From 0a6142a039dfebb086fed6bb7cb38121ed904a33 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 7 Nov 2021 20:20:35 +0800 Subject: [PATCH 001/470] add map function mapFromArrays --- src/Functions/map.cpp | 68 +++++++++++++++++++ .../0_stateless/01651_map_functions.reference | 3 + .../0_stateless/01651_map_functions.sql | 9 ++- 3 files changed, 78 insertions(+), 2 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 03a9da404c2..994eda1ec62 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } namespace @@ -144,6 +145,72 @@ public: } }; +// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays +class FunctionMapFromArrays : public IFunction +{ +public: + static constexpr auto name = "mapFromArrays"; + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires 2 arguments, but {} given", getName(), arguments.size()); + + const auto * keys_type = checkAndGetDataType(arguments[0].get()); + if (!keys_type) + throw Exception{"First argument for function " + getName() + " must be a Array", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + const auto * values_type = checkAndGetDataType(arguments[1].get()); + if (!values_type) + throw Exception{"Second argument for function " + getName() + " must be a Array", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + DataTypes tmp; + const auto & key_type = keys_type->getNestedType(); + const auto & value_type = values_type->getNestedType(); + tmp.emplace_back(key_type); + tmp.emplace_back(value_type); + return std::make_shared(tmp); + } + + ColumnPtr executeImpl( + const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override + { + const auto * col_keys = typeid_cast(arguments[0].column.get()); + const auto * col_values = typeid_cast(arguments[1].column.get()); + if (!col_keys->hasEqualOffsets(*col_values)) + throw Exception("Array arguments for function " + getName() + " must have equal sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + + const auto & data_keys = col_keys->getDataPtr(); + const auto & data_values = col_values->getDataPtr(); + const auto & offsets = col_keys->getOffsetsPtr(); + auto nested_column = ColumnArray::create( + ColumnTuple::create(Columns{std::move(data_keys), std::move(data_values)}), + std::move(offsets)); + return ColumnMap::create(nested_column); + } +}; struct NameMapContains { static constexpr auto name = "mapContains"; }; @@ -528,6 +595,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 06adaf48cfd..30579f8c4ee 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -8,6 +8,8 @@ 0 ['name','age'] ['name','gender'] +{'name':'zhangsan','age':'10'} +{'name':'lisi','gender':'female'} 1 0 0 1 0 1 1 0 0 @@ -21,3 +23,4 @@ {'aa':4,'bb':5} 1 0 {0:0} 1 {0:0} 0 +{'aa':4,'bb':5} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index bbaaf9bee84..83ee42f17a5 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -2,14 +2,15 @@ set allow_experimental_map_type = 1; -- String type drop table if exists table_map; -create table table_map (a Map(String, String), b String) engine = Memory; -insert into table_map values ({'name':'zhangsan', 'age':'10'}, 'name'), ({'name':'lisi', 'gender':'female'},'age'); +create table table_map (a Map(String, String), b String, c Array(String), d Array(String)) engine = Memory; +insert into table_map values ({'name':'zhangsan', 'age':'10'}, 'name', ['name', 'age'], ['zhangsan', '10']), ({'name':'lisi', 'gender':'female'},'age',['name', 'gender'], ['lisi', 'female']); select mapContains(a, 'name') from table_map; select mapContains(a, 'gender') from table_map; select mapContains(a, 'abc') from table_map; select mapContains(a, b) from table_map; select mapContains(a, 10) from table_map; -- { serverError 386 } select mapKeys(a) from table_map; +select mapFromArrays(c, d) from table_map; drop table if exists table_map; CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32) engine = MergeTree order by tuple(); @@ -27,3 +28,7 @@ select map( 'aa', 4, 'bb' , 5) as m, mapKeys(m), mapValues(m); select map( 'aa', 4, 'bb' , 5) as m, mapContains(m, 'aa'), mapContains(m, 'k'); select map(0, 0) as m, mapContains(m, number % 2) from numbers(2); + +select mapFromArrays(['aa', 'bb'], [4, 5]); +select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError 42 } +select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError 190 } From 38a1afaca70d93c562b8e71f05a55fb7465f170b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 7 Nov 2021 20:51:14 +0800 Subject: [PATCH 002/470] add docs --- .../functions/tuple-map-functions.md | 32 +++++++++++++++++++ .../0_stateless/01651_map_functions.reference | 3 ++ .../0_stateless/01651_map_functions.sql | 5 +-- 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 47e33806930..3c86dc9c9bb 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -429,6 +429,38 @@ Result: │ {} │ └────────────────────────────┘ ``` + + +## mapFromArrays {#mapFromArrays} + +**Syntax** + +```sql +mapFromArrays(keys, values) +``` + +**Parameters** + +- `keys` — Given key array to create a map from. +- `values` - Given value array to create a map from. + +**Returned value** + +- A map whose keys and values are from input arrays respectively. + +**Example** + +Query: + +```sql +select mapFromArrays(['a', 'b', 'c'], [1, 2, 3]) +``` + +```text +┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐ +│ {'a':1,'b':2,'c':3} │ +└───────────────────────────────────────────┘ +``` [Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 30579f8c4ee..05dcb8f2cf5 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -19,6 +19,9 @@ [1000] [1001] [1002] +{'1000':'2000','1000':'3000','1000':'4000'} +{'1001':'2002','1001':'3003','1001':'4004'} +{'1002':'2004','1002':'3006','1002':'4008'} {'aa':4,'bb':5} ['aa','bb'] [4,5] {'aa':4,'bb':5} 1 0 {0:0} 1 diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 83ee42f17a5..6ecd5bce494 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -13,13 +13,14 @@ select mapKeys(a) from table_map; select mapFromArrays(c, d) from table_map; drop table if exists table_map; -CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32) engine = MergeTree order by tuple(); -insert into table_map select map(number, number), number, number from numbers(1000, 3); +CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32, d Array(String), e Array(String)) engine = MergeTree order by tuple(); +insert into table_map select map(number, number), number, number, [number, number, number], [number*2, number*3, number*4] from numbers(1000, 3); select mapContains(a, b), mapContains(a, c), mapContains(a, 233) from table_map; select mapContains(a, 'aaa') from table_map; -- { serverError 386 } select mapContains(b, 'aaa') from table_map; -- { serverError 43 } select mapKeys(a) from table_map; select mapValues(a) from table_map; +select mapFromArrays(d, e) from table_map; drop table if exists table_map; From 21aadec53de0d460c48678600e98509ba6f8f0b8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sun, 7 Nov 2021 20:59:46 +0800 Subject: [PATCH 003/470] fix code style --- src/Functions/map.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 994eda1ec62..a3a33453e0c 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -175,7 +175,7 @@ public: if (arguments.size() != 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires 2 arguments, but {} given", getName(), arguments.size()); - + const auto * keys_type = checkAndGetDataType(arguments[0].get()); if (!keys_type) throw Exception{"First argument for function " + getName() + " must be a Array", From 42646c0bba6fe0140fd39567f1407b798c12b5ee Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 12:27:31 +0800 Subject: [PATCH 004/470] fix bug --- src/Functions/map.cpp | 30 ++++++++++++++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index a3a33453e0c..3756c296bb4 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; } namespace @@ -197,8 +198,33 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { - const auto * col_keys = typeid_cast(arguments[0].column.get()); - const auto * col_values = typeid_cast(arguments[1].column.get()); + bool is_keys_const = isColumnConst(*arguments[0].column); + const ColumnArray * col_keys; + if (is_keys_const) + { + auto tmp = arguments[0].column->convertToFullColumnIfConst(); + col_keys = checkAndGetColumn(tmp.get()); + } + else + { + col_keys = checkAndGetColumn(arguments[0].column.get()); + } + + bool is_values_const = isColumnConst(*arguments[1].column); + const ColumnArray * col_values; + if (is_values_const) + { + auto tmp = arguments[1].column->convertToFullColumnIfConst(); + col_values = checkAndGetColumn(tmp.get()); + } + else + { + col_values = checkAndGetColumn(arguments[1].column.get()); + } + + if (!col_keys || !col_values) + throw Exception("Arguments of function " + getName() + " must be array.", ErrorCodes::ILLEGAL_COLUMN); + if (!col_keys->hasEqualOffsets(*col_values)) throw Exception("Array arguments for function " + getName() + " must have equal sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); From 12834902c060dd941ae486330950ff32ed1a1f85 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 17:04:33 +0800 Subject: [PATCH 005/470] fix mem leak --- src/Functions/map.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 3756c296bb4..4f2e072d08a 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -198,24 +198,26 @@ public: ColumnPtr executeImpl( const ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, size_t /* input_rows_count */) const override { + ColumnPtr holder_keys; bool is_keys_const = isColumnConst(*arguments[0].column); const ColumnArray * col_keys; if (is_keys_const) { - auto tmp = arguments[0].column->convertToFullColumnIfConst(); - col_keys = checkAndGetColumn(tmp.get()); + holder_keys = arguments[0].column->convertToFullColumnIfConst(); + col_keys = checkAndGetColumn(holder_keys.get()); } else { col_keys = checkAndGetColumn(arguments[0].column.get()); } + ColumnPtr holder_values; bool is_values_const = isColumnConst(*arguments[1].column); const ColumnArray * col_values; if (is_values_const) { - auto tmp = arguments[1].column->convertToFullColumnIfConst(); - col_values = checkAndGetColumn(tmp.get()); + holder_values = arguments[1].column->convertToFullColumnIfConst(); + col_values = checkAndGetColumn(holder_values.get()); } else { From 8c93e08fd4ff8a4d5ee137e130ab020ba1519ec4 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 8 Nov 2021 17:09:23 +0800 Subject: [PATCH 006/470] add some full columns test cases --- tests/queries/0_stateless/01651_map_functions.reference | 4 ++++ tests/queries/0_stateless/01651_map_functions.sql | 2 ++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 05dcb8f2cf5..284e27a1b0d 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -27,3 +27,7 @@ {0:0} 1 {0:0} 0 {'aa':4,'bb':5} +{'aa':4,'bb':5} +{'aa':4,'bb':5} +{'aa':4,'bb':5} +{'aa':4,'bb':5} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 6ecd5bce494..57023d41470 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -31,5 +31,7 @@ select map( 'aa', 4, 'bb' , 5) as m, mapContains(m, 'aa'), mapContains(m, 'k'); select map(0, 0) as m, mapContains(m, number % 2) from numbers(2); select mapFromArrays(['aa', 'bb'], [4, 5]); +select mapFromArrays(['aa', 'bb'], materialize([4, 5])); +select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(3); select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError 42 } select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError 190 } From fbe4808b6b73523d756bcebeb8ffb0e38b6a2084 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 10 Jan 2023 12:28:13 +0000 Subject: [PATCH 007/470] Add support for custom key in parallel replicas --- src/Core/Settings.h | 2 + src/Core/SettingsEnums.cpp | 4 ++ src/Core/SettingsEnums.h | 9 +++ src/Interpreters/Cluster.cpp | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 15 +++++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 60 +++++++++++++++++-- src/Storages/SelectQueryInfo.h | 2 + 7 files changed, 86 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2357948a1f6..a714fc449b7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -153,6 +153,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ + M(String, parallel_replicas_custom_key, "", "Custom key for parallel replicas", 0) \ + M(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::SAMPLE_KEY, "How to process query using multiple replicas.", 0) \ \ M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 632587106a1..eaeed9dc44c 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -162,4 +162,8 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation , ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, {"kusto", Dialect::kusto}}) + +IMPLEMENT_SETTING_ENUM(ParallelReplicasMode, ErrorCodes::BAD_ARGUMENTS, + {{"sample_key", ParallelReplicasMode::SAMPLE_KEY}, + {"custom_key", ParallelReplicasMode::CUSTOM_KEY}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 97c4275c4d2..a2041634d4b 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -191,4 +191,13 @@ enum class Dialect }; DECLARE_SETTING_ENUM(Dialect) + +enum class ParallelReplicasMode : uint8_t +{ + SAMPLE_KEY, + CUSTOM_KEY, +}; + +DECLARE_SETTING_ENUM(ParallelReplicasMode) + } diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b76434b23e7..9f0a9d3b35c 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -509,7 +509,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, shard_local_addresses.push_back(replica); shard_all_addresses.push_back(replica); } - ConnectionPoolWithFailoverPtr shard_pool = std::make_shared( all_replicas_pools, settings.load_balancing, settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ba2c845ead1..2d0436fa9d3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -319,6 +319,15 @@ static ASTPtr parseAdditionalFilterConditionForTable( return nullptr; } +static ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) +{ + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, setting.data(), setting.data() + setting.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); +} + /// Returns true if we should ignore quotas and limits for a specified table in the system database. static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { @@ -501,6 +510,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); + if (settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && !settings.parallel_replicas_custom_key.value.empty()) + { + query_info.parallel_replica_custom_key_ast = parseParallelReplicaCustomKey( + settings.parallel_replicas_custom_key, *context); + } + auto analyze = [&] (bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1ca1779e4b0..3d48c333727 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -34,6 +34,7 @@ #include #include +#include #include #include #include @@ -44,6 +45,8 @@ #include #include +#include +#include namespace DB { @@ -61,6 +64,7 @@ namespace ErrorCodes extern const int DUPLICATED_PART_UUIDS; extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int PROJECTION_NOT_USED; + extern const int BAD_ARGUMENTS; } @@ -470,6 +474,25 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( return plan; } +namespace +{ + +bool supportsSamplingForParallelReplicas(const SelectQueryInfo & select_query_info, const MergeTreeData & data, const Settings & settings) +{ + if (settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + { + /// maybe just don't use sampling or try to fallback to SAMPLE_KEY? + if (select_query_info.parallel_replica_custom_key_ast == nullptr) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but no 'parallel_replicas_custom_key' defined"); + + return true; + } + + return data.supportsSampling(); +} + +} + MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const SelectQueryInfo & select_query_info, NamesAndTypesList available_real_columns, @@ -587,9 +610,10 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ + bool supports_sampling_for_parallel_replicas = supportsSamplingForParallelReplicas(select_query_info, data, settings); /// Parallel replicas has been requested but there is no way to sample data. /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + if (settings.parallel_replicas_count > 1 && !supports_sampling_for_parallel_replicas && settings.parallel_replica_offset > 0) { LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); @@ -597,16 +621,34 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } - sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); + sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && supports_sampling_for_parallel_replicas); bool no_data = false; /// There is nothing left after sampling. + std::optional parallel_replicas_custom_key_description; + const auto get_sampling_key = [&]() -> const KeyDescription & + { + if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + { + assert(select_query_info.parallel_replica_custom_key_ast); + + LOG_INFO(log, "Using custom key for sampling while processing with multiple replicas"); + + if (!parallel_replicas_custom_key_description) + parallel_replicas_custom_key_description = KeyDescription::getKeyFromAST(select_query_info.parallel_replica_custom_key_ast, metadata_snapshot->columns, context); + + return *parallel_replicas_custom_key_description; + } + + return metadata_snapshot->getSamplingKey(); + }; + if (sampling.use_sampling) { if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) sampling.used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; - const auto & sampling_key = metadata_snapshot->getSamplingKey(); + const auto & sampling_key = get_sampling_key(); DataTypePtr sampling_column_type = sampling_key.data_types[0]; if (sampling_key.data_types.size() == 1) @@ -681,7 +723,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 = sampling_key.definition_ast; if (final) { @@ -693,8 +735,11 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_lower_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable())) + && (settings.parallel_replicas_count <= 1 || settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY)) + { throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + } ASTPtr args = std::make_shared(); args->children.push_back(sampling_key_ast); @@ -711,8 +756,11 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_upper_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable()))) + sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable())) + && (settings.parallel_replicas_count <= 1 || settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY)) + { throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); + } ASTPtr args = std::make_shared(); args->children.push_back(sampling_key_ast); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index a8a8ae4e877..ec3af64d93a 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -209,6 +209,8 @@ struct SelectQueryInfo /// It is needed for PK analysis based on row_level_policy and additional_filters. ASTs filter_asts; + ASTPtr parallel_replica_custom_key_ast; + /// Filter actions dag for current storage ActionsDAGPtr filter_actions_dag; From 19e5988d3312194be35ab0d26c5c2966890a643a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 11 Jan 2023 09:57:13 +0000 Subject: [PATCH 008/470] Better --- src/Interpreters/InterpreterSelectQuery.cpp | 63 ++++++++++++++++--- src/Interpreters/InterpreterSelectQuery.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 60 ++---------------- 3 files changed, 61 insertions(+), 64 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2d0436fa9d3..1cd5d0004fa 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -510,10 +510,37 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - if (settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && !settings.parallel_replicas_custom_key.value.empty()) + + ASTPtr parallel_replicas_custom_filter_ast = nullptr; + if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) { - query_info.parallel_replica_custom_key_ast = parseParallelReplicaCustomKey( + if (settings.parallel_replicas_custom_key.value.empty()) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); + + auto custom_key_ast = parseParallelReplicaCustomKey( settings.parallel_replicas_custom_key, *context); + + // first we do modulo with replica count + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(settings.parallel_replicas_count.value)); + + auto modulo_function = std::make_shared(); + modulo_function->name = "positiveModulo"; + modulo_function->arguments = args; + modulo_function->children.push_back(modulo_function->arguments); + + /// then we compare result to the current replica number (offset) + args = std::make_shared(); + args->children.push_back(modulo_function); + args->children.push_back(std::make_shared(settings.parallel_replica_offset.value)); + + auto equals_function = std::make_shared(); + equals_function->name = "equals"; + equals_function->arguments = args; + equals_function->children.push_back(equals_function->arguments); + + parallel_replicas_custom_filter_ast = equals_function; } auto analyze = [&] (bool try_move_to_prewhere) @@ -653,6 +680,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.push_back(query_info.additional_filter_ast); } + if (parallel_replicas_custom_filter_ast) + { + custom_key_filter_info = generateFilterActions( + table_id, parallel_replicas_custom_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns, + prepared_sets); + + custom_key_filter_info->do_remove_column = true; + query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); + } + source_header = storage_snapshot->getSampleBlockForColumns(required_columns); } @@ -1396,17 +1433,23 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( + auto filter_step = std::make_unique( query_plan.getCurrentDataStream(), - additional_filter_info->actions, - additional_filter_info->column_name, - additional_filter_info->do_remove_column); + new_filter_info->actions, + new_filter_info->column_name, + new_filter_info->do_remove_column); - additional_filter_step->setStepDescription("Additional filter"); - query_plan.addStep(std::move(additional_filter_step)); - } + filter_step->setStepDescription(description); + query_plan.addStep(std::move(filter_step)); + }; + + if (additional_filter_info) + add_filter_step(additional_filter_info, "Additional filter"); + + if (custom_key_filter_info) + add_filter_step(custom_key_filter_info, "Paralel replica custom key filter"); if (expressions.before_array_join) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 761eea8e1b8..5667c0e1fa7 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -224,6 +224,8 @@ private: /// For additional_filter setting. FilterDAGInfoPtr additional_filter_info; + FilterDAGInfoPtr custom_key_filter_info; + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; /// List of columns to read to execute the query. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3d48c333727..d55e3f86f30 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -34,7 +34,6 @@ #include #include -#include #include #include #include @@ -45,8 +44,6 @@ #include #include -#include -#include namespace DB { @@ -64,7 +61,6 @@ namespace ErrorCodes extern const int DUPLICATED_PART_UUIDS; extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int PROJECTION_NOT_USED; - extern const int BAD_ARGUMENTS; } @@ -474,25 +470,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( return plan; } -namespace -{ - -bool supportsSamplingForParallelReplicas(const SelectQueryInfo & select_query_info, const MergeTreeData & data, const Settings & settings) -{ - if (settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) - { - /// maybe just don't use sampling or try to fallback to SAMPLE_KEY? - if (select_query_info.parallel_replica_custom_key_ast == nullptr) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but no 'parallel_replicas_custom_key' defined"); - - return true; - } - - return data.supportsSampling(); -} - -} - MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( const SelectQueryInfo & select_query_info, NamesAndTypesList available_real_columns, @@ -610,10 +587,9 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ - bool supports_sampling_for_parallel_replicas = supportsSamplingForParallelReplicas(select_query_info, data, settings); /// Parallel replicas has been requested but there is no way to sample data. /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && !supports_sampling_for_parallel_replicas && settings.parallel_replica_offset > 0) + if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) { LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); @@ -621,34 +597,16 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( return sampling; } - sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && supports_sampling_for_parallel_replicas); + sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY && data.supportsSampling()); bool no_data = false; /// There is nothing left after sampling. - std::optional parallel_replicas_custom_key_description; - const auto get_sampling_key = [&]() -> const KeyDescription & - { - if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) - { - assert(select_query_info.parallel_replica_custom_key_ast); - - LOG_INFO(log, "Using custom key for sampling while processing with multiple replicas"); - - if (!parallel_replicas_custom_key_description) - parallel_replicas_custom_key_description = KeyDescription::getKeyFromAST(select_query_info.parallel_replica_custom_key_ast, metadata_snapshot->columns, context); - - return *parallel_replicas_custom_key_description; - } - - return metadata_snapshot->getSamplingKey(); - }; - if (sampling.use_sampling) { if (sample_factor_column_queried && relative_sample_size != RelativeSize(0)) sampling.used_sample_factor = 1.0 / boost::rational_cast(relative_sample_size); RelativeSize size_of_universum = 0; - const auto & sampling_key = get_sampling_key(); + const auto & sampling_key = metadata_snapshot->getSamplingKey(); DataTypePtr sampling_column_type = sampling_key.data_types[0]; if (sampling_key.data_types.size() == 1) @@ -723,7 +681,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 = sampling_key.definition_ast; + ASTPtr sampling_key_ast = metadata_snapshot->getSamplingKeyAST(); if (final) { @@ -735,11 +693,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_lower_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable())) - && (settings.parallel_replicas_count <= 1 || settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY)) - { + sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable()))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - } ASTPtr args = std::make_shared(); args->children.push_back(sampling_key_ast); @@ -756,11 +711,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_upper_limit) { if (!key_condition.addCondition( - sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable())) - && (settings.parallel_replicas_count <= 1 || settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY)) - { + sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable()))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); - } ASTPtr args = std::make_shared(); args->children.push_back(sampling_key_ast); From f950b75ef0adb6543b2c1f24117d51b35482f1bd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 11 Jan 2023 10:42:01 +0000 Subject: [PATCH 009/470] Fix --- src/Interpreters/InterpreterSelectQuery.cpp | 9 +++++---- src/Interpreters/InterpreterSelectQuery.h | 3 ++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1cd5d0004fa..8f03d83c1c9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -112,6 +112,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; + extern const int BAD_ARGUMENTS; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -682,11 +683,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (parallel_replicas_custom_filter_ast) { - custom_key_filter_info = generateFilterActions( + parallel_replicas_custom_filter_info = generateFilterActions( table_id, parallel_replicas_custom_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns, prepared_sets); - custom_key_filter_info->do_remove_column = true; + parallel_replicas_custom_filter_info->do_remove_column = true; query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); } @@ -1448,8 +1449,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

Date: Thu, 12 Jan 2023 07:56:15 +0000 Subject: [PATCH 010/470] Add READ_TASKS mode --- src/Core/SettingsEnums.cpp | 3 ++- src/Core/SettingsEnums.h | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 13 +++++++++---- src/Storages/StorageDistributed.cpp | 3 ++- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index eaeed9dc44c..18abac0fe4a 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -165,5 +165,6 @@ IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(ParallelReplicasMode, ErrorCodes::BAD_ARGUMENTS, {{"sample_key", ParallelReplicasMode::SAMPLE_KEY}, - {"custom_key", ParallelReplicasMode::CUSTOM_KEY}}) + {"custom_key", ParallelReplicasMode::CUSTOM_KEY}, + {"read_tasks", ParallelReplicasMode::READ_TASKS}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index a2041634d4b..4acf2889e37 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -196,6 +196,7 @@ enum class ParallelReplicasMode : uint8_t { SAMPLE_KEY, CUSTOM_KEY, + READ_TASKS, }; DECLARE_SETTING_ENUM(ParallelReplicasMode) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d55e3f86f30..3a4e509163b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -589,16 +589,21 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( /// Parallel replicas has been requested but there is no way to sample data. /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY + && !data.supportsSampling() && settings.parallel_replica_offset > 0) { - LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" + LOG_DEBUG( + log, + "Will use no data on this replica because parallel replicas processing has been requested" " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); sampling.read_nothing = true; return sampling; } - sampling.use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY && data.supportsSampling()); - bool no_data = false; /// There is nothing left after sampling. + sampling.use_sampling = relative_sample_size > 0 + || (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY + && data.supportsSampling()); + bool no_data = false; /// There is nothing left after sampling. if (sampling.use_sampling) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 6473e00cdf2..04b98fa8d20 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -697,7 +697,8 @@ void StorageDistributed::read( auto settings = local_context->getSettingsRef(); - bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests; + bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas + && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; if (parallel_replicas) ClusterProxy::executeQueryWithParallelReplicas( From 5589bec33beeced38d3088c8e70800ca5c71a9eb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Jan 2023 11:35:37 +0000 Subject: [PATCH 011/470] Update tests --- ...4_shard_distributed_with_many_replicas.sql | 1 - ...here_max_parallel_replicas_distributed.sql | 2 -- .../01034_sample_final_distributed.sql | 2 -- ...099_parallel_distributed_insert_select.sql | 1 - .../01517_select_final_distributed.sql | 2 -- .../01557_max_parallel_replicas_no_sample.sql | 2 -- .../02221_parallel_replicas_bug.sh | 2 +- ...arallel_reading_from_replicas_benchmark.sh | 1 + .../02404_memory_bound_merging.sql | 1 + ...max_parallel_replicas_custom_key.reference | 0 ...02527_max_parallel_replicas_custom_key.sql | 36 +++++++++++++++++++ 11 files changed, 39 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference create mode 100644 tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index 901b818cbc0..ab4c433ba47 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,6 +1,5 @@ -- Tags: replica, distributed -SET allow_experimental_parallel_reading_from_replicas = 0; SET max_parallel_replicas = 2; DROP TABLE IF EXISTS report; diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index 86c84427297..b26c9af14a9 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -1,7 +1,5 @@ -- Tags: replica, distributed -set allow_experimental_parallel_reading_from_replicas=0; - drop table if exists test_max_parallel_replicas_lr; -- If you wonder why the table is named with "_lr" suffix in this test. diff --git a/tests/queries/0_stateless/01034_sample_final_distributed.sql b/tests/queries/0_stateless/01034_sample_final_distributed.sql index a81fef645db..b784b35cbb3 100644 --- a/tests/queries/0_stateless/01034_sample_final_distributed.sql +++ b/tests/queries/0_stateless/01034_sample_final_distributed.sql @@ -1,7 +1,5 @@ -- Tags: distributed -set allow_experimental_parallel_reading_from_replicas = 0; - drop table if exists sample_final; create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID); insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000); diff --git a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql index aa924218360..a75a5b2c33d 100644 --- a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql +++ b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql @@ -2,7 +2,6 @@ -- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971 -SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525 SET prefer_localhost_replica = 1; DROP TABLE IF EXISTS local_01099_a; diff --git a/tests/queries/0_stateless/01517_select_final_distributed.sql b/tests/queries/0_stateless/01517_select_final_distributed.sql index 701828b0b38..a3d1fcfc185 100644 --- a/tests/queries/0_stateless/01517_select_final_distributed.sql +++ b/tests/queries/0_stateless/01517_select_final_distributed.sql @@ -1,7 +1,5 @@ -- Tags: distributed -SET allow_experimental_parallel_reading_from_replicas = 0; - DROP TABLE IF EXISTS test5346; CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime) diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql index 04777f5b31c..2b1a66147a4 100644 --- a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -1,7 +1,5 @@ -- Tags: replica -SET allow_experimental_parallel_reading_from_replicas=0; - DROP TABLE IF EXISTS t; CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; INSERT INTO t VALUES ('Hello'); diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index cce32bf8272..1a703b20d2a 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 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_mode='read_tasks' -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null 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 2a163746e20..283c51cf0a2 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 @@ -15,6 +15,7 @@ as select * from numbers(1); # Logical error: 'Coordinator for parallel reading from replicas is not initialized'. opts=( --allow_experimental_parallel_reading_from_replicas 1 + --parallel_replicas_mode 'read_tasks' --max_parallel_replicas 3 --iterations 1 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index c41e2d3abae..fb0c65e6a7c 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -49,6 +49,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off -- { echoOff } -- set allow_experimental_parallel_reading_from_replicas = 1; +set parallel_replicas_mode = 'read_tasks'; set max_parallel_replicas = 3; set use_hedged_requests = 0; diff --git a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql new file mode 100644 index 00000000000..b716600b9d5 --- /dev/null +++ b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql @@ -0,0 +1,36 @@ +-- Tags: replica + +DROP TABLE IF EXISTS t; + +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; +INSERT INTO t VALUES ('Hello'); + +SET max_parallel_replicas = 3; +SET parallel_replicas_mode = 'custom_key'; + +SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'sipHash64(x)'; + +DROP TABLE t; + +CREATE TABLE t (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); +INSERT INTO t SELECT toString(number), number FROM numbers(1000); + +SET max_parallel_replicas = 1; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +SET max_parallel_replicas = 2; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +SET max_parallel_replicas = 3; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +DROP TABLE t; From e60c2fa77f6c00fd69ea25e7117bf0f8eeba36bf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Jan 2023 12:47:38 +0000 Subject: [PATCH 012/470] Update reference --- ...02527_max_parallel_replicas_custom_key.reference | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference index e69de29bb2d..c51a7e10fb2 100644 --- a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference +++ b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference @@ -0,0 +1,13 @@ +Hello +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 From 57e771226cb3962941be802e2b272a753024c567 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 16 Jan 2023 10:11:21 +0000 Subject: [PATCH 013/470] Add range mode for custom key --- src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 8 + src/Interpreters/InterpreterSelectQuery.cpp | 186 ++++++++++++++---- ...rallel_replicas_custom_key_range.reference | 13 ++ ...max_parallel_replicas_custom_key_range.sql | 37 ++++ 6 files changed, 214 insertions(+), 37 deletions(-) create mode 100644 tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference create mode 100644 tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 573d742ff58..61d1c62bcff 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -153,7 +153,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ - M(String, parallel_replicas_custom_key, "", "Custom key for parallel replicas", 0) \ + M(String, parallel_replicas_custom_key, "", "Custom key for parallel replicas using modulo operation on the key for assigning work to replicas.", 0) \ + M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::SAMPLE_KEY, "How to process query using multiple replicas.", 0) \ \ M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 33aba922e95..69fd78d7eb3 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -168,6 +168,10 @@ IMPLEMENT_SETTING_ENUM(ParallelReplicasMode, ErrorCodes::BAD_ARGUMENTS, {"custom_key", ParallelReplicasMode::CUSTOM_KEY}, {"read_tasks", ParallelReplicasMode::READ_TASKS}}) +IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, + {{"default", ParallelReplicasCustomKeyFilterType::DEFAULT}, + {"range", ParallelReplicasCustomKeyFilterType::RANGE}}) + IMPLEMENT_SETTING_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS, {{"mmap", LocalFSReadMethod::mmap}, {"pread", LocalFSReadMethod::pread}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 83c12477165..52c1b369f4b 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -202,5 +202,13 @@ enum class ParallelReplicasMode : uint8_t DECLARE_SETTING_ENUM(ParallelReplicasMode) +enum class ParallelReplicasCustomKeyFilterType : uint8_t +{ + DEFAULT, + RANGE, +}; + +DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) + DECLARE_SETTING_ENUM(LocalFSReadMethod) } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 28f63d5967c..6b66c75e54f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -94,6 +94,9 @@ #include #include #include +#include "Core/SettingsEnums.h" + +#include namespace DB { @@ -113,6 +116,7 @@ namespace ErrorCodes extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -228,10 +232,13 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::~InterpreterSelectQuery() = default; +namespace +{ + /** There are no limits on the maximum size of the result for the subquery. * Since the result of the query is not the result of the entire query. */ -static ContextPtr getSubqueryContext(const ContextPtr & context) +ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); @@ -243,7 +250,7 @@ static ContextPtr getSubqueryContext(const ContextPtr & context) return subquery_context; } -static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) +void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) { ASTSelectQuery & select = query->as(); @@ -263,7 +270,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table } /// Checks that the current user has the SELECT privilege. -static void checkAccessRightsForSelect( +void checkAccessRightsForSelect( const ContextPtr & context, const StorageID & table_id, const StorageMetadataPtr & table_metadata, @@ -293,7 +300,7 @@ static void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } -static ASTPtr parseAdditionalFilterConditionForTable( +ASTPtr parseAdditionalFilterConditionForTable( const Map & setting, const DatabaseAndTableWithAlias & target, const Context & context) @@ -320,7 +327,7 @@ static ASTPtr parseAdditionalFilterConditionForTable( return nullptr; } -static ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) +ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) { ParserExpression parser; const auto & settings = context.getSettingsRef(); @@ -329,8 +336,142 @@ static ASTPtr parseParallelReplicaCustomKey(const String & setting, const Contex "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); } +ASTPtr getCustomKeyFilterForParallelReplica(const Settings & settings, const StoragePtr & storage, const ContextPtr & context) +{ + assert(settings.parallel_replicas_count > 1); + + if (settings.parallel_replicas_custom_key.value.empty()) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); + + auto custom_key_ast = parseParallelReplicaCustomKey( + settings.parallel_replicas_custom_key, *context); + + if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + { + // first we do modulo with replica count + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(settings.parallel_replicas_count.value)); + + auto modulo_function = std::make_shared(); + modulo_function->name = "positiveModulo"; + modulo_function->arguments = args; + modulo_function->children.push_back(modulo_function->arguments); + + /// then we compare result to the current replica number (offset) + args = std::make_shared(); + args->children.push_back(modulo_function); + args->children.push_back(std::make_shared(settings.parallel_replica_offset.value)); + + auto equals_function = std::make_shared(); + equals_function->name = "equals"; + equals_function->arguments = args; + equals_function->children.push_back(equals_function->arguments); + + return equals_function; + } + + // create range query + assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + + if (!storage) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); + + KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, storage->getInMemoryMetadataPtr()->columns, context); + + using RelativeSize = boost::rational; + + RelativeSize size_of_universum = 0; + DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; + + if (custom_key_description.data_types.size() == 1) + { + if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, + "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + + RelativeSize relative_range_size = RelativeSize(1) / settings.parallel_replicas_count.value; + RelativeSize relative_range_offset = relative_range_size * RelativeSize(settings.parallel_replica_offset.value); + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + assert(has_lower_limit || has_upper_limit); + + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + if (has_lower_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(lower)); + + lower_function = std::make_shared(); + lower_function->name = "greaterOrEquals"; + lower_function->arguments = args; + lower_function->children.push_back(lower_function->arguments); + + if (!has_upper_limit) + return lower_function; + } + + if (has_upper_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(upper)); + + upper_function = std::make_shared(); + upper_function->name = "less"; + upper_function->arguments = args; + upper_function->children.push_back(upper_function->arguments); + + if (!has_lower_limit) + return upper_function; + } + + assert(has_lower_limit && has_upper_limit); + + ASTPtr args = std::make_shared(); + args->children.push_back(lower_function); + args->children.push_back(upper_function); + + auto filter_function = std::make_shared(); + filter_function->name = "and"; + filter_function->arguments = args; + filter_function->children.push_back(filter_function->arguments); + + return filter_function; +} + /// Returns true if we should ignore quotas and limits for a specified table in the system database. -static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) +bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { if (table_id.database_name == DatabaseCatalog::SYSTEM_DATABASE) { @@ -341,6 +482,8 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) return false; } +} + InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const ContextPtr & context_, @@ -511,38 +654,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - ASTPtr parallel_replicas_custom_filter_ast = nullptr; if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) - { - if (settings.parallel_replicas_custom_key.value.empty()) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); - - auto custom_key_ast = parseParallelReplicaCustomKey( - settings.parallel_replicas_custom_key, *context); - - // first we do modulo with replica count - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(settings.parallel_replicas_count.value)); - - auto modulo_function = std::make_shared(); - modulo_function->name = "positiveModulo"; - modulo_function->arguments = args; - modulo_function->children.push_back(modulo_function->arguments); - - /// then we compare result to the current replica number (offset) - args = std::make_shared(); - args->children.push_back(modulo_function); - args->children.push_back(std::make_shared(settings.parallel_replica_offset.value)); - - auto equals_function = std::make_shared(); - equals_function->name = "equals"; - equals_function->arguments = args; - equals_function->children.push_back(equals_function->arguments); - - parallel_replicas_custom_filter_ast = equals_function; - } + parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica(settings, storage, context); auto analyze = [&] (bool try_move_to_prewhere) { diff --git a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference new file mode 100644 index 00000000000..c51a7e10fb2 --- /dev/null +++ b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference @@ -0,0 +1,13 @@ +Hello +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 +1000 diff --git a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql new file mode 100644 index 00000000000..0c1cb7fb48b --- /dev/null +++ b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql @@ -0,0 +1,37 @@ +-- Tags: replica + +DROP TABLE IF EXISTS t; + +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; +INSERT INTO t VALUES ('Hello'); + +SET max_parallel_replicas = 3; +SET parallel_replicas_mode = 'custom_key'; +SET parallel_replicas_custom_key_filter_type = 'range'; + +SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'sipHash64(x)'; + +DROP TABLE t; + +CREATE TABLE t (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); +INSERT INTO t SELECT toString(number), number FROM numbers(1000); + +SET max_parallel_replicas = 1; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +SET max_parallel_replicas = 2; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +SET max_parallel_replicas = 3; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; + +DROP TABLE t; From 5adf44775830d92e1bfba53ebfba5c71a956d48e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 16 Jan 2023 11:22:12 +0000 Subject: [PATCH 014/470] Use unsigned integer for column --- .../02528_max_parallel_replicas_custom_key_range.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql index 0c1cb7fb48b..a8ef9e58d40 100644 --- a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql +++ b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql @@ -13,7 +13,7 @@ SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_ DROP TABLE t; -CREATE TABLE t (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); +CREATE TABLE t (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); INSERT INTO t SELECT toString(number), number FROM numbers(1000); SET max_parallel_replicas = 1; From bd352068d749d428d909c0b5d81f299a7e85e3f1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Jan 2023 12:34:42 +0000 Subject: [PATCH 015/470] Turn replicas into shard for custom_key --- .../ClusterProxy/executeQuery.cpp | 8 +- src/Interpreters/ClusterProxy/executeQuery.h | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 157 ----------------- src/Storages/StorageDistributed.cpp | 162 +++++++++++++++++- 4 files changed, 169 insertions(+), 161 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 8ea6298c50b..fc490306e08 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -127,7 +127,8 @@ void executeQuery( const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster) + const ClusterPtr & not_optimized_cluster, + std::function add_additional_shard_filter = {}) { const Settings & settings = context->getSettingsRef(); @@ -177,7 +178,10 @@ void executeQuery( visitor.visit(query_ast_for_shard); } else - query_ast_for_shard = query_ast; + query_ast_for_shard = query_ast->clone(); + + if (add_additional_shard_filter) + add_additional_shard_filter(query_ast_for_shard, shard_info.shard_num); stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 662fe47ca65..9519fa00efe 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -50,7 +50,8 @@ void executeQuery( const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, - const ClusterPtr & not_optimized_cluster); + const ClusterPtr & not_optimized_cluster, + std::function add_additional_shard_filter); void executeQueryWithParallelReplicas( diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6b66c75e54f..3afe83ffac3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -327,149 +327,6 @@ ASTPtr parseAdditionalFilterConditionForTable( return nullptr; } -ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) -{ - ParserExpression parser; - const auto & settings = context.getSettingsRef(); - return parseQuery( - parser, setting.data(), setting.data() + setting.size(), - "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); -} - -ASTPtr getCustomKeyFilterForParallelReplica(const Settings & settings, const StoragePtr & storage, const ContextPtr & context) -{ - assert(settings.parallel_replicas_count > 1); - - if (settings.parallel_replicas_custom_key.value.empty()) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); - - auto custom_key_ast = parseParallelReplicaCustomKey( - settings.parallel_replicas_custom_key, *context); - - if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) - { - // first we do modulo with replica count - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(settings.parallel_replicas_count.value)); - - auto modulo_function = std::make_shared(); - modulo_function->name = "positiveModulo"; - modulo_function->arguments = args; - modulo_function->children.push_back(modulo_function->arguments); - - /// then we compare result to the current replica number (offset) - args = std::make_shared(); - args->children.push_back(modulo_function); - args->children.push_back(std::make_shared(settings.parallel_replica_offset.value)); - - auto equals_function = std::make_shared(); - equals_function->name = "equals"; - equals_function->arguments = args; - equals_function->children.push_back(equals_function->arguments); - - return equals_function; - } - - // create range query - assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); - - if (!storage) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); - - KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, storage->getInMemoryMetadataPtr()->columns, context); - - using RelativeSize = boost::rational; - - RelativeSize size_of_universum = 0; - DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - - if (custom_key_description.data_types.size() == 1) - { - if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, - "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - - RelativeSize relative_range_size = RelativeSize(1) / settings.parallel_replicas_count.value; - RelativeSize relative_range_offset = relative_range_size * RelativeSize(settings.parallel_replica_offset.value); - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - assert(has_lower_limit || has_upper_limit); - - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - if (has_lower_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(lower)); - - lower_function = std::make_shared(); - lower_function->name = "greaterOrEquals"; - lower_function->arguments = args; - lower_function->children.push_back(lower_function->arguments); - - if (!has_upper_limit) - return lower_function; - } - - if (has_upper_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(upper)); - - upper_function = std::make_shared(); - upper_function->name = "less"; - upper_function->arguments = args; - upper_function->children.push_back(upper_function->arguments); - - if (!has_lower_limit) - return upper_function; - } - - assert(has_lower_limit && has_upper_limit); - - ASTPtr args = std::make_shared(); - args->children.push_back(lower_function); - args->children.push_back(upper_function); - - auto filter_function = std::make_shared(); - filter_function->name = "and"; - filter_function->arguments = args; - filter_function->children.push_back(filter_function->arguments); - - return filter_function; -} - /// Returns true if we should ignore quotas and limits for a specified table in the system database. bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { @@ -654,10 +511,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - ASTPtr parallel_replicas_custom_filter_ast = nullptr; - if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) - parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica(settings, storage, context); - auto analyze = [&] (bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. @@ -823,16 +676,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.push_back(query_info.additional_filter_ast); } - if (parallel_replicas_custom_filter_ast) - { - parallel_replicas_custom_filter_info = generateFilterActions( - table_id, parallel_replicas_custom_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns, - prepared_sets); - - parallel_replicas_custom_filter_info->do_remove_column = true; - query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); - } - source_header = storage_snapshot->getSampleBlockForColumns(required_columns, parameter_values); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 671ec91609e..56e5bfe9189 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -27,6 +27,7 @@ #include #include #include +#include "Core/SettingsEnums.h" #include #include @@ -88,6 +89,8 @@ #include #include +#include + namespace fs = std::filesystem; @@ -440,6 +443,11 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( const auto & settings = local_context->getSettingsRef(); ClusterPtr cluster = getCluster(); + + // if it's custom_key we will turn replicas into shards and filter specific data on each of them + if (settings.max_parallel_replicas > 1 && cluster->getShardCount() == 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + cluster = cluster->getClusterWithReplicasAsShards(settings); + query_info.cluster = cluster; size_t nodes = getClusterQueriedNodes(settings, cluster); @@ -749,6 +757,158 @@ void StorageDistributed::read( bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; + ParserExpression parser; + auto custom_key_ast = parseQuery( + parser, + settings.parallel_replicas_custom_key.value.data(), + settings.parallel_replicas_custom_key.value.data() + settings.parallel_replicas_custom_key.value.size(), + "parallel replicas custom key", + settings.max_query_size, + settings.max_parser_depth); + + auto shard_count = query_info.getCluster()->getShardCount(); + + std::function add_additional_shard_filter; + if (settings.max_parallel_replicas > 1 + && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + { + add_additional_shard_filter = [&](ASTPtr & query, uint64_t shard_num) + { + ParserExpression parser; + auto custom_key_ast = parseQuery( + parser, settings.parallel_replicas_custom_key.value.data(), settings.parallel_replicas_custom_key.value.data() + settings.parallel_replicas_custom_key.value.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); + + if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + { + // first we do modulo with replica count + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(shard_count)); + + auto modulo_function = std::make_shared(); + modulo_function->name = "positiveModulo"; + modulo_function->arguments = args; + modulo_function->children.push_back(modulo_function->arguments); + + /// then we compare result to the current replica number (offset) + args = std::make_shared(); + args->children.push_back(modulo_function); + args->children.push_back(std::make_shared(shard_num - 1)); + + auto equals_function = std::make_shared(); + equals_function->name = "equals"; + equals_function->arguments = args; + equals_function->children.push_back(equals_function->arguments); + + auto & select_query = query->as(); + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(equals_function)); + } + else + { + assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + auto filter_function = [&] + { + KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); + + using RelativeSize = boost::rational; + + RelativeSize size_of_universum = 0; + DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; + + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (custom_key_description.data_types.size() == 1) + { + if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + + RelativeSize relative_range_size = RelativeSize(1) / query_info.getCluster()->getShardCount(); + RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + assert(has_lower_limit || has_upper_limit); + + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + if (has_lower_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(lower)); + + lower_function = std::make_shared(); + lower_function->name = "greaterOrEquals"; + lower_function->arguments = args; + lower_function->children.push_back(lower_function->arguments); + + if (!has_upper_limit) + return lower_function; + } + + if (has_upper_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(upper)); + + upper_function = std::make_shared(); + upper_function->name = "less"; + upper_function->arguments = args; + upper_function->children.push_back(upper_function->arguments); + + if (!has_lower_limit) + return upper_function; + } + + assert(has_lower_limit && has_upper_limit); + + ASTPtr args = std::make_shared(); + args->children.push_back(lower_function); + args->children.push_back(upper_function); + + auto f = std::make_shared(); + f->name = "and"; + f->arguments = args; + f->children.push_back(f->arguments); + + return f; + }; + + auto & select_query = query->as(); + select_query.setExpression(ASTSelectQuery::Expression::WHERE, filter_function()); + } + }; + } + if (parallel_replicas) ClusterProxy::executeQueryWithParallelReplicas( query_plan, main_table, remote_table_function_ptr, @@ -763,7 +923,7 @@ void StorageDistributed::read( select_stream_factory, log, modified_query_ast, local_context, query_info, sharding_key_expr, sharding_key_column_name, - query_info.cluster); + query_info.cluster, add_additional_shard_filter); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) From dd31de18a3dc1f05a90f646216830d03f91fed03 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Jan 2023 13:31:09 +0000 Subject: [PATCH 016/470] Extend where correctly --- src/Interpreters/InterpreterSelectQuery.cpp | 2 -- src/Storages/StorageDistributed.cpp | 38 ++++++++++++++++----- 2 files changed, 30 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3afe83ffac3..6969a068949 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -115,8 +115,6 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 56e5bfe9189..0309aeabbc7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -133,6 +133,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int UNSUPPORTED_METHOD; + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } namespace ActionLocks @@ -779,6 +780,7 @@ void StorageDistributed::read( parser, settings.parallel_replicas_custom_key.value.data(), settings.parallel_replicas_custom_key.value.data() + settings.parallel_replicas_custom_key.value.size(), "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); + ASTPtr shard_filter = nullptr ; if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) { // first we do modulo with replica count @@ -801,8 +803,7 @@ void StorageDistributed::read( equals_function->arguments = args; equals_function->children.push_back(equals_function->arguments); - auto & select_query = query->as(); - select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(equals_function)); + shard_filter = equals_function; } else { @@ -895,16 +896,37 @@ void StorageDistributed::read( args->children.push_back(lower_function); args->children.push_back(upper_function); - auto f = std::make_shared(); - f->name = "and"; - f->arguments = args; - f->children.push_back(f->arguments); + auto and_function = std::make_shared(); + and_function->name = "and"; + and_function->arguments = args; + and_function->children.push_back(and_function->arguments); - return f; + return and_function; }; + shard_filter = filter_function(); + } + + if (shard_filter) + { auto & select_query = query->as(); - select_query.setExpression(ASTSelectQuery::Expression::WHERE, filter_function()); + + auto where_expression = select_query.where(); + if (where_expression) + { + ASTPtr args = std::make_shared(); + args->children.push_back(where_expression); + args->children.push_back(shard_filter); + + auto and_function = std::make_shared(); + and_function->name = "and"; + and_function->arguments = args; + and_function->children.push_back(and_function->arguments); + + shard_filter = std::move(and_function); + } + + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter)); } }; } From 2a1f35e6612fb07bd12090b9f3563753a0fe7183 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 07:59:57 +0000 Subject: [PATCH 017/470] Revert some changes --- src/Interpreters/InterpreterSelectQuery.cpp | 42 +++++++-------------- 1 file changed, 14 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6969a068949..48326afda45 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -94,9 +94,6 @@ #include #include #include -#include "Core/SettingsEnums.h" - -#include namespace DB { @@ -230,13 +227,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::~InterpreterSelectQuery() = default; -namespace -{ - /** There are no limits on the maximum size of the result for the subquery. * Since the result of the query is not the result of the entire query. */ -ContextPtr getSubqueryContext(const ContextPtr & context) +static ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); @@ -248,7 +242,7 @@ ContextPtr getSubqueryContext(const ContextPtr & context) return subquery_context; } -void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) +static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) { ASTSelectQuery & select = query->as(); @@ -268,7 +262,7 @@ void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, cons } /// Checks that the current user has the SELECT privilege. -void checkAccessRightsForSelect( +static void checkAccessRightsForSelect( const ContextPtr & context, const StorageID & table_id, const StorageMetadataPtr & table_metadata, @@ -298,7 +292,7 @@ void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } -ASTPtr parseAdditionalFilterConditionForTable( +static ASTPtr parseAdditionalFilterConditionForTable( const Map & setting, const DatabaseAndTableWithAlias & target, const Context & context) @@ -326,7 +320,7 @@ ASTPtr parseAdditionalFilterConditionForTable( } /// Returns true if we should ignore quotas and limits for a specified table in the system database. -bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) +static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { if (table_id.database_name == DatabaseCatalog::SYSTEM_DATABASE) { @@ -337,8 +331,6 @@ bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) return false; } -} - InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const ContextPtr & context_, @@ -1417,23 +1409,17 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( - query_plan.getCurrentDataStream(), - new_filter_info->actions, - new_filter_info->column_name, - new_filter_info->do_remove_column); - - filter_step->setStepDescription(description); - query_plan.addStep(std::move(filter_step)); - }; - if (additional_filter_info) - add_filter_step(additional_filter_info, "Additional filter"); + { + auto additional_filter_step = std::make_unique( + query_plan.getCurrentDataStream(), + additional_filter_info->actions, + additional_filter_info->column_name, + additional_filter_info->do_remove_column); - if (parallel_replicas_custom_filter_info) - add_filter_step(parallel_replicas_custom_filter_info, "Parallel replica custom key filter"); + additional_filter_step->setStepDescription("Additional filter"); + query_plan.addStep(std::move(additional_filter_step)); + } if (expressions.before_array_join) { From 1c0a3e38c0adb78ee719310161fdf329393ded55 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 08:13:59 +0000 Subject: [PATCH 018/470] Fix queries with Distributed storage --- .../ClusterProxy/executeQuery.cpp | 31 ++- src/Interpreters/ClusterProxy/executeQuery.h | 3 +- src/Storages/StorageDistributed.cpp | 236 ++++++++---------- 3 files changed, 137 insertions(+), 133 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index fc490306e08..d8d55b5486b 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,7 @@ #include #include + namespace DB { @@ -128,7 +130,7 @@ void executeQuery( const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const ClusterPtr & not_optimized_cluster, - std::function add_additional_shard_filter = {}) + AdditionalShardFilterGenerator shard_filter_generator) { const Settings & settings = context->getSettingsRef(); @@ -180,8 +182,31 @@ void executeQuery( else query_ast_for_shard = query_ast->clone(); - if (add_additional_shard_filter) - add_additional_shard_filter(query_ast_for_shard, shard_info.shard_num); + if (shard_filter_generator) + { + auto shard_filter = shard_filter_generator(shard_info.shard_num); + if (shard_filter) + { + auto & select_query = query_ast_for_shard->as(); + + auto where_expression = select_query.where(); + if (where_expression) + { + ASTPtr args = std::make_shared(); + args->children.push_back(where_expression); + args->children.push_back(shard_filter); + + auto and_function = std::make_shared(); + and_function->name = "and"; + and_function->arguments = args; + and_function->children.push_back(and_function->arguments); + + shard_filter = std::move(and_function); + } + + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter)); + } + } stream_factory.createForShard(shard_info, query_ast_for_shard, main_table, table_func_ptr, diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 9519fa00efe..25827ebf29a 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -37,6 +37,7 @@ class SelectStreamFactory; ContextMutablePtr updateSettingsForCluster( const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info = nullptr, Poco::Logger * log = nullptr); +using AdditionalShardFilterGenerator = std::function; /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. /// `stream_factory` object encapsulates the logic of creating plans for a different type of query /// (currently SELECT, DESCRIBE). @@ -51,7 +52,7 @@ void executeQuery( const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const ClusterPtr & not_optimized_cluster, - std::function add_additional_shard_filter); + AdditionalShardFilterGenerator shard_filter_generator = {}); void executeQueryWithParallelReplicas( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0b36c6c0026..a7c0dafaf62 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -758,22 +758,26 @@ void StorageDistributed::read( bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; - ParserExpression parser; - auto custom_key_ast = parseQuery( - parser, - settings.parallel_replicas_custom_key.value.data(), - settings.parallel_replicas_custom_key.value.data() + settings.parallel_replicas_custom_key.value.size(), - "parallel replicas custom key", - settings.max_query_size, - settings.max_parser_depth); - auto shard_count = query_info.getCluster()->getShardCount(); - std::function add_additional_shard_filter; + ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; if (settings.max_parallel_replicas > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) { - add_additional_shard_filter = [&](ASTPtr & query, uint64_t shard_num) + + const std::string_view custom_key = settings.parallel_replicas_custom_key.value; + assert(!custom_key.empty()); + + ParserExpression parser; + auto custom_key_ast = parseQuery( + parser, + custom_key.data(), + custom_key.data() + custom_key.size(), + "parallel replicas custom key", + settings.max_query_size, + settings.max_parser_depth); + + additional_shard_filter_generator = [&](uint64_t shard_num) -> ASTPtr { ParserExpression parser; auto custom_key_ast = parseQuery( @@ -803,130 +807,104 @@ void StorageDistributed::read( equals_function->arguments = args; equals_function->children.push_back(equals_function->arguments); - shard_filter = equals_function; + return equals_function; } else { assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); - auto filter_function = [&] + + KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); + + using RelativeSize = boost::rational; + + RelativeSize size_of_universum = 0; + DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; + + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (custom_key_description.data_types.size() == 1) { - KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); - - using RelativeSize = boost::rational; - - RelativeSize size_of_universum = 0; - DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - if (custom_key_description.data_types.size() == 1) - { - if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - - RelativeSize relative_range_size = RelativeSize(1) / query_info.getCluster()->getShardCount(); - RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - assert(has_lower_limit || has_upper_limit); - - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - if (has_lower_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(lower)); - - lower_function = std::make_shared(); - lower_function->name = "greaterOrEquals"; - lower_function->arguments = args; - lower_function->children.push_back(lower_function->arguments); - - if (!has_upper_limit) - return lower_function; - } - - if (has_upper_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(upper)); - - upper_function = std::make_shared(); - upper_function->name = "less"; - upper_function->arguments = args; - upper_function->children.push_back(upper_function->arguments); - - if (!has_lower_limit) - return upper_function; - } - - assert(has_lower_limit && has_upper_limit); - - ASTPtr args = std::make_shared(); - args->children.push_back(lower_function); - args->children.push_back(upper_function); - - auto and_function = std::make_shared(); - and_function->name = "and"; - and_function->arguments = args; - and_function->children.push_back(and_function->arguments); - - return and_function; - }; - - shard_filter = filter_function(); - } - - if (shard_filter) - { - auto & select_query = query->as(); - - auto where_expression = select_query.where(); - if (where_expression) - { - ASTPtr args = std::make_shared(); - args->children.push_back(where_expression); - args->children.push_back(shard_filter); - - auto and_function = std::make_shared(); - and_function->name = "and"; - and_function->arguments = args; - and_function->children.push_back(and_function->arguments); - - shard_filter = std::move(and_function); + if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); } - select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter)); + if (size_of_universum == RelativeSize(0)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + + RelativeSize relative_range_size = RelativeSize(1) / query_info.getCluster()->getShardCount(); + RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + assert(has_lower_limit || has_upper_limit); + + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + if (has_lower_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(lower)); + + lower_function = std::make_shared(); + lower_function->name = "greaterOrEquals"; + lower_function->arguments = args; + lower_function->children.push_back(lower_function->arguments); + + if (!has_upper_limit) + return lower_function; + } + + if (has_upper_limit) + { + ASTPtr args = std::make_shared(); + args->children.push_back(custom_key_ast); + args->children.push_back(std::make_shared(upper)); + + upper_function = std::make_shared(); + upper_function->name = "less"; + upper_function->arguments = args; + upper_function->children.push_back(upper_function->arguments); + + if (!has_lower_limit) + return upper_function; + } + + assert(has_lower_limit && has_upper_limit); + + ASTPtr args = std::make_shared(); + args->children.push_back(lower_function); + args->children.push_back(upper_function); + + auto and_function = std::make_shared(); + and_function->name = "and"; + and_function->arguments = args; + and_function->children.push_back(and_function->arguments); + + return and_function; } }; } @@ -945,7 +923,7 @@ void StorageDistributed::read( select_stream_factory, log, modified_query_ast, local_context, query_info, sharding_key_expr, sharding_key_column_name, - query_info.cluster, add_additional_shard_filter); + query_info.cluster, additional_shard_filter_generator); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) From 7a75144ce372da6134ae624668006c1348889d1e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 09:20:40 +0000 Subject: [PATCH 019/470] Refactor --- src/Interpreters/Cluster.cpp | 11 +- src/Interpreters/Cluster.h | 4 +- .../ClusterProxy/executeQuery.cpp | 13 +- src/Storages/StorageDistributed.cpp | 207 +++++++----------- 4 files changed, 92 insertions(+), 143 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 9f0a9d3b35c..7e3e1baf6f2 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -653,9 +653,9 @@ void Cluster::initMisc() } } -std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings) const +std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard) const { - return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings)}; + return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings, max_replicas_from_shard)}; } std::unique_ptr Cluster::getClusterWithSingleShard(size_t index) const @@ -668,7 +668,7 @@ std::unique_ptr Cluster::getClusterWithMultipleShards(const std::vector return std::unique_ptr{ new Cluster(SubclusterTag{}, *this, indices) }; } -Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Settings & settings) +Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard) { if (from.addresses_with_failover.empty()) throw Exception("Cluster is empty", ErrorCodes::LOGICAL_ERROR); @@ -678,6 +678,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti for (size_t shard_index : collections::range(0, from.shards_info.size())) { const auto & replicas = from.addresses_with_failover[shard_index]; + size_t replicas_used = 0; for (const auto & address : replicas) { if (!unique_hosts.emplace(address.host_name, address.port).second) @@ -685,6 +686,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti ShardInfo info; info.shard_num = ++shard_num; + ++replicas_used; if (address.is_local) info.local_addresses.push_back(address); @@ -711,6 +713,9 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti addresses_with_failover.emplace_back(Addresses{address}); shards_info.emplace_back(std::move(info)); + + if (max_replicas_from_shard && replicas_used == max_replicas_from_shard) + break; } } diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index ada04aa1cae..77e87e48ca7 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -250,7 +250,7 @@ public: std::unique_ptr getClusterWithMultipleShards(const std::vector & indices) const; /// Get a new Cluster that contains all servers (all shards with all replicas) from existing cluster as independent shards. - std::unique_ptr getClusterWithReplicasAsShards(const Settings & settings) const; + std::unique_ptr getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard = 0) const; /// Returns false if cluster configuration doesn't allow to use it for cross-replication. /// NOTE: true does not mean, that it's actually a cross-replication cluster. @@ -271,7 +271,7 @@ private: /// For getClusterWithReplicasAsShards implementation struct ReplicasAsShardsTag {}; - Cluster(ReplicasAsShardsTag, const Cluster & from, const Settings & settings); + Cluster(ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard); /// Inter-server secret String secret; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d8d55b5486b..02673b9f7ac 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -191,18 +191,7 @@ void executeQuery( auto where_expression = select_query.where(); if (where_expression) - { - ASTPtr args = std::make_shared(); - args->children.push_back(where_expression); - args->children.push_back(shard_filter); - - auto and_function = std::make_shared(); - and_function->name = "and"; - and_function->arguments = args; - and_function->children.push_back(and_function->arguments); - - shard_filter = std::move(and_function); - } + shard_filter = makeASTFunction("and", where_expression, shard_filter); select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(shard_filter)); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a7c0dafaf62..82c38868cb4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -445,10 +445,6 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr cluster = getCluster(); - // if it's custom_key we will turn replicas into shards and filter specific data on each of them - if (settings.max_parallel_replicas > 1 && cluster->getShardCount() == 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) - cluster = cluster->getClusterWithReplicasAsShards(settings); - query_info.cluster = cluster; size_t nodes = getClusterQueriedNodes(settings, cluster); @@ -758,15 +754,19 @@ void StorageDistributed::read( bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; - auto shard_count = query_info.getCluster()->getShardCount(); - ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; if (settings.max_parallel_replicas > 1 - && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY + && getCluster()->getShardCount() == 1) { + LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into shards"); + + query_info.cluster = getCluster()->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); + query_info.optimized_cluster = nullptr; // it's a single shard cluster so nothing could've been optimized const std::string_view custom_key = settings.parallel_replicas_custom_key.value; - assert(!custom_key.empty()); + if (custom_key.empty()) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); ParserExpression parser; auto custom_key_ast = parseQuery( @@ -777,135 +777,90 @@ void StorageDistributed::read( settings.max_query_size, settings.max_parser_depth); - additional_shard_filter_generator = [&](uint64_t shard_num) -> ASTPtr + additional_shard_filter_generator = [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) mutable -> ASTPtr { - ParserExpression parser; - auto custom_key_ast = parseQuery( - parser, settings.parallel_replicas_custom_key.value.data(), settings.parallel_replicas_custom_key.value.data() + settings.parallel_replicas_custom_key.value.size(), - "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); - - ASTPtr shard_filter = nullptr ; if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) { // first we do modulo with replica count - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(shard_count)); - - auto modulo_function = std::make_shared(); - modulo_function->name = "positiveModulo"; - modulo_function->arguments = args; - modulo_function->children.push_back(modulo_function->arguments); + auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(shard_count)); /// then we compare result to the current replica number (offset) - args = std::make_shared(); - args->children.push_back(modulo_function); - args->children.push_back(std::make_shared(shard_num - 1)); - - auto equals_function = std::make_shared(); - equals_function->name = "equals"; - equals_function->arguments = args; - equals_function->children.push_back(equals_function->arguments); + auto equals_function = makeASTFunction("equals", std::move(modulo_function), std::make_shared(shard_num - 1)); return equals_function; } - else + + assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + + KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); + + using RelativeSize = boost::rational; + + RelativeSize size_of_universum = 0; + DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; + + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (custom_key_description.data_types.size() == 1) { - assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); - - KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); - - using RelativeSize = boost::rational; - - RelativeSize size_of_universum = 0; - DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - if (custom_key_description.data_types.size() == 1) - { - if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - - RelativeSize relative_range_size = RelativeSize(1) / query_info.getCluster()->getShardCount(); - RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - assert(has_lower_limit || has_upper_limit); - - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - if (has_lower_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(lower)); - - lower_function = std::make_shared(); - lower_function->name = "greaterOrEquals"; - lower_function->arguments = args; - lower_function->children.push_back(lower_function->arguments); - - if (!has_upper_limit) - return lower_function; - } - - if (has_upper_limit) - { - ASTPtr args = std::make_shared(); - args->children.push_back(custom_key_ast); - args->children.push_back(std::make_shared(upper)); - - upper_function = std::make_shared(); - upper_function->name = "less"; - upper_function->arguments = args; - upper_function->children.push_back(upper_function->arguments); - - if (!has_lower_limit) - return upper_function; - } - - assert(has_lower_limit && has_upper_limit); - - ASTPtr args = std::make_shared(); - args->children.push_back(lower_function); - args->children.push_back(upper_function); - - auto and_function = std::make_shared(); - and_function->name = "and"; - and_function->arguments = args; - and_function->children.push_back(and_function->arguments); - - return and_function; + if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); + + RelativeSize relative_range_size = RelativeSize(1) / shard_count; + RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + assert(has_lower_limit || has_upper_limit); + + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + if (has_lower_limit) + { + lower_function = makeASTFunction("greaterOrEquals", custom_key_ast, std::make_shared(lower)); + + if (!has_upper_limit) + return lower_function; + } + + if (has_upper_limit) + { + upper_function = makeASTFunction("less", custom_key_ast, std::make_shared(upper)); + + if (!has_lower_limit) + return upper_function; + } + + assert(upper_function && lower_function); + + return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); }; } From 3b0c63551ec244ae1d9ff38e70c04c98b5342b7d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 10:26:38 +0000 Subject: [PATCH 020/470] Combine approaches --- src/Interpreters/InterpreterSelectQuery.cpp | 84 +++++++++++--- .../getCustomKeyFilterForParallelReplicas.cpp | 109 ++++++++++++++++++ .../getCustomKeyFilterForParallelReplicas.h | 21 ++++ src/Storages/StorageDistributed.cpp | 89 +------------- 4 files changed, 206 insertions(+), 97 deletions(-) create mode 100644 src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp create mode 100644 src/Interpreters/getCustomKeyFilterForParallelReplicas.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 48326afda45..3612458196c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -94,6 +95,9 @@ #include #include #include +#include "Core/SettingsEnums.h" + +#include namespace DB { @@ -112,6 +116,8 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -227,10 +233,13 @@ InterpreterSelectQuery::InterpreterSelectQuery( InterpreterSelectQuery::~InterpreterSelectQuery() = default; +namespace +{ + /** There are no limits on the maximum size of the result for the subquery. * Since the result of the query is not the result of the entire query. */ -static ContextPtr getSubqueryContext(const ContextPtr & context) +ContextPtr getSubqueryContext(const ContextPtr & context) { auto subquery_context = Context::createCopy(context); Settings subquery_settings = context->getSettings(); @@ -242,7 +251,7 @@ static ContextPtr getSubqueryContext(const ContextPtr & context) return subquery_context; } -static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) +void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) { ASTSelectQuery & select = query->as(); @@ -262,7 +271,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table } /// Checks that the current user has the SELECT privilege. -static void checkAccessRightsForSelect( +void checkAccessRightsForSelect( const ContextPtr & context, const StorageID & table_id, const StorageMetadataPtr & table_metadata, @@ -292,7 +301,7 @@ static void checkAccessRightsForSelect( context->checkAccess(AccessType::SELECT, table_id, syntax_analyzer_result.requiredSourceColumnsForAccessCheck()); } -static ASTPtr parseAdditionalFilterConditionForTable( +ASTPtr parseAdditionalFilterConditionForTable( const Map & setting, const DatabaseAndTableWithAlias & target, const Context & context) @@ -319,8 +328,20 @@ static ASTPtr parseAdditionalFilterConditionForTable( return nullptr; } +ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) +{ + if (setting.empty()) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); + + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, setting.data(), setting.data() + setting.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); +} + /// Returns true if we should ignore quotas and limits for a specified table in the system database. -static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) +bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { if (table_id.database_name == DatabaseCatalog::SYSTEM_DATABASE) { @@ -331,6 +352,8 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) return false; } +} + InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const ContextPtr & context_, @@ -501,7 +524,24 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.additional_filter_ast = parseAdditionalFilterConditionForTable( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); - auto analyze = [&] (bool try_move_to_prewhere) + ASTPtr parallel_replicas_custom_filter_ast = nullptr; + if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + { + LOG_INFO(log, "Processing query on a replica using custom_key"); + if (!storage) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); + + auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key, *context); + parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + settings.parallel_replicas_custom_key_filter_type, + *storage, + context); + } + + auto analyze = [&](bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; @@ -666,6 +706,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.push_back(query_info.additional_filter_ast); } + if (parallel_replicas_custom_filter_ast) + { + parallel_replicas_custom_filter_info = generateFilterActions( + table_id, parallel_replicas_custom_filter_ast, context, storage, storage_snapshot, metadata_snapshot, required_columns, + prepared_sets); + + parallel_replicas_custom_filter_info->do_remove_column = true; + query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); + } + source_header = storage_snapshot->getSampleBlockForColumns(required_columns, parameter_values); } @@ -1409,17 +1459,23 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( + auto filter_step = std::make_unique( query_plan.getCurrentDataStream(), - additional_filter_info->actions, - additional_filter_info->column_name, - additional_filter_info->do_remove_column); + new_filter_info->actions, + new_filter_info->column_name, + new_filter_info->do_remove_column); - additional_filter_step->setStepDescription("Additional filter"); - query_plan.addStep(std::move(additional_filter_step)); - } + filter_step->setStepDescription(description); + query_plan.addStep(std::move(filter_step)); + }; + + if (additional_filter_info) + add_filter_step(additional_filter_info, "Additional filter"); + + if (parallel_replicas_custom_filter_info) + add_filter_step(parallel_replicas_custom_filter_info, "Parallel replica custom key filter"); if (expressions.before_array_join) { diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp new file mode 100644 index 00000000000..8e7fa8386fe --- /dev/null +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -0,0 +1,109 @@ +#include + +#include +#include + +#include + +#include + + +namespace DB +{ + +ASTPtr getCustomKeyFilterForParallelReplica( + size_t replicas_count, + size_t replica_num, + ASTPtr custom_key_ast, + ParallelReplicasCustomKeyFilterType filter_type, + const IStorage & storage, + const ContextPtr & context) +{ + assert(replicas_count > 1); + if (filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) + { + // first we do modulo with replica count + auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(replicas_count)); + + /// then we compare result to the current replica number (offset) + auto equals_function = makeASTFunction("equals", std::move(modulo_function), std::make_shared(replica_num)); + + return equals_function; + } + + assert(filter_type == ParallelReplicasCustomKeyFilterType::RANGE); + + KeyDescription custom_key_description + = KeyDescription::getKeyFromAST(custom_key_ast, storage.getInMemoryMetadataPtr()->columns, context); + + using RelativeSize = boost::rational; + + RelativeSize size_of_universum = 0; + DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; + + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + if (custom_key_description.data_types.size() == 1) + { + if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + else if (typeid_cast(custom_key_column_type.get())) + size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); + } + + if (size_of_universum == RelativeSize(0)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Invalid custom key column type: {}. Must be one unsigned integer type", + custom_key_column_type->getName()); + + RelativeSize relative_range_size = RelativeSize(1) / replicas_count; + RelativeSize relative_range_offset = relative_range_size * RelativeSize(replica_num); + + /// Calculate the half-interval of `[lower, upper)` column values. + bool has_lower_limit = false; + bool has_upper_limit = false; + + RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; + RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; + + UInt64 lower = boost::rational_cast(lower_limit_rational); + UInt64 upper = boost::rational_cast(upper_limit_rational); + + if (lower > 0) + has_lower_limit = true; + + if (upper_limit_rational < size_of_universum) + has_upper_limit = true; + + assert(has_lower_limit || has_upper_limit); + + /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. + std::shared_ptr lower_function; + std::shared_ptr upper_function; + + if (has_lower_limit) + { + lower_function = makeASTFunction("greaterOrEquals", custom_key_ast, std::make_shared(lower)); + + if (!has_upper_limit) + return lower_function; + } + + if (has_upper_limit) + { + upper_function = makeASTFunction("less", custom_key_ast, std::make_shared(upper)); + + if (!has_lower_limit) + return upper_function; + } + + assert(upper_function && lower_function); + + return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); +} + +} diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h new file mode 100644 index 00000000000..3d830d1d606 --- /dev/null +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +#include +#include "Core/SettingsEnums.h" + +namespace DB +{ + +/// Get AST for filter created from custom_key +/// replica_num is the number of the replica for which we are generating filter starting from 0 +ASTPtr getCustomKeyFilterForParallelReplica( + size_t replicas_count, + size_t replica_num, + ASTPtr custom_key_ast, + ParallelReplicasCustomKeyFilterType filter_type, + const IStorage & storage, + const ContextPtr & context); + +} diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 82c38868cb4..a0aa646869f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -61,6 +61,8 @@ #include #include #include +#include + #include #include #include @@ -777,90 +779,11 @@ void StorageDistributed::read( settings.max_query_size, settings.max_parser_depth); - additional_shard_filter_generator = [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) mutable -> ASTPtr + additional_shard_filter_generator = + [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) -> ASTPtr { - if (settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::DEFAULT) - { - // first we do modulo with replica count - auto modulo_function = makeASTFunction("positiveModulo", custom_key_ast, std::make_shared(shard_count)); - - /// then we compare result to the current replica number (offset) - auto equals_function = makeASTFunction("equals", std::move(modulo_function), std::make_shared(shard_num - 1)); - - return equals_function; - } - - assert(settings.parallel_replicas_custom_key_filter_type == ParallelReplicasCustomKeyFilterType::RANGE); - - KeyDescription custom_key_description = KeyDescription::getKeyFromAST(custom_key_ast, getInMemoryMetadataPtr()->columns, local_context); - - using RelativeSize = boost::rational; - - RelativeSize size_of_universum = 0; - DataTypePtr custom_key_column_type = custom_key_description.data_types[0]; - - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - if (custom_key_description.data_types.size() == 1) - { - if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - else if (typeid_cast(custom_key_column_type.get())) - size_of_universum = RelativeSize(std::numeric_limits::max()) + RelativeSize(1); - } - - if (size_of_universum == RelativeSize(0)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); - - RelativeSize relative_range_size = RelativeSize(1) / shard_count; - RelativeSize relative_range_offset = relative_range_size * RelativeSize(shard_num - 1); - - /// Calculate the half-interval of `[lower, upper)` column values. - bool has_lower_limit = false; - bool has_upper_limit = false; - - RelativeSize lower_limit_rational = relative_range_offset * size_of_universum; - RelativeSize upper_limit_rational = (relative_range_offset + relative_range_size) * size_of_universum; - - UInt64 lower = boost::rational_cast(lower_limit_rational); - UInt64 upper = boost::rational_cast(upper_limit_rational); - - if (lower > 0) - has_lower_limit = true; - - if (upper_limit_rational < size_of_universum) - has_upper_limit = true; - - assert(has_lower_limit || has_upper_limit); - - /// Let's add the conditions to cut off something else when the index is scanned again and when the request is processed. - std::shared_ptr lower_function; - std::shared_ptr upper_function; - - if (has_lower_limit) - { - lower_function = makeASTFunction("greaterOrEquals", custom_key_ast, std::make_shared(lower)); - - if (!has_upper_limit) - return lower_function; - } - - if (has_upper_limit) - { - upper_function = makeASTFunction("less", custom_key_ast, std::make_shared(upper)); - - if (!has_lower_limit) - return upper_function; - } - - assert(upper_function && lower_function); - - return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); + return getCustomKeyFilterForParallelReplica( + shard_count, shard_num - 1, custom_key_ast, settings.parallel_replicas_custom_key_filter_type, *this, local_context); }; } From ddfb913f9991b4e410744ebb0eac260d842df4e2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 11:28:26 +0000 Subject: [PATCH 021/470] better --- src/Interpreters/InterpreterSelectQuery.cpp | 18 +---- .../getCustomKeyFilterForParallelReplicas.cpp | 23 +++++- .../getCustomKeyFilterForParallelReplicas.h | 4 +- src/Storages/StorageDistributed.cpp | 75 ++++++++++--------- 4 files changed, 64 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3612458196c..0aa48ca1998 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -95,9 +95,6 @@ #include #include #include -#include "Core/SettingsEnums.h" - -#include namespace DB { @@ -117,7 +114,6 @@ namespace ErrorCodes extern const int ACCESS_DENIED; extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -328,18 +324,6 @@ ASTPtr parseAdditionalFilterConditionForTable( return nullptr; } -ASTPtr parseParallelReplicaCustomKey(const String & setting, const Context & context) -{ - if (setting.empty()) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); - - ParserExpression parser; - const auto & settings = context.getSettingsRef(); - return parseQuery( - parser, setting.data(), setting.data() + setting.size(), - "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); -} - /// Returns true if we should ignore quotas and limits for a specified table in the system database. bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { @@ -531,7 +515,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!storage) throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); - auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key, *context); + auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key.value, *context); parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( settings.parallel_replicas_count, settings.parallel_replica_offset, diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 8e7fa8386fe..306ca129e16 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -2,6 +2,10 @@ #include #include +#include +#include + +#include #include @@ -11,6 +15,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; +} + ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, @@ -56,7 +66,7 @@ ASTPtr getCustomKeyFilterForParallelReplica( if (size_of_universum == RelativeSize(0)) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid custom key column type: {}. Must be one unsigned integer type", custom_key_column_type->getName()); @@ -106,4 +116,15 @@ ASTPtr getCustomKeyFilterForParallelReplica( return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); } +ASTPtr parseParallelReplicaCustomKey(std::string_view custom_key, const Context & context) +{ + if (custom_key.empty()) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); + + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, custom_key.data(), custom_key.data() + custom_key.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); +} } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 3d830d1d606..251e72e0a97 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -3,7 +3,7 @@ #include #include #include -#include "Core/SettingsEnums.h" +#include namespace DB { @@ -18,4 +18,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( const IStorage & storage, const ContextPtr & context); +ASTPtr parseParallelReplicaCustomKey(std::string_view custom_key, const Context & context); + } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a0aa646869f..e66a363003e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -135,7 +135,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int UNSUPPORTED_METHOD; - extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } namespace ActionLocks @@ -447,29 +446,40 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( ClusterPtr cluster = getCluster(); - query_info.cluster = cluster; - size_t nodes = getClusterQueriedNodes(settings, cluster); - /// Always calculate optimized cluster here, to avoid conditions during read() - /// (Anyway it will be calculated in the read()) - if (nodes > 1 && settings.optimize_skip_unused_shards) + if (settings.max_parallel_replicas > 1 + && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY + && cluster->getShardCount() == 1) { - ClusterPtr optimized_cluster = getOptimizedCluster(local_context, storage_snapshot, query_info.query); - if (optimized_cluster) - { - LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", - makeFormattedListOfShards(optimized_cluster)); + LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards"); - cluster = optimized_cluster; - query_info.optimized_cluster = cluster; + query_info.cluster = cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); + } + else + { + query_info.cluster = cluster; - nodes = getClusterQueriedNodes(settings, cluster); - } - else + if (nodes > 1 && settings.optimize_skip_unused_shards) { - LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", - has_sharding_key ? "" : " (no sharding key)"); + /// Always calculate optimized cluster here, to avoid conditions during read() + /// (Anyway it will be calculated in the read()) + ClusterPtr optimized_cluster = getOptimizedCluster(local_context, storage_snapshot, query_info.query); + if (optimized_cluster) + { + LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}", + makeFormattedListOfShards(optimized_cluster)); + + cluster = optimized_cluster; + query_info.optimized_cluster = cluster; + + nodes = getClusterQueriedNodes(settings, cluster); + } + else + { + LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}", + has_sharding_key ? "" : " (no sharding key)"); + } } } @@ -751,33 +761,21 @@ void StorageDistributed::read( storage_snapshot, processed_stage); - auto settings = local_context->getSettingsRef(); - bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas - && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; if (settings.max_parallel_replicas > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && getCluster()->getShardCount() == 1) { - LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into shards"); + if (query_info.getCluster()->getShardCount() == 1) + { + // we are reading from single shard replica but didn't transform replicas + // into virtual shards with custom_key set + throw Exception(ErrorCodes::LOGICAL_ERROR, "Replicas weren't transformed into virtual shards"); + } - query_info.cluster = getCluster()->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); - query_info.optimized_cluster = nullptr; // it's a single shard cluster so nothing could've been optimized - - const std::string_view custom_key = settings.parallel_replicas_custom_key.value; - if (custom_key.empty()) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); - - ParserExpression parser; - auto custom_key_ast = parseQuery( - parser, - custom_key.data(), - custom_key.data() + custom_key.size(), - "parallel replicas custom key", - settings.max_query_size, - settings.max_parser_depth); + auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key.value, *local_context); additional_shard_filter_generator = [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) -> ASTPtr @@ -787,6 +785,9 @@ void StorageDistributed::read( }; } + bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas + && !settings.use_hedged_requests && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS; + if (parallel_replicas) ClusterProxy::executeQueryWithParallelReplicas( query_plan, main_table, remote_table_function_ptr, From 53b53a1ec90c385c2b68806e4a7f4078ecf64945 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 12:04:07 +0000 Subject: [PATCH 022/470] Add replcia shuffling --- src/Interpreters/Cluster.cpp | 92 ++++++++++++++++++++++++------------ 1 file changed, 61 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 7e3e1baf6f2..97970691025 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -15,6 +15,7 @@ #include #include +#include namespace DB { @@ -677,45 +678,74 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti std::set> unique_hosts; for (size_t shard_index : collections::range(0, from.shards_info.size())) { - const auto & replicas = from.addresses_with_failover[shard_index]; - size_t replicas_used = 0; - for (const auto & address : replicas) + auto create_shards_from_replicas = [&](std::span replicas) { - if (!unique_hosts.emplace(address.host_name, address.port).second) - continue; /// Duplicate host, skip. + for (const auto & address : replicas) + { + if (!unique_hosts.emplace(address.host_name, address.port).second) + continue; /// Duplicate host, skip. - ShardInfo info; - info.shard_num = ++shard_num; - ++replicas_used; + ShardInfo info; + info.shard_num = ++shard_num; - if (address.is_local) - info.local_addresses.push_back(address); + if (address.is_local) + info.local_addresses.push_back(address); - info.all_addresses.push_back(address); + info.all_addresses.push_back(address); - 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.cluster, - address.cluster_secret, - "server", - address.compression, - address.secure, - address.priority); + 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.cluster, + address.cluster_secret, + "server", + address.compression, + address.secure, + address.priority); - info.pool = std::make_shared(ConnectionPoolPtrs{pool}, settings.load_balancing); - info.per_replica_pools = {std::move(pool)}; + info.pool = std::make_shared(ConnectionPoolPtrs{pool}, settings.load_balancing); + info.per_replica_pools = {std::move(pool)}; - addresses_with_failover.emplace_back(Addresses{address}); - shards_info.emplace_back(std::move(info)); + addresses_with_failover.emplace_back(Addresses{address}); + shards_info.emplace_back(std::move(info)); + } + }; - if (max_replicas_from_shard && replicas_used == max_replicas_from_shard) - break; + const auto & replicas = from.addresses_with_failover[shard_index]; + if (!max_replicas_from_shard || replicas.size() <= max_replicas_from_shard) + { + create_shards_from_replicas(replicas); + } + else + { + std::random_device rd; + std::mt19937 gen{rd()}; + auto shuffled_replicas = replicas; + + if (settings.prefer_localhost_replica) + { + auto local_replica = std::find_if(shuffled_replicas.begin(), shuffled_replicas.end(), [](const auto & replica) { return replica.is_local; }); + if (local_replica != shuffled_replicas.end()) + { + std::swap(*shuffled_replicas.begin(), *local_replica); + std::shuffle(shuffled_replicas.begin() + 1, shuffled_replicas.end(), gen); + } + else + { + std::shuffle(shuffled_replicas.begin(), shuffled_replicas.end(), gen); + } + } + else + { + std::shuffle(shuffled_replicas.begin(), shuffled_replicas.end(), gen); + } + + create_shards_from_replicas(std::span{shuffled_replicas.begin(), shuffled_replicas.begin() + max_replicas_from_shard}); } } From d8adff35c3e1851e3f85ffa0ffa3780b0114bd51 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 12:08:32 +0000 Subject: [PATCH 023/470] Fix style --- src/Interpreters/Cluster.cpp | 6 +++++- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 97970691025..9661b3f4e19 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -723,16 +723,20 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti } else { + // shuffle replicas so we don't always pick the same subset std::random_device rd; std::mt19937 gen{rd()}; auto shuffled_replicas = replicas; if (settings.prefer_localhost_replica) { + // force for local replica to always be included auto local_replica = std::find_if(shuffled_replicas.begin(), shuffled_replicas.end(), [](const auto & replica) { return replica.is_local; }); if (local_replica != shuffled_replicas.end()) { - std::swap(*shuffled_replicas.begin(), *local_replica); + if (local_replica != shuffled_replicas.begin()) + std::swap(*shuffled_replicas.begin(), *local_replica); + std::shuffle(shuffled_replicas.begin() + 1, shuffled_replicas.end(), gen); } else diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 306ca129e16..fbc1d214164 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -17,8 +17,8 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; + extern const int BAD_ARGUMENTS; } ASTPtr getCustomKeyFilterForParallelReplica( From d93cb3e1ddb5192f208e259fa57d4ccd5f29dffa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 13:24:35 +0000 Subject: [PATCH 024/470] More correct check --- src/Storages/StorageDistributed.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e66a363003e..ea7cb9d33cb 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -307,6 +307,11 @@ size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & clus return (num_remote_shards + num_local_shards) * settings.max_parallel_replicas; } +bool useVirtualShards(const Settings & settings, const Cluster & cluster) +{ + return settings.max_parallel_replicas > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY + && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; +} } @@ -448,9 +453,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( size_t nodes = getClusterQueriedNodes(settings, cluster); - if (settings.max_parallel_replicas > 1 - && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY - && cluster->getShardCount() == 1) + if (useVirtualShards(settings, *cluster)) { LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards"); @@ -764,13 +767,11 @@ void StorageDistributed::read( auto settings = local_context->getSettingsRef(); ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; - if (settings.max_parallel_replicas > 1 - && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY - && getCluster()->getShardCount() == 1) + if (useVirtualShards(settings, *getCluster())) { if (query_info.getCluster()->getShardCount() == 1) { - // we are reading from single shard replica but didn't transform replicas + // we are reading from single shard with multiple replicas but didn't transform replicas // into virtual shards with custom_key set throw Exception(ErrorCodes::LOGICAL_ERROR, "Replicas weren't transformed into virtual shards"); } From 6394a004fb8014f81baf98b500163a779640478c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 14:47:58 +0000 Subject: [PATCH 025/470] Refactor --- src/Interpreters/Cluster.cpp | 54 +++++++++++++++++++----------------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 9661b3f4e19..76609cf1a5b 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -669,6 +669,33 @@ std::unique_ptr Cluster::getClusterWithMultipleShards(const std::vector return std::unique_ptr{ new Cluster(SubclusterTag{}, *this, indices) }; } +namespace +{ + +void shuffleReplicas(auto & replicas, const Settings & settings) +{ + std::random_device rd; + std::mt19937 gen{rd()}; + + if (settings.prefer_localhost_replica) + { + // force for local replica to always be included + auto local_replica = std::find_if(replicas.begin(), replicas.end(), [](const auto & replica) { return replica.is_local; }); + if (local_replica != replicas.end()) + { + if (local_replica != replicas.begin()) + std::swap(*replicas.begin(), *local_replica); + + std::shuffle(replicas.begin() + 1, replicas.end(), gen); + return; + } + } + + std::shuffle(replicas.begin(), replicas.end(), gen); +} + +} + Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard) { if (from.addresses_with_failover.empty()) @@ -723,32 +750,9 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti } else { - // shuffle replicas so we don't always pick the same subset - std::random_device rd; - std::mt19937 gen{rd()}; auto shuffled_replicas = replicas; - - if (settings.prefer_localhost_replica) - { - // force for local replica to always be included - auto local_replica = std::find_if(shuffled_replicas.begin(), shuffled_replicas.end(), [](const auto & replica) { return replica.is_local; }); - if (local_replica != shuffled_replicas.end()) - { - if (local_replica != shuffled_replicas.begin()) - std::swap(*shuffled_replicas.begin(), *local_replica); - - std::shuffle(shuffled_replicas.begin() + 1, shuffled_replicas.end(), gen); - } - else - { - std::shuffle(shuffled_replicas.begin(), shuffled_replicas.end(), gen); - } - } - else - { - std::shuffle(shuffled_replicas.begin(), shuffled_replicas.end(), gen); - } - + // shuffle replicas so we don't always pick the same subset + shuffleReplicas(shuffled_replicas, settings); create_shards_from_replicas(std::span{shuffled_replicas.begin(), shuffled_replicas.begin() + max_replicas_from_shard}); } } From ad429a9312146bb48058af48283ff627cf094b5c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 19 Jan 2023 15:16:05 +0000 Subject: [PATCH 026/470] Add missing checks --- src/Client/HedgedConnections.cpp | 4 +++- src/Client/MultiplexedConnections.cpp | 5 ++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index c7392a86a7e..12f1850c282 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -175,7 +175,9 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 + && (settings.parallel_replicas_mode != ParallelReplicasMode::READ_TASKS + || !settings.allow_experimental_parallel_reading_from_replicas); if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) { diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 87eda765a7a..81bae7f501b 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -134,6 +134,7 @@ void MultiplexedConnections::sendQuery( } bool parallel_reading_from_replicas = settings.max_parallel_replicas > 1 + && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS && settings.allow_experimental_parallel_reading_from_replicas /// To avoid trying to coordinate with clickhouse-benchmark, /// since it uses the same code. @@ -146,7 +147,9 @@ void MultiplexedConnections::sendQuery( } } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 + && (settings.parallel_replicas_mode != ParallelReplicasMode::READ_TASKS + || !settings.allow_experimental_parallel_reading_from_replicas); size_t num_replicas = replica_states.size(); if (num_replicas > 1) From 9600d16e9f9983c8df4c79b985a97e389a832ace Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 20 Jan 2023 10:09:26 +0000 Subject: [PATCH 027/470] better tests for custom_key --- .../02404_memory_bound_merging.sql | 2 +- ...max_parallel_replicas_custom_key.reference | 13 - ...02527_max_parallel_replicas_custom_key.sql | 36 - ...max_parallel_replicas_custom_key.reference | 659 ++++++++++++++++++ .../02535_max_parallel_replicas_custom_key.sh | 47 ++ 5 files changed, 707 insertions(+), 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference delete mode 100644 tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql create mode 100644 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference create mode 100755 tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index fb0c65e6a7c..f7db31dabb3 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, long create table t(a UInt64, b UInt64) engine=MergeTree order by a; system stop merges t; diff --git a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference deleted file mode 100644 index c51a7e10fb2..00000000000 --- a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.reference +++ /dev/null @@ -1,13 +0,0 @@ -Hello -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 diff --git a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql b/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql deleted file mode 100644 index b716600b9d5..00000000000 --- a/tests/queries/0_stateless/02527_max_parallel_replicas_custom_key.sql +++ /dev/null @@ -1,36 +0,0 @@ --- Tags: replica - -DROP TABLE IF EXISTS t; - -CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; -INSERT INTO t VALUES ('Hello'); - -SET max_parallel_replicas = 3; -SET parallel_replicas_mode = 'custom_key'; - -SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'sipHash64(x)'; - -DROP TABLE t; - -CREATE TABLE t (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); -INSERT INTO t SELECT toString(number), number FROM numbers(1000); - -SET max_parallel_replicas = 1; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -SET max_parallel_replicas = 2; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -SET max_parallel_replicas = 3; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -DROP TABLE t; diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference new file mode 100644 index 00000000000..01f48778b34 --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference @@ -0,0 +1,659 @@ +query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='sipHash64(x)' +filter_type='default' max_replicas=1 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='default' max_replicas=2 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='range' max_replicas=2 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=0 +Hello +(ReadFromRemote) +filter_type='default' max_replicas=1 prefer_localhost_replica=1 +Hello +(Expression) +ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 +filter_type='default' max_replicas=2 prefer_localhost_replica=1 +Hello +(Union) + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=1 +Hello +(Union) + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=1 +Hello +(Expression) +ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 +filter_type='range' max_replicas=2 prefer_localhost_replica=1 +Hello +(Union) + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=1 +Hello +(Union) + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x)' +filter_type='default' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='default' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='range' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='default' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='range' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='y' +filter_type='default' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='default' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='range' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='default' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='range' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + y' +filter_type='default' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='default' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='range' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='default' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='range' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + 1' +filter_type='default' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='default' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=0 +1000 +(ReadFromRemote) +filter_type='range' max_replicas=2 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=0 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='default' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='default' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=1 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform + (ReadFromStorage) + AggregatingTransform + ExpressionTransform + SourceFromSingleChunk 0 → 1 +filter_type='range' max_replicas=2 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 2 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) +filter_type='range' max_replicas=3 prefer_localhost_replica=1 +1000 +(Expression) +ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (Union) + (Aggregating) + AggregatingTransform + (Expression) + ExpressionTransform + (Filter) + FilterTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 + (ReadFromRemote) diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh new file mode 100755 index 00000000000..76138842e9a --- /dev/null +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function run_with_custom_key { + echo "query='$1' with custom_key='$2'" + for prefer_localhost_replica in 0 1; do + for filter_type in 'default' 'range'; do + for max_replicas in {1..3}; do + echo "filter_type='$filter_type' max_replicas=$max_replicas prefer_localhost_replica=$prefer_localhost_replica" + query="$1 SETTINGS max_parallel_replicas=$max_replicas\ + , parallel_replicas_mode='custom_key'\ + , parallel_replicas_custom_key='$2'\ + , parallel_replicas_custom_key_filter_type='$filter_type'\ + , max_threads=4\ + , prefer_localhost_replica=$prefer_localhost_replica" + $CLICKHOUSE_CLIENT --query="$query" + $CLICKHOUSE_CLIENT --query="EXPLAIN PIPELINE $query" + done + done + done +} + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS 02535_custom_key"; + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String) ENGINE = MergeTree ORDER BY x"; +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key VALUES ('Hello')"; + +run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)" "sipHash64(x)" + +$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" + +$CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)" +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number FROM numbers(1000)" + +function run_count_with_custom_key { + run_with_custom_key "SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)" "$1" +} + +run_count_with_custom_key "cityHash64(x)" +run_count_with_custom_key "y" +run_count_with_custom_key "cityHash64(x) + y" +run_count_with_custom_key "cityHash64(x) + 1" + +$CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" From 2cb106f1f9f5e893431e776c66c0b26c689a560a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 20 Jan 2023 12:55:26 +0000 Subject: [PATCH 028/470] Add integration test for custom_key --- .../__init__.py | 0 .../configs/remote_servers.xml | 50 ++++++++++ .../test_parallel_replicas_custom_key/test.py | 94 +++++++++++++++++++ 3 files changed, 144 insertions(+) create mode 100644 tests/integration/test_parallel_replicas_custom_key/__init__.py create mode 100644 tests/integration/test_parallel_replicas_custom_key/configs/remote_servers.xml create mode 100644 tests/integration/test_parallel_replicas_custom_key/test.py diff --git a/tests/integration/test_parallel_replicas_custom_key/__init__.py b/tests/integration/test_parallel_replicas_custom_key/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_parallel_replicas_custom_key/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_custom_key/configs/remote_servers.xml new file mode 100644 index 00000000000..308db461498 --- /dev/null +++ b/tests/integration/test_parallel_replicas_custom_key/configs/remote_servers.xml @@ -0,0 +1,50 @@ + + + + + false + + n1 + 9000 + + + n2 + 9000 + + + + false + + n3 + 9000 + + + n4 + 9000 + + + + + + false + + n1 + 9000 + + + n2 + 9000 + + + n3 + 9000 + + + n4 + 9000 + + + + + + diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py new file mode 100644 index 00000000000..0787234a7b1 --- /dev/null +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -0,0 +1,94 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +n1 = cluster.add_instance( + "n1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +n2 = cluster.add_instance( + "n2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +n3 = cluster.add_instance( + "n3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +n4 = cluster.add_instance( + "n4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True +) +nodes = [n1, n2, n3, n4] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_tables(cluster): + n1.query("DROP TABLE IF EXISTS dist_table") + n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") + + n1.query( + f"CREATE TABLE test_table ON CLUSTER {cluster} (key Int32, value String) Engine=MergeTree ORDER BY (key, sipHash64(value))" + ) + n1.query( + f""" + CREATE TABLE dist_table AS test_table + Engine=Distributed( + {cluster}, + currentDatabase(), + test_table, + rand() + ) + """ + ) + + +def insert_data(cluster, row_num): + create_tables(cluster) + n1.query(f"INSERT INTO dist_table SELECT number, number FROM numbers({row_num})") + n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") + + +@pytest.mark.parametrize("custom_key", ["sipHash64(value)", "key"]) +@pytest.mark.parametrize("filter_type", ["default", "range"]) +@pytest.mark.parametrize( + "cluster", + ["test_multiple_shards_multiple_replicas", "test_single_shard_multiple_replicas"], +) +def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter_type): + for node in nodes: + node.rotate_logs() + + row_num = 1000 + insert_data(cluster, row_num) + assert ( + int( + n1.query( + "SELECT count() FROM dist_table", + settings={ + "prefer_localhost_replica": 0, + "max_parallel_replicas": 3, + "parallel_replicas_mode": "custom_key", + "parallel_replicas_custom_key": custom_key, + "parallel_replicas_custom_key_filter_type": filter_type, + }, + ) + ) + == row_num + ) + + if cluster == "test_multiple_shards_multiple_replicas": + # we simply process query on all replicas for each shard by appending the filter on replica + assert all( + node.contains_in_log("Processing query on a replica using custom_key") + for node in nodes + ) + else: + # we first transform all replicas into shards and then append for each shard filter + assert n1.contains_in_log( + "Single shard cluster used with custom_key, transforming replicas into virtual shards" + ) From ec149c4585c5a7655a275126be59b1694aeafd22 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 20 Jan 2023 12:56:20 +0000 Subject: [PATCH 029/470] Remove old test --- ...rallel_replicas_custom_key_range.reference | 13 ------- ...max_parallel_replicas_custom_key_range.sql | 37 ------------------- 2 files changed, 50 deletions(-) delete mode 100644 tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference delete mode 100644 tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql diff --git a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference deleted file mode 100644 index c51a7e10fb2..00000000000 --- a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.reference +++ /dev/null @@ -1,13 +0,0 @@ -Hello -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 -1000 diff --git a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql b/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql deleted file mode 100644 index a8ef9e58d40..00000000000 --- a/tests/queries/0_stateless/02528_max_parallel_replicas_custom_key_range.sql +++ /dev/null @@ -1,37 +0,0 @@ --- Tags: replica - -DROP TABLE IF EXISTS t; - -CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; -INSERT INTO t VALUES ('Hello'); - -SET max_parallel_replicas = 3; -SET parallel_replicas_mode = 'custom_key'; -SET parallel_replicas_custom_key_filter_type = 'range'; - -SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'sipHash64(x)'; - -DROP TABLE t; - -CREATE TABLE t (x String, y UInt32) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); -INSERT INTO t SELECT toString(number), number FROM numbers(1000); - -SET max_parallel_replicas = 1; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -SET max_parallel_replicas = 2; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -SET max_parallel_replicas = 3; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + y'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x)'; -SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t) SETTINGS parallel_replicas_custom_key = 'cityHash64(x) + 1'; - -DROP TABLE t; From 7bfaf88666a28d76d47a5ba4e3bd2b2925ce90fb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 20 Jan 2023 16:35:14 +0000 Subject: [PATCH 030/470] Fix stateless tests --- .../0_stateless/00124_shard_distributed_with_many_replicas.sql | 1 + .../01034_prewhere_max_parallel_replicas_distributed.sql | 3 +++ tests/queries/0_stateless/01034_sample_final_distributed.sql | 3 +++ .../0_stateless/01099_parallel_distributed_insert_select.sql | 1 + tests/queries/0_stateless/01517_select_final_distributed.sql | 2 ++ .../0_stateless/01557_max_parallel_replicas_no_sample.sql | 1 + .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 1 + 7 files changed, 12 insertions(+) diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index ab4c433ba47..795551e5dfa 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,6 +1,7 @@ -- Tags: replica, distributed SET max_parallel_replicas = 2; +SET parallel_replicas_mode = 'sample_key'; DROP TABLE IF EXISTS report; set allow_deprecated_syntax_for_merge_tree=1; diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index b26c9af14a9..96d52d7e60e 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -9,6 +9,9 @@ CREATE TABLE test_max_parallel_replicas_lr (timestamp UInt64) ENGINE = MergeTree INSERT INTO test_max_parallel_replicas_lr select number as timestamp from system.numbers limit 100; SET max_parallel_replicas = 2; +SET parallel_replicas_mode = 'sample_key'; +SET allow_experimental_parallel_reading_from_replicas = 0; + select count() FROM remote('127.0.0.{2|3}', currentDatabase(), test_max_parallel_replicas_lr) PREWHERE timestamp > 0; drop table test_max_parallel_replicas_lr; diff --git a/tests/queries/0_stateless/01034_sample_final_distributed.sql b/tests/queries/0_stateless/01034_sample_final_distributed.sql index b784b35cbb3..1b1832f7a0d 100644 --- a/tests/queries/0_stateless/01034_sample_final_distributed.sql +++ b/tests/queries/0_stateless/01034_sample_final_distributed.sql @@ -13,7 +13,10 @@ select count() from sample_final sample 1/2; select 'count sample final'; select count() from sample_final final sample 1/2; select 'count final max_parallel_replicas'; + +set allow_experimental_parallel_reading_from_replicas = 0; set max_parallel_replicas=2; +set parallel_replicas_mode='sample_key'; select count() from remote('127.0.0.{2|3}', currentDatabase(), sample_final) final; drop table if exists sample_final; diff --git a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql index a75a5b2c33d..aa924218360 100644 --- a/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql +++ b/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql @@ -2,6 +2,7 @@ -- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971 +SET allow_experimental_parallel_reading_from_replicas = 0; -- see https://github.com/ClickHouse/ClickHouse/issues/34525 SET prefer_localhost_replica = 1; DROP TABLE IF EXISTS local_01099_a; diff --git a/tests/queries/0_stateless/01517_select_final_distributed.sql b/tests/queries/0_stateless/01517_select_final_distributed.sql index a3d1fcfc185..701828b0b38 100644 --- a/tests/queries/0_stateless/01517_select_final_distributed.sql +++ b/tests/queries/0_stateless/01517_select_final_distributed.sql @@ -1,5 +1,7 @@ -- Tags: distributed +SET allow_experimental_parallel_reading_from_replicas = 0; + DROP TABLE IF EXISTS test5346; CREATE TABLE test5346 (`Id` String, `Timestamp` DateTime, `updated` DateTime) diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql index 2b1a66147a4..c44c335700f 100644 --- a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -4,6 +4,7 @@ DROP TABLE IF EXISTS t; CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; INSERT INTO t VALUES ('Hello'); +SET parallel_replicas_mode = 'sample_key'; SET max_parallel_replicas = 3; SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 76138842e9a..e97643bd366 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel, long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 40f42aa0789b6e501e7e13ce391896317778a316 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 22 Jan 2023 14:15:21 +0000 Subject: [PATCH 031/470] Remove pipeline from test --- ...max_parallel_replicas_custom_key.reference | 534 ------------------ .../02535_max_parallel_replicas_custom_key.sh | 2 - 2 files changed, 536 deletions(-) diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference index 01f48778b34..21ed26b7579 100644 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference @@ -1,659 +1,125 @@ query='SELECT * FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='sipHash64(x)' filter_type='default' max_replicas=1 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='default' max_replicas=2 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='range' max_replicas=2 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=0 Hello -(ReadFromRemote) filter_type='default' max_replicas=1 prefer_localhost_replica=1 Hello -(Expression) -ExpressionTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 filter_type='default' max_replicas=2 prefer_localhost_replica=1 Hello -(Union) - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=1 Hello -(Union) - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=1 Hello -(Expression) -ExpressionTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 filter_type='range' max_replicas=2 prefer_localhost_replica=1 Hello -(Union) - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=1 Hello -(Union) - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x)' filter_type='default' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='default' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='range' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='default' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='default' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='range' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='y' filter_type='default' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='default' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='range' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='default' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='default' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='range' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + y' filter_type='default' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='default' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='range' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='default' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='default' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='range' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + 1' filter_type='default' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='default' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=0 1000 -(ReadFromRemote) filter_type='range' max_replicas=2 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=0 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemote) filter_type='default' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='default' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='default' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=1 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform - (ReadFromStorage) - AggregatingTransform - ExpressionTransform - SourceFromSingleChunk 0 → 1 filter_type='range' max_replicas=2 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 2 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) filter_type='range' max_replicas=3 prefer_localhost_replica=1 1000 -(Expression) -ExpressionTransform × 4 - (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (Union) - (Aggregating) - AggregatingTransform - (Expression) - ExpressionTransform - (Filter) - FilterTransform - (ReadFromMergeTree) - MergeTreeInOrder 0 → 1 - (ReadFromRemote) diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index e97643bd366..40f62d39769 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -15,10 +15,8 @@ function run_with_custom_key { , parallel_replicas_mode='custom_key'\ , parallel_replicas_custom_key='$2'\ , parallel_replicas_custom_key_filter_type='$filter_type'\ - , max_threads=4\ , prefer_localhost_replica=$prefer_localhost_replica" $CLICKHOUSE_CLIENT --query="$query" - $CLICKHOUSE_CLIENT --query="EXPLAIN PIPELINE $query" done done done From 3a0941573376d4e2001b9c5937715a2a021a483a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 22 Jan 2023 14:20:25 +0000 Subject: [PATCH 032/470] Refactor integration test --- .../test_parallel_replicas_custom_key/test.py | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 0787234a7b1..0819aceb8c7 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -3,19 +3,12 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -n1 = cluster.add_instance( - "n1", main_configs=["configs/remote_servers.xml"], with_zookeeper=True -) -n2 = cluster.add_instance( - "n2", main_configs=["configs/remote_servers.xml"], with_zookeeper=True -) -n3 = cluster.add_instance( - "n3", main_configs=["configs/remote_servers.xml"], with_zookeeper=True -) -n4 = cluster.add_instance( - "n4", main_configs=["configs/remote_servers.xml"], with_zookeeper=True -) -nodes = [n1, n2, n3, n4] +nodes = [ + cluster.add_instance( + f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True + ) + for i in range(1, 5) +] @pytest.fixture(scope="module", autouse=True) @@ -28,6 +21,7 @@ def start_cluster(): def create_tables(cluster): + n1 = nodes[0] n1.query("DROP TABLE IF EXISTS dist_table") n1.query(f"DROP TABLE IF EXISTS test_table ON CLUSTER {cluster}") @@ -49,6 +43,7 @@ def create_tables(cluster): def insert_data(cluster, row_num): create_tables(cluster) + n1 = nodes[0] n1.query(f"INSERT INTO dist_table SELECT number, number FROM numbers({row_num})") n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") @@ -65,6 +60,8 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter row_num = 1000 insert_data(cluster, row_num) + + n1 = nodes[0] assert ( int( n1.query( From c15128491bd3fc451abd07807e9ed95cee335710 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 23 Jan 2023 17:24:48 +0100 Subject: [PATCH 033/470] Update max_parallel_replicas docs --- docs/en/operations/settings/settings.md | 54 ++++++++++++++++++++++++- docs/en/sql-reference/operators/in.md | 13 ++++-- 2 files changed, 61 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e3f7bc11ddf..9fd4ccafb18 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1221,7 +1221,9 @@ Possible values: Default value: 1. :::warning -Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas). +Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) with [parallel_replicas_mode](#settings-parallel_replicas_mode) set to `sample_key` or `read_tasks`. +If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `custom_key`, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. +If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. ::: ## totals_mode {#totals-mode} @@ -1246,17 +1248,65 @@ Default value: `1`. **Additional Info** -This setting is useful for replicated tables with a sampling key. A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: +This options will produce different results depending on the value of [parallel_replicas_mode](#settings-parallel_replicas_mode). + +### `sample_key` + +If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `sample_key`, this setting is useful for replicated tables with a sampling key. +A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: - The position of the sampling key in the partitioning key does not allow efficient range scans. - Adding a sampling key to the table makes filtering by other columns less efficient. - The sampling key is an expression that is expensive to calculate. - The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. +### `custom_key` + +If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `custom_key`, this setting is useful for any replicated table. +A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) +and [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type). + +Use `default` for [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type) unless the data is split across the entire integer space (e.g. column contains hash values), +then `range` should be used. +Simple expressions using primary keys are preferred. + +If the `custom_key` mode is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. +Otherwise, it will behave same as `sample_key` mode, it will use multiple replicas of each shard. + :::warning This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: +## parallel_replicas_mode {#settings-parallel_replicas_mode} + +Mode of splitting work between replicas. + +Possible values: + +- `sample_key` — Use `SAMPLE` key defined in the `SAMPLE BY` clause to split the work between replicas. +- `custom_key` — Define an arbitrary integer expression to use for splitting work between replicas. +- `read_tasks` — Split tasks for reading physical parts between replicas. + +Default value: `sample_key`. + +## parallel_replicas_custom_key {#settings-parallel_replicas_custom_key} + +Arbitrary integer expression that will be used to split work between replicas. +Used only if `parallel_replicas_mode` is set to `custom_key`. + +Default value: `''`. + +## parallel_replicas_custom_key_filter_type {#settings-parallel_replicas_custom_key_filter_type} + +How to use `parallel_replicas_custom_key` expression for splitting work between replicas. + +Possible values: + +- `default` — Use the default implementation using modulo operation on the `parallel_replicas_custom_key`. +- `range` — Split the entire value space of the expression in the ranges. This type of filtering is useful if values of `parallel_replicas_custom_key` are uniformly spread across the entire integer space, e.g. hash values. + +Default value: `default`. + ## compile_expressions {#compile-expressions} Enables or disables compilation of frequently used simple functions and operators to native code with LLVM at runtime. diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 58119cfc4f5..e1e4118524a 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -233,11 +233,12 @@ If `some_predicate` is not selective enough, it will return large amount of data ### Distributed Subqueries and max_parallel_replicas -When max_parallel_replicas is greater than 1, distributed queries are further transformed. For example, the following: +When [max_parallel_replicas](#settings-max_parallel_replicas) is greater than 1, distributed queries are further transformed. +For example, if [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `sample_key`, the following: ```sql SELECT CounterID, count() FROM distributed_table_1 WHERE UserID IN (SELECT UserID FROM local_table_2 WHERE CounterID < 100) -SETTINGS max_parallel_replicas=3 +SETTINGS max_parallel_replicas=3, parallel_replicas_mode='sample_key' ``` is transformed on each server into @@ -247,8 +248,12 @@ SELECT CounterID, count() FROM local_table_1 WHERE UserID IN (SELECT UserID FROM SETTINGS parallel_replicas_count=3, parallel_replicas_offset=M ``` -where M is between 1 and 3 depending on which replica the local query is executing on. These settings affect every MergeTree-family table in the query and have the same effect as applying `SAMPLE 1/3 OFFSET (M-1)/3` on each table. +where M is between 1 and 3 depending on which replica the local query is executing on. -Therefore adding the max_parallel_replicas setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if local_table_2 does not have a sampling key, incorrect results will be produced. The same rule applies to JOIN. +These settings affect every MergeTree-family table in the query and have the same effect as applying `SAMPLE 1/3 OFFSET (M-1)/3` on each table. + +Therefore adding the [max_parallel_replicas](#settings-max_parallel_replicas) setting will only produce correct results if both tables have the same replication scheme and are sampled by UserID or a subkey of it. In particular, if local_table_2 does not have a sampling key, incorrect results will be produced. The same rule applies to JOIN. One workaround if local_table_2 does not meet the requirements, is to use `GLOBAL IN` or `GLOBAL JOIN`. + +If a table doesn't have a sampling key, more flexible options for [parallel_replicas_mode](#settings-parallel_replicas_mode) can be used that can produce different and more optimal behaviour. From 37b62b3a589d0dd57dcfc72a5043a2608b0c2f20 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 24 Jan 2023 10:46:47 +0000 Subject: [PATCH 034/470] Use Map for custom_key --- docs/en/operations/settings/settings.md | 8 +- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 24 +++--- .../getCustomKeyFilterForParallelReplicas.cpp | 44 +++++++++-- .../getCustomKeyFilterForParallelReplicas.h | 5 +- src/Storages/StorageDistributed.cpp | 78 ++++++++++++++----- .../test_parallel_replicas_custom_key/test.py | 46 ++++++++++- .../02535_max_parallel_replicas_custom_key.sh | 2 +- 8 files changed, 166 insertions(+), 43 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9fd4ccafb18..68690e8a173 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1291,10 +1291,14 @@ Default value: `sample_key`. ## parallel_replicas_custom_key {#settings-parallel_replicas_custom_key} -Arbitrary integer expression that will be used to split work between replicas. +Map of arbitrary integer expression that can be used to split work between replicas for a specific table. +If it's used with `cluster` function, the key can be name of the local table defined inside the `cluster` function. +If it's used with `Distributed` engine, the key can be name of the distributed table, alias or the local table for which the `Distributed` engine is created. +The value can be any integer expression. + Used only if `parallel_replicas_mode` is set to `custom_key`. -Default value: `''`. +Default value: `{}`. ## parallel_replicas_custom_key_filter_type {#settings-parallel_replicas_custom_key_filter_type} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 89686c92960..2c8b1c90faa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -154,7 +154,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ - M(String, parallel_replicas_custom_key, "", "Custom key for parallel replicas using modulo operation on the key for assigning work to replicas.", 0) \ + M(Map, parallel_replicas_custom_key, "", "Custom key for parallel replicas using modulo operation on the key for assigning work to replicas.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ M(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::SAMPLE_KEY, "How to process query using multiple replicas.", 0) \ \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 817d40c086a..d174912f66d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -513,18 +513,20 @@ InterpreterSelectQuery::InterpreterSelectQuery( ASTPtr parallel_replicas_custom_filter_ast = nullptr; if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) { - LOG_INFO(log, "Processing query on a replica using custom_key"); - if (!storage) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, joined_tables.tablesWithColumns().front().table, *context)) + { + LOG_INFO(log, "Processing query on a replica using custom_key"); + if (!storage) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); - auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key.value, *context); - parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( - settings.parallel_replicas_count, - settings.parallel_replica_offset, - std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, - *storage, - context); + parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + settings.parallel_replicas_custom_key_filter_type, + *storage, + context); + } } auto analyze = [&](bool try_move_to_prewhere) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index fbc1d214164..cfc7cfd6194 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -116,15 +116,43 @@ ASTPtr getCustomKeyFilterForParallelReplica( return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); } -ASTPtr parseParallelReplicaCustomKey(std::string_view custom_key, const Context & context) +ASTPtr parseCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context) { - if (custom_key.empty()) - throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Parallel replicas mode set to 'custom_key' but 'parallel_replicas_custom_key' has no value"); + for (size_t i = 0; i < custom_keys.size(); ++i) + { + const auto & tuple = custom_keys[i].safeGet(); + auto & table = tuple.at(0).safeGet(); + auto & filter = tuple.at(1).safeGet(); - ParserExpression parser; - const auto & settings = context.getSettingsRef(); - return parseQuery( - parser, custom_key.data(), custom_key.data() + custom_key.size(), - "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); + if (table == target.alias || + (table == target.table && context.getCurrentDatabase() == target.database) || + (table == target.database + '.' + target.table)) + { + /// Try to parse expression + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, filter.data(), filter.data() + filter.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); + } + } + + return nullptr; } + +bool containsCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context) +{ + for (size_t i = 0; i < custom_keys.size(); ++i) + { + const auto & tuple = custom_keys[i].safeGet(); + auto & table = tuple.at(0).safeGet(); + + if (table == target.alias || + (table == target.table && context.getCurrentDatabase() == target.database) || + (table == target.database + '.' + target.table)) + return true; + } + return false; +} + } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index 251e72e0a97..fc515132487 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -18,6 +19,8 @@ ASTPtr getCustomKeyFilterForParallelReplica( const IStorage & storage, const ContextPtr & context); -ASTPtr parseParallelReplicaCustomKey(std::string_view custom_key, const Context & context); +ASTPtr parseCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context); + +bool containsCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bf9ae2fdd20..7a3bd89e5ff 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -27,7 +27,6 @@ #include #include #include -#include "Core/SettingsEnums.h" #include #include @@ -47,6 +46,7 @@ #include #include #include +#include #include #include #include @@ -80,6 +80,7 @@ #include #include +#include #include #include @@ -91,8 +92,6 @@ #include #include -#include - namespace fs = std::filesystem; @@ -307,13 +306,13 @@ size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & clus return (num_remote_shards + num_local_shards) * settings.max_parallel_replicas; } -bool useVirtualShards(const Settings & settings, const Cluster & cluster) +bool canUseCustomKey(const Settings & settings, const Cluster & cluster) { return settings.max_parallel_replicas > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; } -} +} /// For destruction of std::unique_ptr of type that is incomplete in class definition. StorageDistributed::~StorageDistributed() = default; @@ -453,7 +452,33 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( size_t nodes = getClusterQueriedNodes(settings, cluster); - if (useVirtualShards(settings, *cluster)) + const auto use_virtual_shards = [&] + { + if (!canUseCustomKey(settings, *cluster)) + return false; + + auto distributed_table = DatabaseAndTableWithAlias( + *getTableExpression(query_info.query->as(), 0), local_context->getCurrentDatabase()); + + if (containsCustomKeyForTable(settings.parallel_replicas_custom_key, distributed_table, *local_context)) + { + LOG_INFO(log, "Found custom_key for {}", distributed_table.getQualifiedNamePrefix(false)); + return true; + } + + DatabaseAndTableWithAlias remote_table_info; + remote_table_info.database = remote_database; + remote_table_info.table = remote_table; + if (containsCustomKeyForTable(settings.parallel_replicas_custom_key, remote_table_info, *local_context)) + { + LOG_INFO(log, "Found custom_key for {}", remote_table_info.getQualifiedNamePrefix(false)); + return true; + } + + return false; + }; + + if (use_virtual_shards()) { LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards"); @@ -767,23 +792,40 @@ void StorageDistributed::read( auto settings = local_context->getSettingsRef(); ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; - if (useVirtualShards(settings, *getCluster())) + if (canUseCustomKey(settings, *getCluster())) { - if (query_info.getCluster()->getShardCount() == 1) + const auto get_custom_key_ast = [&]() -> ASTPtr { - // we are reading from single shard with multiple replicas but didn't transform replicas - // into virtual shards with custom_key set - throw Exception(ErrorCodes::LOGICAL_ERROR, "Replicas weren't transformed into virtual shards"); - } + auto distributed_table = DatabaseAndTableWithAlias( + *getTableExpression(query_info.query->as(), 0), local_context->getCurrentDatabase()); + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, distributed_table, *local_context)) + return custom_key_ast; - auto custom_key_ast = parseParallelReplicaCustomKey(settings.parallel_replicas_custom_key.value, *local_context); + DatabaseAndTableWithAlias remote_table_info; + remote_table_info.database = remote_database; + remote_table_info.table = remote_table; + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, remote_table_info, *local_context)) + return custom_key_ast; - additional_shard_filter_generator = - [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) -> ASTPtr - { - return getCustomKeyFilterForParallelReplica( - shard_count, shard_num - 1, custom_key_ast, settings.parallel_replicas_custom_key_filter_type, *this, local_context); + return nullptr; }; + + if (auto custom_key_ast = get_custom_key_ast()) + { + if (query_info.getCluster()->getShardCount() == 1) + { + // we are reading from single shard with multiple replicas but didn't transform replicas + // into virtual shards with custom_key set + throw Exception(ErrorCodes::LOGICAL_ERROR, "Replicas weren't transformed into virtual shards"); + } + + additional_shard_filter_generator = + [&, custom_key_ast = std::move(custom_key_ast), shard_count = query_info.cluster->getShardCount()](uint64_t shard_num) -> ASTPtr + { + return getCustomKeyFilterForParallelReplica( + shard_count, shard_num - 1, custom_key_ast, settings.parallel_replicas_custom_key_filter_type, *this, local_context); + }; + } } bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 0819aceb8c7..c11be610611 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -70,7 +70,7 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter "prefer_localhost_replica": 0, "max_parallel_replicas": 3, "parallel_replicas_mode": "custom_key", - "parallel_replicas_custom_key": custom_key, + "parallel_replicas_custom_key": f"{{'test_table': '{custom_key}'}}", "parallel_replicas_custom_key_filter_type": filter_type, }, ) @@ -89,3 +89,47 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter assert n1.contains_in_log( "Single shard cluster used with custom_key, transforming replicas into virtual shards" ) + + +def test_custom_key_different_table_names(start_cluster): + def run(table_source, table_name_for_custom_key): + for node in nodes: + node.rotate_logs() + + row_num = 1000 + insert_data("test_single_shard_multiple_replicas", row_num) + + n1 = nodes[0] + assert ( + int( + n1.query( + f"SELECT count() FROM {table_source}", + settings={ + "prefer_localhost_replica": 0, + "max_parallel_replicas": 3, + "parallel_replicas_mode": "custom_key", + "parallel_replicas_custom_key": f"{{'{table_name_for_custom_key}': 'sipHash64(value)'}}", + }, + ) + ) + == row_num + ) + + # we first transform all replicas into shards and then append for each shard filter + assert n1.contains_in_log( + "Single shard cluster used with custom_key, transforming replicas into virtual shards" + ) + + run("dist_table", "dist_table") + run("dist_table as d", "d") + run("dist_table as d", "dist_table") + run("dist_table as d", "test_table") + run( + "cluster('test_single_shard_multiple_replicas', default.test_table)", + "test_table", + ) + run("cluster('test_single_shard_multiple_replicas', default.test_table) as d", "d") + run( + "cluster('test_single_shard_multiple_replicas', default.test_table) as d", + "test_table", + ) diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 40f62d39769..b6e5c853dc2 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -13,7 +13,7 @@ function run_with_custom_key { echo "filter_type='$filter_type' max_replicas=$max_replicas prefer_localhost_replica=$prefer_localhost_replica" query="$1 SETTINGS max_parallel_replicas=$max_replicas\ , parallel_replicas_mode='custom_key'\ - , parallel_replicas_custom_key='$2'\ + , parallel_replicas_custom_key={'02535_custom_key': '$2'}\ , parallel_replicas_custom_key_filter_type='$filter_type'\ , prefer_localhost_replica=$prefer_localhost_replica" $CLICKHOUSE_CLIENT --query="$query" From 7df545fbe3af5c6ea688c8ee0a06d1004c6fa349 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 24 Jan 2023 11:21:09 +0000 Subject: [PATCH 035/470] Remove unused define --- src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index cfc7cfd6194..229668ceff4 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; - extern const int BAD_ARGUMENTS; } ASTPtr getCustomKeyFilterForParallelReplica( From ffa3d1d603d0e77b7cf0fa71a70ac3f53e7671ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 24 Jan 2023 13:58:42 +0000 Subject: [PATCH 036/470] Run query on 1 replica if no custom_key --- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 996efe2b4cf..9cecd53f6c8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -511,7 +511,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); ASTPtr parallel_replicas_custom_filter_ast = nullptr; - if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY) + if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) { if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, joined_tables.tablesWithColumns().front().table, *context)) { @@ -527,6 +527,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( *storage, context); } + else if (settings.parallel_replica_offset > 0) + { + LOG_DEBUG( + log, + "Will use no data on this replica because parallel replicas processing with custom_key has been requested" + " (setting 'max_parallel_replicas') but the table does not have custom_key defined for it (settings `parallel_replicas_custom_key`)"); + parallel_replicas_custom_filter_ast = std::make_shared(false); + } } auto analyze = [&](bool try_move_to_prewhere) From 6da9dc2407cd11aa3347f64d3869a0ccfaac48d0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 24 Jan 2023 14:16:26 +0000 Subject: [PATCH 037/470] Correctly handle case with missing custom_key --- .../test_parallel_replicas_custom_key/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index c11be610611..9222f417a94 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -68,7 +68,7 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter "SELECT count() FROM dist_table", settings={ "prefer_localhost_replica": 0, - "max_parallel_replicas": 3, + "max_parallel_replicas": 4, "parallel_replicas_mode": "custom_key", "parallel_replicas_custom_key": f"{{'test_table': '{custom_key}'}}", "parallel_replicas_custom_key_filter_type": filter_type, @@ -92,7 +92,7 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter def test_custom_key_different_table_names(start_cluster): - def run(table_source, table_name_for_custom_key): + def run(table_source, table_name_for_custom_key, should_use_virtual_shard=True): for node in nodes: node.rotate_logs() @@ -106,7 +106,7 @@ def test_custom_key_different_table_names(start_cluster): f"SELECT count() FROM {table_source}", settings={ "prefer_localhost_replica": 0, - "max_parallel_replicas": 3, + "max_parallel_replicas": 4, "parallel_replicas_mode": "custom_key", "parallel_replicas_custom_key": f"{{'{table_name_for_custom_key}': 'sipHash64(value)'}}", }, @@ -116,7 +116,7 @@ def test_custom_key_different_table_names(start_cluster): ) # we first transform all replicas into shards and then append for each shard filter - assert n1.contains_in_log( + assert not should_use_virtual_shard or n1.contains_in_log( "Single shard cluster used with custom_key, transforming replicas into virtual shards" ) @@ -133,3 +133,4 @@ def test_custom_key_different_table_names(start_cluster): "cluster('test_single_shard_multiple_replicas', default.test_table) as d", "test_table", ) + run("dist_table as d", "dist", should_use_virtual_shard=False) From eee7df814926fc323be455c348d0fd8d37e76efc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 1 Feb 2023 13:33:32 +0000 Subject: [PATCH 038/470] WIP on group_by_use_nulls --- src/Analyzer/ColumnNode.h | 6 ++ src/Analyzer/FunctionNode.cpp | 3 +- src/Analyzer/FunctionNode.h | 13 +++- src/Analyzer/IQueryTreeNode.h | 5 ++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 81 +++++++++++++++++++---- src/Interpreters/ActionsDAG.cpp | 32 ++++++++- src/Interpreters/ActionsDAG.h | 8 +++ src/Planner/PlannerActionsVisitor.cpp | 7 +- src/Planner/PlannerExpressionAnalysis.cpp | 5 +- 9 files changed, 139 insertions(+), 21 deletions(-) diff --git a/src/Analyzer/ColumnNode.h b/src/Analyzer/ColumnNode.h index e378bc5f3d0..79c0e23c86f 100644 --- a/src/Analyzer/ColumnNode.h +++ b/src/Analyzer/ColumnNode.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB { @@ -117,6 +118,11 @@ public: return column.type; } + void convertToNullable() override + { + column.type = makeNullableSafe(column.type); + } + void dumpTreeImpl(WriteBuffer & buffer, FormatState & state, size_t indent) const override; protected: diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 7961bfbae31..718dcf4bb58 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -99,7 +99,7 @@ void FunctionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state buffer << ", function_type: " << function_type; if (function) - buffer << ", result_type: " + function->getResultType()->getName(); + buffer << ", result_type: " + getResultType()->getName(); const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) @@ -177,6 +177,7 @@ QueryTreeNodePtr FunctionNode::cloneImpl() const */ result_function->function = function; result_function->kind = kind; + result_function->wrap_with_nullable = wrap_with_nullable; return result_function; } diff --git a/src/Analyzer/FunctionNode.h b/src/Analyzer/FunctionNode.h index 41751ec3f09..0311783b433 100644 --- a/src/Analyzer/FunctionNode.h +++ b/src/Analyzer/FunctionNode.h @@ -8,6 +8,7 @@ #include #include #include +#include #include namespace DB @@ -170,7 +171,16 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Function node with name '{}' is not resolved", function_name); - return function->getResultType(); + auto type = function->getResultType(); + if (wrap_with_nullable) + return makeNullableSafe(type); + return type; + } + + void convertToNullable() override + { + chassert(kind == FunctionKind::ORDINARY); + wrap_with_nullable = true; } void dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const override; @@ -188,6 +198,7 @@ private: String function_name; FunctionKind kind = FunctionKind::UNKNOWN; IResolvedFunctionPtr function; + bool wrap_with_nullable = false; static constexpr size_t parameters_child_index = 0; static constexpr size_t arguments_child_index = 1; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 8aa834e60b7..3c8e73b9e31 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -90,6 +90,11 @@ public: throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method getResultType is not supported for {} query node", getNodeTypeName()); } + virtual void convertToNullable() + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method convertToNullable is not supported for {} query node", getNodeTypeName()); + } + /** Is tree equal to other tree with node root. * * Aliases of query tree nodes are compared during isEqual call. diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7d1f636754c..4c1d6d309d6 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,5 +1,6 @@ #include +#include "Common/logger_useful.h" #include #include @@ -446,6 +447,9 @@ public: alias_name_to_expressions[node_alias].push_back(node); } + if (const auto * function = node->as()) + LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Pushed function {} on stack", function->getFunctionName()); + expressions.emplace_back(node); } @@ -464,6 +468,9 @@ public: alias_name_to_expressions.erase(it); } + if (const auto * function = top_expression->as()) + LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Poped function {} on stack", function->getFunctionName()); + expressions.pop_back(); } @@ -482,6 +489,22 @@ public: return alias_name_to_expressions.contains(alias); } + bool hasAggregateFunction() const + { + const auto & factory = AggregateFunctionFactory::instance(); + for (const auto & node : expressions) + { + const auto * function = node->as(); + if (!function) + continue; + + if (factory.isAggregateFunctionName(function->getFunctionName())) + return true; + LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Function {} is being resolved, but is not aggregate", function->getFunctionName()); + } + return false; + } + QueryTreeNodePtr getExpressionWithAlias(const std::string & alias) const { auto expression_it = alias_name_to_expressions.find(alias); @@ -708,6 +731,8 @@ struct IdentifierResolveScope /// Table expression node to data std::unordered_map table_expression_node_to_data; + QueryTreeNodePtrWithHashSet nullable_group_by_keys; + /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; @@ -1221,7 +1246,6 @@ private: /// Results of scalar sub queries std::unordered_map> scalars; - }; /// Utility functions implementation @@ -3092,6 +3116,11 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook resolve_result.resolve_place = IdentifierResolvePlace::DATABASE_CATALOG; } + if (resolve_result.resolved_identifier + && scope.nullable_group_by_keys.contains(resolve_result.resolved_identifier) + && !scope.expressions_in_resolve_process_stack.hasAggregateFunction()) + resolve_result.resolved_identifier->convertToNullable(); + it->second = resolve_result; /** If identifier was not resolved, or during expression resolution identifier was explicitly added into non cached set, @@ -3099,7 +3128,8 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook */ if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || - !scope.use_identifier_lookup_to_result_cache) + !scope.use_identifier_lookup_to_result_cache || + scope.context->getSettingsRef().group_by_use_nulls) scope.identifier_lookup_to_result.erase(it); return resolve_result; @@ -4627,6 +4657,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } function_node.resolveAsFunction(std::move(function_base)); + if (settings.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) + function_node.convertToNullable(); } catch (Exception & e) { @@ -5873,9 +5905,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier auto & query_node_typed = query_node->as(); const auto & settings = scope.context->getSettingsRef(); - if (settings.group_by_use_nulls) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "GROUP BY use nulls is not supported"); - bool is_rollup_or_cube = query_node_typed.isGroupByWithRollup() || query_node_typed.isGroupByWithCube(); if (query_node_typed.isGroupByWithGroupingSets() && query_node_typed.isGroupByWithTotals()) @@ -6011,16 +6040,11 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveQueryJoinTreeNode(query_node_typed.getJoinTree(), scope, visitor); } - scope.use_identifier_lookup_to_result_cache = true; + if (!settings.group_by_use_nulls) + scope.use_identifier_lookup_to_result_cache = true; /// Resolve query node sections. - auto projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); - if (query_node_typed.getProjection().getNodes().empty()) - throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, - "Empty list of columns in projection. In scope {}", - scope.scope_node->formatASTForErrorMessage()); - if (query_node_typed.hasWith()) resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -6041,6 +6065,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } + + for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes()) + { + for (const auto & group_by_elem : grouping_set->as()->getNodes()) + scope.nullable_group_by_keys.insert(group_by_elem->clone()); + } + // for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes()) + // { + // for (const auto & group_by_elem : grouping_set->as()->getNodes()) + // group_by_elem->convertToNullable(); + // } } else { @@ -6048,6 +6083,14 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier replaceNodesWithPositionalArguments(query_node_typed.getGroupByNode(), query_node_typed.getProjection().getNodes(), scope); resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + if (settings.group_by_use_nulls) + { + for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) + scope.nullable_group_by_keys.insert(group_by_elem->clone()); + // for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) + // group_by_elem->convertToNullable(); + } } } @@ -6100,6 +6143,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validateLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); } + auto projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); + if (query_node_typed.getProjection().getNodes().empty()) + throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, + "Empty list of columns in projection. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + /** Resolve nodes with duplicate aliases. * Table expressions cannot have duplicate aliases. * @@ -6234,7 +6283,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (grouping_set_key->as()) continue; - group_by_keys_nodes.push_back(grouping_set_key); + group_by_keys_nodes.push_back(grouping_set_key->clone()); + if (settings.group_by_use_nulls) + group_by_keys_nodes.back()->convertToNullable(); } } else @@ -6242,7 +6293,9 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (node->as()) continue; - group_by_keys_nodes.push_back(node); + group_by_keys_nodes.push_back(node->clone()); + if (settings.group_by_use_nulls) + group_by_keys_nodes.back()->convertToNullable(); } } diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 5f1398fed39..4e7dfe2bef5 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -199,6 +200,23 @@ const ActionsDAG::Node & ActionsDAG::addFunction( std::move(children), std::move(arguments), std::move(result_name), + function_base->getResultType(), + all_const); +} + +const ActionsDAG::Node & ActionsDAG::addFunction( + const FunctionNode & function, + NodeRawConstPtrs children, + std::string result_name) +{ + auto [arguments, all_const] = getFunctionArguments(children); + + return addFunctionImpl( + function.getFunction(), + std::move(children), + std::move(arguments), + std::move(result_name), + function.getResultType(), all_const); } @@ -214,6 +232,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( std::move(children), std::move(arguments), std::move(result_name), + function_base->getResultType(), all_const); } @@ -238,6 +257,7 @@ const ActionsDAG::Node & ActionsDAG::addFunctionImpl( NodeRawConstPtrs children, ColumnsWithTypeAndName arguments, std::string result_name, + DataTypePtr result_type, bool all_const) { size_t num_arguments = children.size(); @@ -247,7 +267,7 @@ const ActionsDAG::Node & ActionsDAG::addFunctionImpl( node.children = std::move(children); node.function_base = function_base; - node.result_type = node.function_base->getResultType(); + node.result_type = result_type; node.function = node.function_base->prepare(arguments); node.is_deterministic = node.function_base->isDeterministic(); @@ -2236,7 +2256,15 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG( for (const auto & child : node->children) function_children.push_back(node_to_result_node.find(child)->second); - result_node = &result_dag->addFunction(node->function_base, std::move(function_children), {}); + auto [arguments, all_const] = getFunctionArguments(function_children); + + result_node = &result_dag->addFunctionImpl( + node->function_base, + std::move(function_children), + std::move(arguments), + {}, + node->result_type, + all_const); break; } } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 40bc76fe057..0b240feea0e 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -23,6 +23,8 @@ using FunctionBasePtr = std::shared_ptr; class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; +class FunctionNode; + class IDataType; using DataTypePtr = std::shared_ptr; @@ -139,10 +141,15 @@ public: const FunctionOverloadResolverPtr & function, NodeRawConstPtrs children, std::string result_name); + const Node & addFunction( + const FunctionNode & function, + NodeRawConstPtrs children, + std::string result_name); const Node & addFunction( const FunctionBasePtr & function_base, NodeRawConstPtrs children, std::string result_name); + const Node & addCast(const Node & node_to_cast, const DataTypePtr & cast_type); /// Find first column by name in output nodes. This search is linear. @@ -357,6 +364,7 @@ private: NodeRawConstPtrs children, ColumnsWithTypeAndName arguments, std::string result_name, + DataTypePtr result_type, bool all_const); #if USE_EMBEDDED_COMPILER diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 95edd93dd9f..13c88d47ca3 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -29,6 +29,7 @@ #include #include #include +#include namespace DB { @@ -82,6 +83,7 @@ public: node_name, actions_dag->dumpNames()); + LOG_DEBUG(&Poco::Logger::get("ActionsScopeNode"), "Node: {} {}", it->second->result_name, it->second->result_type->getName()); return it->second; } @@ -122,7 +124,7 @@ public: } template - const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, FunctionOrOverloadResolver function) + const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, const FunctionOrOverloadResolver & function) { auto it = node_name_to_node.find(node_name); if (it != node_name_to_node.end()) @@ -225,6 +227,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { auto column_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); + LOG_DEBUG(&Poco::Logger::get("PlannerActionsVisitorImpl"), "Processing column with name: {}", column_node_name); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); @@ -445,7 +448,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } else { - actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction()); + actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node); } size_t actions_stack_size = actions_stack.size(); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index b1017c99c3e..3df6ef640c7 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -299,6 +299,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, { const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + LOG_DEBUG(&Poco::Logger::get("PlannerExpressionAnalysis"), "Projection node: {}", query_node.getProjectionNode()->dumpTree()); auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); auto projection_columns = query_node.getProjectionColumns(); @@ -320,6 +321,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, for (size_t i = 0; i < projection_outputs_size; ++i) { auto & projection_column = projection_columns[i]; + LOG_DEBUG(&Poco::Logger::get("PlannerExpressionAnalysis"), "Projection column {}: {} {}", i, projection_column.name, projection_column.type->getName()); const auto * projection_node = projection_actions_outputs[i]; const auto & projection_node_name = projection_node->result_name; @@ -436,7 +438,6 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo } auto window_analysis_result_optional = analyzeWindow(query_tree, join_tree_input_columns, planner_context, actions_chain); - auto projection_analysis_result = analyzeProjection(query_node, join_tree_input_columns, planner_context, actions_chain); std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) @@ -447,6 +448,8 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (query_node.hasLimitBy()) limit_by_analysis_result_optional = analyzeLimitBy(query_node, join_tree_input_columns, planner_context, actions_chain); + auto projection_analysis_result = analyzeProjection(query_node, join_tree_input_columns, planner_context, actions_chain); + const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; auto project_names_actions = std::make_shared(project_names_input); From f505b798da9282f7b1e2dfa8481bb299a5f365af Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 1 Feb 2023 17:18:15 +0000 Subject: [PATCH 039/470] Fix ActionDAG construction with group_by_use_nulls --- src/Planner/PlannerExpressionAnalysis.cpp | 80 +++++++++++++---------- 1 file changed, 47 insertions(+), 33 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 3df6ef640c7..c7d38839c85 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -16,6 +16,8 @@ #include #include #include +#include "Common/tests/gtest_global_context.h" +#include "DataTypes/DataTypeNullable.h" namespace DB { @@ -33,12 +35,11 @@ namespace * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. */ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, - const ColumnsWithTypeAndName & join_tree_input_columns, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & filter_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & filter_input = current_output_columns; FilterAnalysisResult result; @@ -52,8 +53,8 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ -std::optional analyzeAggregation(const QueryTreeNodePtr & query_tree, - const ColumnsWithTypeAndName & join_tree_input_columns, +std::pair, std::optional> analyzeAggregation(const QueryTreeNodePtr & query_tree, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { @@ -62,15 +63,12 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto aggregate_function_nodes = collectAggregateFunctionNodes(query_tree); auto aggregates_descriptions = extractAggregateDescriptions(aggregate_function_nodes, *planner_context); - ColumnsWithTypeAndName aggregates_columns; - aggregates_columns.reserve(aggregates_descriptions.size()); - for (auto & aggregate_description : aggregates_descriptions) - aggregates_columns.emplace_back(nullptr, aggregate_description.function->getResultType(), aggregate_description.column_name); + ColumnsWithTypeAndName aggregated_columns; + aggregated_columns.reserve(aggregates_descriptions.size()); Names aggregation_keys; - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & group_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & group_by_input = current_output_columns; ActionsDAGPtr before_aggregation_actions = std::make_shared(group_by_input); before_aggregation_actions->getOutputs().clear(); @@ -83,6 +81,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP PlannerActionsVisitor actions_visitor(planner_context); /// Add expressions from GROUP BY + bool group_by_use_nulls = planner_context->getQueryContext()->getSettingsRef().group_by_use_nulls; if (query_node.hasGroupBy()) { @@ -107,6 +106,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; + auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type; + aggregated_columns.emplace_back(nullptr, expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); before_aggregation_actions->getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); @@ -150,6 +151,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; + auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type; + aggregated_columns.emplace_back(nullptr, expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); before_aggregation_actions->getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); @@ -157,6 +160,9 @@ std::optional analyzeAggregation(const QueryTreeNodeP } } + for (auto & aggregate_description : aggregates_descriptions) + aggregated_columns.emplace_back(nullptr, aggregate_description.function->getResultType(), aggregate_description.column_name); + /// Add expressions from aggregate functions arguments for (auto & aggregate_function_node : aggregate_function_nodes) @@ -183,10 +189,10 @@ std::optional analyzeAggregation(const QueryTreeNodeP * With set number, which is used as an additional key at the stage of merging aggregating data. */ if (query_node.isGroupByWithRollup() || query_node.isGroupByWithCube() || query_node.isGroupByWithGroupingSets()) - aggregates_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); + aggregated_columns.emplace_back(nullptr, std::make_shared(), "__grouping_set"); /// Only aggregation keys and aggregates are available for next steps after GROUP BY step - auto aggregate_step = std::make_unique(before_aggregation_actions, ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregates_columns); + auto aggregate_step = std::make_unique(before_aggregation_actions, ActionsChainStep::AvailableOutputColumnsStrategy::OUTPUT_NODES, aggregated_columns); actions_chain.addStep(std::move(aggregate_step)); AggregationAnalysisResult aggregation_analysis_result; @@ -196,14 +202,14 @@ std::optional analyzeAggregation(const QueryTreeNodeP aggregation_analysis_result.grouping_sets_parameters_list = std::move(grouping_sets_parameters_list); aggregation_analysis_result.group_by_with_constant_keys = group_by_with_constant_keys; - return aggregation_analysis_result; + return { aggregation_analysis_result, aggregated_columns }; } /** Construct window analysis result if query tree has window functions. * Actions before window functions are added into actions chain, if result is not null optional. */ std::optional analyzeWindow(const QueryTreeNodePtr & query_tree, - const ColumnsWithTypeAndName & join_tree_input_columns, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { @@ -213,8 +219,7 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query auto window_descriptions = extractWindowDescriptions(window_function_nodes, *planner_context); - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & window_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & window_input = current_output_columns; PlannerActionsVisitor actions_visitor(planner_context); @@ -293,12 +298,11 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query * It is client responsibility to update projection analysis result with project names actions after chain is finalized. */ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, - const ColumnsWithTypeAndName & join_tree_input_columns, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & projection_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & projection_input = current_output_columns; LOG_DEBUG(&Poco::Logger::get("PlannerExpressionAnalysis"), "Projection node: {}", query_node.getProjectionNode()->dumpTree()); auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); @@ -344,12 +348,11 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, * Actions before sort are added into actions chain. */ SortAnalysisResult analyzeSort(const QueryNode & query_node, - const ColumnsWithTypeAndName & join_tree_input_columns, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto *chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & order_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & order_by_input = current_output_columns; ActionsDAGPtr before_sort_actions = std::make_shared(order_by_input); auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); @@ -388,12 +391,11 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, * Actions before limit by are added into actions chain. */ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, - const ColumnsWithTypeAndName & join_tree_input_columns, + const ColumnsWithTypeAndName & current_output_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & limit_by_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & limit_by_input = current_output_columns; auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), limit_by_input, planner_context); Names limit_by_column_names; @@ -420,35 +422,47 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo std::optional where_analysis_result_optional; std::optional where_action_step_index_optional; + ColumnsWithTypeAndName current_output_columns = join_tree_input_columns; + if (query_node.hasWhere()) { - where_analysis_result_optional = analyzeFilter(query_node.getWhere(), join_tree_input_columns, planner_context, actions_chain); + where_analysis_result_optional = analyzeFilter(query_node.getWhere(), current_output_columns, planner_context, actions_chain); where_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } - auto aggregation_analysis_result_optional = analyzeAggregation(query_tree, join_tree_input_columns, planner_context, actions_chain); + auto [aggregation_analysis_result_optional, aggregated_columns_optional] = analyzeAggregation(query_tree, current_output_columns, planner_context, actions_chain); + if (aggregated_columns_optional) + current_output_columns = std::move(*aggregated_columns_optional); std::optional having_analysis_result_optional; std::optional having_action_step_index_optional; if (query_node.hasHaving()) { - having_analysis_result_optional = analyzeFilter(query_node.getHaving(), join_tree_input_columns, planner_context, actions_chain); + having_analysis_result_optional = analyzeFilter(query_node.getHaving(), current_output_columns, planner_context, actions_chain); having_action_step_index_optional = actions_chain.getLastStepIndex(); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } - auto window_analysis_result_optional = analyzeWindow(query_tree, join_tree_input_columns, planner_context, actions_chain); + auto window_analysis_result_optional = analyzeWindow(query_tree, current_output_columns, planner_context, actions_chain); std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) - sort_analysis_result_optional = analyzeSort(query_node, join_tree_input_columns, planner_context, actions_chain); + { + sort_analysis_result_optional = analyzeSort(query_node, current_output_columns, planner_context, actions_chain); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } std::optional limit_by_analysis_result_optional; if (query_node.hasLimitBy()) - limit_by_analysis_result_optional = analyzeLimitBy(query_node, join_tree_input_columns, planner_context, actions_chain); + { + limit_by_analysis_result_optional = analyzeLimitBy(query_node, current_output_columns, planner_context, actions_chain); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + } - auto projection_analysis_result = analyzeProjection(query_node, join_tree_input_columns, planner_context, actions_chain); + auto projection_analysis_result = analyzeProjection(query_node, current_output_columns, planner_context, actions_chain); const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; From 7e40ab7658e1ad02acbd8ddf7067f4091a07338e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 1 Feb 2023 18:04:19 +0000 Subject: [PATCH 040/470] Update output columns after WindowStep --- src/Planner/PlannerExpressionAnalysis.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index c7d38839c85..79feb501f37 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -446,6 +446,8 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo } auto window_analysis_result_optional = analyzeWindow(query_tree, current_output_columns, planner_context, actions_chain); + if (window_analysis_result_optional) + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) From d589864828ed31dcba034947375acf98a6897948 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 1 Feb 2023 18:12:13 +0000 Subject: [PATCH 041/470] Cleanup code --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 14 -------------- src/Planner/PlannerActionsVisitor.cpp | 3 --- src/Planner/PlannerExpressionAnalysis.cpp | 5 +---- 3 files changed, 1 insertion(+), 21 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 4c1d6d309d6..003944fa8b0 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -447,9 +447,6 @@ public: alias_name_to_expressions[node_alias].push_back(node); } - if (const auto * function = node->as()) - LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Pushed function {} on stack", function->getFunctionName()); - expressions.emplace_back(node); } @@ -468,9 +465,6 @@ public: alias_name_to_expressions.erase(it); } - if (const auto * function = top_expression->as()) - LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Poped function {} on stack", function->getFunctionName()); - expressions.pop_back(); } @@ -500,7 +494,6 @@ public: if (factory.isAggregateFunctionName(function->getFunctionName())) return true; - LOG_DEBUG(&Poco::Logger::get("ExpressionsStack"), "Function {} is being resolved, but is not aggregate", function->getFunctionName()); } return false; } @@ -6071,11 +6064,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier for (const auto & group_by_elem : grouping_set->as()->getNodes()) scope.nullable_group_by_keys.insert(group_by_elem->clone()); } - // for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes()) - // { - // for (const auto & group_by_elem : grouping_set->as()->getNodes()) - // group_by_elem->convertToNullable(); - // } } else { @@ -6088,8 +6076,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier { for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) scope.nullable_group_by_keys.insert(group_by_elem->clone()); - // for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) - // group_by_elem->convertToNullable(); } } } diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 13c88d47ca3..6bd245858fe 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -29,7 +29,6 @@ #include #include #include -#include namespace DB { @@ -83,7 +82,6 @@ public: node_name, actions_dag->dumpNames()); - LOG_DEBUG(&Poco::Logger::get("ActionsScopeNode"), "Node: {} {}", it->second->result_name, it->second->result_type->getName()); return it->second; } @@ -227,7 +225,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitColumn(const QueryTreeNodePtr & node) { auto column_node_name = calculateActionNodeName(node, *planner_context, node_to_node_name); - LOG_DEBUG(&Poco::Logger::get("PlannerActionsVisitorImpl"), "Processing column with name: {}", column_node_name); const auto & column_node = node->as(); Int64 actions_stack_size = static_cast(actions_stack.size() - 1); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 79feb501f37..99815d89168 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include @@ -16,8 +17,6 @@ #include #include #include -#include "Common/tests/gtest_global_context.h" -#include "DataTypes/DataTypeNullable.h" namespace DB { @@ -303,7 +302,6 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, ActionsChain & actions_chain) { const auto & projection_input = current_output_columns; - LOG_DEBUG(&Poco::Logger::get("PlannerExpressionAnalysis"), "Projection node: {}", query_node.getProjectionNode()->dumpTree()); auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); auto projection_columns = query_node.getProjectionColumns(); @@ -325,7 +323,6 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, for (size_t i = 0; i < projection_outputs_size; ++i) { auto & projection_column = projection_columns[i]; - LOG_DEBUG(&Poco::Logger::get("PlannerExpressionAnalysis"), "Projection column {}: {} {}", i, projection_column.name, projection_column.type->getName()); const auto * projection_node = projection_actions_outputs[i]; const auto & projection_node_name = projection_node->result_name; From 139a684952d42fd3be4289ca2ada8cfedb7a6591 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 3 Feb 2023 16:00:25 +0000 Subject: [PATCH 042/470] Move analyzeProjection call --- src/Planner/PlannerExpressionAnalysis.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 99815d89168..d0104b0bf2a 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -419,7 +419,8 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo std::optional where_analysis_result_optional; std::optional where_action_step_index_optional; - ColumnsWithTypeAndName current_output_columns = join_tree_input_columns; + const auto * input_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); + ColumnsWithTypeAndName current_output_columns = input_columns ? *input_columns : join_tree_input_columns; if (query_node.hasWhere()) { @@ -446,6 +447,9 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (window_analysis_result_optional) current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + auto projection_analysis_result = analyzeProjection(query_node, current_output_columns, planner_context, actions_chain); + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); + std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) { @@ -461,10 +465,8 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } - auto projection_analysis_result = analyzeProjection(query_node, current_output_columns, planner_context, actions_chain); - const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + const auto & project_names_input = chain_available_output_columns ? *chain_available_output_columns : current_output_columns; auto project_names_actions = std::make_shared(project_names_input); project_names_actions->project(projection_analysis_result.projection_column_names_with_display_aliases); actions_chain.addStep(std::make_unique(project_names_actions)); From a1362b081f5fc4dd8e4fd8762c91800f5e1f0aff Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 3 Feb 2023 17:14:50 +0000 Subject: [PATCH 043/470] Add test --- ...2535_analyzer_group_by_use_nulls.reference | 256 ++++++++++++++++++ .../02535_analyzer_group_by_use_nulls.sql | 85 ++++++ 2 files changed, 341 insertions(+) create mode 100644 tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference create mode 100644 tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference new file mode 100644 index 00000000000..50755627996 --- /dev/null +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.reference @@ -0,0 +1,256 @@ +-- { echoOn } +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +1 1 1 +2 0 2 +3 1 3 +4 0 4 +5 1 5 +6 0 6 +7 1 7 +8 0 8 +9 1 9 +\N \N 45 +set optimize_group_by_function_keys = 0; +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N \N 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 45 +1 0 1 +1 1 1 +2 0 2 +2 0 2 +3 0 3 +3 1 3 +4 0 4 +4 0 4 +5 0 5 +5 1 5 +6 0 6 +6 0 6 +7 0 7 +7 1 7 +8 0 8 +8 0 8 +9 0 9 +9 1 9 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N 0 20 +\N 1 25 +\N \N 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; +0 0 0 +0 0 0 +0 0 20 +0 0 45 +0 1 25 +1 0 1 +1 1 1 +2 0 2 +2 0 2 +3 0 3 +3 1 3 +4 0 4 +4 0 4 +5 0 5 +5 1 5 +6 0 6 +6 0 6 +7 0 7 +7 1 7 +8 0 8 +8 0 8 +9 0 9 +9 1 9 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +\N 0 20 +\N 1 25 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; +0 0 0 +0 0 20 +0 1 25 +1 0 1 +2 0 2 +3 0 3 +4 0 4 +5 0 5 +6 0 6 +7 0 7 +8 0 8 +9 0 9 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N \N 45 + +0 0 45 +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; +0 0 0 +0 \N 0 +1 1 1 +1 \N 1 +2 0 2 +2 \N 2 +3 1 3 +3 \N 3 +4 0 4 +4 \N 4 +5 1 5 +5 \N 5 +6 0 6 +6 \N 6 +7 1 7 +7 \N 7 +8 0 8 +8 \N 8 +9 1 9 +9 \N 9 +\N 0 20 +\N 1 25 +\N \N 45 + +0 0 45 +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY 1, tuple(val) +SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; +0 \N 0 +1 \N 1 +2 \N 2 +3 \N 3 +4 \N 4 +5 \N 5 +6 \N 6 +7 \N 7 +8 \N 8 +9 \N 9 +\N 0 20 +\N 1 25 diff --git a/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql new file mode 100644 index 00000000000..a4d4f2f8bc9 --- /dev/null +++ b/tests/queries/0_stateless/02535_analyzer_group_by_use_nulls.sql @@ -0,0 +1,85 @@ +SET allow_experimental_analyzer=1; + +-- { echoOn } +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +set optimize_group_by_function_keys = 0; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=0; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 1; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls = 0; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY ROLLUP(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT number, number % 2, sum(number) AS val +FROM numbers(10) +GROUP BY CUBE(number, number % 2) WITH TOTALS +ORDER BY (number, number % 2, val) +SETTINGS group_by_use_nulls=1; + +SELECT + number, + number % 2, + sum(number) AS val +FROM numbers(10) +GROUP BY + GROUPING SETS ( + (number), + (number % 2) + ) +ORDER BY 1, tuple(val) +SETTINGS group_by_use_nulls = 1, max_bytes_before_external_sort=10; From 3ea3b3545d8c37e20ce5d5674038807030656590 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 3 Feb 2023 18:02:50 +0000 Subject: [PATCH 044/470] Add statefull test --- .../1_stateful/00173_group_by_use_nulls.sql | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.sql b/tests/queries/1_stateful/00173_group_by_use_nulls.sql index 7acacc4e579..f110ebb0db4 100644 --- a/tests/queries/1_stateful/00173_group_by_use_nulls.sql +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.sql @@ -8,3 +8,17 @@ ORDER BY CounterID ASC LIMIT 10 SETTINGS group_by_use_nulls = 1; + +-- { echoOn } +set allow_experimental_analyzer = 1; + +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM remote('127.0.0.{1,2}', test, hits) +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1; From aca4f08bf5b2aac5fae6cc897cd0be4b9498b65a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 Feb 2023 14:33:45 +0000 Subject: [PATCH 045/470] Correctly calculate size after snapshot load --- src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 9 +++++++ src/Coordination/FourLetterCommand.h | 13 ++++++++++ src/Coordination/KeeperDispatcher.h | 5 ++++ src/Coordination/KeeperServer.cpp | 5 ++++ src/Coordination/KeeperServer.h | 2 ++ src/Coordination/KeeperSnapshotManager.cpp | 28 +++++++++++++--------- src/Coordination/KeeperStateMachine.cpp | 8 +++++++ src/Coordination/KeeperStateMachine.h | 1 + src/Coordination/KeeperStorage.cpp | 20 ++++++++++++++-- src/Coordination/KeeperStorage.h | 5 +++- src/Coordination/SnapshotableHashTable.h | 13 ++++++++++ 12 files changed, 96 insertions(+), 15 deletions(-) diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index e665ccb89c7..f5c79d3be7a 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 3e0e5acee0c..6157daad1cd 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -145,6 +145,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr request_leader_command = std::make_shared(keeper_dispatcher); factory.registerCommand(request_leader_command); + FourLetterCommandPtr recalculate_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(recalculate_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -515,4 +518,10 @@ String RequestLeaderCommand::run() return keeper_dispatcher.requestLeader() ? "Sent leadership request to leader." : "Failed to send leadership request to leader."; } +String RecalculateCommand::run() +{ + keeper_dispatcher.recalculateStorageStats(); + return "ok"; +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index 8a8aacf7a3a..e1fe0333081 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -377,4 +377,17 @@ struct RequestLeaderCommand : public IFourLetterCommand ~RequestLeaderCommand() override = default; }; +/// Request to be leader. +struct RecalculateCommand : public IFourLetterCommand +{ + explicit RecalculateCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "rclc"; } + String run() override; + ~RecalculateCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index ff902d8e036..90965d0934e 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -225,6 +225,11 @@ public: { return server->requestLeader(); } + + void recalculateStorageStats() + { + return server->recalculateStorageStats(); + } }; } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b823fbc697c..05f97313edd 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -947,4 +947,9 @@ bool KeeperServer::requestLeader() return isLeader() || raft_instance->request_leadership(); } +void KeeperServer::recalculateStorageStats() +{ + state_machine->recalculateStorageStats(); +} + } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index feadf3bb7ce..bcff81f66f2 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -137,6 +137,8 @@ public: KeeperLogInfo getKeeperLogInfo(); bool requestLeader(); + + void recalculateStorageStats(); }; } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a3f7dbc2c6a..8b80db3e520 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -361,19 +361,25 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial "If you still want to ignore it, you can set 'keeper_server.ignore_system_path_on_startup' to true", error_msg); } - else if (match_result == EXACT && !is_node_empty(node)) + else if (match_result == EXACT) { - if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + if (!is_node_empty(node)) { - LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); - node = KeeperStorage::Node{}; + if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT) + { + LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); + node = KeeperStorage::Node{}; + } + else + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}. Ignoring it can lead to data loss. " + "If you still want to ignore it, you can set 'keeper_server.ignore_system_path_on_startup' to true", + error_msg); } - else - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "{}. Ignoring it can lead to data loss. " - "If you still want to ignore it, you can set 'keeper_server.ignore_system_path_on_startup' to true", - error_msg); + + // we always ignore the written size for this node + node.recalculateSize(); } storage.container.insertOrReplace(path, node); @@ -390,7 +396,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { auto parent_path = parentPath(itr.key); storage.container.updateValue( - parent_path, [path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseName(path)); }); + parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseName(path), /*update_size*/ version < SnapshotVersion::V4); }); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e591f87c6f1..42ff59f45d9 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -640,4 +640,12 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const return nullptr; } +void KeeperStateMachine::recalculateStorageStats() +{ + std::lock_guard lock(storage_and_responses_lock); + LOG_INFO(log, "Recalculating storage stats"); + storage->recalculateStats(); + LOG_INFO(log, "Done recalculating storage stats"); +} + } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index ffc7fce1cfe..d8181532f09 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -103,6 +103,7 @@ public: uint64_t getKeyArenaSize() const; uint64_t getLatestSnapshotBufSize() const; + void recalculateStorageStats(); private: /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0e4d631938c..b1f3b44b1e1 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -201,9 +201,10 @@ void KeeperStorage::Node::setData(String new_data) data = std::move(new_data); } -void KeeperStorage::Node::addChild(StringRef child_path) +void KeeperStorage::Node::addChild(StringRef child_path, bool update_size) { - size_bytes += sizeof child_path; + if (update_size) [[likely]] + size_bytes += sizeof child_path; children.insert(child_path); } @@ -234,6 +235,16 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) cached_digest = other.cached_digest; } +void KeeperStorage::Node::recalculateSize() +{ + size_bytes = sizeof(Node); + + for (const auto child_path : children) + size_bytes += sizeof child_path; + + size_bytes += data.size(); +} + KeeperStorage::KeeperStorage( int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, const bool initialize_system_nodes) : session_expiry_queue(tick_time_ms), keeper_context(keeper_context_), superdigest(superdigest_) @@ -2407,5 +2418,10 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const return ret; } +void KeeperStorage::recalculateStats() +{ + container.recalculateDataSize(); +} + } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index a40cca8e778..be528072df4 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -47,7 +47,7 @@ public: const auto & getData() const noexcept { return data; } - void addChild(StringRef child_path); + void addChild(StringRef child_path, bool update_size = true); void removeChild(StringRef child_path); @@ -64,6 +64,8 @@ public: // (e.g. we don't need to copy list of children) void shallowCopy(const Node & other); + void recalculateSize(); + private: String data; ChildrenSet children{}; @@ -466,6 +468,7 @@ public: void dumpWatchesByPath(WriteBufferFromOwnString & buf) const; void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const; + void recalculateStats(); private: void removeDigest(const Node & node, std::string_view path); void addDigest(const Node & node, std::string_view path); diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 27572ab86c7..97c7e933334 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -8,6 +8,8 @@ #include #include +#include + namespace DB { @@ -319,6 +321,17 @@ public: return approximate_data_size; } + void recalculateDataSize() + { + approximate_data_size = 0; + for (auto & node : list) + { + node.value.recalculateSize(); + approximate_data_size += node.key.size; + approximate_data_size += node.value.sizeInBytes(); + } + } + uint64_t keyArenaSize() const { return arena.size(); From 9cde46acff502b8ba8972c9c3ccbc13b904b6563 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 Feb 2023 16:09:56 +0000 Subject: [PATCH 046/470] Correctly calculate size on update --- src/Coordination/SnapshotableHashTable.h | 15 +-- .../test_keeper_mntr_data_size/__init__.py | 0 .../configs/enable_keeper.xml | 29 ++++++ .../test_keeper_mntr_data_size/test.py | 93 +++++++++++++++++++ 4 files changed, 130 insertions(+), 7 deletions(-) create mode 100644 tests/integration/test_keeper_mntr_data_size/__init__.py create mode 100644 tests/integration/test_keeper_mntr_data_size/configs/enable_keeper.xml create mode 100644 tests/integration/test_keeper_mntr_data_size/test.py diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 97c7e933334..5b01ca255f8 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -8,8 +8,6 @@ #include #include -#include - namespace DB { @@ -66,7 +64,7 @@ private: /// value_size: size of value to add /// old_value_size: size of value to minus /// old_value_size=0 means there is no old value with the same key. - void updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t old_value_size) + void updateDataSize(OperationType op_type, uint64_t key_size, uint64_t value_size, uint64_t old_value_size, bool remove_old = true) { switch (op_type) { @@ -96,14 +94,14 @@ private: case UPDATE_VALUE: approximate_data_size += key_size; approximate_data_size += value_size; - if (!snapshot_mode) + if (remove_old) { approximate_data_size -= key_size; approximate_data_size -= old_value_size; } break; case ERASE: - if (!snapshot_mode) + if (remove_old) { approximate_data_size -= key_size; approximate_data_size -= old_value_size; @@ -179,7 +177,7 @@ public: list_itr->value = value; } } - updateDataSize(INSERT_OR_REPLACE, key.size(), value.sizeInBytes(), old_value_size); + updateDataSize(INSERT_OR_REPLACE, key.size(), value.sizeInBytes(), old_value_size, !snapshot_mode); } bool erase(const std::string & key) @@ -224,6 +222,7 @@ public: const_iterator ret; + bool remove_old_size = true; if (snapshot_mode) { /// We in snapshot mode but updating some node which is already more @@ -239,6 +238,8 @@ public: auto itr = list.insert(list.end(), std::move(elem_copy)); it->getMapped() = itr; ret = itr; + + remove_old_size = false; } else { @@ -252,7 +253,7 @@ public: ret = list_itr; } - updateDataSize(UPDATE_VALUE, key.size, ret->value.sizeInBytes(), old_value_size); + updateDataSize(UPDATE_VALUE, key.size, ret->value.sizeInBytes(), old_value_size, remove_old_size); return ret; } diff --git a/tests/integration/test_keeper_mntr_data_size/__init__.py b/tests/integration/test_keeper_mntr_data_size/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_mntr_data_size/configs/enable_keeper.xml b/tests/integration/test_keeper_mntr_data_size/configs/enable_keeper.xml new file mode 100644 index 00000000000..a3217b34501 --- /dev/null +++ b/tests/integration/test_keeper_mntr_data_size/configs/enable_keeper.xml @@ -0,0 +1,29 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 10 + 5 + 5000 + 10000 + trace + + + 0 + 0 + 0 + + + + + 1 + localhost + 9234 + + + + diff --git a/tests/integration/test_keeper_mntr_data_size/test.py b/tests/integration/test_keeper_mntr_data_size/test.py new file mode 100644 index 00000000000..8789ca0354c --- /dev/null +++ b/tests/integration/test_keeper_mntr_data_size/test.py @@ -0,0 +1,93 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as keeper_utils +import random +import string +import os +import time +from kazoo.client import KazooClient, KazooState + + +cluster = ClickHouseCluster(__file__) + +# clickhouse itself will use external zookeeper +node = cluster.add_instance( + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, + with_zookeeper=True, +) + + +def random_string(length): + return "".join(random.choices(string.ascii_lowercase + string.digits, k=length)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +def restart_clickhouse(): + node.restart_clickhouse() + keeper_utils.wait_until_connected(cluster, node) + + +def test_mntr_data_size_after_restart(started_cluster): + try: + node_zk = None + node_zk = get_connection_zk("node") + + node_zk.create("/test_mntr_data_size", b"somevalue") + for i in range(100): + node_zk.create( + "/test_mntr_data_size/node" + str(i), random_string(123).encode() + ) + + def get_line_with_size(): + return next( + filter( + lambda line: "zk_approximate_data_size" in line, + keeper_utils.send_4lw_cmd(started_cluster, node, "mntr").split( + "\n" + ), + ), + None, + ) + + line_size_before = get_line_with_size() + assert line_size_before != None + + node_zk.stop() + node_zk.close() + node_zk = None + + restart_clickhouse() + + assert get_line_with_size() == line_size_before + + keeper_utils.send_4lw_cmd(started_cluster, node, "rclc") + assert get_line_with_size() == line_size_before + finally: + try: + if node_zk is not None: + node_zk.stop() + node_zk.close() + except: + pass From b11458f14241f81f22129a1476a9220010c2586e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 Feb 2023 07:51:22 +0000 Subject: [PATCH 047/470] Fix erase --- src/Coordination/SnapshotableHashTable.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 5b01ca255f8..cfa3098b4a1 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -202,7 +202,7 @@ public: list.erase(list_itr); } - updateDataSize(ERASE, key.size(), 0, old_data_size); + updateDataSize(ERASE, key.size(), 0, old_data_size, !snapshot_mode); return true; } From fe48ab2b1d9006cdb8e0d937ad8d9a9370020c1c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Feb 2023 14:58:34 +0000 Subject: [PATCH 048/470] randomize setting 'ratio_of_defaults_for_sparse_serialization' --- tests/clickhouse-test | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4361b64b62f..e84c9beed5d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -511,7 +511,9 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint(0, 1), + "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( + 0, 1 + ), } @staticmethod @@ -524,10 +526,9 @@ class SettingsRandomizer: class MergeTreeSettingsRandomizer: settings = { - # Temporary disable due to large number of failures. TODO: fix. - # "ratio_of_defaults_for_sparse_serialization": threshold_generator( - # 0.1, 0.6, 0.0, 1.0 - # ), + "ratio_of_defaults_for_sparse_serialization": threshold_generator( + 0.3, 0.5, 0.0, 1.0 + ), "prefer_fetch_merged_part_size_threshold": threshold_generator( 0.2, 0.5, 1, 10 * 1024 * 1024 * 1024 ), @@ -1222,7 +1223,7 @@ class TestCase: ) result.check_if_need_retry(args, stdout, stderr, self.runs_count) # to avoid breaking CSV parser - result.description = result.description.replace('\0', '') + result.description = result.description.replace("\0", "") if result.status == TestStatus.FAIL: result.description = self.add_info_about_settings(result.description) @@ -1678,7 +1679,12 @@ def check_server_started(args): print("\nConnection timeout, will not retry") break except Exception as e: - print("\nUexpected exception, will not retry: ", type(e).__name__, ": ", str(e)) + print( + "\nUexpected exception, will not retry: ", + type(e).__name__, + ": ", + str(e), + ) break print("\nAll connection tries failed") @@ -1955,6 +1961,7 @@ def reportCoverage(args): ) ) + def reportLogStats(args): query = """ WITH From 94407682b4c360b47967a0b02d8ca9eb83767317 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Feb 2023 18:06:15 +0000 Subject: [PATCH 049/470] fix function with sparse columns arguments --- src/Columns/ColumnAggregateFunction.h | 5 +++++ src/Columns/ColumnArray.cpp | 5 +++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnCompressed.h | 1 + src/Columns/ColumnConst.h | 5 +++++ src/Columns/ColumnDecimal.h | 5 +++++ src/Columns/ColumnFixedString.h | 5 +++++ src/Columns/ColumnFunction.h | 5 +++++ src/Columns/ColumnLowCardinality.h | 5 +++++ src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.h | 5 +++++ src/Columns/ColumnObject.h | 1 + src/Columns/ColumnSparse.cpp | 13 +++++++++---- src/Columns/ColumnSparse.h | 2 +- src/Columns/ColumnString.h | 5 +++++ src/Columns/ColumnTuple.cpp | 5 +++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnUnique.h | 5 +++++ src/Columns/ColumnVector.h | 5 +++++ src/Columns/IColumn.h | 8 +++++++- src/Columns/IColumnDummy.h | 5 +++++ src/Columns/IColumnImpl.h | 10 ++++++++++ src/Functions/IFunction.cpp | 6 ++++-- 24 files changed, 106 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 38040d65d3b..ae486d5d24e 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -223,6 +223,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getRatioOfDefaultRows is not supported for ColumnAggregateFunction"); } + UInt64 getNumberOfDefaultRows() const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getNumberOfDefaultRows is not supported for ColumnAggregateFunction"); + } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getIndicesOfNonDefaultRows is not supported for ColumnAggregateFunction"); diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 0d085a3275a..cbeebe52141 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -953,6 +953,11 @@ double ColumnArray::getRatioOfDefaultRows(double sample_ratio) const return getRatioOfDefaultRowsImpl(sample_ratio); } +UInt64 ColumnArray::getNumberOfDefaultRows() const +{ + return getNumberOfDefaultRowsImpl(); +} + void ColumnArray::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const { return getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 44652fd0c4b..e60c388831d 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -173,6 +173,7 @@ public: } double getRatioOfDefaultRows(double sample_ratio) const override; + UInt64 getNumberOfDefaultRows() const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index b258dbac878..d747f8ef5f4 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -117,6 +117,7 @@ public: void getExtremes(Field &, Field &) const override { throwMustBeDecompressed(); } size_t byteSizeAt(size_t) const override { throwMustBeDecompressed(); } double getRatioOfDefaultRows(double) const override { throwMustBeDecompressed(); } + UInt64 getNumberOfDefaultRows() const override { throwMustBeDecompressed(); } void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeDecompressed(); } protected: diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index b86ed393e44..fcdcd2ce224 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -253,6 +253,11 @@ public: return data->isDefaultAt(0) ? 1.0 : 0.0; } + UInt64 getNumberOfDefaultRows() const override + { + return data->isDefaultAt(0) ? s : 0; + } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { if (!data->isDefaultAt(0)) diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index d47164a178d..03e0b9be558 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -136,6 +136,11 @@ public: return this->template getRatioOfDefaultRowsImpl(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return this->template getNumberOfDefaultRowsImpl(); + } + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override { return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 7c2d9b1a155..39497e3403e 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -200,6 +200,11 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return getNumberOfDefaultRowsImpl(); + } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { return getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 7b7ceb29a10..a1f6245c2bd 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -168,6 +168,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getRatioOfDefaultRows is not supported for {}", getName()); } + UInt64 getNumberOfDefaultRows() const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getNumberOfDefaultRows is not supported for {}", getName()); + } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getIndicesOfNonDefaultRows is not supported for {}", getName()); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e895bc6b54e..1f2dbdc202e 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -199,6 +199,11 @@ public: return getIndexes().getRatioOfDefaultRows(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return getIndexes().getNumberOfDefaultRows(); + } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { return getIndexes().getIndicesOfNonDefaultRows(indices, from, limit); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 2b5f5f94ee8..b533b68b98d 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -296,6 +296,11 @@ double ColumnMap::getRatioOfDefaultRows(double sample_ratio) const return getRatioOfDefaultRowsImpl(sample_ratio); } +UInt64 ColumnMap::getNumberOfDefaultRows() const +{ + return getNumberOfDefaultRowsImpl(); +} + void ColumnMap::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const { return getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index db918c3db50..a11905fcaa0 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -92,6 +92,7 @@ public: void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override; bool structureEquals(const IColumn & rhs) const override; double getRatioOfDefaultRows(double sample_ratio) const override; + UInt64 getNumberOfDefaultRows() const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; void finalize() override { nested->finalize(); } bool isFinalized() const override { return nested->isFinalized(); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 85bf095a9d1..df0088debc9 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -156,6 +156,11 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return getNumberOfDefaultRowsImpl(); + } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index d09974fcc46..91920908542 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -254,6 +254,7 @@ public: 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: diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 43802b6bbc8..9bc78ef115d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -470,7 +470,7 @@ int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs bool ColumnSparse::hasEqualValues() const { - size_t num_defaults = getNumberOfDefaults(); + size_t num_defaults = getNumberOfDefaultRows(); if (num_defaults == _size) return true; @@ -512,7 +512,7 @@ void ColumnSparse::getPermutationImpl(IColumn::PermutationSortDirection directio else values->getPermutation(direction, stability, limit + 1, null_direction_hint, perm); - size_t num_of_defaults = getNumberOfDefaults(); + size_t num_of_defaults = getNumberOfDefaultRows(); size_t row = 0; const auto & offsets_data = getOffsetsData(); @@ -677,7 +677,7 @@ void ColumnSparse::getExtremes(Field & min, Field & max) const return; } - if (getNumberOfDefaults() == 0) + if (getNumberOfDefaultRows() == 0) { size_t min_idx = 1; size_t max_idx = 1; @@ -709,7 +709,12 @@ void ColumnSparse::getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t double ColumnSparse::getRatioOfDefaultRows(double) const { - return static_cast(getNumberOfDefaults()) / _size; + return static_cast(getNumberOfDefaultRows()) / _size; +} + +UInt64 ColumnSparse::getNumberOfDefaultRows() const +{ + return _size - offsets->size(); } MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & selector) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index ffe392a83c1..d4eb13bf208 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -132,6 +132,7 @@ public: void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override; double getRatioOfDefaultRows(double sample_ratio) const override; + UInt64 getNumberOfDefaultRows() const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; @@ -150,7 +151,6 @@ public: size_t sizeOfValueIfFixed() const override { return values->sizeOfValueIfFixed() + values->sizeOfValueIfFixed(); } bool isCollationSupported() const override { return values->isCollationSupported(); } - size_t getNumberOfDefaults() const { return _size - offsets->size(); } size_t getNumberOfTrailingDefaults() const { return offsets->empty() ? _size : _size - getOffsetsData().back() - 1; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index aa251b1fda0..c9eed212616 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -273,6 +273,11 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return getNumberOfDefaultRowsImpl(); + } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { return getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 903540c1859..369842c7281 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -565,6 +565,11 @@ double ColumnTuple::getRatioOfDefaultRows(double sample_ratio) const return getRatioOfDefaultRowsImpl(sample_ratio); } +UInt64 ColumnTuple::getNumberOfDefaultRows() const +{ + return getNumberOfDefaultRowsImpl(); +} + void ColumnTuple::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const { return getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 25f6328b3fc..281fd94d893 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -102,6 +102,7 @@ public: bool isCollationSupported() const override; ColumnPtr compress() const override; double getRatioOfDefaultRows(double sample_ratio) const override; + UInt64 getNumberOfDefaultRows() const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; void finalize() override; bool isFinalized() const override; diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 8a95726d2be..377255d80c7 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -146,6 +146,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getRatioOfDefaultRows' not implemented for ColumnUnique"); } + UInt64 getNumberOfDefaultRows() const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getNumberOfDefaultRows' not implemented for ColumnUnique"); + } + void getIndicesOfNonDefaultRows(IColumn::Offsets &, size_t, size_t) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getIndicesOfNonDefaultRows' not implemented for ColumnUnique"); diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index ded66430160..bf790423d1d 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -358,6 +358,11 @@ public: return this->template getRatioOfDefaultRowsImpl(sample_ratio); } + UInt64 getNumberOfDefaultRows() const override + { + return this->template getNumberOfDefaultRowsImpl(); + } + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override { return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 53619c73e5b..d777fd7b240 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -441,10 +441,13 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method structureEquals is not supported for {}", getName()); } - /// Returns ration of values in column, that equal to default value of column. + /// Returns ratio of values in column, that are equal to default value of column. /// Checks only @sample_ratio ratio of rows. [[nodiscard]] virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// NOLINT + /// Returns number of values in column, that are equal to default value of column. + [[nodiscard]] virtual UInt64 getNumberOfDefaultRows() const = 0; + /// Returns indices of values in column, that not equal to default value of column. virtual void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const = 0; @@ -584,6 +587,9 @@ protected: template double getRatioOfDefaultRowsImpl(double sample_ratio) const; + template + UInt64 getNumberOfDefaultRowsImpl() const; + template void getIndicesOfNonDefaultRowsImpl(Offsets & indices, size_t from, size_t limit) const; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 0b00ebbdd1f..82d4c857b29 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -168,6 +168,11 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getRatioOfDefaultRows is not supported for {}", getName()); } + UInt64 getNumberOfDefaultRows() const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getNumberOfDefaultRows is not supported for {}", getName()); + } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getIndicesOfNonDefaultRows is not supported for {}", getName()); diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 8537a4c097c..e676373e211 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -194,6 +194,16 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const return static_cast(res) / num_checked_rows; } +template +UInt64 IColumn::getNumberOfDefaultRowsImpl() const +{ + UInt64 res = 0; + size_t num_rows = size(); + for (size_t i = 0; i < num_rows; ++i) + res += static_cast(*this).isDefaultAt(i); + return res; +} + template void IColumn::getIndicesOfNonDefaultRowsImpl(Offsets & indices, size_t from, size_t limit) const { diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 1c30dee0482..2163fa27c51 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -321,7 +321,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, const auto * column_sparse = checkAndGetColumn(arguments[i].column.get()); /// In rare case, when sparse column doesn't have default values, /// it's more convenient to convert it to full before execution of function. - if (column_sparse && column_sparse->getNumberOfDefaults()) + if (column_sparse && column_sparse->getNumberOfDefaultRows()) { sparse_column_position = i; ++num_sparse_columns; @@ -359,7 +359,9 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, return res->cloneResized(input_rows_count); /// If default of sparse column is changed after execution of function, convert to full column. - if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0)) + /// If there are any default in non-zero position after execution of function, convert to full column. + /// Currently there is no easy way to rebuild sparse column with new offsets. + if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); From 1fce8e4eedf14f3fa8c1b9bc8ff90ef7a52a311f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 13 Feb 2023 18:35:34 +0000 Subject: [PATCH 050/470] Fix alias resolve with group_by_use_nulls --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 33 +++++++++++++++-------- src/Planner/PlannerExpressionAnalysis.cpp | 3 ++- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 003944fa8b0..2e7d0b2c6db 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,6 +1,5 @@ #include -#include "Common/logger_useful.h" #include #include @@ -194,7 +193,6 @@ namespace ErrorCodes * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. - * TODO: Support group_by_use_nulls. * TODO: Scalar subqueries cache. */ @@ -676,7 +674,11 @@ struct IdentifierResolveScope if (auto * union_node = scope_node->as()) context = union_node->getContext(); else if (auto * query_node = scope_node->as()) + { context = query_node->getContext(); + group_by_use_nulls = context->getSettingsRef().group_by_use_nulls && + (query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube()); + } } QueryTreeNodePtr scope_node; @@ -729,6 +731,9 @@ struct IdentifierResolveScope /// Use identifier lookup to result cache bool use_identifier_lookup_to_result_cache = true; + /// Apply nullability to aggregation keys + bool group_by_use_nulls = false; + /// Subquery depth size_t subquery_depth = 0; @@ -3122,7 +3127,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || !scope.use_identifier_lookup_to_result_cache || - scope.context->getSettingsRef().group_by_use_nulls) + scope.group_by_use_nulls) scope.identifier_lookup_to_result.erase(it); return resolve_result; @@ -4650,7 +4655,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } function_node.resolveAsFunction(std::move(function_base)); - if (settings.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) + if (scope.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) function_node.convertToNullable(); } catch (Exception & e) @@ -4795,8 +4800,8 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id node = tryResolveIdentifier({unresolved_identifier, IdentifierLookupContext::TABLE_EXPRESSION}, scope).resolved_identifier; /// If table identifier is resolved as CTE clone it and resolve - auto * subquery_node = node->as(); - auto * union_node = node->as(); + auto * subquery_node = node ? node->as() : nullptr; + auto * union_node = node ? node->as() : nullptr; bool resolved_as_cte = (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); if (resolved_as_cte) @@ -4900,6 +4905,12 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id if (result_projection_names.empty()) result_projection_names.push_back(column_node.getColumnName()); + if (scope.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) + { + node = node->clone(); + node->convertToNullable(); + } + break; } case QueryTreeNodeType::FUNCTION: @@ -4986,7 +4997,7 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id /** Update aliases after expression node was resolved. * Do not update node in alias table if we resolve it for duplicate alias. */ - if (!node_alias.empty() && use_alias_table) + if (!node_alias.empty() && use_alias_table && !scope.group_by_use_nulls) { auto it = scope.alias_name_to_expression_node.find(node_alias); if (it != scope.alias_name_to_expression_node.end()) @@ -6033,7 +6044,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveQueryJoinTreeNode(query_node_typed.getJoinTree(), scope, visitor); } - if (!settings.group_by_use_nulls) + if (!scope.group_by_use_nulls) scope.use_identifier_lookup_to_result_cache = true; /// Resolve query node sections. @@ -6072,7 +6083,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveExpressionNodeList(query_node_typed.getGroupByNode(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (settings.group_by_use_nulls) + if (scope.group_by_use_nulls) { for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) scope.nullable_group_by_keys.insert(group_by_elem->clone()); @@ -6270,7 +6281,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier continue; group_by_keys_nodes.push_back(grouping_set_key->clone()); - if (settings.group_by_use_nulls) + if (scope.group_by_use_nulls) group_by_keys_nodes.back()->convertToNullable(); } } @@ -6280,7 +6291,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier continue; group_by_keys_nodes.push_back(node->clone()); - if (settings.group_by_use_nulls) + if (scope.group_by_use_nulls) group_by_keys_nodes.back()->convertToNullable(); } } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index d0104b0bf2a..55a5da247f9 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -80,7 +80,8 @@ std::pair, std::optionalgetQueryContext()->getSettingsRef().group_by_use_nulls; + bool group_by_use_nulls = planner_context->getQueryContext()->getSettingsRef().group_by_use_nulls && + (query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube()); if (query_node.hasGroupBy()) { From 8a3be450c391e020cbb4497b234243e8dbf5d47f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 13 Feb 2023 18:38:07 +0000 Subject: [PATCH 051/470] Add test query --- tests/queries/1_stateful/00173_group_by_use_nulls.sql | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.sql b/tests/queries/1_stateful/00173_group_by_use_nulls.sql index f110ebb0db4..8531e9efaf8 100644 --- a/tests/queries/1_stateful/00173_group_by_use_nulls.sql +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.sql @@ -9,6 +9,17 @@ ORDER BY LIMIT 10 SETTINGS group_by_use_nulls = 1; +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM test.hits +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1 FORMAT Null; + -- { echoOn } set allow_experimental_analyzer = 1; From e460aa6272578f53f85a3cb3e108cf0b411abc7a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Feb 2023 16:00:51 +0000 Subject: [PATCH 052/470] Fix warning --- src/Planner/PlannerActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 6bd245858fe..c9bae0b2c2a 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -327,7 +327,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack.pop_back(); // TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver. - actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), std::move(function_capture)); + actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture); size_t actions_stack_size = actions_stack.size(); for (size_t i = level + 1; i < actions_stack_size; ++i) From 3730ea388f4e9de81dce547f309f24801d724d2b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 15 Feb 2023 19:37:54 +0000 Subject: [PATCH 053/470] fix issues with sparse columns --- src/Client/ClientBase.cpp | 2 +- src/Columns/ColumnAggregateFunction.h | 4 ++-- src/Columns/IColumnImpl.h | 11 +++++------ src/Interpreters/Aggregator.cpp | 4 ++++ .../0_stateless/02432_s3_parallel_parts_cleanup.sql | 4 ++-- 5 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bc8c43af8c6..6c6d22bc037 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -488,7 +488,7 @@ void ClientBase::onTotals(Block & block, ASTPtr parsed_query) void ClientBase::onExtremes(Block & block, ASTPtr parsed_query) { initOutputFormat(block, parsed_query); - output_format->setExtremes(block); + output_format->setExtremes(materializeBlock(block)); } diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index ae486d5d24e..f9ce45708c9 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -220,12 +220,12 @@ public: double getRatioOfDefaultRows(double) const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getRatioOfDefaultRows is not supported for ColumnAggregateFunction"); + return 0.0; } UInt64 getNumberOfDefaultRows() const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getNumberOfDefaultRows is not supported for ColumnAggregateFunction"); + return 0; } void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index e676373e211..20aaefae39c 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -164,13 +164,9 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of 'sample_ratio' must be in interval (0.0; 1.0], but got: {}", sample_ratio); - /// Randomize a little to avoid boundary effects. - std::uniform_int_distribution dist(1, static_cast(1.0 / sample_ratio)); - size_t num_rows = size(); - size_t num_sampled_rows = static_cast(num_rows * sample_ratio); - size_t num_checked_rows = dist(thread_local_rng); - num_sampled_rows = std::min(num_sampled_rows + dist(thread_local_rng), num_rows); + size_t num_sampled_rows = std::min(static_cast(num_rows * sample_ratio), num_rows); + size_t num_checked_rows = 0; size_t res = 0; if (num_sampled_rows == num_rows) @@ -191,6 +187,9 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const } } + if (num_checked_rows == 0) + return 0.0; + return static_cast(res) / num_checked_rows; } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index da4442f99d3..de172a6d7ac 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1416,6 +1416,10 @@ void Aggregator::prepareAggregateInstructions( materialized_columns.push_back(columns.at(pos)->convertToFullColumnIfConst()); aggregate_columns[i][j] = materialized_columns.back().get(); + /// Sparse columns without defaults may be handled incorrectly. + if (aggregate_columns[i][j]->getNumberOfDefaultRows() == 0) + allow_sparse_arguments = false; + auto full_column = allow_sparse_arguments ? aggregate_columns[i][j]->getPtr() : recursiveRemoveSparse(aggregate_columns[i][j]->getPtr()); diff --git a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql index 235a3335d9d..3688a649d5e 100644 --- a/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql +++ b/tests/queries/0_stateless/02432_s3_parallel_parts_cleanup.sql @@ -7,7 +7,7 @@ drop table if exists rmt2; create table rmt (n int, m int, k int) engine=ReplicatedMergeTree('/test/02432/{database}', '1') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, - max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_compact_part=0, min_rows_for_compact_part=0; + max_replicated_merges_in_queue=0, max_replicated_mutations_in_queue=0, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; insert into rmt(n, m) values (1, 42); insert into rmt(n, m) values (2, 42); @@ -37,7 +37,7 @@ select count(), sum(n), sum(m) from rmt; create table rmt2 (n int, m int, k String) engine=ReplicatedMergeTree('/test/02432/{database}', '2') order by tuple() settings storage_policy = 's3_cache', allow_remote_fs_zero_copy_replication=1, max_part_removal_threads=10, concurrent_part_removal_threshold=1, cleanup_delay_period=1, cleanup_delay_period_random_add=1, - min_bytes_for_compact_part=0, min_rows_for_compact_part=0, max_replicated_merges_in_queue=1, + min_bytes_for_wide_part=0, min_rows_for_wide_part=0, max_replicated_merges_in_queue=1, old_parts_lifetime=0; alter table rmt2 modify column k Nullable(String); From c24d68e7c9bb03d12d61fb5e64023daea26875c6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Feb 2023 21:53:53 +0000 Subject: [PATCH 054/470] fix tests with sparse columns --- src/Columns/IColumnImpl.h | 6 ++++-- .../0_stateless/01780_column_sparse_distinct.reference | 1 + .../queries/0_stateless/01780_column_sparse_distinct.sql | 2 +- .../0_stateless/02179_sparse_columns_detach.reference | 8 ++++---- tests/queries/0_stateless/02179_sparse_columns_detach.sql | 8 ++++---- 5 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 20aaefae39c..0eab9452813 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -164,12 +164,14 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of 'sample_ratio' must be in interval (0.0; 1.0], but got: {}", sample_ratio); + static constexpr auto max_number_of_rows_for_full_search = 1000; + size_t num_rows = size(); size_t num_sampled_rows = std::min(static_cast(num_rows * sample_ratio), num_rows); size_t num_checked_rows = 0; size_t res = 0; - if (num_sampled_rows == num_rows) + if (num_sampled_rows == num_rows || num_rows <= max_number_of_rows_for_full_search) { for (size_t i = 0; i < num_rows; ++i) res += static_cast(*this).isDefaultAt(i); @@ -177,7 +179,7 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const } else if (num_sampled_rows != 0) { - for (size_t i = num_checked_rows; i < num_rows; ++i) + for (size_t i = 0; i < num_rows; ++i) { if (num_checked_rows * num_rows <= i * num_sampled_rows) { diff --git a/tests/queries/0_stateless/01780_column_sparse_distinct.reference b/tests/queries/0_stateless/01780_column_sparse_distinct.reference index bb0cebc6540..beb45208e7b 100644 --- a/tests/queries/0_stateless/01780_column_sparse_distinct.reference +++ b/tests/queries/0_stateless/01780_column_sparse_distinct.reference @@ -5,3 +5,4 @@ all_2_2_0 v Sparse 2 3 4 +5 diff --git a/tests/queries/0_stateless/01780_column_sparse_distinct.sql b/tests/queries/0_stateless/01780_column_sparse_distinct.sql index 502ca7600d4..e98bada1aac 100644 --- a/tests/queries/0_stateless/01780_column_sparse_distinct.sql +++ b/tests/queries/0_stateless/01780_column_sparse_distinct.sql @@ -7,7 +7,7 @@ SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; SYSTEM STOP MERGES t_sparse_distinct; -INSERT INTO t_sparse_distinct SELECT number, number % 5 FROM numbers(100000); +INSERT INTO t_sparse_distinct SELECT number, number % 6 FROM numbers(100000); INSERT INTO t_sparse_distinct SELECT number, number % 100 = 0 FROM numbers(100000); SELECT name, column, serialization_kind diff --git a/tests/queries/0_stateless/02179_sparse_columns_detach.reference b/tests/queries/0_stateless/02179_sparse_columns_detach.reference index 2f9714f7a97..04a9b10c09f 100644 --- a/tests/queries/0_stateless/02179_sparse_columns_detach.reference +++ b/tests/queries/0_stateless/02179_sparse_columns_detach.reference @@ -1,12 +1,12 @@ -1000 +954 id Default s Sparse -1000 +954 id Default s Sparse -1000 +954 id Default s Sparse -1000 +954 id Default s Sparse diff --git a/tests/queries/0_stateless/02179_sparse_columns_detach.sql b/tests/queries/0_stateless/02179_sparse_columns_detach.sql index 4720e6720ba..2ae088fedb4 100644 --- a/tests/queries/0_stateless/02179_sparse_columns_detach.sql +++ b/tests/queries/0_stateless/02179_sparse_columns_detach.sql @@ -4,8 +4,8 @@ CREATE TABLE t_sparse_detach(id UInt64, s String) ENGINE = MergeTree ORDER BY id SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; -INSERT INTO t_sparse_detach SELECT number, number % 20 = 0 ? toString(number) : '' FROM numbers(10000); -INSERT INTO t_sparse_detach SELECT number, number % 20 = 0 ? toString(number) : '' FROM numbers(10000); +INSERT INTO t_sparse_detach SELECT number, number % 21 = 0 ? toString(number) : '' FROM numbers(10000); +INSERT INTO t_sparse_detach SELECT number, number % 21 = 0 ? toString(number) : '' FROM numbers(10000); OPTIMIZE TABLE t_sparse_detach FINAL; @@ -30,8 +30,8 @@ ALTER TABLE t_sparse_detach MODIFY SETTING vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 1; -INSERT INTO t_sparse_detach SELECT number, number % 20 = 0 ? toString(number) : '' FROM numbers(10000); -INSERT INTO t_sparse_detach SELECT number, number % 20 = 0 ? toString(number) : '' FROM numbers(10000); +INSERT INTO t_sparse_detach SELECT number, number % 21 = 0 ? toString(number) : '' FROM numbers(10000); +INSERT INTO t_sparse_detach SELECT number, number % 21 = 0 ? toString(number) : '' FROM numbers(10000); OPTIMIZE TABLE t_sparse_detach FINAL; From 81f5f9708df0048010d42a2ddb334c6c1add22bb Mon Sep 17 00:00:00 2001 From: artem-yadr Date: Fri, 17 Feb 2023 01:39:23 +0300 Subject: [PATCH 055/470] Added support for MongoDB Replica Set URI with enumiration --- .../MongoDB/include/Poco/MongoDB/Connection.h | 278 +++++++++--------- base/poco/MongoDB/src/Connection.cpp | 196 ++++++++---- 2 files changed, 283 insertions(+), 191 deletions(-) diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index de669aa90dd..319e1f93ba0 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -18,150 +18,154 @@ #define MongoDB_Connection_INCLUDED -#include "Poco/MongoDB/RequestMessage.h" -#include "Poco/MongoDB/ResponseMessage.h" -#include "Poco/Mutex.h" #include "Poco/Net/SocketAddress.h" #include "Poco/Net/StreamSocket.h" +#include "Poco/Mutex.h" +#include "Poco/MongoDB/RequestMessage.h" +#include "Poco/MongoDB/ResponseMessage.h" -namespace Poco +namespace Poco { +namespace MongoDB { + + +class MongoDB_API Connection + /// Represents a connection to a MongoDB server + /// using the MongoDB wire protocol. + /// + /// See https://docs.mongodb.com/manual/reference/mongodb-wire-protocol/ + /// for more information on the wire protocol. { -namespace MongoDB +public: + typedef Poco::SharedPtr Ptr; + + class MongoDB_API SocketFactory + { + public: + SocketFactory(); + /// Creates the SocketFactory. + + virtual ~SocketFactory(); + /// Destroys the SocketFactory. + + virtual Poco::Net::StreamSocket createSocket(const std::string& host, int port, Poco::Timespan connectTimeout, bool secure); + /// Creates a Poco::Net::StreamSocket (if secure is false), or a + /// Poco::Net::SecureStreamSocket (if secure is true) connected to the + /// given host and port number. + /// + /// The default implementation will throw a Poco::NotImplementedException + /// if secure is true. + }; + + Connection(); + /// Creates an unconnected Connection. + /// + /// Use this when you want to connect later on. + + Connection(const std::string& hostAndPort); + /// Creates a Connection connected to the given MongoDB instance at host:port. + /// + /// The host and port must be separated with a colon. + + Connection(const std::string& uri, SocketFactory& socketFactory); + /// Creates a Connection connected to the given MongoDB instance at the + /// given URI. + /// + /// See the corresponding connect() method for more information. + + Connection(const std::string& host, int port); + /// Creates a Connection connected to the given MongoDB instance at host and port. + + Connection(const Poco::Net::SocketAddress& addrs); + /// Creates a Connection connected to the given MongoDB instance at the given address. + + Connection(const Poco::Net::StreamSocket& socket); + /// Creates a Connection connected to the given MongoDB instance using the given socket, + /// which must already be connected. + + virtual ~Connection(); + /// Destroys the Connection. + + Poco::Net::SocketAddress address() const; + /// Returns the address of the MongoDB server. + + std::string uri() const; + /// Returns the uri on which the connection was made. + + void connect(const std::string& hostAndPort); + /// Connects to the given MongoDB server. + /// + /// The host and port must be separated with a colon. + + void connect(const std::string& uri, SocketFactory& socketFactory); + /// Connects to the given MongoDB instance at the given URI. + /// + /// The URI must be in standard MongoDB connection string URI format: + /// + /// mongodb://:@hostname.com:/database-name?options + /// + /// The following options are supported: + /// + /// - ssl: If ssl=true is specified, a custom SocketFactory subclass creating + /// a SecureStreamSocket must be supplied. + /// - connectTimeoutMS: Socket connection timeout in milliseconds. + /// - socketTimeoutMS: Socket send/receive timeout in milliseconds. + /// - authMechanism: Authentication mechanism. Only "SCRAM-SHA-1" (default) + /// and "MONGODB-CR" are supported. + /// + /// Unknown options are silently ignored. + /// + /// Will also attempt to authenticate using the specified credentials, + /// using Database::authenticate(). + /// + /// Throws a Poco::NoPermissionException if authentication fails. + + void connect(const std::string& host, int port); + /// Connects to the given MongoDB server. + + void connect(const Poco::Net::SocketAddress& addrs); + /// Connects to the given MongoDB server. + + void connect(const Poco::Net::StreamSocket& socket); + /// Connects using an already connected socket. + + void disconnect(); + /// Disconnects from the MongoDB server. + + void sendRequest(RequestMessage& request); + /// Sends a request to the MongoDB server. + /// + /// Used for one-way requests without a response. + + void sendRequest(RequestMessage& request, ResponseMessage& response); + /// Sends a request to the MongoDB server and receives the response. + /// + /// Use this when a response is expected: only a "query" or "getmore" + /// request will return a response. + +protected: + void connect(); + +private: + Poco::Net::SocketAddress _address; + Poco::Net::StreamSocket _socket; + std::string _uri; +}; + + +// +// inlines +// +inline Net::SocketAddress Connection::address() const { - - - class MongoDB_API Connection - /// Represents a connection to a MongoDB server - /// using the MongoDB wire protocol. - /// - /// See https://docs.mongodb.com/manual/reference/mongodb-wire-protocol/ - /// for more information on the wire protocol. - { - public: - typedef Poco::SharedPtr Ptr; - - class MongoDB_API SocketFactory - { - public: - SocketFactory(); - /// Creates the SocketFactory. - - virtual ~SocketFactory(); - /// Destroys the SocketFactory. - - virtual Poco::Net::StreamSocket createSocket(const std::string & host, int port, Poco::Timespan connectTimeout, bool secure); - /// Creates a Poco::Net::StreamSocket (if secure is false), or a - /// Poco::Net::SecureStreamSocket (if secure is true) connected to the - /// given host and port number. - /// - /// The default implementation will throw a Poco::NotImplementedException - /// if secure is true. - }; - - Connection(); - /// Creates an unconnected Connection. - /// - /// Use this when you want to connect later on. - - Connection(const std::string & hostAndPort); - /// Creates a Connection connected to the given MongoDB instance at host:port. - /// - /// The host and port must be separated with a colon. - - Connection(const std::string & uri, SocketFactory & socketFactory); - /// Creates a Connection connected to the given MongoDB instance at the - /// given URI. - /// - /// See the corresponding connect() method for more information. - - Connection(const std::string & host, int port); - /// Creates a Connection connected to the given MongoDB instance at host and port. - - Connection(const Poco::Net::SocketAddress & addrs); - /// Creates a Connection connected to the given MongoDB instance at the given address. - - Connection(const Poco::Net::StreamSocket & socket); - /// Creates a Connection connected to the given MongoDB instance using the given socket, - /// which must already be connected. - - virtual ~Connection(); - /// Destroys the Connection. - - Poco::Net::SocketAddress address() const; - /// Returns the address of the MongoDB server. - - void connect(const std::string & hostAndPort); - /// Connects to the given MongoDB server. - /// - /// The host and port must be separated with a colon. - - void connect(const std::string & uri, SocketFactory & socketFactory); - /// Connects to the given MongoDB instance at the given URI. - /// - /// The URI must be in standard MongoDB connection string URI format: - /// - /// mongodb://:@hostname.com:/database-name?options - /// - /// The following options are supported: - /// - /// - ssl: If ssl=true is specified, a custom SocketFactory subclass creating - /// a SecureStreamSocket must be supplied. - /// - connectTimeoutMS: Socket connection timeout in milliseconds. - /// - socketTimeoutMS: Socket send/receive timeout in milliseconds. - /// - authMechanism: Authentication mechanism. Only "SCRAM-SHA-1" (default) - /// and "MONGODB-CR" are supported. - /// - /// Unknown options are silently ignored. - /// - /// Will also attempt to authenticate using the specified credentials, - /// using Database::authenticate(). - /// - /// Throws a Poco::NoPermissionException if authentication fails. - - void connect(const std::string & host, int port); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::SocketAddress & addrs); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::StreamSocket & socket); - /// Connects using an already connected socket. - - void disconnect(); - /// Disconnects from the MongoDB server. - - void sendRequest(RequestMessage & request); - /// Sends a request to the MongoDB server. - /// - /// Used for one-way requests without a response. - - void sendRequest(RequestMessage & request, ResponseMessage & response); - /// Sends a request to the MongoDB server and receives the response. - /// - /// Use this when a response is expected: only a "query" or "getmore" - /// request will return a response. - - protected: - void connect(); - - private: - Poco::Net::SocketAddress _address; - Poco::Net::StreamSocket _socket; - }; - - - // - // inlines - // - inline Net::SocketAddress Connection::address() const - { - return _address; - } - - + return _address; } -} // namespace Poco::MongoDB +inline std::string Connection::uri() const +{ + return _uri; +} + +} } // namespace Poco::MongoDB #endif // MongoDB_Connection_INCLUDED diff --git a/base/poco/MongoDB/src/Connection.cpp b/base/poco/MongoDB/src/Connection.cpp index 56bb192cec2..411b59c774e 100644 --- a/base/poco/MongoDB/src/Connection.cpp +++ b/base/poco/MongoDB/src/Connection.cpp @@ -145,71 +145,159 @@ void Connection::connect(const Poco::Net::StreamSocket& socket) void Connection::connect(const std::string& uri, SocketFactory& socketFactory) { - Poco::URI theURI(uri); - if (theURI.getScheme() != "mongodb") throw Poco::UnknownURISchemeException(uri); + std::vector strAddresses; + std::string newURI; - std::string userInfo = theURI.getUserInfo(); - std::string host = theURI.getHost(); - Poco::UInt16 port = theURI.getPort(); - if (port == 0) port = 27017; + if (uri.find(',') != std::string::npos) + { + size_t pos; + size_t head = 0; + if ((pos = uri.find("@")) != std::string::npos) + { + head = pos + 1; + } + else if ((pos = uri.find("://")) != std::string::npos) + { + head = pos + 3; + } - std::string databaseName = theURI.getPath(); - if (!databaseName.empty() && databaseName[0] == '/') databaseName.erase(0, 1); - if (databaseName.empty()) databaseName = "admin"; + std::string tempstr; + std::string::const_iterator it = uri.begin(); + it += head; + size_t tail = head; + for (;it != uri.end() && *it != '?' && *it != '/'; ++it) + { + tempstr += *it; + tail++; + } - bool ssl = false; - Poco::Timespan connectTimeout; - Poco::Timespan socketTimeout; - std::string authMechanism = Database::AUTH_SCRAM_SHA1; + it = tempstr.begin(); + std::string token; + for (;it != tempstr.end(); ++it) + { + if (*it == ',') + { + newURI = uri.substr(0, head) + token + uri.substr(tail, uri.length()); + strAddresses.push_back(newURI); + token = ""; + } + else + { + token += *it; + } + } + newURI = uri.substr(0, head) + token + uri.substr(tail, uri.length()); + strAddresses.push_back(newURI); + } + else + { + strAddresses.push_back(uri); + } - Poco::URI::QueryParameters params = theURI.getQueryParameters(); - for (Poco::URI::QueryParameters::const_iterator it = params.begin(); it != params.end(); ++it) - { - if (it->first == "ssl") - { - ssl = (it->second == "true"); - } - else if (it->first == "connectTimeoutMS") - { - connectTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); - } - else if (it->first == "socketTimeoutMS") - { - socketTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); - } - else if (it->first == "authMechanism") - { - authMechanism = it->second; - } - } + newURI = strAddresses.front(); + Poco::URI theURI(newURI); + if (theURI.getScheme() != "mongodb") throw Poco::UnknownURISchemeException(uri); - connect(socketFactory.createSocket(host, port, connectTimeout, ssl)); + std::string userInfo = theURI.getUserInfo(); + std::string databaseName = theURI.getPath(); + if (!databaseName.empty() && databaseName[0] == '/') databaseName.erase(0, 1); + if (databaseName.empty()) databaseName = "admin"; - if (socketTimeout > 0) - { - _socket.setSendTimeout(socketTimeout); - _socket.setReceiveTimeout(socketTimeout); - } + bool ssl = false; + Poco::Timespan connectTimeout; + Poco::Timespan socketTimeout; + std::string authMechanism = Database::AUTH_SCRAM_SHA1; + std::string readPreference="primary"; - if (!userInfo.empty()) - { - std::string username; - std::string password; - std::string::size_type pos = userInfo.find(':'); - if (pos != std::string::npos) - { - username.assign(userInfo, 0, pos++); - password.assign(userInfo, pos, userInfo.size() - pos); - } - else username = userInfo; + Poco::URI::QueryParameters params = theURI.getQueryParameters(); + for (Poco::URI::QueryParameters::const_iterator it = params.begin(); it != params.end(); ++it) + { + if (it->first == "ssl") + { + ssl = (it->second == "true"); + } + else if (it->first == "connectTimeoutMS") + { + connectTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); + } + else if (it->first == "socketTimeoutMS") + { + socketTimeout = static_cast(1000)*Poco::NumberParser::parse(it->second); + } + else if (it->first == "authMechanism") + { + authMechanism = it->second; + } + else if (it->first == "readPreference") + { + readPreference= it->second; + } + } - Database database(databaseName); - if (!database.authenticate(*this, username, password, authMechanism)) - throw Poco::NoPermissionException(Poco::format("Access to MongoDB database %s denied for user %s", databaseName, username)); - } + for (std::vector::const_iterator it = strAddresses.cbegin();it != strAddresses.cend(); ++it) + { + newURI = *it; + Poco::URI theURI(newURI); + + std::string host = theURI.getHost(); + Poco::UInt16 port = theURI.getPort(); + if (port == 0) port = 27017; + + connect(socketFactory.createSocket(host, port, connectTimeout, ssl)); + _uri = newURI; + if (socketTimeout > 0) + { + _socket.setSendTimeout(socketTimeout); + _socket.setReceiveTimeout(socketTimeout); + } + if (strAddresses.size() > 1) + { + Poco::MongoDB::QueryRequest request("admin.$cmd"); + request.setNumberToReturn(1); + request.selector().add("isMaster", 1); + Poco::MongoDB::ResponseMessage response; + + sendRequest(request, response); + _uri = newURI; + if (!response.documents().empty()) + { + Poco::MongoDB::Document::Ptr doc = response.documents()[0]; + if (doc->get("ismaster") && readPreference == "primary") + { + break; + } + else if (!doc->get("ismaster") && readPreference == "secondary") + { + break; + } + else if (it + 1 == strAddresses.cend()) + { + throw Poco::URISyntaxException(uri); + } + } + } + } + if (!userInfo.empty()) + { + std::string username; + std::string password; + std::string::size_type pos = userInfo.find(':'); + if (pos != std::string::npos) + { + username.assign(userInfo, 0, pos++); + password.assign(userInfo, pos, userInfo.size() - pos); + } + else username = userInfo; + + Database database(databaseName); + + if (!database.authenticate(*this, username, password, authMechanism)) + throw Poco::NoPermissionException(Poco::format("Access to MongoDB database %s denied for user %s", databaseName, username)); + } } + void Connection::disconnect() { _socket.close(); From 4a2efc339bdc25ae5332b3f6fcd08a48e4530653 Mon Sep 17 00:00:00 2001 From: artem-yadr Date: Fri, 17 Feb 2023 01:47:36 +0300 Subject: [PATCH 056/470] Header fix --- .../MongoDB/include/Poco/MongoDB/Connection.h | 280 +++++++++--------- 1 file changed, 142 insertions(+), 138 deletions(-) diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index 319e1f93ba0..699b9652eaf 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -18,154 +18,158 @@ #define MongoDB_Connection_INCLUDED -#include "Poco/Net/SocketAddress.h" -#include "Poco/Net/StreamSocket.h" -#include "Poco/Mutex.h" #include "Poco/MongoDB/RequestMessage.h" #include "Poco/MongoDB/ResponseMessage.h" +#include "Poco/Mutex.h" +#include "Poco/Net/SocketAddress.h" +#include "Poco/Net/StreamSocket.h" -namespace Poco { -namespace MongoDB { - - -class MongoDB_API Connection - /// Represents a connection to a MongoDB server - /// using the MongoDB wire protocol. - /// - /// See https://docs.mongodb.com/manual/reference/mongodb-wire-protocol/ - /// for more information on the wire protocol. +namespace Poco { -public: - typedef Poco::SharedPtr Ptr; - - class MongoDB_API SocketFactory - { - public: - SocketFactory(); - /// Creates the SocketFactory. - - virtual ~SocketFactory(); - /// Destroys the SocketFactory. - - virtual Poco::Net::StreamSocket createSocket(const std::string& host, int port, Poco::Timespan connectTimeout, bool secure); - /// Creates a Poco::Net::StreamSocket (if secure is false), or a - /// Poco::Net::SecureStreamSocket (if secure is true) connected to the - /// given host and port number. - /// - /// The default implementation will throw a Poco::NotImplementedException - /// if secure is true. - }; - - Connection(); - /// Creates an unconnected Connection. - /// - /// Use this when you want to connect later on. - - Connection(const std::string& hostAndPort); - /// Creates a Connection connected to the given MongoDB instance at host:port. - /// - /// The host and port must be separated with a colon. - - Connection(const std::string& uri, SocketFactory& socketFactory); - /// Creates a Connection connected to the given MongoDB instance at the - /// given URI. - /// - /// See the corresponding connect() method for more information. - - Connection(const std::string& host, int port); - /// Creates a Connection connected to the given MongoDB instance at host and port. - - Connection(const Poco::Net::SocketAddress& addrs); - /// Creates a Connection connected to the given MongoDB instance at the given address. - - Connection(const Poco::Net::StreamSocket& socket); - /// Creates a Connection connected to the given MongoDB instance using the given socket, - /// which must already be connected. - - virtual ~Connection(); - /// Destroys the Connection. - - Poco::Net::SocketAddress address() const; - /// Returns the address of the MongoDB server. - - std::string uri() const; - /// Returns the uri on which the connection was made. - - void connect(const std::string& hostAndPort); - /// Connects to the given MongoDB server. - /// - /// The host and port must be separated with a colon. - - void connect(const std::string& uri, SocketFactory& socketFactory); - /// Connects to the given MongoDB instance at the given URI. - /// - /// The URI must be in standard MongoDB connection string URI format: - /// - /// mongodb://:@hostname.com:/database-name?options - /// - /// The following options are supported: - /// - /// - ssl: If ssl=true is specified, a custom SocketFactory subclass creating - /// a SecureStreamSocket must be supplied. - /// - connectTimeoutMS: Socket connection timeout in milliseconds. - /// - socketTimeoutMS: Socket send/receive timeout in milliseconds. - /// - authMechanism: Authentication mechanism. Only "SCRAM-SHA-1" (default) - /// and "MONGODB-CR" are supported. - /// - /// Unknown options are silently ignored. - /// - /// Will also attempt to authenticate using the specified credentials, - /// using Database::authenticate(). - /// - /// Throws a Poco::NoPermissionException if authentication fails. - - void connect(const std::string& host, int port); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::SocketAddress& addrs); - /// Connects to the given MongoDB server. - - void connect(const Poco::Net::StreamSocket& socket); - /// Connects using an already connected socket. - - void disconnect(); - /// Disconnects from the MongoDB server. - - void sendRequest(RequestMessage& request); - /// Sends a request to the MongoDB server. - /// - /// Used for one-way requests without a response. - - void sendRequest(RequestMessage& request, ResponseMessage& response); - /// Sends a request to the MongoDB server and receives the response. - /// - /// Use this when a response is expected: only a "query" or "getmore" - /// request will return a response. - -protected: - void connect(); - -private: - Poco::Net::SocketAddress _address; - Poco::Net::StreamSocket _socket; - std::string _uri; -}; - - -// -// inlines -// -inline Net::SocketAddress Connection::address() const +namespace MongoDB { - return _address; -} -inline std::string Connection::uri() const + + + class MongoDB_API Connection + /// Represents a connection to a MongoDB server + /// using the MongoDB wire protocol. + /// + /// See https://docs.mongodb.com/manual/reference/mongodb-wire-protocol/ + /// for more information on the wire protocol. + { + public: + typedef Poco::SharedPtr Ptr; + + class MongoDB_API SocketFactory + { + public: + SocketFactory(); + /// Creates the SocketFactory. + + virtual ~SocketFactory(); + /// Destroys the SocketFactory. + + virtual Poco::Net::StreamSocket createSocket(const std::string & host, int port, Poco::Timespan connectTimeout, bool secure); + /// Creates a Poco::Net::StreamSocket (if secure is false), or a + /// Poco::Net::SecureStreamSocket (if secure is true) connected to the + /// given host and port number. + /// + /// The default implementation will throw a Poco::NotImplementedException + /// if secure is true. + }; + + Connection(); + /// Creates an unconnected Connection. + /// + /// Use this when you want to connect later on. + + Connection(const std::string & hostAndPort); + /// Creates a Connection connected to the given MongoDB instance at host:port. + /// + /// The host and port must be separated with a colon. + + Connection(const std::string & uri, SocketFactory & socketFactory); + /// Creates a Connection connected to the given MongoDB instance at the + /// given URI. + /// + /// See the corresponding connect() method for more information. + + Connection(const std::string & host, int port); + /// Creates a Connection connected to the given MongoDB instance at host and port. + + Connection(const Poco::Net::SocketAddress & addrs); + /// Creates a Connection connected to the given MongoDB instance at the given address. + + Connection(const Poco::Net::StreamSocket & socket); + /// Creates a Connection connected to the given MongoDB instance using the given socket, + /// which must already be connected. + + virtual ~Connection(); + /// Destroys the Connection. + + Poco::Net::SocketAddress address() const; + /// Returns the address of the MongoDB server. + + std::string uri() const; + /// Returns the uri on which the connection was made. + + void connect(const std::string & hostAndPort); + /// Connects to the given MongoDB server. + /// + /// The host and port must be separated with a colon. + + void connect(const std::string & uri, SocketFactory & socketFactory); + /// Connects to the given MongoDB instance at the given URI. + /// + /// The URI must be in standard MongoDB connection string URI format: + /// + /// mongodb://:@hostname.com:/database-name?options + /// + /// The following options are supported: + /// + /// - ssl: If ssl=true is specified, a custom SocketFactory subclass creating + /// a SecureStreamSocket must be supplied. + /// - connectTimeoutMS: Socket connection timeout in milliseconds. + /// - socketTimeoutMS: Socket send/receive timeout in milliseconds. + /// - authMechanism: Authentication mechanism. Only "SCRAM-SHA-1" (default) + /// and "MONGODB-CR" are supported. + /// + /// Unknown options are silently ignored. + /// + /// Will also attempt to authenticate using the specified credentials, + /// using Database::authenticate(). + /// + /// Throws a Poco::NoPermissionException if authentication fails. + + void connect(const std::string & host, int port); + /// Connects to the given MongoDB server. + + void connect(const Poco::Net::SocketAddress & addrs); + /// Connects to the given MongoDB server. + + void connect(const Poco::Net::StreamSocket & socket); + /// Connects using an already connected socket. + + void disconnect(); + /// Disconnects from the MongoDB server. + + void sendRequest(RequestMessage & request); + /// Sends a request to the MongoDB server. + /// + /// Used for one-way requests without a response. + + void sendRequest(RequestMessage & request, ResponseMessage & response); + /// Sends a request to the MongoDB server and receives the response. + /// + /// Use this when a response is expected: only a "query" or "getmore" + /// request will return a response. + + protected: + void connect(); + + private: + Poco::Net::SocketAddress _address; + Poco::Net::StreamSocket _socket; + std::string _uri; + }; + + + // + // inlines + // + inline Net::SocketAddress Connection::address() const + { + return _address; + } + inline std::string Connection::uri() const { return _uri; } -} } // namespace Poco::MongoDB + +} +} // namespace Poco::MongoDB #endif // MongoDB_Connection_INCLUDED From 83dbde476b1a9554efa0d1edb6d93e9460964a17 Mon Sep 17 00:00:00 2001 From: artem-yadr Date: Fri, 17 Feb 2023 01:53:42 +0300 Subject: [PATCH 057/470] move brackets and delete newline --- base/poco/MongoDB/src/Connection.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/base/poco/MongoDB/src/Connection.cpp b/base/poco/MongoDB/src/Connection.cpp index 411b59c774e..c75211cf9d7 100644 --- a/base/poco/MongoDB/src/Connection.cpp +++ b/base/poco/MongoDB/src/Connection.cpp @@ -297,7 +297,6 @@ void Connection::connect(const std::string& uri, SocketFactory& socketFactory) } - void Connection::disconnect() { _socket.close(); From 4999d6cb1a0398328ddad048a90d53a0cab89ec7 Mon Sep 17 00:00:00 2001 From: artem-yadr <84010375+artem-yadr@users.noreply.github.com> Date: Fri, 17 Feb 2023 01:58:31 +0300 Subject: [PATCH 058/470] Update Connection.h --- base/poco/MongoDB/include/Poco/MongoDB/Connection.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index 699b9652eaf..8fd9c7919e4 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -163,9 +163,9 @@ namespace MongoDB return _address; } inline std::string Connection::uri() const -{ + { return _uri; -} + } } From 08734d4dc0a0ef5f46838248138488e880f757f2 Mon Sep 17 00:00:00 2001 From: artem-yadr Date: Fri, 17 Feb 2023 14:56:21 +0300 Subject: [PATCH 059/470] poco changes are now used in MongoDBDictionarySource --- src/Dictionaries/MongoDBDictionarySource.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index bec566c29a6..f61efcab026 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -114,7 +114,11 @@ MongoDBDictionarySource::MongoDBDictionarySource( { if (!uri.empty()) { - Poco::URI poco_uri(uri); + // Connect with URI. + Poco::MongoDB::Connection::SocketFactory socket_factory; + connection->connect(uri, socket_factory); + + Poco::URI poco_uri(connection.uri()); // Parse database from URI. This is required for correctness -- the // cursor is created using database name and collection name, so we have @@ -134,10 +138,6 @@ MongoDBDictionarySource::MongoDBDictionarySource( { user.resize(separator); } - - // Connect with URI. - Poco::MongoDB::Connection::SocketFactory socket_factory; - connection->connect(uri, socket_factory); } else { From 6b6daf7ae8bdaa98081bb5c2bff259f02522d13f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 17 Feb 2023 14:45:08 +0000 Subject: [PATCH 060/470] Automatic style fix --- tests/ci/workflow_approve_rerun_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index b563a9786c4..fb14dfd2258 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -123,7 +123,7 @@ TRUSTED_CONTRIBUTORS = { "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse Employee - "myrrc", # Mike Kot, DoubleCloud + "myrrc", # Mike Kot, DoubleCloud ] } From 046e2bb8eac0127f56fdaaedf54bee3cbb8f4fea Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 17 Feb 2023 16:50:20 +0000 Subject: [PATCH 061/470] fix issues with sparse columns --- src/Columns/ColumnSparse.cpp | 6 ++++-- .../01201_read_single_thread_in_order.sql | 2 +- .../01551_mergetree_read_in_order_spread.sql | 2 +- .../01710_projection_optimize_materialize.sql | 2 +- .../0_stateless/02008_materialize_column.sql | 16 ++++++++-------- .../0_stateless/02402_merge_engine_with_view.sql | 4 ++-- 6 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 9bc78ef115d..0e408fca467 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -450,11 +450,13 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, { const auto & rhs_sparse = assert_cast(rhs); PaddedPODArray nested_result; - values->compareColumn(rhs_sparse.getValuesColumn(), rhs_sparse.getValueIndex(rhs_row_num), + values->compareColumn( + rhs_sparse.getValuesColumn(), + rhs_sparse.getValueIndex(rhs_row_num), nullptr, nested_result, direction, nan_direction_hint); const auto & offsets_data = getOffsetsData(); - compare_results.resize_fill(_size, nested_result[0]); + std::fill(compare_results.begin(), compare_results.end(), nested_result[0]); for (size_t i = 0; i < offsets_data.size(); ++i) compare_results[offsets_data[i]] = nested_result[i + 1]; } diff --git a/tests/queries/0_stateless/01201_read_single_thread_in_order.sql b/tests/queries/0_stateless/01201_read_single_thread_in_order.sql index bfe03192891..71437b2e04c 100644 --- a/tests/queries/0_stateless/01201_read_single_thread_in_order.sql +++ b/tests/queries/0_stateless/01201_read_single_thread_in_order.sql @@ -6,7 +6,7 @@ CREATE TABLE t ) ENGINE = MergeTree ORDER BY number -SETTINGS index_granularity = 128; +SETTINGS index_granularity = 128, ratio_of_defaults_for_sparse_serialization = 1.0; SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; INSERT INTO t SELECT number FROM numbers(10000000); diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql index 1d21d861e20..41d4ff68824 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-random-merge-tree-settings DROP TABLE IF EXISTS data_01551; diff --git a/tests/queries/0_stateless/01710_projection_optimize_materialize.sql b/tests/queries/0_stateless/01710_projection_optimize_materialize.sql index d8251aabaf6..92d3ead828c 100644 --- a/tests/queries/0_stateless/01710_projection_optimize_materialize.sql +++ b/tests/queries/0_stateless/01710_projection_optimize_materialize.sql @@ -1,6 +1,6 @@ drop table if exists z; -create table z (pk Int64, d Date, id UInt64, c UInt64) Engine MergeTree partition by d order by pk ; +create table z (pk Int64, d Date, id UInt64, c UInt64) Engine MergeTree partition by d order by pk settings ratio_of_defaults_for_sparse_serialization = 1.0; insert into z select number, '2021-10-24', intDiv (number, 10000), 1 from numbers(1000000); optimize table z final; diff --git a/tests/queries/0_stateless/02008_materialize_column.sql b/tests/queries/0_stateless/02008_materialize_column.sql index 8a8eb2afe83..a78920d2525 100644 --- a/tests/queries/0_stateless/02008_materialize_column.sql +++ b/tests/queries/0_stateless/02008_materialize_column.sql @@ -8,33 +8,33 @@ INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20; ALTER TABLE tmp MATERIALIZE COLUMN x; -- { serverError 36 } ALTER TABLE tmp ADD COLUMN s String DEFAULT toString(x); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(arraySort(groupArray(x))), groupArray(s) FROM tmp; ALTER TABLE tmp MODIFY COLUMN s String DEFAULT toString(x+1); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp MATERIALIZE COLUMN s; ALTER TABLE tmp MODIFY COLUMN s String DEFAULT toString(x+2); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp MATERIALIZE COLUMN s; ALTER TABLE tmp MODIFY COLUMN s String DEFAULT toString(x+3); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp DROP COLUMN s; ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp MODIFY COLUMN s String MATERIALIZED toString(x+1); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp MATERIALIZE COLUMN s; ALTER TABLE tmp MODIFY COLUMN s String MATERIALIZED toString(x+2); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp MATERIALIZE COLUMN s; ALTER TABLE tmp MODIFY COLUMN s String MATERIALIZED toString(x+3); -SELECT groupArray(x), groupArray(s) FROM tmp; +SELECT arraySort(groupArray(x)), groupArray(s) FROM tmp; ALTER TABLE tmp DROP COLUMN s; DROP TABLE tmp; diff --git a/tests/queries/0_stateless/02402_merge_engine_with_view.sql b/tests/queries/0_stateless/02402_merge_engine_with_view.sql index 64822784845..ae9de1426e7 100644 --- a/tests/queries/0_stateless/02402_merge_engine_with_view.sql +++ b/tests/queries/0_stateless/02402_merge_engine_with_view.sql @@ -1,7 +1,7 @@ -- #40014 -CREATE TABLE m0 (id UInt64) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1; +CREATE TABLE m0 (id UInt64) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1, ratio_of_defaults_for_sparse_serialization = 1.0; INSERT INTO m0 SELECT number FROM numbers(10); -CREATE TABLE m1 (id UInt64, s String) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1; +CREATE TABLE m1 (id UInt64, s String) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1, ratio_of_defaults_for_sparse_serialization = 1.0; INSERT INTO m1 SELECT number, 'boo' FROM numbers(10); CREATE VIEW m1v AS SELECT id FROM m1; From 8da3594cd8463eb0974d79d3829f5ece93cc7477 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Feb 2023 17:42:56 +0000 Subject: [PATCH 062/470] Fix IPv4/IPv6 serialization/deserialization in binary formats --- src/Formats/CapnProtoUtils.cpp | 2 + src/Functions/array/FunctionArrayMapped.h | 1 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 37 ++++++++ .../Formats/Impl/AvroRowInputFormat.cpp | 11 +++ .../Formats/Impl/AvroRowOutputFormat.cpp | 14 +++ .../Impl/BSONEachRowRowInputFormat.cpp | 92 +++++++++++++------ .../Impl/BSONEachRowRowOutputFormat.cpp | 18 ++++ .../Formats/Impl/CHColumnToArrowColumn.cpp | 54 +++++++++++ .../Formats/Impl/CapnProtoRowInputFormat.cpp | 3 + .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 5 + .../Formats/Impl/MsgPackRowInputFormat.cpp | 11 +++ .../Formats/Impl/MsgPackRowOutputFormat.cpp | 12 +++ .../Formats/Impl/ORCBlockOutputFormat.cpp | 15 +++ .../02566_ipv4_ipv6_binary_formats.reference | 18 ++++ .../02566_ipv4_ipv6_binary_formats.sh | 45 +++++++++ .../format_schemas/02566_ipv4_ipv6.capnp | 6 ++ 16 files changed, 316 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference create mode 100755 tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh create mode 100644 tests/queries/0_stateless/format_schemas/02566_ipv4_ipv6.capnp diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index e5f619faff5..a557b762672 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -317,6 +317,7 @@ static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr case TypeIndex::UInt16: return capnp_type.isUInt16(); case TypeIndex::DateTime: [[fallthrough]]; + case TypeIndex::IPv4: [[fallthrough]]; case TypeIndex::UInt32: return capnp_type.isUInt32(); case TypeIndex::UInt64: @@ -355,6 +356,7 @@ static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr case TypeIndex::LowCardinality: return checkCapnProtoType(capnp_type, assert_cast(data_type.get())->getDictionaryType(), mode, error_message, column_name); case TypeIndex::FixedString: [[fallthrough]]; + case TypeIndex::IPv6: [[fallthrough]]; case TypeIndex::String: return capnp_type.isText() || capnp_type.isData(); default: diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 5092698d01c..5d758826f6a 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index ef25b5a332f..b7a914427a2 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -527,6 +528,38 @@ static std::shared_ptr getNestedArrowColumn(std::shared_ptr return std::make_shared(array_vector); } +static ColumnWithTypeAndName readIPv6ColumnFromBinaryData(std::shared_ptr & arrow_column, const String & column_name) +{ + size_t total_size = 0; + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) + { + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + const size_t chunk_length = chunk.length(); + + for (size_t i = 0; i != chunk_length; ++i) + { + /// If at least one value size is not 16 bytes, fallback to reading String column and further cast to IPv6. + if (chunk.value_length(i) != sizeof(IPv6)) + return readColumnWithStringData(arrow_column, column_name); + } + total_size += chunk_length; + } + + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & data = assert_cast(*internal_column).getData(); + data.reserve(total_size * sizeof(IPv6)); + + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) + { + auto & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr buffer = chunk.value_data(); + const auto * raw_data = reinterpret_cast(buffer->data()); + data.insert_assume_reserved(raw_data, raw_data + chunk.length()); + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + static ColumnWithTypeAndName readColumnFromArrowColumn( std::shared_ptr & arrow_column, const std::string & column_name, @@ -558,7 +591,11 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( { case arrow::Type::STRING: case arrow::Type::BINARY: + { + if (type_hint && isIPv6(type_hint)) + return readIPv6ColumnFromBinaryData(arrow_column, column_name); return readColumnWithStringData(arrow_column, column_name); + } case arrow::Type::FIXED_SIZE_BINARY: return readColumnWithFixedStringData(arrow_column, column_name); case arrow::Type::LARGE_BINARY: diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index eacd29e0db1..26f5a76776e 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -145,6 +145,9 @@ static void insertNumber(IColumn & column, WhichDataType type, T value) case TypeIndex::DateTime64: assert_cast &>(column).insertValue(static_cast(value)); break; + case TypeIndex::IPv4: + assert_cast(column).insertValue(IPv4(static_cast(value))); + break; default: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Type is not compatible with Avro"); } @@ -386,6 +389,14 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node column.insertData(reinterpret_cast(tmp_fixed.data()), tmp_fixed.size()); }; } + else if (target.isIPv6() && fixed_size == sizeof(IPv6)) + { + return [tmp_fixed = std::vector(fixed_size)](IColumn & column, avro::Decoder & decoder) mutable + { + decoder.decodeFixed(tmp_fixed.size(), tmp_fixed); + column.insertData(reinterpret_cast(tmp_fixed.data()), tmp_fixed.size()); + }; + } break; } case avro::AVRO_SYMBOLIC: diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 8483a91df62..1ca56a1c5cc 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -127,6 +127,11 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF { encoder.encodeInt(assert_cast(column).getElement(row_num)); }}; + case TypeIndex::IPv4: + return {avro::IntSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) + { + encoder.encodeInt(assert_cast(column).getElement(row_num)); + }}; case TypeIndex::Int32: return {avro::IntSchema(), [](const IColumn & column, size_t row_num, avro::Encoder & encoder) { @@ -205,6 +210,15 @@ AvroSerializer::SchemaWithSerializeFn AvroSerializer::createSchemaWithSerializeF encoder.encodeFixed(reinterpret_cast(s.data()), s.size()); }}; } + case TypeIndex::IPv6: + { + auto schema = avro::FixedSchema(sizeof(IPv6), "ipv6_" + toString(type_name_increment)); + return {schema, [](const IColumn & column, size_t row_num, avro::Encoder & encoder) + { + const std::string_view & s = assert_cast(column).getDataAt(row_num).toView(); + encoder.encodeFixed(reinterpret_cast(s.data()), s.size()); + }}; + } case TypeIndex::Enum8: { auto schema = avro::EnumSchema("enum8_" + toString(type_name_increment)); /// type names must be different for different types. diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index c4ffce2bc65..e0064c282b4 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -151,6 +151,17 @@ static void readAndInsertInteger(ReadBuffer & in, IColumn & column, const DataTy } } +static void readAndInsertIPv4(ReadBuffer & in, IColumn & column, BSONType bson_type) +{ + /// We expect BSON type Int32 as IPv4 value. + if (bson_type != BSONType::INT32) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert BSON Int32 into column with type IPv4"); + + UInt32 value; + readBinary(value, in); + assert_cast(column).insertValue(IPv4(value)); +} + template static void readAndInsertDouble(ReadBuffer & in, IColumn & column, const DataTypePtr & data_type, BSONType bson_type) { @@ -296,37 +307,52 @@ static void readAndInsertString(ReadBuffer & in, IColumn & column, BSONType bson } } +static void readAndInsertIPv6(ReadBuffer & in, IColumn & column, BSONType bson_type) +{ + if (bson_type != BSONType::BINARY) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert BSON {} into IPv6 column", getBSONTypeName(bson_type)); + + auto size = readBSONSize(in); + auto subtype = getBSONBinarySubtype(readBSONType(in)); + if (subtype != BSONBinarySubtype::BINARY) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert BSON Binary subtype {} into IPv6 column", getBSONBinarySubtypeName(subtype)); + + if (size != sizeof(IPv6)) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Cannot parse value of type IPv6, size of binary data is not equal to the binary size of IPv6 value: {} != {}", + size, + sizeof(IPv6)); + + IPv6 value; + readBinary(value, in); + assert_cast(column).insertValue(value); +} + + static void readAndInsertUUID(ReadBuffer & in, IColumn & column, BSONType bson_type) { - if (bson_type == BSONType::BINARY) - { - auto size = readBSONSize(in); - auto subtype = getBSONBinarySubtype(readBSONType(in)); - if (subtype == BSONBinarySubtype::UUID || subtype == BSONBinarySubtype::UUID_OLD) - { - if (size != sizeof(UUID)) - throw Exception( - ErrorCodes::INCORRECT_DATA, - "Cannot parse value of type UUID, size of binary data is not equal to the binary size of UUID value: {} != {}", - size, - sizeof(UUID)); - - UUID value; - readBinary(value, in); - assert_cast(column).insertValue(value); - } - else - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Cannot insert BSON Binary subtype {} into UUID column", - getBSONBinarySubtypeName(subtype)); - } - } - else - { + if (bson_type != BSONType::BINARY) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert BSON {} into UUID column", getBSONTypeName(bson_type)); - } + + auto size = readBSONSize(in); + auto subtype = getBSONBinarySubtype(readBSONType(in)); + if (subtype != BSONBinarySubtype::UUID && subtype != BSONBinarySubtype::UUID_OLD) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Cannot insert BSON Binary subtype {} into UUID column", + getBSONBinarySubtypeName(subtype)); + + if (size != sizeof(UUID)) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Cannot parse value of type UUID, size of binary data is not equal to the binary size of UUID value: {} != {}", + size, + sizeof(UUID)); + + UUID value; + readBinary(value, in); + assert_cast(column).insertValue(value); } void BSONEachRowRowInputFormat::readArray(IColumn & column, const DataTypePtr & data_type, BSONType bson_type) @@ -591,6 +617,16 @@ bool BSONEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr & readAndInsertString(*in, column, bson_type); return true; } + case TypeIndex::IPv4: + { + readAndInsertIPv4(*in, column, bson_type); + return true; + } + case TypeIndex::IPv6: + { + readAndInsertIPv6(*in, column, bson_type); + return true; + } case TypeIndex::UUID: { readAndInsertUUID(*in, column, bson_type); diff --git a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp index 211021b0d78..95dd3079687 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowOutputFormat.cpp @@ -124,6 +124,7 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co case TypeIndex::Date: [[fallthrough]]; case TypeIndex::Date32: [[fallthrough]]; case TypeIndex::Decimal32: [[fallthrough]]; + case TypeIndex::IPv4: [[fallthrough]]; case TypeIndex::Int32: { return size + sizeof(Int32); @@ -168,6 +169,10 @@ size_t BSONEachRowRowOutputFormat::countBSONFieldSize(const IColumn & column, co const auto & string_column = assert_cast(column); return size + sizeof(BSONSizeT) + string_column.getN() + 1; // Size of data + data + \0 or BSON subtype (in case of BSON binary) } + case TypeIndex::IPv6: + { + return size + sizeof(BSONSizeT) + 1 + sizeof(IPv6); // Size of data + BSON binary subtype + 16 bytes of value + } case TypeIndex::UUID: { return size + sizeof(BSONSizeT) + 1 + sizeof(UUID); // Size of data + BSON binary subtype + 16 bytes of value @@ -371,6 +376,19 @@ void BSONEachRowRowOutputFormat::serializeField(const IColumn & column, const Da writeBSONString(column, row_num, name, out, settings.bson.output_string_as_string); break; } + case TypeIndex::IPv4: + { + writeBSONNumber(BSONType::INT32, column, row_num, name, out); + break; + } + case TypeIndex::IPv6: + { + writeBSONTypeAndKeyName(BSONType::BINARY, name, out); + writeBSONSize(sizeof(IPv6), out); + writeBSONType(BSONBinarySubtype::BINARY, out); + writeBinary(assert_cast(column).getElement(row_num), out); + break; + } case TypeIndex::UUID: { writeBSONTypeAndKeyName(BSONType::BINARY, name, out); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index aef54516627..8698b343eb3 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -434,6 +434,46 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } + static void fillArrowArrayWithIPv6ColumnData( + ColumnPtr write_column, + const PaddedPODArray * null_bytemap, + const String & format_name, + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) + { + const auto & internal_column = assert_cast(*write_column); + const auto & internal_data = internal_column.getData(); + size_t fixed_length = sizeof(IPv6); + arrow::FixedSizeBinaryBuilder & builder = assert_cast(*array_builder); + arrow::Status status; + + PaddedPODArray arrow_null_bytemap = revertNullByteMap(null_bytemap, start, end); + const UInt8 * arrow_null_bytemap_raw_ptr = arrow_null_bytemap.empty() ? nullptr : arrow_null_bytemap.data(); + + const uint8_t * data_start = reinterpret_cast(internal_data.data()) + start * fixed_length; + status = builder.AppendValues(data_start, end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); + checkStatus(status, write_column->getName(), format_name); + } + + static void fillArrowArrayWithIPv4ColumnData( + ColumnPtr write_column, + const PaddedPODArray * null_bytemap, + const String & format_name, + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) + { + const auto & internal_data = assert_cast(*write_column).getData(); + auto & builder = assert_cast(*array_builder); + arrow::Status status; + + PaddedPODArray arrow_null_bytemap = revertNullByteMap(null_bytemap, start, end); + const UInt8 * arrow_null_bytemap_raw_ptr = arrow_null_bytemap.empty() ? nullptr : arrow_null_bytemap.data(); + status = builder.AppendValues(&(internal_data.data() + start)->toUnderType(), end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); + checkStatus(status, write_column->getName(), format_name); + } + static void fillArrowArrayWithDateColumnData( ColumnPtr write_column, const PaddedPODArray * null_bytemap, @@ -541,6 +581,14 @@ namespace DB else fillArrowArrayWithStringColumnData(column, null_bytemap, format_name, array_builder, start, end); } + else if (isIPv6(column_type)) + { + fillArrowArrayWithIPv6ColumnData(column, null_bytemap, format_name, array_builder, start, end); + } + else if (isIPv4(column_type)) + { + fillArrowArrayWithIPv4ColumnData(column, null_bytemap, format_name, array_builder, start, end); + } else if (isDate(column_type)) { fillArrowArrayWithDateColumnData(column, null_bytemap, format_name, array_builder, start, end); @@ -781,6 +829,12 @@ namespace DB if (isBool(column_type)) return arrow::boolean(); + if (isIPv6(column_type)) + return arrow::fixed_size_binary(sizeof(IPv6)); + + if (isIPv4(column_type)) + return arrow::uint32(); + const std::string type_name = column_type->getFamilyName(); if (const auto * arrow_type_it = std::find_if( internal_type_to_arrow_type.begin(), diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 58ace9cfca5..9f37bcc3219 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -128,6 +128,9 @@ static void insertUnsignedInteger(IColumn & column, const DataTypePtr & column_t case TypeIndex::UInt64: assert_cast(column).insertValue(value); break; + case TypeIndex::IPv4: + assert_cast(column).insertValue(IPv4(static_cast(value))); + break; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not an unsigned integer."); } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index dcbd5db5f9b..c0f61bbd586 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -111,7 +111,12 @@ static std::optional convertToDynamicValue( case capnp::DynamicValue::Type::INT: return capnp::DynamicValue::Reader(column->getInt(row_num)); case capnp::DynamicValue::Type::UINT: + { + /// IPv4 column doesn't support getUInt method. + if (isIPv4(data_type)) + return capnp::DynamicValue::Reader(assert_cast(column.get())->getElement(row_num)); return capnp::DynamicValue::Reader(column->getUInt(row_num)); + } case capnp::DynamicValue::Type::BOOL: return capnp::DynamicValue::Reader(column->getBool(row_num)); case capnp::DynamicValue::Type::FLOAT: diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index f337eedbb05..5605b87f3a9 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -162,6 +162,11 @@ static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value) assert_cast(column).insertValue(value); break; } + case TypeIndex::IPv4: + { + assert_cast(column).insertValue(IPv4(static_cast(value))); + break; + } default: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack integer into column with type {}.", type->getName()); } @@ -190,6 +195,12 @@ static void insertString(IColumn & column, DataTypePtr type, const char * value, return; } + if (isIPv6(type) && bin) + { + assert_cast(column).insertData(value, size); + return; + } + if (!isStringOrFixedString(type)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack string into column with type {}.", type->getName()); diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index a1ed45ec40f..07951d42bc6 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -56,6 +56,11 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_uint32(assert_cast(column).getElement(row_num)); return; } + case TypeIndex::IPv4: + { + packer.pack_uint32(assert_cast(column).getElement(row_num)); + return; + } case TypeIndex::UInt64: { packer.pack_uint64(assert_cast(column).getElement(row_num)); @@ -110,6 +115,13 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr packer.pack_bin_body(string.data(), static_cast(string.size())); return; } + case TypeIndex::IPv6: + { + const std::string_view & data = assert_cast(column).getDataAt(row_num).toView(); + packer.pack_bin(static_cast(data.size())); + packer.pack_bin_body(data.data(), static_cast(data.size())); + return; + } case TypeIndex::Array: { auto nested_type = assert_cast(*data_type).getNestedType(); diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 42c3e178436..86d9560beb9 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -75,6 +75,7 @@ std::unique_ptr ORCBlockOutputFormat::getORCType(const DataTypePtr & return orc::createPrimitiveType(orc::TypeKind::SHORT); } case TypeIndex::UInt32: [[fallthrough]]; + case TypeIndex::IPv4: [[fallthrough]]; case TypeIndex::Int32: { return orc::createPrimitiveType(orc::TypeKind::INT); @@ -109,6 +110,10 @@ std::unique_ptr ORCBlockOutputFormat::getORCType(const DataTypePtr & return orc::createPrimitiveType(orc::TypeKind::STRING); return orc::createPrimitiveType(orc::TypeKind::BINARY); } + case TypeIndex::IPv6: + { + return orc::createPrimitiveType(orc::TypeKind::BINARY); + } case TypeIndex::Nullable: { return getORCType(removeNullable(type)); @@ -309,6 +314,11 @@ void ORCBlockOutputFormat::writeColumn( writeNumbers(orc_column, column, null_bytemap, [](const UInt32 & value){ return value; }); break; } + case TypeIndex::IPv4: + { + writeNumbers(orc_column, column, null_bytemap, [](const IPv4 & value){ return value.toUnderType(); }); + break; + } case TypeIndex::Int64: { writeNumbers(orc_column, column, null_bytemap, [](const Int64 & value){ return value; }); @@ -339,6 +349,11 @@ void ORCBlockOutputFormat::writeColumn( writeStrings(orc_column, column, null_bytemap); break; } + case TypeIndex::IPv6: + { + writeStrings(orc_column, column, null_bytemap); + break; + } case TypeIndex::DateTime: { writeDateTimes( diff --git a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference new file mode 100644 index 00000000000..e228d911715 --- /dev/null +++ b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference @@ -0,0 +1,18 @@ +CapnProto +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +Avro +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +Arrow +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +Parquet +ipv6 Nullable(FixedString(16)) +ipv4 Nullable(Int64) +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +ORC +ipv6 Nullable(String) +ipv4 Nullable(Int32) +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +BSONEachRow +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 +MsgPack +2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 diff --git a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh new file mode 100755 index 00000000000..e84c06027e8 --- /dev/null +++ b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh @@ -0,0 +1,45 @@ +#!/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 "CapnProto" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format CapnProto settings format_schema='format_schemas/02566_ipv4_ipv6:Message'" > 02566_ipv4_ipv6_data.capnp +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.capnp, auto, 'ipv6 IPv6, ipv4 IPv4') settings format_schema='format_schemas/02566_ipv4_ipv6:Message'" +rm 02566_ipv4_ipv6_data.capnp + +echo "Avro" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format Avro" > 02566_ipv4_ipv6_data.avro +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.avro, auto, 'ipv6 IPv6, ipv4 IPv4')" +rm 02566_ipv4_ipv6_data.avro + +echo "Arrow" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format Arrow" > 02566_ipv4_ipv6_data.arrow +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.arrow, auto, 'ipv6 IPv6, ipv4 IPv4')" +rm 02566_ipv4_ipv6_data.arrow + +echo "Parquet" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format Parquet" > 02566_ipv4_ipv6_data.parquet +${CLICKHOUSE_LOCAL} -q "desc file(02566_ipv4_ipv6_data.parquet)" +${CLICKHOUSE_LOCAL} -q "select ipv6, toIPv4(ipv4) from file(02566_ipv4_ipv6_data.parquet, auto, 'ipv6 IPv6, ipv4 UInt32')" +rm 02566_ipv4_ipv6_data.parquet + +echo "ORC" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format ORC" > 02566_ipv4_ipv6_data.orc +${CLICKHOUSE_LOCAL} -q "desc file(02566_ipv4_ipv6_data.orc)" +${CLICKHOUSE_LOCAL} -q "select ipv6, toIPv4(ipv4) from file(02566_ipv4_ipv6_data.orc, auto, 'ipv6 IPv6, ipv4 UInt32')" +rm 02566_ipv4_ipv6_data.orc + +echo "BSONEachRow" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format BSONEachRow" > 02566_ipv4_ipv6_data.bson +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.bson, auto, 'ipv6 IPv6, ipv4 IPv4')" +rm 02566_ipv4_ipv6_data.bson + +echo "MsgPack" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format MsgPack" > 02566_ipv4_ipv6_data.msgpack +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.msgpack, auto, 'ipv6 IPv6, ipv4 IPv4')" +rm 02566_ipv4_ipv6_data.msgpack + + diff --git a/tests/queries/0_stateless/format_schemas/02566_ipv4_ipv6.capnp b/tests/queries/0_stateless/format_schemas/02566_ipv4_ipv6.capnp new file mode 100644 index 00000000000..f999043e2d2 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02566_ipv4_ipv6.capnp @@ -0,0 +1,6 @@ +@0xb6ecde1cd54a101d; + +struct Message { + ipv4 @0 :UInt32; + ipv6 @1 :Data; +} From e37f6b545775783bac0a21e8e951c01586ab1778 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 20 Feb 2023 19:50:25 +0000 Subject: [PATCH 063/470] Update docs --- docs/en/interfaces/formats.md | 209 ++++++++++++++++++---------------- 1 file changed, 111 insertions(+), 98 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index e94c6377ae9..f1935131491 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1232,50 +1232,52 @@ Each row is formatted as a single document and each column is formatted as a sin For output it uses the following correspondence between ClickHouse types and BSON types: -| ClickHouse type | BSON Type | -|-----------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------| -| [Bool](/docs/en/sql-reference/data-types/boolean.md) | `\x08` boolean | -| [Int8/UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | -| [Int16UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | -| [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | -| [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 | -| [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 | -| [Float32/Float64](/docs/en/sql-reference/data-types/float.md) | `\x01` double | -| [Date](/docs/en/sql-reference/data-types/date.md)/[Date32](/docs/en/sql-reference/data-types/date32.md) | `\x10` int32 | -| [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `\x12` int64 | -| [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `\x09` datetime | -| [Decimal32](/docs/en/sql-reference/data-types/decimal.md) | `\x10` int32 | -| [Decimal64](/docs/en/sql-reference/data-types/decimal.md) | `\x12` int64 | -| [Decimal128](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 16 | -| [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 32 | -| [Int128/UInt128](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 16 | -| [Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 32 | +| ClickHouse type | BSON Type | +|-----------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------| +| [Bool](/docs/en/sql-reference/data-types/boolean.md) | `\x08` boolean | +| [Int8/UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | +| [Int16UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | +| [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `\x10` int32 | +| [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 | +| [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `\x12` int64 | +| [Float32/Float64](/docs/en/sql-reference/data-types/float.md) | `\x01` double | +| [Date](/docs/en/sql-reference/data-types/date.md)/[Date32](/docs/en/sql-reference/data-types/date32.md) | `\x10` int32 | +| [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `\x12` int64 | +| [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `\x09` datetime | +| [Decimal32](/docs/en/sql-reference/data-types/decimal.md) | `\x10` int32 | +| [Decimal64](/docs/en/sql-reference/data-types/decimal.md) | `\x12` int64 | +| [Decimal128](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 16 | +| [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `\x05` binary, `\x00` binary subtype, size = 32 | +| [Int128/UInt128](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 16 | +| [Int256/UInt256](/docs/en/sql-reference/data-types/int-uint.md) | `\x05` binary, `\x00` binary subtype, size = 32 | | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `\x05` binary, `\x00` binary subtype or \x02 string if setting output_format_bson_string_as_string is enabled | -| [UUID](/docs/en/sql-reference/data-types/uuid.md) | `\x05` binary, `\x04` uuid subtype, size = 16 | -| [Array](/docs/en/sql-reference/data-types/array.md) | `\x04` array | -| [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x04` array | -| [Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x03` document | -| [Map](/docs/en/sql-reference/data-types/map.md) (with String keys) | `\x03` document | +| [UUID](/docs/en/sql-reference/data-types/uuid.md) | `\x05` binary, `\x04` uuid subtype, size = 16 | +| [Array](/docs/en/sql-reference/data-types/array.md) | `\x04` array | +| [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x04` array | +| [Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | `\x03` document | +| [Map](/docs/en/sql-reference/data-types/map.md) (with String keys) | `\x03` document | +| [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `\x10` int32 | +| [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `\x05` binary, `\x00` binary subtype | For input it uses the following correspondence between BSON types and ClickHouse types: -| BSON Type | ClickHouse Type | -|------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `\x01` double | [Float32/Float64](/docs/en/sql-reference/data-types/float.md) | -| `\x02` string | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x03` document | [Map](/docs/en/sql-reference/data-types/map.md)/[Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | -| `\x04` array | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) | -| `\x05` binary, `\x00` binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x05` binary, `\x02` old binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x05` binary, `\x03` old uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | -| `\x05` binary, `\x04` uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | -| `\x07` ObjectId | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x08` boolean | [Bool](/docs/en/sql-reference/data-types/boolean.md) | -| `\x09` datetime | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | -| `\x0A` null value | [NULL](/docs/en/sql-reference/data-types/nullable.md) | -| `\x0D` JavaScript code | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x0E` symbol | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | -| `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md) | +| BSON Type | ClickHouse Type | +|------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `\x01` double | [Float32/Float64](/docs/en/sql-reference/data-types/float.md) | +| `\x02` string | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| `\x03` document | [Map](/docs/en/sql-reference/data-types/map.md)/[Named Tuple](/docs/en/sql-reference/data-types/tuple.md) | +| `\x04` array | [Array](/docs/en/sql-reference/data-types/array.md)/[Tuple](/docs/en/sql-reference/data-types/tuple.md) | +| `\x05` binary, `\x00` binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md)/[IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | +| `\x05` binary, `\x02` old binary subtype | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| `\x05` binary, `\x03` old uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | +| `\x05` binary, `\x04` uuid subtype | [UUID](/docs/en/sql-reference/data-types/uuid.md) | +| `\x07` ObjectId | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| `\x08` boolean | [Bool](/docs/en/sql-reference/data-types/boolean.md) | +| `\x09` datetime | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | +| `\x0A` null value | [NULL](/docs/en/sql-reference/data-types/nullable.md) | +| `\x0D` JavaScript code | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| `\x0E` symbol | [String](/docs/en/sql-reference/data-types/string.md)/[FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| `\x10` int32 | [Int32/UInt32](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal32](/docs/en/sql-reference/data-types/decimal.md)/[IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | | `\x12` int64 | [Int64/UInt64](/docs/en/sql-reference/data-types/int-uint.md)/[Decimal64](/docs/en/sql-reference/data-types/decimal.md)/[DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | Other BSON types are not supported. Also, it performs conversion between different integer types (for example, you can insert BSON int32 value into ClickHouse UInt8). @@ -1608,23 +1610,25 @@ See also [Format Schema](#formatschema). The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| CapnProto data type (`INSERT`) | ClickHouse data type | CapnProto data type (`SELECT`) | -|--------------------------------|-----------------------------------------------------------|--------------------------------| -| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | -| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | -| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) | `UINT16` | -| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | -| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | -| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | -| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | -| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `INT64` | -| `FLOAT32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` | -| `FLOAT64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` | -| `TEXT, DATA` | [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `TEXT, DATA` | -| `union(T, Void), union(Void, T)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` | -| `ENUM` | [Enum(8\|16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` | -| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | -| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | +| CapnProto data type (`INSERT`) | ClickHouse data type | CapnProto data type (`SELECT`) | +|----------------------------------|------------------------------------------------------------------------------------------------------------------------|------------------------------| +| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | +| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | +| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) | `UINT16` | +| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | +| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md), [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | +| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | +| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | +| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `INT64` | +| `FLOAT32` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` | +| `FLOAT64` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` | +| `TEXT, DATA` | [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `TEXT, DATA` | +| `union(T, Void), union(Void, T)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(T, Void), union(Void, T)` | +| `ENUM` | [Enum(8\ |16)](/docs/en/sql-reference/data-types/enum.md) | `ENUM` | +| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | +| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | +| `DATA` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `DATA` | For working with `Enum` in CapnProto format use the [format_capn_proto_enum_comparising_mode](/docs/en/operations/settings/settings-formats.md/#format_capn_proto_enum_comparising_mode) setting. @@ -1804,21 +1808,23 @@ ClickHouse Avro format supports reading and writing [Avro data files](https://av The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` | -|---------------------------------------------|----------------------------------------------------------------------------------------------------|------------------------------| -| `boolean`, `int`, `long`, `float`, `double` | [Int(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md) | `int` | -| `boolean`, `int`, `long`, `float`, `double` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `long` | -| `boolean`, `int`, `long`, `float`, `double` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float` | -| `boolean`, `int`, `long`, `float`, `double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `double` | -| `bytes`, `string`, `fixed`, `enum` | [String](/docs/en/sql-reference/data-types/string.md) | `bytes` or `string` \* | -| `bytes`, `string`, `fixed` | [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) | `fixed(N)` | -| `enum` | [Enum(8\|16)](/docs/en/sql-reference/data-types/enum.md) | `enum` | -| `array(T)` | [Array(T)](/docs/en/sql-reference/data-types/array.md) | `array(T)` | -| `union(null, T)`, `union(T, null)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(null, T)` | -| `null` | [Nullable(Nothing)](/docs/en/sql-reference/data-types/special-data-types/nothing.md) | `null` | -| `int (date)` \** | [Date](/docs/en/sql-reference/data-types/date.md) | `int (date)` \** | -| `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* | -| `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* | +| Avro data type `INSERT` | ClickHouse data type | Avro data type `SELECT` | +|---------------------------------------------|-----------------------------------------------------------------------------------------------------------------|-------------------------------------------------| +| `boolean`, `int`, `long`, `float`, `double` | [Int(8\ | 16\ |32)](/docs/en/sql-reference/data-types/int-uint.md), [UInt(8\|16\|32)](/docs/en/sql-reference/data-types/int-uint.md) | `int` | +| `boolean`, `int`, `long`, `float`, `double` | [Int64](/docs/en/sql-reference/data-types/int-uint.md), [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `long` | +| `boolean`, `int`, `long`, `float`, `double` | [Float32](/docs/en/sql-reference/data-types/float.md) | `float` | +| `boolean`, `int`, `long`, `float`, `double` | [Float64](/docs/en/sql-reference/data-types/float.md) | `double` | +| `bytes`, `string`, `fixed`, `enum` | [String](/docs/en/sql-reference/data-types/string.md) | `bytes` or `string` \* | +| `bytes`, `string`, `fixed` | [FixedString(N)](/docs/en/sql-reference/data-types/fixedstring.md) | `fixed(N)` | +| `enum` | [Enum(8\ | 16)](/docs/en/sql-reference/data-types/enum.md) | `enum` | +| `array(T)` | [Array(T)](/docs/en/sql-reference/data-types/array.md) | `array(T)` | +| `union(null, T)`, `union(T, null)` | [Nullable(T)](/docs/en/sql-reference/data-types/date.md) | `union(null, T)` | +| `null` | [Nullable(Nothing)](/docs/en/sql-reference/data-types/special-data-types/nothing.md) | `null` | +| `int (date)` \** | [Date](/docs/en/sql-reference/data-types/date.md) | `int (date)` \** | +| `long (timestamp-millis)` \** | [DateTime64(3)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-millis)` \* | +| `long (timestamp-micros)` \** | [DateTime64(6)](/docs/en/sql-reference/data-types/datetime.md) | `long (timestamp-micros)` \* | +| `int` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `int` | +| `fixed(16)` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `fixed(16)` | \* `bytes` is default, controlled by [output_format_avro_string_column_pattern](/docs/en/operations/settings/settings-formats.md/#output_format_avro_string_column_pattern) \** [Avro logical types](https://avro.apache.org/docs/current/spec.html#Logical+Types) @@ -1918,28 +1924,30 @@ Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) | -|-----------------------------------------------|-----------------------------------------------------------------|------------------------------| -| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | -| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | -| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | -| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | -| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | -| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | -| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | -| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | -| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | -| `FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT` | -| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `DOUBLE` | -| `DATE` | [Date32](/docs/en/sql-reference/data-types/date.md) | `DATE` | -| `TIME (ms)` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | -| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `TIMESTAMP` | -| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | -| `STRING`, `BINARY`, `FIXED_LENGTH_BYTE_ARRAY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `FIXED_LENGTH_BYTE_ARRAY` | -| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | -| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | -| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | -| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | +| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) | +|----------------------------------------------------|-----------------------------------------------------------------|------------------------------| +| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | +| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | +| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | +| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | +| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | +| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | +| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | +| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | +| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | +| `FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT` | +| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `DOUBLE` | +| `DATE` | [Date32](/docs/en/sql-reference/data-types/date.md) | `DATE` | +| `TIME (ms)` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | +| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `TIMESTAMP` | +| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | +| `STRING`, `BINARY`, `FIXED_LENGTH_BYTE_ARRAY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `FIXED_LENGTH_BYTE_ARRAY` | +| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | +| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | +| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | +| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | +| `FIXED_LENGTH_BYTE_ARRAY` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `FIXED_LENGTH_BYTE_ARRAY` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -2006,6 +2014,8 @@ The table below shows supported data types and how they match ClickHouse [data t | `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | | `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | | `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | +| `UINT32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `UINT32` | +| `FIXED_SIZE_BINARY`, `BINARY` | [IPv6](/docs/en/sql-reference/data-types/domains/ipv6.md) | `FIXED_SIZE_BINARY` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -2053,8 +2063,8 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| ORC data type (`INSERT`) | ClickHouse data type | ORC data type (`SELECT`) | -|---------------------------------------|---------------------------------------------------------|--------------------------| +| ORC data type (`INSERT`) | ClickHouse data type | ORC data type (`SELECT`) | +|---------------------------------------|---------------------------------------------------------------|--------------------------| | `Boolean` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `Boolean` | | `Tinyint` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `Tinyint` | | `Smallint` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `Smallint` | @@ -2069,6 +2079,7 @@ The table below shows supported data types and how they match ClickHouse [data t | `List` | [Array](/docs/en/sql-reference/data-types/array.md) | `List` | | `Struct` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `Struct` | | `Map` | [Map](/docs/en/sql-reference/data-types/map.md) | `Map` | +| `-` | [IPv4](/docs/en/sql-reference/data-types/int-uint.md) | `Int` | Other types are not supported. @@ -2263,8 +2274,8 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data ### Data Types Matching {#data-types-matching-msgpack} -| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) | -|--------------------------------------------------------------------|-----------------------------------------------------------|------------------------------------| +| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) | +|--------------------------------------------------------------------|-----------------------------------------------------------------|------------------------------------| | `uint N`, `positive fixint` | [UIntN](/docs/en/sql-reference/data-types/int-uint.md) | `uint N` | | `int N`, `negative fixint` | [IntN](/docs/en/sql-reference/data-types/int-uint.md) | `int N` | | `bool` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `uint 8` | @@ -2277,6 +2288,8 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data | `uint 64` | [DateTime64](/docs/en/sql-reference/data-types/datetime.md) | `uint 64` | | `fixarray`, `array 16`, `array 32` | [Array](/docs/en/sql-reference/data-types/array.md) | `fixarray`, `array 16`, `array 32` | | `fixmap`, `map 16`, `map 32` | [Map](/docs/en/sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` | +| `uint 32` | [IPv4](/docs/en/sql-reference/data-types/domains/ipv4.md) | `uint 32` | +| `bin 8` | [String](/docs/en/sql-reference/data-types/string.md) | `bin 8` | Example: From bf020298100a51ed57f10b0d1f9e2eceedd802bb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 Feb 2023 12:19:58 +0000 Subject: [PATCH 064/470] Add pass for transforming or equality chain to single IN --- .../Passes/OrEqualityChainToInPass.cpp | 117 ++++++++++++++++++ src/Analyzer/Passes/OrEqualityChainToInPass.h | 18 +++ src/Analyzer/QueryTreePassManager.cpp | 4 +- 3 files changed, 138 insertions(+), 1 deletion(-) create mode 100644 src/Analyzer/Passes/OrEqualityChainToInPass.cpp create mode 100644 src/Analyzer/Passes/OrEqualityChainToInPass.h diff --git a/src/Analyzer/Passes/OrEqualityChainToInPass.cpp b/src/Analyzer/Passes/OrEqualityChainToInPass.cpp new file mode 100644 index 00000000000..212e8252ce4 --- /dev/null +++ b/src/Analyzer/Passes/OrEqualityChainToInPass.cpp @@ -0,0 +1,117 @@ +#include + +#include + +#include +#include +#include +#include + +namespace DB +{ + +class OrEqualityChainToInVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + + explicit OrEqualityChainToInVisitor(ContextPtr context) + : Base(std::move(context)) + {} + + + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + + if (!function_node || function_node->getFunctionName() != "or") + return; + + QueryTreeNodes or_operands; + + QueryTreeNodePtrWithHashMap node_to_equals_functions; + + for (const auto & argument : function_node->getArguments()) + { + auto * argument_function = argument->as(); + if (!argument_function) + { + or_operands.push_back(argument); + continue; + } + + /// collect all equality checks (x = value) + if (argument_function->getFunctionName() != "equals") + { + or_operands.push_back(argument); + continue; + } + + const auto & equals_arguments = argument_function->getArguments().getNodes(); + const auto & lhs = equals_arguments[0]; + + const auto * rhs_literal = equals_arguments[1]->as(); + if (!rhs_literal) + { + or_operands.push_back(argument); + continue; + } + + node_to_equals_functions[lhs].push_back(argument); + } + + auto in_function_resolver = FunctionFactory::instance().get("in", getContext()); + + for (auto & [lhs, equals_functions] : node_to_equals_functions) + { + const auto & settings = getSettings(); + if (equals_functions.size() < settings.optimize_min_equality_disjunction_chain_length && !lhs.node->getResultType()->lowCardinality()) + { + std::move(equals_functions.begin(), equals_functions.end(), std::back_inserter(or_operands)); + continue; + } + + Tuple args; + args.reserve(equals_functions.size()); + /// first we create tuple from RHS of equals functions + for (const auto & equals : equals_functions) + { + const auto * equals_function = equals->as(); + assert(equals_function && equals_function->getFunctionName() == "equals"); + const auto * rhs_literal = equals_function->getArguments().getNodes()[1]->as(); + assert(rhs_literal); + args.push_back(rhs_literal->getValue()); + } + + auto rhs_node = std::make_shared(std::move(args)); + + auto in_function = std::make_shared("in"); + + QueryTreeNodes in_arguments; + in_arguments.reserve(2); + in_arguments.push_back(lhs.node); + in_arguments.push_back(std::move(rhs_node)); + + in_function->getArguments().getNodes() = std::move(in_arguments); + in_function->resolveAsFunction(in_function_resolver); + + or_operands.push_back(std::move(in_function)); + } + + if (or_operands.size() == 1) + or_operands.push_back(std::make_shared(static_cast(0))); + + auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); + function_node->getArguments().getNodes() = std::move(or_operands); + function_node->resolveAsFunction(or_function_resolver); + } +}; + + +void OrEqualityChainToInPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + OrEqualityChainToInVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/OrEqualityChainToInPass.h b/src/Analyzer/Passes/OrEqualityChainToInPass.h new file mode 100644 index 00000000000..86c9c9efd57 --- /dev/null +++ b/src/Analyzer/Passes/OrEqualityChainToInPass.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +namespace DB +{ + +class OrEqualityChainToInPass final : public IQueryTreePass +{ +public: + String getName() override { return "OrEqualityChainToIn"; } + + String getDescription() override { return "Transform all the 'or's with equality check to a single IN function"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 588457f90f7..e4bc919e051 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace DB { @@ -145,7 +146,6 @@ private: /** ClickHouse query tree pass manager. * * TODO: Support _shard_num into shardNum() rewriting. - * TODO: Support logical expressions optimizer. * TODO: Support setting convert_query_to_cnf. * TODO: Support setting optimize_using_constraints. * TODO: Support setting optimize_substitute_columns. @@ -262,6 +262,8 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } From bdad6a480ed157e0727d449de413a7dac2590dab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 Feb 2023 12:56:09 +0000 Subject: [PATCH 065/470] Add tests --- src/Analyzer/Passes/OrEqualityChainToInPass.h | 20 + ...00621_regression_for_in_operator.reference | 44 ++ .../00621_regression_for_in_operator.sql | 4 + .../00736_disjunction_optimisation.reference | 426 ++++++++++++++++-- .../00736_disjunction_optimisation.sql | 24 + ...ssions_optimizer_low_cardinality.reference | 42 ++ ..._expressions_optimizer_low_cardinality.sql | 2 + 7 files changed, 526 insertions(+), 36 deletions(-) diff --git a/src/Analyzer/Passes/OrEqualityChainToInPass.h b/src/Analyzer/Passes/OrEqualityChainToInPass.h index 86c9c9efd57..2a9d6818a4f 100644 --- a/src/Analyzer/Passes/OrEqualityChainToInPass.h +++ b/src/Analyzer/Passes/OrEqualityChainToInPass.h @@ -5,6 +5,26 @@ namespace DB { +/** + * This pass replaces chains of equality functions inside an OR with a single IN operator. + * The replacement is done if: + * - rhs of the equality function is a literal + * - length of chain is at least 'optimize_min_equality_disjunction_chain_length' long OR lhs is LowCardinality + * + * E.g. (optimize_min_equality_disjunction_chain_length = 2) + * ------------------------------- + * SELECT * + * FROM table + * WHERE a = 1 OR b = 'test' OR a = 2; + * + * will be transformed into + * + * SELECT * + * FROM TABLE + * WHERE b = 'test' OR a IN (1, 2); + * ------------------------------- + */ + class OrEqualityChainToInPass final : public IQueryTreePass { public: diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index 90f0a70449a..c9eb7a08fc6 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -3,3 +3,47 @@ 2 2 2 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.regression_for_in_operator_view + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: g, result_type: String, source_id: 3 + CONSTANT id: 9, constant_value: Tuple_(\'5\', \'6\'), constant_value_type: Tuple(String, String) + CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 +2 +2 +QUERY id: 0 + PROJECTION COLUMNS + count() UInt64 + PROJECTION + LIST id: 1, nodes: 1 + FUNCTION id: 2, function_name: count, function_type: aggregate, result_type: UInt64 + JOIN TREE + TABLE id: 3, table_name: default.regression_for_in_operator_view + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: g, result_type: String, source_id: 3 + CONSTANT id: 9, constant_value: \'5\', constant_value_type: String + FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 8, column_name: g, result_type: String, source_id: 3 + CONSTANT id: 12, constant_value: \'6\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.sql b/tests/queries/0_stateless/00621_regression_for_in_operator.sql index 273f930a90f..db1bcb4a39a 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.sql +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.sql @@ -12,9 +12,13 @@ SELECT count() FROM regression_for_in_operator_view WHERE g IN ('5','6'); SET optimize_min_equality_disjunction_chain_length = 1; SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6'; +SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; SET optimize_min_equality_disjunction_chain_length = 3; SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6'; +SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN QUERY TREE SELECT count() FROM regression_for_in_operator_view WHERE g = '5' OR g = '6' SETTINGS allow_experimental_analyzer = 1; DROP TABLE regression_for_in_operator_view; DROP TABLE regression_for_in_operator; diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index afd698b425e..4210571842f 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -25,6 +25,97 @@ 3 21 3 22 3 23 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.bug + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 + JOIN TREE + QUERY id: 3, is_subquery: 1 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 + COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 + JOIN TREE + TABLE id: 7, table_name: default.bug + WHERE + FUNCTION id: 9, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 10, nodes: 2 + FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 + CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + WHERE + FUNCTION id: 16, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 1 1 23 0 0 1 @@ -34,42 +125,6 @@ 3 1 21 1 1 1 3 1 22 0 1 1 3 1 23 0 0 1 -21 1 -22 1 -23 1 -21 1 -22 1 -23 1 -21 1 -22 1 -23 1 -1 21 -1 22 -1 23 -2 21 -2 22 -2 23 -3 21 -3 22 -3 23 -1 21 -1 22 -1 23 -2 21 -2 22 -2 23 -3 21 -3 22 -3 23 -1 21 -1 22 -1 23 -2 21 -2 22 -2 23 -3 21 -3 22 -3 23 1 1 21 1 1 1 1 1 22 0 1 1 1 1 23 0 0 1 @@ -79,6 +134,53 @@ 3 1 21 1 1 1 3 1 22 0 1 1 3 1 23 0 0 1 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + or(equals(k, 1), equals(k, 2), equals(k, 3)) UInt8 + s UInt64 + equals(s, 21) UInt8 + or(equals(s, 21), equals(s, 22)) UInt8 + or(equals(s, 21), equals(s, 22), equals(s, 23)) UInt8 + PROJECTION + LIST id: 1, nodes: 6 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_21, constant_value_type: UInt8 + FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 18, constant_value: Tuple_(UInt64_21, UInt64_22), constant_value_type: Tuple(UInt8, UInt8) + CONSTANT id: 19, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 2 + FUNCTION id: 22, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 24, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 25, constant_value: UInt64_0, constant_value_type: UInt8 + JOIN TREE + TABLE id: 3, table_name: default.bug + SETTINGS allow_experimental_analyzer=1 21 1 22 1 23 1 @@ -88,3 +190,255 @@ 21 1 22 1 23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +QUERY id: 0 + PROJECTION COLUMNS + s UInt64 + or(equals(s, 21), equals(s, 22), equals(s, 23)) UInt8 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + JOIN TREE + TABLE id: 3, table_name: default.bug + SETTINGS allow_experimental_analyzer=1 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.bug + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 10, nodes: 2 + COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +1 21 +1 22 +1 23 +2 21 +2 22 +2 23 +3 21 +3 22 +3 23 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + COLUMN id: 4, column_name: s, result_type: UInt64, source_id: 3 + JOIN TREE + QUERY id: 3, is_subquery: 1 + PROJECTION COLUMNS + k UInt64 + s UInt64 + PROJECTION + LIST id: 5, nodes: 2 + COLUMN id: 6, column_name: k, result_type: UInt64, source_id: 7 + COLUMN id: 8, column_name: s, result_type: UInt64, source_id: 7 + JOIN TREE + TABLE id: 7, table_name: default.bug + WHERE + FUNCTION id: 9, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 10, nodes: 2 + FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 + CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + WHERE + FUNCTION id: 16, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 +1 1 21 1 1 1 +1 1 22 0 1 1 +1 1 23 0 0 1 +2 1 21 1 1 1 +2 1 22 0 1 1 +2 1 23 0 0 1 +3 1 21 1 1 1 +3 1 22 0 1 1 +3 1 23 0 0 1 +1 1 21 1 1 1 +1 1 22 0 1 1 +1 1 23 0 0 1 +2 1 21 1 1 1 +2 1 22 0 1 1 +2 1 23 0 0 1 +3 1 21 1 1 1 +3 1 22 0 1 1 +3 1 23 0 0 1 +QUERY id: 0 + PROJECTION COLUMNS + k UInt64 + or(equals(k, 1), equals(k, 2), equals(k, 3)) UInt8 + s UInt64 + equals(s, 21) UInt8 + or(equals(s, 21), equals(s, 22)) UInt8 + or(equals(s, 21), equals(s, 22), equals(s, 23)) UInt8 + PROJECTION + LIST id: 1, nodes: 6 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 13, constant_value: UInt64_21, constant_value_type: UInt8 + FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_21, constant_value_type: UInt8 + FUNCTION id: 19, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 20, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_22, constant_value_type: UInt8 + FUNCTION id: 22, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + FUNCTION id: 24, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 25, nodes: 2 + COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 26, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 27, constant_value: UInt64_0, constant_value_type: UInt8 + JOIN TREE + TABLE id: 3, table_name: default.bug + SETTINGS allow_experimental_analyzer=1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +QUERY id: 0 + PROJECTION COLUMNS + s UInt64 + or(equals(s, 21), equals(s, 22), equals(s, 23)) UInt8 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + JOIN TREE + TABLE id: 3, table_name: default.bug + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.sql b/tests/queries/0_stateless/00736_disjunction_optimisation.sql index 700221ef7f0..e52ed2ab45b 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.sql +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.sql @@ -5,17 +5,41 @@ insert into bug values(1,21),(1,22),(1,23),(2,21),(2,22),(2,23),(3,21),(3,22),(3 set optimize_min_equality_disjunction_chain_length = 2; select * from bug; + select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); +select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; + select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); +select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; + select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug; +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; + select s, (s=21 or s=22 or s=23) from bug; +select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; set optimize_min_equality_disjunction_chain_length = 3; select * from bug; + select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23); +select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from bug where (k =1 or k=2 or k =3) and (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; + select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23); +select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; +explain query tree select * from (select * from bug where k=1 or k=2 or k=3) where (s=21 or s=22 or s=23) SETTINGS allow_experimental_analyzer = 1;; + select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug; +select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +explain query tree select k, (k=1 or k=2 or k=3), s, (s=21), (s=21 or s=22), (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; + select s, (s=21 or s=22 or s=23) from bug; +select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; DROP TABLE bug; diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index dcfcac737c3..086c0a49b59 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -1,6 +1,48 @@ SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a IN (\'x\', \'y\') +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE (b = 0) OR (b = 1) +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + FUNCTION id: 10, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 8, column_name: b, result_type: UInt32, source_id: 3 + CONSTANT id: 12, constant_value: UInt64_1, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql index be355a05675..f5368679f60 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql @@ -4,7 +4,9 @@ CREATE TABLE t_logical_expressions_optimizer_low_cardinality (a LowCardinality(S -- LowCardinality case, ignore optimize_min_equality_disjunction_chain_length limit, optimzer applied EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y'; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y' SETTINGS allow_experimental_analyzer = 1; -- Non-LowCardinality case, optimizer not applied for short chains EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1 SETTINGS allow_experimental_analyzer = 1; DROP TABLE t_logical_expressions_optimizer_low_cardinality; From b66a5bdcf0366b751ca5ad1ecbf00cf6e80ddd6a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 21 Feb 2023 15:13:29 +0000 Subject: [PATCH 066/470] fix mutations with sparse columns --- src/DataTypes/DataTypeTuple.cpp | 14 +++ src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.cpp | 9 ++ src/DataTypes/IDataType.h | 1 + .../Serializations/SerializationInfo.h | 1 + .../Serializations/SerializationInfoTuple.cpp | 13 +++ .../Serializations/SerializationInfoTuple.h | 2 + src/Storages/AlterCommands.cpp | 7 +- .../MergedColumnOnlyOutputStream.cpp | 1 - src/Storages/MergeTree/MutateTask.cpp | 48 +++++++++-- ...02662_sparse_columns_mutations_1.reference | 13 +++ .../02662_sparse_columns_mutations_1.sql | 49 +++++++++++ ...02662_sparse_columns_mutations_2.reference | 6 ++ .../02662_sparse_columns_mutations_2.sql | 33 +++++++ ...02662_sparse_columns_mutations_3.reference | 11 +++ .../02662_sparse_columns_mutations_3.sql | 85 +++++++++++++++++++ 16 files changed, 280 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_1.reference create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_1.sql create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_2.reference create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_2.sql create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_3.reference create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_3.sql diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 768f87fe3d4..1e28af3ee54 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -312,6 +312,20 @@ MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const Seriali return std::make_shared(std::move(infos), names, settings); } +MutableSerializationInfoPtr DataTypeTuple::cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const +{ + const auto & old_info_tuple = assert_cast(old_info); + assert(old_info_tuple.getNumElements() == elems.size()); + + MutableSerializationInfos infos; + infos.reserve(elems.size()); + for (size_t i = 0; i < elems.size(); ++i) + infos.push_back(elems[i]->cloneSerializationInfo(*old_info_tuple.getElementInfo(i), settings)); + + return std::make_shared(std::move(infos), names, settings); +} + + SerializationInfoPtr DataTypeTuple::getSerializationInfo(const IColumn & column) const { if (const auto * column_const = checkAndGetColumn(&column)) diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 152f21015f5..9f3860f78db 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -57,6 +57,7 @@ public: SerializationPtr doGetDefaultSerialization() const override; SerializationPtr getSerialization(const SerializationInfo & info) const override; MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override; + MutableSerializationInfoPtr cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const override; SerializationInfoPtr getSerializationInfo(const IColumn & column) const override; const DataTypePtr & getElement(size_t i) const { return elems[i]; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index e0612fbbf36..5cd0b6f659c 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -191,6 +191,15 @@ MutableSerializationInfoPtr IDataType::createSerializationInfo(const Serializati return std::make_shared(ISerialization::Kind::DEFAULT, settings); } +MutableSerializationInfoPtr IDataType::cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const +{ + auto new_kind = old_info.getKind(); + if (new_kind == ISerialization::Kind::SPARSE && !supportsSparseSerialization()) + new_kind = ISerialization::Kind::DEFAULT; + + return std::make_shared(new_kind, settings); +} + SerializationInfoPtr IDataType::getSerializationInfo(const IColumn & column) const { if (const auto * column_const = checkAndGetColumn(&column)) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index bafe03dbc3a..32e3b53fc01 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -104,6 +104,7 @@ public: Names getSubcolumnNames() const; virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const; + virtual MutableSerializationInfoPtr cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const; virtual SerializationInfoPtr getSerializationInfo(const IColumn & column) const; /// TODO: support more types. diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index a4a5685253f..4ae5cf8c193 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -51,6 +51,7 @@ public: virtual ~SerializationInfo() = default; virtual bool hasCustomSerialization() const { return kind != ISerialization::Kind::DEFAULT; } + virtual bool structureEquals(const SerializationInfo & rhs) const { return typeid(SerializationInfo) == typeid(rhs); } virtual void add(const IColumn & column); virtual void add(const SerializationInfo & other); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 6c326743e8a..1a9639a1566 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -28,6 +28,19 @@ bool SerializationInfoTuple::hasCustomSerialization() const return std::any_of(elems.begin(), elems.end(), [](const auto & elem) { return elem->hasCustomSerialization(); }); } +bool SerializationInfoTuple::structureEquals(const SerializationInfo & rhs) const +{ + const auto * rhs_tuple = typeid_cast(&rhs); + if (!rhs_tuple || elems.size() != rhs_tuple->elems.size()) + return false; + + for (size_t i = 0; i < elems.size(); ++i) + if (!elems[i]->structureEquals(*rhs_tuple->elems[i])) + return false; + + return true; +} + void SerializationInfoTuple::add(const IColumn & column) { SerializationInfo::add(column); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.h b/src/DataTypes/Serializations/SerializationInfoTuple.h index b01c629d2ff..ef288948b9a 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.h +++ b/src/DataTypes/Serializations/SerializationInfoTuple.h @@ -11,6 +11,7 @@ public: SerializationInfoTuple(MutableSerializationInfos elems_, Names names_, const Settings & settings_); bool hasCustomSerialization() const override; + bool structureEquals(const SerializationInfo & rhs) const override; void add(const IColumn & column) override; void add(const SerializationInfo & other) override; @@ -25,6 +26,7 @@ public: Poco::JSON::Object toJSON() const override; void fromJSON(const Poco::JSON::Object & object) override; + size_t getNumElements() const { return elems.size(); } const MutableSerializationInfoPtr & getElementInfo(size_t i) const { return elems[i]; } ISerialization::Kind getElementKind(size_t i) const { return elems[i]->getKind(); } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index da11a87eb4d..aff17465466 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -719,7 +719,7 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) { typeid(DataTypeUInt16), typeid(DataTypeDate) }, }; - /// Unwrap some nested and check for valid conevrsions + /// Unwrap some nested and check for valid conversions while (true) { /// types are equal, obviously pure metadata alter @@ -749,10 +749,9 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) const auto * nullable_from = typeid_cast(from); const auto * nullable_to = typeid_cast(to); - if (nullable_to) + if (nullable_from && nullable_to) { - /// Here we allow a conversion X -> Nullable(X) to make a metadata-only conversion. - from = nullable_from ? nullable_from->getNestedType().get() : from; + from = nullable_from->getNestedType().get(); to = nullable_to->getNestedType().get(); continue; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 03829f1daf9..f6fc40884a1 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -86,7 +86,6 @@ MergedColumnOnlyOutputStream::fillChecksums( } new_part->setColumns(columns, serialization_infos); - return checksums; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 4b9265488e7..8d95a76b5da 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -203,20 +203,50 @@ getColumnsForNewDataPart( } SerializationInfoByName new_serialization_infos; - for (const auto & [name, info] : serialization_infos) + for (const auto & [name, old_info] : serialization_infos) { if (removed_columns.contains(name)) continue; auto it = renamed_columns_from_to.find(name); - if (it != renamed_columns_from_to.end()) - new_serialization_infos.emplace(it->second, info); - else - new_serialization_infos.emplace(name, info); + auto new_name = it == renamed_columns_from_to.end() ? name : it->second; + + if (!updated_header.has(new_name)) + { + new_serialization_infos.emplace(new_name, old_info); + continue; + } + + auto old_type = part_columns.getPhysical(name); + auto new_type = updated_header.getByName(new_name).type; + + if (!new_type->supportsSparseSerialization()) + continue; + + SerializationInfo::Settings settings + { + .ratio_of_defaults_for_sparse = source_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .choose_kind = false + }; + + auto new_info = new_type->createSerializationInfo(settings); + if (!old_info->structureEquals(*new_info)) + { + new_serialization_infos.emplace(new_name, std::move(new_info)); + continue; + } + + if (!old_info->hasCustomSerialization()) + { + new_serialization_infos.emplace(new_name, old_info); + continue; + } + + new_info = new_type->cloneSerializationInfo(*old_info, settings); + new_serialization_infos.emplace(new_name, std::move(new_info)); } - /// In compact parts we read all columns, because they all stored in a - /// single file + /// In compact parts we read all columns, because they all stored in a single file if (!isWidePart(source_part) || !isFullPartStorage(source_part->getDataPartStorage())) return {updated_header.getNamesAndTypesList(), new_serialization_infos}; @@ -1221,8 +1251,8 @@ private: skip_part_indices, ctx->compression_codec, ctx->txn, - false, - false, + /*reset_columns=*/ true, + /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings()); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_1.reference b/tests/queries/0_stateless/02662_sparse_columns_mutations_1.reference new file mode 100644 index 00000000000..3f5c8b6ed1f --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_1.reference @@ -0,0 +1,13 @@ +1_1_1_0 String Sparse +477 ['','foo'] +1_1_1_0_2 Nullable(String) Default +477 ['','foo'] +1_1_1_0_2 Nullable(String) Default +2_3_3_0 Nullable(String) Default +954 ['','foo'] +1_1_1_0_4 String Default +2_3_3_0_4 String Default +954 ['','foo'] +1_1_1_1_4 String Sparse +2_3_3_1_4 String Sparse +954 ['','foo'] diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_1.sql b/tests/queries/0_stateless/02662_sparse_columns_mutations_1.sql new file mode 100644 index 00000000000..3bf37e8e62b --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_1.sql @@ -0,0 +1,49 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_mutations_1; + +CREATE TABLE t_sparse_mutations_1 (key UInt8, id UInt64, s String) +ENGINE = MergeTree ORDER BY id PARTITION BY key +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_mutations_1 SELECT 1, number, if (number % 21 = 0, 'foo', '') FROM numbers (10000); + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_1' AND column = 's' AND active +ORDER BY name; + +SELECT countIf(s = 'foo'), arraySort(groupUniqArray(s)) FROM t_sparse_mutations_1; + +ALTER TABLE t_sparse_mutations_1 MODIFY COLUMN s Nullable(String); + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_1' AND column = 's' AND active +ORDER BY name; + +SELECT countIf(s = 'foo'), arraySort(groupUniqArray(s)) FROM t_sparse_mutations_1; + +INSERT INTO t_sparse_mutations_1 SELECT 2, number, if (number % 21 = 0, 'foo', '') FROM numbers (10000); + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_1' AND column = 's' AND active +ORDER BY name; + +SELECT countIf(s = 'foo'), arraySort(groupUniqArray(s)) FROM t_sparse_mutations_1; + +ALTER TABLE t_sparse_mutations_1 MODIFY COLUMN s String; + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_1' AND column = 's' AND active +ORDER BY name; + +SELECT countIf(s = 'foo'), arraySort(groupUniqArray(s)) FROM t_sparse_mutations_1; + +OPTIMIZE TABLE t_sparse_mutations_1 FINAL; + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_1' AND column = 's' AND active +ORDER BY name; + +SELECT countIf(s = 'foo'), arraySort(groupUniqArray(s)) FROM t_sparse_mutations_1; + +DROP TABLE t_sparse_mutations_1; diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_2.reference b/tests/queries/0_stateless/02662_sparse_columns_mutations_2.reference new file mode 100644 index 00000000000..64eb0119982 --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_2.reference @@ -0,0 +1,6 @@ +String Default +10000 49995000 +String Default +770 3848845 +String Sparse +770 3848845 diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_2.sql b/tests/queries/0_stateless/02662_sparse_columns_mutations_2.sql new file mode 100644 index 00000000000..561bd164200 --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_2.sql @@ -0,0 +1,33 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_mutations_2; + +CREATE TABLE t_sparse_mutations_2 (key UInt8, id UInt64, s String) +ENGINE = MergeTree ORDER BY id PARTITION BY key +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_mutations_2 SELECT 1, number, toString(number) FROM numbers (10000); + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_2' AND column = 's' AND active +ORDER BY name; + +SELECT count(), sum(s::UInt64) FROM t_sparse_mutations_2 WHERE s != ''; + +ALTER TABLE t_sparse_mutations_2 UPDATE s = '' WHERE id % 13 != 0; + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_2' AND column = 's' AND active +ORDER BY name; + +SELECT count(), sum(s::UInt64) FROM t_sparse_mutations_2 WHERE s != ''; + +OPTIMIZE TABLE t_sparse_mutations_2 FINAL; + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_2' AND column = 's' AND active +ORDER BY name; + +SELECT count(), sum(s::UInt64) FROM t_sparse_mutations_2 WHERE s != ''; + +DROP TABLE t_sparse_mutations_2; diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_3.reference b/tests/queries/0_stateless/02662_sparse_columns_mutations_3.reference new file mode 100644 index 00000000000..1501fd27fd5 --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_3.reference @@ -0,0 +1,11 @@ +String Default +Tuple(UInt64, UInt64, String, String, String) Default ['1','2','3','4','5'] ['UInt64','UInt64','String','String','String'] ['Default','Default','Default','Default','Default'] +10000 0 ['1'] ['0'] [''] +Tuple(UInt64, UInt64, String, String, String) Default ['1','2','3','4','5'] ['UInt64','UInt64','String','String','String'] ['Default','Sparse','Default','Default','Sparse'] +10000 0 ['1'] ['0'] [''] +Tuple(UInt64, UInt64, UInt64, UInt64, String) Default ['1','2','3','4','5'] ['UInt64','UInt64','UInt64','UInt64','String'] ['Default','Sparse','Default','Default','Sparse'] +10000 0 10000 0 [''] +Tuple(UInt64, UInt64, UInt64, UInt64, String) Default ['1','2','3','4','5'] ['UInt64','UInt64','UInt64','UInt64','String'] ['Default','Sparse','Default','Sparse','Sparse'] +10000 0 10000 0 [''] +Tuple(Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String)) Default ['1','1.null','2','2.null','3','3.null','4','4.null','5','5.null'] ['Nullable(UInt64)','UInt8','Nullable(UInt64)','UInt8','Nullable(UInt64)','UInt8','Nullable(UInt64)','UInt8','Nullable(String)','UInt8'] ['Default','Default','Default','Default','Default','Default','Default','Default','Default','Default'] +10000 0 10000 0 [''] diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_3.sql b/tests/queries/0_stateless/02662_sparse_columns_mutations_3.sql new file mode 100644 index 00000000000..6e66336dcbc --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_3.sql @@ -0,0 +1,85 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_mutations_3; + +CREATE TABLE t_sparse_mutations_3 (key UInt8, id UInt64, s String) +ENGINE = MergeTree ORDER BY id PARTITION BY key +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_mutations_3 SELECT 1, number, toString(tuple(1, 0, '1', '0', '')) FROM numbers (10000); + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN s Tuple(UInt64, UInt64, String, String, String); + +SELECT + type, + serialization_kind, + subcolumns.names, + subcolumns.types, + subcolumns.serializations +FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +SELECT sum(s.1), sum(s.2), groupUniqArray(s.3), groupUniqArray(s.4), groupUniqArray(s.5) FROM t_sparse_mutations_3; + +OPTIMIZE TABLE t_sparse_mutations_3 FINAL; + +SELECT + type, + serialization_kind, + subcolumns.names, + subcolumns.types, + subcolumns.serializations +FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +SELECT sum(s.1), sum(s.2), groupUniqArray(s.3), groupUniqArray(s.4), groupUniqArray(s.5) FROM t_sparse_mutations_3; + +ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN s Tuple(UInt64, UInt64, UInt64, UInt64, String); + +SELECT + type, + serialization_kind, + subcolumns.names, + subcolumns.types, + subcolumns.serializations +FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +SELECT sum(s.1), sum(s.2), sum(s.3), sum(s.4), groupUniqArray(s.5) FROM t_sparse_mutations_3; + +OPTIMIZE TABLE t_sparse_mutations_3 FINAL; + +SELECT + type, + serialization_kind, + subcolumns.names, + subcolumns.types, + subcolumns.serializations +FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +SELECT sum(s.1), sum(s.2), sum(s.3), sum(s.4), groupUniqArray(s.5) FROM t_sparse_mutations_3; + +ALTER TABLE t_sparse_mutations_3 MODIFY COLUMN s Tuple(Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(UInt64), Nullable(String)); + +SELECT + type, + serialization_kind, + subcolumns.names, + subcolumns.types, + subcolumns.serializations +FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_3' AND column = 's' AND active +ORDER BY name; + +SELECT sum(s.1), sum(s.2), sum(s.3), sum(s.4), groupUniqArray(s.5) FROM t_sparse_mutations_3; + +DROP TABLE t_sparse_mutations_3; From a110e0f0225acdc2868423b08efc8b23fe1ad697 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 15:58:56 +0100 Subject: [PATCH 067/470] Allow to ignore errors while pushing to MATERILIZED VIEW This can be useful in the following scenarious: - you want to duplicate the data to another table and you don't care about the errors - you want to duplicate system.*_log to another server, you are adding materialized view that will push to Distributed table, but you don't want to miss original blocks in the local system.*_log - you want to push some data to a 3d party service, using i.e. URL engine. Signed-off-by: Azat Khuzhin --- .../sql-reference/statements/create/view.md | 2 ++ src/Core/Settings.h | 1 + .../Transforms/buildPushingToViewsChain.cpp | 16 ++++++++-- ...materialized_views_ignore_errors.reference | 17 +++++++++++ ...02572_materialized_views_ignore_errors.sql | 30 +++++++++++++++++++ 5 files changed, 63 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference create mode 100644 tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 527b31b36a4..35242ec7a93 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -66,6 +66,8 @@ A materialized view is implemented as follows: when inserting data to the table Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in the `SELECT` query result, ClickHouse uses a default value, even if the column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. + +By default if the will be an error during pushing to the materialized view the data will not be inserted into the table to which the materialized view is attached. You can change this by setting `materialized_views_ignore_errors=true` setting for your `INSERT` query. ::: If you specify `POPULATE`, the existing table data is inserted into the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3002bcc8b49..ac9a169e2a8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -508,6 +508,7 @@ class IColumn; M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 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, 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/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index a58d70a8428..154986045d9 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -173,7 +174,7 @@ class FinalizingViewsTransform final : public IProcessor static InputPorts initPorts(std::vector headers); public: - FinalizingViewsTransform(std::vector headers, ViewsDataPtr data); + FinalizingViewsTransform(std::vector headers, ViewsDataPtr data, bool materialized_views_ignore_errors_); String getName() const override { return "FinalizingViewsTransform"; } Status prepare() override; @@ -184,6 +185,7 @@ private: ViewsDataPtr views_data; std::vector statuses; std::exception_ptr any_exception; + bool materialized_views_ignore_errors; }; @@ -407,7 +409,7 @@ Chain buildPushingToViewsChain( headers.push_back(chain.getOutputHeader()); auto copying_data = std::make_shared(storage_header, views_data); - auto finalizing_views = std::make_shared(std::move(headers), views_data); + auto finalizing_views = std::make_shared(std::move(headers), views_data, settings.materialized_views_ignore_errors); auto out = copying_data->getOutputs().begin(); auto in = finalizing_views->getInputs().begin(); @@ -684,10 +686,11 @@ void PushingToWindowViewSink::consume(Chunk chunk) } -FinalizingViewsTransform::FinalizingViewsTransform(std::vector headers, ViewsDataPtr data) +FinalizingViewsTransform::FinalizingViewsTransform(std::vector headers, ViewsDataPtr data, bool materialized_views_ignore_errors_) : IProcessor(initPorts(std::move(headers)), {Block()}) , output(outputs.front()) , views_data(std::move(data)) + , materialized_views_ignore_errors(materialized_views_ignore_errors_) { statuses.resize(views_data->views.size()); } @@ -788,6 +791,13 @@ void FinalizingViewsTransform::work() auto & status = statuses[i]; ++i; + if (status.exception && materialized_views_ignore_errors) + { + auto exception = addStorageToException(status.exception, view.table_id); + tryLogException(exception, &Poco::Logger::get("PushingToViews"), "Cannot push to the storage, ignoring the error"); + continue; + } + if (status.exception) { if (!any_exception) diff --git a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference new file mode 100644 index 00000000000..537865e10da --- /dev/null +++ b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference @@ -0,0 +1,17 @@ +-- { echoOn } +select * from data_02572 order by key; +insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } +select * from data_02572 order by key; +1 +insert into data_02572 settings materialized_views_ignore_errors=1 values (2); +select * from data_02572 order by key; +1 +2 +create table receiver_02572 as data_02572; +insert into data_02572 values (3); +select * from data_02572 order by key; +1 +2 +3 +select * from receiver_02572 order by key; +3 diff --git a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql new file mode 100644 index 00000000000..41fa9e32d1c --- /dev/null +++ b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql @@ -0,0 +1,30 @@ +set prefer_localhost_replica=1; + +drop table if exists data_02572; +drop table if exists proxy_02572; +drop table if exists push_to_proxy_mv_02572; +drop table if exists receiver_02572; + +create table data_02572 (key Int) engine=Memory(); + +create table proxy_02572 (key Int) engine=Distributed('test_shard_localhost', currentDatabase(), 'receiver_02572'); +-- ensure that insert fails +insert into proxy_02572 values (1); -- { serverError UNKNOWN_TABLE } + +-- proxy data with MV +create materialized view push_to_proxy_mv_02572 to proxy_02572 as select * from data_02572; + +-- { echoOn } +select * from data_02572 order by key; + +insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } +select * from data_02572 order by key; + +insert into data_02572 settings materialized_views_ignore_errors=1 values (2); +select * from data_02572 order by key; + +create table receiver_02572 as data_02572; + +insert into data_02572 values (3); +select * from data_02572 order by key; +select * from receiver_02572 order by key; From 6391e2daecad874e29ae444b24f537a64758de0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 16:27:30 +0100 Subject: [PATCH 068/470] Set materialized_views_ignore_errors unconditionally for system.*_log tables Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/statements/create/view.md | 2 ++ src/Interpreters/SystemLog.cpp | 2 ++ ...s_materialized_views_ignore_errors.reference | 0 ...em_logs_materialized_views_ignore_errors.sql | 17 +++++++++++++++++ 4 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference create mode 100644 tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 35242ec7a93..5d54662326a 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -68,6 +68,8 @@ Materialized views in ClickHouse use **column names** instead of column order du Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. By default if the will be an error during pushing to the materialized view the data will not be inserted into the table to which the materialized view is attached. You can change this by setting `materialized_views_ignore_errors=true` setting for your `INSERT` query. + +`materialized_views_ignore_errors` set to `true` by default for `system.*_log` tables. ::: If you specify `POPULATE`, the existing table data is inserted into the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 9d8547abcf2..78513920236 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -426,6 +426,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, // we need query context to do inserts to target table with MV containing subqueries or joins auto insert_context = Context::createCopy(context); insert_context->makeQueryContext(); + /// We always want to deliver the data to the original table regardless of the MVs + insert_context->setSetting("materialized_views_ignore_errors", true); InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql new file mode 100644 index 00000000000..d26f86962c4 --- /dev/null +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql @@ -0,0 +1,17 @@ +-- Tags: no-parallel +-- Tag no-parallel: due to attaching to system.query_log + +-- Attach MV to system.query_log and check that writing query_log will not fail + +drop table if exists log_proxy_02572; +drop table if exists push_to_logs_proxy_mv_02572; + +create table log_proxy_02572 as system.query_log engine=Distributed('test_shard_localhost', currentDatabase(), 'receiver_02572'); +create materialized view push_to_logs_proxy_mv_02572 to log_proxy_02572 as select * from system.query_log; + +set log_queries=1; +system flush logs; +system flush logs; + +drop table log_proxy_02572; +drop table push_to_logs_proxy_mv_02572; From 81435282ec2665ba189511a9427a7ccf6692e963 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 17:04:58 +0100 Subject: [PATCH 069/470] Add an entry to system.query_views_log regardless of materialized_views_ignore_errors Signed-off-by: Azat Khuzhin --- .../Transforms/buildPushingToViewsChain.cpp | 15 ++++++--------- ...2_materialized_views_ignore_errors.reference | 17 ++++++++++++++--- .../02572_materialized_views_ignore_errors.sql | 16 +++++++++++++--- 3 files changed, 33 insertions(+), 15 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 154986045d9..13c4806590e 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -737,7 +737,7 @@ IProcessor::Status FinalizingViewsTransform::prepare() else statuses[i].exception = data.exception; - if (i == 0 && statuses[0].is_first) + if (i == 0 && statuses[0].is_first && !materialized_views_ignore_errors) { output.pushData(std::move(data)); return Status::PortFull; @@ -754,7 +754,7 @@ IProcessor::Status FinalizingViewsTransform::prepare() if (!statuses.empty()) return Status::Ready; - if (any_exception) + if (any_exception && !materialized_views_ignore_errors) output.pushException(any_exception); output.finish(); @@ -791,19 +791,16 @@ void FinalizingViewsTransform::work() auto & status = statuses[i]; ++i; - if (status.exception && materialized_views_ignore_errors) - { - auto exception = addStorageToException(status.exception, view.table_id); - tryLogException(exception, &Poco::Logger::get("PushingToViews"), "Cannot push to the storage, ignoring the error"); - continue; - } - if (status.exception) { if (!any_exception) any_exception = status.exception; view.setException(addStorageToException(status.exception, view.table_id)); + + /// Exception will be ignored, it is saved here for the system.query_views_log + if (materialized_views_ignore_errors) + tryLogException(view.exception, &Poco::Logger::get("PushingToViews"), "Cannot push to the storage, ignoring the error"); } else { diff --git a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference index 537865e10da..fc2e6b78122 100644 --- a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference +++ b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.reference @@ -1,10 +1,21 @@ -- { echoOn } select * from data_02572 order by key; -insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } -select * from data_02572 order by key; -1 insert into data_02572 settings materialized_views_ignore_errors=1 values (2); select * from data_02572 order by key; +2 +-- check system.query_views_log +system flush logs; +-- lower(status) to pass through clickhouse-test "exception" check +select lower(status::String), errorCodeToName(exception_code) +from system.query_views_log where + view_name = concatWithSeparator('.', currentDatabase(), 'push_to_proxy_mv_02572') and + view_target = concatWithSeparator('.', currentDatabase(), 'proxy_02572') + order by event_date, event_time +; +exceptionwhileprocessing UNKNOWN_TABLE +-- materialized_views_ignore_errors=0 +insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } +select * from data_02572 order by key; 1 2 create table receiver_02572 as data_02572; diff --git a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql index 41fa9e32d1c..2d1f824b9b1 100644 --- a/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql +++ b/tests/queries/0_stateless/02572_materialized_views_ignore_errors.sql @@ -17,11 +17,21 @@ create materialized view push_to_proxy_mv_02572 to proxy_02572 as select * from -- { echoOn } select * from data_02572 order by key; -insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } -select * from data_02572 order by key; - insert into data_02572 settings materialized_views_ignore_errors=1 values (2); select * from data_02572 order by key; +-- check system.query_views_log +system flush logs; +-- lower(status) to pass through clickhouse-test "exception" check +select lower(status::String), errorCodeToName(exception_code) +from system.query_views_log where + view_name = concatWithSeparator('.', currentDatabase(), 'push_to_proxy_mv_02572') and + view_target = concatWithSeparator('.', currentDatabase(), 'proxy_02572') + order by event_date, event_time +; + +-- materialized_views_ignore_errors=0 +insert into data_02572 values (1); -- { serverError UNKNOWN_TABLE } +select * from data_02572 order by key; create table receiver_02572 as data_02572; From 83b2f6434107be0852b07b466faeb42f77e8496f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 17:23:26 +0100 Subject: [PATCH 070/470] Improve 02572_system_logs_materialized_views_ignore_errors test Signed-off-by: Azat Khuzhin --- ...tem_logs_materialized_views_ignore_errors.reference | 2 ++ ...72_system_logs_materialized_views_ignore_errors.sql | 10 +++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference index e69de29bb2d..d5446e756a3 100644 --- a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference @@ -0,0 +1,2 @@ +10 querystart OK +10 queryfinish OK diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql index d26f86962c4..b7362ac1c33 100644 --- a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql @@ -3,15 +3,23 @@ -- Attach MV to system.query_log and check that writing query_log will not fail +set log_queries=1; + drop table if exists log_proxy_02572; drop table if exists push_to_logs_proxy_mv_02572; create table log_proxy_02572 as system.query_log engine=Distributed('test_shard_localhost', currentDatabase(), 'receiver_02572'); create materialized view push_to_logs_proxy_mv_02572 to log_proxy_02572 as select * from system.query_log; -set log_queries=1; +select 1 format Null; system flush logs; system flush logs; drop table log_proxy_02572; drop table push_to_logs_proxy_mv_02572; + +system flush logs; +-- lower() to pass through clickhouse-test "exception" check +select count(), lower(type::String), errorCodeToName(exception_code) + from system.query_log + where current_database = currentDatabase() group by 2, 3; From 4f8da66bb51c0e5f59674b2e2f86dfe49840fe08 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 21 Feb 2023 18:25:10 +0000 Subject: [PATCH 071/470] fix unit test with sparse columns --- src/Columns/ColumnSparse.cpp | 1 + src/Columns/tests/gtest_column_sparse.cpp | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0e408fca467..fbab61c879d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -456,6 +456,7 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, nullptr, nested_result, direction, nan_direction_hint); const auto & offsets_data = getOffsetsData(); + compare_results.resize(size()); std::fill(compare_results.begin(), compare_results.end(), nested_result[0]); for (size_t i = 0; i < offsets_data.size(); ++i) compare_results[offsets_data[i]] = nested_result[i + 1]; diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index b5b226c6862..6062ea51941 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -327,4 +327,3 @@ TEST(ColumnSparse, GetPermutation) } #undef DUMP_COLUMN -#undef DUMP_NON_DEFAULTS From 3a58951f6446b3aeca13cdc671322f459c4f77a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Feb 2023 19:10:43 +0100 Subject: [PATCH 072/470] Improve documentation for materialized_views_ignore_errors Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/statements/create/view.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 5d54662326a..6e412c13291 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -67,9 +67,11 @@ Materialized views in ClickHouse use **column names** instead of column order du Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. -By default if the will be an error during pushing to the materialized view the data will not be inserted into the table to which the materialized view is attached. You can change this by setting `materialized_views_ignore_errors=true` setting for your `INSERT` query. +Materialized views in ClickHouse does not have deterministic behaviour in case of errors. This means that blocks that had been already written will be preserved in the destination table, but all blocks after error will not. -`materialized_views_ignore_errors` set to `true` by default for `system.*_log` tables. +By default if pushing to one of views fails, then the INSERT query will fail too, and some blocks may not be written to the destination table. This can be changed using `materialized_views_ignore_errors` setting (you should set it for `INSERT` query), if you will set `materialized_views_ignore_errors=true`, then any errors while pushing to views will be ignored and all blocks will be written to the destination table. + +Also note, that `materialized_views_ignore_errors` set to `true` by default for `system.*_log` tables. ::: If you specify `POPULATE`, the existing table data is inserted into the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. From f5c232707d7e948776a67e761bcf65e19d08e05f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 Feb 2023 20:04:40 +0100 Subject: [PATCH 073/470] cancel insert queries correctly --- .../PushingAsyncPipelineExecutor.cpp | 4 +- .../Executors/PushingPipelineExecutor.cpp | 4 +- src/Server/GRPCServer.cpp | 5 +- src/Server/TCPHandler.cpp | 15 ++++- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- ...ert_cancellation_native_protocol.reference | 2 + ...434_insert_cancellation_native_protocol.sh | 65 +++++++++++++++++++ 7 files changed, 93 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference create mode 100755 tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 54c1e7bf30f..20f47c6b54c 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -139,9 +139,11 @@ PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(QueryPipeline & pipel PushingAsyncPipelineExecutor::~PushingAsyncPipelineExecutor() { + /// It must be finalized explicitly. Otherwise we cancel it assuming it's due to an exception. + chassert(finished || std::uncaught_exceptions()); try { - finish(); + cancel(); } catch (...) { diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index d9a14704cd0..ca60932ed6f 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -63,9 +63,11 @@ PushingPipelineExecutor::PushingPipelineExecutor(QueryPipeline & pipeline_) : pi PushingPipelineExecutor::~PushingPipelineExecutor() { + /// It must be finalized explicitly. Otherwise we cancel it assuming it's due to an exception. + chassert(finished || std::uncaught_exceptions()); try { - finish(); + cancel(); } catch (...) { diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 595f5a8c2b7..9a4e43b2fde 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -984,7 +984,10 @@ namespace executor.push(block); } - executor.finish(); + if (isQueryCancelled()) + executor.cancel(); + else + executor.finish(); } void Call::initializePipeline(const Block & header) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f76c342fa9a..f1340764e00 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -109,6 +109,7 @@ namespace ErrorCodes extern const int UNEXPECTED_PACKET_FROM_CLIENT; extern const int UNKNOWN_PROTOCOL; extern const int AUTHENTICATION_FAILED; + extern const int QUERY_WAS_CANCELLED; } TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) @@ -658,6 +659,7 @@ bool TCPHandler::readDataNext() { LOG_INFO(log, "Client has dropped the connection, cancel the query."); state.is_connection_closed = true; + state.is_cancelled = true; break; } @@ -701,6 +703,9 @@ void TCPHandler::readData() while (readDataNext()) ; + + if (state.is_cancelled) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); } @@ -711,6 +716,9 @@ void TCPHandler::skipData() while (readDataNext()) ; + + if (state.is_cancelled) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); } @@ -747,7 +755,10 @@ void TCPHandler::processInsertQuery() while (readDataNext()) executor.push(std::move(state.block_for_insert)); - executor.finish(); + if (state.is_cancelled) + executor.cancel(); + else + executor.finish(); }; if (num_threads > 1) @@ -1314,6 +1325,8 @@ bool TCPHandler::receivePacket() std::this_thread::sleep_for(ms); } + state.is_cancelled = true; + return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db99007cb4f..b8e889d6182 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -794,7 +794,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_WARNING(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); continue; } else if (code != Coordination::Error::ZOK) @@ -874,7 +874,7 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada case Coordination::Error::ZNODEEXISTS: throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, "Replica {} already exists", replica_path); case Coordination::Error::ZBADVERSION: - LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time"); + LOG_INFO(log, "Retrying createReplica(), because some other replicas were created at the same time"); break; case Coordination::Error::ZNONODE: throw Exception(ErrorCodes::ALL_REPLICAS_LOST, "Table {} was suddenly removed", zookeeper_path); diff --git a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference b/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference new file mode 100644 index 00000000000..e3038e03530 --- /dev/null +++ b/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference @@ -0,0 +1,2 @@ +5000000 +5000000 diff --git a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh b/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh new file mode 100755 index 00000000000..4eb02c38c1a --- /dev/null +++ b/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash +# shellcheck disable=SC2009 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" +export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" + +$CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE +$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' +$CLICKHOUSE_CLIENT --max_block_size=100000 --min_chunk_bytes_for_parallel_parsing=10000 -q 'insert into dedup_test format TSV' < $DATA_FILE +$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' + +function thread_insert +{ + # supress "Killed" messages from bash + function wrap + { + $CLICKHOUSE_CLIENT --max_block_size=100000 --min_chunk_bytes_for_parallel_parsing=10000 --query_id="$ID" -q 'insert into dedup_test format TSV' < $DATA_FILE + } + export -f wrap + while true; do + export ID="$TEST_MARK$RANDOM" + bash -c wrap 2>&1| grep -Fav "Killed" + done +} + +function thread_select +{ + while true; do + $CLICKHOUSE_CLIENT -q "with (select count() from dedup_test) as c select throwIf(c != 5000000, 'Expected 5000000 rows, got ' || toString(c)) format Null" + sleep 0.$RANDOM; + done +} + +function thread_cancel +{ + while true; do + SIGNAL="INT" + if (( RANDOM % 2 )); then + SIGNAL="KILL" + fi + PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') + if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID" || echo "$PID"; fi + sleep 0.$RANDOM; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + done +} + +export -f thread_insert; +export -f thread_select; +export -f thread_cancel; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread_insert & +timeout $TIMEOUT bash -c thread_select & +timeout $TIMEOUT bash -c thread_cancel & + +wait + +$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' From 592af6d652b69c8e5b18e2a8578ac76cbfcffe71 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Feb 2023 02:34:03 +0100 Subject: [PATCH 074/470] fix incomplete interst through http --- src/Common/StatusFile.cpp | 2 +- src/Core/MySQL/IMySQLReadPacket.cpp | 4 +- src/IO/LimitReadBuffer.cpp | 24 +++++-- src/IO/LimitReadBuffer.h | 8 ++- src/Interpreters/executeQuery.cpp | 2 +- src/Server/HTTP/HTTPServerRequest.cpp | 15 +++- src/Server/MySQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- .../02435_insert_cancellation_http.reference | 2 + .../02435_insert_cancellation_http.sh | 69 +++++++++++++++++++ 11 files changed, 114 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02435_insert_cancellation_http.reference create mode 100755 tests/queries/0_stateless/02435_insert_cancellation_http.sh diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index d90d50ff96d..e71056fcaa3 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, false); + LimitReadBuffer limit_in(in, 1024, /* trow_exception */ false, /* exact_limit */ false); readStringUntilEOF(contents, limit_in); } diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 74f309d0294..c99ae11688b 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -33,13 +33,13 @@ void IMySQLReadPacket::readPayloadWithUnpacked(ReadBuffer & in) void LimitedReadPacket::readPayload(ReadBuffer &in, uint8_t &sequence_id) { - LimitReadBuffer limited(in, 10000, true, "too long MySQL packet."); + LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); IMySQLReadPacket::readPayload(limited, sequence_id); } void LimitedReadPacket::readPayloadWithUnpacked(ReadBuffer & in) { - LimitReadBuffer limited(in, 10000, true, "too long MySQL packet."); + LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); IMySQLReadPacket::readPayloadWithUnpacked(limited); } diff --git a/src/IO/LimitReadBuffer.cpp b/src/IO/LimitReadBuffer.cpp index 6b3c383c753..96e2c359b58 100644 --- a/src/IO/LimitReadBuffer.cpp +++ b/src/IO/LimitReadBuffer.cpp @@ -9,6 +9,7 @@ namespace DB namespace ErrorCodes { extern const int LIMIT_EXCEEDED; + extern const int CANNOT_READ_ALL_DATA; } @@ -21,14 +22,22 @@ bool LimitReadBuffer::nextImpl() if (bytes >= limit) { + if (exact_limit && bytes == limit) + return false; + + if (exact_limit && bytes != limit) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected data, got {} bytes, expected {}", bytes, limit); + if (throw_exception) throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Limit for LimitReadBuffer exceeded: {}", exception_message); - else - return false; + + return false; } if (!in->next()) { + if (exact_limit && bytes != limit) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, limit); /// Clearing the buffer with existing data. set(in->position(), 0); return false; @@ -43,12 +52,13 @@ bool LimitReadBuffer::nextImpl() } -LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::string exception_message_) +LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) : ReadBuffer(in_ ? in_->position() : nullptr, 0) , in(in_) , owns_in(owns) , limit(limit_) , throw_exception(throw_exception_) + , exact_limit(exact_limit_) , exception_message(std::move(exception_message_)) { assert(in); @@ -61,14 +71,14 @@ LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, boo } -LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, std::string exception_message_) - : LimitReadBuffer(&in_, false, limit_, throw_exception_, exception_message_) +LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) + : LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_) { } -LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, std::string exception_message_) - : LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exception_message_) +LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) + : LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_) { } diff --git a/src/IO/LimitReadBuffer.h b/src/IO/LimitReadBuffer.h index 92e5fbb0aa6..0f99bf2a08d 100644 --- a/src/IO/LimitReadBuffer.h +++ b/src/IO/LimitReadBuffer.h @@ -13,8 +13,9 @@ namespace DB class LimitReadBuffer : public ReadBuffer { public: - LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, std::string exception_message_ = {}); - LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, std::string exception_message_ = {}); + LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_ = {}); + LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, + std::string exception_message_ = {}); ~LimitReadBuffer() override; private: @@ -23,9 +24,10 @@ private: UInt64 limit; bool throw_exception; + bool exact_limit; std::string exception_message; - LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::string exception_message_); + LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_); bool nextImpl() override; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3bb46462353..aeaacf35051 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1237,7 +1237,7 @@ void executeQuery( /// If not - copy enough data into 'parse_buf'. WriteBufferFromVector> out(parse_buf); - LimitReadBuffer limit(istr, max_query_size + 1, false); + LimitReadBuffer limit(istr, max_query_size + 1, /* trow_exception */ false, /* exact_limit */ false); copyData(limit, out); out.finalize(); diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index a82eb95aee1..d2e19d594dc 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -12,6 +12,8 @@ #include #include +#include + #if USE_SSL #include #include @@ -44,12 +46,23 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse readRequest(*in); /// Try parse according to RFC7230 + /// If a client crashes, most systems will gracefully terminate the connection with FIN just like it's done on close(). + /// So we will get 0 from recv(...) and will not be able to understand that something went wrong (well, we probably + /// will get RST later on attempt to write to the socket that closed on the other side, but it will happen when the query is finished). + /// If we are extremely unlucky and data format is TSV, for example, then we may stop parsing exactly between rows + /// and decide that it's EOF (but it is not). It may break deduplication, because clients cannot control it + /// and retry with exactly the same (incomplete) set of rows. + /// That's why we have to check body size if it's provided. if (getChunkedTransferEncoding()) stream = std::make_unique(std::move(in), context->getMaxChunkSize()); else if (hasContentLength()) - stream = std::make_unique(std::move(in), getContentLength(), false); + stream = std::make_unique(std::move(in), getContentLength(), /* trow_exception */ true, /* exact_limit */ true); else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) + { stream = std::move(in); + LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length, " + "it may be impossible to distinguish graceful EOF from abnormal connection loss"); + } else /// We have to distinguish empty buffer and nullptr. stream = std::make_unique(); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 3715d658730..279896da9fa 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -155,7 +155,7 @@ void MySQLHandler::run() payload.readStrict(command); // For commands which are executed without MemoryTracker. - LimitReadBuffer limited_payload(payload, 10000, true, "too long MySQL packet."); + LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); LOG_DEBUG(log, "Received command: {}. Connection id: {}.", static_cast(static_cast(command)), connection_id); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f1340764e00..3e89d1810c7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1057,7 +1057,7 @@ bool TCPHandler::receiveProxyHeader() /// Only PROXYv1 is supported. /// Validation of protocol is not fully performed. - LimitReadBuffer limit_in(*in, 107, true); /// Maximum length from the specs. + LimitReadBuffer limit_in(*in, 107, /* trow_exception */ true, /* exact_limit */ false); /// Maximum length from the specs. assertString("PROXY ", limit_in); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index f2eca42ed0b..388faf1aeb5 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -118,7 +118,7 @@ private: if (limited_by_file_size) { - limited.emplace(*plain, file_size - offset, false); + limited.emplace(*plain, file_size - offset, /* trow_exception */ false, /* exact_limit */ false); compressed.emplace(*limited); } else diff --git a/tests/queries/0_stateless/02435_insert_cancellation_http.reference b/tests/queries/0_stateless/02435_insert_cancellation_http.reference new file mode 100644 index 00000000000..e3038e03530 --- /dev/null +++ b/tests/queries/0_stateless/02435_insert_cancellation_http.reference @@ -0,0 +1,2 @@ +5000000 +5000000 diff --git a/tests/queries/0_stateless/02435_insert_cancellation_http.sh b/tests/queries/0_stateless/02435_insert_cancellation_http.sh new file mode 100755 index 00000000000..653ec59f491 --- /dev/null +++ b/tests/queries/0_stateless/02435_insert_cancellation_http.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# shellcheck disable=SC2009 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" +export TEST_MARK="02435_insert_${CLICKHOUSE_DATABASE}_" + +$CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE +$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' +$CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE +$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' + +function thread_insert +{ + # supress "Killed" messages from bash + function wrap + { + if (( RANDOM % 2 )); then + $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + else + $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + fi + } + export -f wrap + while true; do + export ID="$TEST_MARK$RANDOM" + bash -c wrap 2>&1| grep -Fav "Killed" + done +} + +function thread_select +{ + while true; do + $CLICKHOUSE_CLIENT -q "with (select count() from dedup_test) as c select throwIf(c != 5000000, 'Expected 5000000 rows, got ' || toString(c)) format Null" + sleep 0.$RANDOM; + done +} + +function thread_cancel +{ + while true; do + SIGNAL="INT" + if (( RANDOM % 2 )); then + SIGNAL="KILL" + fi + PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') + if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID" || echo "$PID"; fi + sleep 0.$RANDOM; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + done +} + +export -f thread_insert; +export -f thread_select; +export -f thread_cancel; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread_insert & +timeout $TIMEOUT bash -c thread_select & +timeout $TIMEOUT bash -c thread_cancel & + +wait + +$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' From a3d5a76b37252e93f929f4737753f5fb9a156cc0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 22 Feb 2023 08:07:16 +0100 Subject: [PATCH 075/470] Do not run 02572_system_logs_materialized_views_ignore_errors for Replicated CI: https://s3.amazonaws.com/clickhouse-test-reports/46658/3a58951f6446b3aeca13cdc671322f459c4f77a7/stateless_tests__release__databasereplicated__[1/4].html Signed-off-by: Azat Khuzhin --- .../02572_system_logs_materialized_views_ignore_errors.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql index b7362ac1c33..9568bc7af1a 100644 --- a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql @@ -1,5 +1,6 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-replicated-database -- Tag no-parallel: due to attaching to system.query_log +-- Tag no-replicated-database: Replicated database will has extra queries -- Attach MV to system.query_log and check that writing query_log will not fail From d0b50adffa98b6db67586098430d8f08f970e174 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 22 Feb 2023 11:00:14 +0100 Subject: [PATCH 076/470] Remove materialized_views_ignore_errors from FinalizingViewsTransform Signed-off-by: Azat Khuzhin --- .../Transforms/buildPushingToViewsChain.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 13c4806590e..dce2c61457c 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -174,7 +174,7 @@ class FinalizingViewsTransform final : public IProcessor static InputPorts initPorts(std::vector headers); public: - FinalizingViewsTransform(std::vector headers, ViewsDataPtr data, bool materialized_views_ignore_errors_); + FinalizingViewsTransform(std::vector headers, ViewsDataPtr data); String getName() const override { return "FinalizingViewsTransform"; } Status prepare() override; @@ -185,7 +185,6 @@ private: ViewsDataPtr views_data; std::vector statuses; std::exception_ptr any_exception; - bool materialized_views_ignore_errors; }; @@ -409,7 +408,7 @@ Chain buildPushingToViewsChain( headers.push_back(chain.getOutputHeader()); auto copying_data = std::make_shared(storage_header, views_data); - auto finalizing_views = std::make_shared(std::move(headers), views_data, settings.materialized_views_ignore_errors); + auto finalizing_views = std::make_shared(std::move(headers), views_data); auto out = copying_data->getOutputs().begin(); auto in = finalizing_views->getInputs().begin(); @@ -686,11 +685,10 @@ void PushingToWindowViewSink::consume(Chunk chunk) } -FinalizingViewsTransform::FinalizingViewsTransform(std::vector headers, ViewsDataPtr data, bool materialized_views_ignore_errors_) +FinalizingViewsTransform::FinalizingViewsTransform(std::vector headers, ViewsDataPtr data) : IProcessor(initPorts(std::move(headers)), {Block()}) , output(outputs.front()) , views_data(std::move(data)) - , materialized_views_ignore_errors(materialized_views_ignore_errors_) { statuses.resize(views_data->views.size()); } @@ -712,6 +710,7 @@ IProcessor::Status FinalizingViewsTransform::prepare() if (!output.canPush()) return Status::PortFull; + bool materialized_views_ignore_errors = views_data->context->getSettingsRef().materialized_views_ignore_errors; size_t num_finished = 0; size_t pos = 0; for (auto & input : inputs) @@ -785,6 +784,8 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St void FinalizingViewsTransform::work() { + bool materialized_views_ignore_errors = views_data->context->getSettingsRef().materialized_views_ignore_errors; + size_t i = 0; for (auto & view : views_data->views) { From e660c0838ca7ddf597f0a7d54b52e305b482a474 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 22 Feb 2023 17:54:35 +0100 Subject: [PATCH 077/470] fix multipart requests --- src/Common/StatusFile.cpp | 2 +- src/Core/ExternalTable.cpp | 6 ++-- src/Core/MySQL/IMySQLReadPacket.cpp | 4 +-- src/IO/LimitReadBuffer.cpp | 19 +++++++----- src/IO/LimitReadBuffer.h | 9 +++--- src/IO/examples/limit_read_buffer.cpp | 4 +-- src/IO/examples/limit_read_buffer2.cpp | 6 ++-- src/Interpreters/executeQuery.cpp | 2 +- src/Server/HTTP/HTMLForm.cpp | 16 ++++++++-- src/Server/HTTP/HTMLForm.h | 3 ++ src/Server/HTTP/HTTPServerRequest.cpp | 11 +++++-- src/Server/MySQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- .../02435_insert_cancellation_http.sh | 30 ++++++++++++------- 15 files changed, 76 insertions(+), 42 deletions(-) diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index e71056fcaa3..a9ffce7ddf8 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 */ false); + LimitReadBuffer limit_in(in, 1024, /* trow_exception */ false, /* exact_limit */ {}); readStringUntilEOF(contents, limit_in); } diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 217f8808185..36dcc677589 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -34,7 +34,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { initReadBuffer(); initSampleBlock(); - auto input = context->getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); + auto input = context->getInputFormat(format, *read_buffer, sample_block, context->getSettingsRef().get("max_block_size").get()); auto data = std::make_unique(); data->pipe = std::make_unique(); @@ -135,7 +135,9 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, if (settings.http_max_multipart_form_data_size) read_buffer = std::make_unique( stream, settings.http_max_multipart_form_data_size, - true, "the maximum size of multipart/form-data. This limit can be tuned by 'http_max_multipart_form_data_size' setting"); + /* trow_exception */ true, /* exact_limit */ std::optional(), + "the maximum size of multipart/form-data. " + "This limit can be tuned by 'http_max_multipart_form_data_size' setting"); else read_buffer = wrapReadBufferReference(stream); diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index c99ae11688b..39b2e5bbfb5 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -33,13 +33,13 @@ void IMySQLReadPacket::readPayloadWithUnpacked(ReadBuffer & in) void LimitedReadPacket::readPayload(ReadBuffer &in, uint8_t &sequence_id) { - LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); + LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet."); IMySQLReadPacket::readPayload(limited, sequence_id); } void LimitedReadPacket::readPayloadWithUnpacked(ReadBuffer & in) { - LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); + LimitReadBuffer limited(in, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet."); IMySQLReadPacket::readPayloadWithUnpacked(limited); } diff --git a/src/IO/LimitReadBuffer.cpp b/src/IO/LimitReadBuffer.cpp index 96e2c359b58..e14112f8d19 100644 --- a/src/IO/LimitReadBuffer.cpp +++ b/src/IO/LimitReadBuffer.cpp @@ -22,11 +22,11 @@ bool LimitReadBuffer::nextImpl() if (bytes >= limit) { - if (exact_limit && bytes == limit) + if (exact_limit && bytes == *exact_limit) return false; - if (exact_limit && bytes != limit) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected data, got {} bytes, expected {}", bytes, limit); + if (exact_limit && bytes != *exact_limit) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected data, got {} bytes, expected {}", bytes, *exact_limit); if (throw_exception) throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Limit for LimitReadBuffer exceeded: {}", exception_message); @@ -36,8 +36,8 @@ bool LimitReadBuffer::nextImpl() if (!in->next()) { - if (exact_limit && bytes != limit) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, limit); + if (exact_limit && bytes != *exact_limit) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, got {} of {} bytes", bytes, *exact_limit); /// Clearing the buffer with existing data. set(in->position(), 0); return false; @@ -52,7 +52,8 @@ bool LimitReadBuffer::nextImpl() } -LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) +LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, + std::optional exact_limit_, std::string exception_message_) : ReadBuffer(in_ ? in_->position() : nullptr, 0) , in(in_) , owns_in(owns) @@ -71,13 +72,15 @@ LimitReadBuffer::LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, boo } -LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) +LimitReadBuffer::LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, + std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(&in_, false, limit_, throw_exception_, exact_limit_, exception_message_) { } -LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_) +LimitReadBuffer::LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, + std::optional exact_limit_, std::string exception_message_) : LimitReadBuffer(in_.release(), true, limit_, throw_exception_, exact_limit_, exception_message_) { } diff --git a/src/IO/LimitReadBuffer.h b/src/IO/LimitReadBuffer.h index 0f99bf2a08d..15885c1d850 100644 --- a/src/IO/LimitReadBuffer.h +++ b/src/IO/LimitReadBuffer.h @@ -13,8 +13,9 @@ namespace DB class LimitReadBuffer : public ReadBuffer { public: - LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_ = {}); - LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, bool exact_limit_, + LimitReadBuffer(ReadBuffer & in_, UInt64 limit_, bool throw_exception_, + std::optional exact_limit_, std::string exception_message_ = {}); + LimitReadBuffer(std::unique_ptr in_, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_ = {}); ~LimitReadBuffer() override; @@ -24,10 +25,10 @@ private: UInt64 limit; bool throw_exception; - bool exact_limit; + std::optional exact_limit; std::string exception_message; - LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, bool exact_limit_, std::string exception_message_); + LimitReadBuffer(ReadBuffer * in_, bool owns, UInt64 limit_, bool throw_exception_, std::optional exact_limit_, std::string exception_message_); bool nextImpl() override; }; diff --git a/src/IO/examples/limit_read_buffer.cpp b/src/IO/examples/limit_read_buffer.cpp index 559c87353f0..9c57c175620 100644 --- a/src/IO/examples/limit_read_buffer.cpp +++ b/src/IO/examples/limit_read_buffer.cpp @@ -24,13 +24,13 @@ int main(int argc, char ** argv) writeCString("--- first ---\n", out); { - LimitReadBuffer limit_in(in, limit, false); + LimitReadBuffer limit_in(in, limit, /* trow_exception */ false, /* exact_limit */ {}); copyData(limit_in, out); } writeCString("\n--- second ---\n", out); { - LimitReadBuffer limit_in(in, limit, false); + LimitReadBuffer limit_in(in, limit, /* trow_exception */ false, /* exact_limit */ {}); copyData(limit_in, out); } diff --git a/src/IO/examples/limit_read_buffer2.cpp b/src/IO/examples/limit_read_buffer2.cpp index ac7c43d764c..a0369047d3a 100644 --- a/src/IO/examples/limit_read_buffer2.cpp +++ b/src/IO/examples/limit_read_buffer2.cpp @@ -27,7 +27,7 @@ try ReadBuffer in(src.data(), src.size(), 0); - LimitReadBuffer limit_in(in, 1, false); + LimitReadBuffer limit_in(in, 1, /* trow_exception */ false, /* exact_limit */ {}); { WriteBufferFromString out(dst); @@ -55,7 +55,7 @@ try char x; readChar(x, in); - LimitReadBuffer limit_in(in, 1, false); + LimitReadBuffer limit_in(in, 1, /* trow_exception */ false, /* exact_limit */ {}); copyData(limit_in, out); @@ -85,7 +85,7 @@ try ReadBuffer in(src.data(), src.size(), 0); { - LimitReadBuffer limit_in(in, 1, false); + LimitReadBuffer limit_in(in, 1, /* trow_exception */ false, /* exact_limit */ {}); char x; readChar(x, limit_in); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index aeaacf35051..7fcdf1ff0dc 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1237,7 +1237,7 @@ void executeQuery( /// If not - copy enough data into 'parse_buf'. WriteBufferFromVector> out(parse_buf); - LimitReadBuffer limit(istr, max_query_size + 1, /* trow_exception */ false, /* exact_limit */ false); + LimitReadBuffer limit(istr, max_query_size + 1, /* trow_exception */ false, /* exact_limit */ {}); copyData(limit, out); out.finalize(); diff --git a/src/Server/HTTP/HTMLForm.cpp b/src/Server/HTTP/HTMLForm.cpp index d9d897d20c4..1abf9e5b83e 100644 --- a/src/Server/HTTP/HTMLForm.cpp +++ b/src/Server/HTTP/HTMLForm.cpp @@ -20,6 +20,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; +} + namespace { @@ -229,6 +234,11 @@ void HTMLForm::readMultipart(ReadBuffer & in_, PartHandler & handler) if (!in.skipToNextBoundary()) break; } + + /// It's important to check, because we could get "fake" EOF and incomplete request if a client suddenly died in the middle. + if (!in.isActualEOF()) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF, " + "did not find the last boundary while parsing a multipart HTTP request"); } @@ -244,7 +254,8 @@ bool HTMLForm::MultipartReadBuffer::skipToNextBoundary() if (in.eof()) return false; - assert(boundary_hit); + chassert(boundary_hit); + chassert(!found_last_boundary); boundary_hit = false; @@ -255,7 +266,8 @@ bool HTMLForm::MultipartReadBuffer::skipToNextBoundary() { set(in.position(), 0); next(); /// We need to restrict our buffer to size of next available line. - return !startsWith(line, boundary + "--"); + found_last_boundary = startsWith(line, boundary + "--"); + return !found_last_boundary; } } diff --git a/src/Server/HTTP/HTMLForm.h b/src/Server/HTTP/HTMLForm.h index 16889b41d80..c75dafccaf0 100644 --- a/src/Server/HTTP/HTMLForm.h +++ b/src/Server/HTTP/HTMLForm.h @@ -108,10 +108,13 @@ public: /// Returns false if last boundary found. bool skipToNextBoundary(); + bool isActualEOF() const { return found_last_boundary; } + private: PeekableReadBuffer in; const std::string boundary; bool boundary_hit = true; + bool found_last_boundary = false; std::string readLine(bool append_crlf); diff --git a/src/Server/HTTP/HTTPServerRequest.cpp b/src/Server/HTTP/HTTPServerRequest.cpp index d2e19d594dc..c9ffa3a4c3b 100644 --- a/src/Server/HTTP/HTTPServerRequest.cpp +++ b/src/Server/HTTP/HTTPServerRequest.cpp @@ -56,12 +56,17 @@ HTTPServerRequest::HTTPServerRequest(HTTPContextPtr context, HTTPServerResponse if (getChunkedTransferEncoding()) stream = std::make_unique(std::move(in), context->getMaxChunkSize()); else if (hasContentLength()) - stream = std::make_unique(std::move(in), getContentLength(), /* trow_exception */ true, /* exact_limit */ true); + { + size_t content_length = getContentLength(); + stream = std::make_unique(std::move(in), content_length, + /* trow_exception */ true, /* exact_limit */ content_length); + } else if (getMethod() != HTTPRequest::HTTP_GET && getMethod() != HTTPRequest::HTTP_HEAD && getMethod() != HTTPRequest::HTTP_DELETE) { stream = std::move(in); - LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length, " - "it may be impossible to distinguish graceful EOF from abnormal connection loss"); + if (!startsWith(getContentType(), "multipart/form-data")) + LOG_WARNING(&Poco::Logger::get("HTTPServerRequest"), "Got an HTTP request with no content length " + "and no chunked/multipart encoding, it may be impossible to distinguish graceful EOF from abnormal connection loss"); } else /// We have to distinguish empty buffer and nullptr. diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 279896da9fa..4dcbca93e28 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -155,7 +155,7 @@ void MySQLHandler::run() payload.readStrict(command); // For commands which are executed without MemoryTracker. - LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ false, "too long MySQL packet."); + LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet."); LOG_DEBUG(log, "Received command: {}. Connection id: {}.", static_cast(static_cast(command)), connection_id); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3e89d1810c7..bf5578f2610 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1057,7 +1057,7 @@ bool TCPHandler::receiveProxyHeader() /// Only PROXYv1 is supported. /// Validation of protocol is not fully performed. - LimitReadBuffer limit_in(*in, 107, /* trow_exception */ true, /* exact_limit */ false); /// Maximum length from the specs. + LimitReadBuffer limit_in(*in, 107, /* trow_exception */ true, /* exact_limit */ {}); /// Maximum length from the specs. assertString("PROXY ", limit_in); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 388faf1aeb5..014a0ff2424 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -118,7 +118,7 @@ private: if (limited_by_file_size) { - limited.emplace(*plain, file_size - offset, /* trow_exception */ false, /* exact_limit */ false); + limited.emplace(*plain, file_size - offset, /* trow_exception */ false, /* exact_limit */ std::optional()); compressed.emplace(*limited); } else diff --git a/tests/queries/0_stateless/02435_insert_cancellation_http.sh b/tests/queries/0_stateless/02435_insert_cancellation_http.sh index 653ec59f491..e3a1645db63 100755 --- a/tests/queries/0_stateless/02435_insert_cancellation_http.sh +++ b/tests/queries/0_stateless/02435_insert_cancellation_http.sh @@ -10,24 +10,32 @@ export TEST_MARK="02435_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' -$CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + +function insert_data +{ + SETTINGS="query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0" + TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=1100000&max_block_size=1100000&min_insert_block_size_bytes=0&min_insert_block_size_rows=1100000&max_insert_block_size=1100000" + TYPE=$(( RANDOM % 3 )) + if [[ "$TYPE" -eq 0 ]]; then + $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + elif [[ "$TYPE" -eq 1 ]]; then + $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + else + $CLICKHOUSE_CURL -sS -F 'file=@-' "$CLICKHOUSE_URL&$TRASH_SETTINGS&file_format=TSV&file_types=UInt64" -X POST --form-string 'query=insert into dedup_test select * from file' < $DATA_FILE + fi +} + +export -f insert_data + +insert_data $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' function thread_insert { # supress "Killed" messages from bash - function wrap - { - if (( RANDOM % 2 )); then - $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE - else - $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0&query=insert+into+dedup_test+format+TSV" < $DATA_FILE - fi - } - export -f wrap while true; do export ID="$TEST_MARK$RANDOM" - bash -c wrap 2>&1| grep -Fav "Killed" + bash -c insert_data 2>&1| grep -Fav "Killed" done } From f49a67752ea252a189f073bb329396cdbea58b2f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Feb 2023 13:28:36 +0000 Subject: [PATCH 078/470] Support constant on LHS --- ...ass.cpp => LogicalExpressionOptimizer.cpp} | 53 ++++++++++--------- ...ass.h => LogicalExpressionOptimizerPass.h} | 6 +-- src/Analyzer/QueryTreePassManager.cpp | 4 +- .../00736_disjunction_optimisation.reference | 37 +++++++++++++ .../00736_disjunction_optimisation.sql | 4 ++ ...ssions_optimizer_low_cardinality.reference | 22 ++++++++ ..._expressions_optimizer_low_cardinality.sql | 2 + 7 files changed, 98 insertions(+), 30 deletions(-) rename src/Analyzer/Passes/{OrEqualityChainToInPass.cpp => LogicalExpressionOptimizer.cpp} (63%) rename src/Analyzer/Passes/{OrEqualityChainToInPass.h => LogicalExpressionOptimizerPass.h} (81%) diff --git a/src/Analyzer/Passes/OrEqualityChainToInPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp similarity index 63% rename from src/Analyzer/Passes/OrEqualityChainToInPass.cpp rename to src/Analyzer/Passes/LogicalExpressionOptimizer.cpp index 212e8252ce4..f9744cdc8d0 100644 --- a/src/Analyzer/Passes/OrEqualityChainToInPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -10,12 +10,12 @@ namespace DB { -class OrEqualityChainToInVisitor : public InDepthQueryTreeVisitorWithContext +class LogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWithContext { public: - using Base = InDepthQueryTreeVisitorWithContext; + using Base = InDepthQueryTreeVisitorWithContext; - explicit OrEqualityChainToInVisitor(ContextPtr context) + explicit LogicalExpressionOptimizerVisitor(ContextPtr context) : Base(std::move(context)) {} @@ -34,38 +34,32 @@ public: for (const auto & argument : function_node->getArguments()) { auto * argument_function = argument->as(); - if (!argument_function) + if (!argument_function || argument_function->getFunctionName() != "equals") { or_operands.push_back(argument); continue; } /// collect all equality checks (x = value) - if (argument_function->getFunctionName() != "equals") - { - or_operands.push_back(argument); - continue; - } const auto & equals_arguments = argument_function->getArguments().getNodes(); const auto & lhs = equals_arguments[0]; + const auto & rhs = equals_arguments[1]; - const auto * rhs_literal = equals_arguments[1]->as(); - if (!rhs_literal) - { + if (lhs->as()) + node_to_equals_functions[rhs].push_back(argument); + else if (rhs->as()) + node_to_equals_functions[lhs].push_back(argument); + else or_operands.push_back(argument); - continue; - } - - node_to_equals_functions[lhs].push_back(argument); } auto in_function_resolver = FunctionFactory::instance().get("in", getContext()); - for (auto & [lhs, equals_functions] : node_to_equals_functions) + for (auto & [expression, equals_functions] : node_to_equals_functions) { const auto & settings = getSettings(); - if (equals_functions.size() < settings.optimize_min_equality_disjunction_chain_length && !lhs.node->getResultType()->lowCardinality()) + if (equals_functions.size() < settings.optimize_min_equality_disjunction_chain_length && !expression.node->getResultType()->lowCardinality()) { std::move(equals_functions.begin(), equals_functions.end(), std::back_inserter(or_operands)); continue; @@ -78,9 +72,18 @@ public: { const auto * equals_function = equals->as(); assert(equals_function && equals_function->getFunctionName() == "equals"); - const auto * rhs_literal = equals_function->getArguments().getNodes()[1]->as(); - assert(rhs_literal); - args.push_back(rhs_literal->getValue()); + + const auto & equals_arguments = equals_function->getArguments().getNodes(); + if (const auto * rhs_literal = equals_arguments[1]->as()) + { + args.push_back(rhs_literal->getValue()); + } + else + { + const auto * lhs_literal = equals_arguments[0]->as(); + assert(lhs_literal); + args.push_back(lhs_literal->getValue()); + } } auto rhs_node = std::make_shared(std::move(args)); @@ -89,7 +92,7 @@ public: QueryTreeNodes in_arguments; in_arguments.reserve(2); - in_arguments.push_back(lhs.node); + in_arguments.push_back(expression.node); in_arguments.push_back(std::move(rhs_node)); in_function->getArguments().getNodes() = std::move(in_arguments); @@ -108,9 +111,9 @@ public: }; -void OrEqualityChainToInPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +void LogicalExpressionOptimizerPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - OrEqualityChainToInVisitor visitor(std::move(context)); + LogicalExpressionOptimizerVisitor visitor(std::move(context)); visitor.visit(query_tree_node); } diff --git a/src/Analyzer/Passes/OrEqualityChainToInPass.h b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h similarity index 81% rename from src/Analyzer/Passes/OrEqualityChainToInPass.h rename to src/Analyzer/Passes/LogicalExpressionOptimizerPass.h index 2a9d6818a4f..327ae3c3fee 100644 --- a/src/Analyzer/Passes/OrEqualityChainToInPass.h +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h @@ -8,7 +8,7 @@ namespace DB /** * This pass replaces chains of equality functions inside an OR with a single IN operator. * The replacement is done if: - * - rhs of the equality function is a literal + * - rhs of the equality function is a constant * - length of chain is at least 'optimize_min_equality_disjunction_chain_length' long OR lhs is LowCardinality * * E.g. (optimize_min_equality_disjunction_chain_length = 2) @@ -25,10 +25,10 @@ namespace DB * ------------------------------- */ -class OrEqualityChainToInPass final : public IQueryTreePass +class LogicalExpressionOptimizerPass final : public IQueryTreePass { public: - String getName() override { return "OrEqualityChainToIn"; } + String getName() override { return "LogicalExpressionOptimizer"; } String getDescription() override { return "Transform all the 'or's with equality check to a single IN function"; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index dbbedfd1aaa..da419f0e5bd 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -39,7 +39,7 @@ #include #include #include -#include +#include namespace DB @@ -264,7 +264,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index 4210571842f..fa395dccdc8 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -442,3 +442,40 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +21 1 +22 1 +23 1 +QUERY id: 0 + PROJECTION COLUMNS + s UInt64 + or(equals(s, 21), equals(22, s), equals(23, s)) UInt8 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + JOIN TREE + TABLE id: 3, table_name: default.bug + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.sql b/tests/queries/0_stateless/00736_disjunction_optimisation.sql index e52ed2ab45b..e5bfc81f7ae 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.sql +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.sql @@ -42,4 +42,8 @@ select s, (s=21 or s=22 or s=23) from bug; select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; explain query tree select s, (s=21 or s=22 or s=23) from bug SETTINGS allow_experimental_analyzer = 1;; +select s, (s=21 or 22=s or 23=s) from bug; +select s, (s=21 or 22=s or 23=s) from bug SETTINGS allow_experimental_analyzer = 1;; +explain query tree select s, (s=21 or 22=s or 23=s) from bug SETTINGS allow_experimental_analyzer = 1;; + DROP TABLE bug; diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 086c0a49b59..617bdde183e 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -22,6 +22,28 @@ QUERY id: 0 SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality +WHERE (a = \'x\') OR (\'y\' = a) +QUERY id: 0 + PROJECTION COLUMNS + a LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality + WHERE + FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) + CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 +SELECT a +FROM t_logical_expressions_optimizer_low_cardinality WHERE (b = 0) OR (b = 1) QUERY id: 0 PROJECTION COLUMNS diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql index f5368679f60..14f8ad830e7 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.sql @@ -5,6 +5,8 @@ CREATE TABLE t_logical_expressions_optimizer_low_cardinality (a LowCardinality(S -- LowCardinality case, ignore optimize_min_equality_disjunction_chain_length limit, optimzer applied EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y'; EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR a = 'y' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a; +EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE a = 'x' OR 'y' = a SETTINGS allow_experimental_analyzer = 1; -- Non-LowCardinality case, optimizer not applied for short chains EXPLAIN SYNTAX SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1; EXPLAIN QUERY TREE SELECT a FROM t_logical_expressions_optimizer_low_cardinality WHERE b = 0 OR b = 1 SETTINGS allow_experimental_analyzer = 1; From 88f322ace267dc1c194f85db5be1e94ba29c05c8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 23 Feb 2023 14:50:58 +0100 Subject: [PATCH 079/470] Try fix test --- tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh index e84c06027e8..d27a2f9fcbb 100755 --- a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh +++ b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh echo "CapnProto" -${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format CapnProto settings format_schema='format_schemas/02566_ipv4_ipv6:Message'" > 02566_ipv4_ipv6_data.capnp -${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.capnp, auto, 'ipv6 IPv6, ipv4 IPv4') settings format_schema='format_schemas/02566_ipv4_ipv6:Message'" +${CLICKHOUSE_LOCAL} -q "select '2001:db8:11a3:9d7:1f34:8a2e:7a0:765d'::IPv6 as ipv6, '127.0.0.1'::IPv4 as ipv4 format CapnProto settings format_schema='$CURDIR/format_schemas/02566_ipv4_ipv6:Message'" > 02566_ipv4_ipv6_data.capnp +${CLICKHOUSE_LOCAL} -q "select * from file(02566_ipv4_ipv6_data.capnp, auto, 'ipv6 IPv6, ipv4 IPv4') settings format_schema='$CURDIR/format_schemas/02566_ipv4_ipv6:Message'" rm 02566_ipv4_ipv6_data.capnp echo "Avro" From e37a3801ca4478bb88d21cc9d8eda029ba4dbc8a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 23 Feb 2023 16:04:51 +0000 Subject: [PATCH 080/470] Add new logical optimizations --- .../Passes/LogicalExpressionOptimizer.cpp | 148 ++++++++++++++++-- ...00621_regression_for_in_operator.reference | 4 +- .../00736_disjunction_optimisation.reference | 64 ++++---- ...ssions_optimizer_low_cardinality.reference | 8 +- ...imizer_removing_redundant_checks.reference | 0 ...al_optimizer_removing_redundant_checks.sql | 26 +++ 6 files changed, 202 insertions(+), 48 deletions(-) create mode 100644 tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference create mode 100644 tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp index f9744cdc8d0..c29d38c7865 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp @@ -7,6 +7,8 @@ #include #include +#include + namespace DB { @@ -17,21 +19,115 @@ public: explicit LogicalExpressionOptimizerVisitor(ContextPtr context) : Base(std::move(context)) + , cast_function_resolver(FunctionFactory::instance().get("_CAST", getContext())) {} - void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); - if (!function_node || function_node->getFunctionName() != "or") + if (!function_node) return; + if (function_node->getFunctionName() == "or") + { + tryReplaceOrEqualsChainWithIn(node); + return; + } + + if (function_node->getFunctionName() == "and") + { + tryReplaceAndEqualsChainsWithConstant(node); + return; + } + } +private: + void tryReplaceAndEqualsChainsWithConstant(QueryTreeNodePtr & node) + { + auto & function_node = node->as(); + assert(function_node.getFunctionName() == "and"); + + if (function_node.getResultType()->isNullable()) + return; + + QueryTreeNodes and_operands; + + QueryTreeNodePtrWithHashMap node_to_constants; + + for (const auto & argument : function_node.getArguments()) + { + auto * argument_function = argument->as(); + if (!argument_function || argument_function->getFunctionName() != "equals") + { + and_operands.push_back(argument); + continue; + } + + const auto & equals_arguments = argument_function->getArguments().getNodes(); + const auto & lhs = equals_arguments[0]; + const auto & rhs = equals_arguments[1]; + + const auto has_and_with_different_constant = [&](const QueryTreeNodePtr & expression, const ConstantNode * constant) + { + if (auto it = node_to_constants.find(expression); it != node_to_constants.end()) + { + if (!it->second->isEqual(*constant)) + return true; + } + else + { + node_to_constants.emplace(expression, constant); + and_operands.push_back(argument); + } + + return false; + }; + + bool collapse_to_false = false; + + if (const auto * lhs_literal = lhs->as()) + { + collapse_to_false = has_and_with_different_constant(rhs, lhs_literal); + } + else if (const auto * rhs_literal = rhs->as()) + { + collapse_to_false = has_and_with_different_constant(lhs, rhs_literal); + } + else + continue; + + if (collapse_to_false) + { + auto false_value = std::make_shared(0u, function_node.getResultType()); + auto false_node = std::make_shared(std::move(false_value)); + node = std::move(false_node); + return; + } + } + + if (and_operands.size() == 1) + { + assert(!function_node.getResultType()->isNullable()); + resolveAsCast(function_node, std::move(and_operands[0])); + return; + } + + auto and_function_resolver = FunctionFactory::instance().get("and", getContext()); + function_node.getArguments().getNodes() = std::move(and_operands); + function_node.resolveAsFunction(and_function_resolver); + } + + void tryReplaceOrEqualsChainWithIn(QueryTreeNodePtr & node) + { + auto & function_node = node->as(); + assert(function_node.getFunctionName() == "or"); + QueryTreeNodes or_operands; QueryTreeNodePtrWithHashMap node_to_equals_functions; + QueryTreeNodePtrWithHashMap node_to_constants; - for (const auto & argument : function_node->getArguments()) + for (const auto & argument : function_node.getArguments()) { auto * argument_function = argument->as(); if (!argument_function || argument_function->getFunctionName() != "equals") @@ -46,10 +142,20 @@ public: const auto & lhs = equals_arguments[0]; const auto & rhs = equals_arguments[1]; - if (lhs->as()) - node_to_equals_functions[rhs].push_back(argument); - else if (rhs->as()) - node_to_equals_functions[lhs].push_back(argument); + const auto add_equals_function_if_not_present = [&](const auto & expression_node, const ConstantNode * constant) + { + auto & constant_set = node_to_constants[expression_node]; + if (!constant_set.contains(constant)) + { + constant_set.insert(constant); + node_to_equals_functions[expression_node].push_back(argument); + } + }; + + if (const auto * lhs_literal = lhs->as()) + add_equals_function_if_not_present(rhs, lhs_literal); + else if (const auto * rhs_literal = rhs->as()) + add_equals_function_if_not_present(lhs, rhs_literal); else or_operands.push_back(argument); } @@ -102,12 +208,34 @@ public: } if (or_operands.size() == 1) - or_operands.push_back(std::make_shared(static_cast(0))); + { + assert(!function_node.getResultType()->isNullable()); + resolveAsCast(function_node, std::move(or_operands[0])); + return; + } auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); - function_node->getArguments().getNodes() = std::move(or_operands); - function_node->resolveAsFunction(or_function_resolver); + function_node.getArguments().getNodes() = std::move(or_operands); + function_node.resolveAsFunction(or_function_resolver); } + + void resolveAsCast(FunctionNode & function_node, QueryTreeNodePtr operand) + { + std::string cast_type = function_node.getResultType()->getName(); + auto cast_type_constant_value = std::make_shared(std::move(cast_type), std::make_shared()); + auto cast_type_constant_node = std::make_shared(std::move(cast_type_constant_value)); + + QueryTreeNodes arguments; + arguments.reserve(2); + arguments.push_back(std::move(operand)); + arguments.push_back(std::move(cast_type_constant_node)); + + function_node.getArguments().getNodes() = std::move(arguments); + + function_node.resolveAsFunction(cast_function_resolver); + } + + const FunctionOverloadResolverPtr cast_function_resolver; }; diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index c9eb7a08fc6..54b345b65c0 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -12,7 +12,7 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.regression_for_in_operator_view WHERE - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -20,7 +20,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 8, column_name: g, result_type: String, source_id: 3 CONSTANT id: 9, constant_value: Tuple_(\'5\', \'6\'), constant_value_type: Tuple(String, String) - CONSTANT id: 10, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 10, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 2 2 diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index fa395dccdc8..4b899527913 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -39,7 +39,7 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 6, nodes: 2 - FUNCTION id: 7, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 7, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 8, nodes: 2 FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 @@ -47,8 +47,8 @@ QUERY id: 0 LIST id: 10, nodes: 2 COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 - FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + CONSTANT id: 13, constant_value: \'UInt8\', constant_value_type: String + FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 @@ -56,7 +56,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 20, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -96,7 +96,7 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: default.bug WHERE - FUNCTION id: 9, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 9, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 10, nodes: 2 FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 @@ -104,9 +104,9 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 15, constant_value: \'UInt8\', constant_value_type: String WHERE - FUNCTION id: 16, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 16, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 17, nodes: 2 FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 @@ -114,7 +114,7 @@ QUERY id: 0 LIST id: 19, nodes: 2 COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 22, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 @@ -145,7 +145,7 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 6 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -153,14 +153,14 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 12, nodes: 2 COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 13, constant_value: UInt64_21, constant_value_type: UInt8 - FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 @@ -168,8 +168,8 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 18, constant_value: Tuple_(UInt64_21, UInt64_22), constant_value_type: Tuple(UInt8, UInt8) - CONSTANT id: 19, constant_value: UInt64_0, constant_value_type: UInt8 - FUNCTION id: 20, function_name: or, function_type: ordinary, result_type: UInt8 + CONSTANT id: 19, constant_value: \'UInt8\', constant_value_type: String + FUNCTION id: 20, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 21, nodes: 2 FUNCTION id: 22, function_name: in, function_type: ordinary, result_type: UInt8 @@ -177,7 +177,7 @@ QUERY id: 0 LIST id: 23, nodes: 2 COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 24, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 25, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 25, constant_value: \'UInt8\', constant_value_type: String JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -206,7 +206,7 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -214,7 +214,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -259,7 +259,7 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 6, nodes: 2 - FUNCTION id: 7, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 7, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 8, nodes: 2 FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 @@ -267,8 +267,8 @@ QUERY id: 0 LIST id: 10, nodes: 2 COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 13, constant_value: UInt64_0, constant_value_type: UInt8 - FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 + CONSTANT id: 13, constant_value: \'UInt8\', constant_value_type: String + FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 @@ -276,7 +276,7 @@ QUERY id: 0 LIST id: 17, nodes: 2 COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 20, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 20, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -316,7 +316,7 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: default.bug WHERE - FUNCTION id: 9, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 9, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 10, nodes: 2 FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 @@ -324,9 +324,9 @@ QUERY id: 0 LIST id: 12, nodes: 2 COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 15, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 15, constant_value: \'UInt8\', constant_value_type: String WHERE - FUNCTION id: 16, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 16, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 17, nodes: 2 FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 @@ -334,7 +334,7 @@ QUERY id: 0 LIST id: 19, nodes: 2 COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 22, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 22, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 @@ -365,7 +365,7 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 6 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -373,7 +373,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS @@ -393,7 +393,7 @@ QUERY id: 0 LIST id: 20, nodes: 2 COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 21, constant_value: UInt64_22, constant_value_type: UInt8 - FUNCTION id: 22, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 22, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 23, nodes: 2 FUNCTION id: 24, function_name: in, function_type: ordinary, result_type: UInt8 @@ -401,7 +401,7 @@ QUERY id: 0 LIST id: 25, nodes: 2 COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 26, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 27, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 27, constant_value: \'UInt8\', constant_value_type: String JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -430,7 +430,7 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -438,7 +438,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -467,7 +467,7 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -475,7 +475,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 617bdde183e..22bd68e0ac1 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -10,7 +10,7 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -18,7 +18,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality @@ -32,7 +32,7 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: or, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 @@ -40,7 +40,7 @@ QUERY id: 0 LIST id: 7, nodes: 2 COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) - CONSTANT id: 9, constant_value: UInt64_0, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql new file mode 100644 index 00000000000..f20ef412215 --- /dev/null +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.sql @@ -0,0 +1,26 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS 02668_logical_optimizer; + +CREATE TABLE 02668_logical_optimizer +(a Int32, b LowCardinality(String)) +ENGINE=Memory; + +INSERT INTO 02668_logical_optimizer VALUES (1, 'test'), (2, 'test2'), (3, 'another'); + +SET optimize_min_equality_disjunction_chain_length = 2; + +SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 3 = a OR 1 = a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 3 = a OR 1 = a; + +SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 1 = a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 OR 1 = a; + +SELECT * FROM 02668_logical_optimizer WHERE a = 1 AND 2 = a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 1 AND 2 = a; + +SELECT * FROM 02668_logical_optimizer WHERE 3 = a AND b = 'another' AND a = 3; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 3 AND b = 'another' AND a = 3; + +SELECT * FROM 02668_logical_optimizer WHERE a = 2 AND 2 = a; +EXPLAIN QUERY TREE SELECT * FROM 02668_logical_optimizer WHERE a = 2 AND 2 = a; From d21a6a3ba40f2259aba17ff26acf930814a35f16 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Feb 2023 20:42:46 +0100 Subject: [PATCH 081/470] fix --- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Executors/PushingPipelineExecutor.cpp | 2 +- ..._cancel_insert_when_client_dies.reference} | 1 + ...> 02434_cancel_insert_when_client_dies.sh} | 29 +++++++-- ...434_insert_cancellation_native_protocol.sh | 65 ------------------- .../02435_insert_cancellation_http.reference | 2 - utils/check-mysql-binlog/main.cpp | 3 +- 7 files changed, 27 insertions(+), 77 deletions(-) rename tests/queries/0_stateless/{02434_insert_cancellation_native_protocol.reference => 02434_cancel_insert_when_client_dies.reference} (88%) rename tests/queries/0_stateless/{02435_insert_cancellation_http.sh => 02434_cancel_insert_when_client_dies.sh} (59%) delete mode 100755 tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh delete mode 100644 tests/queries/0_stateless/02435_insert_cancellation_http.reference diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 20f47c6b54c..70815bb8b3b 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -140,7 +140,7 @@ PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(QueryPipeline & pipel PushingAsyncPipelineExecutor::~PushingAsyncPipelineExecutor() { /// It must be finalized explicitly. Otherwise we cancel it assuming it's due to an exception. - chassert(finished || std::uncaught_exceptions()); + chassert(finished || std::uncaught_exceptions() || std::current_exception()); try { cancel(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index ca60932ed6f..696932932df 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -64,7 +64,7 @@ PushingPipelineExecutor::PushingPipelineExecutor(QueryPipeline & pipeline_) : pi PushingPipelineExecutor::~PushingPipelineExecutor() { /// It must be finalized explicitly. Otherwise we cancel it assuming it's due to an exception. - chassert(finished || std::uncaught_exceptions()); + chassert(finished || std::uncaught_exceptions() || std::current_exception()); try { cancel(); diff --git a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.reference similarity index 88% rename from tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference rename to tests/queries/0_stateless/02434_cancel_insert_when_client_dies.reference index e3038e03530..d2475419998 100644 --- a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.reference +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.reference @@ -1,2 +1,3 @@ 5000000 5000000 +1 diff --git a/tests/queries/0_stateless/02435_insert_cancellation_http.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh similarity index 59% rename from tests/queries/0_stateless/02435_insert_cancellation_http.sh rename to tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index e3a1645db63..c2cf98f712f 100755 --- a/tests/queries/0_stateless/02435_insert_cancellation_http.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -6,19 +7,24 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" -export TEST_MARK="02435_insert_${CLICKHOUSE_DATABASE}_" +export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' function insert_data { - SETTINGS="query_id=$ID&max_insert_block_size=100000&input_format_parallel_parsing=0" - TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=1100000&max_block_size=1100000&min_insert_block_size_bytes=0&min_insert_block_size_rows=1100000&max_insert_block_size=1100000" - TYPE=$(( RANDOM % 3 )) + SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000" + # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) + TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" + TYPE=$(( RANDOM % 4 )) + if [[ "$TYPE" -eq 0 ]]; then - $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + # client will send 10000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) + $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" -q 'insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 1 ]]; then + $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + elif [[ "$TYPE" -eq 2 ]]; then $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE else $CLICKHOUSE_CURL -sS -F 'file=@-' "$CLICKHOUSE_URL&$TRASH_SETTINGS&file_format=TSV&file_types=UInt64" -X POST --form-string 'query=insert into dedup_test select * from file' < $DATA_FILE @@ -27,6 +33,7 @@ function insert_data export -f insert_data +ID="02434_insert_init_${CLICKHOUSE_DATABASE}_$RANDOM" insert_data $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' @@ -66,12 +73,20 @@ export -f thread_insert; export -f thread_select; export -f thread_cancel; -TIMEOUT=30 +TIMEOUT=40 # 10 seconds for each TYPE timeout $TIMEOUT bash -c thread_insert & timeout $TIMEOUT bash -c thread_select & -timeout $TIMEOUT bash -c thread_cancel & +timeout $TIMEOUT bash -c thread_cancel 2> /dev/null & wait $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' + +$CLICKHOUSE_CLIENT -q 'system flush logs' + +# We have to ignore stderr from thread_cancel, because our CI finds a bug in ps... +# So use this query to check that thread_cancel do something +$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( + message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or + message like '%Connection reset by peer%')" diff --git a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh b/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh deleted file mode 100755 index 4eb02c38c1a..00000000000 --- a/tests/queries/0_stateless/02434_insert_cancellation_native_protocol.sh +++ /dev/null @@ -1,65 +0,0 @@ -#!/usr/bin/env bash -# shellcheck disable=SC2009 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" -export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" - -$CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE -$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' -$CLICKHOUSE_CLIENT --max_block_size=100000 --min_chunk_bytes_for_parallel_parsing=10000 -q 'insert into dedup_test format TSV' < $DATA_FILE -$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' - -function thread_insert -{ - # supress "Killed" messages from bash - function wrap - { - $CLICKHOUSE_CLIENT --max_block_size=100000 --min_chunk_bytes_for_parallel_parsing=10000 --query_id="$ID" -q 'insert into dedup_test format TSV' < $DATA_FILE - } - export -f wrap - while true; do - export ID="$TEST_MARK$RANDOM" - bash -c wrap 2>&1| grep -Fav "Killed" - done -} - -function thread_select -{ - while true; do - $CLICKHOUSE_CLIENT -q "with (select count() from dedup_test) as c select throwIf(c != 5000000, 'Expected 5000000 rows, got ' || toString(c)) format Null" - sleep 0.$RANDOM; - done -} - -function thread_cancel -{ - while true; do - SIGNAL="INT" - if (( RANDOM % 2 )); then - SIGNAL="KILL" - fi - PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') - if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID" || echo "$PID"; fi - sleep 0.$RANDOM; - sleep 0.$RANDOM; - sleep 0.$RANDOM; - done -} - -export -f thread_insert; -export -f thread_select; -export -f thread_cancel; - -TIMEOUT=30 - -timeout $TIMEOUT bash -c thread_insert & -timeout $TIMEOUT bash -c thread_select & -timeout $TIMEOUT bash -c thread_cancel & - -wait - -$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' diff --git a/tests/queries/0_stateless/02435_insert_cancellation_http.reference b/tests/queries/0_stateless/02435_insert_cancellation_http.reference deleted file mode 100644 index e3038e03530..00000000000 --- a/tests/queries/0_stateless/02435_insert_cancellation_http.reference +++ /dev/null @@ -1,2 +0,0 @@ -5000000 -5000000 diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index 7dd387ba5be..cf2a27e8aac 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -17,7 +17,8 @@ static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( std::shared_ptr & last_table_map_event, bool exist_checksum) { DB::MySQLReplication::BinlogEventPtr event; - DB::ReadBufferPtr limit_read_buffer = std::make_shared(payload, header.event_size - 19, false); + DB::ReadBufferPtr limit_read_buffer = std::make_shared(payload, header.event_size - 19, + /* trow_exception */ false, /* exact_limit */ {}); DB::ReadBufferPtr event_payload = std::make_shared(*limit_read_buffer, exist_checksum ? 4 : 0); switch (header.type) From 240e0070e5b4ea55b22a687fcad375131cbdad70 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 23 Feb 2023 23:07:35 +0100 Subject: [PATCH 082/470] fix --- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index ac8101d4ca2..78b173de6dc 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -210,7 +210,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// to avoid buffering of huge amount of data in memory. auto read_buf = getReadBufferFromASTInsertQuery(query); - LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, false); + LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* trow_exception */ false, /* exact_limit */ {}); WriteBufferFromString write_buf(bytes); copyData(limit_buf, write_buf); From 5e5e802348af021c6dbeefd5a05e2ad98c1a21bf Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 23 Feb 2023 22:54:50 +0000 Subject: [PATCH 083/470] return back optimization for ALTER MODIFY Nullable --- src/Storages/MergeTree/MutateTask.cpp | 45 +++++++++++++++++-- .../02669_alter_modify_to_nullable.reference | 8 ++++ .../02669_alter_modify_to_nullable.sql | 31 +++++++++++++ 3 files changed, 80 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02669_alter_modify_to_nullable.reference create mode 100644 tests/queries/0_stateless/02669_alter_modify_to_nullable.sql diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8d95a76b5da..de623c58538 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include @@ -1568,6 +1569,45 @@ bool MutateTask::execute() return false; } +static bool canSkipConversionToNullable(const MergeTreeDataPartPtr & part, const MutationCommand & command) +{ + if (command.type != MutationCommand::READ_COLUMN) + return false; + + auto part_column = part->tryGetColumn(command.column_name); + if (!part_column) + return false; + + /// For ALTER MODIFY COLUMN from 'Type' to 'Nullable(Type)' we can skip mutatation and + /// apply only metadata conversion. But it doesn't work for custom serialization. + const auto * to_nullable = typeid_cast(command.data_type.get()); + if (!to_nullable) + return false; + + if (!part_column->type->equals(*to_nullable->getNestedType())) + return false; + + auto serialization = part->getSerialization(command.column_name); + if (serialization->getKind() != ISerialization::Kind::DEFAULT) + return false; + + return true; +} + +static bool canSkipMutationCommandForPart(const MergeTreeDataPartPtr & part, const MutationCommand & command, const ContextPtr & context) +{ + if (command.partition) + { + auto command_partition_id = part->storage.getPartitionIDFromQuery(command.partition, context); + if (part->info.partition_id != command_partition_id) + return true; + } + + if (canSkipConversionToNullable(part, command)) + return true; + + return false; +} bool MutateTask::prepare() { @@ -1586,11 +1626,8 @@ bool MutateTask::prepare() context_for_reading->setSetting("force_primary_key", false); for (const auto & command : *ctx->commands) - { - if (command.partition == nullptr || ctx->source_part->info.partition_id == ctx->data->getPartitionIDFromQuery( - command.partition, context_for_reading)) + if (!canSkipMutationCommandForPart(ctx->source_part, command, context_for_reading)) 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)) diff --git a/tests/queries/0_stateless/02669_alter_modify_to_nullable.reference b/tests/queries/0_stateless/02669_alter_modify_to_nullable.reference new file mode 100644 index 00000000000..aff80e1d699 --- /dev/null +++ b/tests/queries/0_stateless/02669_alter_modify_to_nullable.reference @@ -0,0 +1,8 @@ +1_1_1_0 String Default +2_2_2_0 String Sparse +20000 10435 ['','bar','foo'] +1_1_1_0_3 String Default +2_2_2_0_3 Nullable(String) Default +20000 10435 ['','bar','foo'] +1_1_1_0_3 0 +2_2_2_0_3 10000 diff --git a/tests/queries/0_stateless/02669_alter_modify_to_nullable.sql b/tests/queries/0_stateless/02669_alter_modify_to_nullable.sql new file mode 100644 index 00000000000..862280fd7cd --- /dev/null +++ b/tests/queries/0_stateless/02669_alter_modify_to_nullable.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS t_modify_to_nullable; + +CREATE TABLE t_modify_to_nullable (key UInt64, id UInt64, s String) +ENGINE = MergeTree ORDER BY id PARTITION BY key +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_modify_to_nullable SELECT 1, number, 'foo' FROM numbers(10000); +INSERT INTO t_modify_to_nullable SELECT 2, number, if (number % 23 = 0, 'bar', '') FROM numbers(10000); + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_modify_to_nullable' AND column = 's' AND active +ORDER BY name; + +SELECT count(s), countIf(s != ''), arraySort(groupUniqArray(s)) FROM t_modify_to_nullable; + +SET mutations_sync = 2; +ALTER TABLE t_modify_to_nullable MODIFY COLUMN s Nullable(String); + +SELECT name, type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_modify_to_nullable' AND column = 's' AND active +ORDER BY name; + +SELECT count(s), countIf(s != ''), arraySort(groupUniqArray(s)) FROM t_modify_to_nullable; + +SYSTEM FLUSH LOGS; + +SELECT part_name, read_rows FROM system.part_log +WHERE database = currentDatabase() AND table = 't_modify_to_nullable' AND event_type = 'MutatePart' +ORDER BY part_name; + +DROP TABLE t_modify_to_nullable; From 4de9c9152940e334bb7a971b1bf6fc6bacb6499c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Feb 2023 01:48:06 +0100 Subject: [PATCH 084/470] add a test for transactions --- .../02434_cancel_insert_when_client_dies.sh | 2 +- ...02435_rollback_cancelled_queries.reference | 3 + .../02435_rollback_cancelled_queries.sh | 118 ++++++++++++++++++ 3 files changed, 122 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02435_rollback_cancelled_queries.reference create mode 100755 tests/queries/0_stateless/02435_rollback_cancelled_queries.sh diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index c2cf98f712f..6cae90a3cc3 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -62,7 +62,7 @@ function thread_cancel SIGNAL="KILL" fi PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') - if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID" || echo "$PID"; fi + if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID"; fi sleep 0.$RANDOM; sleep 0.$RANDOM; sleep 0.$RANDOM; diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference new file mode 100644 index 00000000000..94c627bb5d8 --- /dev/null +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference @@ -0,0 +1,3 @@ +1000000 +0 1 +1 diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh new file mode 100755 index 00000000000..bc16c54cd1e --- /dev/null +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -0,0 +1,118 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-ordinary-database +# shellcheck disable=SC2009 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export DATA_FILE="$CLICKHOUSE_TMP/deduptest.tsv" +export TEST_MARK="02435_insert_${CLICKHOUSE_DATABASE}_" +export SESSION="02435_session_${CLICKHOUSE_DATABASE}" + +$CLICKHOUSE_CLIENT -q 'select * from numbers(1000000) format TSV' > $DATA_FILE +$CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by sin(A) partition by intDiv(A, 100000)' + +function insert_data +{ + IMPLICIT=$(( RANDOM % 2 )) + SESSION_ID="${SESSION}_$RANDOM$RANDOM$RANDOM" + TXN_SETTINGS="session_id=$SESSION_ID&throw_on_unsupported_query_inside_transaction=0" + BEGIN="" + COMMIT="" + SETTINGS="query_id=$ID&$TXN_SETTINGS&max_insert_block_size=110000&min_insert_block_size_rows=110000" + if [[ "$IMPLICIT" -eq 0 ]]; then + $CLICKHOUSE_CURL -sS -d 'begin transaction' "$CLICKHOUSE_URL&$TXN_SETTINGS" + BEGIN="begin transaction;" + COMMIT=$(echo -ne "\n\ncommit") + else + TXN_SETTINGS="$TXN_SETTINGS&implicit_transaction=1" + fi + + SETTINGS="query_id=$ID&$TXN_SETTINGS&max_insert_block_size=110000&min_insert_block_size_rows=110000" + # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) + TRASH_SETTINGS="query_id=$ID&$TXN_SETTINGS&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" + TYPE=$(( RANDOM % 6 )) + + if [[ "$TYPE" -eq 0 ]]; then + $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + elif [[ "$TYPE" -eq 1 ]]; then + $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + elif [[ "$TYPE" -eq 2 ]]; then + $CLICKHOUSE_CURL -sS -F 'file=@-' "$CLICKHOUSE_URL&$TRASH_SETTINGS&file_format=TSV&file_types=UInt64" -X POST --form-string 'query=insert into dedup_test select * from file' < $DATA_FILE + else + # client will send 1000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) + $CLICKHOUSE_CLIENT --query_id="$ID" --throw_on_unsupported_query_inside_transaction=0 --implicit_transaction="$IMPLICIT" \ + --max_block_size=1000 --max_insert_block_size=1000 --multiquery -q \ + "${BEGIN}insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV$COMMIT" < $DATA_FILE \ + | grep -Fv "Transaction is not in RUNNING state" + fi + + if [[ "$IMPLICIT" -eq 0 ]]; then + $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS" | grep -Faq "Transaction is not in RUNNING state" && $CLICKHOUSE_CURL -sS -d 'rollback' "$CLICKHOUSE_URL&$TXN_SETTINGS" + fi +} + +export -f insert_data + +ID="02435_insert_init_${CLICKHOUSE_DATABASE}_$RANDOM" +insert_data +$CLICKHOUSE_CLIENT -q 'select count() from dedup_test' + +function thread_insert +{ + # supress "Killed" messages from bash + while true; do + export ID="$TEST_MARK$RANDOM" + bash -c insert_data 2>&1| grep -Fav "Killed" + done +} + +function thread_select +{ + while true; do + $CLICKHOUSE_CLIENT --implicit_transaction=1 -q "with (select count() from dedup_test) as c select throwIf(c % 1000000 != 0, 'Expected 1000000 * N rows, got ' || toString(c)) format Null" + sleep 0.$RANDOM; + done +} + +function thread_cancel +{ + while true; do + SIGNAL="INT" + if (( RANDOM % 2 )); then + SIGNAL="KILL" + fi + PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') + if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID"; fi + sleep 0.$RANDOM; + done +} + +export -f thread_insert; +export -f thread_select; +export -f thread_cancel; + +TIMEOUT=20 # 5 seconds for each TYPE + +timeout $TIMEOUT bash -c thread_insert & +timeout $TIMEOUT bash -c thread_select & +timeout $TIMEOUT bash -c thread_cancel 2> /dev/null & + +wait +wait_for_queries_to_finish + +$CLICKHOUSE_CLIENT -q 'system flush logs' + +ID="02435_insert_last_${CLICKHOUSE_DATABASE}_$RANDOM" +insert_data + +$CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select count() % 1000000, count() > 0 from dedup_test' + +# We have to ignore stderr from thread_cancel, because our CI finds a bug in ps... +# So use this query to check that thread_cancel do something +$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( + message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or + message like '%Connection reset by peer%')" + +$CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" From 6f4c742408f77202c0740829d0d10d2d756ea9e9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Feb 2023 02:42:45 +0100 Subject: [PATCH 085/470] fix --- .../0_stateless/02435_rollback_cancelled_queries.reference | 2 +- .../queries/0_stateless/02435_rollback_cancelled_queries.sh | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference index 94c627bb5d8..2d32c17ec7c 100644 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference @@ -1,3 +1,3 @@ 1000000 -0 1 +0 1 diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index bc16c54cd1e..18317960cdf 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -42,7 +42,7 @@ function insert_data $CLICKHOUSE_CURL -sS -F 'file=@-' "$CLICKHOUSE_URL&$TRASH_SETTINGS&file_format=TSV&file_types=UInt64" -X POST --form-string 'query=insert into dedup_test select * from file' < $DATA_FILE else # client will send 1000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) - $CLICKHOUSE_CLIENT --query_id="$ID" --throw_on_unsupported_query_inside_transaction=0 --implicit_transaction="$IMPLICIT" \ + $CLICKHOUSE_CLIENT --stacktrace --query_id="$ID" --throw_on_unsupported_query_inside_transaction=0 --implicit_transaction="$IMPLICIT" \ --max_block_size=1000 --max_insert_block_size=1000 --multiquery -q \ "${BEGIN}insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV$COMMIT" < $DATA_FILE \ | grep -Fv "Transaction is not in RUNNING state" @@ -107,7 +107,8 @@ $CLICKHOUSE_CLIENT -q 'system flush logs' ID="02435_insert_last_${CLICKHOUSE_DATABASE}_$RANDOM" insert_data -$CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select count() % 1000000, count() > 0 from dedup_test' +$CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ + || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" # We have to ignore stderr from thread_cancel, because our CI finds a bug in ps... # So use this query to check that thread_cancel do something From 7122ebab4d696b45add7f3d40b066e31a4093b18 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Feb 2023 03:06:33 +0100 Subject: [PATCH 086/470] fix clickhouse-test --- tests/clickhouse-test | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f5d1099c3f4..cfd2546bbdd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -73,13 +73,7 @@ def stringhash(s): # First and last lines of the log def trim_for_log(s): - if not s: - return s - lines = s.splitlines() - if len(lines) > 100: - return "\n".join(lines[:50] + ["#" * 100] + lines[-50:]) - else: - return "\n".join(lines) + return s class HTTPError(Exception): From 8227a6e7bfbdaf0d9eb9f71be909dccf3013a3cc Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Feb 2023 09:12:08 +0000 Subject: [PATCH 087/470] Fix tests --- .../Passes/LogicalExpressionOptimizer.cpp | 30 +-- .../Passes/LogicalExpressionOptimizerPass.h | 50 ++++- ...00621_regression_for_in_operator.reference | 10 +- .../00736_disjunction_optimisation.reference | 210 ++++++------------ ...ssions_optimizer_low_cardinality.reference | 20 +- ...imizer_removing_redundant_checks.reference | 89 ++++++++ 6 files changed, 221 insertions(+), 188 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp index c29d38c7865..85b44de74da 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp @@ -19,7 +19,6 @@ public: explicit LogicalExpressionOptimizerVisitor(ContextPtr context) : Base(std::move(context)) - , cast_function_resolver(FunctionFactory::instance().get("_CAST", getContext())) {} void visitImpl(QueryTreeNodePtr & node) @@ -86,15 +85,11 @@ private: bool collapse_to_false = false; if (const auto * lhs_literal = lhs->as()) - { collapse_to_false = has_and_with_different_constant(rhs, lhs_literal); - } else if (const auto * rhs_literal = rhs->as()) - { collapse_to_false = has_and_with_different_constant(lhs, rhs_literal); - } else - continue; + and_operands.push_back(argument); if (collapse_to_false) { @@ -108,7 +103,7 @@ private: if (and_operands.size() == 1) { assert(!function_node.getResultType()->isNullable()); - resolveAsCast(function_node, std::move(and_operands[0])); + node = std::move(and_operands[0]); return; } @@ -210,7 +205,7 @@ private: if (or_operands.size() == 1) { assert(!function_node.getResultType()->isNullable()); - resolveAsCast(function_node, std::move(or_operands[0])); + node = std::move(or_operands[0]); return; } @@ -218,27 +213,8 @@ private: function_node.getArguments().getNodes() = std::move(or_operands); function_node.resolveAsFunction(or_function_resolver); } - - void resolveAsCast(FunctionNode & function_node, QueryTreeNodePtr operand) - { - std::string cast_type = function_node.getResultType()->getName(); - auto cast_type_constant_value = std::make_shared(std::move(cast_type), std::make_shared()); - auto cast_type_constant_node = std::make_shared(std::move(cast_type_constant_value)); - - QueryTreeNodes arguments; - arguments.reserve(2); - arguments.push_back(std::move(operand)); - arguments.push_back(std::move(cast_type_constant_node)); - - function_node.getArguments().getNodes() = std::move(arguments); - - function_node.resolveAsFunction(cast_function_resolver); - } - - const FunctionOverloadResolverPtr cast_function_resolver; }; - void LogicalExpressionOptimizerPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { LogicalExpressionOptimizerVisitor visitor(std::move(context)); diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h index 327ae3c3fee..b436c94fe4c 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h @@ -6,10 +6,12 @@ namespace DB { /** - * This pass replaces chains of equality functions inside an OR with a single IN operator. + * This pass tries to do optimizations on logical expression + * + * Replaces chains of equality functions inside an OR with a single IN operator. * The replacement is done if: - * - rhs of the equality function is a constant - * - length of chain is at least 'optimize_min_equality_disjunction_chain_length' long OR lhs is LowCardinality + * - one of the operands of the equality function is a constant + * - length of chain is at least 'optimize_min_equality_disjunction_chain_length' long OR the expression has type of LowCardinality * * E.g. (optimize_min_equality_disjunction_chain_length = 2) * ------------------------------- @@ -23,6 +25,48 @@ namespace DB * FROM TABLE * WHERE b = 'test' OR a IN (1, 2); * ------------------------------- + * + * Removes duplicate OR checks + * ------------------------------- + * SELECT * + * FROM table + * WHERE a = 1 OR b = 'test' OR a = 1; + * + * will be transformed into + * + * SELECT * + * FROM TABLE + * WHERE a = 1 OR b = 'test'; + * ------------------------------- + * + * Replaces AND chains with a single constant. + * The replacement is done if: + * - one of the operands of the equality function is a constant + * - constants are different for same expression + * ------------------------------- + * SELECT * + * FROM table + * WHERE a = 1 AND b = 'test' AND a = 2; + * + * will be transformed into + * + * SELECT * + * FROM TABLE + * WHERE 0; + * ------------------------------- + * + * Removes duplicate AND checks + * ------------------------------- + * SELECT * + * FROM table + * WHERE a = 1 AND b = 'test' AND a = 1; + * + * will be transformed into + * + * SELECT * + * FROM TABLE + * WHERE a = 1 AND b = 'test'; + * ------------------------------- */ class LogicalExpressionOptimizerPass final : public IQueryTreePass diff --git a/tests/queries/0_stateless/00621_regression_for_in_operator.reference b/tests/queries/0_stateless/00621_regression_for_in_operator.reference index 54b345b65c0..ab8bcf307eb 100644 --- a/tests/queries/0_stateless/00621_regression_for_in_operator.reference +++ b/tests/queries/0_stateless/00621_regression_for_in_operator.reference @@ -12,15 +12,11 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.regression_for_in_operator_view WHERE - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 8, column_name: g, result_type: String, source_id: 3 - CONSTANT id: 9, constant_value: Tuple_(\'5\', \'6\'), constant_value_type: Tuple(String, String) - CONSTANT id: 10, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 6, column_name: g, result_type: String, source_id: 3 + CONSTANT id: 7, constant_value: Tuple_(\'5\', \'6\'), constant_value_type: Tuple(String, String) SETTINGS allow_experimental_analyzer=1 2 2 diff --git a/tests/queries/0_stateless/00736_disjunction_optimisation.reference b/tests/queries/0_stateless/00736_disjunction_optimisation.reference index 4b899527913..84477a64057 100644 --- a/tests/queries/0_stateless/00736_disjunction_optimisation.reference +++ b/tests/queries/0_stateless/00736_disjunction_optimisation.reference @@ -39,24 +39,16 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 6, nodes: 2 - FUNCTION id: 7, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 7, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 8, nodes: 2 - FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 - CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 13, constant_value: \'UInt8\', constant_value_type: String - FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 + COLUMN id: 9, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 10, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 15, nodes: 2 - FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 20, constant_value: \'UInt8\', constant_value_type: String + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 14, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -96,25 +88,17 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: default.bug WHERE - FUNCTION id: 9, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 10, nodes: 2 - FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 12, nodes: 2 - COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 - CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 15, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) WHERE - FUNCTION id: 16, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 13, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 22, constant_value: \'UInt8\', constant_value_type: String + LIST id: 14, nodes: 2 + COLUMN id: 15, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) SETTINGS allow_experimental_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 @@ -145,39 +129,27 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 6 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 8, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 10, constant_value: UInt64_21, constant_value_type: UInt8 + FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 12, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_21, constant_value_type: UInt8 - FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 13, constant_value: Tuple_(UInt64_21, UInt64_22), constant_value_type: Tuple(UInt8, UInt8) + FUNCTION id: 14, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 15, nodes: 2 - FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 18, constant_value: Tuple_(UInt64_21, UInt64_22), constant_value_type: Tuple(UInt8, UInt8) - CONSTANT id: 19, constant_value: \'UInt8\', constant_value_type: String - FUNCTION id: 20, function_name: _CAST, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 21, nodes: 2 - FUNCTION id: 22, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 23, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 24, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 25, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -206,15 +178,11 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -259,24 +227,16 @@ QUERY id: 0 FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 6, nodes: 2 - FUNCTION id: 7, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 7, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 8, nodes: 2 - FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 10, nodes: 2 - COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 3 - CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 13, constant_value: \'UInt8\', constant_value_type: String - FUNCTION id: 14, function_name: _CAST, function_type: ordinary, result_type: UInt8 + COLUMN id: 9, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 10, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 15, nodes: 2 - FUNCTION id: 16, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 17, nodes: 2 - COLUMN id: 18, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 19, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 20, constant_value: \'UInt8\', constant_value_type: String + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 14, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) SETTINGS allow_experimental_analyzer=1 1 21 1 22 @@ -316,25 +276,17 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: default.bug WHERE - FUNCTION id: 9, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 9, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 10, nodes: 2 - FUNCTION id: 11, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 12, nodes: 2 - COLUMN id: 13, column_name: k, result_type: UInt64, source_id: 7 - CONSTANT id: 14, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 15, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 11, column_name: k, result_type: UInt64, source_id: 7 + CONSTANT id: 12, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) WHERE - FUNCTION id: 16, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 13, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 17, nodes: 2 - FUNCTION id: 18, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 19, nodes: 2 - COLUMN id: 20, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 22, constant_value: \'UInt8\', constant_value_type: String + LIST id: 14, nodes: 2 + COLUMN id: 15, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 16, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) SETTINGS allow_experimental_analyzer=1 1 1 21 1 1 1 1 1 22 0 1 1 @@ -365,43 +317,35 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 6 COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 + COLUMN id: 2, column_name: k, result_type: UInt64, source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(UInt64_1, UInt64_2, UInt64_3), constant_value_type: Tuple(UInt8, UInt8, UInt8) + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + FUNCTION id: 8, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 10, constant_value: UInt64_21, constant_value_type: UInt8 + FUNCTION id: 11, function_name: or, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 12, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 13, constant_value: UInt64_21, constant_value_type: UInt8 - FUNCTION id: 14, function_name: or, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 15, nodes: 2 + FUNCTION id: 13, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_21, constant_value_type: UInt8 FUNCTION id: 16, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 17, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 18, constant_value: UInt64_21, constant_value_type: UInt8 - FUNCTION id: 19, function_name: equals, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 20, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 21, constant_value: UInt64_22, constant_value_type: UInt8 - FUNCTION id: 22, function_name: _CAST, function_type: ordinary, result_type: UInt8 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_22, constant_value_type: UInt8 + FUNCTION id: 19, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 23, nodes: 2 - FUNCTION id: 24, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 25, nodes: 2 - COLUMN id: 10, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 26, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 27, constant_value: \'UInt8\', constant_value_type: String + LIST id: 20, nodes: 2 + COLUMN id: 7, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 21, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -430,15 +374,11 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 @@ -467,15 +407,11 @@ QUERY id: 0 PROJECTION LIST id: 1, nodes: 2 COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 2, column_name: s, result_type: UInt64, source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(UInt64_21, UInt64_22, UInt64_23), constant_value_type: Tuple(UInt8, UInt8, UInt8) JOIN TREE TABLE id: 3, table_name: default.bug SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference index 22bd68e0ac1..84589668d64 100644 --- a/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference +++ b/tests/queries/0_stateless/02477_logical_expressions_optimizer_low_cardinality.reference @@ -10,15 +10,11 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality @@ -32,15 +28,11 @@ QUERY id: 0 JOIN TREE TABLE id: 3, table_name: default.t_logical_expressions_optimizer_low_cardinality WHERE - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: UInt8 + FUNCTION id: 4, function_name: in, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 5, nodes: 2 - FUNCTION id: 6, function_name: in, function_type: ordinary, result_type: UInt8 - ARGUMENTS - LIST id: 7, nodes: 2 - COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 - CONSTANT id: 8, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) - CONSTANT id: 9, constant_value: \'UInt8\', constant_value_type: String + COLUMN id: 2, column_name: a, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 6, constant_value: Tuple_(\'x\', \'y\'), constant_value_type: Tuple(String, String) SETTINGS allow_experimental_analyzer=1 SELECT a FROM t_logical_expressions_optimizer_low_cardinality diff --git a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference index e69de29bb2d..d083e178586 100644 --- a/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference +++ b/tests/queries/0_stateless/02668_logical_optimizer_removing_redundant_checks.reference @@ -0,0 +1,89 @@ +1 test +3 another +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(UInt64_1, UInt64_3), constant_value_type: Tuple(UInt8, UInt8) +1 test +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02668_logical_optimizer + WHERE + CONSTANT id: 5, constant_value: UInt64_0, constant_value_type: UInt8 +3 another +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + FUNCTION id: 7, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + COLUMN id: 9, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 10, constant_value: UInt64_3, constant_value_type: UInt8 + FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: LowCardinality(UInt8) + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: b, result_type: LowCardinality(String), source_id: 3 + CONSTANT id: 14, constant_value: \'another\', constant_value_type: String +2 test2 +QUERY id: 0 + PROJECTION COLUMNS + a Int32 + b LowCardinality(String) + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: a, result_type: Int32, source_id: 3 + COLUMN id: 4, column_name: b, result_type: LowCardinality(String), source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.02668_logical_optimizer + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: a, result_type: Int32, source_id: 3 + CONSTANT id: 8, constant_value: UInt64_2, constant_value_type: UInt8 From a587bcbb8db253647949b6fb657ae9ba85bcdc9b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Feb 2023 12:14:51 +0000 Subject: [PATCH 088/470] Use initial range for fetching file size --- src/IO/ReadWriteBufferFromHTTP.h | 28 ++++++++++++++++++++-------- 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 5bc72c5ff62..dd8a3376607 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -118,6 +118,7 @@ namespace detail bool use_external_buffer; size_t offset_from_begin_pos = 0; + const Range initial_read_range; Range read_range; std::optional file_size; @@ -145,6 +146,7 @@ namespace detail size_t getOffset() const { return getRangeBegin() + offset_from_begin_pos; } + template std::istream * callImpl(Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) { // With empty path poco will send "POST HTTP/1.1" its bug. @@ -154,6 +156,12 @@ namespace detail Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header + Range range; + if constexpr (use_initial_range) + range = Range{initial_read_range.begin.value_or(0), initial_read_range.end}; + else + range = Range{getOffset(), read_range.end}; + if (out_stream_callback) request.setChunkedTransferEncoding(true); @@ -163,10 +171,10 @@ namespace detail if (withPartialContent()) { String range_header_value; - if (read_range.end) - range_header_value = fmt::format("bytes={}-{}", getOffset(), *read_range.end); + if (range.end) + range_header_value = fmt::format("bytes={}-{}", *range.begin, *range.end); else - range_header_value = fmt::format("bytes={}-", getOffset()); + range_header_value = fmt::format("bytes={}-", *range.begin); LOG_TEST(log, "Adding header: Range: {}", range_header_value); request.set("Range", range_header_value); } @@ -206,7 +214,7 @@ namespace detail return *file_size; Poco::Net::HTTPResponse response; - getHeadResponse(response); + getHeadResponse(response); if (response.hasContentLength()) { @@ -235,13 +243,14 @@ namespace detail InitializeError initialization_error = InitializeError::NONE; private: + template void getHeadResponse(Poco::Net::HTTPResponse & response) { for (size_t i = 0; i < settings.http_max_tries; ++i) { try { - callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD); + callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD); break; } catch (const Poco::Exception & e) @@ -296,6 +305,7 @@ namespace detail , remote_host_filter {remote_host_filter_} , buffer_size {buffer_size_} , use_external_buffer {use_external_buffer_} + , initial_read_range(read_range_) , read_range(read_range_) , http_skip_not_found_url(http_skip_not_found_url_) , settings {settings_} @@ -359,9 +369,10 @@ namespace detail return location_uri; } + template void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { - call(response, method_, throw_on_all_errors); + call(response, method_, throw_on_all_errors); Poco::URI prev_uri = uri; while (isRedirect(response.getStatus())) @@ -373,15 +384,16 @@ namespace detail session->updateSession(uri_redirect); - istr = callImpl(uri_redirect, response, method); + istr = callImpl(uri_redirect, response, method); } } + template void call(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { try { - istr = callImpl(saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); + istr = callImpl(saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); } catch (...) { From c66d5bc687aa070ea9dcb6c497040754b601d78e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 24 Feb 2023 12:51:36 +0000 Subject: [PATCH 089/470] better --- src/IO/ReadWriteBufferFromHTTP.h | 31 +++++++++++++++++++------------ 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index dd8a3376607..e4c48fab741 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -133,13 +133,13 @@ namespace detail ReadSettings settings; Poco::Logger * log; - bool withPartialContent() const + bool withPartialContent(const Range & range) const { /** * Add range header if we have some passed range (for disk web) * or if we want to retry GET request on purpose. */ - return read_range.begin || read_range.end || retry_with_range_header; + return range.begin || range.end || retry_with_range_header; } size_t getRangeBegin() const { return read_range.begin.value_or(0); } @@ -156,11 +156,6 @@ namespace detail Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header - Range range; - if constexpr (use_initial_range) - range = Range{initial_read_range.begin.value_or(0), initial_read_range.end}; - else - range = Range{getOffset(), read_range.end}; if (out_stream_callback) request.setChunkedTransferEncoding(true); @@ -168,13 +163,25 @@ namespace detail for (auto & [header, value] : http_header_entries) request.set(header, value); - if (withPartialContent()) + std::optional range; + if constexpr (use_initial_range) + { + if (withPartialContent(initial_read_range)) + range = initial_read_range; + } + else + { + if (withPartialContent(read_range)) + range = Range{getOffset(), read_range.end}; + } + + if (range) { String range_header_value; - if (range.end) - range_header_value = fmt::format("bytes={}-{}", *range.begin, *range.end); + if (range->end) + range_header_value = fmt::format("bytes={}-{}", *range->begin, *range->end); else - range_header_value = fmt::format("bytes={}-", *range.begin); + range_header_value = fmt::format("bytes={}-", *range->begin); LOG_TEST(log, "Adding header: Range: {}", range_header_value); request.set("Range", range_header_value); } @@ -448,7 +455,7 @@ namespace detail if (response.hasContentLength()) LOG_DEBUG(log, "Received response with content length: {}", response.getContentLength()); - if (withPartialContent() && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) + if (withPartialContent(read_range) && response.getStatus() != Poco::Net::HTTPResponse::HTTPStatus::HTTP_PARTIAL_CONTENT) { /// Having `200 OK` instead of `206 Partial Content` is acceptable in case we retried with range.begin == 0. if (read_range.begin && *read_range.begin != 0) From 12b38ece1d6924e0e7718eb57be41c2e83931cea Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 24 Feb 2023 13:07:09 +0000 Subject: [PATCH 090/470] Fix build --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 1427e6098ed..cb851c4a1e9 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -433,6 +433,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { decoder.decodeFixed(tmp_fixed.size(), tmp_fixed); column.insertData(reinterpret_cast(tmp_fixed.data()), tmp_fixed.size()); + return true; }; } break; From 5024f11adc2557b1fbf775b312b3d290bfea97fc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 24 Feb 2023 17:18:45 +0000 Subject: [PATCH 091/470] fix order in serialization info --- src/DataTypes/Serializations/SerializationInfo.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 4ae5cf8c193..7f73d053f1b 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -86,7 +86,8 @@ using MutableSerializationInfoPtr = std::shared_ptr; using SerializationInfos = std::vector; using MutableSerializationInfos = std::vector; -class SerializationInfoByName : public std::unordered_map +/// The order is important because info is serialized to part metadata. +class SerializationInfoByName : public std::map { public: SerializationInfoByName() = default; From 3a947ecddc5a9f30e91ef74e4f54da97b8c91f9e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 24 Feb 2023 19:17:44 +0000 Subject: [PATCH 092/470] do not allow const and non-deterministic secondary indexes --- src/Storages/IndicesDescription.cpp | 10 +------ src/Storages/MergeTree/MergeTreeData.cpp | 1 - src/Storages/MergeTree/MergeTreeIndices.cpp | 26 +++++++++++++++++++ .../02670_constant_skip_index.reference | 0 .../0_stateless/02670_constant_skip_index.sql | 25 ++++++++++++++++++ 5 files changed, 52 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02670_constant_skip_index.reference create mode 100644 tests/queries/0_stateless/02670_constant_skip_index.sql diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index 2e07aceeaa9..00a3636b605 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -94,15 +94,7 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast auto syntax = TreeRewriter(context).analyze(expr_list, columns.getAllPhysical()); result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true); - Block block_without_columns = result.expression->getSampleBlock(); - - for (size_t i = 0; i < block_without_columns.columns(); ++i) - { - const auto & column = block_without_columns.getByPosition(i); - result.column_names.emplace_back(column.name); - result.data_types.emplace_back(column.type); - result.sample_block.insert(ColumnWithTypeAndName(column.type->createColumn(), column.type, column.name)); - } + result.sample_block = result.expression->getSampleBlock(); const auto & definition_arguments = index_definition->type->arguments; if (definition_arguments) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0d4e54453d7..f49ecba1385 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -524,7 +524,6 @@ void MergeTreeData::checkProperties( for (const auto & index : new_metadata.secondary_indices) { - MergeTreeIndexFactory::instance().validate(index, attach); if (indices_names.find(index.name) != indices_names.end()) diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 2be9ecd8de3..6ae96d00171 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -35,6 +35,7 @@ MergeTreeIndexPtr MergeTreeIndexFactory::get( { auto it = creators.find(index.type); if (it == creators.end()) + { throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown Index type '{}'. Available index types: {}", index.type, std::accumulate(creators.cbegin(), creators.cend(), std::string{}, @@ -46,6 +47,7 @@ MergeTreeIndexPtr MergeTreeIndexFactory::get( return left + ", " + right.first; }) ); + } return it->second(index); } @@ -61,8 +63,31 @@ MergeTreeIndices MergeTreeIndexFactory::getMany(const std::vectorhasArrayJoin()) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Secondary index '{}' cannot contain array joins", index.name); + + try + { + index.expression->assertDeterministic(); + } + catch (Exception & e) + { + e.addMessage(fmt::format("for secondary index '{}'", index.name)); + throw; + } + + for (const auto & elem : index.sample_block) + if (elem.column && (isColumnConst(*elem.column) || elem.column->isDummy())) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Secondary index '{}' cannot contain constants", index.name); + } + auto it = validators.find(index.type); if (it == validators.end()) + { throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown Index type '{}'. Available index types: {}", index.type, std::accumulate( @@ -77,6 +102,7 @@ void MergeTreeIndexFactory::validate(const IndexDescription & index, bool attach return left + ", " + right.first; }) ); + } it->second(index, attach); } diff --git a/tests/queries/0_stateless/02670_constant_skip_index.reference b/tests/queries/0_stateless/02670_constant_skip_index.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02670_constant_skip_index.sql b/tests/queries/0_stateless/02670_constant_skip_index.sql new file mode 100644 index 00000000000..97dd2ab33c9 --- /dev/null +++ b/tests/queries/0_stateless/02670_constant_skip_index.sql @@ -0,0 +1,25 @@ + +DROP TABLE IF EXISTS t_constant_index; + +CREATE TABLE t_constant_index +( + id UInt64, + INDEX t_constant_index 'foo' TYPE set(2) GRANULARITY 1 +) ENGINE = MergeTree +ORDER BY id; -- { serverError INCORRECT_QUERY } + +CREATE TABLE t_constant_index +( + id UInt64, + INDEX t_constant_index id + rand() TYPE set(2) GRANULARITY 1 +) ENGINE = MergeTree +ORDER BY id; -- { serverError BAD_ARGUMENTS } + +CREATE TABLE t_constant_index +( + id UInt64, + INDEX t_constant_index id * 2 TYPE set(2) GRANULARITY 1 +) ENGINE = MergeTree +ORDER BY id; + +DROP TABLE t_constant_index; From cad1e0b7684a691116cef95f1c418df8ef4be740 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 25 Feb 2023 01:18:34 +0100 Subject: [PATCH 093/470] fix --- src/IO/WriteBufferFromPocoSocket.cpp | 11 +++++--- src/IO/WriteBufferFromPocoSocket.h | 1 + src/Interpreters/executeQuery.cpp | 15 ++++++----- src/QueryPipeline/BlockIO.cpp | 25 ++++++++++++++++++- src/QueryPipeline/BlockIO.h | 3 ++- src/Server/TCPHandler.cpp | 16 +++++++++--- src/Storages/Distributed/DistributedSink.cpp | 20 +++++++++++++++ src/Storages/Distributed/DistributedSink.h | 2 ++ .../test_insert_into_distributed/test.py | 2 +- .../02434_cancel_insert_when_client_dies.sh | 13 +++++++--- .../02435_rollback_cancelled_queries.sh | 2 +- utils/check-mysql-binlog/main.cpp | 2 +- 12 files changed, 89 insertions(+), 23 deletions(-) diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index e01245849ae..b15149c6f88 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -64,7 +64,8 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::Net::NetException & e) { - throw NetException(ErrorCodes::NETWORK_ERROR, "{}, while writing to socket ({})", e.displayText(), peer_address.toString()); + throw NetException(ErrorCodes::NETWORK_ERROR, "{}, while writing to socket ({} -> {})", e.displayText(), + our_address.toString(), peer_address.toString()); } catch (const Poco::TimeoutException &) { @@ -74,18 +75,20 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::IOException & e) { - throw NetException(ErrorCodes::NETWORK_ERROR, "{}, while writing to socket ({})", e.displayText(), peer_address.toString()); + throw NetException(ErrorCodes::NETWORK_ERROR, "{}, while writing to socket ({} -> {})", e.displayText(), + our_address.toString(), peer_address.toString()); } if (res < 0) - throw NetException(ErrorCodes::CANNOT_WRITE_TO_SOCKET, "Cannot write to socket ({})", peer_address.toString()); + throw NetException(ErrorCodes::CANNOT_WRITE_TO_SOCKET, "Cannot write to socket ({} -> {})", + our_address.toString(), peer_address.toString()); bytes_written += res; } } WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) - : BufferWithOwnMemory(buf_size), socket(socket_), peer_address(socket.peerAddress()) + : BufferWithOwnMemory(buf_size), socket(socket_), peer_address(socket.peerAddress()), our_address(socket.address()) { } diff --git a/src/IO/WriteBufferFromPocoSocket.h b/src/IO/WriteBufferFromPocoSocket.h index 295ca16ecaf..ffe1176c8cd 100644 --- a/src/IO/WriteBufferFromPocoSocket.h +++ b/src/IO/WriteBufferFromPocoSocket.h @@ -28,6 +28,7 @@ protected: * (getpeername will return an error). */ Poco::Net::SocketAddress peer_address; + Poco::Net::SocketAddress our_address; }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f3d83a37c59..5e1e4cb58fb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -176,7 +176,7 @@ static void setExceptionStackTrace(QueryLogElement & elem) /// Log exception (with query info) into text log (not into system table). -static void logException(ContextPtr context, QueryLogElement & elem) +static void logException(ContextPtr context, QueryLogElement & elem, bool log_error = true) { String comment; if (!elem.log_comment.empty()) @@ -187,7 +187,7 @@ static void logException(ContextPtr context, QueryLogElement & elem) PreformattedMessage message; message.format_string = elem.exception_format_string; - if (elem.stack_trace.empty()) + if (elem.stack_trace.empty() || !log_error) message.text = fmt::format("{} (from {}){} (in query: {})", elem.exception, context->getClientInfo().current_address.toString(), comment, @@ -201,7 +201,10 @@ static void logException(ContextPtr context, QueryLogElement & elem) toOneLineQuery(elem.query), elem.stack_trace); - LOG_ERROR(&Poco::Logger::get("executeQuery"), message); + if (log_error) + LOG_ERROR(&Poco::Logger::get("executeQuery"), message); + else + LOG_INFO(&Poco::Logger::get("executeQuery"), message); } static void onExceptionBeforeStart( @@ -1101,7 +1104,7 @@ static std::tuple executeQueryImpl( quota(quota), status_info_to_query_log, implicit_txn_control, - query_span]() mutable + query_span](bool log_error) mutable { if (implicit_txn_control) { @@ -1139,9 +1142,9 @@ static std::tuple executeQueryImpl( elem.query_duration_ms = start_watch.elapsedMilliseconds(); } - if (current_settings.calculate_text_stack_trace) + if (current_settings.calculate_text_stack_trace && log_error) setExceptionStackTrace(elem); - logException(context, elem); + logException(context, elem, log_error); /// In case of exception we log internal queries also if (log_queries && elem.type >= log_queries_min_type && static_cast(elem.query_duration_ms) >= log_queries_min_query_duration_ms) diff --git a/src/QueryPipeline/BlockIO.cpp b/src/QueryPipeline/BlockIO.cpp index 9af7cd2b772..231c369707e 100644 --- a/src/QueryPipeline/BlockIO.cpp +++ b/src/QueryPipeline/BlockIO.cpp @@ -4,6 +4,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; +} void BlockIO::reset() { @@ -58,7 +62,26 @@ void BlockIO::onFinish() void BlockIO::onException() { if (exception_callback) - exception_callback(); + exception_callback(/* log_error */ true); + + pipeline.reset(); +} + +void BlockIO::onCancelOrConnectionLoss() +{ + /// Query was not finished gracefully, so we should call exception_callback + /// But we don't have a real exception + if (exception_callback) + { + try + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled or a client has unexpectedly dropped the connection"); + } + catch (...) + { + exception_callback(/* log_error */ false); + } + } pipeline.reset(); } diff --git a/src/QueryPipeline/BlockIO.h b/src/QueryPipeline/BlockIO.h index 4c8d29d0ba8..ff85a0d6772 100644 --- a/src/QueryPipeline/BlockIO.h +++ b/src/QueryPipeline/BlockIO.h @@ -26,13 +26,14 @@ struct BlockIO /// Callbacks for query logging could be set here. std::function finish_callback; - std::function exception_callback; + std::function exception_callback; /// When it is true, don't bother sending any non-empty blocks to the out stream bool null_format = false; void onFinish(); void onException(); + void onCancelOrConnectionLoss(); /// Set is_all_data_sent in system.processes for this query. void setAllDataSent() const; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1f06f6a4ae9..9f53e418aec 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -416,17 +416,25 @@ void TCPHandler::runImpl() after_check_cancelled.restart(); after_send_progress.restart(); + auto finish_or_cancel = [this]() + { + if (state.is_cancelled) + state.io.onCancelOrConnectionLoss(); + else + state.io.onFinish(); + }; + if (state.io.pipeline.pushing()) { /// FIXME: check explicitly that insert query suggests to receive data via native protocol, state.need_receive_data_for_insert = true; processInsertQuery(); - state.io.onFinish(); + finish_or_cancel(); } else if (state.io.pipeline.pulling()) { processOrdinaryQueryWithProcessors(); - state.io.onFinish(); + finish_or_cancel(); } else if (state.io.pipeline.completed()) { @@ -455,7 +463,7 @@ void TCPHandler::runImpl() executor.execute(); } - state.io.onFinish(); + finish_or_cancel(); std::lock_guard lock(task_callback_mutex); @@ -469,7 +477,7 @@ void TCPHandler::runImpl() } else { - state.io.onFinish(); + finish_or_cancel(); } /// Do it before sending end of stream, to have a chance to show log message in client. diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index bac13ea37cf..b2bbd02f879 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -569,6 +569,26 @@ void DistributedSink::onFinish() } } +void DistributedSink::onCancel() +{ + if (pool && !pool->finished()) + { + try + { + pool->wait(); + } + catch (...) + { + tryLogCurrentException(storage.log); + } + } + + for (auto & shard_jobs : per_shard_jobs) + for (JobReplica & job : shard_jobs.replicas_jobs) + if (job.executor) + job.executor->cancel(); +} + IColumn::Selector DistributedSink::createSelector(const Block & source_block) const { diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index af0c64cbd78..325d5859289 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -54,6 +54,8 @@ public: void onFinish() override; private: + void onCancel() override; + IColumn::Selector createSelector(const Block & source_block) const; void writeAsync(const Block & block); diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index a52809f817c..3bee2149387 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -288,7 +288,7 @@ def test_inserts_single_replica_no_internal_replication(started_cluster): "prefer_localhost_replica": "0", }, ) - assert node2.query("SELECT count(*) FROM single_replicated").strip() == "1" + assert node2.query("SELECT count(*) FROM single_replicated").strip() == "0" finally: node2.query("TRUNCATE TABLE single_replicated") diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 6cae90a3cc3..2e653f2f1a3 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -11,20 +11,25 @@ export TEST_MARK="02434_insert_${CLICKHOUSE_DATABASE}_" $CLICKHOUSE_CLIENT -q 'select * from numbers(5000000) format TSV' > $DATA_FILE $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order by A settings non_replicated_deduplication_window=1000;' +$CLICKHOUSE_CLIENT -q "create table dedup_dist(A Int64) Engine = Distributed('test_cluster_one_shard_two_replicas', currentDatabase(), dedup_test)" function insert_data { SETTINGS="query_id=$ID&max_insert_block_size=110000&min_insert_block_size_rows=110000" # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) TRASH_SETTINGS="query_id=$ID&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" - TYPE=$(( RANDOM % 4 )) + TYPE=$(( RANDOM % 5 )) if [[ "$TYPE" -eq 0 ]]; then # client will send 10000-rows blocks, server will squash them into 110000-rows blocks (more chances to catch a bug on query cancellation) - $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" -q 'insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE + $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" \ + -q 'insert into dedup_test settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 1 ]]; then - $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + $CLICKHOUSE_CLIENT --max_block_size=10000 --max_insert_block_size=10000 --query_id="$ID" --prefer_localhost_replica="$(( RANDOM % 2))" \ + -q 'insert into dedup_dist settings max_insert_block_size=110000, min_insert_block_size_rows=110000 format TSV' < $DATA_FILE elif [[ "$TYPE" -eq 2 ]]; then + $CLICKHOUSE_CURL -sS -X POST --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE + elif [[ "$TYPE" -eq 3 ]]; then $CLICKHOUSE_CURL -sS -X POST -H "Transfer-Encoding: chunked" --data-binary @- "$CLICKHOUSE_URL&$SETTINGS&query=insert+into+dedup_test+format+TSV" < $DATA_FILE else $CLICKHOUSE_CURL -sS -F 'file=@-' "$CLICKHOUSE_URL&$TRASH_SETTINGS&file_format=TSV&file_types=UInt64" -X POST --form-string 'query=insert into dedup_test select * from file' < $DATA_FILE @@ -73,7 +78,7 @@ export -f thread_insert; export -f thread_select; export -f thread_cancel; -TIMEOUT=40 # 10 seconds for each TYPE +TIMEOUT=40 timeout $TIMEOUT bash -c thread_insert & timeout $TIMEOUT bash -c thread_select & diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 18317960cdf..a69e526c1c8 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -114,6 +114,6 @@ $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 # So use this query to check that thread_cancel do something $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or - message like '%Connection reset by peer%')" + message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index cf2a27e8aac..68558340180 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -18,7 +18,7 @@ static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( { DB::MySQLReplication::BinlogEventPtr event; DB::ReadBufferPtr limit_read_buffer = std::make_shared(payload, header.event_size - 19, - /* trow_exception */ false, /* exact_limit */ {}); + /* trow_exception */ false, /* exact_limit */ std::nullopt); DB::ReadBufferPtr event_payload = std::make_shared(*limit_read_buffer, exist_checksum ? 4 : 0); switch (header.type) From 940035f1086a7e7c6bdb375304d9166024ebddd4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 25 Feb 2023 01:31:38 +0100 Subject: [PATCH 094/470] fix --- .../queries/0_stateless/02434_cancel_insert_when_client_dies.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 2e653f2f1a3..2a17095b267 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -40,6 +40,7 @@ export -f insert_data ID="02434_insert_init_${CLICKHOUSE_DATABASE}_$RANDOM" insert_data +$CLICKHOUSE_CLIENT -q "system flush distributed dedup_dist" $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' function thread_insert From 4b5d62d5bf34b534693b9f8f579ed87ecf25f1e5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 26 Feb 2023 22:19:01 +0000 Subject: [PATCH 095/470] Analyzer: trivial count optimization --- src/Planner/PlannerJoinTree.cpp | 126 ++++++++++++++++++++++++++++++++ 1 file changed, 126 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 59b09f91888..36eb5c59ef5 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -39,6 +39,14 @@ #include #include #include +#include +#include +#include "Analyzer/AggregationUtils.h" +#include "Analyzer/FunctionNode.h" +#include +#include +#include +#include namespace DB { @@ -143,6 +151,119 @@ NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage return result; } +bool applyTrivialCountIfPossible( + QueryPlan & query_plan, + const TableNode & table_node, + const SelectQueryInfo & select_query_info, + const ContextPtr & query_context, + const Names & columns_names) +{ + const auto & settings = query_context->getSettingsRef(); + if (!settings.optimize_trivial_count_query) + return false; + + const auto & storage = table_node.getStorage(); + const auto & storage_snapshot = table_node.getStorageSnapshot(); + + bool can_apply_trivial_count = (settings.max_parallel_replicas <= 1) && // + !settings.allow_experimental_query_deduplication && // + !settings.empty_result_for_aggregation_by_empty_set && // + storage && // + storage->getName() != "MaterializedMySQL" && // + !storage->hasLightweightDeletedMask() && // + select_query_info.filter_asts.empty() && // ??? + select_query_info.has_aggregates; + if (!can_apply_trivial_count) + return false; + + QueryTreeNodes aggregates = collectAggregateFunctionNodes(select_query_info.query_tree); + if (aggregates.size() != 1) + return false; + + auto & main_query_node = select_query_info.query_tree->as(); + /// dump main query tree + { + WriteBufferFromOwnString buffer; + IQueryTreeNode::FormatState format_state; + main_query_node.dumpTreeImpl(buffer, format_state, 0); + + LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "main_query_node:\n{}", buffer.str()); + LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Projection column:\n{}", main_query_node.getProjectionColumns().front().dump()); + } + + { + WriteBufferFromOwnString buffer; + buffer << columns_names; + LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", buffer.str()); + } + + const auto * function_node = typeid_cast(aggregates.front().get()); + if (!function_node) + return false; + + if (!function_node->getAggregateFunction()) + return false; + + LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Aggregation: {}", function_node->getFunctionName()); + + const auto * count_func = typeid_cast(function_node->getAggregateFunction().get()); + if (!count_func) + return false; + + /// get number of rows + std::optional num_rows{}; + // if (!query_tree. prewhere() && !query.where() && !context->getCurrentTransaction()) + if (!main_query_node.hasPrewhere() && !main_query_node.hasWhere()) + { + num_rows = storage->totalRows(settings); + } + // else // It's possible to optimize count() given only partition predicates + // { + // SelectQueryInfo temp_query_info; + // temp_query_info.query = query_ptr; + // temp_query_info.syntax_analyzer_result = syntax_analyzer_result; + // temp_query_info.prepared_sets = query_analyzer->getPreparedSets(); + // num_rows = storage->totalRowsByPartitionPredicate(temp_query_info, context); + // } + + if (!num_rows) + return false; + + LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Number of rows: {}", num_rows.value()); + + /// set aggregation state + const AggregateFunctionCount & agg_count = *count_func; + std::vector state(agg_count.sizeOfData()); + AggregateDataPtr place = state.data(); + agg_count.create(place); + // SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); + agg_count.set(place, num_rows.value()); + + auto column = ColumnAggregateFunction::create(function_node->getAggregateFunction()); + column->insertFrom(place); + + /// get count() argument type + DataTypes argument_types; + argument_types.reserve(columns_names.size()); + { + const Block source_header = storage_snapshot->getSampleBlockForColumns(columns_names); + for (const auto & column_name : columns_names) + argument_types.push_back(source_header.getByName(column_name).type); + } + + Block block_with_count{ + {std::move(column), + std::make_shared(function_node->getAggregateFunction(), argument_types, Array{}), + columns_names.front()}}; + + auto source = std::make_shared(block_with_count); + auto prepared_count = std::make_unique(Pipe(std::move(source))); + prepared_count->setStepDescription("Optimized trivial count"); + query_plan.addStep(std::move(prepared_count)); + + return true; +} + JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expression, const SelectQueryInfo & select_query_info, const SelectQueryOptions & select_query_options, @@ -287,6 +408,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl table_expression_data.addColumn(additional_column_to_read, column_identifier); } + /// apply trivial_count optimization if possible + if (is_single_table_expression && table_node + && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names)) + return {std::move(query_plan), from_stage}; + bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(columns_names); if (need_rewrite_query_with_final) { From 1e4d9e2421091468bddcce8f59191b87865a05c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 26 Feb 2023 22:37:06 +0000 Subject: [PATCH 096/470] try to fix --- src/Planner/PlannerJoinTree.cpp | 50 ++++++++++++++++++--------------- 1 file changed, 28 insertions(+), 22 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 36eb5c59ef5..593bdd7dece 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -408,11 +408,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl table_expression_data.addColumn(additional_column_to_read, column_identifier); } - /// apply trivial_count optimization if possible - if (is_single_table_expression && table_node - && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names)) - return {std::move(query_plan), from_stage}; - bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(columns_names); if (need_rewrite_query_with_final) { @@ -434,28 +429,39 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } } - storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + /// apply trivial_count optimization if possible + bool is_trivial_count_applied = (is_single_table_expression && table_node + && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names)); - if (query_plan.isInitialized()) + if (is_trivial_count_applied) { - /** Specify the number of threads only if it wasn't specified in storage. - * - * But in case of remote query and prefer_localhost_replica=1 (default) - * The inner local query (that is done in the same process, without - * network interaction), it will setMaxThreads earlier and distributed - * query will not update it. - */ - if (!query_plan.getMaxThreads() || is_remote) - query_plan.setMaxThreads(max_threads_execute_query); + from_stage = QueryProcessingStage::WithMergeableState; } else { - /// Create step which reads from empty source if storage has no data. - auto source_header = storage_snapshot->getSampleBlockForColumns(columns_names); - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); + storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + + if (query_plan.isInitialized()) + { + /** Specify the number of threads only if it wasn't specified in storage. + * + * But in case of remote query and prefer_localhost_replica=1 (default) + * The inner local query (that is done in the same process, without + * network interaction), it will setMaxThreads earlier and distributed + * query will not update it. + */ + if (!query_plan.getMaxThreads() || is_remote) + query_plan.setMaxThreads(max_threads_execute_query); + } + else + { + /// Create step which reads from empty source if storage has no data. + auto source_header = storage_snapshot->getSampleBlockForColumns(columns_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } } } else if (query_node || union_node) From f9a324c47a50e67dfe1933677924e08f921a40e7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 Feb 2023 09:48:24 +0000 Subject: [PATCH 097/470] Make diagnostic traces debug level --- src/Planner/PlannerJoinTree.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 593bdd7dece..b1e22b40d69 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -187,14 +187,14 @@ bool applyTrivialCountIfPossible( IQueryTreeNode::FormatState format_state; main_query_node.dumpTreeImpl(buffer, format_state, 0); - LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "main_query_node:\n{}", buffer.str()); - LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Projection column:\n{}", main_query_node.getProjectionColumns().front().dump()); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "main_query_node:\n{}", buffer.str()); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Projection column:\n{}", main_query_node.getProjectionColumns().front().dump()); } { WriteBufferFromOwnString buffer; buffer << columns_names; - LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", buffer.str()); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", buffer.str()); } const auto * function_node = typeid_cast(aggregates.front().get()); @@ -204,7 +204,7 @@ bool applyTrivialCountIfPossible( if (!function_node->getAggregateFunction()) return false; - LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Aggregation: {}", function_node->getFunctionName()); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Aggregation: {}", function_node->getFunctionName()); const auto * count_func = typeid_cast(function_node->getAggregateFunction().get()); if (!count_func) @@ -229,7 +229,7 @@ bool applyTrivialCountIfPossible( if (!num_rows) return false; - LOG_ERROR(&Poco::Logger::get(__PRETTY_FUNCTION__), "Number of rows: {}", num_rows.value()); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Number of rows: {}", num_rows.value()); /// set aggregation state const AggregateFunctionCount & agg_count = *count_func; From cac9e96d0415f52d27a96c873c981f37bd8a2ed7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 Feb 2023 10:56:59 +0000 Subject: [PATCH 098/470] Fix: do not apply if FINAL --- src/Planner/PlannerJoinTree.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index b1e22b40d69..9c005314421 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -162,6 +162,10 @@ bool applyTrivialCountIfPossible( if (!settings.optimize_trivial_count_query) return false; + /// can't apply if FINAL + if (select_query_info.table_expression_modifiers.has_value() && select_query_info.table_expression_modifiers.value().hasFinal()) + return false; + const auto & storage = table_node.getStorage(); const auto & storage_snapshot = table_node.getStorageSnapshot(); @@ -430,8 +434,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } /// apply trivial_count optimization if possible - bool is_trivial_count_applied = (is_single_table_expression && table_node - && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names)); + bool is_trivial_count_applied = is_single_table_expression && table_node + && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names); if (is_trivial_count_applied) { From 67099f2a65a78e8dc898510a4f700b36603fd067 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 Feb 2023 12:53:09 +0000 Subject: [PATCH 099/470] Tests --- .../02674_trivial_count_analyzer.reference | 48 +++++++++++++++++++ .../02674_trivial_count_analyzer.sql | 46 ++++++++++++++++++ 2 files changed, 94 insertions(+) create mode 100644 tests/queries/0_stateless/02674_trivial_count_analyzer.reference create mode 100644 tests/queries/0_stateless/02674_trivial_count_analyzer.sql diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.reference b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference new file mode 100644 index 00000000000..2a94fd59d7b --- /dev/null +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference @@ -0,0 +1,48 @@ +-- { echoOn } +set allow_experimental_analyzer=1; +set optimize_trivial_count_query=1; +create table m3(a Int64, b UInt64) Engine=MergeTree order by tuple(); +select count() from m3; +0 +insert into m3 values (0,0); +insert into m3 values (-1,1); +select trimBoth(explain) from (explain select count() from m3) where explain like '%ReadFromPreparedSource (Optimized trivial count)%'; +ReadFromPreparedSource (Optimized trivial count) +select count() from m3; +2 +select count(*) from m3; +2 +select count(a) from m3; +2 +select count(b) from m3; +2 +select count() + 1 from m3; +3 +-- drop table m3; + +-- checking queries with FINAL +create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b); +SYSTEM STOP MERGES replacing_m3; +select count() from replacing_m3; +0 +insert into replacing_m3 values (0,0); +insert into replacing_m3 values (0,0); +insert into replacing_m3 values (-1,1); +insert into replacing_m3 values (-2,2); +select trimBoth(explain) from (explain select count() from replacing_m3) where explain like '%ReadFromPreparedSource (Optimized trivial count)%'; +ReadFromPreparedSource (Optimized trivial count) +select count() from replacing_m3; +4 +select count(*) from replacing_m3; +4 +select count(a) from replacing_m3; +4 +select count(b) from replacing_m3; +4 +set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL +select count() from replacing_m3 FINAL; +3 +select count(a) from replacing_m3 FINAL; +3 +select count(b) from replacing_m3 FINAL; +3 diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.sql b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql new file mode 100644 index 00000000000..d4a686e6eff --- /dev/null +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql @@ -0,0 +1,46 @@ +drop table if exists m3; +drop table if exists replacing_m3; + +-- { echoOn } +set allow_experimental_analyzer=1; +set optimize_trivial_count_query=1; + +create table m3(a Int64, b UInt64) Engine=MergeTree order by tuple(); + +select count() from m3; + +insert into m3 values (0,0); +insert into m3 values (-1,1); + +select trimBoth(explain) from (explain select count() from m3) where explain like '%ReadFromPreparedSource (Optimized trivial count)%'; +select count() from m3; +select count(*) from m3; +select count(a) from m3; +select count(b) from m3; +select count() + 1 from m3; + +-- drop table m3; + +-- checking queries with FINAL +create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b); +SYSTEM STOP MERGES replacing_m3; + +select count() from replacing_m3; + +insert into replacing_m3 values (0,0); +insert into replacing_m3 values (0,0); +insert into replacing_m3 values (-1,1); +insert into replacing_m3 values (-2,2); + +select trimBoth(explain) from (explain select count() from replacing_m3) where explain like '%ReadFromPreparedSource (Optimized trivial count)%'; +select count() from replacing_m3; +select count(*) from replacing_m3; +select count(a) from replacing_m3; +select count(b) from replacing_m3; + +set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL +select count() from replacing_m3 FINAL; +select count(a) from replacing_m3 FINAL; +select count(b) from replacing_m3 FINAL; + +-- drop table replacing_m3; From 7684153100907ad3987980a66b79824158d10dac Mon Sep 17 00:00:00 2001 From: artem-yadr <84010375+artem-yadr@users.noreply.github.com> Date: Mon, 27 Feb 2023 16:18:08 +0300 Subject: [PATCH 100/470] Redeclaration error fix --- base/poco/MongoDB/src/Connection.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/MongoDB/src/Connection.cpp b/base/poco/MongoDB/src/Connection.cpp index c75211cf9d7..38c31d2250a 100644 --- a/base/poco/MongoDB/src/Connection.cpp +++ b/base/poco/MongoDB/src/Connection.cpp @@ -237,7 +237,7 @@ void Connection::connect(const std::string& uri, SocketFactory& socketFactory) for (std::vector::const_iterator it = strAddresses.cbegin();it != strAddresses.cend(); ++it) { newURI = *it; - Poco::URI theURI(newURI); + theURI = Poco::URI(newURI); std::string host = theURI.getHost(); Poco::UInt16 port = theURI.getPort(); From 385beda8888dcabc895d0538914b3ce747c7ea67 Mon Sep 17 00:00:00 2001 From: artem-yadr <84010375+artem-yadr@users.noreply.github.com> Date: Mon, 27 Feb 2023 16:25:12 +0300 Subject: [PATCH 101/470] Update base/poco/MongoDB/include/Poco/MongoDB/Connection.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- base/poco/MongoDB/include/Poco/MongoDB/Connection.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index 8fd9c7919e4..5c5f6f76638 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -91,7 +91,7 @@ namespace MongoDB Poco::Net::SocketAddress address() const; /// Returns the address of the MongoDB server. - std::string uri() const; + const std::string & uri() const; /// Returns the uri on which the connection was made. void connect(const std::string & hostAndPort); From 0de2eb8a795908132ed2b6eaaa4bd09fb76e9f2d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 27 Feb 2023 14:45:39 +0000 Subject: [PATCH 102/470] Update version_date.tsv and changelogs after v22.8.14.53-lts --- docs/changelogs/v22.8.14.53-lts.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 41 insertions(+) create mode 100644 docs/changelogs/v22.8.14.53-lts.md diff --git a/docs/changelogs/v22.8.14.53-lts.md b/docs/changelogs/v22.8.14.53-lts.md new file mode 100644 index 00000000000..5978080fa3a --- /dev/null +++ b/docs/changelogs/v22.8.14.53-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.8.14.53-lts (4ea67c40077) FIXME as compared to v22.8.13.20-lts (e4817946d18) + +#### Performance Improvement +* Backported in [#45845](https://github.com/ClickHouse/ClickHouse/issues/45845): Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46374](https://github.com/ClickHouse/ClickHouse/issues/46374): Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#46358](https://github.com/ClickHouse/ClickHouse/issues/46358): Allow using Vertical merge algorithm with parts in Compact format. This will allow ClickHouse server to use much less memory for background operations. This closes [#46084](https://github.com/ClickHouse/ClickHouse/issues/46084). [#46282](https://github.com/ClickHouse/ClickHouse/pull/46282) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement +* Backported in [#46112](https://github.com/ClickHouse/ClickHouse/issues/46112): Remove the dependency on the `adduser` tool from the packages, because we don't use it. This fixes [#44934](https://github.com/ClickHouse/ClickHouse/issues/44934). [#45011](https://github.com/ClickHouse/ClickHouse/pull/45011) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46482](https://github.com/ClickHouse/ClickHouse/issues/46482): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46505](https://github.com/ClickHouse/ClickHouse/issues/46505): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#45908](https://github.com/ClickHouse/ClickHouse/issues/45908): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#46238](https://github.com/ClickHouse/ClickHouse/issues/46238): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#45727](https://github.com/ClickHouse/ClickHouse/issues/45727): Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). +* Backported in [#46394](https://github.com/ClickHouse/ClickHouse/issues/46394): Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#46442](https://github.com/ClickHouse/ClickHouse/issues/46442): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46674](https://github.com/ClickHouse/ClickHouse/issues/46674): Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46879](https://github.com/ClickHouse/ClickHouse/issues/46879): Fix MSan report in the `maxIntersections` function. This closes [#43126](https://github.com/ClickHouse/ClickHouse/issues/43126). [#46847](https://github.com/ClickHouse/ClickHouse/pull/46847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46871](https://github.com/ClickHouse/ClickHouse/issues/46871): Fix a bug in the `Map` data type. This closes [#46855](https://github.com/ClickHouse/ClickHouse/issues/46855). [#46856](https://github.com/ClickHouse/ClickHouse/pull/46856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3814e94bf24..50937edb449 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -25,6 +25,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 v22.8.11.15-lts 2022-12-08 From 7b85b35534fd65fae459e5ce13b5328da7d6c6b0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 27 Feb 2023 15:11:25 +0000 Subject: [PATCH 103/470] fix skip indexes --- src/Storages/IndicesDescription.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index 00a3636b605..591a9082f7a 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -96,6 +96,12 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true); result.sample_block = result.expression->getSampleBlock(); + for (const auto & elem : result.sample_block) + { + result.column_names.push_back(elem.name); + result.data_types.push_back(elem.type); + } + const auto & definition_arguments = index_definition->type->arguments; if (definition_arguments) { From 2e921e3d6b4abcdf61ff4eec0405038f528175c0 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Feb 2023 16:00:19 +0000 Subject: [PATCH 104/470] Fix date and int inference from string in JSON --- src/Formats/SchemaInferenceUtils.cpp | 1 + src/Processors/Formats/ISchemaReader.h | 4 ++-- .../02674_date_int_string_json_inference.reference | 1 + .../0_stateless/02674_date_int_string_json_inference.sql | 2 ++ 4 files changed, 6 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02674_date_int_string_json_inference.reference create mode 100644 tests/queries/0_stateless/02674_date_int_string_json_inference.sql diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 7a242a9f81c..00eb686385d 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -131,6 +131,7 @@ namespace type_indexes.erase(TypeIndex::Date); type_indexes.erase(TypeIndex::DateTime); + type_indexes.insert(TypeIndex::String); return; } diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index edc5c6068c3..330acc99369 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -170,10 +170,10 @@ void chooseResultColumnType( ErrorCodes::TYPE_MISMATCH, "Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. " "You can specify the type for this column using setting schema_inference_hints", - type->getName(), + new_type->getName(), column_name, row, - new_type->getName()); + type->getName()); } } diff --git a/tests/queries/0_stateless/02674_date_int_string_json_inference.reference b/tests/queries/0_stateless/02674_date_int_string_json_inference.reference new file mode 100644 index 00000000000..2e89d6a15a4 --- /dev/null +++ b/tests/queries/0_stateless/02674_date_int_string_json_inference.reference @@ -0,0 +1 @@ +x Nullable(String) diff --git a/tests/queries/0_stateless/02674_date_int_string_json_inference.sql b/tests/queries/0_stateless/02674_date_int_string_json_inference.sql new file mode 100644 index 00000000000..21abf763cbf --- /dev/null +++ b/tests/queries/0_stateless/02674_date_int_string_json_inference.sql @@ -0,0 +1,2 @@ +desc format(JSONEachRow, '{"x" : "2020-01-01"}, {"x" : "1000"}') + From e01c9ff5ec080ca9537843e058ea9e0ccea42ac6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 Feb 2023 17:37:29 +0000 Subject: [PATCH 105/470] Fixes: group by, final --- src/Planner/PlannerJoinTree.cpp | 34 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9c005314421..8b7489d52c8 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -154,7 +154,7 @@ NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage bool applyTrivialCountIfPossible( QueryPlan & query_plan, const TableNode & table_node, - const SelectQueryInfo & select_query_info, + const QueryTreeNodePtr & query_tree, const ContextPtr & query_context, const Names & columns_names) { @@ -163,28 +163,26 @@ bool applyTrivialCountIfPossible( return false; /// can't apply if FINAL - if (select_query_info.table_expression_modifiers.has_value() && select_query_info.table_expression_modifiers.value().hasFinal()) + if (table_node.getTableExpressionModifiers().has_value() && table_node.getTableExpressionModifiers()->hasFinal()) + return false; + + auto & main_query_node = query_tree->as(); + if (main_query_node.hasGroupBy()) return false; const auto & storage = table_node.getStorage(); - const auto & storage_snapshot = table_node.getStorageSnapshot(); - - bool can_apply_trivial_count = (settings.max_parallel_replicas <= 1) && // - !settings.allow_experimental_query_deduplication && // - !settings.empty_result_for_aggregation_by_empty_set && // - storage && // - storage->getName() != "MaterializedMySQL" && // - !storage->hasLightweightDeletedMask() && // - select_query_info.filter_asts.empty() && // ??? - select_query_info.has_aggregates; - if (!can_apply_trivial_count) + if (!storage || storage->hasLightweightDeletedMask()) return false; - QueryTreeNodes aggregates = collectAggregateFunctionNodes(select_query_info.query_tree); + if (settings.max_parallel_replicas > 1 || // + settings.allow_experimental_query_deduplication || // + settings.empty_result_for_aggregation_by_empty_set) + return false; + + QueryTreeNodes aggregates = collectAggregateFunctionNodes(query_tree); if (aggregates.size() != 1) return false; - auto & main_query_node = select_query_info.query_tree->as(); /// dump main query tree { WriteBufferFromOwnString buffer; @@ -250,7 +248,7 @@ bool applyTrivialCountIfPossible( DataTypes argument_types; argument_types.reserve(columns_names.size()); { - const Block source_header = storage_snapshot->getSampleBlockForColumns(columns_names); + const Block source_header = table_node.getStorageSnapshot()->getSampleBlockForColumns(columns_names); for (const auto & column_name : columns_names) argument_types.push_back(source_header.getByName(column_name).type); } @@ -434,8 +432,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } /// apply trivial_count optimization if possible - bool is_trivial_count_applied = is_single_table_expression && table_node - && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info, planner_context->getQueryContext(), columns_names); + bool is_trivial_count_applied = is_single_table_expression && table_node && select_query_info.has_aggregates + && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), columns_names); if (is_trivial_count_applied) { From ab899bf2f3220e07b4316c1db772eb2cde185087 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 27 Feb 2023 19:28:19 +0000 Subject: [PATCH 106/470] Allow types conversion in Native input format --- .../operations/settings/settings-formats.md | 12 ++++++-- src/Core/Settings.h | 2 ++ src/Core/SettingsChangesHistory.h | 3 +- src/DataTypes/DataTypeLowCardinality.h | 2 +- .../DataTypeLowCardinalityHelpers.cpp | 8 ++--- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 5 ++++ src/Formats/NativeReader.cpp | 30 +++++++++++++++++-- src/Formats/NativeReader.h | 2 ++ src/Processors/Formats/Impl/NativeFormat.cpp | 1 + .../Algorithms/AggregatingSortedAlgorithm.cpp | 2 +- .../Algorithms/SummingSortedAlgorithm.cpp | 2 +- .../02567_native_type_conversions.reference | 3 ++ .../02567_native_type_conversions.sh | 12 ++++++++ 14 files changed, 72 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02567_native_type_conversions.reference create mode 100755 tests/queries/0_stateless/02567_native_type_conversions.sh diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index c6eb8c1f2c0..a7a7d78ba4d 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1468,7 +1468,7 @@ Default value: `65505`. The name of table that will be used in the output INSERT statement. -Default value: `'table''`. +Default value: `table`. ### output_format_sql_insert_include_column_names {#output_format_sql_insert_include_column_names} @@ -1508,4 +1508,12 @@ Disabled by default. The maximum allowed size for String in RowBinary format. It prevents allocating large amount of memory in case of corrupted data. 0 means there is no limit. -Default value: `1GiB` +Default value: `1GiB`. + +## Native format settings {#native-format-settings} + +### input_format_native_allow_types_conversion {#input_format_native_allow_types_conversion} + +Allow types conversion in Native input format between columns from input data and requested columns. + +Enabled by default. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c1b396778ab..43207b517ca 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -819,6 +819,8 @@ class IColumn; M(UInt64, input_format_csv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in CSV format", 0) \ M(UInt64, input_format_tsv_skip_first_lines, 0, "Skip specified number of lines at the beginning of data in TSV format", 0) \ \ + M(Bool, input_format_native_allow_types_conversion, true, "Allow data types conversion in Native input format", 0) \ + \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c9d68b94a5e..b153372dd92 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.3", {{"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}}}, + {"23.3", {{"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 format"}}}, {"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"}, diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 57f67ddad7a..d301a0f5443 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -86,6 +86,6 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type); ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column); /// Convert column of type from_type to type to_type by converting nested LowCardinality columns. -ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type); +ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type); } diff --git a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index 8a61afee420..98eb76267a4 100644 --- a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -113,7 +113,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) return column; } -ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) +ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) { if (!column) return column; @@ -128,7 +128,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & if (const auto * column_const = typeid_cast(column.get())) { const auto & nested = column_const->getDataColumnPtr(); - auto nested_no_lc = recursiveTypeConversion(nested, from_type, to_type); + auto nested_no_lc = recursiveLowCardinalityTypeConversion(nested, from_type, to_type); if (nested.get() == nested_no_lc.get()) return column; @@ -164,7 +164,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & const auto & nested_to = to_array_type->getNestedType(); return ColumnArray::create( - recursiveTypeConversion(column_array->getDataPtr(), nested_from, nested_to), + recursiveLowCardinalityTypeConversion(column_array->getDataPtr(), nested_from, nested_to), column_array->getOffsetsPtr()); } } @@ -187,7 +187,7 @@ ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & for (size_t i = 0; i < columns.size(); ++i) { auto & element = columns[i]; - auto element_no_lc = recursiveTypeConversion(element, from_elements.at(i), to_elements.at(i)); + auto element_no_lc = recursiveLowCardinalityTypeConversion(element, from_elements.at(i), to_elements.at(i)); if (element.get() != element_no_lc.get()) { element = element_no_lc; diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a22926973ed..f9e704e0445 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -190,6 +190,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string; format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference; format_settings.max_binary_string_size = settings.format_binary_max_string_size; + format_settings.native.allow_types_conversion = settings.input_format_native_allow_types_conversion; format_settings.max_parser_depth = context->getSettingsRef().max_parser_depth; format_settings.client_protocol_version = context->getClientProtocolVersion(); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 5e95c629c7c..7ed0e7d069e 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -326,6 +326,11 @@ struct FormatSettings bool output_string_as_string; bool skip_fields_with_unsupported_types_in_schema_inference; } bson; + + struct + { + bool allow_types_conversion = true; + } native; }; } diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 9f8d4ba1930..eca88a41c13 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -15,6 +15,8 @@ #include #include +#include + namespace DB { @@ -39,12 +41,14 @@ NativeReader::NativeReader( UInt64 server_revision_, bool skip_unknown_columns_, bool null_as_default_, + bool allow_types_conversion_, BlockMissingValues * block_missing_values_) : istr(istr_) , header(header_) , server_revision(server_revision_) , skip_unknown_columns(skip_unknown_columns_) , null_as_default(null_as_default_) + , allow_types_conversion(allow_types_conversion_) , block_missing_values(block_missing_values_) { } @@ -204,11 +208,31 @@ Block NativeReader::read() if (null_as_default) insertNullAsDefaultIfNeeded(column, header_column, header.getPositionByName(column.name), block_missing_values); - /// Support insert from old clients without low cardinality type. if (!header_column.type->equals(*column.type)) { - column.column = recursiveTypeConversion(column.column, column.type, header.safeGetByPosition(i).type); - column.type = header.safeGetByPosition(i).type; + if (allow_types_conversion) + { + try + { + column.column = castColumn(column, header_column.type); + } + catch (Exception & e) + { + e.addMessage(fmt::format( + "while converting column \"{}\" from type {} to type {}", + column.name, + column.type->getName(), + header_column.type->getName())); + throw; + } + } + else + { + /// Support insert from old clients without low cardinality type. + column.column = recursiveLowCardinalityTypeConversion(column.column, column.type, header_column.type); + } + + column.type = header_column.type; } } else diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 2d8b16e06eb..3cec4afd997 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -30,6 +30,7 @@ public: UInt64 server_revision_, bool skip_unknown_columns_ = false, bool null_as_default_ = false, + bool allow_types_conversion_ = false, BlockMissingValues * block_missing_values_ = nullptr); /// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read. @@ -51,6 +52,7 @@ private: UInt64 server_revision; bool skip_unknown_columns = false; bool null_as_default = false; + bool allow_types_conversion = false; BlockMissingValues * block_missing_values = nullptr; bool use_index = false; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index bd1b13ce2ef..d3fd9ef73e1 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -23,6 +23,7 @@ public: 0, settings.skip_unknown_fields, settings.null_as_default, + settings.native.allow_types_conversion, settings.defaults_for_omitted_fields ? &block_missing_values : nullptr)) , header(header_) {} diff --git a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp index 560be60987b..ef103eb508c 100644 --- a/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/AggregatingSortedAlgorithm.cpp @@ -117,7 +117,7 @@ static void postprocessChunk(Chunk & chunk, const AggregatingSortedAlgorithm::Co { const auto & from_type = desc.nested_type; const auto & to_type = desc.real_type; - columns[desc.column_number] = recursiveTypeConversion(columns[desc.column_number], from_type, to_type); + columns[desc.column_number] = recursiveLowCardinalityTypeConversion(columns[desc.column_number], from_type, to_type); } } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 0f1775d4ac0..d8e95e6b950 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -450,7 +450,7 @@ static void postprocessChunk( { const auto & from_type = desc.nested_type; const auto & to_type = desc.real_type; - res_columns[desc.column_numbers[0]] = recursiveTypeConversion(column, from_type, to_type); + res_columns[desc.column_numbers[0]] = recursiveLowCardinalityTypeConversion(column, from_type, to_type); } else res_columns[desc.column_numbers[0]] = std::move(column); diff --git a/tests/queries/0_stateless/02567_native_type_conversions.reference b/tests/queries/0_stateless/02567_native_type_conversions.reference new file mode 100644 index 00000000000..5c223870c11 --- /dev/null +++ b/tests/queries/0_stateless/02567_native_type_conversions.reference @@ -0,0 +1,3 @@ +1 +42 +1 diff --git a/tests/queries/0_stateless/02567_native_type_conversions.sh b/tests/queries/0_stateless/02567_native_type_conversions.sh new file mode 100755 index 00000000000..976c42f07c1 --- /dev/null +++ b/tests/queries/0_stateless/02567_native_type_conversions.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select 42::UInt8 as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=0 2>&1 | grep "TYPE_MISMATCH" -c + +$CLICKHOUSE_LOCAL -q "select 42::UInt8 as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=1 + +$CLICKHOUSE_LOCAL -q "select 'Hello' as x format Native" | $CLICKHOUSE_LOCAL --structure="x UInt64" --input-format="Native" -q "select * from table" --input_format_native_allow_types_conversion=1 2>&1 | grep 'while converting column "x" from type String to type UInt64' -c + From d5469c0ab49106ad55aef68a8eaaec17d59d4081 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Feb 2023 00:07:50 +0100 Subject: [PATCH 107/470] fix clickhouse-test --- tests/clickhouse-test | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index cfd2546bbdd..06a30317346 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -73,7 +73,14 @@ def stringhash(s): # First and last lines of the log def trim_for_log(s): - return s + if not s: + return s + lines = s.splitlines() + if len(lines) > 10000: + separator = "-" * 40 + str(len(lines) - 10000) + " lines are hidden" + "-" * 40 + return "\n".join(lines[:5000] + [] + [separator] + [] + lines[-5000:]) + else: + return "\n".join(lines) class HTTPError(Exception): From 5fb1f20882677f66ca5c50decd01dcaae4677cab Mon Sep 17 00:00:00 2001 From: AndyB Date: Mon, 27 Feb 2023 23:16:34 +0000 Subject: [PATCH 108/470] LOG_DEBUG over LOG_INFO --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 435401796a0..f3c103f6af2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -995,7 +995,7 @@ static std::tuple executeQueryImpl( { double elapsed_seconds = static_cast(info.elapsed_microseconds) / 1000000.0; double rows_per_second = static_cast(elem.read_rows) / elapsed_seconds; - LOG_INFO( + LOG_DEBUG( &Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", elem.read_rows, From 42c5634ae4af14602250e2938d28d70c064d6247 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 Feb 2023 11:35:36 +0000 Subject: [PATCH 109/470] Remove debug logging --- src/Planner/PlannerJoinTree.cpp | 39 ++++++++++----------------------- 1 file changed, 12 insertions(+), 27 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 8b7489d52c8..11944b4b71d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -39,14 +39,14 @@ #include #include #include + #include -#include -#include "Analyzer/AggregationUtils.h" -#include "Analyzer/FunctionNode.h" -#include +#include +#include #include #include #include +#include namespace DB { @@ -183,22 +183,6 @@ bool applyTrivialCountIfPossible( if (aggregates.size() != 1) return false; - /// dump main query tree - { - WriteBufferFromOwnString buffer; - IQueryTreeNode::FormatState format_state; - main_query_node.dumpTreeImpl(buffer, format_state, 0); - - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "main_query_node:\n{}", buffer.str()); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Projection column:\n{}", main_query_node.getProjectionColumns().front().dump()); - } - - { - WriteBufferFromOwnString buffer; - buffer << columns_names; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "{}", buffer.str()); - } - const auto * function_node = typeid_cast(aggregates.front().get()); if (!function_node) return false; @@ -206,19 +190,22 @@ bool applyTrivialCountIfPossible( if (!function_node->getAggregateFunction()) return false; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Aggregation: {}", function_node->getFunctionName()); - const auto * count_func = typeid_cast(function_node->getAggregateFunction().get()); if (!count_func) return false; /// get number of rows std::optional num_rows{}; - // if (!query_tree. prewhere() && !query.where() && !context->getCurrentTransaction()) - if (!main_query_node.hasPrewhere() && !main_query_node.hasWhere()) + /// Transaction check here is necessary because + /// MergeTree maintains total count for all parts in Active state and it simply returns that number for trivial select count() from table query. + /// But if we have current transaction, then we should return number of rows in current snapshot (that may include parts in Outdated state), + /// so we have to use totalRowsByPartitionPredicate() instead of totalRows even for trivial query + /// See https://github.com/ClickHouse/ClickHouse/pull/24258/files#r828182031 + if (!main_query_node.hasPrewhere() && !main_query_node.hasWhere() && !query_context->getCurrentTransaction()) { num_rows = storage->totalRows(settings); } + // TODO: // else // It's possible to optimize count() given only partition predicates // { // SelectQueryInfo temp_query_info; @@ -231,14 +218,12 @@ bool applyTrivialCountIfPossible( if (!num_rows) return false; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "Number of rows: {}", num_rows.value()); - /// set aggregation state const AggregateFunctionCount & agg_count = *count_func; std::vector state(agg_count.sizeOfData()); AggregateDataPtr place = state.data(); agg_count.create(place); - // SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); + SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); agg_count.set(place, num_rows.value()); auto column = ColumnAggregateFunction::create(function_node->getAggregateFunction()); From 919122948f73d391e938f4bffcdaebdad3a7a256 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 28 Feb 2023 14:22:44 +0000 Subject: [PATCH 110/470] More fixes and add tests --- src/IO/ReadWriteBufferFromHTTP.h | 135 ++++++++++-------- .../test_storage_url/configs/conf.xml | 4 + tests/integration/test_storage_url/test.py | 26 ++++ 3 files changed, 104 insertions(+), 61 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 58abf4a6317..edf7b3f5f95 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -49,22 +49,18 @@ namespace ErrorCodes extern const int UNKNOWN_FILE_SIZE; } -template -class UpdatableSessionBase +template +class UpdatableSession { -protected: - SessionPtr session; - UInt64 redirects{0}; - Poco::URI initial_uri; - ConnectionTimeouts timeouts; - UInt64 max_redirects; - public: - virtual void buildNewSession(const Poco::URI & uri) = 0; + using SessionPtr = typename TSessionFactory::SessionType; - explicit UpdatableSessionBase(const Poco::URI uri, const ConnectionTimeouts & timeouts_, UInt64 max_redirects_) - : initial_uri{uri}, timeouts{timeouts_}, max_redirects{max_redirects_} + explicit UpdatableSession(const Poco::URI & uri, UInt64 max_redirects_, std::shared_ptr session_factory_) + : max_redirects{max_redirects_} + , initial_uri(uri) + , session_factory(std::move(session_factory_)) { + session = session_factory->buildNewSession(uri); } SessionPtr getSession() { return session; } @@ -73,16 +69,21 @@ public: { ++redirects; if (redirects <= max_redirects) - { - buildNewSession(uri); - } + session = session_factory->buildNewSession(uri); else - { throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects while trying to access {}", initial_uri.toString()); - } } - virtual ~UpdatableSessionBase() = default; + std::shared_ptr> clone(const Poco::URI & uri) + { + return std::make_shared>(uri, max_redirects, session_factory); + } +private: + SessionPtr session; + UInt64 redirects{0}; + UInt64 max_redirects; + Poco::URI initial_uri; + std::shared_ptr session_factory; }; @@ -147,7 +148,7 @@ namespace detail size_t getOffset() const { return getRangeBegin() + offset_from_begin_pos; } template - std::istream * callImpl(Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) + std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) { // With empty path poco will send "POST HTTP/1.1" its bug. if (uri_.getPath().empty()) @@ -156,7 +157,6 @@ namespace detail Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header - if (out_stream_callback) request.setChunkedTransferEncoding(true); @@ -191,8 +191,7 @@ namespace detail LOG_TRACE(log, "Sending request to {}", uri_.toString()); - auto sess = session->getSession(); - + auto sess = current_session->getSession(); try { auto & stream_out = sess->sendRequest(request); @@ -200,11 +199,13 @@ namespace detail if (out_stream_callback) out_stream_callback(stream_out); - istr = receiveResponse(*sess, request, response, true); + auto result_istr = receiveResponse(*sess, request, response, true); response.getCookies(cookies); - content_encoding = response.get("Content-Encoding", ""); - return istr; + if constexpr (!use_initial_range) + content_encoding = response.get("Content-Encoding", ""); + + return result_istr; } catch (const Poco::Exception & e) { @@ -362,7 +363,7 @@ namespace detail non_retriable_errors.begin(), non_retriable_errors.end(), [&](const auto status) { return http_status != status; }); } - Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) + static Poco::URI getUriAfterRedirect(const Poco::URI & prev_uri, Poco::Net::HTTPResponse & response) { auto location = response.get("Location"); auto location_uri = Poco::URI(location); @@ -379,7 +380,14 @@ namespace detail template void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { - call(response, method_, throw_on_all_errors); + UpdatableSessionPtr current_session = nullptr; + + if (use_initial_range) + current_session = session->clone(uri); + else + current_session = session; + + call(current_session, response, method_, throw_on_all_errors); Poco::URI prev_uri = uri; while (isRedirect(response.getStatus())) @@ -389,18 +397,22 @@ namespace detail if (remote_host_filter) remote_host_filter->checkURL(uri_redirect); - session->updateSession(uri_redirect); + current_session->updateSession(uri_redirect); - istr = callImpl(uri_redirect, response, method); + auto result_istr = callImpl(current_session, uri_redirect, response, method); + if (!use_initial_range) + istr = result_istr; } } template - void call(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) + void call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { try { - istr = callImpl(saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); + auto result_istr = callImpl(current_session, saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); + if (!use_initial_range) + istr = result_istr; } catch (...) { @@ -436,7 +448,7 @@ namespace detail { Poco::Net::HTTPResponse response; - call(response, method); + call(session, response, method); if (initialization_error != InitializeError::NONE) return; @@ -448,7 +460,7 @@ namespace detail session->updateSession(uri_redirect); - istr = callImpl(uri_redirect, response, method); + istr = callImpl(session, uri_redirect, response, method); saved_uri_redirect = uri_redirect; } @@ -731,23 +743,24 @@ namespace detail }; } -class UpdatableSession : public UpdatableSessionBase +class SessionFactory { - using Parent = UpdatableSessionBase; - public: - UpdatableSession(const Poco::URI uri, const ConnectionTimeouts & timeouts_, const UInt64 max_redirects_) - : Parent(uri, timeouts_, max_redirects_) - { - session = makeHTTPSession(initial_uri, timeouts); - } + explicit SessionFactory(const ConnectionTimeouts & timeouts_) + : timeouts(timeouts_) + {} - void buildNewSession(const Poco::URI & uri) override { session = makeHTTPSession(uri, timeouts); } + using SessionType = HTTPSessionPtr; + + SessionType buildNewSession(const Poco::URI & uri) { return makeHTTPSession(uri, timeouts); } +private: + ConnectionTimeouts timeouts; }; -class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase> +class ReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> { - using Parent = detail::ReadWriteBufferFromHTTPBase>; + using SessionType = UpdatableSession; + using Parent = detail::ReadWriteBufferFromHTTPBase>; public: ReadWriteBufferFromHTTP( @@ -766,7 +779,7 @@ public: bool use_external_buffer_ = false, bool skip_not_found_url_ = false) : Parent( - std::make_shared(uri_, timeouts, max_redirects), + std::make_shared(uri_, max_redirects, std::make_shared(timeouts)), uri_, credentials_, method_, @@ -878,27 +891,27 @@ private: bool skip_not_found_url; }; -class UpdatablePooledSession : public UpdatableSessionBase +class PooledSessionFactory { - using Parent = UpdatableSessionBase; - -private: - size_t per_endpoint_pool_size; - public: - explicit UpdatablePooledSession( - const Poco::URI uri, const ConnectionTimeouts & timeouts_, const UInt64 max_redirects_, size_t per_endpoint_pool_size_) - : Parent(uri, timeouts_, max_redirects_), per_endpoint_pool_size{per_endpoint_pool_size_} - { - session = makePooledHTTPSession(initial_uri, timeouts, per_endpoint_pool_size); - } + explicit PooledSessionFactory( + const ConnectionTimeouts & timeouts_, size_t per_endpoint_pool_size_) + : timeouts(timeouts_) + , per_endpoint_pool_size(per_endpoint_pool_size_) + {} - void buildNewSession(const Poco::URI & uri) override { session = makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size); } + using SessionType = PooledHTTPSessionPtr; + + SessionType buildNewSession(const Poco::URI & uri) { return makePooledHTTPSession(uri, timeouts, per_endpoint_pool_size); } +private: + ConnectionTimeouts timeouts; + size_t per_endpoint_pool_size; }; -class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase> +class PooledReadWriteBufferFromHTTP : public detail::ReadWriteBufferFromHTTPBase>> { - using Parent = detail::ReadWriteBufferFromHTTPBase>; + using SessionType = UpdatableSession; + using Parent = detail::ReadWriteBufferFromHTTPBase>; public: explicit PooledReadWriteBufferFromHTTP( @@ -911,7 +924,7 @@ public: const UInt64 max_redirects = 0, size_t max_connections_per_endpoint = DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT) : Parent( - std::make_shared(uri_, timeouts_, max_redirects, max_connections_per_endpoint), + std::make_shared(uri_, max_redirects, std::make_shared(timeouts_, max_connections_per_endpoint)), uri_, credentials_, method_, diff --git a/tests/integration/test_storage_url/configs/conf.xml b/tests/integration/test_storage_url/configs/conf.xml index 790752c5c04..a9a78f9de71 100644 --- a/tests/integration/test_storage_url/configs/conf.xml +++ b/tests/integration/test_storage_url/configs/conf.xml @@ -6,5 +6,9 @@ TSV column1 UInt32, column2 UInt32, column3 UInt32 + + PUT + column1 UInt32, column2 UInt32, column3 UInt32 + diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 5591e63400c..570f54f354a 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -75,3 +75,29 @@ def test_table_function_url_access_rights(): ["c3", "Nullable(Int64)"], ] ) + + +@pytest.mark.parametrize("file_format", ["Parquet", "CSV", "TSV", "JSONEachRow"]) +def test_file_formats(file_format): + url = f"http://nginx:80/{file_format}_file" + + values = ', '.join([f"({i}, {i + 1}, {i + 2})" for i in range(100)]) + node1.query( + f"insert into table function url(url_file, url = '{url}', format = '{file_format}') values", + stdin=values + ) + + for download_threads in [1, 4, 16]: + result = node1.query( + f""" +SELECT * +FROM url('{url}', '{file_format}') +LIMIT 10 +SETTINGS remote_read_min_bytes_for_seek = 1, max_read_buffer_size = 1, max_download_buffer_size = 1, max_download_threads = {download_threads} +""") + + expected_result = "" + for i in range(10): + expected_result += f"{i}\t{i + 1}\t{i + 2}\n" + + assert result == expected_result From 38d026ca6e627149eee5cd714be87a1b06a68b88 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 28 Feb 2023 14:29:24 +0000 Subject: [PATCH 111/470] Automatic style fix --- tests/integration/test_storage_url/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 570f54f354a..f3ed087d3d0 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -81,10 +81,10 @@ def test_table_function_url_access_rights(): def test_file_formats(file_format): url = f"http://nginx:80/{file_format}_file" - values = ', '.join([f"({i}, {i + 1}, {i + 2})" for i in range(100)]) + values = ", ".join([f"({i}, {i + 1}, {i + 2})" for i in range(100)]) node1.query( - f"insert into table function url(url_file, url = '{url}', format = '{file_format}') values", - stdin=values + f"insert into table function url(url_file, url = '{url}', format = '{file_format}') values", + stdin=values, ) for download_threads in [1, 4, 16]: @@ -94,7 +94,8 @@ SELECT * FROM url('{url}', '{file_format}') LIMIT 10 SETTINGS remote_read_min_bytes_for_seek = 1, max_read_buffer_size = 1, max_download_buffer_size = 1, max_download_threads = {download_threads} -""") +""" + ) expected_result = "" for i in range(10): From d953baa7e359bf768cabc4d7bab799eec8a56ce0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 28 Feb 2023 16:12:51 +0000 Subject: [PATCH 112/470] Review improvements + fixup --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 60 +++++++++++------------ 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 78291d29908..24e94f095e3 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -445,6 +445,12 @@ public: alias_name_to_expressions[node_alias].push_back(node); } + if (const auto * function = node->as()) + { + if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->getFunctionName())) + ++aggregate_functions_counter; + } + expressions.emplace_back(node); } @@ -463,6 +469,12 @@ public: alias_name_to_expressions.erase(it); } + if (const auto * function = top_expression->as()) + { + if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->getFunctionName())) + --aggregate_functions_counter; + } + expressions.pop_back(); } @@ -483,17 +495,7 @@ public: bool hasAggregateFunction() const { - const auto & factory = AggregateFunctionFactory::instance(); - for (const auto & node : expressions) - { - const auto * function = node->as(); - if (!function) - continue; - - if (factory.isAggregateFunctionName(function->getFunctionName())) - return true; - } - return false; + return aggregate_functions_counter > 0; } QueryTreeNodePtr getExpressionWithAlias(const std::string & alias) const @@ -542,6 +544,7 @@ public: private: QueryTreeNodes expressions; + size_t aggregate_functions_counter = 0; std::unordered_map alias_name_to_expressions; }; @@ -3114,11 +3117,6 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook resolve_result.resolve_place = IdentifierResolvePlace::DATABASE_CATALOG; } - if (resolve_result.resolved_identifier - && scope.nullable_group_by_keys.contains(resolve_result.resolved_identifier) - && !scope.expressions_in_resolve_process_stack.hasAggregateFunction()) - resolve_result.resolved_identifier->convertToNullable(); - it->second = resolve_result; /** If identifier was not resolved, or during expression resolution identifier was explicitly added into non cached set, @@ -3126,8 +3124,7 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook */ if (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || - !scope.use_identifier_lookup_to_result_cache || - scope.group_by_use_nulls) + !scope.use_identifier_lookup_to_result_cache) scope.identifier_lookup_to_result.erase(it); return resolve_result; @@ -4655,8 +4652,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } function_node.resolveAsFunction(std::move(function_base)); - if (scope.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) - function_node.convertToNullable(); } catch (Exception & e) { @@ -4905,12 +4900,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id if (result_projection_names.empty()) result_projection_names.push_back(column_node.getColumnName()); - if (scope.group_by_use_nulls && scope.nullable_group_by_keys.contains(node)) - { - node = node->clone(); - node->convertToNullable(); - } - break; } case QueryTreeNodeType::FUNCTION: @@ -4994,6 +4983,14 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id } } + if (node + && scope.nullable_group_by_keys.contains(node) + && !scope.expressions_in_resolve_process_stack.hasAggregateFunction()) + { + node = node->clone(); + node->convertToNullable(); + } + /** Update aliases after expression node was resolved. * Do not update node in alias table if we resolve it for duplicate alias. */ @@ -6074,10 +6071,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier resolveExpressionNodeList(grouping_sets_keys_list_node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); } - for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes()) + if (scope.group_by_use_nulls) { - for (const auto & group_by_elem : grouping_set->as()->getNodes()) - scope.nullable_group_by_keys.insert(group_by_elem->clone()); + for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes()) + { + for (const auto & group_by_elem : grouping_set->as()->getNodes()) + scope.nullable_group_by_keys.insert(group_by_elem); + } } } else @@ -6090,7 +6090,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (scope.group_by_use_nulls) { for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes()) - scope.nullable_group_by_keys.insert(group_by_elem->clone()); + scope.nullable_group_by_keys.insert(group_by_elem); } } } From f53b65a7cc2568d1c4b6850d1e57a88f94dc51ea Mon Sep 17 00:00:00 2001 From: artem-yadr <84010375+artem-yadr@users.noreply.github.com> Date: Tue, 28 Feb 2023 21:01:03 +0300 Subject: [PATCH 113/470] small fix --- base/poco/MongoDB/include/Poco/MongoDB/Connection.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h index 5c5f6f76638..dcb813b75bc 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/Connection.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/Connection.h @@ -162,7 +162,7 @@ namespace MongoDB { return _address; } - inline std::string Connection::uri() const + inline const std::string & Connection::uri() const { return _uri; } From 3f892e52abaf1791b6f1dbe27b3ed6ddad20e7f1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Jan 2023 13:20:38 +0100 Subject: [PATCH 114/470] Revert "Revert "Merge pull request #44922 from azat/dist/async-INSERT-metrics"" This is the revert of revert since there will be follow up patches to address the issues. This reverts commit a55798626a491139d84692a3c546f27cdde71665. --- src/Storages/Distributed/DirectoryMonitor.cpp | 298 +++++++++--------- src/Storages/Distributed/DirectoryMonitor.h | 27 +- src/Storages/Distributed/DistributedSink.cpp | 20 +- src/Storages/StorageDistributed.cpp | 14 +- src/Storages/StorageDistributed.h | 2 +- ..._INSERT_block_structure_mismatch.reference | 4 +- 6 files changed, 190 insertions(+), 175 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index cb6659e59ce..f2a3471d839 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -59,6 +59,7 @@ namespace ErrorCodes extern const int TOO_MANY_PARTITIONS; extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -363,18 +364,22 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( const std::string & relative_path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, - BackgroundSchedulePool & bg_pool) + BackgroundSchedulePool & bg_pool, + bool initialize_from_disk) : storage(storage_) , pool(std::move(pool_)) , disk(disk_) , relative_path(relative_path_) , path(fs::path(disk->getPath()) / relative_path / "") + , broken_relative_path(fs::path(relative_path) / "broken") + , broken_path(fs::path(path) / "broken" / "") , should_batch_inserts(storage.getDistributedSettingsRef().monitor_batch_inserts) , split_batch_on_failure(storage.getDistributedSettingsRef().monitor_split_batch_on_failure) , dir_fsync(storage.getDistributedSettingsRef().fsync_directories) , min_batched_block_size_rows(storage.getContext()->getSettingsRef().min_insert_block_size_rows) , min_batched_block_size_bytes(storage.getContext()->getSettingsRef().min_insert_block_size_bytes) , current_batch_file_path(path + "current_batch.txt") + , pending_files(std::numeric_limits::max()) , default_sleep_time(storage.getDistributedSettingsRef().monitor_sleep_time_ms.totalMilliseconds()) , sleep_time(default_sleep_time) , max_sleep_time(storage.getDistributedSettingsRef().monitor_max_sleep_time_ms.totalMilliseconds()) @@ -383,6 +388,11 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) , metric_broken_files(CurrentMetrics::BrokenDistributedFilesToInsert, 0) { + fs::create_directory(broken_path); + + if (initialize_from_disk) + initializeFilesFromDisk(); + task_handle = bg_pool.createTask(getLoggerName() + "/Bg", [this]{ run(); }); task_handle->activateAndSchedule(); } @@ -390,35 +400,29 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor() { - if (!quit) + if (!pending_files.isFinished()) { - quit = true; + pending_files.clearAndFinish(); task_handle->deactivate(); } } void StorageDistributedDirectoryMonitor::flushAllData() { - if (quit) + if (pending_files.isFinished()) return; std::lock_guard lock{mutex}; - - const auto & files = getFiles(); - if (!files.empty()) - { - processFiles(files); - - /// Update counters. - getFiles(); - } + if (!hasPendingFiles()) + return; + processFiles(); } void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() { - if (!quit) + if (!pending_files.isFinished()) { - quit = true; + pending_files.clearAndFinish(); task_handle->deactivate(); } @@ -432,19 +436,21 @@ void StorageDistributedDirectoryMonitor::run() std::lock_guard lock{mutex}; bool do_sleep = false; - while (!quit) + while (!pending_files.isFinished()) { do_sleep = true; - const auto & files = getFiles(); - if (files.empty()) + if (!hasPendingFiles()) break; if (!monitor_blocker.isCancelled()) { try { - do_sleep = !processFiles(files); + processFiles(); + /// No errors while processing existing files. + /// Let's see maybe there are more files to process. + do_sleep = false; std::lock_guard status_lock(status_mutex); status.last_exception = std::exception_ptr{}; @@ -469,9 +475,7 @@ void StorageDistributedDirectoryMonitor::run() } } else - { LOG_DEBUG(log, "Skipping send data over distributed table."); - } const auto now = std::chrono::system_clock::now(); if (now - last_decrease_time > decrease_error_count_period) @@ -486,10 +490,7 @@ void StorageDistributedDirectoryMonitor::run() break; } - /// Update counters. - getFiles(); - - if (!quit && do_sleep) + if (!pending_files.isFinished() && do_sleep) task_handle->scheduleAfter(sleep_time.count()); } @@ -567,41 +568,83 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri settings.distributed_replica_error_cap); } - -std::map StorageDistributedDirectoryMonitor::getFiles() +bool StorageDistributedDirectoryMonitor::hasPendingFiles() const { - std::map files; + return fs::exists(current_batch_file_path) || !current_batch_file.empty() || !pending_files.empty(); +} + +void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() +{ + /// NOTE: This method does not requires to hold status_mutex, hence, no TSA + /// annotations in the header file. fs::directory_iterator end; - for (fs::directory_iterator it{path}; it != end; ++it) + + /// Initialize pending files { - const auto & file_path_str = it->path(); - if (!it->is_directory() && startsWith(fs::path(file_path_str).extension(), ".bin")) + size_t bytes_count = 0; + + for (fs::directory_iterator it{path}; it != end; ++it) { - files[parse(fs::path(file_path_str).stem())] = file_path_str; + const auto & file_path = it->path(); + const auto & base_name = file_path.stem().string(); + if (!it->is_directory() && startsWith(fs::path(file_path).extension(), ".bin") && parse(base_name)) + { + const std::string & file_path_str = file_path.string(); + if (!pending_files.push(file_path_str)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file"); + bytes_count += fs::file_size(file_path); + } + else if (base_name != "tmp" && base_name != "broken") + { + /// It is OK to log current_batch.txt here too (useful for debugging). + LOG_WARNING(log, "Unexpected file {} in {}", file_path.string(), path); + } } + + LOG_TRACE(log, "Files set to {}", pending_files.size()); + LOG_TRACE(log, "Bytes set to {}", bytes_count); + + metric_pending_files.changeTo(pending_files.size()); + status.files_count = pending_files.size(); + status.bytes_count = bytes_count; } - return files; + /// Initialize broken files + { + size_t broken_bytes_count = 0; + size_t broken_files = 0; + + for (fs::directory_iterator it{broken_path}; it != end; ++it) + { + const auto & file_path = it->path(); + if (!it->is_directory() && startsWith(fs::path(file_path).extension(), ".bin") && parse(file_path.stem())) + broken_bytes_count += fs::file_size(file_path); + else + LOG_WARNING(log, "Unexpected file {} in {}", file_path.string(), broken_path); + } + + LOG_TRACE(log, "Broken files set to {}", broken_files); + LOG_TRACE(log, "Broken bytes set to {}", broken_bytes_count); + + metric_broken_files.changeTo(broken_files); + status.broken_files_count = broken_files; + status.broken_bytes_count = broken_bytes_count; + } } -bool StorageDistributedDirectoryMonitor::processFiles(const std::map & files) +void StorageDistributedDirectoryMonitor::processFiles() { if (should_batch_inserts) - { - processFilesWithBatching(files); - } + processFilesWithBatching(); else { - for (const auto & file : files) - { - if (quit) - return true; + /// Process unprocessed file. + if (!current_batch_file.empty()) + processFile(current_batch_file); - processFile(file.second); - } + while (pending_files.tryPop(current_batch_file)) + processFile(current_batch_file); } - - return true; } void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path) @@ -648,7 +691,11 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa thread_trace_context->root_span.addAttribute(std::current_exception()); e.addMessage(fmt::format("While sending {}", file_path)); - maybeMarkAsBroken(file_path, e); + if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) + { + markAsBroken(file_path); + current_batch_file.clear(); + } throw; } catch (...) @@ -661,6 +708,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); markAsSend(file_path); + current_batch_file.clear(); LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds()); } @@ -700,25 +748,19 @@ struct StorageDistributedDirectoryMonitor::BatchHeader struct StorageDistributedDirectoryMonitor::Batch { - /// File indexes for this batch. - std::vector file_indices; size_t total_rows = 0; size_t total_bytes = 0; bool recovered = false; StorageDistributedDirectoryMonitor & parent; - /// Information about all available indexes (not only for the current batch). - const std::map & file_index_to_path; + std::vector files; bool split_batch_on_failure = true; bool fsync = false; bool dir_fsync = false; - Batch( - StorageDistributedDirectoryMonitor & parent_, - const std::map & file_index_to_path_) + explicit Batch(StorageDistributedDirectoryMonitor & parent_) : parent(parent_) - , file_index_to_path(file_index_to_path_) , split_batch_on_failure(parent.split_batch_on_failure) , fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert) , dir_fsync(parent.dir_fsync) @@ -733,7 +775,7 @@ struct StorageDistributedDirectoryMonitor::Batch void send() { - if (file_indices.empty()) + if (files.empty()) return; CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; @@ -776,7 +818,7 @@ struct StorageDistributedDirectoryMonitor::Batch } catch (const Exception & e) { - if (split_batch_on_failure && file_indices.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException())) + if (split_batch_on_failure && files.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException())) { tryLogCurrentException(parent.log, "Trying to split batch due to"); sendSeparateFiles(); @@ -796,49 +838,28 @@ struct StorageDistributedDirectoryMonitor::Batch } else { - std::vector files; - for (auto file_index_info : file_indices | boost::adaptors::indexed()) - { - if (file_index_info.index() > 8) - { - files.push_back("..."); - break; - } - - auto file_index = file_index_info.value(); - auto file_path = file_index_to_path.find(file_index); - if (file_path != file_index_to_path.end()) - files.push_back(file_path->second); - else - files.push_back(fmt::format("#{}.bin (deleted)", file_index)); - } - e.addMessage(fmt::format("While sending batch, size: {}, files: {}", file_indices.size(), fmt::join(files, "\n"))); - + e.addMessage(fmt::format("While sending a batch of {} files, files: {}", files.size(), fmt::join(files, "\n"))); throw; } } if (!batch_broken) { - LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", file_indices.size(), watch.elapsedMilliseconds()); + LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", files.size(), watch.elapsedMilliseconds()); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - for (UInt64 file_index : file_indices) - parent.markAsSend(file_index_to_path.at(file_index)); + for (const auto & file : files) + parent.markAsSend(file); } else if (!batch_marked_as_broken) { - LOG_ERROR(parent.log, "Marking a batch of {} files as broken.", file_indices.size()); + LOG_ERROR(parent.log, "Marking a batch of {} files as broken, files: {}", files.size(), fmt::join(files, "\n")); - for (UInt64 file_idx : file_indices) - { - auto file_path = file_index_to_path.find(file_idx); - if (file_path != file_index_to_path.end()) - parent.markAsBroken(file_path->second); - } + for (const auto & file : files) + parent.markAsBroken(file); } - file_indices.clear(); + files.clear(); total_rows = 0; total_bytes = 0; recovered = false; @@ -848,8 +869,11 @@ struct StorageDistributedDirectoryMonitor::Batch void writeText(WriteBuffer & out) { - for (UInt64 file_idx : file_indices) - out << file_idx << '\n'; + for (const auto & file : files) + { + UInt64 file_index = parse(fs::path(file).stem()); + out << file_index << '\n'; + } } void readText(ReadBuffer & in) @@ -858,8 +882,9 @@ struct StorageDistributedDirectoryMonitor::Batch { UInt64 idx; in >> idx >> "\n"; - file_indices.push_back(idx); + files.push_back(fmt::format("{}/{}.bin", parent.path, idx)); } + recovered = true; } @@ -871,14 +896,9 @@ private: IConnectionPool::Entry connection; - for (UInt64 file_idx : file_indices) + for (const auto & file : files) { - auto file_path = file_index_to_path.find(file_idx); - if (file_path == file_index_to_path.end()) - throw Exception(ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO, - "Failed to send batch: file with index {} is absent", file_idx); - - ReadBufferFromFile in(file_path->second); + ReadBufferFromFile in(file); const auto & distributed_header = readDistributedHeader(in, parent.log); OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, @@ -892,7 +912,7 @@ private: compression_expected = connection->getCompression() == Protocol::Compression::Enable; LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", - file_indices.size(), + files.size(), connection->getDescription(), formatReadableQuantity(total_rows), formatReadableSizeWithBinarySuffix(total_bytes)); @@ -913,19 +933,11 @@ private: { size_t broken_files = 0; - for (UInt64 file_idx : file_indices) + for (const auto & file : files) { - auto file_path = file_index_to_path.find(file_idx); - if (file_path == file_index_to_path.end()) - { - LOG_ERROR(parent.log, "Failed to send one file from batch: file with index {} is absent", file_idx); - ++broken_files; - continue; - } - try { - ReadBufferFromFile in(file_path->second); + ReadBufferFromFile in(file); const auto & distributed_header = readDistributedHeader(in, parent.log); // this function is called in a separated thread, so we set up the trace context from the file @@ -947,9 +959,11 @@ private: } catch (Exception & e) { - e.addMessage(fmt::format("While sending {}", file_path->second)); - parent.maybeMarkAsBroken(file_path->second, e); - ++broken_files; + if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) + { + parent.markAsBroken(file); + ++broken_files; + } } } @@ -1029,13 +1043,18 @@ std::shared_ptr StorageDistributedDirectoryMonitor::createSourceFromFil return std::make_shared(file_name); } -bool StorageDistributedDirectoryMonitor::addAndSchedule(size_t file_size, size_t ms) +bool StorageDistributedDirectoryMonitor::addAndSchedule(const std::string & file_path, size_t file_size, size_t ms) { - if (quit) + /// NOTE: It is better not to throw in this case, since the file is already + /// on disk (see DistributedSink), and it will be processed next time. + if (pending_files.isFinished()) return false; + if (!pending_files.push(file_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file"); + { - std::lock_guard status_lock(status_mutex); + std::lock_guard lock(status_mutex); metric_pending_files.add(); status.bytes_count += file_size; ++status.files_count; @@ -1051,33 +1070,25 @@ StorageDistributedDirectoryMonitor::Status StorageDistributedDirectoryMonitor::g return current_status; } -void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map & files) +void StorageDistributedDirectoryMonitor::processFilesWithBatching() { - std::unordered_set file_indices_to_skip; - + /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. if (fs::exists(current_batch_file_path)) { - /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. - Batch batch(*this, files); + Batch batch(*this); ReadBufferFromFile in{current_batch_file_path}; batch.readText(in); - file_indices_to_skip.insert(batch.file_indices.begin(), batch.file_indices.end()); batch.send(); + + auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); + fs::remove(current_batch_file_path); } std::unordered_map header_to_batch; - for (const auto & file : files) + std::string file_path; + while (pending_files.tryPop(file_path)) { - if (quit) - return; - - UInt64 file_idx = file.first; - const String & file_path = file.second; - - if (file_indices_to_skip.contains(file_idx)) - continue; - size_t total_rows = 0; size_t total_bytes = 0; Block header; @@ -1116,8 +1127,9 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map } catch (const Exception & e) { - if (maybeMarkAsBroken(file_path, e)) + if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) { + markAsBroken(file_path); tryLogCurrentException(log, "File is marked broken due to"); continue; } @@ -1131,9 +1143,9 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map std::move(distributed_header.client_info), std::move(header) ); - Batch & batch = header_to_batch.try_emplace(batch_header, *this, files).first->second; + Batch & batch = header_to_batch.try_emplace(batch_header, *this).first->second; - batch.file_indices.push_back(file_idx); + batch.files.push_back(file_path); batch.total_rows += total_rows; batch.total_bytes += total_bytes; @@ -1161,16 +1173,10 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_path) { - const auto last_path_separator_pos = file_path.rfind('/'); - const auto & base_path = file_path.substr(0, last_path_separator_pos + 1); - const auto & file_name = file_path.substr(last_path_separator_pos + 1); - const String & broken_path = fs::path(base_path) / "broken/"; - const String & broken_file_path = fs::path(broken_path) / file_name; - - fs::create_directory(broken_path); + const String & broken_file_path = fs::path(broken_path) / fs::path(file_path).filename(); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, fs::path(relative_path) / "broken/"); + auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, broken_relative_path); { std::lock_guard status_lock(status_mutex); @@ -1204,21 +1210,9 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat fs::remove(file_path); } -bool StorageDistributedDirectoryMonitor::maybeMarkAsBroken(const std::string & file_path, const Exception & e) -{ - /// Mark file as broken if necessary. - if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) - { - markAsBroken(file_path); - return true; - } - else - return false; -} - std::string StorageDistributedDirectoryMonitor::getLoggerName() const { - return storage.getStorageID().getFullTableName() + ".DirectoryMonitor"; + return storage.getStorageID().getFullTableName() + ".DirectoryMonitor." + disk->getName(); } void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_relative_path) diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 030d6acf6e2..313125e16bd 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -38,7 +39,8 @@ public: const std::string & relative_path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, - BackgroundSchedulePool & bg_pool); + BackgroundSchedulePool & bg_pool, + bool initialize_from_disk); ~StorageDistributedDirectoryMonitor(); @@ -53,7 +55,7 @@ public: static std::shared_ptr createSourceFromFile(const String & file_name); /// For scheduling via DistributedSink. - bool addAndSchedule(size_t file_size, size_t ms); + bool addAndSchedule(const std::string & file_path, size_t file_size, size_t ms); struct InternalStatus { @@ -79,14 +81,15 @@ public: private: void run(); - std::map getFiles(); - bool processFiles(const std::map & files); + bool hasPendingFiles() const; + + void initializeFilesFromDisk(); + void processFiles(); void processFile(const std::string & file_path); - void processFilesWithBatching(const std::map & files); + void processFilesWithBatching(); void markAsBroken(const std::string & file_path); void markAsSend(const std::string & file_path); - bool maybeMarkAsBroken(const std::string & file_path, const Exception & e); std::string getLoggerName() const; @@ -96,25 +99,33 @@ private: DiskPtr disk; std::string relative_path; std::string path; + std::string broken_relative_path; + std::string broken_path; const bool should_batch_inserts = false; const bool split_batch_on_failure = true; const bool dir_fsync = false; const size_t min_batched_block_size_rows = 0; const size_t min_batched_block_size_bytes = 0; - String current_batch_file_path; + + /// This is pending data (due to some error) for should_batch_inserts==true + std::string current_batch_file_path; + /// This is pending data (due to some error) for should_batch_inserts==false + std::string current_batch_file; struct BatchHeader; struct Batch; std::mutex status_mutex; + InternalStatus status; + ConcurrentBoundedQueue pending_files; + const std::chrono::milliseconds default_sleep_time; std::chrono::milliseconds sleep_time; const std::chrono::milliseconds max_sleep_time; std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; - std::atomic quit {false}; std::mutex mutex; Poco::Logger * log; ActionBlocker & monitor_blocker; diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index bac13ea37cf..edbb2acc923 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -724,6 +724,9 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const return guard; }; + std::vector bin_files; + bin_files.reserve(dir_names.size()); + auto it = dir_names.begin(); /// on first iteration write block to a temporary directory for subsequent /// hardlinking to ensure the inode is not freed until we're done @@ -802,8 +805,8 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const } // Create hardlink here to reuse increment number - const std::string block_file_path(fs::path(path) / file_name); - createHardLink(first_file_tmp_path, block_file_path); + bin_files.push_back(fs::path(path) / file_name); + createHardLink(first_file_tmp_path, bin_files.back()); auto dir_sync_guard = make_directory_sync_guard(*it); } ++it; @@ -814,8 +817,8 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const const std::string path(fs::path(disk_path) / (data_path + *it)); fs::create_directory(path); - const std::string block_file_path(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")); - createHardLink(first_file_tmp_path, block_file_path); + bin_files.push_back(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")); + createHardLink(first_file_tmp_path, bin_files.back()); auto dir_sync_guard = make_directory_sync_guard(*it); } @@ -826,10 +829,13 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const /// Notify auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms; - for (const auto & dir_name : dir_names) + for (size_t i = 0; i < dir_names.size(); ++i) { - auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name); - directory_monitor.addAndSchedule(file_size, sleep_ms.totalMilliseconds()); + const auto & dir_name = dir_names[i]; + const auto & bin_file = bin_files[i]; + + auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name, /* startup= */ false); + directory_monitor.addAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds()); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5516d6cadf0..a38beef983c 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1208,12 +1208,15 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) const auto & dir_path = it->path(); if (std::filesystem::is_directory(dir_path)) { + /// Created by DistributedSink const auto & tmp_path = dir_path / "tmp"; - - /// "tmp" created by DistributedSink if (std::filesystem::is_directory(tmp_path) && std::filesystem::is_empty(tmp_path)) std::filesystem::remove(tmp_path); + const auto & broken_path = dir_path / "broken"; + if (std::filesystem::is_directory(broken_path) && std::filesystem::is_empty(broken_path)) + std::filesystem::remove(broken_path); + if (std::filesystem::is_empty(dir_path)) { LOG_DEBUG(log, "Removing {} (used for async INSERT into Distributed)", dir_path.string()); @@ -1222,14 +1225,14 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) } else { - requireDirectoryMonitor(disk, dir_path.filename().string()); + requireDirectoryMonitor(disk, dir_path.filename().string(), /* startup= */ true); } } } } -StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const DiskPtr & disk, const std::string & name) +StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const DiskPtr & disk, const std::string & name, bool startup) { const std::string & disk_path = disk->getPath(); const std::string key(disk_path + name); @@ -1243,7 +1246,8 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor( *this, disk, relative_data_path + name, node_data.connection_pool, monitors_blocker, - getContext()->getDistributedSchedulePool()); + getContext()->getDistributedSchedulePool(), + /* initialize_from_disk= */ startup); } return *node_data.directory_monitor; } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 66fd7b77889..25a752fe795 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -166,7 +166,7 @@ private: /// create directory monitors for each existing subdirectory void createDirectoryMonitors(const DiskPtr & disk); /// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name - StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const DiskPtr & disk, const std::string & name); + StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const DiskPtr & disk, const std::string & name, bool startup); /// Return list of metrics for all created monitors /// (note that monitors are created lazily, i.e. until at least one INSERT executed) diff --git a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.reference b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.reference index f3be69d3279..b0d8284faa5 100644 --- a/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.reference +++ b/tests/queries/0_stateless/01791_dist_INSERT_block_structure_mismatch.reference @@ -1,7 +1,7 @@ DistributedSink: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 1)), implicit conversion will be done. DistributedSink: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 1)), implicit conversion will be done. - default.dist_01683.DirectoryMonitor: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done - default.dist_01683.DirectoryMonitor: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done + default.dist_01683.DirectoryMonitor.default: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done + default.dist_01683.DirectoryMonitor.default: Structure does not match (remote: n Int8 Int8(size = 0), local: n UInt64 UInt64(size = 0)), implicit conversion will be done 1 1 2 From 16646c0923338adfd20e2853de4a1b9373534d58 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:07:44 +0100 Subject: [PATCH 115/470] Rename DirectoryMonitor::current_batch_file to current_file Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 14 +++++++------- src/Storages/Distributed/DirectoryMonitor.h | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f2a3471d839..e536f1e63d8 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -570,7 +570,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri bool StorageDistributedDirectoryMonitor::hasPendingFiles() const { - return fs::exists(current_batch_file_path) || !current_batch_file.empty() || !pending_files.empty(); + return fs::exists(current_batch_file_path) || !current_file.empty() || !pending_files.empty(); } void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() @@ -639,11 +639,11 @@ void StorageDistributedDirectoryMonitor::processFiles() else { /// Process unprocessed file. - if (!current_batch_file.empty()) - processFile(current_batch_file); + if (!current_file.empty()) + processFile(current_file); - while (pending_files.tryPop(current_batch_file)) - processFile(current_batch_file); + while (pending_files.tryPop(current_file)) + processFile(current_file); } } @@ -694,7 +694,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) { markAsBroken(file_path); - current_batch_file.clear(); + current_file.clear(); } throw; } @@ -708,7 +708,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); markAsSend(file_path); - current_batch_file.clear(); + current_file.clear(); LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds()); } diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 313125e16bd..06843b77a4b 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -111,7 +111,7 @@ private: /// This is pending data (due to some error) for should_batch_inserts==true std::string current_batch_file_path; /// This is pending data (due to some error) for should_batch_inserts==false - std::string current_batch_file; + std::string current_file; struct BatchHeader; struct Batch; From ef1e642e059510ff182a6a9dc97d34fdb7c1a071 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:12:50 +0100 Subject: [PATCH 116/470] Add log message to StorageDistributedDirectoryMonitor::addAndSchedule() Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index e536f1e63d8..c04e54b6bba 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1048,7 +1048,10 @@ bool StorageDistributedDirectoryMonitor::addAndSchedule(const std::string & file /// NOTE: It is better not to throw in this case, since the file is already /// on disk (see DistributedSink), and it will be processed next time. if (pending_files.isFinished()) + { + LOG_DEBUG(log, "File {} had not been scheduled, since the table had been detached", file_path); return false; + } if (!pending_files.push(file_path)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file"); From f0a2efa63014ff1349340e28647ab067698cf974 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:20:28 +0100 Subject: [PATCH 117/470] Always manipulate with absolute file paths in DirectoryMonitor Otherwise on batch restore we can get the difference in file paths. Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 16 ++++++++++------ src/Storages/Distributed/DirectoryMonitor.h | 3 ++- src/Storages/Distributed/DistributedSink.cpp | 2 +- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index c04e54b6bba..eead8c8ea42 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -573,6 +573,12 @@ bool StorageDistributedDirectoryMonitor::hasPendingFiles() const return fs::exists(current_batch_file_path) || !current_file.empty() || !pending_files.empty(); } +void StorageDistributedDirectoryMonitor::addFile(const std::string & file_path) +{ + if (!pending_files.push(fs::absolute(file_path).string())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot schedule a file '{}'", file_path); +} + void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() { /// NOTE: This method does not requires to hold status_mutex, hence, no TSA @@ -591,8 +597,7 @@ void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() if (!it->is_directory() && startsWith(fs::path(file_path).extension(), ".bin") && parse(base_name)) { const std::string & file_path_str = file_path.string(); - if (!pending_files.push(file_path_str)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file"); + addFile(file_path_str); bytes_count += fs::file_size(file_path); } else if (base_name != "tmp" && base_name != "broken") @@ -882,7 +887,7 @@ struct StorageDistributedDirectoryMonitor::Batch { UInt64 idx; in >> idx >> "\n"; - files.push_back(fmt::format("{}/{}.bin", parent.path, idx)); + files.push_back(fs::absolute(fmt::format("{}/{}.bin", parent.path, idx)).string()); } recovered = true; @@ -1043,7 +1048,7 @@ std::shared_ptr StorageDistributedDirectoryMonitor::createSourceFromFil return std::make_shared(file_name); } -bool StorageDistributedDirectoryMonitor::addAndSchedule(const std::string & file_path, size_t file_size, size_t ms) +bool StorageDistributedDirectoryMonitor::addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms) { /// NOTE: It is better not to throw in this case, since the file is already /// on disk (see DistributedSink), and it will be processed next time. @@ -1053,8 +1058,7 @@ bool StorageDistributedDirectoryMonitor::addAndSchedule(const std::string & file return false; } - if (!pending_files.push(file_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add pending file"); + addFile(file_path); { std::lock_guard lock(status_mutex); diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 06843b77a4b..9b1596d45e3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -55,7 +55,7 @@ public: static std::shared_ptr createSourceFromFile(const String & file_name); /// For scheduling via DistributedSink. - bool addAndSchedule(const std::string & file_path, size_t file_size, size_t ms); + bool addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms); struct InternalStatus { @@ -83,6 +83,7 @@ private: bool hasPendingFiles() const; + void addFile(const std::string & file_path); void initializeFilesFromDisk(); void processFiles(); void processFile(const std::string & file_path); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index edbb2acc923..c9c235596db 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -835,7 +835,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const const auto & bin_file = bin_files[i]; auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name, /* startup= */ false); - directory_monitor.addAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds()); + directory_monitor.addFileAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds()); } } From 13a3e03f19b13f0210e93e28f433e3abee7b3fd9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:27:59 +0100 Subject: [PATCH 118/470] Introduce StorageDistributedDirectoryMonitor::Batch::{de,}serialize() Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 51 +++++++++++-------- 1 file changed, 30 insertions(+), 21 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index eead8c8ea42..fe5018db821 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -793,24 +793,7 @@ struct StorageDistributedDirectoryMonitor::Batch /// we must try to re-send exactly the same batches. /// So we save contents of the current batch into the current_batch_file_path file /// and truncate it afterwards if all went well. - - /// Temporary file is required for atomicity. - String tmp_file{parent.current_batch_file_path + ".tmp"}; - - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - if (fs::exists(tmp_file)) - LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); - - { - WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT}; - writeText(out); - - out.finalize(); - if (fsync) - out.sync(); - } - - fs::rename(tmp_file, parent.current_batch_file_path); + serialize(); } bool batch_broken = false; @@ -872,6 +855,34 @@ struct StorageDistributedDirectoryMonitor::Batch fs::resize_file(parent.current_batch_file_path, 0); } + void serialize() + { + /// Temporary file is required for atomicity. + String tmp_file{parent.current_batch_file_path + ".tmp"}; + + auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); + if (fs::exists(tmp_file)) + LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); + + { + WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT}; + writeText(out); + + out.finalize(); + if (fsync) + out.sync(); + } + + fs::rename(tmp_file, parent.current_batch_file_path); + } + + void deserialize() + { + ReadBufferFromFile in{parent.current_batch_file_path}; + readText(in); + } + +private: void writeText(WriteBuffer & out) { for (const auto & file : files) @@ -893,7 +904,6 @@ struct StorageDistributedDirectoryMonitor::Batch recovered = true; } -private: void sendBatch() { std::unique_ptr remote; @@ -1083,8 +1093,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() if (fs::exists(current_batch_file_path)) { Batch batch(*this); - ReadBufferFromFile in{current_batch_file_path}; - batch.readText(in); + batch.deserialize(); batch.send(); auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); From 0c19a75a1c3225d8ec6625e2ca37434245f63a4c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:29:59 +0100 Subject: [PATCH 119/470] Add log message for batch restore Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index fe5018db821..c86661eb2ec 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1092,6 +1092,8 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. if (fs::exists(current_batch_file_path)) { + LOG_DEBUG(log, "Restoring the batch"); + Batch batch(*this); batch.deserialize(); batch.send(); From 22a39e29f760969607b728a9de671c18071cbfb4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 10:41:38 +0100 Subject: [PATCH 120/470] Add a comment for StorageDistributedDirectoryMonitor::Batch::recovered Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index c86661eb2ec..06aa5219be0 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -755,6 +755,7 @@ struct StorageDistributedDirectoryMonitor::Batch { size_t total_rows = 0; size_t total_bytes = 0; + /// Does the batch had been created from the files in current_batch.txt? bool recovered = false; StorageDistributedDirectoryMonitor & parent; From 325a7b23050f3964704f87fabfacc92b0b737e9a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 16:53:51 +0100 Subject: [PATCH 121/470] Separate out DistributedHeader as DistributedAsyncInsertHeader Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 17 ++- .../DistributedAsyncInsertHeader.cpp | 109 ++++++++++++++++++ .../DistributedAsyncInsertHeader.h | 38 ++++++ 3 files changed, 155 insertions(+), 9 deletions(-) create mode 100644 src/Storages/Distributed/DistributedAsyncInsertHeader.cpp create mode 100644 src/Storages/Distributed/DistributedAsyncInsertHeader.h diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 06aa5219be0..469f2f90954 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -12,10 +12,9 @@ #include #include #include -#include +#include #include #include -#include #include #include #include @@ -277,7 +276,7 @@ namespace return nullptr; } - void writeAndConvert(RemoteInserter & remote, const DistributedHeader & distributed_header, ReadBufferFromFile & in) + void writeAndConvert(RemoteInserter & remote, const DistributedAsyncInsertHeader & distributed_header, ReadBufferFromFile & in) { CompressedReadBuffer decompressing_in(in); NativeReader block_in(decompressing_in, distributed_header.revision); @@ -296,7 +295,7 @@ namespace } void writeRemoteConvert( - const DistributedHeader & distributed_header, + const DistributedAsyncInsertHeader & distributed_header, RemoteInserter & remote, bool compression_expected, ReadBufferFromFile & in, @@ -662,7 +661,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; ReadBufferFromFile in(file_path); - const auto & distributed_header = readDistributedHeader(in, log); + const auto & distributed_header = readDistributedAsyncInsertHeader(in, log); thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, @@ -915,7 +914,7 @@ private: for (const auto & file : files) { ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedHeader(in, parent.log); + const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, @@ -954,7 +953,7 @@ private: try { ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedHeader(in, parent.log); + const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); // this function is called in a separated thread, so we set up the trace context from the file OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, @@ -1111,12 +1110,12 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() size_t total_rows = 0; size_t total_bytes = 0; Block header; - DistributedHeader distributed_header; + DistributedAsyncInsertHeader distributed_header; try { /// Determine metadata of the current file and check if it is not broken. ReadBufferFromFile in{file_path}; - distributed_header = readDistributedHeader(in, log); + distributed_header = readDistributedAsyncInsertHeader(in, log); if (distributed_header.rows) { diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp new file mode 100644 index 00000000000..19235c91cc6 --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -0,0 +1,109 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; + extern const int CHECKSUM_DOESNT_MATCH; +} + +DistributedAsyncInsertHeader readDistributedAsyncInsertHeader(ReadBufferFromFile & in, Poco::Logger * log) +{ + DistributedAsyncInsertHeader distributed_header; + + UInt64 query_size; + readVarUInt(query_size, in); + + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER) + { + /// Read the header as a string. + String header_data; + readStringBinary(header_data, in); + + /// Check the checksum of the header. + CityHash_v1_0_2::uint128 expected_checksum; + readPODBinary(expected_checksum, in); + CityHash_v1_0_2::uint128 calculated_checksum = + CityHash_v1_0_2::CityHash128(header_data.data(), header_data.size()); + if (expected_checksum != calculated_checksum) + { + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Checksum of extra info doesn't match: corrupted data. Reference: {}{}. Actual: {}{}.", + getHexUIntLowercase(expected_checksum.first), getHexUIntLowercase(expected_checksum.second), + getHexUIntLowercase(calculated_checksum.first), getHexUIntLowercase(calculated_checksum.second)); + } + + /// Read the parts of the header. + ReadBufferFromString header_buf(header_data); + + readVarUInt(distributed_header.revision, header_buf); + if (DBMS_TCP_PROTOCOL_VERSION < distributed_header.revision) + { + LOG_WARNING(log, "ClickHouse shard version is older than ClickHouse initiator version. It may lack support for new features."); + } + + readStringBinary(distributed_header.insert_query, header_buf); + distributed_header.insert_settings.read(header_buf); + + if (header_buf.hasPendingData()) + distributed_header.client_info.read(header_buf, distributed_header.revision); + + if (header_buf.hasPendingData()) + { + readVarUInt(distributed_header.rows, header_buf); + readVarUInt(distributed_header.bytes, header_buf); + readStringBinary(distributed_header.block_header_string, header_buf); + } + + if (header_buf.hasPendingData()) + { + NativeReader header_block_in(header_buf, distributed_header.revision); + distributed_header.block_header = header_block_in.read(); + if (!distributed_header.block_header) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read header from the {} batch. Data was written with protocol version {}, current version: {}", + in.getFileName(), distributed_header.revision, DBMS_TCP_PROTOCOL_VERSION); + } + + if (header_buf.hasPendingData()) + { + readVarUInt(distributed_header.shard_num, header_buf); + readStringBinary(distributed_header.cluster, header_buf); + readStringBinary(distributed_header.distributed_table, header_buf); + readStringBinary(distributed_header.remote_table, header_buf); + } + + /// Add handling new data here, for example: + /// + /// if (header_buf.hasPendingData()) + /// readVarUInt(my_new_data, header_buf); + /// + /// And note that it is safe, because we have checksum and size for header. + + return distributed_header; + } + + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT) + { + distributed_header.insert_settings.read(in, SettingsWriteFormat::BINARY); + readStringBinary(distributed_header.insert_query, in); + return distributed_header; + } + + distributed_header.insert_query.resize(query_size); + in.readStrict(distributed_header.insert_query.data(), query_size); + + return distributed_header; +} + +} diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.h b/src/Storages/Distributed/DistributedAsyncInsertHeader.h new file mode 100644 index 00000000000..2c7a6477b6c --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ReadBufferFromFile; + +/// Header for the binary files that are stored on disk for async INSERT into Distributed. +struct DistributedAsyncInsertHeader +{ + UInt64 revision = 0; + Settings insert_settings; + std::string insert_query; + ClientInfo client_info; + + /// .bin file cannot have zero rows/bytes. + size_t rows = 0; + size_t bytes = 0; + + UInt32 shard_num = 0; + std::string cluster; + std::string distributed_table; + std::string remote_table; + + /// dumpStructure() of the header -- obsolete + std::string block_header_string; + Block block_header; +}; + +DistributedAsyncInsertHeader readDistributedAsyncInsertHeader(ReadBufferFromFile & in, Poco::Logger * log); + +} From 33b13549ad6f39fabad4474ae6055458ce8a041f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 17:01:41 +0100 Subject: [PATCH 122/470] Separate out DirectoryMonitorSource as DistributedAsyncInsertSource Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 70 ------------------- .../DistributedAsyncInsertSource.cpp | 62 ++++++++++++++++ .../DistributedAsyncInsertSource.h | 28 ++++++++ src/Storages/StorageFile.cpp | 7 +- 4 files changed, 93 insertions(+), 74 deletions(-) create mode 100644 src/Storages/Distributed/DistributedAsyncInsertSource.cpp create mode 100644 src/Storages/Distributed/DistributedAsyncInsertSource.h diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 469f2f90954..c51e9c6e061 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -988,76 +988,6 @@ private: } }; -class DirectoryMonitorSource : public ISource -{ -public: - - struct Data - { - std::unique_ptr in; - std::unique_ptr decompressing_in; - std::unique_ptr block_in; - - Poco::Logger * log = nullptr; - - Block first_block; - - explicit Data(const String & file_name) - { - in = std::make_unique(file_name); - decompressing_in = std::make_unique(*in); - log = &Poco::Logger::get("DirectoryMonitorSource"); - - auto distributed_header = readDistributedHeader(*in, log); - block_in = std::make_unique(*decompressing_in, distributed_header.revision); - - first_block = block_in->read(); - } - - Data(Data &&) = default; - }; - - explicit DirectoryMonitorSource(const String & file_name) - : DirectoryMonitorSource(Data(file_name)) - { - } - - explicit DirectoryMonitorSource(Data data_) - : ISource(data_.first_block.cloneEmpty()) - , data(std::move(data_)) - { - } - - String getName() const override { return "DirectoryMonitorSource"; } - -protected: - Chunk generate() override - { - if (data.first_block) - { - size_t num_rows = data.first_block.rows(); - Chunk res(data.first_block.getColumns(), num_rows); - data.first_block.clear(); - return res; - } - - auto block = data.block_in->read(); - if (!block) - return {}; - - size_t num_rows = block.rows(); - return Chunk(block.getColumns(), num_rows); - } - -private: - Data data; -}; - -std::shared_ptr StorageDistributedDirectoryMonitor::createSourceFromFile(const String & file_name) -{ - return std::make_shared(file_name); -} - bool StorageDistributedDirectoryMonitor::addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms) { /// NOTE: It is better not to throw in this case, since the file is already diff --git a/src/Storages/Distributed/DistributedAsyncInsertSource.cpp b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp new file mode 100644 index 00000000000..782cbf9c026 --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp @@ -0,0 +1,62 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct DistributedAsyncInsertSource::Data +{ + Poco::Logger * log = nullptr; + + ReadBufferFromFile in; + CompressedReadBuffer decompressing_in; + NativeReader block_in; + + Block first_block; + + explicit Data(const String & file_name) + : log(&Poco::Logger::get("DistributedAsyncInsertSource")) + , in(file_name) + , decompressing_in(in) + , block_in(decompressing_in, readDistributedAsyncInsertHeader(in, log).revision) + , first_block(block_in.read()) + { + } +}; + +DistributedAsyncInsertSource::DistributedAsyncInsertSource(const String & file_name) + : DistributedAsyncInsertSource(std::make_unique(file_name)) +{ +} + +DistributedAsyncInsertSource::DistributedAsyncInsertSource(std::unique_ptr data_) + : ISource(data_->first_block.cloneEmpty()) + , data(std::move(data_)) +{ +} + +DistributedAsyncInsertSource::~DistributedAsyncInsertSource() = default; + +Chunk DistributedAsyncInsertSource::generate() +{ + if (data->first_block) + { + size_t num_rows = data->first_block.rows(); + Chunk res(data->first_block.getColumns(), num_rows); + data->first_block.clear(); + return res; + } + + auto block = data->block_in.read(); + if (!block) + return {}; + + size_t num_rows = block.rows(); + return Chunk(block.getColumns(), num_rows); +} + +} diff --git a/src/Storages/Distributed/DistributedAsyncInsertSource.h b/src/Storages/Distributed/DistributedAsyncInsertSource.h new file mode 100644 index 00000000000..35f846151dc --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertSource.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Source for the Distributed engine on-disk file for async INSERT. +class DistributedAsyncInsertSource : public ISource +{ + struct Data; + explicit DistributedAsyncInsertSource(std::unique_ptr data); + +public: + explicit DistributedAsyncInsertSource(const String & file_name); + ~DistributedAsyncInsertSource() override; + String getName() const override { return "DistributedAsyncInsertSource"; } + +protected: + Chunk generate() override; + +private: + std::unique_ptr data; +}; + +} diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e2a2f84bc72..5fd5664b9e6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -368,8 +368,7 @@ ColumnsDescription StorageFile::getTableStructureFromFile( if (paths.empty()) throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Cannot get table structure from file, because no files match specified name"); - auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]); - return ColumnsDescription(source->getOutputs().front().getHeader().getNamesAndTypesList()); + return ColumnsDescription(DistributedAsyncInsertSource(paths[0]).getOutputs().front().getHeader().getNamesAndTypesList()); } if (paths.empty() && !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format)) @@ -597,7 +596,7 @@ public: /// Special case for distributed format. Defaults are not needed here. if (storage->format_name == "Distributed") { - pipeline = std::make_unique(StorageDistributedDirectoryMonitor::createSourceFromFile(current_path)); + pipeline = std::make_unique(std::make_shared(current_path)); reader = std::make_unique(*pipeline); continue; } From 1c4659b8e705b9a4c46929524bd392eaa6609f1d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 18:12:42 +0100 Subject: [PATCH 123/470] Separate out Batch as DistributedAsyncInsertBatch (and also some helpers) Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DirectoryMonitor.cpp | 598 +++--------------- src/Storages/Distributed/DirectoryMonitor.h | 8 +- .../DistributedAsyncInsertBatch.cpp | 277 ++++++++ .../Distributed/DistributedAsyncInsertBatch.h | 44 ++ .../DistributedAsyncInsertHelpers.cpp | 124 ++++ .../DistributedAsyncInsertHelpers.h | 35 + src/Storages/StorageDistributed.h | 1 + 7 files changed, 559 insertions(+), 528 deletions(-) create mode 100644 src/Storages/Distributed/DistributedAsyncInsertBatch.cpp create mode 100644 src/Storages/Distributed/DistributedAsyncInsertBatch.h create mode 100644 src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp create mode 100644 src/Storages/Distributed/DistributedAsyncInsertHelpers.h diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index c51e9c6e061..d5091003cb3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1,6 +1,19 @@ +#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -21,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -41,13 +55,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_READ_ALL_DATA; - extern const int UNKNOWN_CODEC; - extern const int CANNOT_DECOMPRESS; - extern const int CHECKSUM_DOESNT_MATCH; - extern const int TOO_LARGE_SIZE_COMPRESSED; - extern const int ATTEMPT_TO_READ_AFTER_EOF; - extern const int EMPTY_DATA_PASSED; extern const int INCORRECT_FILE_NAME; extern const int MEMORY_LIMIT_EXCEEDED; extern const int DISTRIBUTED_BROKEN_BATCH_INFO; @@ -64,296 +71,64 @@ namespace ErrorCodes namespace { - constexpr const std::chrono::minutes decrease_error_count_period{5}; - template - ConnectionPoolPtrs createPoolsForAddresses(const std::string & name, PoolFactory && factory, const Cluster::ShardsInfo & shards_info, Poco::Logger * log) +template +ConnectionPoolPtrs createPoolsForAddresses(const std::string & name, PoolFactory && factory, const Cluster::ShardsInfo & shards_info, Poco::Logger * log) +{ + ConnectionPoolPtrs pools; + + auto make_connection = [&](const Cluster::Address & address) { - ConnectionPoolPtrs pools; - - auto make_connection = [&](const Cluster::Address & address) + try { - try - { - pools.emplace_back(factory(address)); - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::INCORRECT_FILE_NAME) - { - tryLogCurrentException(log); - return; - } - throw; - } - }; - - for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it) - { - const std::string & dirname = boost::copy_range(*it); - Cluster::Address address = Cluster::Address::fromFullString(dirname); - if (address.shard_index && dirname.ends_with("_all_replicas")) - { - if (address.shard_index > shards_info.size()) - { - LOG_ERROR(log, "No shard with shard_index={} ({})", address.shard_index, name); - continue; - } - - const auto & shard_info = shards_info[address.shard_index - 1]; - size_t replicas = shard_info.per_replica_pools.size(); - - for (size_t replica_index = 1; replica_index <= replicas; ++replica_index) - { - address.replica_index = static_cast(replica_index); - make_connection(address); - } - } - else - make_connection(address); + pools.emplace_back(factory(address)); } - - return pools; - } - - void assertChecksum(CityHash_v1_0_2::uint128 expected, CityHash_v1_0_2::uint128 calculated) - { - if (expected != calculated) + catch (const Exception & e) { - throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, - "Checksum of extra info doesn't match: corrupted data. Reference: {}{}. Actual: {}{}.", - getHexUIntLowercase(expected.first), getHexUIntLowercase(expected.second), - getHexUIntLowercase(calculated.first), getHexUIntLowercase(calculated.second)); + if (e.code() == ErrorCodes::INCORRECT_FILE_NAME) + { + tryLogCurrentException(log); + return; + } + throw; } - } - - struct DistributedHeader - { - UInt64 revision = 0; - Settings insert_settings; - std::string insert_query; - ClientInfo client_info; - - /// .bin file cannot have zero rows/bytes. - size_t rows = 0; - size_t bytes = 0; - - UInt32 shard_num = 0; - std::string cluster; - std::string distributed_table; - std::string remote_table; - - /// dumpStructure() of the header -- obsolete - std::string block_header_string; - Block block_header; }; - DistributedHeader readDistributedHeader(ReadBufferFromFile & in, Poco::Logger * log) + for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it) { - DistributedHeader distributed_header; - - UInt64 query_size; - readVarUInt(query_size, in); - - if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER) + const std::string & dirname = boost::copy_range(*it); + Cluster::Address address = Cluster::Address::fromFullString(dirname); + if (address.shard_index && dirname.ends_with("_all_replicas")) { - /// Read the header as a string. - String header_data; - readStringBinary(header_data, in); - - /// Check the checksum of the header. - CityHash_v1_0_2::uint128 checksum; - readPODBinary(checksum, in); - assertChecksum(checksum, CityHash_v1_0_2::CityHash128(header_data.data(), header_data.size())); - - /// Read the parts of the header. - ReadBufferFromString header_buf(header_data); - - readVarUInt(distributed_header.revision, header_buf); - if (DBMS_TCP_PROTOCOL_VERSION < distributed_header.revision) + if (address.shard_index > shards_info.size()) { - LOG_WARNING(log, "ClickHouse shard version is older than ClickHouse initiator version. It may lack support for new features."); + LOG_ERROR(log, "No shard with shard_index={} ({})", address.shard_index, name); + continue; } - readStringBinary(distributed_header.insert_query, header_buf); - distributed_header.insert_settings.read(header_buf); + const auto & shard_info = shards_info[address.shard_index - 1]; + size_t replicas = shard_info.per_replica_pools.size(); - if (header_buf.hasPendingData()) - distributed_header.client_info.read(header_buf, distributed_header.revision); - - if (header_buf.hasPendingData()) + for (size_t replica_index = 1; replica_index <= replicas; ++replica_index) { - readVarUInt(distributed_header.rows, header_buf); - readVarUInt(distributed_header.bytes, header_buf); - readStringBinary(distributed_header.block_header_string, header_buf); + address.replica_index = static_cast(replica_index); + make_connection(address); } - - if (header_buf.hasPendingData()) - { - NativeReader header_block_in(header_buf, distributed_header.revision); - distributed_header.block_header = header_block_in.read(); - if (!distributed_header.block_header) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, - "Cannot read header from the {} batch. Data was written with protocol version {}, current version: {}", - in.getFileName(), distributed_header.revision, DBMS_TCP_PROTOCOL_VERSION); - } - - if (header_buf.hasPendingData()) - { - readVarUInt(distributed_header.shard_num, header_buf); - readStringBinary(distributed_header.cluster, header_buf); - readStringBinary(distributed_header.distributed_table, header_buf); - readStringBinary(distributed_header.remote_table, header_buf); - } - - /// Add handling new data here, for example: - /// - /// if (header_buf.hasPendingData()) - /// readVarUInt(my_new_data, header_buf); - /// - /// And note that it is safe, because we have checksum and size for header. - - return distributed_header; } - - if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT) - { - distributed_header.insert_settings.read(in, SettingsWriteFormat::BINARY); - readStringBinary(distributed_header.insert_query, in); - return distributed_header; - } - - distributed_header.insert_query.resize(query_size); - in.readStrict(distributed_header.insert_query.data(), query_size); - - return distributed_header; + else + make_connection(address); } - /// 'remote_error' argument is used to decide whether some errors should be - /// ignored or not, in particular: - /// - /// - ATTEMPT_TO_READ_AFTER_EOF should not be ignored - /// if we receive it from remote (receiver), since: - /// - the sender will got ATTEMPT_TO_READ_AFTER_EOF when the client just go away, - /// i.e. server had been restarted - /// - since #18853 the file will be checked on the sender locally, and - /// if there is something wrong with the file itself, we will receive - /// ATTEMPT_TO_READ_AFTER_EOF not from the remote at first - /// and mark batch as broken. - bool isFileBrokenErrorCode(int code, bool remote_error) - { - return code == ErrorCodes::CHECKSUM_DOESNT_MATCH - || code == ErrorCodes::EMPTY_DATA_PASSED - || code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED - || code == ErrorCodes::CANNOT_READ_ALL_DATA - || code == ErrorCodes::UNKNOWN_CODEC - || code == ErrorCodes::CANNOT_DECOMPRESS - || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO - || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES - || (!remote_error && code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); - } + return pools; +} - /// Can the batch be split and send files from batch one-by-one instead? - bool isSplittableErrorCode(int code, bool remote) - { - return code == ErrorCodes::MEMORY_LIMIT_EXCEEDED - /// FunctionRange::max_elements and similar - || code == ErrorCodes::ARGUMENT_OUT_OF_BOUND - || code == ErrorCodes::TOO_MANY_PARTS - || code == ErrorCodes::TOO_MANY_BYTES - || code == ErrorCodes::TOO_MANY_ROWS_OR_BYTES - || code == ErrorCodes::TOO_MANY_PARTITIONS - || code == ErrorCodes::DISTRIBUTED_TOO_MANY_PENDING_BYTES - || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO - || isFileBrokenErrorCode(code, remote) - ; - } +uint64_t doubleToUInt64(double d) +{ + if (d >= static_cast(std::numeric_limits::max())) + return std::numeric_limits::max(); + return static_cast(d); +} - SyncGuardPtr getDirectorySyncGuard(bool dir_fsync, const DiskPtr & disk, const String & path) - { - if (dir_fsync) - return disk->getDirectorySyncGuard(path); - return nullptr; - } - - void writeAndConvert(RemoteInserter & remote, const DistributedAsyncInsertHeader & distributed_header, ReadBufferFromFile & in) - { - CompressedReadBuffer decompressing_in(in); - NativeReader block_in(decompressing_in, distributed_header.revision); - - while (Block block = block_in.read()) - { - auto converting_dag = ActionsDAG::makeConvertingActions( - block.cloneEmpty().getColumnsWithTypeAndName(), - remote.getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto converting_actions = std::make_shared(std::move(converting_dag)); - converting_actions->execute(block); - remote.write(block); - } - } - - void writeRemoteConvert( - const DistributedAsyncInsertHeader & distributed_header, - RemoteInserter & remote, - bool compression_expected, - ReadBufferFromFile & in, - Poco::Logger * log) - { - if (!remote.getHeader()) - { - CheckingCompressedReadBuffer checking_in(in); - remote.writePrepared(checking_in); - return; - } - - /// This is old format, that does not have header for the block in the file header, - /// applying ConvertingTransform in this case is not a big overhead. - /// - /// Anyway we can get header only from the first block, which contain all rows anyway. - if (!distributed_header.block_header) - { - LOG_TRACE(log, "Processing batch {} with old format (no header)", in.getFileName()); - - writeAndConvert(remote, distributed_header, in); - return; - } - - if (!blocksHaveEqualStructure(distributed_header.block_header, remote.getHeader())) - { - LOG_WARNING(log, - "Structure does not match (remote: {}, local: {}), implicit conversion will be done", - remote.getHeader().dumpStructure(), distributed_header.block_header.dumpStructure()); - - writeAndConvert(remote, distributed_header, in); - return; - } - - /// If connection does not use compression, we have to uncompress the data. - if (!compression_expected) - { - writeAndConvert(remote, distributed_header, in); - return; - } - - if (distributed_header.revision != remote.getServerRevision()) - { - writeAndConvert(remote, distributed_header, in); - return; - } - - /// Otherwise write data as it was already prepared (more efficient path). - CheckingCompressedReadBuffer checking_in(in); - remote.writePrepared(checking_in); - } - - uint64_t doubleToUInt64(double d) - { - if (d >= static_cast(std::numeric_limits::max())) - return std::numeric_limits::max(); - return static_cast(d); - } } @@ -425,13 +200,15 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() task_handle->deactivate(); } - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); + auto dir_sync_guard = getDirectorySyncGuard(relative_path); fs::remove_all(path); } void StorageDistributedDirectoryMonitor::run() { + constexpr const std::chrono::minutes decrease_error_count_period{5}; + std::lock_guard lock{mutex}; bool do_sleep = false; @@ -695,7 +472,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa thread_trace_context->root_span.addAttribute(std::current_exception()); e.addMessage(fmt::format("While sending {}", file_path)); - if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) + if (isDistributedSendBroken(e.code(), e.isRemoteException())) { markAsBroken(file_path); current_file.clear(); @@ -710,7 +487,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa throw; } - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); + auto dir_sync_guard = getDirectorySyncGuard(relative_path); markAsSend(file_path); current_file.clear(); LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds()); @@ -750,244 +527,6 @@ struct StorageDistributedDirectoryMonitor::BatchHeader }; }; -struct StorageDistributedDirectoryMonitor::Batch -{ - size_t total_rows = 0; - size_t total_bytes = 0; - /// Does the batch had been created from the files in current_batch.txt? - bool recovered = false; - - StorageDistributedDirectoryMonitor & parent; - std::vector files; - - bool split_batch_on_failure = true; - bool fsync = false; - bool dir_fsync = false; - - explicit Batch(StorageDistributedDirectoryMonitor & parent_) - : parent(parent_) - , split_batch_on_failure(parent.split_batch_on_failure) - , fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert) - , dir_fsync(parent.dir_fsync) - {} - - bool isEnoughSize() const - { - return (!parent.min_batched_block_size_rows && !parent.min_batched_block_size_bytes) - || (parent.min_batched_block_size_rows && total_rows >= parent.min_batched_block_size_rows) - || (parent.min_batched_block_size_bytes && total_bytes >= parent.min_batched_block_size_bytes); - } - - void send() - { - if (files.empty()) - return; - - CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; - - Stopwatch watch; - - if (!recovered) - { - /// For deduplication in Replicated tables to work, in case of error - /// we must try to re-send exactly the same batches. - /// So we save contents of the current batch into the current_batch_file_path file - /// and truncate it afterwards if all went well. - serialize(); - } - - bool batch_broken = false; - bool batch_marked_as_broken = false; - try - { - try - { - sendBatch(); - } - catch (const Exception & e) - { - if (split_batch_on_failure && files.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException())) - { - tryLogCurrentException(parent.log, "Trying to split batch due to"); - sendSeparateFiles(); - } - else - throw; - } - } - catch (Exception & e) - { - if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) - { - tryLogCurrentException(parent.log, "Failed to send batch due to"); - batch_broken = true; - if (!e.isRemoteException() && e.code() == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES) - batch_marked_as_broken = true; - } - else - { - e.addMessage(fmt::format("While sending a batch of {} files, files: {}", files.size(), fmt::join(files, "\n"))); - throw; - } - } - - if (!batch_broken) - { - LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", files.size(), watch.elapsedMilliseconds()); - - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - for (const auto & file : files) - parent.markAsSend(file); - } - else if (!batch_marked_as_broken) - { - LOG_ERROR(parent.log, "Marking a batch of {} files as broken, files: {}", files.size(), fmt::join(files, "\n")); - - for (const auto & file : files) - parent.markAsBroken(file); - } - - files.clear(); - total_rows = 0; - total_bytes = 0; - recovered = false; - - fs::resize_file(parent.current_batch_file_path, 0); - } - - void serialize() - { - /// Temporary file is required for atomicity. - String tmp_file{parent.current_batch_file_path + ".tmp"}; - - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, parent.disk, parent.relative_path); - if (fs::exists(tmp_file)) - LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); - - { - WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT}; - writeText(out); - - out.finalize(); - if (fsync) - out.sync(); - } - - fs::rename(tmp_file, parent.current_batch_file_path); - } - - void deserialize() - { - ReadBufferFromFile in{parent.current_batch_file_path}; - readText(in); - } - -private: - void writeText(WriteBuffer & out) - { - for (const auto & file : files) - { - UInt64 file_index = parse(fs::path(file).stem()); - out << file_index << '\n'; - } - } - - void readText(ReadBuffer & in) - { - while (!in.eof()) - { - UInt64 idx; - in >> idx >> "\n"; - files.push_back(fs::absolute(fmt::format("{}/{}.bin", parent.path, idx)).string()); - } - - recovered = true; - } - - void sendBatch() - { - std::unique_ptr remote; - bool compression_expected = false; - - IConnectionPool::Entry connection; - - for (const auto & file : files) - { - ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); - - OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - parent.storage.getContext()->getOpenTelemetrySpanLog()); - - if (!remote) - { - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); - connection = parent.pool->get(timeouts); - compression_expected = connection->getCompression() == Protocol::Compression::Enable; - - LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", - files.size(), - connection->getDescription(), - formatReadableQuantity(total_rows), - formatReadableSizeWithBinarySuffix(total_bytes)); - - remote = std::make_unique(*connection, timeouts, - distributed_header.insert_query, - distributed_header.insert_settings, - distributed_header.client_info); - } - writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log); - } - - if (remote) - remote->onFinish(); - } - - void sendSeparateFiles() - { - size_t broken_files = 0; - - for (const auto & file : files) - { - try - { - ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); - - // this function is called in a separated thread, so we set up the trace context from the file - OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - parent.storage.getContext()->getOpenTelemetrySpanLog()); - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); - auto connection = parent.pool->get(timeouts); - bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; - - RemoteInserter remote(*connection, timeouts, - distributed_header.insert_query, - distributed_header.insert_settings, - distributed_header.client_info); - - writeRemoteConvert(distributed_header, remote, compression_expected, in, parent.log); - remote.onFinish(); - } - catch (Exception & e) - { - if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) - { - parent.markAsBroken(file); - ++broken_files; - } - } - } - - if (broken_files) - throw Exception(ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES, - "Failed to send {} files", broken_files); - } -}; - bool StorageDistributedDirectoryMonitor::addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms) { /// NOTE: It is better not to throw in this case, since the file is already @@ -1024,15 +563,15 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() { LOG_DEBUG(log, "Restoring the batch"); - Batch batch(*this); + DistributedAsyncInsertBatch batch(*this); batch.deserialize(); batch.send(); - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); + auto dir_sync_guard = getDirectorySyncGuard(relative_path); fs::remove(current_batch_file_path); } - std::unordered_map header_to_batch; + std::unordered_map header_to_batch; std::string file_path; while (pending_files.tryPop(file_path)) @@ -1075,7 +614,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() } catch (const Exception & e) { - if (isFileBrokenErrorCode(e.code(), e.isRemoteException())) + if (isDistributedSendBroken(e.code(), e.isRemoteException())) { markAsBroken(file_path); tryLogCurrentException(log, "File is marked broken due to"); @@ -1091,7 +630,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() std::move(distributed_header.client_info), std::move(header) ); - Batch & batch = header_to_batch.try_emplace(batch_header, *this).first->second; + DistributedAsyncInsertBatch & batch = header_to_batch.try_emplace(batch_header, *this).first->second; batch.files.push_back(file_path); batch.total_rows += total_rows; @@ -1105,12 +644,12 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() for (auto & kv : header_to_batch) { - Batch & batch = kv.second; + DistributedAsyncInsertBatch & batch = kv.second; batch.send(); } { - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); + auto dir_sync_guard = getDirectorySyncGuard(relative_path); /// current_batch.txt will not exist if there was no send /// (this is the case when all batches that was pending has been marked as pending) @@ -1123,8 +662,8 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p { const String & broken_file_path = fs::path(broken_path) / fs::path(file_path).filename(); - auto dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, relative_path); - auto broken_dir_sync_guard = getDirectorySyncGuard(dir_fsync, disk, broken_relative_path); + auto dir_sync_guard = getDirectorySyncGuard(relative_path); + auto broken_dir_sync_guard = getDirectorySyncGuard(broken_relative_path); { std::lock_guard status_lock(status_mutex); @@ -1158,6 +697,13 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat fs::remove(file_path); } +SyncGuardPtr StorageDistributedDirectoryMonitor::getDirectorySyncGuard(const std::string & dir_path) +{ + if (dir_fsync) + return disk->getDirectorySyncGuard(dir_path); + return nullptr; +} + std::string StorageDistributedDirectoryMonitor::getLoggerName() const { return storage.getStorageID().getFullTableName() + ".DirectoryMonitor." + disk->getName(); diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index 9b1596d45e3..8515f5a16a1 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -3,11 +3,11 @@ #include #include #include - +#include +#include #include #include #include -#include namespace CurrentMetrics { class Increment; } @@ -32,6 +32,8 @@ class ISource; */ class StorageDistributedDirectoryMonitor { + friend class DistributedAsyncInsertBatch; + public: StorageDistributedDirectoryMonitor( StorageDistributed & storage_, @@ -92,6 +94,8 @@ private: void markAsBroken(const std::string & file_path); void markAsSend(const std::string & file_path); + SyncGuardPtr getDirectorySyncGuard(const std::string & path); + std::string getLoggerName() const; StorageDistributed & storage; diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp new file mode 100644 index 00000000000..5f9e78a72e2 --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -0,0 +1,277 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace +{ + +namespace fs = std::filesystem; + +} + +namespace CurrentMetrics +{ + extern const Metric DistributedSend; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int MEMORY_LIMIT_EXCEEDED; + extern const int DISTRIBUTED_BROKEN_BATCH_INFO; + extern const int DISTRIBUTED_BROKEN_BATCH_FILES; + extern const int TOO_MANY_PARTS; + extern const int TOO_MANY_BYTES; + extern const int TOO_MANY_ROWS_OR_BYTES; + extern const int TOO_MANY_PARTITIONS; + extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +/// Can the batch be split and send files from batch one-by-one instead? +bool isSplittableErrorCode(int code, bool remote) +{ + return code == ErrorCodes::MEMORY_LIMIT_EXCEEDED + /// FunctionRange::max_elements and similar + || code == ErrorCodes::ARGUMENT_OUT_OF_BOUND + || code == ErrorCodes::TOO_MANY_PARTS + || code == ErrorCodes::TOO_MANY_BYTES + || code == ErrorCodes::TOO_MANY_ROWS_OR_BYTES + || code == ErrorCodes::TOO_MANY_PARTITIONS + || code == ErrorCodes::DISTRIBUTED_TOO_MANY_PENDING_BYTES + || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO + || isDistributedSendBroken(code, remote) + ; +} + +DistributedAsyncInsertBatch::DistributedAsyncInsertBatch(StorageDistributedDirectoryMonitor & parent_) + : parent(parent_) + , split_batch_on_failure(parent.split_batch_on_failure) + , fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert) + , dir_fsync(parent.dir_fsync) +{} + +bool DistributedAsyncInsertBatch::isEnoughSize() const +{ + return (!parent.min_batched_block_size_rows && !parent.min_batched_block_size_bytes) + || (parent.min_batched_block_size_rows && total_rows >= parent.min_batched_block_size_rows) + || (parent.min_batched_block_size_bytes && total_bytes >= parent.min_batched_block_size_bytes); +} + +void DistributedAsyncInsertBatch::send() +{ + if (files.empty()) + return; + + CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; + + Stopwatch watch; + + if (!recovered) + { + /// For deduplication in Replicated tables to work, in case of error + /// we must try to re-send exactly the same batches. + /// So we save contents of the current batch into the current_batch_file_path file + /// and truncate it afterwards if all went well. + serialize(); + } + + bool batch_broken = false; + bool batch_marked_as_broken = false; + try + { + try + { + sendBatch(); + } + catch (const Exception & e) + { + if (split_batch_on_failure && files.size() > 1 && isSplittableErrorCode(e.code(), e.isRemoteException())) + { + tryLogCurrentException(parent.log, "Trying to split batch due to"); + sendSeparateFiles(); + } + else + throw; + } + } + catch (Exception & e) + { + if (isDistributedSendBroken(e.code(), e.isRemoteException())) + { + tryLogCurrentException(parent.log, "Failed to send batch due to"); + batch_broken = true; + if (!e.isRemoteException() && e.code() == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES) + batch_marked_as_broken = true; + } + else + { + e.addMessage(fmt::format("While sending a batch of {} files, files: {}", files.size(), fmt::join(files, "\n"))); + throw; + } + } + + if (!batch_broken) + { + LOG_TRACE(parent.log, "Sent a batch of {} files (took {} ms).", files.size(), watch.elapsedMilliseconds()); + + auto dir_sync_guard = parent.getDirectorySyncGuard(parent.relative_path); + for (const auto & file : files) + parent.markAsSend(file); + } + else if (!batch_marked_as_broken) + { + LOG_ERROR(parent.log, "Marking a batch of {} files as broken, files: {}", files.size(), fmt::join(files, "\n")); + + for (const auto & file : files) + parent.markAsBroken(file); + } + + files.clear(); + total_rows = 0; + total_bytes = 0; + recovered = false; + + fs::resize_file(parent.current_batch_file_path, 0); +} + +void DistributedAsyncInsertBatch::serialize() +{ + /// Temporary file is required for atomicity. + String tmp_file{parent.current_batch_file_path + ".tmp"}; + + auto dir_sync_guard = parent.getDirectorySyncGuard(parent.relative_path); + if (fs::exists(tmp_file)) + LOG_ERROR(parent.log, "Temporary file {} exists. Unclean shutdown?", backQuote(tmp_file)); + + { + WriteBufferFromFile out{tmp_file, O_WRONLY | O_TRUNC | O_CREAT}; + writeText(out); + + out.finalize(); + if (fsync) + out.sync(); + } + + fs::rename(tmp_file, parent.current_batch_file_path); +} + +void DistributedAsyncInsertBatch::deserialize() +{ + ReadBufferFromFile in{parent.current_batch_file_path}; + readText(in); +} + +void DistributedAsyncInsertBatch::writeText(WriteBuffer & out) +{ + for (const auto & file : files) + { + UInt64 file_index = parse(fs::path(file).stem()); + out << file_index << '\n'; + } +} + +void DistributedAsyncInsertBatch::readText(ReadBuffer & in) +{ + while (!in.eof()) + { + UInt64 idx; + in >> idx >> "\n"; + files.push_back(fs::absolute(fmt::format("{}/{}.bin", parent.path, idx)).string()); + } + + recovered = true; +} + +void DistributedAsyncInsertBatch::sendBatch() +{ + std::unique_ptr remote; + bool compression_expected = false; + + IConnectionPool::Entry connection; + + for (const auto & file : files) + { + ReadBufferFromFile in(file); + const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); + + OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); + + if (!remote) + { + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); + connection = parent.pool->get(timeouts); + compression_expected = connection->getCompression() == Protocol::Compression::Enable; + + LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", + files.size(), + connection->getDescription(), + formatReadableQuantity(total_rows), + formatReadableSizeWithBinarySuffix(total_bytes)); + + remote = std::make_unique(*connection, timeouts, + distributed_header.insert_query, + distributed_header.insert_settings, + distributed_header.client_info); + } + writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log); + } + + if (remote) + remote->onFinish(); +} + +void DistributedAsyncInsertBatch::sendSeparateFiles() +{ + size_t broken_files = 0; + + for (const auto & file : files) + { + try + { + ReadBufferFromFile in(file); + const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); + + // This function is called in a separated thread, so we set up the trace context from the file + OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, + distributed_header.client_info.client_trace_context, + parent.storage.getContext()->getOpenTelemetrySpanLog()); + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); + auto connection = parent.pool->get(timeouts); + bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; + + RemoteInserter remote(*connection, timeouts, + distributed_header.insert_query, + distributed_header.insert_settings, + distributed_header.client_info); + + writeRemoteConvert(distributed_header, remote, compression_expected, in, parent.log); + remote.onFinish(); + } + catch (Exception & e) + { + if (isDistributedSendBroken(e.code(), e.isRemoteException())) + { + parent.markAsBroken(file); + ++broken_files; + } + } + } + + if (broken_files) + throw Exception(ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES, + "Failed to send {} files", broken_files); +} + +} diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.h b/src/Storages/Distributed/DistributedAsyncInsertBatch.h new file mode 100644 index 00000000000..8b3320155fa --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageDistributedDirectoryMonitor; +class WriteBuffer; +class ReadBuffer; + +class DistributedAsyncInsertBatch +{ +public: + explicit DistributedAsyncInsertBatch(StorageDistributedDirectoryMonitor & parent_); + + bool isEnoughSize() const; + void send(); + + void serialize(); + void deserialize(); + + size_t total_rows = 0; + size_t total_bytes = 0; + std::vector files; + +private: + void writeText(WriteBuffer & out); + void readText(ReadBuffer & in); + void sendBatch(); + void sendSeparateFiles(); + + StorageDistributedDirectoryMonitor & parent; + + /// Does the batch had been created from the files in current_batch.txt? + bool recovered = false; + + bool split_batch_on_failure = true; + bool fsync = false; + bool dir_fsync = false; +}; + +} diff --git a/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp b/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp new file mode 100644 index 00000000000..98073ba1e08 --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertHelpers.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; + extern const int UNKNOWN_CODEC; + extern const int CANNOT_DECOMPRESS; + extern const int CHECKSUM_DOESNT_MATCH; + extern const int TOO_LARGE_SIZE_COMPRESSED; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int EMPTY_DATA_PASSED; + extern const int DISTRIBUTED_BROKEN_BATCH_INFO; + extern const int DISTRIBUTED_BROKEN_BATCH_FILES; +} + +/// 'remote_error' argument is used to decide whether some errors should be +/// ignored or not, in particular: +/// +/// - ATTEMPT_TO_READ_AFTER_EOF should not be ignored +/// if we receive it from remote (receiver), since: +/// - the sender will got ATTEMPT_TO_READ_AFTER_EOF when the client just go away, +/// i.e. server had been restarted +/// - since #18853 the file will be checked on the sender locally, and +/// if there is something wrong with the file itself, we will receive +/// ATTEMPT_TO_READ_AFTER_EOF not from the remote at first +/// and mark batch as broken. +bool isDistributedSendBroken(int code, bool remote_error) +{ + return code == ErrorCodes::CHECKSUM_DOESNT_MATCH + || code == ErrorCodes::EMPTY_DATA_PASSED + || code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED + || code == ErrorCodes::CANNOT_READ_ALL_DATA + || code == ErrorCodes::UNKNOWN_CODEC + || code == ErrorCodes::CANNOT_DECOMPRESS + || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_INFO + || code == ErrorCodes::DISTRIBUTED_BROKEN_BATCH_FILES + || (!remote_error && code == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF); +} + +void writeAndConvert(RemoteInserter & remote, const DistributedAsyncInsertHeader & distributed_header, ReadBufferFromFile & in) +{ + CompressedReadBuffer decompressing_in(in); + NativeReader block_in(decompressing_in, distributed_header.revision); + + while (Block block = block_in.read()) + { + auto converting_dag = ActionsDAG::makeConvertingActions( + block.cloneEmpty().getColumnsWithTypeAndName(), + remote.getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(block); + remote.write(block); + } +} + +void writeRemoteConvert( + const DistributedAsyncInsertHeader & distributed_header, + RemoteInserter & remote, + bool compression_expected, + ReadBufferFromFile & in, + Poco::Logger * log) +{ + if (!remote.getHeader()) + { + CheckingCompressedReadBuffer checking_in(in); + remote.writePrepared(checking_in); + return; + } + + /// This is old format, that does not have header for the block in the file header, + /// applying ConvertingTransform in this case is not a big overhead. + /// + /// Anyway we can get header only from the first block, which contain all rows anyway. + if (!distributed_header.block_header) + { + LOG_TRACE(log, "Processing batch {} with old format (no header)", in.getFileName()); + + writeAndConvert(remote, distributed_header, in); + return; + } + + if (!blocksHaveEqualStructure(distributed_header.block_header, remote.getHeader())) + { + LOG_WARNING(log, + "Structure does not match (remote: {}, local: {}), implicit conversion will be done", + remote.getHeader().dumpStructure(), distributed_header.block_header.dumpStructure()); + + writeAndConvert(remote, distributed_header, in); + return; + } + + /// If connection does not use compression, we have to uncompress the data. + if (!compression_expected) + { + writeAndConvert(remote, distributed_header, in); + return; + } + + if (distributed_header.revision != remote.getServerRevision()) + { + writeAndConvert(remote, distributed_header, in); + return; + } + + /// Otherwise write data as it was already prepared (more efficient path). + CheckingCompressedReadBuffer checking_in(in); + remote.writePrepared(checking_in); +} + +} diff --git a/src/Storages/Distributed/DistributedAsyncInsertHelpers.h b/src/Storages/Distributed/DistributedAsyncInsertHelpers.h new file mode 100644 index 00000000000..9543450418c --- /dev/null +++ b/src/Storages/Distributed/DistributedAsyncInsertHelpers.h @@ -0,0 +1,35 @@ +#pragma once + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +struct DistributedAsyncInsertHeader; +class ReadBufferFromFile; +class RemoteInserter; + +/// 'remote_error' argument is used to decide whether some errors should be +/// ignored or not, in particular: +/// +/// - ATTEMPT_TO_READ_AFTER_EOF should not be ignored +/// if we receive it from remote (receiver), since: +/// - the sender will got ATTEMPT_TO_READ_AFTER_EOF when the client just go away, +/// i.e. server had been restarted +/// - since #18853 the file will be checked on the sender locally, and +/// if there is something wrong with the file itself, we will receive +/// ATTEMPT_TO_READ_AFTER_EOF not from the remote at first +/// and mark batch as broken. +bool isDistributedSendBroken(int code, bool remote_error); + +void writeRemoteConvert( + const DistributedAsyncInsertHeader & distributed_header, + RemoteInserter & remote, + bool compression_expected, + ReadBufferFromFile & in, + Poco::Logger * log); + +} diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 25a752fe795..432aee047e8 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -38,6 +38,7 @@ using ExpressionActionsPtr = std::shared_ptr; class StorageDistributed final : public IStorage, WithContext { friend class DistributedSink; + friend class DistributedAsyncInsertBatch; friend class StorageDistributedDirectoryMonitor; friend class StorageSystemDistributionQueue; From b5434eac3b189a6c588dbc17fba43e180809ed07 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 18:55:14 +0100 Subject: [PATCH 124/470] Rename StorageDistributedDirectoryMonitor to DistributedAsyncInsertDirectoryQueue Since #44922 it is not a directory monitor anymore. v2: Remove unused error codes v3: Contains some header fixes due to conflicts with master Signed-off-by: Azat Khuzhin --- src/Loggers/OwnSplitChannel.cpp | 2 +- .../DistributedAsyncInsertBatch.cpp | 4 +- .../Distributed/DistributedAsyncInsertBatch.h | 6 +- ... DistributedAsyncInsertDirectoryQueue.cpp} | 66 +++++++------------ ...=> DistributedAsyncInsertDirectoryQueue.h} | 25 +++++-- src/Storages/Distributed/DistributedSink.cpp | 4 +- src/Storages/StorageDistributed.cpp | 22 +++---- src/Storages/StorageDistributed.h | 14 ++-- .../System/StorageSystemDistributionQueue.cpp | 3 +- src/TableFunctions/ITableFunctionFileLike.cpp | 1 - ...etry_insert_on_distributed_table.reference | 4 +- 11 files changed, 71 insertions(+), 80 deletions(-) rename src/Storages/Distributed/{DirectoryMonitor.cpp => DistributedAsyncInsertDirectoryQueue.cpp} (90%) rename src/Storages/Distributed/{DirectoryMonitor.h => DistributedAsyncInsertDirectoryQueue.h} (81%) diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 7974d5212e1..7a7456823ed 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -56,7 +56,7 @@ void OwnSplitChannel::tryLogSplit(const Poco::Message & msg) /// breaking some functionality because of unexpected "File not /// found" (or similar) error. /// - /// For example StorageDistributedDirectoryMonitor will mark batch + /// For example DistributedAsyncInsertDirectoryQueue will mark batch /// as broken, some MergeTree code can also be affected. /// /// Also note, that we cannot log the exception here, since this diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 5f9e78a72e2..80350600caa 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -52,7 +52,7 @@ bool isSplittableErrorCode(int code, bool remote) ; } -DistributedAsyncInsertBatch::DistributedAsyncInsertBatch(StorageDistributedDirectoryMonitor & parent_) +DistributedAsyncInsertBatch::DistributedAsyncInsertBatch(DistributedAsyncInsertDirectoryQueue & parent_) : parent(parent_) , split_batch_on_failure(parent.split_batch_on_failure) , fsync(parent.storage.getDistributedSettingsRef().fsync_after_insert) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.h b/src/Storages/Distributed/DistributedAsyncInsertBatch.h index 8b3320155fa..867a0de89fa 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.h +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.h @@ -6,14 +6,14 @@ namespace DB { -class StorageDistributedDirectoryMonitor; +class DistributedAsyncInsertDirectoryQueue; class WriteBuffer; class ReadBuffer; class DistributedAsyncInsertBatch { public: - explicit DistributedAsyncInsertBatch(StorageDistributedDirectoryMonitor & parent_); + explicit DistributedAsyncInsertBatch(DistributedAsyncInsertDirectoryQueue & parent_); bool isEnoughSize() const; void send(); @@ -31,7 +31,7 @@ private: void sendBatch(); void sendSeparateFiles(); - StorageDistributedDirectoryMonitor & parent; + DistributedAsyncInsertDirectoryQueue & parent; /// Does the batch had been created from the files in current_batch.txt? bool recovered = false; diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp similarity index 90% rename from src/Storages/Distributed/DirectoryMonitor.cpp rename to src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d5091003cb3..2d25c1822d8 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -22,19 +21,9 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include #include #include #include @@ -56,15 +45,6 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_FILE_NAME; - extern const int MEMORY_LIMIT_EXCEEDED; - extern const int DISTRIBUTED_BROKEN_BATCH_INFO; - extern const int DISTRIBUTED_BROKEN_BATCH_FILES; - extern const int TOO_MANY_PARTS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int TOO_MANY_PARTITIONS; - extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; - extern const int ARGUMENT_OUT_OF_BOUND; extern const int LOGICAL_ERROR; } @@ -132,7 +112,7 @@ uint64_t doubleToUInt64(double d) } -StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( +DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( StorageDistributed & storage_, const DiskPtr & disk_, const std::string & relative_path_, @@ -172,7 +152,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( } -StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor() +DistributedAsyncInsertDirectoryQueue::~DistributedAsyncInsertDirectoryQueue() { if (!pending_files.isFinished()) { @@ -181,7 +161,7 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor() } } -void StorageDistributedDirectoryMonitor::flushAllData() +void DistributedAsyncInsertDirectoryQueue::flushAllData() { if (pending_files.isFinished()) return; @@ -192,7 +172,7 @@ void StorageDistributedDirectoryMonitor::flushAllData() processFiles(); } -void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() +void DistributedAsyncInsertDirectoryQueue::shutdownAndDropAllData() { if (!pending_files.isFinished()) { @@ -205,7 +185,7 @@ void StorageDistributedDirectoryMonitor::shutdownAndDropAllData() } -void StorageDistributedDirectoryMonitor::run() +void DistributedAsyncInsertDirectoryQueue::run() { constexpr const std::chrono::minutes decrease_error_count_period{5}; @@ -271,7 +251,7 @@ void StorageDistributedDirectoryMonitor::run() } -ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage) +ConnectionPoolPtr DistributedAsyncInsertDirectoryQueue::createPool(const std::string & name, const StorageDistributed & storage) { const auto pool_factory = [&storage, &name] (const Cluster::Address & address) -> ConnectionPoolPtr { @@ -344,18 +324,18 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri settings.distributed_replica_error_cap); } -bool StorageDistributedDirectoryMonitor::hasPendingFiles() const +bool DistributedAsyncInsertDirectoryQueue::hasPendingFiles() const { return fs::exists(current_batch_file_path) || !current_file.empty() || !pending_files.empty(); } -void StorageDistributedDirectoryMonitor::addFile(const std::string & file_path) +void DistributedAsyncInsertDirectoryQueue::addFile(const std::string & file_path) { if (!pending_files.push(fs::absolute(file_path).string())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot schedule a file '{}'", file_path); } -void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() +void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() { /// NOTE: This method does not requires to hold status_mutex, hence, no TSA /// annotations in the header file. @@ -413,7 +393,7 @@ void StorageDistributedDirectoryMonitor::initializeFilesFromDisk() status.broken_bytes_count = broken_bytes_count; } } -void StorageDistributedDirectoryMonitor::processFiles() +void DistributedAsyncInsertDirectoryQueue::processFiles() { if (should_batch_inserts) processFilesWithBatching(); @@ -428,7 +408,7 @@ void StorageDistributedDirectoryMonitor::processFiles() } } -void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path) +void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_path) { OpenTelemetry::TracingContextHolderPtr thread_trace_context; @@ -493,7 +473,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds()); } -struct StorageDistributedDirectoryMonitor::BatchHeader +struct DistributedAsyncInsertDirectoryQueue::BatchHeader { Settings settings; String query; @@ -527,7 +507,7 @@ struct StorageDistributedDirectoryMonitor::BatchHeader }; }; -bool StorageDistributedDirectoryMonitor::addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms) +bool DistributedAsyncInsertDirectoryQueue::addFileAndSchedule(const std::string & file_path, size_t file_size, size_t ms) { /// NOTE: It is better not to throw in this case, since the file is already /// on disk (see DistributedSink), and it will be processed next time. @@ -549,14 +529,14 @@ bool StorageDistributedDirectoryMonitor::addFileAndSchedule(const std::string & return task_handle->scheduleAfter(ms, false); } -StorageDistributedDirectoryMonitor::Status StorageDistributedDirectoryMonitor::getStatus() +DistributedAsyncInsertDirectoryQueue::Status DistributedAsyncInsertDirectoryQueue::getStatus() { std::lock_guard status_lock(status_mutex); Status current_status{status, path, monitor_blocker.isCancelled()}; return current_status; } -void StorageDistributedDirectoryMonitor::processFilesWithBatching() +void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() { /// Possibly, we failed to send a batch on the previous iteration. Try to send exactly the same batch. if (fs::exists(current_batch_file_path)) @@ -658,7 +638,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching() } } -void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_path) +void DistributedAsyncInsertDirectoryQueue::markAsBroken(const std::string & file_path) { const String & broken_file_path = fs::path(broken_path) / fs::path(file_path).filename(); @@ -683,7 +663,7 @@ void StorageDistributedDirectoryMonitor::markAsBroken(const std::string & file_p LOG_ERROR(log, "Renamed `{}` to `{}`", file_path, broken_file_path); } -void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_path) +void DistributedAsyncInsertDirectoryQueue::markAsSend(const std::string & file_path) { size_t file_size = fs::file_size(file_path); @@ -697,19 +677,19 @@ void StorageDistributedDirectoryMonitor::markAsSend(const std::string & file_pat fs::remove(file_path); } -SyncGuardPtr StorageDistributedDirectoryMonitor::getDirectorySyncGuard(const std::string & dir_path) +SyncGuardPtr DistributedAsyncInsertDirectoryQueue::getDirectorySyncGuard(const std::string & dir_path) { if (dir_fsync) return disk->getDirectorySyncGuard(dir_path); return nullptr; } -std::string StorageDistributedDirectoryMonitor::getLoggerName() const +std::string DistributedAsyncInsertDirectoryQueue::getLoggerName() const { return storage.getStorageID().getFullTableName() + ".DirectoryMonitor." + disk->getName(); } -void StorageDistributedDirectoryMonitor::updatePath(const std::string & new_relative_path) +void DistributedAsyncInsertDirectoryQueue::updatePath(const std::string & new_relative_path) { task_handle->deactivate(); std::lock_guard lock{mutex}; diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h similarity index 81% rename from src/Storages/Distributed/DirectoryMonitor.h rename to src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index 8515f5a16a1..90e3d563b4b 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -27,15 +27,28 @@ using ProcessorPtr = std::shared_ptr; class ISource; -/** Details of StorageDistributed. - * This type is not designed for standalone use. - */ -class StorageDistributedDirectoryMonitor +/** Queue for async INSERT Into Distributed engine (insert_distributed_sync=0). + * + * Files are added from two places: + * - from filesystem at startup (StorageDistributed::startup()) + * - on INSERT via DistributedSink + * + * Later, in background, those files will be send to the remote nodes. + * + * The behaviour of this queue can be configured via the following settings: + * - distributed_directory_monitor_batch_inserts + * - distributed_directory_monitor_split_batch_on_failure + * - distributed_directory_monitor_sleep_time_ms + * - distributed_directory_monitor_max_sleep_time_ms + * NOTE: It worth to rename the settings too + * ("directory_monitor" in settings looks too internal). + */ +class DistributedAsyncInsertDirectoryQueue { friend class DistributedAsyncInsertBatch; public: - StorageDistributedDirectoryMonitor( + DistributedAsyncInsertDirectoryQueue( StorageDistributed & storage_, const DiskPtr & disk_, const std::string & relative_path_, @@ -44,7 +57,7 @@ public: BackgroundSchedulePool & bg_pool, bool initialize_from_disk); - ~StorageDistributedDirectoryMonitor(); + ~DistributedAsyncInsertDirectoryQueue(); static ConnectionPoolPtr createPool(const std::string & name, const StorageDistributed & storage); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index c9c235596db..38aa26fbe0c 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -834,7 +834,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const const auto & dir_name = dir_names[i]; const auto & bin_file = bin_files[i]; - auto & directory_monitor = storage.requireDirectoryMonitor(disk, dir_name, /* startup= */ false); + auto & directory_monitor = storage.getDirectoryQueue(disk, dir_name, /* startup= */ false); directory_monitor.addFileAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds()); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a38beef983c..d95c703216e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1094,7 +1094,7 @@ void StorageDistributed::initializeFromDisk() { pool.scheduleOrThrowOnError([&]() { - createDirectoryMonitors(disk); + initializeDirectoryQueuesForDisk(disk); }); } pool.wait(); @@ -1133,7 +1133,7 @@ void StorageDistributed::shutdown() void StorageDistributed::drop() { // Some INSERT in-between shutdown() and drop() can call - // requireDirectoryMonitor() again, so call shutdown() to clear them, but + // getDirectoryQueue() again, so call shutdown() to clear them, but // when the drop() (this function) executed none of INSERT is allowed in // parallel. // @@ -1196,7 +1196,7 @@ StoragePolicyPtr StorageDistributed::getStoragePolicy() const return storage_policy; } -void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) +void StorageDistributed::initializeDirectoryQueuesForDisk(const DiskPtr & disk) { const std::string path(disk->getPath() + relative_data_path); fs::create_directories(path); @@ -1225,14 +1225,14 @@ void StorageDistributed::createDirectoryMonitors(const DiskPtr & disk) } else { - requireDirectoryMonitor(disk, dir_path.filename().string(), /* startup= */ true); + getDirectoryQueue(disk, dir_path.filename().string(), /* startup= */ true); } } } } -StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(const DiskPtr & disk, const std::string & name, bool startup) +DistributedAsyncInsertDirectoryQueue & StorageDistributed::getDirectoryQueue(const DiskPtr & disk, const std::string & name, bool startup) { const std::string & disk_path = disk->getPath(); const std::string key(disk_path + name); @@ -1241,8 +1241,8 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor( auto & node_data = cluster_nodes_data[key]; if (!node_data.directory_monitor) { - node_data.connection_pool = StorageDistributedDirectoryMonitor::createPool(name, *this); - node_data.directory_monitor = std::make_unique( + node_data.connection_pool = DistributedAsyncInsertDirectoryQueue::createPool(name, *this); + node_data.directory_monitor = std::make_unique( *this, disk, relative_data_path + name, node_data.connection_pool, monitors_blocker, @@ -1252,9 +1252,9 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor( return *node_data.directory_monitor; } -std::vector StorageDistributed::getDirectoryMonitorsStatuses() const +std::vector StorageDistributed::getDirectoryQueueStatuses() const { - std::vector statuses; + std::vector statuses; std::lock_guard lock(cluster_nodes_mutex); statuses.reserve(cluster_nodes_data.size()); for (const auto & node : cluster_nodes_data) @@ -1265,7 +1265,7 @@ std::vector StorageDistributed::getD std::optional StorageDistributed::totalBytes(const Settings &) const { UInt64 total_bytes = 0; - for (const auto & status : getDirectoryMonitorsStatuses()) + for (const auto & status : getDirectoryQueueStatuses()) total_bytes += status.bytes_count; return total_bytes; } @@ -1426,7 +1426,7 @@ void StorageDistributed::flushClusterNodesAllData(ContextPtr local_context) /// Sync SYSTEM FLUSH DISTRIBUTED with TRUNCATE auto table_lock = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); - std::vector> directory_monitors; + std::vector> directory_monitors; { std::lock_guard lock(cluster_nodes_mutex); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 432aee047e8..df4c16f0f67 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -39,7 +39,7 @@ class StorageDistributed final : public IStorage, WithContext { friend class DistributedSink; friend class DistributedAsyncInsertBatch; - friend class StorageDistributedDirectoryMonitor; + friend class DistributedAsyncInsertDirectoryQueue; friend class StorageSystemDistributionQueue; public: @@ -165,15 +165,15 @@ private: const String & getRelativeDataPath() const { return relative_data_path; } /// create directory monitors for each existing subdirectory - void createDirectoryMonitors(const DiskPtr & disk); - /// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name - StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const DiskPtr & disk, const std::string & name, bool startup); + void initializeDirectoryQueuesForDisk(const DiskPtr & disk); + /// ensure directory queue thread and connection pool created by disk and subdirectory name + DistributedAsyncInsertDirectoryQueue & getDirectoryQueue(const DiskPtr & disk, const std::string & name, bool startup); /// Return list of metrics for all created monitors /// (note that monitors are created lazily, i.e. until at least one INSERT executed) /// /// Used by StorageSystemDistributionQueue - std::vector getDirectoryMonitorsStatuses() const; + std::vector getDirectoryQueueStatuses() const; static IColumn::Selector createSelector(ClusterPtr cluster, const ColumnWithTypeAndName & result); /// Apply the following settings: @@ -248,7 +248,7 @@ private: struct ClusterNodeData { - std::shared_ptr directory_monitor; + std::shared_ptr directory_monitor; ConnectionPoolPtr connection_pool; }; std::unordered_map cluster_nodes_data; diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 34cff7df65d..d57269f0638 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -174,7 +173,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, Cont auto & distributed_table = dynamic_cast(*tables[database][table]); - for (const auto & status : distributed_table.getDirectoryMonitorsStatuses()) + for (const auto & status : distributed_table.getDirectoryQueueStatuses()) { size_t col_num = 0; res_columns[col_num++]->insert(database); diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 8cbffc10e5a..bbaf2b68418 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference index dde07d4540d..81a1f62ca29 100644 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference @@ -1,7 +1,7 @@ {"operation_name":"void DB::DistributedSink::writeToLocal(const Cluster::ShardInfo &, const DB::Block &, size_t)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} {"operation_name":"void DB::DistributedSink::writeToLocal(const Cluster::ShardInfo &, const DB::Block &, size_t)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} -{"operation_name":"void DB::StorageDistributedDirectoryMonitor::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} -{"operation_name":"void DB::StorageDistributedDirectoryMonitor::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} +{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} +{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} {"operation_name":"auto DB::DistributedSink::runWritingJob(DB::DistributedSink::JobReplica &, const DB::Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} {"operation_name":"auto DB::DistributedSink::runWritingJob(DB::DistributedSink::JobReplica &, const DB::Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} {"operation_name":"auto DB::DistributedSink::runWritingJob(DB::DistributedSink::JobReplica &, const DB::Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} From e10fb142fd124455b12d0b359c9866f0349ef26c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 20:37:12 +0100 Subject: [PATCH 125/470] Fix race for distributed sends from disk Before it was initialized from disk only on startup, but if some INSERT can create the object before, then, it will lead to the situation when it will not be initialized. Signed-off-by: Azat Khuzhin --- .../DistributedAsyncInsertDirectoryQueue.cpp | 6 +-- .../DistributedAsyncInsertDirectoryQueue.h | 3 +- src/Storages/Distributed/DistributedSink.cpp | 38 +++++++++---------- src/Storages/StorageDistributed.cpp | 7 ++-- src/Storages/StorageDistributed.h | 8 +++- 5 files changed, 30 insertions(+), 32 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 2d25c1822d8..3993e9065d5 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -118,8 +118,7 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( const std::string & relative_path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, - BackgroundSchedulePool & bg_pool, - bool initialize_from_disk) + BackgroundSchedulePool & bg_pool) : storage(storage_) , pool(std::move(pool_)) , disk(disk_) @@ -144,8 +143,7 @@ DistributedAsyncInsertDirectoryQueue::DistributedAsyncInsertDirectoryQueue( { fs::create_directory(broken_path); - if (initialize_from_disk) - initializeFilesFromDisk(); + initializeFilesFromDisk(); task_handle = bg_pool.createTask(getLoggerName() + "/Bg", [this]{ run(); }); task_handle->activateAndSchedule(); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index 90e3d563b4b..de8bb813824 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -54,8 +54,7 @@ public: const std::string & relative_path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, - BackgroundSchedulePool & bg_pool, - bool initialize_from_disk); + BackgroundSchedulePool & bg_pool); ~DistributedAsyncInsertDirectoryQueue(); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 38aa26fbe0c..3c4e493c34a 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -724,8 +724,8 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const return guard; }; - std::vector bin_files; - bin_files.reserve(dir_names.size()); + auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds(); + size_t file_size; auto it = dir_names.begin(); /// on first iteration write block to a temporary directory for subsequent @@ -804,10 +804,16 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const out.sync(); } + file_size = fs::file_size(first_file_tmp_path); + // Create hardlink here to reuse increment number - bin_files.push_back(fs::path(path) / file_name); - createHardLink(first_file_tmp_path, bin_files.back()); - auto dir_sync_guard = make_directory_sync_guard(*it); + auto bin_file = (fs::path(path) / file_name).string(); + auto & directory_queue = storage.getDirectoryQueue(disk, *it); + { + createHardLink(first_file_tmp_path, bin_file); + auto dir_sync_guard = make_directory_sync_guard(*it); + } + directory_queue.addFileAndSchedule(bin_file, file_size, sleep_ms); } ++it; @@ -817,26 +823,18 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const const std::string path(fs::path(disk_path) / (data_path + *it)); fs::create_directory(path); - bin_files.push_back(fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")); - createHardLink(first_file_tmp_path, bin_files.back()); - auto dir_sync_guard = make_directory_sync_guard(*it); + auto bin_file = (fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")).string(); + auto & directory_monitor = storage.getDirectoryQueue(disk, *it); + { + createHardLink(first_file_tmp_path, bin_file); + auto dir_sync_guard = make_directory_sync_guard(*it); + } + directory_monitor.addFileAndSchedule(bin_file, file_size, sleep_ms); } - auto file_size = fs::file_size(first_file_tmp_path); /// remove the temporary file, enabling the OS to reclaim inode after all threads /// have removed their corresponding files fs::remove(first_file_tmp_path); - - /// Notify - auto sleep_ms = context->getSettingsRef().distributed_directory_monitor_sleep_time_ms; - for (size_t i = 0; i < dir_names.size(); ++i) - { - const auto & dir_name = dir_names[i]; - const auto & bin_file = bin_files[i]; - - auto & directory_monitor = storage.getDirectoryQueue(disk, dir_name, /* startup= */ false); - directory_monitor.addFileAndSchedule(bin_file, file_size, sleep_ms.totalMilliseconds()); - } } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index d95c703216e..9f674ce3bed 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1225,14 +1225,14 @@ void StorageDistributed::initializeDirectoryQueuesForDisk(const DiskPtr & disk) } else { - getDirectoryQueue(disk, dir_path.filename().string(), /* startup= */ true); + getDirectoryQueue(disk, dir_path.filename().string()); } } } } -DistributedAsyncInsertDirectoryQueue & StorageDistributed::getDirectoryQueue(const DiskPtr & disk, const std::string & name, bool startup) +DistributedAsyncInsertDirectoryQueue & StorageDistributed::getDirectoryQueue(const DiskPtr & disk, const std::string & name) { const std::string & disk_path = disk->getPath(); const std::string key(disk_path + name); @@ -1246,8 +1246,7 @@ DistributedAsyncInsertDirectoryQueue & StorageDistributed::getDirectoryQueue(con *this, disk, relative_data_path + name, node_data.connection_pool, monitors_blocker, - getContext()->getDistributedSchedulePool(), - /* initialize_from_disk= */ startup); + getContext()->getDistributedSchedulePool()); } return *node_data.directory_monitor; } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index df4c16f0f67..3a7fae44708 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -166,8 +166,12 @@ private: /// create directory monitors for each existing subdirectory void initializeDirectoryQueuesForDisk(const DiskPtr & disk); - /// ensure directory queue thread and connection pool created by disk and subdirectory name - DistributedAsyncInsertDirectoryQueue & getDirectoryQueue(const DiskPtr & disk, const std::string & name, bool startup); + + /// Get directory queue thread and connection pool created by disk and subdirectory name + /// + /// Used for the INSERT into Distributed in case of insert_distributed_sync==1, from DistributedSink. + DistributedAsyncInsertDirectoryQueue & getDirectoryQueue(const DiskPtr & disk, const std::string & name); + /// Return list of metrics for all created monitors /// (note that monitors are created lazily, i.e. until at least one INSERT executed) From e83699a8d3b72de7ab5ca3295dcabc59492a7fc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 20:51:04 +0100 Subject: [PATCH 126/470] Improve comment for DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 3993e9065d5..6530ebc578a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -335,8 +335,9 @@ void DistributedAsyncInsertDirectoryQueue::addFile(const std::string & file_path void DistributedAsyncInsertDirectoryQueue::initializeFilesFromDisk() { - /// NOTE: This method does not requires to hold status_mutex, hence, no TSA - /// annotations in the header file. + /// NOTE: This method does not requires to hold status_mutex (because this + /// object is not in the list that the caller may iterate over), hence, no + /// TSA annotations in the header file. fs::directory_iterator end; From a76d7b22c1ff11c775986cef17d91c500ca9e7ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 20:51:16 +0100 Subject: [PATCH 127/470] Use existing public methods of StorageDistributed in DistributedSink Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedSink.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 3c4e493c34a..14febb1a2b4 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -340,9 +340,9 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si size_t rows = shard_block.rows(); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); - span.addAttribute("clickhouse.cluster", this->storage.cluster_name); - span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); - span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); + span.addAttribute("clickhouse.cluster", storage.cluster_name); + span.addAttribute("clickhouse.distributed", storage.getStorageID().getFullNameNotQuoted()); + span.addAttribute("clickhouse.remote", [this]() { return storage.getRemoteDatabaseName() + "." + storage.getRemoteTableName(); }); span.addAttribute("clickhouse.rows", rows); span.addAttribute("clickhouse.bytes", [&shard_block]() { return toString(shard_block.bytes()); }); @@ -476,7 +476,7 @@ void DistributedSink::writeSync(const Block & block) span.addAttribute("clickhouse.start_shard", start); span.addAttribute("clickhouse.end_shard", end); - span.addAttribute("db.statement", this->query_string); + span.addAttribute("db.statement", query_string); if (num_shards > 1) { @@ -659,9 +659,9 @@ void DistributedSink::writeToLocal(const Cluster::ShardInfo & shard_info, const { OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); span.addAttribute("clickhouse.shard_num", shard_info.shard_num); - span.addAttribute("clickhouse.cluster", this->storage.cluster_name); - span.addAttribute("clickhouse.distributed", this->storage.getStorageID().getFullNameNotQuoted()); - span.addAttribute("clickhouse.remote", [this]() { return storage.remote_database + "." + storage.remote_table; }); + span.addAttribute("clickhouse.cluster", storage.cluster_name); + span.addAttribute("clickhouse.distributed", storage.getStorageID().getFullNameNotQuoted()); + span.addAttribute("clickhouse.remote", [this]() { return storage.getRemoteDatabaseName() + "." + storage.getRemoteTableName(); }); span.addAttribute("clickhouse.rows", [&block]() { return toString(block.rows()); }); span.addAttribute("clickhouse.bytes", [&block]() { return toString(block.bytes()); }); @@ -782,9 +782,9 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const } writeVarUInt(shard_info.shard_num, header_buf); - writeStringBinary(this->storage.cluster_name, header_buf); - writeStringBinary(this->storage.getStorageID().getFullNameNotQuoted(), header_buf); - writeStringBinary(this->storage.remote_database + "." + this->storage.remote_table, header_buf); + writeStringBinary(storage.cluster_name, header_buf); + writeStringBinary(storage.getStorageID().getFullNameNotQuoted(), header_buf); + writeStringBinary(storage.getRemoteDatabaseName() + "." + storage.getRemoteTableName(), header_buf); /// Add new fields here, for example: /// writeVarUInt(my_new_data, header_buf); From 00115c6615c311faffc40d24476c1ef51d40312c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 20:54:49 +0100 Subject: [PATCH 128/470] Rename readDistributedAsyncInsertHeader() Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 4 ++-- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 4 ++-- src/Storages/Distributed/DistributedAsyncInsertHeader.cpp | 2 +- src/Storages/Distributed/DistributedAsyncInsertHeader.h | 4 ++-- src/Storages/Distributed/DistributedAsyncInsertSource.cpp | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 80350600caa..948b4186ec6 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -201,7 +201,7 @@ void DistributedAsyncInsertBatch::sendBatch() for (const auto & file : files) { ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); + const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, @@ -240,7 +240,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles() try { ReadBufferFromFile in(file); - const auto & distributed_header = readDistributedAsyncInsertHeader(in, parent.log); + const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); // This function is called in a separated thread, so we set up the trace context from the file OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 6530ebc578a..b58096182ec 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -417,7 +417,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_ CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; ReadBufferFromFile in(file_path); - const auto & distributed_header = readDistributedAsyncInsertHeader(in, log); + const auto & distributed_header = DistributedAsyncInsertHeader::read(in, log); thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, distributed_header.client_info.client_trace_context, @@ -563,7 +563,7 @@ void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() { /// Determine metadata of the current file and check if it is not broken. ReadBufferFromFile in{file_path}; - distributed_header = readDistributedAsyncInsertHeader(in, log); + distributed_header = DistributedAsyncInsertHeader::read(in, log); if (distributed_header.rows) { diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index 19235c91cc6..9584d1eebdf 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; } -DistributedAsyncInsertHeader readDistributedAsyncInsertHeader(ReadBufferFromFile & in, Poco::Logger * log) +DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFile & in, Poco::Logger * log) { DistributedAsyncInsertHeader distributed_header; diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.h b/src/Storages/Distributed/DistributedAsyncInsertHeader.h index 2c7a6477b6c..dc048107a47 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.h +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.h @@ -31,8 +31,8 @@ struct DistributedAsyncInsertHeader /// dumpStructure() of the header -- obsolete std::string block_header_string; Block block_header; + + static DistributedAsyncInsertHeader read(ReadBufferFromFile & in, Poco::Logger * log); }; -DistributedAsyncInsertHeader readDistributedAsyncInsertHeader(ReadBufferFromFile & in, Poco::Logger * log); - } diff --git a/src/Storages/Distributed/DistributedAsyncInsertSource.cpp b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp index 782cbf9c026..7992636ac11 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertSource.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertSource.cpp @@ -22,7 +22,7 @@ struct DistributedAsyncInsertSource::Data : log(&Poco::Logger::get("DistributedAsyncInsertSource")) , in(file_name) , decompressing_in(in) - , block_in(decompressing_in, readDistributedAsyncInsertHeader(in, log).revision) + , block_in(decompressing_in, DistributedAsyncInsertHeader::read(in, log).revision) , first_block(block_in.read()) { } From 263c042c6aa6103e5be221fe36c83be593056ac6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 21:01:54 +0100 Subject: [PATCH 129/470] Fix opentelemetry for distributed batch sends Signed-off-by: Azat Khuzhin --- .../DistributedAsyncInsertBatch.cpp | 79 ++++++++++++------- .../DistributedAsyncInsertDirectoryQueue.cpp | 12 +-- .../DistributedAsyncInsertHeader.cpp | 16 ++++ .../DistributedAsyncInsertHeader.h | 7 ++ 4 files changed, 77 insertions(+), 37 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 948b4186ec6..10bf9b8d66e 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -198,37 +198,59 @@ void DistributedAsyncInsertBatch::sendBatch() IConnectionPool::Entry connection; - for (const auto & file : files) + /// Since the batch is sent as a whole (in case of failure, the whole batch + /// will be repeated), we need to mark the whole batch as failed in case of + /// error). + std::vector tracing_contexts; + UInt64 batch_start_time = clock_gettime_ns(); + + try { - ReadBufferFromFile in(file); - const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); - - OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - parent.storage.getContext()->getOpenTelemetrySpanLog()); - - if (!remote) + for (const auto & file : files) { - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); - connection = parent.pool->get(timeouts); - compression_expected = connection->getCompression() == Protocol::Compression::Enable; + ReadBufferFromFile in(file); + const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); - LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", - files.size(), - connection->getDescription(), - formatReadableQuantity(total_rows), - formatReadableSizeWithBinarySuffix(total_bytes)); + tracing_contexts.emplace_back(distributed_header.createTracingContextHolder( + parent.storage.getContext()->getOpenTelemetrySpanLog())); + tracing_contexts.back()->root_span.addAttribute("clickhouse.distributed_batch_start_time", batch_start_time); - remote = std::make_unique(*connection, timeouts, - distributed_header.insert_query, - distributed_header.insert_settings, - distributed_header.client_info); + if (!remote) + { + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); + connection = parent.pool->get(timeouts); + compression_expected = connection->getCompression() == Protocol::Compression::Enable; + + LOG_DEBUG(parent.log, "Sending a batch of {} files to {} ({} rows, {} bytes).", + files.size(), + connection->getDescription(), + formatReadableQuantity(total_rows), + formatReadableSizeWithBinarySuffix(total_bytes)); + + remote = std::make_unique(*connection, timeouts, + distributed_header.insert_query, + distributed_header.insert_settings, + distributed_header.client_info); + } + writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log); } - writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log); - } - if (remote) - remote->onFinish(); + if (remote) + remote->onFinish(); + } + catch (...) + { + try + { + for (auto & tracing_context : tracing_contexts) + tracing_context->root_span.addAttribute(std::current_exception()); + } + catch (...) + { + tryLogCurrentException(parent.log, "Cannot append exception to tracing context"); + } + throw; + } } void DistributedAsyncInsertBatch::sendSeparateFiles() @@ -237,14 +259,15 @@ void DistributedAsyncInsertBatch::sendSeparateFiles() for (const auto & file : files) { + OpenTelemetry::TracingContextHolderPtr trace_context; + try { ReadBufferFromFile in(file); const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); // This function is called in a separated thread, so we set up the trace context from the file - OpenTelemetry::TracingContextHolder thread_trace_context(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, + trace_context = distributed_header.createTracingContextHolder( parent.storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); @@ -261,6 +284,8 @@ void DistributedAsyncInsertBatch::sendSeparateFiles() } catch (Exception & e) { + trace_context->root_span.addAttribute(std::current_exception()); + if (isDistributedSendBroken(e.code(), e.isRemoteException())) { parent.markAsBroken(file); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index b58096182ec..31c621571ab 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -418,16 +418,8 @@ void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_ ReadBufferFromFile in(file_path); const auto & distributed_header = DistributedAsyncInsertHeader::read(in, log); - - thread_trace_context = std::make_unique(__PRETTY_FUNCTION__, - distributed_header.client_info.client_trace_context, - this->storage.getContext()->getOpenTelemetrySpanLog()); - thread_trace_context->root_span.addAttribute("clickhouse.shard_num", distributed_header.shard_num); - thread_trace_context->root_span.addAttribute("clickhouse.cluster", distributed_header.cluster); - thread_trace_context->root_span.addAttribute("clickhouse.distributed", distributed_header.distributed_table); - thread_trace_context->root_span.addAttribute("clickhouse.remote", distributed_header.remote_table); - thread_trace_context->root_span.addAttribute("clickhouse.rows", distributed_header.rows); - thread_trace_context->root_span.addAttribute("clickhouse.bytes", distributed_header.bytes); + thread_trace_context = + distributed_header.createTracingContextHolder(storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); auto connection = pool->get(timeouts, &distributed_header.insert_settings); diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index 9584d1eebdf..b5c16558eb2 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -106,4 +107,19 @@ DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFi return distributed_header; } +OpenTelemetry::TracingContextHolderPtr DistributedAsyncInsertHeader::createTracingContextHolder(std::shared_ptr open_telemetry_span_log) const +{ + OpenTelemetry::TracingContextHolderPtr trace_context = std::make_unique( + __PRETTY_FUNCTION__, + client_info.client_trace_context, + std::move(open_telemetry_span_log)); + trace_context->root_span.addAttribute("clickhouse.shard_num", shard_num); + trace_context->root_span.addAttribute("clickhouse.cluster", cluster); + trace_context->root_span.addAttribute("clickhouse.distributed", distributed_table); + trace_context->root_span.addAttribute("clickhouse.remote", remote_table); + trace_context->root_span.addAttribute("clickhouse.rows", rows); + trace_context->root_span.addAttribute("clickhouse.bytes", bytes); + return trace_context; +} + } diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.h b/src/Storages/Distributed/DistributedAsyncInsertHeader.h index dc048107a47..dfc059d7083 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.h +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.h @@ -11,6 +11,12 @@ namespace DB class ReadBufferFromFile; +namespace OpenTelemetry +{ +struct TracingContextHolder; +using TracingContextHolderPtr = std::unique_ptr; +} + /// Header for the binary files that are stored on disk for async INSERT into Distributed. struct DistributedAsyncInsertHeader { @@ -33,6 +39,7 @@ struct DistributedAsyncInsertHeader Block block_header; static DistributedAsyncInsertHeader read(ReadBufferFromFile & in, Poco::Logger * log); + OpenTelemetry::TracingContextHolderPtr createTracingContextHolder(std::shared_ptr open_telemetry_span_log) const; }; } From 572dd76eae2b9e3090e07c388c6baa142a095a57 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 21:53:01 +0100 Subject: [PATCH 130/470] Extend ConcurrentBoundedQueue to deque Signed-off-by: Azat Khuzhin --- src/Common/ConcurrentBoundedQueue.h | 59 +++++++++++++++++++---------- 1 file changed, 39 insertions(+), 20 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index fd4a2d5790b..665be69a28f 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -18,7 +18,8 @@ template class ConcurrentBoundedQueue { private: - std::queue queue; + using Container = std::deque; + Container queue; mutable std::mutex queue_mutex; std::condition_variable push_condition; @@ -28,7 +29,7 @@ private: size_t max_fill = 0; - template + template bool emplaceImpl(std::optional timeout_milliseconds, Args &&...args) { { @@ -51,13 +52,17 @@ private: if (is_finished) return false; - queue.emplace(std::forward(args)...); + if constexpr (back) + queue.emplace_back(std::forward(args)...); + else + queue.emplace_front(std::forward(args)...); } pop_condition.notify_one(); return true; } + template bool popImpl(T & x, std::optional timeout_milliseconds) { { @@ -80,8 +85,16 @@ private: if (is_finished && queue.empty()) return false; - detail::moveOrCopyIfThrow(std::move(queue.front()), x); - queue.pop(); + if constexpr (front) + { + detail::moveOrCopyIfThrow(std::move(queue.front()), x); + queue.pop_front(); + } + else + { + detail::moveOrCopyIfThrow(std::move(queue.back()), x); + queue.pop_back(); + } } push_condition.notify_one(); @@ -94,6 +107,12 @@ public: : max_fill(max_fill_) {} + /// Returns false if queue is finished + [[nodiscard]] bool pushFront(const T & x) + { + return emplaceImpl(/* timeout_milliseconds= */ std::nullopt , x); + } + /// Returns false if queue is finished [[nodiscard]] bool push(const T & x) { @@ -109,37 +128,37 @@ public: template [[nodiscard]] bool emplace(Args &&... args) { - return emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); - } - - /// Returns false if queue is finished and empty - [[nodiscard]] bool pop(T & x) - { - return popImpl(x, std::nullopt /*timeout in milliseconds*/); + return emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); } /// Returns false if queue is finished or object was not pushed during timeout [[nodiscard]] bool tryPush(const T & x, UInt64 milliseconds = 0) { - return emplaceImpl(milliseconds, x); + return emplaceImpl(milliseconds, x); } [[nodiscard]] bool tryPush(T && x, UInt64 milliseconds = 0) { - return emplaceImpl(milliseconds, std::move(x)); + return emplaceImpl(milliseconds, std::move(x)); } /// Returns false if queue is finished or object was not emplaced during timeout template [[nodiscard]] bool tryEmplace(UInt64 milliseconds, Args &&... args) { - return emplaceImpl(milliseconds, std::forward(args...)); + return emplaceImpl(milliseconds, std::forward(args...)); + } + + /// Returns false if queue is finished and empty + [[nodiscard]] bool pop(T & x) + { + return popImpl(x, std::nullopt /*timeout in milliseconds*/); } /// Returns false if queue is (finished and empty) or (object was not popped during timeout) [[nodiscard]] bool tryPop(T & x, UInt64 milliseconds) { - return popImpl(x, milliseconds); + return popImpl(x, milliseconds); } /// Returns false if queue is empty. @@ -153,7 +172,7 @@ public: return false; detail::moveOrCopyIfThrow(std::move(queue.front()), x); - queue.pop(); + queue.pop_front(); } push_condition.notify_one(); @@ -222,7 +241,7 @@ public: if (is_finished) return; - std::queue empty_queue; + Container empty_queue; queue.swap(empty_queue); } @@ -235,7 +254,7 @@ public: { std::lock_guard lock(queue_mutex); - std::queue empty_queue; + Container empty_queue; queue.swap(empty_queue); is_finished = true; } From 752d27d663afd9b1b7ce384e9b9b84bf9f4e370e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 22:00:57 +0100 Subject: [PATCH 131/470] Fix lossing files during distributed batch send v2: do not suppress exceptions in case of errors Signed-off-by: Azat Khuzhin --- .../DistributedAsyncInsertDirectoryQueue.cpp | 128 ++++++++++-------- 1 file changed, 73 insertions(+), 55 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 31c621571ab..32d1823b289 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -545,78 +545,96 @@ void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() std::unordered_map header_to_batch; std::string file_path; - while (pending_files.tryPop(file_path)) + + try { - size_t total_rows = 0; - size_t total_bytes = 0; - Block header; - DistributedAsyncInsertHeader distributed_header; - try + while (pending_files.tryPop(file_path)) { - /// Determine metadata of the current file and check if it is not broken. - ReadBufferFromFile in{file_path}; - distributed_header = DistributedAsyncInsertHeader::read(in, log); - - if (distributed_header.rows) + size_t total_rows = 0; + size_t total_bytes = 0; + Block header; + DistributedAsyncInsertHeader distributed_header; + try { - total_rows += distributed_header.rows; - total_bytes += distributed_header.bytes; - } + /// Determine metadata of the current file and check if it is not broken. + ReadBufferFromFile in{file_path}; + distributed_header = DistributedAsyncInsertHeader::read(in, log); - if (distributed_header.block_header) - header = distributed_header.block_header; - - if (!total_rows || !header) - { - LOG_DEBUG(log, "Processing batch {} with old format (no header/rows)", in.getFileName()); - - CompressedReadBuffer decompressing_in(in); - NativeReader block_in(decompressing_in, distributed_header.revision); - - while (Block block = block_in.read()) + if (distributed_header.rows) { - total_rows += block.rows(); - total_bytes += block.bytes(); + total_rows += distributed_header.rows; + total_bytes += distributed_header.bytes; + } - if (!header) - header = block.cloneEmpty(); + if (distributed_header.block_header) + header = distributed_header.block_header; + + if (!total_rows || !header) + { + LOG_DEBUG(log, "Processing batch {} with old format (no header/rows)", in.getFileName()); + + CompressedReadBuffer decompressing_in(in); + NativeReader block_in(decompressing_in, distributed_header.revision); + + while (Block block = block_in.read()) + { + total_rows += block.rows(); + total_bytes += block.bytes(); + + if (!header) + header = block.cloneEmpty(); + } } } - } - catch (const Exception & e) - { - if (isDistributedSendBroken(e.code(), e.isRemoteException())) + catch (const Exception & e) { - markAsBroken(file_path); - tryLogCurrentException(log, "File is marked broken due to"); - continue; + if (isDistributedSendBroken(e.code(), e.isRemoteException())) + { + markAsBroken(file_path); + tryLogCurrentException(log, "File is marked broken due to"); + continue; + } + else + throw; + } + + BatchHeader batch_header( + std::move(distributed_header.insert_settings), + std::move(distributed_header.insert_query), + std::move(distributed_header.client_info), + std::move(header) + ); + DistributedAsyncInsertBatch & batch = header_to_batch.try_emplace(batch_header, *this).first->second; + + batch.files.push_back(file_path); + batch.total_rows += total_rows; + batch.total_bytes += total_bytes; + + if (batch.isEnoughSize()) + { + batch.send(); } - else - throw; } - BatchHeader batch_header( - std::move(distributed_header.insert_settings), - std::move(distributed_header.insert_query), - std::move(distributed_header.client_info), - std::move(header) - ); - DistributedAsyncInsertBatch & batch = header_to_batch.try_emplace(batch_header, *this).first->second; - - batch.files.push_back(file_path); - batch.total_rows += total_rows; - batch.total_bytes += total_bytes; - - if (batch.isEnoughSize()) + for (auto & kv : header_to_batch) { + DistributedAsyncInsertBatch & batch = kv.second; batch.send(); } } - - for (auto & kv : header_to_batch) + catch (...) { - DistributedAsyncInsertBatch & batch = kv.second; - batch.send(); + /// Revert uncommitted files. + for (const auto & [_, batch] : header_to_batch) + { + for (const auto & file : batch.files) + { + if (!pending_files.pushFront(file)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot re-schedule a file '{}'", file); + } + } + /// Rethrow exception + throw; } { From 16bfef3c8ae2f2af3f8a680ae6f4ed4e871ba6cd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Jan 2023 22:21:48 +0100 Subject: [PATCH 132/470] Fix processing current_batch.txt on init Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 32d1823b289..97aaa8fba54 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -550,6 +550,12 @@ void DistributedAsyncInsertDirectoryQueue::processFilesWithBatching() { while (pending_files.tryPop(file_path)) { + if (!fs::exists(file_path)) + { + LOG_WARNING(log, "File {} does not exists, likely due to current_batch.txt processing", file_path); + continue; + } + size_t total_rows = 0; size_t total_bytes = 0; Block header; From 7063c20b3c4e288397096431c93118f75eea9ca5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Feb 2023 16:43:08 +0100 Subject: [PATCH 133/470] Change noisy "Skipping send data over distributed table." message to test Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 97aaa8fba54..0e31a759c4f 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -229,7 +229,7 @@ void DistributedAsyncInsertDirectoryQueue::run() } } else - LOG_DEBUG(log, "Skipping send data over distributed table."); + LOG_TEST(log, "Skipping send data over distributed table."); const auto now = std::chrono::system_clock::now(); if (now - last_decrease_time > decrease_error_count_period) From 591fca57f3fd1c072078f386b339157bc42d9ad7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 18 Feb 2023 22:27:45 +0100 Subject: [PATCH 134/470] Fix function names for opentelemetry spans in StorageDistributed Fixes: 02417_opentelemetry_insert_on_distributed_table Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedAsyncInsertBatch.cpp | 2 ++ .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 5 +++-- src/Storages/Distributed/DistributedAsyncInsertHeader.cpp | 4 ++-- src/Storages/Distributed/DistributedAsyncInsertHeader.h | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp index 10bf9b8d66e..bf410eed6cc 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertBatch.cpp @@ -212,6 +212,7 @@ void DistributedAsyncInsertBatch::sendBatch() const auto & distributed_header = DistributedAsyncInsertHeader::read(in, parent.log); tracing_contexts.emplace_back(distributed_header.createTracingContextHolder( + __PRETTY_FUNCTION__, parent.storage.getContext()->getOpenTelemetrySpanLog())); tracing_contexts.back()->root_span.addAttribute("clickhouse.distributed_batch_start_time", batch_start_time); @@ -268,6 +269,7 @@ void DistributedAsyncInsertBatch::sendSeparateFiles() // This function is called in a separated thread, so we set up the trace context from the file trace_context = distributed_header.createTracingContextHolder( + __PRETTY_FUNCTION__, parent.storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index 0e31a759c4f..c6f675533c6 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -418,8 +418,9 @@ void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_ ReadBufferFromFile in(file_path); const auto & distributed_header = DistributedAsyncInsertHeader::read(in, log); - thread_trace_context = - distributed_header.createTracingContextHolder(storage.getContext()->getOpenTelemetrySpanLog()); + thread_trace_context = distributed_header.createTracingContextHolder( + __PRETTY_FUNCTION__, + storage.getContext()->getOpenTelemetrySpanLog()); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(distributed_header.insert_settings); auto connection = pool->get(timeouts, &distributed_header.insert_settings); diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp index b5c16558eb2..018c1d863bb 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.cpp @@ -107,10 +107,10 @@ DistributedAsyncInsertHeader DistributedAsyncInsertHeader::read(ReadBufferFromFi return distributed_header; } -OpenTelemetry::TracingContextHolderPtr DistributedAsyncInsertHeader::createTracingContextHolder(std::shared_ptr open_telemetry_span_log) const +OpenTelemetry::TracingContextHolderPtr DistributedAsyncInsertHeader::createTracingContextHolder(const char * function, std::shared_ptr open_telemetry_span_log) const { OpenTelemetry::TracingContextHolderPtr trace_context = std::make_unique( - __PRETTY_FUNCTION__, + function, client_info.client_trace_context, std::move(open_telemetry_span_log)); trace_context->root_span.addAttribute("clickhouse.shard_num", shard_num); diff --git a/src/Storages/Distributed/DistributedAsyncInsertHeader.h b/src/Storages/Distributed/DistributedAsyncInsertHeader.h index dfc059d7083..a7330fa5ef1 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertHeader.h +++ b/src/Storages/Distributed/DistributedAsyncInsertHeader.h @@ -39,7 +39,7 @@ struct DistributedAsyncInsertHeader Block block_header; static DistributedAsyncInsertHeader read(ReadBufferFromFile & in, Poco::Logger * log); - OpenTelemetry::TracingContextHolderPtr createTracingContextHolder(std::shared_ptr open_telemetry_span_log) const; + OpenTelemetry::TracingContextHolderPtr createTracingContextHolder(const char * function, std::shared_ptr open_telemetry_span_log) const; }; } From d06a4b50d60e6ccb6e8907559abd0a8c5b27edae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Feb 2023 22:40:54 +0100 Subject: [PATCH 135/470] Latest review fixes (variable naming: s/monitor/queue) Signed-off-by: Azat Khuzhin --- src/Storages/Distributed/DistributedSink.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 14febb1a2b4..1c372e87495 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -705,7 +705,7 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const CompressionCodecPtr compression_codec = CompressionCodecFactory::instance().get(compression_method, compression_level); /// tmp directory is used to ensure atomicity of transactions - /// and keep monitor thread out from reading incomplete data + /// and keep directory queue thread out from reading incomplete data std::string first_file_tmp_path; auto reservation = storage.getStoragePolicy()->reserveAndCheck(block.bytes()); @@ -824,12 +824,12 @@ void DistributedSink::writeToShard(const Cluster::ShardInfo & shard_info, const fs::create_directory(path); auto bin_file = (fs::path(path) / (toString(storage.file_names_increment.get()) + ".bin")).string(); - auto & directory_monitor = storage.getDirectoryQueue(disk, *it); + auto & directory_queue = storage.getDirectoryQueue(disk, *it); { createHardLink(first_file_tmp_path, bin_file); auto dir_sync_guard = make_directory_sync_guard(*it); } - directory_monitor.addFileAndSchedule(bin_file, file_size, sleep_ms); + directory_queue.addFileAndSchedule(bin_file, file_size, sleep_ms); } /// remove the temporary file, enabling the OS to reclaim inode after all threads From 99329d868232d9377d7f808763e951e6f15fd71c Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Fri, 10 Feb 2023 13:55:04 +0100 Subject: [PATCH 136/470] Improve tests for Distributed INSERT Signed-off-by: Azat Khuzhin --- .../configs/overrides_1.xml | 2 - .../configs/overrides_2.xml | 2 - .../test.py | 113 +++++++++++------- .../02536_distributed_detach_table.reference | 2 + .../02536_distributed_detach_table.sql | 16 +++ ...ed_loosing_files_after_exception.reference | 16 +++ ...buted_loosing_files_after_exception.sql.j2 | 32 +++++ 7 files changed, 135 insertions(+), 48 deletions(-) create mode 100644 tests/queries/0_stateless/02536_distributed_detach_table.reference create mode 100644 tests/queries/0_stateless/02536_distributed_detach_table.sql create mode 100644 tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.reference create mode 100644 tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_1.xml b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_1.xml index 397e05e7a60..a79ce3de1fc 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_1.xml +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_1.xml @@ -3,8 +3,6 @@ 0 - - 1 1 diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_2.xml b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_2.xml index 2ffd5beaf8d..8279fcdbe6d 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_2.xml +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/configs/overrides_2.xml @@ -3,8 +3,6 @@ 0 - - 1 0 diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index a47268b06fd..faa38af6533 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -18,61 +18,86 @@ node2 = cluster.add_instance( ) +def get_test_settings(): + settings = {"monitor_batch_inserts": [0, 1]} + return [(k, v) for k, values in settings.items() for v in values] + + +def drop_tables(): + tables = ["null_", "dist", "data", "mv", "dist_data"] + query = "\n".join([f"drop table if exists {table};" for table in tables]) + for _, node in cluster.instances.items(): + node.query(query) + + +def create_tables(**dist_settings): + drop_tables() + _settings_values = ",".join([f"{k}={v}" for k, v in dist_settings.items()]) + _settings = f"settings {_settings_values}" if _settings_values else "" + for _, node in cluster.instances.items(): + node.query( + f""" + create table null_ (key Int, value Int) engine=Null(); + create table dist as null_ engine=Distributed(test_cluster, currentDatabase(), null_, key) {_settings}; + create table data (key Int, uniq_values Int) engine=Memory(); + create materialized view mv to data as select key, uniqExact(value) uniq_values from null_ group by key; + system stop distributed sends dist; + + create table dist_data as data engine=Distributed(test_cluster, currentDatabase(), data); + """ + ) + + @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - - for _, node in cluster.instances.items(): - node.query( - """ - create table null_ (key Int, value Int) engine=Null(); - create table dist as null_ engine=Distributed(test_cluster, currentDatabase(), null_, key); - create table data (key Int, uniq_values Int) engine=Memory(); - create materialized view mv to data as select key, uniqExact(value) uniq_values from null_ group by key; - system stop distributed sends dist; - - create table dist_data as data engine=Distributed(test_cluster, currentDatabase(), data); - """ - ) - yield cluster finally: + drop_tables() cluster.shutdown() def test_distributed_directory_monitor_split_batch_on_failure_OFF(started_cluster): - for i in range(0, 100): - limit = 100e3 - node2.query( - f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}", - settings={ - # max_memory_usage is the limit for the batch on the remote node - # (local query should not be affected since 30MB is enough for 100K rows) - "max_memory_usage": "30Mi", - "max_untracked_memory": "0", - }, - ) - # "Received from" is mandatory, since the exception should be thrown on the remote node. - with pytest.raises( - QueryRuntimeException, - match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: .*while pushing to view default\.mv", - ): + for setting, setting_value in get_test_settings(): + create_tables(**{setting: setting_value}) + for i in range(0, 100): + limit = 100e3 + node2.query( + f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}", + settings={ + # max_memory_usage is the limit for the batch on the remote node + # (local query should not be affected since 30MB is enough for 100K rows) + "max_memory_usage": "30Mi", + "max_untracked_memory": "0", + }, + ) + # "Received from" is mandatory, since the exception should be thrown on the remote node. + if setting == "monitor_batch_inserts" and setting_value == 1: + with pytest.raises( + QueryRuntimeException, + match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: .*while pushing to view default\.mv", + ): + node2.query("system flush distributed dist") + assert int(node2.query("select count() from dist_data")) == 0 + continue node2.query("system flush distributed dist") - assert int(node2.query("select count() from dist_data")) == 0 + assert int(node2.query("select count() from dist_data")) == 100000 def test_distributed_directory_monitor_split_batch_on_failure_ON(started_cluster): - for i in range(0, 100): - limit = 100e3 - node1.query( - f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}", - settings={ - # max_memory_usage is the limit for the batch on the remote node - # (local query should not be affected since 30MB is enough for 100K rows) - "max_memory_usage": "30Mi", - "max_untracked_memory": "0", - }, - ) - node1.query("system flush distributed dist") - assert int(node1.query("select count() from dist_data")) == 100000 + for setting, setting_value in get_test_settings(): + create_tables(**{setting: setting_value}) + for i in range(0, 100): + limit = 100e3 + node1.query( + f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}", + settings={ + # max_memory_usage is the limit for the batch on the remote node + # (local query should not be affected since 30MB is enough for 100K rows) + "max_memory_usage": "30Mi", + "max_untracked_memory": "0", + }, + ) + node1.query("system flush distributed dist") + assert int(node1.query("select count() from dist_data")) == 100000 diff --git a/tests/queries/0_stateless/02536_distributed_detach_table.reference b/tests/queries/0_stateless/02536_distributed_detach_table.reference new file mode 100644 index 00000000000..f09bace4421 --- /dev/null +++ b/tests/queries/0_stateless/02536_distributed_detach_table.reference @@ -0,0 +1,2 @@ +0 0 +10 20 diff --git a/tests/queries/0_stateless/02536_distributed_detach_table.sql b/tests/queries/0_stateless/02536_distributed_detach_table.sql new file mode 100644 index 00000000000..92bee1ee544 --- /dev/null +++ b/tests/queries/0_stateless/02536_distributed_detach_table.sql @@ -0,0 +1,16 @@ +-- test detach distributed table with pending files +CREATE TABLE test_02536 (n Int8) ENGINE=MergeTree() ORDER BY tuple(); +CREATE TABLE test_dist_02536 (n Int8) ENGINE=Distributed(test_cluster_two_shards, currentDatabase(), test_02536, rand()); +SYSTEM STOP DISTRIBUTED SENDS test_dist_02536; + +INSERT INTO test_dist_02536 SELECT number FROM numbers(5) SETTINGS prefer_localhost_replica=0; +SELECT count(n), sum(n) FROM test_dist_02536; -- 0 0 + +DETACH TABLE test_dist_02536; +ATTACH TABLE test_dist_02536; + +SYSTEM FLUSH DISTRIBUTED test_dist_02536; + +SELECT count(n), sum(n) FROM test_dist_02536; -- 10 20 +DROP TABLE test_02536; +DROP TABLE test_dist_02536; diff --git a/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.reference b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.reference new file mode 100644 index 00000000000..7793e91fcb6 --- /dev/null +++ b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.reference @@ -0,0 +1,16 @@ +monitor_batch_insert=0 +1 2 +1 0 +-- { echoOn } +SELECT sum(key), count(key) FROM dist; +2 2 +SELECT sum(key), count(key) FROM underlying; +2 2 +monitor_batch_insert=1 +1 2 +1 0 +-- { echoOn } +SELECT sum(key), count(key) FROM dist; +2 2 +SELECT sum(key), count(key) FROM underlying; +2 2 diff --git a/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 new file mode 100644 index 00000000000..4f8cf1ccffe --- /dev/null +++ b/tests/queries/0_stateless/02537_distributed_loosing_files_after_exception.sql.j2 @@ -0,0 +1,32 @@ +{% for setting in [0, 1] %} +-- Testing that distributed table doesn't loose file after inserts which contain errors + +SELECT 'monitor_batch_insert={{ setting }}'; + +DROP TABLE IF EXISTS dist; +DROP TABLE IF EXISTS underlying; + +CREATE TABLE dist (key Int) ENGINE=Distributed(test_shard_localhost, currentDatabase(), underlying) SETTINGS monitor_batch_inserts={{ setting }}; +SYSTEM STOP DISTRIBUTED SENDS dist; + +INSERT INTO dist SETTINGS prefer_localhost_replica=0, max_threads=1 VALUES (1); +INSERT INTO dist SETTINGS prefer_localhost_replica=0, max_threads=2 VALUES (1); + +SYSTEM FLUSH DISTRIBUTED dist; -- { serverError UNKNOWN_TABLE } +-- check the second since after using queue it may got lost from it +SYSTEM FLUSH DISTRIBUTED dist; -- { serverError UNKNOWN_TABLE } + +SELECT is_blocked, data_files FROM system.distribution_queue WHERE database = currentDatabase() AND table = 'dist'; + +CREATE TABLE underlying (key Int) ENGINE=Memory(); +SYSTEM FLUSH DISTRIBUTED dist; + +-- all data should be flushed +SELECT is_blocked, data_files FROM system.distribution_queue WHERE database = currentDatabase() AND table = 'dist'; + +-- { echoOn } +SELECT sum(key), count(key) FROM dist; +SELECT sum(key), count(key) FROM underlying; +-- { echoOff } + +{% endfor %} From c797122dbe4fa97e726536cc5b6784e16a11dd44 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Mar 2023 08:45:50 +0000 Subject: [PATCH 137/470] Better --- src/Coordination/KeeperStorage.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b1f3b44b1e1..33b2a91d8bf 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -238,10 +238,7 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) void KeeperStorage::Node::recalculateSize() { size_bytes = sizeof(Node); - - for (const auto child_path : children) - size_bytes += sizeof child_path; - + size_bytes += children.size() * sizeof(decltype(children)::value_type); size_bytes += data.size(); } From e1352adced7eb8f5e5201fb12c1634903ade6a59 Mon Sep 17 00:00:00 2001 From: artem-yadr <84010375+artem-yadr@users.noreply.github.com> Date: Wed, 1 Mar 2023 12:50:03 +0300 Subject: [PATCH 138/470] Update MongoDBDictionarySource.cpp --- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index f61efcab026..a9555a94304 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -118,7 +118,7 @@ MongoDBDictionarySource::MongoDBDictionarySource( Poco::MongoDB::Connection::SocketFactory socket_factory; connection->connect(uri, socket_factory); - Poco::URI poco_uri(connection.uri()); + Poco::URI poco_uri(connection->uri()); // Parse database from URI. This is required for correctness -- the // cursor is created using database name and collection name, so we have From 67b271b1fa39719f69154d68ba660770b86e189e Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Wed, 1 Mar 2023 12:20:49 +0100 Subject: [PATCH 139/470] Keep indentation on Enter in the web UI --- programs/server/play.html | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/programs/server/play.html b/programs/server/play.html index c511d13cf91..d90293fbd1b 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -669,6 +669,29 @@ elem.selectionStart = selection_start + 4; elem.selectionEnd = selection_start + 4; + e.preventDefault(); + return false; + } else if (e.key === 'Enter') { + // If the user presses Enter, and the previous line starts with spaces, + // then we will insert the same number of spaces. + const elem = e.target; + if (elem.selectionStart !== elem.selectionEnd) { + // If there is a selection, then we will not insert spaces. + return; + } + const cursor_pos = elem.selectionStart; + + const elem_value = elem.value; + const text_before_cursor = elem_value.substring(0, cursor_pos); + const text_after_cursor = elem_value.substring(cursor_pos); + const prev_lines = text_before_cursor.split('\n'); + const prev_line = prev_lines.pop(); + const lead_spaces = prev_line.match(/^\s*/)[0]; + // Add leading spaces to the current line. + elem.value = text_before_cursor + '\n' + lead_spaces + text_after_cursor; + elem.selectionStart = cursor_pos + lead_spaces.length + 1; + elem.selectionEnd = elem.selectionStart; + e.preventDefault(); return false; } From 43dbd7bc2522ca174ef5065550e5a8031390af94 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 1 Mar 2023 12:37:34 +0100 Subject: [PATCH 140/470] Terminate long-running offline non-busy runners in EC2 --- tests/ci/ci_runners_metrics_lambda/app.py | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py index 44cae9221f5..341e1b674ec 100644 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ b/tests/ci/ci_runners_metrics_lambda/app.py @@ -37,15 +37,17 @@ RunnerDescriptions = List[RunnerDescription] def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: + """Returns instances that are offline/dead in EC2, or not found in EC2""" ids = { runner.name: runner for runner in runners # Only `i-deadbead123` are valid names for an instance ID - if runner.offline and not runner.busy and runner.name.startswith("i-") + if runner.name.startswith("i-") and runner.offline and not runner.busy } if not ids: return [] + # Delete all offline runners with wrong name result_to_delete = [ runner for runner in runners @@ -57,7 +59,7 @@ def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: i = 0 inc = 100 - print("Checking ids", ids.keys()) + print("Checking ids: ", " ".join(ids.keys())) instances_statuses = [] while i < len(ids.keys()): try: @@ -106,6 +108,9 @@ def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] )["Reservations"] lost_instances = [] + offline_runners = [ + runner.name for runner in runners if runner.offline and not runner.busy + ] # Here we refresh the runners to get the most recent state now = datetime.now().timestamp() @@ -126,6 +131,10 @@ def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[dict]: ): continue + if instance["InstanceId"] in offline_runners: + lost_instances.append(instance) + continue + if instance["State"]["Name"] == "running" and ( not [ runner From 32c198be45477782ec31fb864894500336216a71 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Mar 2023 12:47:31 +0000 Subject: [PATCH 141/470] trace raft logs --- tests/config/config.d/keeper_port.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 2066dedfa56..03f80bfbe62 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -10,11 +10,12 @@ 100000 10000 - true + false 240000 1000000000000000 100000 + trace 0 From 98004eec6b69d28ae735e75d263089ed9822e422 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 1 Mar 2023 13:08:24 +0000 Subject: [PATCH 142/470] Fix NOT_IMPLEMENTED error with CROSS JOIN and algorithm = auto --- src/Interpreters/ExpressionAnalyzer.cpp | 1 + src/Planner/PlannerJoins.cpp | 8 +++-- .../0_stateless/00202_cross_join.reference | 30 +++++++++++++++++++ .../queries/0_stateless/00202_cross_join.sql | 6 ++++ 4 files changed, 43 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2b88ff6a353..67aace815dc 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1089,6 +1089,7 @@ static std::shared_ptr chooseJoinAlgorithm( if (MergeJoin::isSupported(analyzed_join)) return std::make_shared(analyzed_join, right_sample_block); + return std::make_shared(analyzed_join, right_sample_block); } throw Exception(ErrorCodes::NOT_IMPLEMENTED, diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e1c137ddfb8..2a7bd49d6a3 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -655,7 +655,7 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo return std::make_shared(table_join, right_table_expression_header); } - if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH)) + if (!table_join->oneDisjunct() && !table_join->isEnabledAlgorithm(JoinAlgorithm::HASH) && !table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); /// Direct JOIN with special storages that support key value access. For example JOIN with Dictionary @@ -708,7 +708,11 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo } if (table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO)) - return std::make_shared(table_join, right_table_expression_header); + { + if (MergeJoin::isSupported(table_join)) + return std::make_shared(table_join, right_table_expression_header); + return std::make_shared(table_join, right_table_expression_header); + } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Can't execute any of specified algorithms for specified strictness/kind and right storage type"); diff --git a/tests/queries/0_stateless/00202_cross_join.reference b/tests/queries/0_stateless/00202_cross_join.reference index 122cf0a6e06..a8db281730a 100644 --- a/tests/queries/0_stateless/00202_cross_join.reference +++ b/tests/queries/0_stateless/00202_cross_join.reference @@ -13,3 +13,33 @@ 2 2 2 3 2 4 +0 0 +0 1 +0 2 +0 3 +0 4 +1 0 +1 1 +1 2 +1 3 +1 4 +2 0 +2 1 +2 2 +2 3 +2 4 +0 0 +0 1 +0 2 +0 3 +0 4 +1 0 +1 1 +1 2 +1 3 +1 4 +2 0 +2 1 +2 2 +2 3 +2 4 diff --git a/tests/queries/0_stateless/00202_cross_join.sql b/tests/queries/0_stateless/00202_cross_join.sql index ed435d90021..8d62c56b3f1 100644 --- a/tests/queries/0_stateless/00202_cross_join.sql +++ b/tests/queries/0_stateless/00202_cross_join.sql @@ -1 +1,7 @@ SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; + +SET join_algorithm = 'auto'; +SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; + +SET allow_experimental_analyzer = 1; +SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2; From f2e1d6d4025a3dcabf0f18828803d31252fc1bd9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 1 Mar 2023 13:11:32 +0000 Subject: [PATCH 143/470] Update version_date.tsv and changelogs after v23.2.2.20-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.2.2.20-stable.md | 30 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 3 +++ 5 files changed, 36 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.2.2.20-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 09395befdad..532f1531bb8 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 472f25eed2d..f56e177bbe9 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5dbb244c298..1372288978d 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.2.2.20-stable.md b/docs/changelogs/v23.2.2.20-stable.md new file mode 100644 index 00000000000..60aeaa66cbf --- /dev/null +++ b/docs/changelogs/v23.2.2.20-stable.md @@ -0,0 +1,30 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.2.20-stable (f6c269c8df2) FIXME as compared to v23.2.1.2537-stable (52bf836e03a) + +#### Improvement +* Backported in [#46914](https://github.com/ClickHouse/ClickHouse/issues/46914): Allow PREWHERE for Merge with different DEFAULT expression for column. [#46831](https://github.com/ClickHouse/ClickHouse/pull/46831) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#47022](https://github.com/ClickHouse/ClickHouse/issues/47022): Allow IPv4 in range(). [#46995](https://github.com/ClickHouse/ClickHouse/pull/46995) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). + +#### Bug Fix +* Backported in [#46828](https://github.com/ClickHouse/ClickHouse/issues/46828): Combined PREWHERE column accumulated from multiple PREWHERE in some cases didn't contain 0's from previous steps. The fix is to apply final filter if we know that it wasn't applied from more than 1 last step. [#46785](https://github.com/ClickHouse/ClickHouse/pull/46785) ([Alexander Gololobov](https://github.com/davenger)). + +#### Build/Testing/Packaging Improvement +* Backported in [#47062](https://github.com/ClickHouse/ClickHouse/issues/47062): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#46895](https://github.com/ClickHouse/ClickHouse/issues/46895): Fixed a bug in automatic retries of `DROP TABLE` query with `ReplicatedMergeTree` tables and `Atomic` databases. In rare cases it could lead to `Can't get data for node /zk_path/log_pointer` and `The specified key does not exist` errors if ZooKeeper session expired during DROP and a new replicated table with the same path in ZooKeeper was created in parallel. [#46384](https://github.com/ClickHouse/ClickHouse/pull/46384) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#46865](https://github.com/ClickHouse/ClickHouse/issues/46865): Fix a bug in the `Map` data type. This closes [#46855](https://github.com/ClickHouse/ClickHouse/issues/46855). [#46856](https://github.com/ClickHouse/ClickHouse/pull/46856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46958](https://github.com/ClickHouse/ClickHouse/issues/46958): Fix result of LIKE predicates which translate to substring searches and contain quoted non-LIKE metacharacters. [#46875](https://github.com/ClickHouse/ClickHouse/pull/46875) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* More concise logging at trace level for PREWHERE steps [#46771](https://github.com/ClickHouse/ClickHouse/pull/46771) ([Alexander Gololobov](https://github.com/davenger)). +* Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3814e94bf24..8a25ceb13cf 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,6 @@ +v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 @@ -25,6 +27,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 v22.8.11.15-lts 2022-12-08 From a898dd556fb475020a6a7fcb1fdf58ac43d826f2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 1 Mar 2023 13:12:22 +0000 Subject: [PATCH 144/470] Update version_date.tsv and changelogs after v23.1.4.58-stable --- docs/changelogs/v23.1.4.58-stable.md | 47 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 2 files changed, 49 insertions(+) create mode 100644 docs/changelogs/v23.1.4.58-stable.md diff --git a/docs/changelogs/v23.1.4.58-stable.md b/docs/changelogs/v23.1.4.58-stable.md new file mode 100644 index 00000000000..d1ffe87f58e --- /dev/null +++ b/docs/changelogs/v23.1.4.58-stable.md @@ -0,0 +1,47 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.1.4.58-stable (9ed562163a5) FIXME as compared to v23.1.3.5-stable (548b494bcce) + +#### Performance Improvement +* Backported in [#46380](https://github.com/ClickHouse/ClickHouse/issues/46380): Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Improvement +* Backported in [#46985](https://github.com/ClickHouse/ClickHouse/issues/46985): - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). +* Backported in [#46778](https://github.com/ClickHouse/ClickHouse/issues/46778): Backward compatibility for T64 codec support for IPv4. [#46747](https://github.com/ClickHouse/ClickHouse/pull/46747) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#47020](https://github.com/ClickHouse/ClickHouse/issues/47020): Allow IPv4 in range(). [#46995](https://github.com/ClickHouse/ClickHouse/pull/46995) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). + +#### Build/Testing/Packaging Improvement +* Backported in [#46031](https://github.com/ClickHouse/ClickHouse/issues/46031): Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46477](https://github.com/ClickHouse/ClickHouse/issues/46477): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46511](https://github.com/ClickHouse/ClickHouse/issues/46511): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#46228](https://github.com/ClickHouse/ClickHouse/issues/46228): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#46967](https://github.com/ClickHouse/ClickHouse/issues/46967): Backward compatibility - allow implicit narrowing conversion from UInt64 to IPv4 - required for "INSERT ... VALUES ..." expression. [#45865](https://github.com/ClickHouse/ClickHouse/pull/45865) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#46220](https://github.com/ClickHouse/ClickHouse/issues/46220): Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#46751](https://github.com/ClickHouse/ClickHouse/issues/46751): Follow-up fix for Replace domain IP types (IPv4, IPv6) with native https://github.com/ClickHouse/ClickHouse/pull/43221. [#46087](https://github.com/ClickHouse/ClickHouse/pull/46087) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#46448](https://github.com/ClickHouse/ClickHouse/issues/46448): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46680](https://github.com/ClickHouse/ClickHouse/issues/46680): Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46873](https://github.com/ClickHouse/ClickHouse/issues/46873): Fix a bug in the `Map` data type. This closes [#46855](https://github.com/ClickHouse/ClickHouse/issues/46855). [#46856](https://github.com/ClickHouse/ClickHouse/pull/46856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46956](https://github.com/ClickHouse/ClickHouse/issues/46956): Fix result of LIKE predicates which translate to substring searches and contain quoted non-LIKE metacharacters. [#46875](https://github.com/ClickHouse/ClickHouse/pull/46875) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Support DELETE ON CLUSTER [#45786](https://github.com/ClickHouse/ClickHouse/pull/45786) ([Alexander Gololobov](https://github.com/davenger)). +* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Wait for background tasks in ~UploadHelper [#46334](https://github.com/ClickHouse/ClickHouse/pull/46334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Improve install_check.py [#46458](https://github.com/ClickHouse/ClickHouse/pull/46458) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3814e94bf24..6cae29d53a8 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,4 +1,5 @@ v23.2.1.2537-stable 2023-02-23 +v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 @@ -25,6 +26,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 v22.8.11.15-lts 2022-12-08 From e8847f2fb6be0c951cc4e7a13cec580c8da88989 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 21:15:01 +0800 Subject: [PATCH 145/470] change as request --- .../functions/tuple-map-functions.md | 64 ++++++++++--------- src/Functions/map.cpp | 51 ++++++--------- .../0_stateless/01651_map_functions.reference | 2 + .../0_stateless/01651_map_functions.sql | 6 +- 4 files changed, 59 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index d59204a7482..28f2f930d90 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -66,6 +66,40 @@ Result: - [Map(key, value)](../../sql-reference/data-types/map.md) data type +## mapFromArrays + +mapFromArrays merges an array of keys and an array of values into a Map. + +**Syntax** + +```sql +mapFromArrays(keys, values) +``` + +Alias: `MAP_FROM_ARRAYS(keys, values)` + +**Parameters** +- `keys` — Given key array to create a map from. The nested type of array must be: [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md) +- `values` - Given value array to create a map from. + +**Returned value** + +- A map whose keys and values are constructed from the key and value arrays + +**Example** + +Query: + +```sql +select mapFromArrays(['a', 'b', 'c'], [1, 2, 3]) +``` + +```text +┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐ +│ {'a':1,'b':2,'c':3} │ +└───────────────────────────────────────────┘ +``` + ## mapAdd Collect all the keys and sum corresponding values. @@ -431,36 +465,6 @@ Result: ``` -## mapFromArrays {#mapFromArrays} - -**Syntax** - -```sql -mapFromArrays(keys, values) -``` - -**Parameters** - -- `keys` — Given key array to create a map from. -- `values` - Given value array to create a map from. - -**Returned value** - -- A map whose keys and values are from input arrays respectively. - -**Example** - -Query: - -```sql -select mapFromArrays(['a', 'b', 'c'], [1, 2, 3]) -``` - -```text -┌─mapFromArrays(['a', 'b', 'c'], [1, 2, 3])─┐ -│ {'a':1,'b':2,'c':3} │ -└───────────────────────────────────────────┘ -``` ## mapApply diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 78985bd18fc..549de200bea 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int ILLEGAL_COLUMN; } @@ -149,26 +149,16 @@ public: } }; -// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays +/// mapFromArrays(keys, values) is a function that allows you to make key-value pair from a pair of arrays class FunctionMapFromArrays : public IFunction { public: static constexpr auto name = "mapFromArrays"; - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + String getName() const override { return name; } - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } + size_t getNumberOfArguments() const override { return 2; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } @@ -177,25 +167,22 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Function {} requires 2 arguments, but {} given", getName(), arguments.size()); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires 2 arguments, but {} given", + getName(), + arguments.size()); const auto * keys_type = checkAndGetDataType(arguments[0].get()); if (!keys_type) - throw Exception{"First argument for function " + getName() + " must be a Array", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an Array", getName()); const auto * values_type = checkAndGetDataType(arguments[1].get()); if (!values_type) - throw Exception{"Second argument for function " + getName() + " must be a Array", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be an Array", getName()); - DataTypes tmp; - const auto & key_type = keys_type->getNestedType(); - const auto & value_type = values_type->getNestedType(); - tmp.emplace_back(key_type); - tmp.emplace_back(value_type); - return std::make_shared(tmp); + DataTypes key_value_types{keys_type->getNestedType(), values_type->getNestedType()}; + return std::make_shared(key_value_types); } ColumnPtr executeImpl( @@ -228,17 +215,15 @@ public: } if (!col_keys || !col_values) - throw Exception("Arguments of function " + getName() + " must be array.", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Arguments of function {} must be array", getName()); if (!col_keys->hasEqualOffsets(*col_values)) - throw Exception("Array arguments for function " + getName() + " must have equal sizes", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Array arguments for function {} must have equal sizes", getName()); const auto & data_keys = col_keys->getDataPtr(); const auto & data_values = col_values->getDataPtr(); const auto & offsets = col_keys->getOffsetsPtr(); - auto nested_column = ColumnArray::create( - ColumnTuple::create(Columns{std::move(data_keys), std::move(data_values)}), - std::move(offsets)); + auto nested_column = ColumnArray::create(ColumnTuple::create(Columns{data_keys, data_values}), offsets); return ColumnMap::create(nested_column); } }; @@ -745,6 +730,8 @@ REGISTER_FUNCTION(Map) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerAlias("MAP_FROM_ARRAYS", "mapFromArrays"); + } } diff --git a/tests/queries/0_stateless/01651_map_functions.reference b/tests/queries/0_stateless/01651_map_functions.reference index 284e27a1b0d..f7fd3503327 100644 --- a/tests/queries/0_stateless/01651_map_functions.reference +++ b/tests/queries/0_stateless/01651_map_functions.reference @@ -31,3 +31,5 @@ {'aa':4,'bb':5} {'aa':4,'bb':5} {'aa':4,'bb':5} +{'aa':4,'bb':5} +{'aa':4,'bb':5} diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 57023d41470..4d2bb3a0812 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -31,7 +31,9 @@ select map( 'aa', 4, 'bb' , 5) as m, mapContains(m, 'aa'), mapContains(m, 'k'); select map(0, 0) as m, mapContains(m, number % 2) from numbers(2); select mapFromArrays(['aa', 'bb'], [4, 5]); -select mapFromArrays(['aa', 'bb'], materialize([4, 5])); -select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(3); +select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(2); +select mapFromArrays(materialize(['aa', 'bb']), [4, 5]) from numbers(2); +select mapFromArrays(materialize(['aa', 'bb']), materialize([4, 5])) from numbers(2); select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError 42 } select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError 190 } +select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError 36 } From 598efe19858a0ca3aee6be015a99ebdf92c5ca67 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 21:17:05 +0800 Subject: [PATCH 146/470] change as request --- tests/queries/0_stateless/01651_map_functions.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index 4d2bb3a0812..f09739cc0d1 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -34,6 +34,8 @@ select mapFromArrays(['aa', 'bb'], [4, 5]); select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(2); select mapFromArrays(materialize(['aa', 'bb']), [4, 5]) from numbers(2); select mapFromArrays(materialize(['aa', 'bb']), materialize([4, 5])) from numbers(2); +select mapFromArrays('aa', [4, 5]); -- { serverError 43 } +select mapFromArrays(['aa', 'bb'], 5); -- { serverError 43 } select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError 42 } select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError 190 } select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError 36 } From 119ea045acf5edffb8e141b6abd285a8bfc5cdb8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 1 Mar 2023 21:19:39 +0800 Subject: [PATCH 147/470] change as request --- .../queries/0_stateless/01651_map_functions.sql | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01651_map_functions.sql b/tests/queries/0_stateless/01651_map_functions.sql index f09739cc0d1..848b6932fe0 100644 --- a/tests/queries/0_stateless/01651_map_functions.sql +++ b/tests/queries/0_stateless/01651_map_functions.sql @@ -8,7 +8,7 @@ select mapContains(a, 'name') from table_map; select mapContains(a, 'gender') from table_map; select mapContains(a, 'abc') from table_map; select mapContains(a, b) from table_map; -select mapContains(a, 10) from table_map; -- { serverError 386 } +select mapContains(a, 10) from table_map; -- { serverError NO_COMMON_TYPE } select mapKeys(a) from table_map; select mapFromArrays(c, d) from table_map; drop table if exists table_map; @@ -16,8 +16,8 @@ drop table if exists table_map; CREATE TABLE table_map (a Map(UInt8, Int), b UInt8, c UInt32, d Array(String), e Array(String)) engine = MergeTree order by tuple(); insert into table_map select map(number, number), number, number, [number, number, number], [number*2, number*3, number*4] from numbers(1000, 3); select mapContains(a, b), mapContains(a, c), mapContains(a, 233) from table_map; -select mapContains(a, 'aaa') from table_map; -- { serverError 386 } -select mapContains(b, 'aaa') from table_map; -- { serverError 43 } +select mapContains(a, 'aaa') from table_map; -- { serverError NO_COMMON_TYPE } +select mapContains(b, 'aaa') from table_map; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } select mapKeys(a) from table_map; select mapValues(a) from table_map; select mapFromArrays(d, e) from table_map; @@ -34,8 +34,8 @@ select mapFromArrays(['aa', 'bb'], [4, 5]); select mapFromArrays(['aa', 'bb'], materialize([4, 5])) from numbers(2); select mapFromArrays(materialize(['aa', 'bb']), [4, 5]) from numbers(2); select mapFromArrays(materialize(['aa', 'bb']), materialize([4, 5])) from numbers(2); -select mapFromArrays('aa', [4, 5]); -- { serverError 43 } -select mapFromArrays(['aa', 'bb'], 5); -- { serverError 43 } -select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError 42 } -select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError 190 } -select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError 36 } +select mapFromArrays('aa', [4, 5]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select mapFromArrays(['aa', 'bb'], 5); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select mapFromArrays(['aa', 'bb'], [4, 5], [6, 7]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select mapFromArrays(['aa', 'bb'], [4, 5, 6]); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } +select mapFromArrays([[1,2], [3,4]], [4, 5, 6]); -- { serverError BAD_ARGUMENTS } From 50d033c563c58d5925a19eff01464616fe16ea7f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 1 Mar 2023 13:48:23 +0000 Subject: [PATCH 148/470] Update version_date.tsv and changelogs after v22.12.4.76-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v22.12.4.76-stable.md | 55 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 4 ++ 5 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v22.12.4.76-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 09395befdad..532f1531bb8 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 472f25eed2d..f56e177bbe9 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5dbb244c298..1372288978d 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.2.20" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v22.12.4.76-stable.md b/docs/changelogs/v22.12.4.76-stable.md new file mode 100644 index 00000000000..79569ff841e --- /dev/null +++ b/docs/changelogs/v22.12.4.76-stable.md @@ -0,0 +1,55 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.12.4.76-stable (cb5772db805) FIXME as compared to v22.12.3.5-stable (893de538f02) + +#### Performance Improvement +* Backported in [#45704](https://github.com/ClickHouse/ClickHouse/issues/45704): Fixed performance of short `SELECT` queries that read from tables with large number of`Array`/`Map`/`Nested` columns. [#45630](https://github.com/ClickHouse/ClickHouse/pull/45630) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46378](https://github.com/ClickHouse/ClickHouse/issues/46378): Fix too big memory usage for vertical merges on non-remote disk. Respect `max_insert_delayed_streams_for_parallel_write` for the remote disk. [#46275](https://github.com/ClickHouse/ClickHouse/pull/46275) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Bug Fix +* Backported in [#45672](https://github.com/ClickHouse/ClickHouse/issues/45672): Fix wiping sensitive info in logs. [#45603](https://github.com/ClickHouse/ClickHouse/pull/45603) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Build/Testing/Packaging Improvement +* Backported in [#45200](https://github.com/ClickHouse/ClickHouse/issues/45200): Fix zookeeper downloading, update the version, and optimize the image size. [#44853](https://github.com/ClickHouse/ClickHouse/pull/44853) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46116](https://github.com/ClickHouse/ClickHouse/issues/46116): Remove the dependency on the `adduser` tool from the packages, because we don't use it. This fixes [#44934](https://github.com/ClickHouse/ClickHouse/issues/44934). [#45011](https://github.com/ClickHouse/ClickHouse/pull/45011) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46035](https://github.com/ClickHouse/ClickHouse/issues/46035): Add systemd.service file for clickhouse-keeper. Fixes [#44293](https://github.com/ClickHouse/ClickHouse/issues/44293). [#45568](https://github.com/ClickHouse/ClickHouse/pull/45568) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46484](https://github.com/ClickHouse/ClickHouse/issues/46484): Get rid of unnecessary build for standalone clickhouse-keeper. [#46367](https://github.com/ClickHouse/ClickHouse/pull/46367) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#46509](https://github.com/ClickHouse/ClickHouse/issues/46509): Some time ago the ccache compression was changed to `zst`, but `gz` archives are downloaded by default. It fixes it by prioritizing zst archive. [#46490](https://github.com/ClickHouse/ClickHouse/pull/46490) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#47058](https://github.com/ClickHouse/ClickHouse/issues/47058): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#45904](https://github.com/ClickHouse/ClickHouse/issues/45904): Fixed bug with non-parsable default value for EPHEMERAL column in table metadata. [#44026](https://github.com/ClickHouse/ClickHouse/pull/44026) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Backported in [#45321](https://github.com/ClickHouse/ClickHouse/issues/45321): Fixed a bug in normalization of a `DEFAULT` expression in `CREATE TABLE` statement. The second argument of function `in` (or the right argument of operator `IN`) might be replaced with the result of its evaluation during CREATE query execution. Fixes [#44496](https://github.com/ClickHouse/ClickHouse/issues/44496). [#44547](https://github.com/ClickHouse/ClickHouse/pull/44547) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#45000](https://github.com/ClickHouse/ClickHouse/issues/45000): Another fix for `Cannot read all data` error which could happen while reading `LowCardinality` dictionary from remote fs. Fixes [#44709](https://github.com/ClickHouse/ClickHouse/issues/44709). [#44875](https://github.com/ClickHouse/ClickHouse/pull/44875) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#45553](https://github.com/ClickHouse/ClickHouse/issues/45553): Fix `SELECT ... FROM system.dictionaries` exception when there is a dictionary with a bad structure (e.g. incorrect type in xml config). [#45399](https://github.com/ClickHouse/ClickHouse/pull/45399) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#46226](https://github.com/ClickHouse/ClickHouse/issues/46226): A couple of seg faults have been reported around `c-ares`. All of the recent stack traces observed fail on inserting into `std::unodered_set<>`. I believe I have found the root cause of this, it seems to be unprocessed queries. Prior to this PR, CH calls `poll` to wait on the file descriptors in the `c-ares` channel. According to the [poll docs](https://man7.org/linux/man-pages/man2/poll.2.html), a negative return value means an error has ocurred. Because of this, we would abort the execution and return failure. The problem is that `poll` will also return a negative value if a system interrupt occurs. A system interrupt does not mean the processing has failed or ended, but we would abort it anyways because we were checking for negative values. Once the execution is aborted, the whole stack is destroyed, which includes the `std::unordered_set` passed to the `void *` parameter of the c-ares callback. Once c-ares completed the request, the callback would be invoked and would access an invalid memory address causing a segfault. [#45629](https://github.com/ClickHouse/ClickHouse/pull/45629) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#46218](https://github.com/ClickHouse/ClickHouse/issues/46218): Fix reading of non existing nested columns with multiple level in compact parts. [#46045](https://github.com/ClickHouse/ClickHouse/pull/46045) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#46446](https://github.com/ClickHouse/ClickHouse/issues/46446): Fix possible `LOGICAL_ERROR` in asynchronous inserts with invalid data sent in format `VALUES`. [#46350](https://github.com/ClickHouse/ClickHouse/pull/46350) ([Anton Popov](https://github.com/CurtizJ)). +* Backported in [#46678](https://github.com/ClickHouse/ClickHouse/issues/46678): Fix an invalid processing of constant `LowCardinality` argument in function `arrayMap`. This bug could lead to a segfault in release, and logical error `Bad cast` in debug build. [#46569](https://github.com/ClickHouse/ClickHouse/pull/46569) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46872](https://github.com/ClickHouse/ClickHouse/issues/46872): Fix a bug in the `Map` data type. This closes [#46855](https://github.com/ClickHouse/ClickHouse/issues/46855). [#46856](https://github.com/ClickHouse/ClickHouse/pull/46856) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#46954](https://github.com/ClickHouse/ClickHouse/issues/46954): Fix result of LIKE predicates which translate to substring searches and contain quoted non-LIKE metacharacters. [#46875](https://github.com/ClickHouse/ClickHouse/pull/46875) ([Robert Schulze](https://github.com/rschu1ze)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Automatically merge green backport PRs and green approved PRs [#41110](https://github.com/ClickHouse/ClickHouse/pull/41110) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Improve release scripts [#45074](https://github.com/ClickHouse/ClickHouse/pull/45074) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Another attempt to fix automerge, or at least to have debug footprint [#45476](https://github.com/ClickHouse/ClickHouse/pull/45476) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add check for running workflows to merge_pr.py [#45803](https://github.com/ClickHouse/ClickHouse/pull/45803) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of progress timestamps in release publishing [#45818](https://github.com/ClickHouse/ClickHouse/pull/45818) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add necessary dependency for sanitizers [#45959](https://github.com/ClickHouse/ClickHouse/pull/45959) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add helping logging to auto-merge script [#46080](https://github.com/ClickHouse/ClickHouse/pull/46080) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix write buffer destruction order for vertical merge. [#46205](https://github.com/ClickHouse/ClickHouse/pull/46205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Improve install_check.py [#46458](https://github.com/ClickHouse/ClickHouse/pull/46458) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix dependencies for InstallPackagesTestAarch64 [#46597](https://github.com/ClickHouse/ClickHouse/pull/46597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Get rid of legacy DocsReleaseChecks [#46665](https://github.com/ClickHouse/ClickHouse/pull/46665) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3814e94bf24..4cc74097371 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,7 +1,10 @@ +v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 +v22.12.4.76-stable 2023-03-01 v22.12.3.5-stable 2023-01-10 v22.12.2.25-stable 2023-01-06 v22.12.1.1752-stable 2022-12-15 @@ -25,6 +28,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 v22.8.11.15-lts 2022-12-08 From b11ed5910727f549a68d1a1dba8819505a8949f2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 1 Mar 2023 15:37:03 +0100 Subject: [PATCH 149/470] remove ps --- .../0_stateless/02434_cancel_insert_when_client_dies.sh | 7 +++---- .../0_stateless/02435_rollback_cancelled_queries.sh | 9 ++++----- tests/queries/shell_config.sh | 3 ++- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 2a17095b267..c4d7367ba14 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -67,7 +67,7 @@ function thread_cancel if (( RANDOM % 2 )); then SIGNAL="KILL" fi - PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') + PID=$(grep -Fa "$TEST_MARK" /proc/*/cmdline | grep -Fav grep | grep -Eoa "/proc/[0-9]*/cmdline:" | grep -Eo "[0-9]*" | head -1) if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID"; fi sleep 0.$RANDOM; sleep 0.$RANDOM; @@ -91,8 +91,7 @@ $CLICKHOUSE_CLIENT -q 'select count() from dedup_test' $CLICKHOUSE_CLIENT -q 'system flush logs' -# We have to ignore stderr from thread_cancel, because our CI finds a bug in ps... -# So use this query to check that thread_cancel do something +# Ensure that thread_cancel actually did something $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or - message like '%Connection reset by peer%')" + message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index a69e526c1c8..b639cd5ef70 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -83,7 +83,7 @@ function thread_cancel if (( RANDOM % 2 )); then SIGNAL="KILL" fi - PID=$(ps -ef | grep "$TEST_MARK" | grep -v grep | awk '{print $2}') + PID=$(grep -Fa "$TEST_MARK" /proc/*/cmdline | grep -Fav grep | grep -Eoa "/proc/[0-9]*/cmdline:" | grep -Eo "[0-9]*" | head -1) if [ ! -z "$PID" ]; then kill -s "$SIGNAL" "$PID"; fi sleep 0.$RANDOM; done @@ -93,14 +93,13 @@ export -f thread_insert; export -f thread_select; export -f thread_cancel; -TIMEOUT=20 # 5 seconds for each TYPE +TIMEOUT=20 timeout $TIMEOUT bash -c thread_insert & timeout $TIMEOUT bash -c thread_select & timeout $TIMEOUT bash -c thread_cancel 2> /dev/null & wait -wait_for_queries_to_finish $CLICKHOUSE_CLIENT -q 'system flush logs' @@ -110,10 +109,10 @@ insert_data $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" -# We have to ignore stderr from thread_cancel, because our CI finds a bug in ps... -# So use this query to check that thread_cancel do something +# Ensure that thread_cancel actually did something $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" +wait_for_queries_to_finish 30 $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 3006b74d3f9..ffa286a95cd 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -136,12 +136,13 @@ function clickhouse_client_removed_host_parameter() function wait_for_queries_to_finish() { + local max_tries="${1:-20}" # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query NOT LIKE '%system.processes%'") -ne 0 ]]; do sleep 0.5; num_tries=$((num_tries+1)) - if [ $num_tries -eq 20 ]; then + if [ $num_tries -eq $max_tries ]; then $CLICKHOUSE_CLIENT -q "SELECT * FROM system.processes WHERE current_database=currentDatabase() AND query NOT LIKE '%system.processes%' FORMAT Vertical" break fi From 6ebee202bd398542b2867d455b6b193e87b627cf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Mar 2023 14:47:15 +0000 Subject: [PATCH 150/470] Flush buffer when no sync --- src/Coordination/Changelog.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 899310dc591..ddedae4fa0f 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -211,9 +211,14 @@ public: void flush() { auto * file_buffer = tryGetFileBuffer(); - /// Fsync file system if needed - if (file_buffer && log_file_settings.force_sync) - file_buffer->sync(); + if (file_buffer) + { + /// Fsync file system if needed + if (log_file_settings.force_sync) + file_buffer->sync(); + else + file_buffer->next(); + } } uint64_t getStartIndex() const From 6ee65fa1dc3bef79cd6989a66999eca04fce32ee Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 1 Mar 2023 16:41:51 +0100 Subject: [PATCH 151/470] fix shellcheck --- .../0_stateless/01169_alter_partition_isolation_stress.sh | 1 + .../queries/0_stateless/01171_mv_select_insert_isolation_long.sh | 1 + tests/queries/0_stateless/01174_select_insert_isolation.sh | 1 + tests/queries/shell_config.sh | 1 + 4 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh index a385fc81fe4..508ad05224c 100755 --- a/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh +++ b/tests/queries/0_stateless/01169_alter_partition_isolation_stress.sh @@ -2,6 +2,7 @@ # Tags: long, no-replicated-database, no-ordinary-database # shellcheck disable=SC2015 +# shellcheck disable=SC2119 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh index 12b654f4215..199c2b5389f 100755 --- a/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh +++ b/tests/queries/0_stateless/01171_mv_select_insert_isolation_long.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash # Tags: long, no-parallel, no-ordinary-database # Test is too heavy, avoid parallel run in Flaky Check +# shellcheck disable=SC2119 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01174_select_insert_isolation.sh b/tests/queries/0_stateless/01174_select_insert_isolation.sh index dc5c1d7a722..29ccfbb1ccb 100755 --- a/tests/queries/0_stateless/01174_select_insert_isolation.sh +++ b/tests/queries/0_stateless/01174_select_insert_isolation.sh @@ -2,6 +2,7 @@ # Tags: long, no-ordinary-database # shellcheck disable=SC2015 +# shellcheck disable=SC2119 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index ffa286a95cd..ef70c82aefc 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# shellcheck disable=SC2120 # Don't check for ODR violation, since we may test shared build with ASAN export ASAN_OPTIONS=detect_odr_violation=0 From 96e7454df7cd2fd7eaaec97084b679b48be8416b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 1 Mar 2023 16:31:50 +0000 Subject: [PATCH 152/470] Fix test --- tests/queries/0_stateless/00965_logs_level_bugfix.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/00965_logs_level_bugfix.reference b/tests/queries/0_stateless/00965_logs_level_bugfix.reference index 52396b3fe79..affd41b780b 100644 --- a/tests/queries/0_stateless/00965_logs_level_bugfix.reference +++ b/tests/queries/0_stateless/00965_logs_level_bugfix.reference @@ -2,7 +2,6 @@ . . - . - From c59220274b873a3fd09187a4bb2021bba2402833 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 1 Mar 2023 17:53:48 +0100 Subject: [PATCH 153/470] release lambda resources before detaching thread group --- src/Interpreters/threadPoolCallbackRunner.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 7990c538c03..1d2f67e8e59 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -25,8 +25,16 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & CurrentThread::attachTo(thread_group); SCOPE_EXIT_SAFE({ + { + /// Release all captutred resources before detaching thread group + /// Releasing has to use proper memory tracker which has been set here before callback + + [[maybe_unused]] auto tmp = std::move(callback); + } + if (thread_group) CurrentThread::detachQueryIfNotDetached(); + }); setThreadName(thread_name.data()); From 9f109f0876c84c2f104ffbb96a7e82ed62328095 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 16 Feb 2023 11:34:36 +0100 Subject: [PATCH 154/470] Fix JOIN tests with invalid USING --- tests/queries/0_stateless/00057_join_aliases.sql | 4 ++-- .../00099_join_many_blocks_segfault.sql | 7 +++++-- .../00122_join_with_subquery_with_subquery.sql | 2 +- .../queries/0_stateless/00138_table_aliases.sql | 2 +- .../0_stateless/00162_shard_global_join.sql | 3 ++- .../00163_shard_join_with_empty_table.sql | 16 ++++++++-------- 6 files changed, 19 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/00057_join_aliases.sql b/tests/queries/0_stateless/00057_join_aliases.sql index b994e26a747..6f283094396 100644 --- a/tests/queries/0_stateless/00057_join_aliases.sql +++ b/tests/queries/0_stateless/00057_join_aliases.sql @@ -1,6 +1,6 @@ SELECT * FROM ( - SELECT number, number / 2 AS n, j1, j2 - FROM system.numbers + SELECT number, n, j1, j2 + FROM (SELECT number, number / 2 AS n FROM system.numbers) js1 ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) js2 USING n LIMIT 10 ) ORDER BY n diff --git a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql index 7219bcecf55..31dec093e61 100644 --- a/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql +++ b/tests/queries/0_stateless/00099_join_many_blocks_segfault.sql @@ -1,7 +1,10 @@ SELECT - 1 AS DomainID, + DomainID, Domain -FROM system.one +FROM +( + SELECT 1 AS DomainID FROM system.one +) js1 ANY LEFT JOIN ( SELECT diff --git a/tests/queries/0_stateless/00122_join_with_subquery_with_subquery.sql b/tests/queries/0_stateless/00122_join_with_subquery_with_subquery.sql index 0cb3de94a55..add31112579 100644 --- a/tests/queries/0_stateless/00122_join_with_subquery_with_subquery.sql +++ b/tests/queries/0_stateless/00122_join_with_subquery_with_subquery.sql @@ -1 +1 @@ -SELECT 1 AS k FROM system.one ANY LEFT JOIN (SELECT k FROM (SELECT 1 AS k, 2 AS x)) js2 USING k; +SELECT k FROM (SELECT 1 AS k FROM system.one) js1 ANY LEFT JOIN (SELECT k FROM (SELECT 1 AS k, 2 AS x)) js2 USING k; diff --git a/tests/queries/0_stateless/00138_table_aliases.sql b/tests/queries/0_stateless/00138_table_aliases.sql index 58fe2ac65e9..1d168250266 100644 --- a/tests/queries/0_stateless/00138_table_aliases.sql +++ b/tests/queries/0_stateless/00138_table_aliases.sql @@ -1,2 +1,2 @@ SELECT * FROM `system`.`one` AS `xxx`; -SELECT 1 AS k, s FROM `system`.`one` AS `xxx` ANY LEFT JOIN (SELECT 1 AS k, 'Hello' AS s) AS `yyy` USING k; +SELECT k, s FROM (SELECT 1 AS k FROM `system`.`one`) AS `xxx` ANY LEFT JOIN (SELECT 1 AS k, 'Hello' AS s) AS `yyy` USING k; diff --git a/tests/queries/0_stateless/00162_shard_global_join.sql b/tests/queries/0_stateless/00162_shard_global_join.sql index 040eb0e7c03..0bf80e70dda 100644 --- a/tests/queries/0_stateless/00162_shard_global_join.sql +++ b/tests/queries/0_stateless/00162_shard_global_join.sql @@ -1,3 +1,4 @@ -- Tags: shard -SELECT toFloat64(dummy + 2) AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) jr1 GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) jr2 USING n LIMIT 10; +SELECT n, j1, j2 FROM (SELECT toFloat64(dummy + 2) AS n FROM remote('127.0.0.{2,3}', system.one)) jr1 +GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) jr2 USING n LIMIT 10; diff --git a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql index 777b2443416..07242f7a8d4 100644 --- a/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql +++ b/tests/queries/0_stateless/00163_shard_join_with_empty_table.sql @@ -4,29 +4,29 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; SELECT * FROM ( - SELECT number, number / 2 AS n, j1, j2 - FROM remote('127.0.0.{2,3}', system.numbers) + SELECT number, n, j1, j2 + FROM (SELECT number, number / 2 AS n FROM remote('127.0.0.{2,3}', system.numbers)) ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10 ) ORDER BY number; SELECT * FROM ( - SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 - FROM remote('127.0.0.{2,3}', system.one) + SELECT number, n, j1, j2 + FROM (SELECT dummy + 2 AS number, number / 2 AS n FROM remote('127.0.0.{2,3}', system.one)) ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10 ) ORDER BY number; SELECT * FROM ( - SELECT number, number / 2 AS n, j1, j2 - FROM remote('127.0.0.{2,3}', system.numbers) + SELECT number, n, j1, j2 + FROM (SELECT number, number / 2 AS n FROM remote('127.0.0.{2,3}', system.numbers)) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10 ) ORDER BY number; SELECT * FROM ( - SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 - FROM remote('127.0.0.{2,3}', system.one) + SELECT number, n, j1, j2 + FROM (SELECT dummy + 2 AS number, number / 2 AS n FROM remote('127.0.0.{2,3}', system.one)) GLOBAL ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10 ) ORDER BY number; From f1093716f0457c5427168f2a318bf6bb6e624d72 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 13:45:34 +0100 Subject: [PATCH 155/470] Analyzer MutationsInterpreter fix --- src/Interpreters/MutationsInterpreter.cpp | 24 +++++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 034f327db91..0b52a1a51bc 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -172,6 +172,21 @@ ASTPtr prepareQueryAffectedAST(const std::vector & commands, co return select; } +QueryTreeNodePtr prepareQueryAffectedQueryTree(const std::vector & commands, const StoragePtr & storage, ContextPtr context) +{ + auto ast = prepareQueryAffectedAST(commands, storage, context); + auto query_tree = buildQueryTree(ast, context); + + auto & query_node = query_tree->as(); + query_node.getJoinTree() = std::make_shared(storage, context); + + QueryTreePassManager query_tree_pass_manager(context); + addQueryTreePasses(query_tree_pass_manager); + query_tree_pass_manager.run(query_tree); + + return query_tree; +} + ColumnDependencies getAllColumnDependencies(const StorageMetadataPtr & metadata_snapshot, const NameSet & updated_columns) { NameSet new_updated_columns = updated_columns; @@ -231,18 +246,15 @@ bool isStorageTouchedByMutations( std::optional interpreter_select_query; BlockIO io; - ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), context); - if (context->getSettingsRef().allow_experimental_analyzer) { - InterpreterSelectQueryAnalyzer interpreter(select_query, - context, - storage_from_part, - SelectQueryOptions().ignoreLimits().ignoreProjections()); + auto select_query_tree = prepareQueryAffectedQueryTree(commands, storage.shared_from_this(), context); + InterpreterSelectQueryAnalyzer interpreter(select_query_tree, context, SelectQueryOptions().ignoreLimits().ignoreProjections()); io = interpreter.execute(); } else { + ASTPtr select_query = prepareQueryAffectedAST(commands, storage.shared_from_this(), 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. From 0ee0324c2cf599838bfd28127eef3641d159d55b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 18:52:39 +0100 Subject: [PATCH 156/470] Analyzer ARRAY JOIN conversion to AST fix --- src/Analyzer/ArrayJoinNode.cpp | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 2157b5edf6f..490e227d617 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -5,8 +5,10 @@ #include #include +#include #include +#include namespace DB { @@ -52,8 +54,24 @@ ASTPtr ArrayJoinNode::toASTImpl() const auto array_join_ast = std::make_shared(); array_join_ast->kind = is_left ? ASTArrayJoin::Kind::Left : ASTArrayJoin::Kind::Inner; - const auto & join_expression_list_node = getJoinExpressionsNode(); - array_join_ast->children.push_back(join_expression_list_node->toAST()); + auto array_join_expressions_ast = std::make_shared(); + const auto & array_join_expressions = getJoinExpressions().getNodes(); + + for (const auto & array_join_expression : array_join_expressions) + { + ASTPtr array_join_expression_ast; + + auto * column_node = array_join_expression->as(); + if (column_node && column_node->getExpression()) + array_join_expression_ast = column_node->getExpression()->toAST(); + else + array_join_expression_ast = array_join_expression->toAST(); + + array_join_expression_ast->setAlias(array_join_expression->getAlias()); + array_join_expressions_ast->children.push_back(std::move(array_join_expression_ast)); + } + + array_join_ast->children.push_back(std::move(array_join_expressions_ast)); array_join_ast->expression_list = array_join_ast->children.back(); ASTPtr tables_in_select_query_ast = std::make_shared(); From 3fbb7f89e5a438ff357506663a0afeab8395a37a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 21:14:22 +0100 Subject: [PATCH 157/470] Fixed tests --- .../00674_join_on_syntax.reference | 4 +-- .../0_stateless/00674_join_on_syntax.sql | 1 + .../0_stateless/00702_join_with_using.sql | 2 +- .../0_stateless/00722_inner_join.reference | 30 +++++++++---------- .../queries/0_stateless/00722_inner_join.sql | 2 ++ .../00818_inner_join_bug_3567.reference | 28 ++++++++--------- .../0_stateless/00818_inner_join_bug_3567.sql | 2 ++ ...full_join_wrong_columns_in_block.reference | 22 +++++++------- ...00819_full_join_wrong_columns_in_block.sql | 1 + .../00820_multiple_joins.reference | 4 +-- .../0_stateless/00820_multiple_joins.sql | 2 ++ ...le_joins_subquery_requires_alias.reference | 4 +-- ...multiple_joins_subquery_requires_alias.sql | 2 ++ .../00834_hints_for_type_function_typos.sh | 2 +- .../00835_if_generic_case.reference | 2 +- .../0_stateless/00835_if_generic_case.sql | 2 ++ .../01429_join_on_error_messages.sql | 6 ++-- .../01508_explain_header.reference | 2 +- .../01527_bad_aggregation_in_lambda.sql | 4 ++- 19 files changed, 69 insertions(+), 53 deletions(-) diff --git a/tests/queries/0_stateless/00674_join_on_syntax.reference b/tests/queries/0_stateless/00674_join_on_syntax.reference index 10c31e56fe8..7bad0431b8f 100644 --- a/tests/queries/0_stateless/00674_join_on_syntax.reference +++ b/tests/queries/0_stateless/00674_join_on_syntax.reference @@ -49,8 +49,8 @@ duplicate column names {"a1":1,"tab1_copy.a1":2} {"a1":1,"tab1_copy.a1":2} {"a1":1,"copy.a1":2} -{"a1":1,"copy.a1":2} -{"a1":1,"copy.a1":2} +{"a1":1,"tab1_copy.a1":2} +{"a1":1,"tab1_copy.a1":2} subquery 1 1 diff --git a/tests/queries/0_stateless/00674_join_on_syntax.sql b/tests/queries/0_stateless/00674_join_on_syntax.sql index cdca4e27a71..9ff26db1536 100644 --- a/tests/queries/0_stateless/00674_join_on_syntax.sql +++ b/tests/queries/0_stateless/00674_join_on_syntax.sql @@ -1,4 +1,5 @@ SET joined_subquery_requires_alias = 0; +SET allow_experimental_analyzer = 1; drop table if exists tab1; drop table if exists tab2; diff --git a/tests/queries/0_stateless/00702_join_with_using.sql b/tests/queries/0_stateless/00702_join_with_using.sql index 4f81e757dfc..462524e1922 100644 --- a/tests/queries/0_stateless/00702_join_with_using.sql +++ b/tests/queries/0_stateless/00702_join_with_using.sql @@ -7,7 +7,7 @@ CREATE TABLE using2(a UInt8, b UInt8) ENGINE=Memory; INSERT INTO using1 VALUES (1, 1) (2, 2) (3, 3); INSERT INTO using2 VALUES (4, 4) (2, 2) (3, 3); -SELECT * FROM using1 ALL LEFT JOIN (SELECT * FROM using2) js2 USING (a, a, a, b, b, b, a, a) ORDER BY a; +SELECT * FROM using1 ALL LEFT JOIN (SELECT * FROM using2) js2 USING (a, b) ORDER BY a; DROP TABLE using1; DROP TABLE using2; diff --git a/tests/queries/0_stateless/00722_inner_join.reference b/tests/queries/0_stateless/00722_inner_join.reference index b5e8a77a20d..86c07e6e84e 100644 --- a/tests/queries/0_stateless/00722_inner_join.reference +++ b/tests/queries/0_stateless/00722_inner_join.reference @@ -16,24 +16,24 @@ ┌─x──────┬─name─┐ │ system │ one │ └────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ +┌─database─┬─t.name─┐ +│ system │ one │ +└──────────┴────────┘ ┌─db.x───┬─name─┐ │ system │ one │ └────────┴──────┘ -┌─db.name─┬─name─┐ -│ system │ one │ -└─────────┴──────┘ -┌─db.name─┬─name─┐ -│ system │ one │ -└─────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ +┌─db.name─┬─t.name─┐ +│ system │ one │ +└─────────┴────────┘ +┌─db.name─┬─t.name─┐ +│ system │ one │ +└─────────┴────────┘ +┌─t.database─┬─t.name─┐ +│ system │ one │ +└────────────┴────────┘ +┌─database─┬─t.name─┐ +│ system │ one │ +└──────────┴────────┘ 2 2 2 diff --git a/tests/queries/0_stateless/00722_inner_join.sql b/tests/queries/0_stateless/00722_inner_join.sql index eee164ed1e1..75ef40ff2b7 100644 --- a/tests/queries/0_stateless/00722_inner_join.sql +++ b/tests/queries/0_stateless/00722_inner_join.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS one; CREATE TABLE one(dummy UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/00818_inner_join_bug_3567.reference b/tests/queries/0_stateless/00818_inner_join_bug_3567.reference index 173e53ebc34..41e0d8ea43a 100644 --- a/tests/queries/0_stateless/00818_inner_join_bug_3567.reference +++ b/tests/queries/0_stateless/00818_inner_join_bug_3567.reference @@ -3,17 +3,17 @@ │ b │ 2018-01-01 │ │ c │ 2018-01-01 │ └───┴────────────┘ -┌─c─┬─a─┬──────────d─┬─a─┬──────────b─┐ -│ B │ b │ 2018-01-01 │ B │ 2018-01-01 │ -│ C │ c │ 2018-01-01 │ C │ 2018-01-01 │ -│ D │ d │ 2018-01-01 │ D │ 2018-01-01 │ -└───┴───┴────────────┴───┴────────────┘ -┌─a─┬──────────b─┬─c─┬──────────d─┐ -│ a │ 2018-01-01 │ │ 1970-01-01 │ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ -└───┴────────────┴───┴────────────┘ -┌─a─┬──────────b─┬─c─┬──────────d─┐ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ -└───┴────────────┴───┴────────────┘ +┌─c─┬─table2.a─┬──────────d─┬─a─┬──────────b─┐ +│ B │ b │ 2018-01-01 │ B │ 2018-01-01 │ +│ C │ c │ 2018-01-01 │ C │ 2018-01-01 │ +│ D │ d │ 2018-01-01 │ D │ 2018-01-01 │ +└───┴──────────┴────────────┴───┴────────────┘ +┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ +│ a │ 2018-01-01 │ │ 1970-01-01 │ │ +│ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ +│ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ +└───┴────────────┴───┴────────────┴───┘ +┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ +│ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ +│ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ +└───┴────────────┴───┴────────────┴───┘ diff --git a/tests/queries/0_stateless/00818_inner_join_bug_3567.sql b/tests/queries/0_stateless/00818_inner_join_bug_3567.sql index 1c851d40f47..cc0b63f9def 100644 --- a/tests/queries/0_stateless/00818_inner_join_bug_3567.sql +++ b/tests/queries/0_stateless/00818_inner_join_bug_3567.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference index afbe9855519..b6d689e3e74 100644 --- a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference +++ b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference @@ -1,11 +1,11 @@ -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x -1 x x +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y +1 x y diff --git a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql index 14a7424e634..cdb9e57d17f 100644 --- a/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql +++ b/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql @@ -1,5 +1,6 @@ SET any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; +SET allow_experimental_analyzer = 1; SELECT * FROM (SELECT 1 AS a, 'x' AS b) join (SELECT 1 as a, 'y' as b) using a; SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a; diff --git a/tests/queries/0_stateless/00820_multiple_joins.reference b/tests/queries/0_stateless/00820_multiple_joins.reference index f9d9a664784..3d7054bacfc 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.reference +++ b/tests/queries/0_stateless/00820_multiple_joins.reference @@ -32,13 +32,13 @@ 6 6 60 60 12 12 120 120 18 18 180 180 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┐ +┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 6 │ 6 │ 60 │ 60 │ 600 │ │ 12 │ 12 │ 120 │ 120 │ 1200 │ │ 18 │ 18 │ 180 │ 180 │ 1800 │ └──────┴──────┴──────┴──────┴──────┘ -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┐ +┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 6 │ 6 │ 60 │ 60 │ 600 │ │ 12 │ 12 │ 120 │ 120 │ 1200 │ diff --git a/tests/queries/0_stateless/00820_multiple_joins.sql b/tests/queries/0_stateless/00820_multiple_joins.sql index af13e6f1d8b..5c7a7bebb0b 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.sql +++ b/tests/queries/0_stateless/00820_multiple_joins.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference index 6d317230813..11755d6bc8b 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference @@ -32,13 +32,13 @@ 6 6 60 60 12 12 120 120 18 18 180 180 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┐ +┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 6 │ 6 │ 60 │ 60 │ 600 │ │ 12 │ 12 │ 120 │ 120 │ 1200 │ │ 18 │ 18 │ 180 │ 180 │ 1800 │ └──────┴──────┴──────┴──────┴──────┘ -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┐ +┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ │ 0 │ 0 │ 0 │ 0 │ 0 │ │ 6 │ 6 │ 60 │ 60 │ 600 │ │ 12 │ 12 │ 120 │ 120 │ 1200 │ diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql index 612e75e6d4d..3da2cad4eff 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; DROP TABLE IF EXISTS table3; diff --git a/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh b/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh index 41a35d908d1..b013993059f 100755 --- a/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh +++ b/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "select c23ount(*) from system.functions;" 2>&1 | grep "Ma $CLICKHOUSE_CLIENT -q "select cunt(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['count'" &>/dev/null; $CLICKHOUSE_CLIENT -q "select positin(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['position'" &>/dev/null; $CLICKHOUSE_CLIENT -q "select POSITIO(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['position'" &>/dev/null; -$CLICKHOUSE_CLIENT -q "select fount(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['count'" | grep "Maybe you meant: \['round'" | grep "Or unknown aggregate function" &>/dev/null; +$CLICKHOUSE_CLIENT -q "select fount(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['count'" &>/dev/null; $CLICKHOUSE_CLIENT -q "select positin(*) from system.functions;" 2>&1 | grep -v "Or unknown aggregate function" &>/dev/null; $CLICKHOUSE_CLIENT -q "select pov(*) from system.functions;" 2>&1 | grep "Maybe you meant: \['pow'\]" &>/dev/null; $CLICKHOUSE_CLIENT -q "select getColumnStructure('abc');" 2>&1 | grep "Maybe you meant: \['dumpColumnStructure'\]" &>/dev/null; diff --git a/tests/queries/0_stateless/00835_if_generic_case.reference b/tests/queries/0_stateless/00835_if_generic_case.reference index 45ee4651e17..84c036b17ca 100644 --- a/tests/queries/0_stateless/00835_if_generic_case.reference +++ b/tests/queries/0_stateless/00835_if_generic_case.reference @@ -2,7 +2,7 @@ 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 -2000-01-01 00:00:00 2000-01-02 2000-01-02 +2000-01-01 00:00:00 2000-01-02 2000-01-02 00:00:00 2000-01-01 00:00:00 2000-01-02 2000-01-02 2000-01-01 00:00:00 2000-01-02 2000-01-02 2000-01-01 00:00:00 2000-01-02 2000-01-02 diff --git a/tests/queries/0_stateless/00835_if_generic_case.sql b/tests/queries/0_stateless/00835_if_generic_case.sql index 63baffcf17d..3d7f128f4c1 100644 --- a/tests/queries/0_stateless/00835_if_generic_case.sql +++ b/tests/queries/0_stateless/00835_if_generic_case.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + SELECT toDateTime('2000-01-01 00:00:00', 'Asia/Istanbul') AS x, toDate('2000-01-02') AS y, x > y ? x : y AS z; SELECT materialize(toDateTime('2000-01-01 00:00:00', 'Asia/Istanbul')) AS x, toDate('2000-01-02') AS y, x > y ? x : y AS z; SELECT toDateTime('2000-01-01 00:00:00', 'Asia/Istanbul') AS x, materialize(toDate('2000-01-02')) AS y, x > y ? x : y AS z; diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index cf9aac0e4da..9b8688c8415 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -1,8 +1,10 @@ +SET allow_experimental_analyzer = 1; + SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (arrayJoin([1]) = B.b); -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (A.a = arrayJoin([1])); -- { serverError 403 } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError 62 } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError 62 } +SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError 42 } +SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError 42 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError 403 } diff --git a/tests/queries/0_stateless/01508_explain_header.reference b/tests/queries/0_stateless/01508_explain_header.reference index 7510e67c643..206bd4afdab 100644 --- a/tests/queries/0_stateless/01508_explain_header.reference +++ b/tests/queries/0_stateless/01508_explain_header.reference @@ -1,4 +1,4 @@ -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + (Projection + Change column names to column identifiers))) Header: x UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/01527_bad_aggregation_in_lambda.sql b/tests/queries/0_stateless/01527_bad_aggregation_in_lambda.sql index 10ffca22bd1..e16f6396a2c 100644 --- a/tests/queries/0_stateless/01527_bad_aggregation_in_lambda.sql +++ b/tests/queries/0_stateless/01527_bad_aggregation_in_lambda.sql @@ -1 +1,3 @@ -SELECT arrayMap(x -> x * sum(x), range(10)); -- { serverError 47 } +SET allow_experimental_analyzer = 1; + +SELECT arrayMap(x -> x * sum(x), range(10)); -- { serverError 10 } From 8665077b5ce8f4590b41848aef5698358d1fd95c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 21:15:30 +0100 Subject: [PATCH 158/470] Planner add logging --- src/Planner/Planner.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index f0fe44e368f..fff539f59d6 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1012,6 +1012,11 @@ void Planner::buildQueryPlanIfNeeded() if (query_plan.isInitialized()) return; + LOG_TRACE(&Poco::Logger::get("Planner"), "Query {} to stage {}{}", + query_tree->formatConvertedASTForErrorMessage(), + QueryProcessingStage::toString(select_query_options.to_stage), + select_query_options.only_analyze ? " only analyze" : ""); + if (query_tree->getNodeType() == QueryTreeNodeType::UNION) buildPlanForUnionNode(); else @@ -1200,6 +1205,12 @@ void Planner::buildPlanForQueryNode() query_plan = std::move(join_tree_query_plan.query_plan); } + LOG_TRACE(&Poco::Logger::get("Planner"), "Query {} from stage {} to stage {}{}", + query_tree->formatConvertedASTForErrorMessage(), + QueryProcessingStage::toString(from_stage), + QueryProcessingStage::toString(select_query_options.to_stage), + select_query_options.only_analyze ? " only analyze" : ""); + if (select_query_options.to_stage == QueryProcessingStage::FetchColumns) return; From 54b704254d0063050241016731dcb8d2b62da986 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 21:20:49 +0100 Subject: [PATCH 159/470] Analyzer support InterpreterCreateQuery --- src/Interpreters/InterpreterCreateQuery.cpp | 31 +++++++++++++++++++-- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 559fc3532f5..29b7a4db609 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -730,7 +731,21 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } else if (create.select) { - Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), getContext(), false /* is_subquery */, create.isParameterizedView()); + + Block as_select_sample; + + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + as_select_sample = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), + getContext(), + false /* is_subquery */, + create.isParameterizedView()); + } + properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); } else if (create.as_table_function) @@ -1184,8 +1199,18 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) getContext() )) { - Block input_block = InterpreterSelectWithUnionQuery( - create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); + 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(); From 1b946b314ecf8c710ac80e21236b3ee8c50af17d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 21:31:21 +0100 Subject: [PATCH 160/470] Analyzer convert LIMIT OFFSET expressions --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 45d8154f39b..2cbdab81c73 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1156,7 +1156,7 @@ private: static void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope); - static void validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); + static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope); static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); @@ -2091,7 +2091,7 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ } } -void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) +void QueryAnalyzer::convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope) { const auto * limit_offset_constant_node = expression_node->as(); if (!limit_offset_constant_node || !isNativeNumber(removeNullable(limit_offset_constant_node->getResultType()))) @@ -2101,11 +2101,17 @@ void QueryAnalyzer::validateLimitOffsetExpression(QueryTreeNodePtr & expression_ expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - Field converted = convertFieldToType(limit_offset_constant_node->getValue(), DataTypeUInt64()); - if (converted.isNull()) + Field converted_value = convertFieldToType(limit_offset_constant_node->getValue(), DataTypeUInt64()); + if (converted_value.isNull()) throw Exception(ErrorCodes::INVALID_LIMIT_EXPRESSION, "{} numeric constant expression is not representable as UInt64", expression_description); + + auto constant_value = std::make_shared(std::move(converted_value), std::make_shared()); + auto result_constant_node = std::make_shared(std::move(constant_value)); + result_constant_node->getSourceExpression() = limit_offset_constant_node->getSourceExpression(); + + expression_node = std::move(result_constant_node); } void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope) @@ -6473,13 +6479,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasLimitByLimit()) { resolveExpressionNode(query_node_typed.getLimitByLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - validateLimitOffsetExpression(query_node_typed.getLimitByLimit(), "LIMIT BY LIMIT", scope); + convertLimitOffsetExpression(query_node_typed.getLimitByLimit(), "LIMIT BY LIMIT", scope); } if (query_node_typed.hasLimitByOffset()) { resolveExpressionNode(query_node_typed.getLimitByOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - validateLimitOffsetExpression(query_node_typed.getLimitByOffset(), "LIMIT BY OFFSET", scope); + convertLimitOffsetExpression(query_node_typed.getLimitByOffset(), "LIMIT BY OFFSET", scope); } if (query_node_typed.hasLimitBy()) @@ -6493,13 +6499,13 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier if (query_node_typed.hasLimit()) { resolveExpressionNode(query_node_typed.getLimit(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - validateLimitOffsetExpression(query_node_typed.getLimit(), "LIMIT", scope); + convertLimitOffsetExpression(query_node_typed.getLimit(), "LIMIT", scope); } if (query_node_typed.hasOffset()) { resolveExpressionNode(query_node_typed.getOffset(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - validateLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); + convertLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); } /** Resolve nodes with duplicate aliases. From 2672f24ced7b6a2c4754990524bab31f04a97187 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 19 Feb 2023 21:36:59 +0100 Subject: [PATCH 161/470] Analyzer cyclic aliases fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 50 +++++++++++++------ .../00818_alias_bug_4110.reference | 1 + .../0_stateless/00818_alias_bug_4110.sql | 10 ++-- 3 files changed, 41 insertions(+), 20 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2cbdab81c73..c97b167348b 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -361,6 +361,12 @@ struct IdentifierResolveResult } }; +struct IdentifierResolveState +{ + IdentifierResolveResult resolve_result; + bool cyclic_identifier_resolve = false; +}; + struct IdentifierResolveSettings { /// Allow to check join tree during identifier resolution @@ -687,7 +693,7 @@ struct IdentifierResolveScope ContextPtr context; /// Identifier lookup to result - std::unordered_map identifier_lookup_to_result; + std::unordered_map identifier_lookup_to_resolve_state; /// Lambda argument can be expression like constant, column, or it can be function std::unordered_map expression_argument_name_to_node; @@ -799,11 +805,11 @@ struct IdentifierResolveScope [[maybe_unused]] void dump(WriteBuffer & buffer) const { buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n'; - buffer << "Identifier lookup to result " << identifier_lookup_to_result.size() << '\n'; - for (const auto & [identifier, result] : identifier_lookup_to_result) + buffer << "Identifier lookup to resolve state " << identifier_lookup_to_resolve_state.size() << '\n'; + for (const auto & [identifier, state] : identifier_lookup_to_resolve_state) { buffer << "Identifier " << identifier.dump() << " resolve result "; - result.dump(buffer); + state.resolve_result.dump(buffer); buffer << '\n'; } @@ -3316,21 +3322,28 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { - auto it = scope.identifier_lookup_to_result.find(identifier_lookup); - if (it != scope.identifier_lookup_to_result.end()) + auto it = scope.identifier_lookup_to_resolve_state.find(identifier_lookup); + if (it != scope.identifier_lookup_to_resolve_state.end()) { - if (!it->second.resolved_identifier) + if (it->second.cyclic_identifier_resolve) throw Exception(ErrorCodes::CYCLIC_ALIASES, "Cyclic aliases for identifier '{}'. In scope {}", identifier_lookup.identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); - if (scope.use_identifier_lookup_to_result_cache && !scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) - return it->second; - } + if (!it->second.resolve_result.isResolved()) + it->second.cyclic_identifier_resolve = true; - auto [insert_it, _] = scope.identifier_lookup_to_result.insert({identifier_lookup, IdentifierResolveResult()}); - it = insert_it; + if (it->second.resolve_result.isResolved() && + scope.use_identifier_lookup_to_result_cache && + !scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup)) + return it->second.resolve_result; + } + else + { + auto [insert_it, _] = scope.identifier_lookup_to_resolve_state.insert({identifier_lookup, IdentifierResolveState()}); + it = insert_it; + } /// Resolve identifier from current scope @@ -3409,15 +3422,18 @@ IdentifierResolveResult QueryAnalyzer::tryResolveIdentifier(const IdentifierLook resolve_result.resolve_place = IdentifierResolvePlace::DATABASE_CATALOG; } - it->second = resolve_result; + bool was_cyclic_identifier_resolve = it->second.cyclic_identifier_resolve; + if (!was_cyclic_identifier_resolve) + it->second.resolve_result = resolve_result; + it->second.cyclic_identifier_resolve = false; /** If identifier was not resolved, or during expression resolution identifier was explicitly added into non cached set, * or identifier caching was disabled in resolve scope we remove identifier lookup result from identifier lookup to result table. */ - if (!resolve_result.resolved_identifier || + if (!was_cyclic_identifier_resolve && (!resolve_result.resolved_identifier || scope.non_cached_identifier_lookups_during_expression_resolve.contains(identifier_lookup) || - !scope.use_identifier_lookup_to_result_cache) - scope.identifier_lookup_to_result.erase(it); + !scope.use_identifier_lookup_to_result_cache)) + scope.identifier_lookup_to_resolve_state.erase(it); return resolve_result; } @@ -4987,6 +5003,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi */ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression) { + checkStackSize(); + auto resolved_expression_it = resolved_expressions.find(node); if (resolved_expression_it != resolved_expressions.end()) { diff --git a/tests/queries/0_stateless/00818_alias_bug_4110.reference b/tests/queries/0_stateless/00818_alias_bug_4110.reference index 210fc67db66..e6013d269c2 100644 --- a/tests/queries/0_stateless/00818_alias_bug_4110.reference +++ b/tests/queries/0_stateless/00818_alias_bug_4110.reference @@ -4,6 +4,7 @@ 11 12 12 11 10 10 +10 11 11 12 11 10 12 11 12 diff --git a/tests/queries/0_stateless/00818_alias_bug_4110.sql b/tests/queries/0_stateless/00818_alias_bug_4110.sql index df7e70cb275..9f3657221e4 100644 --- a/tests/queries/0_stateless/00818_alias_bug_4110.sql +++ b/tests/queries/0_stateless/00818_alias_bug_4110.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + select s.a as a, s.a + 1 as b from (select 10 as a) s; select s.a + 1 as a, s.a as b from (select 10 as a) s; select s.a + 1 as a, s.a + 1 as b from (select 10 as a) s; @@ -5,15 +7,15 @@ select s.a + 1 as b, s.a + 2 as a from (select 10 as a) s; select s.a + 2 as b, s.a + 1 as a from (select 10 as a) s; select a, a as a from (select 10 as a); -select s.a, a, a + 1 as a from (select 10 as a) as s; -- { serverError 352 } +select s.a, a, a + 1 as a from (select 10 as a) as s; select s.a + 2 as b, b - 1 as a from (select 10 as a) s; select s.a as a, s.a + 2 as b from (select 10 as a) s; select s.a + 1 as a, s.a + 2 as b from (select 10 as a) s; select a + 1 as a, a + 1 as b from (select 10 as a); select a + 1 as b, b + 1 as a from (select 10 as a); -- { serverError 174 } -select 10 as a, a + 1 as a; -- { serverError 179 } -with 10 as a select a as a; -- { serverError 179 } -with 10 as a select a + 1 as a; -- { serverError 179 } +select 10 as a, a + 1 as a; -- { serverError 47 } +with 10 as a select a as a; -- { serverError 47 } +with 10 as a select a + 1 as a; -- { serverError 47 } SELECT 0 as t FROM (SELECT 1 as t) as inn WHERE inn.t = 1; SELECT sum(value) as value FROM (SELECT 1 as value) as data WHERE data.value > 0; From 40fc36bb6f7a04524368aa445e78d9742e24195a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Feb 2023 19:26:39 +0100 Subject: [PATCH 162/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Planner/PlannerJoinTree.cpp | 3 +- .../00800_low_cardinality_join.sql | 3 +- .../00800_low_cardinality_merge_join.sql.j2 | 4 ++- .../00834_limit_with_constant_expressions.sql | 4 ++- .../01000_subquery_requires_alias.reference | 4 +-- .../01000_subquery_requires_alias.sql | 1 + .../01009_insert_select_data_loss.sql | 2 +- .../01010_pm_join_all_join_bug.sql | 4 ++- .../queries/0_stateless/01010_pmj_on_disk.sql | 12 +++---- .../01064_pm_all_join_const_and_nullable.sql | 32 +++++++++---------- .../0_stateless/01095_tpch_like_smoke.sql | 8 +++-- .../0_stateless/01120_join_constants.sql | 4 +-- .../01129_dict_get_join_lose_constness.sql | 4 +-- .../0_stateless/01223_dist_on_dist.sql | 4 +-- .../0_stateless/01232_untuple.reference | 2 +- tests/queries/0_stateless/01232_untuple.sql | 2 ++ .../0_stateless/01474_bad_global_join.sql | 4 ++- .../01671_merge_join_and_constants.reference | 10 +++--- .../01671_merge_join_and_constants.sql | 2 ++ .../01890_cross_join_explain_crash.reference | 1 + .../01890_cross_join_explain_crash.sql | 5 ++- ...obal_subqueries_subquery_or_table_name.sql | 4 ++- .../0_stateless/02116_tuple_element.sql | 6 ++-- .../02136_scalar_read_rows_json.reference | 2 +- .../02136_scalar_read_rows_json.sh | 2 +- .../02154_dictionary_get_http_json.reference | 8 ++--- .../02154_dictionary_get_http_json.sh | 1 + tests/queries/0_stateless/02163_operators.sql | 2 +- .../02234_clickhouse_local_test_mode.sh | 2 +- ..._column_matcher_and_column_transformer.sql | 4 ++- ...37_multiple_joins_original_names.reference | 1 + .../02337_multiple_joins_original_names.sql | 4 ++- ...497_if_transform_strings_to_enum.reference | 24 +++++++------- 34 files changed, 102 insertions(+), 75 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c97b167348b..0e47aec7c20 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -3647,7 +3647,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveQualifiedMatcher(Qu if (!table_expression_node) { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Qualified matcher {} does not find table. In scope {}", matcher_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f2259ca7e33..02c396542b1 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -533,7 +533,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (auto & output_node : cast_actions_dag->getOutputs()) { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name) && + output_node->result_type->canBeInsideNullable()) output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); } diff --git a/tests/queries/0_stateless/00800_low_cardinality_join.sql b/tests/queries/0_stateless/00800_low_cardinality_join.sql index 30fb40a56e9..9c1fd9b7ad3 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -1,3 +1,4 @@ +set allow_experimental_analyzer = 1; set joined_subquery_requires_alias = 0; select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) using val; @@ -10,7 +11,7 @@ select * from (select toLowCardinality(toNullable(dummy)) as val from system.one select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; select '-'; -select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError 352 } +select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError 403 } select * from (select dummy as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1; select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1; select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1; diff --git a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 index 8b7856b7738..8e2037480c7 100644 --- a/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 +++ b/tests/queries/0_stateless/00800_low_cardinality_merge_join.sql.j2 @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + {% for join_algorithm in ['partial_merge', 'full_sorting_merge', 'grace_hash'] -%} SET join_algorithm = '{{ join_algorithm }}'; @@ -12,7 +14,7 @@ SELECT * FROM (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one SELECT * FROM (SELECT toLowCardinality(dummy) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s2 USING val ORDER BY val; SELECT * FROM (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(toNullable(dummy)) AS val FROM system.one) s2 USING val ORDER BY val; SELECT '-'; -SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS val FROM system.one) s2 ON val + 0 = val * 1 ORDER BY val; -- { serverError 352 } +SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS val FROM system.one) s2 ON val + 0 = val * 1 ORDER BY val; -- { serverError 403 } SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val; SELECT * FROM (SELECT toLowCardinality(dummy) AS val FROM system.one) s1 ANY LEFT JOIN (SELECT dummy AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val; SELECT * FROM (SELECT dummy AS val FROM system.one) s1 ANY LEFT JOIN (SELECT toLowCardinality(dummy) AS rval FROM system.one) s2 ON val + 0 = rval * 1 ORDER BY val; diff --git a/tests/queries/0_stateless/00834_limit_with_constant_expressions.sql b/tests/queries/0_stateless/00834_limit_with_constant_expressions.sql index 12104145237..1d3429436e7 100644 --- a/tests/queries/0_stateless/00834_limit_with_constant_expressions.sql +++ b/tests/queries/0_stateless/00834_limit_with_constant_expressions.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + SELECT number FROM numbers(10) LIMIT 0 + 1; SELECT number FROM numbers(10) LIMIT 1 - 1; SELECT number FROM numbers(10) LIMIT 2 - 1; @@ -9,7 +11,7 @@ SELECT number FROM numbers(10) LIMIT now(); -- { serverError 440 } SELECT number FROM numbers(10) LIMIT today(); -- { serverError 440 } SELECT number FROM numbers(10) LIMIT toUInt8('1'); SELECT number FROM numbers(10) LIMIT toFloat32('1'); -SELECT number FROM numbers(10) LIMIT rand(); -- { serverError 36 } +SELECT number FROM numbers(10) LIMIT rand(); -- { serverError 440 } SELECT count() <= 1 FROM (SELECT number FROM numbers(10) LIMIT randConstant() % 2); diff --git a/tests/queries/0_stateless/01000_subquery_requires_alias.reference b/tests/queries/0_stateless/01000_subquery_requires_alias.reference index 8018af28430..b6ceb2a2a72 100644 --- a/tests/queries/0_stateless/01000_subquery_requires_alias.reference +++ b/tests/queries/0_stateless/01000_subquery_requires_alias.reference @@ -1,2 +1,2 @@ -1 2 3 -1 2 3 +2 1 3 +2 1 3 diff --git a/tests/queries/0_stateless/01000_subquery_requires_alias.sql b/tests/queries/0_stateless/01000_subquery_requires_alias.sql index d14d4637be8..27320fab933 100644 --- a/tests/queries/0_stateless/01000_subquery_requires_alias.sql +++ b/tests/queries/0_stateless/01000_subquery_requires_alias.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET joined_subquery_requires_alias = 1; SELECT * FROM (SELECT 1 as A, 2 as B) X diff --git a/tests/queries/0_stateless/01009_insert_select_data_loss.sql b/tests/queries/0_stateless/01009_insert_select_data_loss.sql index a153d7dbc30..cbf87c4d0f1 100644 --- a/tests/queries/0_stateless/01009_insert_select_data_loss.sql +++ b/tests/queries/0_stateless/01009_insert_select_data_loss.sql @@ -1,7 +1,7 @@ drop table if exists tab; create table tab (x UInt64) engine = MergeTree order by tuple(); -insert into tab select number as n from numbers(20) nums +insert into tab select n from (SELECT number AS n FROM numbers(20)) nums semi left join (select number * 10 as n from numbers(2)) js2 using(n) settings max_block_size = 5; select * from tab order by x; diff --git a/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql b/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql index 18a67f41194..d192b9a8b64 100644 --- a/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql +++ b/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS ints; CREATE TABLE ints (i64 Int64, i32 Int32) ENGINE = Memory; @@ -10,6 +12,6 @@ SELECT '-'; SELECT * FROM ints l INNER JOIN ints r USING i64 ORDER BY l.i32, r.i32; SELECT '-'; -SELECT count() FROM ( SELECT [1], count(1) ) AS t1 ALL RIGHT JOIN ( SELECT number AS s FROM numbers(2) ) AS t2 USING (s); -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } +SELECT count() FROM ( SELECT [1], count(1) ) AS t1 ALL RIGHT JOIN ( SELECT number AS s FROM numbers(2) ) AS t2 USING (s); -- { serverError UNKNOWN_IDENTIFIER } DROP TABLE ints; diff --git a/tests/queries/0_stateless/01010_pmj_on_disk.sql b/tests/queries/0_stateless/01010_pmj_on_disk.sql index d4fb9184896..4925f78f82f 100644 --- a/tests/queries/0_stateless/01010_pmj_on_disk.sql +++ b/tests/queries/0_stateless/01010_pmj_on_disk.sql @@ -1,6 +1,6 @@ SET join_algorithm = 'hash'; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number as n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) @@ -10,7 +10,7 @@ ORDER BY n; SET max_rows_in_join = 1000; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) @@ -20,7 +20,7 @@ ORDER BY n; -- { serverError 191 } SET join_algorithm = 'partial_merge'; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number as n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) @@ -30,7 +30,7 @@ ORDER BY n; SET partial_merge_join_optimizations = 1; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) @@ -40,7 +40,7 @@ ORDER BY n; SET join_algorithm = 'auto'; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) @@ -50,7 +50,7 @@ ORDER BY n; SET max_rows_in_join = '10'; -SELECT number as n, j FROM numbers(4) nums +SELECT n, j FROM (SELECT number AS n FROM numbers(4)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number + 10 AS j FROM numbers(4000) diff --git a/tests/queries/0_stateless/01064_pm_all_join_const_and_nullable.sql b/tests/queries/0_stateless/01064_pm_all_join_const_and_nullable.sql index 10306777f25..bc5e1d68290 100644 --- a/tests/queries/0_stateless/01064_pm_all_join_const_and_nullable.sql +++ b/tests/queries/0_stateless/01064_pm_all_join_const_and_nullable.sql @@ -1,87 +1,87 @@ SET join_algorithm = 'partial_merge'; SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(1) AS k FROM numbers(1)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT 1 as k FROM numbers(1) nums +SELECT k FROM (SELECT 1 AS k FROM numbers(1)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT 1 as k FROM numbers(1) nums +SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j USING k); SELECT 'first nullable'; SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(toNullable(1)) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(toNullable(1)) AS k FROM numbers(1)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(toNullable(1)) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT toNullable(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT toNullable(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums JOIN (SELECT 1 AS k, number n FROM numbers(100000)) j USING k); SELECT 'second nullable'; SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(1) as k FROM numbers(1)) nums JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT 1 as k FROM numbers(1) nums +SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT 1 as k FROM numbers(1) nums +SELECT k FROM (SELECT 1 as k FROM numbers(1)) nums JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j USING k); SELECT 'both nullable'; SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(toNullable(1)) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT materialize(toNullable(1)) as k FROM numbers(1) nums +SELECT k FROM (SELECT materialize(toNullable(1)) as k FROM numbers(1)) nums JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT toNullable(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums JOIN (SELECT materialize(toNullable(1)) AS k, number n FROM numbers(100000)) j USING k); SELECT count(1), uniqExact(1) FROM ( -SELECT toNullable(1) as k FROM numbers(1) nums +SELECT k FROM (SELECT toNullable(1) as k FROM numbers(1)) nums JOIN (SELECT toNullable(1) AS k, number n FROM numbers(100000)) j USING k); diff --git a/tests/queries/0_stateless/01095_tpch_like_smoke.sql b/tests/queries/0_stateless/01095_tpch_like_smoke.sql index 1ac9ec229f0..7ab57766605 100644 --- a/tests/queries/0_stateless/01095_tpch_like_smoke.sql +++ b/tests/queries/0_stateless/01095_tpch_like_smoke.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS part; DROP TABLE IF EXISTS supplier; DROP TABLE IF EXISTS partsupp; @@ -180,7 +182,7 @@ order by n_name, s_name, p_partkey -limit 100; -- { serverError 47 } +limit 100; -- { serverError 1 } select 3; select @@ -598,7 +600,7 @@ where lineitem where l_partkey = p_partkey - ); -- { serverError 47 } + ); -- { serverError 1 } select 18; select @@ -709,7 +711,7 @@ where and s_nationkey = n_nationkey and n_name = 'CANADA' order by - s_name; -- { serverError 47 } + s_name; -- { serverError 1 } select 21, 'fail: exists, not exists'; -- TODO -- select diff --git a/tests/queries/0_stateless/01120_join_constants.sql b/tests/queries/0_stateless/01120_join_constants.sql index fdf297f5934..d8c8b5757f8 100644 --- a/tests/queries/0_stateless/01120_join_constants.sql +++ b/tests/queries/0_stateless/01120_join_constants.sql @@ -1,7 +1,7 @@ SELECT t1.*, t2.*, - 'world', + 'world' AS constant, isConstant('world') FROM ( @@ -19,7 +19,7 @@ LEFT JOIN SELECT t1.*, t2.*, - 123, + 123 AS constant, isConstant('world') FROM ( diff --git a/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql b/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql index 4f7197dc5ce..fd3e12f7a15 100644 --- a/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql +++ b/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql @@ -13,9 +13,9 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' PASSWORD '' TAB LIFETIME(0) LAYOUT(hashed()); -SELECT dictGetInt32('system.dict1', 'element_count', toUInt64(dict_key)) AS join_key, +SELECT join_key, toTimeZone(dictGetDateTime('system.dict1', 'loading_start_time', toUInt64(dict_key)), 'UTC') AS datetime -FROM (select 1 AS dict_key) js1 +FROM (select dictGetInt32('system.dict1', 'element_count', toUInt64(dict_key)) AS join_key, 1 AS dict_key) js1 LEFT JOIN (SELECT toInt32(2) AS join_key) js2 USING (join_key) WHERE now() >= datetime; diff --git a/tests/queries/0_stateless/01223_dist_on_dist.sql b/tests/queries/0_stateless/01223_dist_on_dist.sql index 65a240fd48b..4cd8ffd7a03 100644 --- a/tests/queries/0_stateless/01223_dist_on_dist.sql +++ b/tests/queries/0_stateless/01223_dist_on_dist.sql @@ -33,9 +33,9 @@ select 'cluster() GROUP BY ORDER BY'; select * from cluster(test_cluster_two_shards, currentDatabase(), dist_01223) group by key order by key; select 'LEFT JOIN'; -select toInt32(number) key, b.key from numbers(2) a left join (select distinct * from dist_01223) b using key order by b.key; +select a.key, b.key from (SELECT toInt32(number) key from numbers(2)) a left join (select distinct * from dist_01223) b using key order by b.key; select 'RIGHT JOIN'; -select toInt32(number) key, b.key from numbers(2) a right join (select distinct * from dist_01223) b using key order by b.key; +select a.key, b.key from (SELECT toInt32(number) key from numbers(2)) a right join (select distinct * from dist_01223) b using key order by b.key; -- more data for GROUP BY insert into data_01223 select number%3 from numbers(30); diff --git a/tests/queries/0_stateless/01232_untuple.reference b/tests/queries/0_stateless/01232_untuple.reference index 8e1f97d2585..7bb80bf618f 100644 --- a/tests/queries/0_stateless/01232_untuple.reference +++ b/tests/queries/0_stateless/01232_untuple.reference @@ -2,7 +2,7 @@ hello 1 3 world 9 9 (0,1) -key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 1) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 2) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 3) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 4) tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), 5) +key tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'1\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'2\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'3\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'4\') tupleElement(argMax(tuple(v1, v2, v3, v4, v5), v1), \'5\') 1 20 20 10 20 30 2 11 20 10 20 30 3 70 20 10 20 30 diff --git a/tests/queries/0_stateless/01232_untuple.sql b/tests/queries/0_stateless/01232_untuple.sql index 92150e92b29..ccefd13a772 100644 --- a/tests/queries/0_stateless/01232_untuple.sql +++ b/tests/queries/0_stateless/01232_untuple.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + select untuple((* except (b),)) from (select 1 a, 2 b, 3 c); select 'hello', untuple((* except (b),)), 'world' from (select 1 a, 2 b, 3 c); select argMax(untuple(x)) from (select (number, number + 1) as x from numbers(10)); diff --git a/tests/queries/0_stateless/01474_bad_global_join.sql b/tests/queries/0_stateless/01474_bad_global_join.sql index b65b4e804a8..2251c1797cf 100644 --- a/tests/queries/0_stateless/01474_bad_global_join.sql +++ b/tests/queries/0_stateless/01474_bad_global_join.sql @@ -1,5 +1,7 @@ -- Tags: global +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS dist_table; @@ -10,7 +12,7 @@ INSERT INTO local_table SELECT number AS id, toString(number) AS val FROM number CREATE TABLE dist_table AS local_table ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), local_table); -SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN numbers(100) AS t USING id; -- { serverError 284 } +SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN numbers(100) AS t USING id; -- { serverError 47 } SELECT uniq(d.val) FROM dist_table AS d GLOBAL LEFT JOIN local_table AS t USING id; DROP TABLE local_table; diff --git a/tests/queries/0_stateless/01671_merge_join_and_constants.reference b/tests/queries/0_stateless/01671_merge_join_and_constants.reference index efd814df893..f82033ab3c7 100644 --- a/tests/queries/0_stateless/01671_merge_join_and_constants.reference +++ b/tests/queries/0_stateless/01671_merge_join_and_constants.reference @@ -1,6 +1,6 @@ -┌─a─┬──────────b─┬─c─┬──────────d─┬─t2.'0.10'─┐ -│ a │ 2018-01-01 │ │ 1970-01-01 │ │ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10 │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10 │ -└───┴────────────┴───┴────────────┴───────────┘ +┌─a─┬──────────b─┬─c─┬──────────d─┬─'0.10'─┬─c─┐ +│ a │ 2018-01-01 │ │ 1970-01-01 │ │ │ +│ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10 │ B │ +│ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10 │ C │ +└───┴────────────┴───┴────────────┴────────┴───┘ \N \N \N \N 0 0 diff --git a/tests/queries/0_stateless/01671_merge_join_and_constants.sql b/tests/queries/0_stateless/01671_merge_join_and_constants.sql index e9a60f11875..5cabd6f7f06 100644 --- a/tests/queries/0_stateless/01671_merge_join_and_constants.sql +++ b/tests/queries/0_stateless/01671_merge_join_and_constants.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS table1; DROP TABLE IF EXISTS table2; diff --git a/tests/queries/0_stateless/01890_cross_join_explain_crash.reference b/tests/queries/0_stateless/01890_cross_join_explain_crash.reference index 76315843adb..c08ba9c71b3 100644 --- a/tests/queries/0_stateless/01890_cross_join_explain_crash.reference +++ b/tests/queries/0_stateless/01890_cross_join_explain_crash.reference @@ -1,3 +1,4 @@ +1 1 1 2 1 1 1 2 1 1 1 2 diff --git a/tests/queries/0_stateless/01890_cross_join_explain_crash.sql b/tests/queries/0_stateless/01890_cross_join_explain_crash.sql index 20a1956ea6b..bb2bc606870 100644 --- a/tests/queries/0_stateless/01890_cross_join_explain_crash.sql +++ b/tests/queries/0_stateless/01890_cross_join_explain_crash.sql @@ -1,8 +1,7 @@ +SET allow_experimental_analyzer = 1; SET joined_subquery_requires_alias = 0; -select * FROM (SELECT 1), (SELECT 1), (SELECT 1); -- { serverError 352 } --- This queries work by luck. --- Feel free to remove then if it is the only failed test. +select * FROM (SELECT 1), (SELECT 1), (SELECT 1); select * from (select 2), (select 1) as a, (select 1) as b; select * from (select 1) as a, (select 2), (select 1) as b; select * from (select 1) as a, (select 1) as b, (select 2); diff --git a/tests/queries/0_stateless/02002_global_subqueries_subquery_or_table_name.sql b/tests/queries/0_stateless/02002_global_subqueries_subquery_or_table_name.sql index 7d9c08c7cb6..75eff67970e 100644 --- a/tests/queries/0_stateless/02002_global_subqueries_subquery_or_table_name.sql +++ b/tests/queries/0_stateless/02002_global_subqueries_subquery_or_table_name.sql @@ -1,7 +1,9 @@ -- Tags: global +SET allow_experimental_analyzer = 1; + SELECT cityHash64(number GLOBAL IN (NULL, -2147483648, -9223372036854775808), nan, 1024, NULL, NULL, 1.000100016593933, NULL), (NULL, cityHash64(inf, -2147483648, NULL, NULL, 10.000100135803223), cityHash64(1.1754943508222875e-38, NULL, NULL, NULL), 2147483647) FROM cluster(test_cluster_two_shards_localhost, numbers((NULL, cityHash64(0., 65536, NULL, NULL, 10000000000., NULL), 0) GLOBAL IN (some_identifier), 65536)) -WHERE number GLOBAL IN [1025] --{serverError 284} +WHERE number GLOBAL IN [1025] --{serverError 36} diff --git a/tests/queries/0_stateless/02116_tuple_element.sql b/tests/queries/0_stateless/02116_tuple_element.sql index 4ce6e5cf136..b43c9133e6b 100644 --- a/tests/queries/0_stateless/02116_tuple_element.sql +++ b/tests/queries/0_stateless/02116_tuple_element.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS t_tuple_element; CREATE TABLE t_tuple_element(t1 Tuple(a UInt32, s String), t2 Tuple(UInt32, String)) ENGINE = Memory; @@ -16,7 +18,7 @@ EXPLAIN SYNTAX SELECT tupleElement(t1, 'a') FROM t_tuple_element; SELECT tupleElement(number, 1) FROM numbers(1); -- { serverError 43 } SELECT tupleElement(t1) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 47 } +SELECT tupleElement(t1, 'b') FROM t_tuple_element; -- { serverError 10 } SELECT tupleElement(t1, 0) FROM t_tuple_element; -- { serverError 127 } SELECT tupleElement(t1, 3) FROM t_tuple_element; -- { serverError 127 } SELECT tupleElement(t1, materialize('a')) FROM t_tuple_element; -- { serverError 43 } @@ -28,7 +30,7 @@ SELECT tupleElement(t2, 1) FROM t_tuple_element; EXPLAIN SYNTAX SELECT tupleElement(t2, 1) FROM t_tuple_element; SELECT tupleElement(t2) FROM t_tuple_element; -- { serverError 42 } -SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 47 } +SELECT tupleElement(t2, 'a') FROM t_tuple_element; -- { serverError 10 } SELECT tupleElement(t2, 0) FROM t_tuple_element; -- { serverError 127 } SELECT tupleElement(t2, 3) FROM t_tuple_element; -- { serverError 127 } SELECT tupleElement(t2, materialize(1)) FROM t_tuple_element; -- { serverError 43 } diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.reference b/tests/queries/0_stateless/02136_scalar_read_rows_json.reference index 49020a4432f..f8335125233 100644 --- a/tests/queries/0_stateless/02136_scalar_read_rows_json.reference +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.reference @@ -30,7 +30,7 @@ "meta": [ { - "type": "Tuple(UInt64, UInt64)" + "type": "Tuple(`max(number)` UInt64, `count(number)` UInt64)" } ], diff --git a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh index 34b4b6909b5..1fe345d266d 100755 --- a/tests/queries/0_stateless/02136_scalar_read_rows_json.sh +++ b/tests/queries/0_stateless/02136_scalar_read_rows_json.sh @@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "#1" ${CLICKHOUSE_CLIENT} --query='SELECT count() FROM numbers(100) FORMAT JSON;' | grep -a -v "elapsed" echo "#2" -${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" +${CLICKHOUSE_CLIENT} --query='SELECT (SELECT max(number), count(number) FROM numbers(100000) as n) SETTINGS max_block_size = 65505, allow_experimental_analyzer = 1 FORMAT JSON;' | grep -a -v "elapsed" | grep -v "_subquery" diff --git a/tests/queries/0_stateless/02154_dictionary_get_http_json.reference b/tests/queries/0_stateless/02154_dictionary_get_http_json.reference index 7106f551cd7..201d3b122e8 100644 --- a/tests/queries/0_stateless/02154_dictionary_get_http_json.reference +++ b/tests/queries/0_stateless/02154_dictionary_get_http_json.reference @@ -3,11 +3,11 @@ "meta": [ { - "name": "dictGet(02154_test_dictionary, 'value', toUInt64(0))", + "name": "dictGet('02154_test_dictionary', 'value', toUInt64(0))", "type": "String" }, { - "name": "dictGet(02154_test_dictionary, 'value', toUInt64(1))", + "name": "dictGet('02154_test_dictionary', 'value', toUInt64(1))", "type": "String" } ], @@ -15,8 +15,8 @@ "data": [ { - "dictGet(02154_test_dictionary, 'value', toUInt64(0))": "Value", - "dictGet(02154_test_dictionary, 'value', toUInt64(1))": "" + "dictGet('02154_test_dictionary', 'value', toUInt64(0))": "Value", + "dictGet('02154_test_dictionary', 'value', toUInt64(1))": "" } ], diff --git a/tests/queries/0_stateless/02154_dictionary_get_http_json.sh b/tests/queries/0_stateless/02154_dictionary_get_http_json.sh index a2bce866c76..fbaf67fff2f 100755 --- a/tests/queries/0_stateless/02154_dictionary_get_http_json.sh +++ b/tests/queries/0_stateless/02154_dictionary_get_http_json.sh @@ -32,6 +32,7 @@ $CLICKHOUSE_CLIENT -q """ echo """ SELECT dictGet(02154_test_dictionary, 'value', toUInt64(0)), dictGet(02154_test_dictionary, 'value', toUInt64(1)) + SETTINGS allow_experimental_analyzer = 1 FORMAT JSON """ | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&wait_end_of_query=1&output_format_write_statistics=0" -d @- diff --git a/tests/queries/0_stateless/02163_operators.sql b/tests/queries/0_stateless/02163_operators.sql index 4968e448ab2..b2414bb197e 100644 --- a/tests/queries/0_stateless/02163_operators.sql +++ b/tests/queries/0_stateless/02163_operators.sql @@ -1,2 +1,2 @@ WITH 2 AS `b.c`, [4, 5] AS a, 6 AS u, 3 AS v, 2 AS d, TRUE AS e, 1 AS f, 0 AS g, 2 AS h, 'Hello' AS i, 'World' AS j, TIMESTAMP '2022-02-02 02:02:02' AS w, [] AS k, (1, 2) AS l, 2 AS m, 3 AS n, [] AS o, [1] AS p, 1 AS q, q AS r, 1 AS s, 1 AS t -SELECT INTERVAL CASE CASE WHEN NOT -a[b.c] * u DIV v + d IS NOT NULL AND e OR f BETWEEN g AND h THEN i ELSE j END WHEN w THEN k END || [l, (m, n)] MINUTE IS NULL OR NOT o::Array(INT) = p <> q < r > s != t AS upyachka; +SELECT INTERVAL CASE CASE WHEN NOT -a[`b.c`] * u DIV v + d IS NOT NULL AND e OR f BETWEEN g AND h THEN i ELSE j END WHEN w THEN k END || [l, (m, n)] MINUTE IS NULL OR NOT o::Array(INT) = p <> q < r > s != t AS upyachka; diff --git a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh index f736751726d..16200399099 100755 --- a/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh +++ b/tests/queries/0_stateless/02234_clickhouse_local_test_mode.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --query="SELECT n" 2>&1 | grep -q "Code: 47. DB::Exception: Missing columns:" && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_LOCAL --query="SELECT n SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "Code: 47. DB::Exception:" && echo 'OK' || echo 'FAIL' ||: $CLICKHOUSE_LOCAL --query="SELECT n -- { serverError 47 }" diff --git a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql index f0c0e2bae46..245b2cc97e3 100644 --- a/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql +++ b/tests/queries/0_stateless/02271_fix_column_matcher_and_column_transformer.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS github_events; CREATE TABLE github_events @@ -59,6 +61,6 @@ CREATE TABLE github_events ) ENGINE = MergeTree ORDER BY (event_type, repo_name, created_at); -with top_repos as ( select repo_name from github_events where event_type = 'WatchEvent' and toDate(created_at) = today() - 1 group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toMonday(created_at) = toMonday(today() - interval 1 week) group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toYear(created_at) = toYear(today()) - 1 group by repo_name order by count() desc limit 100 ), last_day as ( select repo_name, count() as count_last_day, rowNumberInAllBlocks() + 1 as position_last_day from github_events where repo_name in (select repo_name from top_repos) and toDate(created_at) = today() - 1 group by repo_name order by count_last_day desc ), last_week as ( select repo_name, count() as count_last_week, rowNumberInAllBlocks() + 1 as position_last_week from github_events where repo_name in (select repo_name from top_repos) and toMonday(created_at) = toMonday(today()) - interval 1 week group by repo_name order by count_last_week desc ), last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc ) select d.repo_name, columns(count) from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name FORMAT TabSeparatedWithNamesAndTypes; -- { serverError 47 } +with top_repos as ( select repo_name from github_events where event_type = 'WatchEvent' and toDate(created_at) = today() - 1 group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toMonday(created_at) = toMonday(today() - interval 1 week) group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count() desc limit 100 union distinct select repo_name from github_events where event_type = 'WatchEvent' and toYear(created_at) = toYear(today()) - 1 group by repo_name order by count() desc limit 100 ), last_day as ( select repo_name, count() as count_last_day, rowNumberInAllBlocks() + 1 as position_last_day from github_events where repo_name in (select repo_name from top_repos) and toDate(created_at) = today() - 1 group by repo_name order by count_last_day desc ), last_week as ( select repo_name, count() as count_last_week, rowNumberInAllBlocks() + 1 as position_last_week from github_events where repo_name in (select repo_name from top_repos) and toMonday(created_at) = toMonday(today()) - interval 1 week group by repo_name order by count_last_week desc ), last_month as ( select repo_name, count() as count_last_month, rowNumberInAllBlocks() + 1 as position_last_month from github_events where repo_name in (select repo_name from top_repos) and toStartOfMonth(created_at) = toStartOfMonth(today()) - interval 1 month group by repo_name order by count_last_month desc ) select d.repo_name, columns(count) from last_day d join last_week w on d.repo_name = w.repo_name join last_month m on d.repo_name = m.repo_name; DROP TABLE github_events; diff --git a/tests/queries/0_stateless/02337_multiple_joins_original_names.reference b/tests/queries/0_stateless/02337_multiple_joins_original_names.reference index 6ed281c757a..e8183f05f5d 100644 --- a/tests/queries/0_stateless/02337_multiple_joins_original_names.reference +++ b/tests/queries/0_stateless/02337_multiple_joins_original_names.reference @@ -1,2 +1,3 @@ 1 1 +1 diff --git a/tests/queries/0_stateless/02337_multiple_joins_original_names.sql b/tests/queries/0_stateless/02337_multiple_joins_original_names.sql index afafee9f8eb..63bbfe6873d 100644 --- a/tests/queries/0_stateless/02337_multiple_joins_original_names.sql +++ b/tests/queries/0_stateless/02337_multiple_joins_original_names.sql @@ -1,11 +1,13 @@ -- https://github.com/ClickHouse/ClickHouse/issues/34697 +SET allow_experimental_analyzer = 1; + SELECT table1_id FROM ( SELECT first.table1_id FROM (SELECT number+1 as table1_id FROM numbers(1)) as first JOIN (SELECT number+1 as table2_id FROM numbers(1)) as second ON first.table1_id = second.table2_id JOIN (SELECT number+1 as table3_id FROM numbers(1)) as third ON first.table1_id = third.table3_id -); -- { serverError UNKNOWN_IDENTIFIER } +); SELECT table1_id FROM ( SELECT first.table1_id diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference index c6265e195c4..f5284f38b86 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference @@ -37,7 +37,7 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: system.numbers LIMIT - CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 google google google @@ -80,7 +80,7 @@ QUERY id: 0 JOIN TREE TABLE id: 9, table_name: system.numbers LIMIT - CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other1 other1 google1 @@ -124,7 +124,7 @@ QUERY id: 0 JOIN TREE TABLE id: 9, table_name: system.numbers LIMIT - CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google1 google1 google1 @@ -171,7 +171,7 @@ QUERY id: 0 JOIN TREE TABLE id: 11, table_name: system.numbers LIMIT - CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 google google google @@ -225,7 +225,7 @@ QUERY id: 0 JOIN TREE TABLE id: 12, table_name: system.numbers LIMIT - CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 22, constant_value: UInt64_10, constant_value_type: UInt64 other other google @@ -276,7 +276,7 @@ QUERY id: 0 JOIN TREE TABLE id: 10, table_name: system.numbers LIMIT - CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 20, constant_value: UInt64_10, constant_value_type: UInt64 google google google google google google @@ -343,7 +343,7 @@ QUERY id: 0 JOIN TREE TABLE id: 9, table_name: system.numbers LIMIT - CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 19, constant_value: UInt64_10, constant_value_type: UInt64 other other other other google google @@ -404,7 +404,7 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: system.numbers LIMIT - CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 \N \N \N @@ -444,7 +444,7 @@ QUERY id: 0 JOIN TREE TABLE id: 11, table_name: system.numbers LIMIT - CONSTANT id: 12, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 12, constant_value: UInt64_10, constant_value_type: UInt64 \N \N \N @@ -473,7 +473,7 @@ QUERY id: 0 JOIN TREE TABLE id: 5, table_name: system.numbers LIMIT - CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64 other other google @@ -502,7 +502,7 @@ QUERY id: 0 JOIN TREE TABLE id: 5, table_name: system.numbers LIMIT - CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 9, constant_value: UInt64_10, constant_value_type: UInt64 google google google @@ -534,4 +534,4 @@ QUERY id: 0 JOIN TREE TABLE id: 7, table_name: system.numbers LIMIT - CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt8 + CONSTANT id: 11, constant_value: UInt64_10, constant_value_type: UInt64 From 6209d13ad535f24b079c6b2a9d29dcea0dbc242b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 21 Feb 2023 21:50:24 +0100 Subject: [PATCH 163/470] Analyzer support _shardNum column to shardNum() function rewrite --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 21 +++++- .../Passes/ShardNumColumnToFunctionPass.cpp | 67 +++++++++++++++++++ .../Passes/ShardNumColumnToFunctionPass.h | 21 ++++++ src/Analyzer/QueryTreePassManager.cpp | 5 +- src/Functions/getScalar.cpp | 7 +- .../01018_Distributed__shard_num.reference | 2 +- .../01018_Distributed__shard_num.sql | 3 +- .../0_stateless/02163_shard_num.reference | 5 +- tests/queries/0_stateless/02163_shard_num.sql | 9 ++- 9 files changed, 129 insertions(+), 11 deletions(-) create mode 100644 src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp create mode 100644 src/Analyzer/Passes/ShardNumColumnToFunctionPass.h diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 0e47aec7c20..147c0660bc2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4928,11 +4928,15 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi { auto function_base = function->build(argument_columns); + /// Do not constant fold get scalar functions + bool is_get_scalar_function = function_name == "__getScalar" || function_name == "shardNum" || + function_name == "shardCount"; + /** If function is suitable for constant folding try to convert it to constant. * Example: SELECT plus(1, 1); * Result: SELECT 2; */ - if (function_base->isSuitableForConstantFolding()) + if (function_base->isSuitableForConstantFolding() && !is_get_scalar_function) { auto result_type = function_base->getResultType(); auto executable_function = function_base->prepare(argument_columns); @@ -6247,6 +6251,21 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, { validateTableExpressionModifiers(join_tree_node, scope); initializeTableExpressionData(join_tree_node, scope); + + auto & query_node = scope.scope_node->as(); + auto & mutable_context = query_node.getMutableContext(); + + if (!mutable_context->isDistributed()) + { + bool is_distributed = false; + + if (auto * table_node = join_tree_node->as()) + is_distributed = table_node->getStorage()->isRemote(); + else if (auto * table_function_node = join_tree_node->as()) + is_distributed = table_function_node->getStorage()->isRemote(); + + mutable_context->setDistributed(is_distributed); + } } auto add_table_expression_alias_into_scope = [&](const QueryTreeNodePtr & table_expression_node) diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp new file mode 100644 index 00000000000..b28816e8ff3 --- /dev/null +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -0,0 +1,67 @@ +#include + +#include + +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class ShardNumColumnToFunctionVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) const + { + auto * column_node = node->as(); + if (!column_node) + return; + + const auto & column = column_node->getColumn(); + if (column.name != "_shard_num") + return; + + auto column_source = column_node->getColumnSource(); + + auto * table_node = column_source->as(); + auto * table_function_node = column_source->as(); + if (!table_node && !table_function_node) + return; + + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + if (!storage->isRemote()) + return; + + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + if (!storage->isVirtualColumn(column.name, storage_snapshot->getMetadataForQuery())) + return; + + auto function_node = std::make_shared("shardNum"); + auto function = FunctionFactory::instance().get(function_node->getFunctionName(), getContext()); + function_node->resolveAsFunction(function->build(function_node->getArgumentColumns())); + node = std::move(function_node); + } +}; + +} + +void ShardNumColumnToFunctionPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + ShardNumColumnToFunctionVisitor visitor(context); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h new file mode 100644 index 00000000000..83b974954fa --- /dev/null +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +/** Rewrite _shard_num column into shardNum() function. + */ +class ShardNumColumnToFunctionPass final : public IQueryTreePass +{ +public: + String getName() override { return "ShardNumColumnToFunctionPass"; } + + String getDescription() override { return "Rewrite _shard_num column into shardNum() function"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 9ba18e27f73..dfac2f499dd 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -40,6 +40,7 @@ #include #include #include +#include namespace DB @@ -147,7 +148,6 @@ private: /** ClickHouse query tree pass manager. * - * TODO: Support _shard_num into shardNum() rewriting. * TODO: Support logical expressions optimizer. * TODO: Support setting convert_query_to_cnf. * TODO: Support setting optimize_using_constraints. @@ -266,10 +266,9 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } } diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index 1a26ac5f4da..d72c84b8528 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -121,7 +121,12 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - return ColumnConst::create(scalar.column, input_rows_count); + auto result = ColumnConst::create(scalar.column, input_rows_count); + + if (!isSuitableForConstantFolding()) + return result->convertToFullColumnIfConst(); + + return result; } private: diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.reference b/tests/queries/0_stateless/01018_Distributed__shard_num.reference index 46963a006ec..e50ae93402b 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.reference +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.reference @@ -85,7 +85,7 @@ SELECT a._shard_num, a.key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), FROM dist_1 a JOIN system.clusters b ON a._shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 47; } +WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 403 } SELECT 'dist_3'; dist_3 SELECT * FROM dist_3; diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/tests/queries/0_stateless/01018_Distributed__shard_num.sql index d3f4e1ac527..ce522227006 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.sql +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -2,6 +2,7 @@ -- make the order static SET max_threads = 1; +SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS mem1; DROP TABLE IF EXISTS mem2; @@ -79,7 +80,7 @@ SELECT a._shard_num, a.key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), FROM dist_1 a JOIN system.clusters b ON a._shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 47; } +WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 403 } SELECT 'dist_3'; SELECT * FROM dist_3; diff --git a/tests/queries/0_stateless/02163_shard_num.reference b/tests/queries/0_stateless/02163_shard_num.reference index a109d5d2b6b..f93803789a1 100644 --- a/tests/queries/0_stateless/02163_shard_num.reference +++ b/tests/queries/0_stateless/02163_shard_num.reference @@ -1,4 +1,5 @@ --- { echo } +-- { echoOn } + SELECT shardNum() AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) GROUP BY _shard_num; 2 1 1 1 @@ -14,4 +15,4 @@ SELECT _shard_num AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, SELECT a._shard_num AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) a GROUP BY shard_num; 2 1 1 1 -SELECT _shard_num FROM remote('127.1', system.one) AS a INNER JOIN (SELECT _shard_num FROM system.one) AS b USING (dummy); -- { serverError UNKNOWN_IDENTIFIER } +SELECT _shard_num FROM remote('127.1', system.one) AS a INNER JOIN (SELECT _shard_num FROM system.one) AS b USING (dummy); -- { serverError UNSUPPORTED_METHOD } diff --git a/tests/queries/0_stateless/02163_shard_num.sql b/tests/queries/0_stateless/02163_shard_num.sql index 27d40b3c976..65a2a8ca7c2 100644 --- a/tests/queries/0_stateless/02163_shard_num.sql +++ b/tests/queries/0_stateless/02163_shard_num.sql @@ -1,7 +1,12 @@ --- { echo } +SET allow_experimental_analyzer = 1; + +-- { echoOn } + SELECT shardNum() AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) GROUP BY _shard_num; SELECT shardNum() AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) GROUP BY shard_num; SELECT _shard_num AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) GROUP BY _shard_num; SELECT _shard_num AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) GROUP BY shard_num; SELECT a._shard_num AS shard_num, sum(1) as rows FROM remote('127.{1,2}', system, one) a GROUP BY shard_num; -SELECT _shard_num FROM remote('127.1', system.one) AS a INNER JOIN (SELECT _shard_num FROM system.one) AS b USING (dummy); -- { serverError UNKNOWN_IDENTIFIER } +SELECT _shard_num FROM remote('127.1', system.one) AS a INNER JOIN (SELECT _shard_num FROM system.one) AS b USING (dummy); -- { serverError UNSUPPORTED_METHOD } + +-- { echoOff } From bff55d5b2a493d1041f26edfacf0cde6ffa319e0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 21 Feb 2023 21:51:48 +0100 Subject: [PATCH 164/470] Planner table expression data column order fix --- src/Planner/PlannerJoinTree.cpp | 2 +- src/Planner/TableExpressionData.h | 31 ++++---- .../02514_analyzer_drop_join_on.reference | 72 +++++++++---------- 3 files changed, 52 insertions(+), 53 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 02c396542b1..a890780659c 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -259,7 +259,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - Names columns_names = table_expression_data.getColumnNames(); + auto columns_names = table_expression_data.getColumnNames(); /** The current user must have the SELECT privilege. * We do not check access rights for table functions because they have been already checked in ITableFunction::execute(). diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 6b4a9b4748d..e828f128e38 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -63,9 +63,7 @@ public: if (hasColumn(column.name)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Column with name {} already exists"); - column_name_to_column.emplace(column.name, column); - column_name_to_column_identifier.emplace(column.name, column_identifier); - column_identifier_to_column_name.emplace(column_identifier, column.name); + addColumnImpl(column, column_identifier); } /** Add column if it does not exists in table expression data. @@ -76,9 +74,7 @@ public: if (hasColumn(column.name)) return; - column_name_to_column.emplace(column.name, column); - column_name_to_column_identifier.emplace(column.name, column_identifier); - column_identifier_to_column_name.emplace(column_identifier, column.name); + addColumnImpl(column, column_identifier); } /// Add alias column name @@ -100,15 +96,9 @@ public: } /// Get column names - Names getColumnNames() const + const Names & getColumnNames() const { - Names result; - result.reserve(column_name_to_column.size()); - - for (const auto & [column_name, _] : column_name_to_column) - result.push_back(column_name); - - return result; + return column_names; } ColumnIdentifiers getColumnIdentifiers() const @@ -142,7 +132,6 @@ public: auto it = column_name_to_column.find(column_name); if (it == column_name_to_column.end()) { - auto column_names = getColumnNames(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Column for column name {} does not exists. There are only column names: {}", column_name, @@ -172,7 +161,6 @@ public: auto it = column_name_to_column_identifier.find(column_name); if (it == column_name_to_column_identifier.end()) { - auto column_names = getColumnNames(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Column identifier for column name {} does not exists. There are only column names: {}", column_name, @@ -240,6 +228,17 @@ public: } private: + void addColumnImpl(const NameAndTypePair & column, const ColumnIdentifier & column_identifier) + { + column_names.push_back(column.name); + column_name_to_column.emplace(column.name, column); + column_name_to_column_identifier.emplace(column.name, column_identifier); + column_identifier_to_column_name.emplace(column_identifier, column.name); + } + + /// Valid for table, table function, array join, query, union nodes + Names column_names; + /// Valid for table, table function, array join, query, union nodes ColumnNameToColumn column_name_to_column; diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 0f6fa7e2e66..af6f07e3f95 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -24,15 +24,15 @@ Header: count() UInt64 Header: default.a.a2_4 String default.b.b1_0 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_4 String - default.a.a1_1 UInt64 + Header: default.a.a1_1 UInt64 + default.a.a2_4 String default.b.b1_0 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a2_4 String - default.a.a1_1 UInt64 + Header: default.a.a1_1 UInt64 + default.a.a2_4 String ReadFromStorage (Memory) - Header: a2 String - a1 UInt64 + Header: a1 UInt64 + a2 String Expression ((JOIN actions + Change column names to column identifiers)) Header: default.b.b1_0 UInt64 ReadFromStorage (Memory) @@ -52,27 +52,27 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 default.d.d2_1 String Expression (DROP unused columns after JOIN) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 Expression (DROP unused columns after JOIN) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 Expression (Change column names to column identifiers) - Header: default.a.k_2 UInt64 - default.a.a2_0 String + Header: default.a.a2_0 String + default.a.k_2 UInt64 ReadFromStorage (Memory) - Header: k UInt64 - a2 String + Header: a2 String + k UInt64 Expression (Change column names to column identifiers) Header: default.b.k_3 UInt64 ReadFromStorage (Memory) @@ -82,11 +82,11 @@ Header: a2 String ReadFromStorage (Memory) Header: k UInt64 Expression (Change column names to column identifiers) - Header: default.d.k_5 UInt64 - default.d.d2_1 String + Header: default.d.d2_1 String + default.d.k_5 UInt64 ReadFromStorage (Memory) - Header: k UInt64 - d2 String + Header: d2 String + k UInt64 EXPLAIN PLAN header = 1 SELECT b.bx FROM a JOIN (SELECT b1, b2 || 'x' AS bx FROM b ) AS b ON b.b1 = a.a1 @@ -123,28 +123,28 @@ Header: bx String b.bx_0 String b.b1_1 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_6 String - default.a.a1_2 UInt64 + Header: default.a.a1_2 UInt64 + default.a.a2_6 String b.bx_0 String b.b1_1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a2_6 String - default.a.a1_2 UInt64 + Header: default.a.a1_2 UInt64 + default.a.a2_6 String ReadFromStorage (Memory) - Header: a2 String - a1 UInt64 + Header: a1 UInt64 + a2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) Header: b.b1_1 UInt64 b.bx_0 String ReadFromStorage (Memory) - Header: b2 String - b1 UInt64 + Header: b1 UInt64 + b2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.c.c2_5 String - default.c.c1_3 UInt64 + Header: default.c.c1_3 UInt64 + default.c.c2_5 String ReadFromStorage (Memory) - Header: c2 String - c1 UInt64 + Header: c1 UInt64 + c2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) Header: d.d1_4 UInt64 ReadFromStorage (SystemNumbers) From 07e114d8f6e6655571bcfd6d1c379604e1553fd5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 22 Feb 2023 15:03:16 +0100 Subject: [PATCH 165/470] Analyzer WITH FILL and INTERPOLATE fix --- src/Planner/Planner.cpp | 14 ++-- src/Planner/PlannerExpressionAnalysis.cpp | 69 +++++++++++++++++-- src/Planner/PlannerExpressionAnalysis.h | 1 + .../0_stateless/02233_interpolate_1.sql | 4 +- 4 files changed, 75 insertions(+), 13 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fff539f59d6..c5c9703ef6a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -591,20 +591,24 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, if (query_node.hasInterpolate()) { auto interpolate_actions_dag = std::make_shared(); + auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); + for (auto & query_plan_column : query_plan_columns) + { + query_plan_column.column = nullptr; + interpolate_actions_dag->addInput(query_plan_column); + } auto & interpolate_list_node = query_node.getInterpolate()->as(); auto & interpolate_list_nodes = interpolate_list_node.getNodes(); if (interpolate_list_nodes.empty()) { - auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); - for (auto & query_plan_column : query_plan_columns) + for (const auto * input_node : interpolate_actions_dag->getInputs()) { - if (column_names_with_fill.contains(query_plan_column.name)) + if (column_names_with_fill.contains(input_node->result_name)) continue; - const auto * input_action_node = &interpolate_actions_dag->addInput(query_plan_column); - interpolate_actions_dag->getOutputs().push_back(input_action_node); + interpolate_actions_dag->getOutputs().push_back(input_node); } } else diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index a813d3b753a..0e97e69cd67 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -359,7 +359,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, before_sort_actions_outputs.clear(); PlannerActionsVisitor actions_visitor(planner_context); - + bool has_with_fill = false; std::unordered_set before_sort_actions_dag_output_node_names; /** We add only sort node sort expression in before ORDER BY actions DAG. @@ -370,6 +370,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, { auto & sort_node_typed = sort_node->as(); auto expression_dag_nodes = actions_visitor.visit(before_sort_actions, sort_node_typed.getExpression()); + has_with_fill |= sort_node_typed.withFill(); for (auto & action_dag_node : expression_dag_nodes) { @@ -381,10 +382,55 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, } } + if (has_with_fill) + { + for (auto & output_node : before_sort_actions_outputs) + output_node = &before_sort_actions->materializeNode(*output_node); + } + + /// We add only INPUT columns necessary for INTERPOLATE expression in before ORDER BY actions DAG + if (query_node.hasInterpolate()) + { + auto & interpolate_list_node = query_node.getInterpolate()->as(); + + PlannerActionsVisitor interpolate_actions_visitor(planner_context); + auto interpolate_actions_dag = std::make_shared(); + + for (auto & interpolate_node : interpolate_list_node.getNodes()) + { + auto & interpolate_node_typed = interpolate_node->as(); + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); + } + + std::unordered_map before_sort_actions_inputs_name_to_node; + for (const auto & node : before_sort_actions->getInputs()) + before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); + + for (const auto & node : interpolate_actions_dag->getNodes()) + { + if (before_sort_actions_dag_output_node_names.contains(node.result_name) || + node.type != ActionsDAG::ActionType::INPUT) + continue; + + auto input_node_it = before_sort_actions_inputs_name_to_node.find(node.result_name); + if (input_node_it == before_sort_actions_inputs_name_to_node.end()) + { + auto input_column = ColumnWithTypeAndName{node.column, node.result_type, node.result_name}; + const auto * input_node = &before_sort_actions->addInput(std::move(input_column)); + auto [it, _] = before_sort_actions_inputs_name_to_node.emplace(node.result_name, input_node); + input_node_it = it; + } + + before_sort_actions_outputs.push_back(input_node_it->second); + before_sort_actions_dag_output_node_names.insert(node.result_name); + } + } + auto actions_step_before_sort = std::make_unique(before_sort_actions); actions_chain.addStep(std::move(actions_step_before_sort)); - return SortAnalysisResult{std::move(before_sort_actions)}; + return SortAnalysisResult{std::move(before_sort_actions), has_with_fill}; } /** Construct limit by analysis result. @@ -488,21 +534,30 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo const auto * chain_available_output_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); auto project_names_input = chain_available_output_columns ? *chain_available_output_columns : join_tree_input_columns; + bool has_with_fill = sort_analysis_result_optional.has_value() && sort_analysis_result_optional->has_with_fill; - /** If there is DISTINCT we must preserve non constant projection output columns + /** If there is WITH FILL we must use non constant projection columns. + * + * Example: SELECT 1 AS value ORDER BY value ASC WITH FILL FROM 0 TO 5 STEP 1; + * + * If there is DISTINCT we must preserve non constant projection output columns * in project names actions, to prevent removing of unused expressions during chain finalize. * * Example: SELECT DISTINCT id, 1 AS value FROM test_table ORDER BY id; */ - if (query_node.isDistinct()) + if (has_with_fill || query_node.isDistinct()) { std::unordered_set projection_column_names; - for (auto & [column_name, _] : projection_analysis_result.projection_column_names_with_display_aliases) - projection_column_names.insert(column_name); + + if (query_node.isDistinct()) + { + for (auto & [column_name, _] : projection_analysis_result.projection_column_names_with_display_aliases) + projection_column_names.insert(column_name); + } for (auto & column : project_names_input) { - if (projection_column_names.contains(column.name)) + if (has_with_fill || projection_column_names.contains(column.name)) column.column = nullptr; } } diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index b6d6ed96e9d..792cfdec2ff 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -48,6 +48,7 @@ struct WindowAnalysisResult struct SortAnalysisResult { ActionsDAGPtr before_order_by_actions; + bool has_with_fill = false; }; struct LimitByAnalysisResult diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index 229c36e23fb..b08f4a23898 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + # Test WITH FILL without INTERPOLATE SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 @@ -36,7 +38,7 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } # Test INTERPOLATE with aliased column SELECT n, source, inter + 1 AS inter_p FROM ( From 51ee007e01e37625f2a15838849766676a641770 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 24 Feb 2023 13:46:09 +0100 Subject: [PATCH 166/470] Fixed tests --- .../Passes/CustomizeFunctionsPass.cpp | 63 ----------- .../Passes/NormalizeCountVariantsPass.cpp | 3 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 100 +++++++++++++----- src/Interpreters/InterpreterCreateQuery.cpp | 6 -- .../getHeaderForProcessingStage.cpp | 39 +------ src/Planner/Planner.cpp | 38 ++----- src/Planner/PlannerJoinTree.cpp | 63 +++++++---- src/Planner/Utils.cpp | 34 ++++++ src/Planner/Utils.h | 6 ++ .../optimizePrimaryKeyCondition.cpp | 17 +-- .../QueryPlan/ReadFromMergeTree.cpp | 4 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 19 +--- .../QueryPlan/SourceStepWithFilter.h | 45 ++++++++ src/Storages/StorageDistributed.cpp | 42 +------- src/Storages/StorageDummy.cpp | 56 ++++++++++ src/Storages/StorageDummy.h | 74 +++++++++++++ src/Storages/StorageMerge.cpp | 8 +- src/Storages/StorageMerge.h | 15 +-- .../queries/0_stateless/00390_array_sort.sql | 4 +- .../00740_database_in_nested_view.sql | 1 + ...00936_function_result_with_operator_in.sql | 5 +- ...62_pm_all_join_with_block_continuation.sql | 26 ++--- ...2_multiple_joins_with_union_join.reference | 4 +- .../01852_multiple_joins_with_union_join.sql | 4 +- .../01936_quantiles_cannot_return_null.sql | 8 +- .../02339_analyzer_matcher_basic.reference | 27 +++-- .../02339_analyzer_matcher_basic.sql | 26 +++-- .../02378_analyzer_projection_names.reference | 2 +- 28 files changed, 425 insertions(+), 314 deletions(-) create mode 100644 src/Processors/QueryPlan/SourceStepWithFilter.h create mode 100644 src/Storages/StorageDummy.cpp create mode 100644 src/Storages/StorageDummy.h diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp index 3333008a0c0..55307fd39e7 100644 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp @@ -34,53 +34,6 @@ public: auto function_name = function_node->getFunctionName(); auto function_name_lowercase = Poco::toLower(function_name); - if (function_node->isAggregateFunction() || function_node->isWindowFunction()) - { - auto count_distinct_implementation_function_name = String(settings.count_distinct_implementation); - - /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal - if (function_name_lowercase.ends_with("ifdistinct")) - { - size_t prefix_length = function_name_lowercase.size() - strlen("ifdistinct"); - auto updated_function_name = function_name_lowercase.substr(0, prefix_length) + "DistinctIf"; - resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - - /** Move -OrNull suffix ahead, this should execute after add -OrNull suffix. - * Used to rewrite aggregate functions with -OrNull suffix in some cases. - * Example: sumIfOrNull. - * Result: sumOrNullIf. - */ - if (function_name.ends_with("OrNull")) - { - auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(function_name); - if (function_properies && !function_properies->returns_default_when_only_null) - { - size_t function_name_size = function_name.size(); - - static constexpr std::array suffixes_to_replace = {"MergeState", "Merge", "State", "If"}; - for (const auto & suffix : suffixes_to_replace) - { - auto suffix_string_value = String(suffix); - auto suffix_to_check = suffix_string_value + "OrNull"; - - if (!function_name.ends_with(suffix_to_check)) - continue; - - auto updated_function_name = function_name.substr(0, function_name_size - suffix_to_check.size()) + "OrNull" + suffix_string_value; - resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - break; - } - } - } - - return; - } - if (settings.transform_null_in) { auto function_result_type = function_node->getResultType(); @@ -106,22 +59,6 @@ public: } } - static inline void resolveAggregateOrWindowFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) - { - auto function_aggregate_function = function_node.getAggregateFunction(); - - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, - function_aggregate_function->getArgumentTypes(), - function_aggregate_function->getParameters(), - properties); - - if (function_node.isAggregateFunction()) - function_node.resolveAsAggregateFunction(std::move(aggregate_function)); - else if (function_node.isWindowFunction()) - function_node.resolveAsWindowFunction(std::move(aggregate_function)); - } - inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const { auto function = FunctionFactory::instance().get(function_name, getContext()); diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 705018d108f..d36be98751c 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -46,8 +46,7 @@ public: } else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && - first_argument_constant_literal.get() == 1 && - !getSettings().aggregate_functions_null_for_empty) + first_argument_constant_literal.get() == 1) { resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 147c0660bc2..9d1a096e582 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1168,9 +1168,11 @@ private: static void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); + static std::pair recursivelyCollectMaxOrdinaryExpressions(QueryTreeNodePtr & node, QueryTreeNodes & into); + static void expandGroupByAll(QueryNode & query_tree_node_typed); - static std::pair recursivelyCollectMaxOrdinaryExpressions(QueryTreeNodePtr & node, QueryTreeNodes & into); + static std::string rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context); /// Resolve identifier functions @@ -2238,9 +2240,72 @@ void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed) for (auto & node : projection_list.getNodes()) recursivelyCollectMaxOrdinaryExpressions(node, group_by_nodes); - } +std::string QueryAnalyzer::rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, const ContextPtr & context) +{ + std::string result_aggregate_function_name = aggregate_function_name; + auto aggregate_function_name_lowercase = Poco::toLower(aggregate_function_name); + + const auto & settings = context->getSettingsRef(); + + if (aggregate_function_name_lowercase == "countdistinct") + { + result_aggregate_function_name = settings.count_distinct_implementation; + } + else if (aggregate_function_name_lowercase == "countdistinctif" || aggregate_function_name_lowercase == "countifdistinct") + { + result_aggregate_function_name = settings.count_distinct_implementation; + result_aggregate_function_name += "If"; + } + + /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal + if (result_aggregate_function_name.ends_with("ifdistinct")) + { + size_t prefix_length = result_aggregate_function_name.size() - strlen("ifdistinct"); + result_aggregate_function_name = result_aggregate_function_name.substr(0, prefix_length) + "DistinctIf"; + } + + bool need_add_or_null = settings.aggregate_functions_null_for_empty && !result_aggregate_function_name.ends_with("OrNull"); + if (need_add_or_null) + { + auto properties = AggregateFunctionFactory::instance().tryGetProperties(result_aggregate_function_name); + if (!properties->returns_default_when_only_null) + result_aggregate_function_name += "OrNull"; + } + + /** Move -OrNull suffix ahead, this should execute after add -OrNull suffix. + * Used to rewrite aggregate functions with -OrNull suffix in some cases. + * Example: sumIfOrNull. + * Result: sumOrNullIf. + */ + if (result_aggregate_function_name.ends_with("OrNull")) + { + auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(result_aggregate_function_name); + if (function_properies && !function_properies->returns_default_when_only_null) + { + size_t function_name_size = result_aggregate_function_name.size(); + + static constexpr std::array suffixes_to_replace = {"MergeState", "Merge", "State", "If"}; + for (const auto & suffix : suffixes_to_replace) + { + auto suffix_string_value = String(suffix); + auto suffix_to_check = suffix_string_value + "OrNull"; + + if (!result_aggregate_function_name.ends_with(suffix_to_check)) + continue; + + result_aggregate_function_name = result_aggregate_function_name.substr(0, function_name_size - suffix_to_check.size()); + result_aggregate_function_name += "OrNull"; + result_aggregate_function_name += suffix_string_value; + + break; + } + } + } + + return result_aggregate_function_name; +} /// Resolve identifier functions implementation @@ -3003,7 +3068,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo { resolved_identifier = resolved_identifier->clone(); auto & resolved_column = resolved_identifier->as(); - resolved_column.setColumnType(makeNullableSafe(resolved_column.getColumnType())); + resolved_column.setColumnType(makeNullableOrLowCardinalityNullable(resolved_column.getColumnType())); } return resolved_identifier; @@ -3988,6 +4053,9 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I node = replace_expression->clone(); node_projection_names = resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + if (node_projection_names.size() == 1) + node_projection_names[0] = column_name; + execute_replace_transformer = true; } @@ -4678,8 +4746,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } } - const auto & settings = scope.context->getSettingsRef(); - if (function_node.isWindowFunction()) { if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) @@ -4694,12 +4760,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi "Window function '{}' does not support lambda arguments", function_name); - bool need_add_or_null = settings.aggregate_functions_null_for_empty && !function_name.ends_with("OrNull"); + std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, scope.context); AggregateFunctionProperties properties; - auto aggregate_function = need_add_or_null - ? AggregateFunctionFactory::instance().get(function_name + "OrNull", argument_types, parameters, properties) - : AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); + auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, parameters, properties); function_node.resolveAsWindowFunction(std::move(aggregate_function)); @@ -4758,24 +4822,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi "Aggregate function '{}' does not support lambda arguments", function_name); - auto function_name_lowercase = Poco::toLower(function_name); - - if (function_name_lowercase == "countdistinct") - { - function_name = scope.context->getSettingsRef().count_distinct_implementation; - } - else if (function_name_lowercase == "countdistinctif" || function_name_lowercase == "countifdistinct") - { - function_name = scope.context->getSettingsRef().count_distinct_implementation; - function_name += "If"; - } - - bool need_add_or_null = settings.aggregate_functions_null_for_empty && !function_name.ends_with("OrNull"); - if (need_add_or_null) - function_name += "OrNull"; + std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, scope.context); AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); + auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, parameters, properties); function_node.resolveAsAggregateFunction(std::move(aggregate_function)); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 29b7a4db609..edc6cc92fd7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1224,12 +1224,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) output_columns.push_back(output_column->cloneEmpty()); } } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); } } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index ab10bd2024c..3f52b2f5099 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -83,44 +84,6 @@ bool removeJoin(ASTSelectQuery & select, TreeRewriterResult & rewriter_result, C return true; } -class StorageDummy : public IStorage -{ -public: - StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_) - : IStorage(table_id_) - { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - setInMemoryMetadata(storage_metadata); - } - - std::string getName() const override { return "StorageDummy"; } - - bool supportsSampling() const override { return true; } - bool supportsFinal() const override { return true; } - bool supportsPrewhere() const override { return true; } - bool supportsSubcolumns() const override { return true; } - bool supportsDynamicSubcolumns() const override { return true; } - bool canMoveConditionsToPrewhere() const override { return false; } - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "StorageDummy does not support getQueryProcessingStage method"); - } - - Pipe read(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 - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "StorageDummy does not support read method"); - } -}; - Block getHeaderForProcessingStage( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c5c9703ef6a..49120c91db2 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1178,36 +1178,14 @@ void Planner::buildPlanForQueryNode() collectTableExpressionData(query_tree, *planner_context); collectSets(query_tree, *planner_context); - QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; - - if (select_query_options.only_analyze) - { - Block join_tree_block; - - for (const auto & [_, table_expression_data] : planner_context->getTableExpressionNodeToData()) - { - for (const auto & [column_name, column] : table_expression_data.getColumnNameToColumn()) - { - const auto & column_identifier = table_expression_data.getColumnIdentifierOrThrow(column_name); - join_tree_block.insert(ColumnWithTypeAndName(column.type, column_identifier)); - } - } - - auto read_nothing_step = std::make_unique(join_tree_block); - read_nothing_step->setStepDescription("Read nothing"); - query_plan.addStep(std::move(read_nothing_step)); - } - else - { - auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); - auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, - select_query_info, - select_query_options, - top_level_identifiers, - planner_context); - from_stage = join_tree_query_plan.from_stage; - query_plan = std::move(join_tree_query_plan.query_plan); - } + auto top_level_identifiers = collectTopLevelColumnIdentifiers(query_tree, planner_context); + auto join_tree_query_plan = buildJoinTreeQueryPlan(query_tree, + select_query_info, + select_query_options, + top_level_identifiers, + planner_context); + auto from_stage = join_tree_query_plan.from_stage; + query_plan = std::move(join_tree_query_plan.query_plan); LOG_TRACE(&Poco::Logger::get("Planner"), "Query {} from stage {} to stage {}{}", query_tree->formatConvertedASTForErrorMessage(), diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a890780659c..f8e4a3b7021 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include @@ -257,8 +257,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl else table_expression_query_info.table_expression_modifiers = table_function_node->getTableExpressionModifiers(); - from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); - auto columns_names = table_expression_data.getColumnNames(); /** The current user must have the SELECT privilege. @@ -308,7 +306,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } } - storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + if (!select_query_options.only_analyze) + { + from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); + storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + } if (query_plan.isInitialized()) { @@ -324,7 +326,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } else { - /// Create step which reads from empty source if storage has no data. + /// Create step which reads from empty source if storage has no data auto source_header = storage_snapshot->getSampleBlockForColumns(columns_names); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); @@ -334,22 +336,37 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } else if (query_node || union_node) { - if (table_expression_data.getColumnNames().empty()) + if (!select_query_options.only_analyze) { - const auto & projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); - NamesAndTypesList projection_columns_list(projection_columns.begin(), projection_columns.end()); - auto additional_column_to_read = ExpressionActions::getSmallestColumn(projection_columns_list); + if (table_expression_data.getColumnNames().empty()) + { + const auto & projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + NamesAndTypesList projection_columns_list(projection_columns.begin(), projection_columns.end()); + auto additional_column_to_read = ExpressionActions::getSmallestColumn(projection_columns_list); - const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); - table_expression_data.addColumn(additional_column_to_read, column_identifier); + const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); + table_expression_data.addColumn(additional_column_to_read, column_identifier); + } + + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner(table_expression, subquery_options, planner_context->getGlobalPlannerContext()); + /// Propagate storage limits to subquery + subquery_planner.addStorageLimits(*select_query_info.storage_limits); + subquery_planner.buildQueryPlanIfNeeded(); + query_plan = std::move(subquery_planner).extractQueryPlan(); } + else + { + auto projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + Block source_header; + for (auto & projection_column : projection_columns) + source_header.insert(ColumnWithTypeAndName(nullptr, projection_column.type, projection_column.name)); - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner(table_expression, subquery_options, planner_context->getGlobalPlannerContext()); - /// Propagate storage limits to subquery - subquery_planner.addStorageLimits(*select_query_info.storage_limits); - subquery_planner.buildQueryPlanIfNeeded(); - query_plan = std::move(subquery_planner).extractQueryPlan(); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } } else { @@ -533,9 +550,15 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (auto & output_node : cast_actions_dag->getOutputs()) { - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name) && - output_node->result_type->canBeInsideNullable()) - output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(output_node->result_name)) + { + DataTypePtr type_to_check = output_node->result_type; + if (const auto * type_to_check_low_cardinality = typeid_cast(type_to_check.get())) + type_to_check = type_to_check_low_cardinality->getDictionaryType(); + + if (type_to_check->canBeInsideNullable()) + output_node = &cast_actions_dag->addFunction(to_nullable_function, {output_node}, output_node->result_name); + } } cast_actions_dag->projectInput(); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 6df024fee4c..5c5eadac55d 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -13,12 +13,17 @@ #include +#include + #include +#include #include #include #include #include +#include +#include #include #include @@ -357,4 +362,33 @@ std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & return predicate_value > 0; } +QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, + const ContextPtr & context, + ResultReplacementMap * result_replacement_map) +{ + auto & query_node_typed = query_node->as(); + auto table_expressions = extractTableExpressions(query_node_typed.getJoinTree()); + std::unordered_map replacement_map; + + for (auto & table_expression : table_expressions) + { + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + if (!table_node && !table_function_node) + continue; + + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + auto storage_dummy = std::make_shared(storage_snapshot->storage.getStorageID(), + storage_snapshot->metadata->getColumns()); + auto dummy_table_node = std::make_shared(std::move(storage_dummy), context); + + if (result_replacement_map) + result_replacement_map->emplace(table_expression, dummy_table_node); + + replacement_map.emplace(table_expression.get(), std::move(dummy_table_node)); + } + + return query_node->cloneAndReplace(replacement_map); +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index e304ae13eb5..0520bd67d26 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -66,4 +66,10 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con /// Try extract boolean constant from condition node std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node); +/// Replace tables nodes and table function nodes with dummy table nodes +using ResultReplacementMap = std::unordered_map; +QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, + const ContextPtr & context, + ResultReplacementMap * result_replacement_map = nullptr); + } diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index cac4944bad7..753940339ea 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -1,8 +1,7 @@ #include #include #include -#include -#include +#include #include #include @@ -13,21 +12,15 @@ void optimizePrimaryKeyCondition(const Stack & stack) { const auto & frame = stack.back(); - auto * read_from_merge_tree = typeid_cast(frame.node->step.get()); - auto * read_from_merge = typeid_cast(frame.node->step.get()); - - if (!read_from_merge && !read_from_merge_tree) + auto * source_step_with_filter = dynamic_cast(frame.node->step.get()); + if (!source_step_with_filter) return; for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) - { - if (read_from_merge_tree) - read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); - if (read_from_merge) - read_from_merge->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); - } + source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, /// So this is likely not needed. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c70f6c6da30..fe279f3ab84 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -148,7 +148,7 @@ ReadFromMergeTree::ReadFromMergeTree( Poco::Logger * log_, MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading) - : ISourceStep(DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader( + : SourceStepWithFilter(DataStream{.header = IMergeTreeSelectAlgorithm::transformHeader( storage_snapshot_->getSampleBlockForColumns(real_column_names_), getPrewhereInfoFromQueryInfo(query_info_), data_.getPartitionValueType(), @@ -1095,7 +1095,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge return selectRangesToRead( std::move(parts), prewhere_info, - added_filter_nodes, + filter_nodes, storage_snapshot->metadata, storage_snapshot->getMetadataForQuery(), query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 4fd37fd6edf..9b28e903d21 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include @@ -29,7 +29,7 @@ using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptrfindInOutputs(column_name)); - } - - void addFilterNodes(const ActionDAGNodes & filter_nodes) - { - for (const auto & node : filter_nodes.nodes) - added_filter_nodes.nodes.push_back(node); - } - StorageID getStorageID() const { return data.getStorageID(); } UInt64 getSelectedParts() const { return selected_parts; } UInt64 getSelectedRows() const { return selected_rows; } @@ -203,9 +191,6 @@ private: PrewhereInfoPtr prewhere_info; ExpressionActionsSettings actions_settings; - std::vector added_filter_dags; - ActionDAGNodes added_filter_nodes; - StorageSnapshotPtr storage_snapshot; StorageMetadataPtr metadata_for_reading; diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h new file mode 100644 index 00000000000..a363451fff2 --- /dev/null +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Source step that can use filters for more efficient pipeline initialization. + * Filters must be added before pipeline initialization. + */ +class SourceStepWithFilter : public ISourceStep +{ +public: + using Base = ISourceStep; + using Base::Base; + + const std::vector & getFilters() const + { + return filter_dags; + } + + const ActionDAGNodes & getFilterNodes() const + { + return filter_nodes; + } + + void addFilter(ActionsDAGPtr filter_dag, std::string column_name) + { + filter_nodes.nodes.push_back(&filter_dag->findInOutputs(column_name)); + filter_dags.push_back(std::move(filter_dag)); + } + + void addFilter(ActionsDAGPtr filter_dag, const ActionsDAG::Node * filter_node) + { + filter_nodes.nodes.push_back(filter_node); + filter_dags.push_back(std::move(filter_dag)); + } + +protected: + std::vector filter_dags; + ActionDAGNodes filter_nodes; +}; + +} diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5516d6cadf0..286742586f2 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include @@ -133,7 +134,6 @@ namespace ErrorCodes extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; - extern const int UNSUPPORTED_METHOD; } namespace ActionLocks @@ -622,44 +622,6 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( namespace { -class StorageDistributedLocal : public IStorage -{ -public: - StorageDistributedLocal(const StorageID & table_id_, const ColumnsDescription & columns_) - : IStorage(table_id_) - { - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - setInMemoryMetadata(storage_metadata); - } - - std::string getName() const override { return "StorageDistributedLocal"; } - - bool supportsSampling() const override { return true; } - bool supportsFinal() const override { return true; } - bool supportsPrewhere() const override { return true; } - bool supportsSubcolumns() const override { return true; } - bool supportsDynamicSubcolumns() const override { return true; } - bool canMoveConditionsToPrewhere() const override { return false; } - - QueryProcessingStage::Enum - getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "StorageDistributedLocal does not support getQueryProcessingStage method"); - } - - Pipe read(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 - { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "StorageDistributedLocal does not support read method"); - } -}; - QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, @@ -687,7 +649,7 @@ QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const Sele auto resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); auto storage = DatabaseCatalog::instance().tryGetTable(resolved_remote_storage_id, query_context); if (!storage) - storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); + storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); replacement_table_expression = std::make_shared(std::move(storage), query_context); } diff --git a/src/Storages/StorageDummy.cpp b/src/Storages/StorageDummy.cpp new file mode 100644 index 00000000000..e5f3b0b4d8e --- /dev/null +++ b/src/Storages/StorageDummy.cpp @@ -0,0 +1,56 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_) + : IStorage(table_id_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); +} + +QueryProcessingStage::Enum StorageDummy::getQueryProcessingStage( + ContextPtr, + QueryProcessingStage::Enum, + const StorageSnapshotPtr &, + SelectQueryInfo &) const +{ + return QueryProcessingStage::FetchColumns; +} + +void StorageDummy::read(QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo &, + ContextPtr, + QueryProcessingStage::Enum, + size_t, + size_t) +{ + query_plan.addStep(std::make_unique(*this, storage_snapshot, column_names)); +} + +ReadFromDummy::ReadFromDummy(const StorageDummy & storage_, + StorageSnapshotPtr storage_snapshot_, + Names column_names_) + : SourceStepWithFilter(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(column_names_)}) + , storage(storage_) + , storage_snapshot(std::move(storage_snapshot_)) + , column_names(std::move(column_names_)) +{} + +void ReadFromDummy::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipe pipe(std::make_shared(getOutputStream().header)); + pipeline.init(std::move(pipe)); +} + +} diff --git a/src/Storages/StorageDummy.h b/src/Storages/StorageDummy.h new file mode 100644 index 00000000000..a7beef9d531 --- /dev/null +++ b/src/Storages/StorageDummy.h @@ -0,0 +1,74 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +class StorageDummy : public IStorage +{ +public: + StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_); + + std::string getName() const override { return "StorageDummy"; } + + bool supportsSampling() const override { return true; } + bool supportsFinal() const override { return true; } + bool supportsPrewhere() const override { return true; } + bool supportsSubcolumns() const override { return true; } + bool supportsDynamicSubcolumns() const override { return true; } + bool canMoveConditionsToPrewhere() const override { return false; } + + QueryProcessingStage::Enum getQueryProcessingStage( + ContextPtr local_context, + QueryProcessingStage::Enum to_stage, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info) const override; + + void read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; +}; + +class ReadFromDummy : public SourceStepWithFilter +{ +public: + explicit ReadFromDummy(const StorageDummy & storage_, + StorageSnapshotPtr storage_snapshot_, + Names column_names_); + + const StorageDummy & getStorage() const + { + return storage; + } + + const StorageSnapshotPtr & getStorageSnapshot() const + { + return storage_snapshot; + } + + const Names & getColumnNames() const + { + return column_names; + } + + String getName() const override { return "ReadFromDummy"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + +private: + const StorageDummy & storage; + StorageSnapshotPtr storage_snapshot; + Names column_names; +}; + +} diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0ca29e2826a..af5a32d47f7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -384,7 +384,7 @@ ReadFromMerge::ReadFromMerge( const SelectQueryInfo & query_info_, ContextMutablePtr context_, QueryProcessingStage::Enum processed_stage) - : ISourceStep(DataStream{.header = common_header_}) + : SourceStepWithFilter(DataStream{.header = common_header_}) , required_max_block_size(max_block_size) , requested_num_streams(num_streams) , common_header(std::move(common_header_)) @@ -691,7 +691,11 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( return {}; if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) - read_from_merge_tree->addFilterNodes(added_filter_nodes); + { + size_t filters_dags_size = filter_dags.size(); + for (size_t i = 0; i < filters_dags_size; ++i) + read_from_merge_tree->addFilter(filter_dags[i], filter_nodes.nodes[i]); + } builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 03c1719dfef..4bc47375047 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -115,7 +115,7 @@ private: friend class ReadFromMerge; }; -class ReadFromMerge final : public ISourceStep +class ReadFromMerge final : public SourceStepWithFilter { public: static constexpr auto name = "ReadFromMerge"; @@ -141,12 +141,6 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void addFilter(ActionsDAGPtr expression, std::string column_name) - { - added_filter_dags.push_back(expression); - added_filter_nodes.nodes.push_back(&expression->findInOutputs(column_name)); - } - const StorageListWithLocks & getSelectedTables() const { return selected_tables; } /// Returns `false` if requested reading cannot be performed. @@ -169,11 +163,6 @@ private: ContextMutablePtr context; QueryProcessingStage::Enum common_processed_stage; - std::vector added_filter_dags; - ActionDAGNodes added_filter_nodes; - - std::string added_filter_column_name; - InputOrderInfoPtr order_info; struct AliasData diff --git a/tests/queries/0_stateless/00390_array_sort.sql b/tests/queries/0_stateless/00390_array_sort.sql index 8c2307eb3bd..1518761b166 100644 --- a/tests/queries/0_stateless/00390_array_sort.sql +++ b/tests/queries/0_stateless/00390_array_sort.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + SELECT [2, 1, 3] AS arr, arraySort(arr), arrayReverseSort(arr), arraySort(x -> -x, arr); SELECT materialize([2, 1, 3]) AS arr, arraySort(arr), arrayReverseSort(arr), arraySort(x -> -x, arr); @@ -51,4 +53,4 @@ SELECT arrayPartialSort(2, [1,2,3], [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUM SELECT arrayPartialSort(2, [1,2,3], 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayPartialSort(arraySort([1,2,3]), [1,2,3]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayMap(x -> range(x), [4, 1, 2, 3]) AS arr, 100 AS lim, arrayResize(arrayPartialSort(arrayPartialSort(lim, arr), arr), lim), arrayResize(arrayPartialReverseSort(lim, arr), lim), arrayResize(arrayPartialSort(x -> (-length(x)), lim, arr), lim); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayPartialReverseSort(arraySort((x, y) -> y, [NULL, NULL], [NULL, NULL]), arr), arrayMap(x -> toString(x), [257, -9223372036854775807, 2, -2147483648, 2147483648, NULL, 65536, -2147483648, 2, 65535]) AS arr, NULL, 100 AS lim, 65536, arrayResize(arrayPartialSort(x -> reverse(x), lim, arr), lim) GROUP BY [NULL, 1023, -2, NULL, 255, '0', NULL, 9223372036854775806] WITH ROLLUP; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayPartialReverseSort(arraySort((x, y) -> y, [NULL, NULL], [NULL, NULL]), arr), arrayMap(x -> toString(x), [257, -9223372036854775807, 2, -2147483648, 2147483648, NULL, 65536, -2147483648, 2, 65535]) AS arr, NULL, 100 AS lim, 65536, arrayResize(arrayPartialSort(x -> reverse(x), lim, arr), lim) GROUP BY [NULL, 1023, -2, NULL, 255, '0', NULL, 9223372036854775806] WITH ROLLUP; -- { serverError NO_COMMON_TYPE } diff --git a/tests/queries/0_stateless/00740_database_in_nested_view.sql b/tests/queries/0_stateless/00740_database_in_nested_view.sql index 1890baafaa1..e2debe2859b 100644 --- a/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +DROP DATABASE IF EXISTS test_00740; CREATE DATABASE test_00740; USE test_00740; diff --git a/tests/queries/0_stateless/00936_function_result_with_operator_in.sql b/tests/queries/0_stateless/00936_function_result_with_operator_in.sql index dfb19c1f3ec..9d976e5333d 100644 --- a/tests/queries/0_stateless/00936_function_result_with_operator_in.sql +++ b/tests/queries/0_stateless/00936_function_result_with_operator_in.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET force_primary_key = 1; DROP TABLE IF EXISTS samples; @@ -22,8 +23,8 @@ SELECT 'a' IN splitByChar('c', 'abcdef'); SELECT 'errors:'; -- non-constant expressions in the right side of IN -SELECT count() FROM samples WHERE 1 IN range(samples.value); -- { serverError 47 } -SELECT count() FROM samples WHERE 1 IN range(rand() % 1000); -- { serverError 36 } +SELECT count() FROM samples WHERE 1 IN range(samples.value); -- { serverError 1 } +SELECT count() FROM samples WHERE 1 IN range(rand() % 1000); -- { serverError 1 } -- index is not used SELECT count() FROM samples WHERE value IN range(3); -- { serverError 277 } diff --git a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql index 15a28e92386..0732db73e38 100644 --- a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql +++ b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql @@ -2,32 +2,32 @@ SET max_memory_usage = 50000000; SET join_algorithm = 'partial_merge'; SELECT 'defaults'; - + SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); -- errors SET max_joined_block_size_rows = 0; - + SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -- { serverError 241 } SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -- { serverError 241 } @@ -35,17 +35,17 @@ SELECT 'max_joined_block_size_rows = 2000'; SET max_joined_block_size_rows = 2000; SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums + SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); @@ -53,16 +53,16 @@ SELECT 'max_rows_in_join = 1000'; SET max_rows_in_join = 1000; SELECT count(1) FROM ( - SELECT materialize(1) as k, n FROM numbers(10) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT materialize(1) as k, n FROM numbers(1000000) nums + SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000000)) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); diff --git a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.reference b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.reference index eb37a198435..85e6138dc5d 100644 --- a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.reference +++ b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.reference @@ -1,2 +1,2 @@ -1 1 1 1 -1 1 1 1 +1 1 1 +1 1 1 diff --git a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql index 39818fa51c2..8c6937eb581 100644 --- a/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql +++ b/tests/queries/0_stateless/01852_multiple_joins_with_union_join.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS v1; DROP TABLE IF EXISTS v2; @@ -17,7 +19,7 @@ JOIN v1 AS t2 USING (id) CROSS JOIN v2 AS n1 CROSS JOIN v2 AS n2; -SELECT * FROM v1 AS t1 JOIN v1 AS t2 USING (id) JOIN v1 AS t3 USING (value); -- { serverError NOT_IMPLEMENTED } +SELECT * FROM v1 AS t1 JOIN v1 AS t2 USING (id) JOIN v1 AS t3 USING (value); -- { serverError UNKNOWN_IDENTIFIER } DROP TABLE IF EXISTS v1; DROP TABLE IF EXISTS v2; diff --git a/tests/queries/0_stateless/01936_quantiles_cannot_return_null.sql b/tests/queries/0_stateless/01936_quantiles_cannot_return_null.sql index 81ac6224268..46fd8adf570 100644 --- a/tests/queries/0_stateless/01936_quantiles_cannot_return_null.sql +++ b/tests/queries/0_stateless/01936_quantiles_cannot_return_null.sql @@ -1,9 +1,9 @@ set aggregate_functions_null_for_empty=0; -SELECT quantiles(0.95)(x) FROM (SELECT 1 x WHERE 0); -SELECT quantiles(0.95)(number) FROM (SELECT number FROM numbers(10) WHERE number > 10); +SELECT quantiles(0.95)(x) FROM (SELECT 1 x WHERE 0); +SELECT quantiles(0.95)(number) FROM (SELECT number FROM numbers(10) WHERE number > 10); set aggregate_functions_null_for_empty=1; -SELECT quantiles(0.95)(x) FROM (SELECT 1 x WHERE 0); -SELECT quantiles(0.95)(number) FROM (SELECT number FROM numbers(10) WHERE number > 10); +SELECT quantiles(0.95)(x) FROM (SELECT 1 x WHERE 0); +SELECT quantiles(0.95)(number) FROM (SELECT number FROM numbers(10) WHERE number > 10); diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference index 3ca0d303793..7e263647f98 100644 --- a/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.reference @@ -38,6 +38,21 @@ id UInt64 value String 0 Value Database and table qualified matchers +-- +id UInt64 +value String +0 Value +-- +id UInt64 +0 +-- +id UInt64 +value String +0 Value +-- +id UInt64 +value String +0 Value APPLY transformer -- toString(id) String @@ -74,20 +89,20 @@ toString(value) String Value REPLACE transformer -- -5 UInt8 +id UInt8 value String 5 Value -- -5 UInt8 +id UInt8 value String 5 Value -- -5 UInt8 -6 UInt8 +id UInt8 +value UInt8 5 6 -- -5 UInt8 -6 UInt8 +id UInt8 +value UInt8 5 6 Combine EXCEPT, REPLACE, APPLY transformers -- diff --git a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql index 9d7c486b28a..a09325fa43b 100644 --- a/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql +++ b/tests/queries/0_stateless/02339_analyzer_matcher_basic.sql @@ -80,27 +80,25 @@ CREATE TABLE 02339_db.test_table INSERT INTO 02339_db.test_table VALUES (0, 'Value'); --- TODO: Qualified COLUMNS where identifier has more than 2 parts are not supported on parser level +SELECT '--'; --- SELECT '--'; +DESCRIBE (SELECT 02339_db.test_table.* FROM 02339_db.test_table); +SELECT 02339_db.test_table.* FROM 02339_db.test_table; --- DESCRIBE (SELECT 02339_db.test_table.* FROM 02339_db.test_table); --- SELECT 02339_db.test_table.* FROM 02339_db.test_table; +SELECT '--'; --- SELECT '--'; +DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table); +SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table; --- DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table); --- SELECT 02339_db.test_table.COLUMNS(id) FROM 02339_db.test_table; +SELECT '--'; --- SELECT '--'; +DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table); +SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table; --- DESCRIBE (SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table); --- SELECT 02339_db.test_table.COLUMNS(id), 02339_db.test_table.COLUMNS(value) FROM 02339_db.test_table; +SELECT '--'; --- SELECT '--'; - --- DESCRIBE (SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table); --- SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table; +DESCRIBE (SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table); +SELECT 02339_db.test_table.COLUMNS('i'), 02339_db.test_table.COLUMNS('v') FROM 02339_db.test_table; DROP TABLE 02339_db.test_table; DROP DATABASE 02339_db; diff --git a/tests/queries/0_stateless/02378_analyzer_projection_names.reference b/tests/queries/0_stateless/02378_analyzer_projection_names.reference index 9e72fe0d100..a82d4d4c5d2 100644 --- a/tests/queries/0_stateless/02378_analyzer_projection_names.reference +++ b/tests/queries/0_stateless/02378_analyzer_projection_names.reference @@ -136,7 +136,7 @@ plus(id, id) UInt64 SELECT '--'; -- DESCRIBE (SELECT test_table.* REPLACE id + (id AS id_alias) AS id, id_alias FROM test_table); -plus(id, id_alias) UInt64 +id UInt64 value String id_alias UInt64 SELECT 'Matcher'; From 14e01566a099a6b7861a5cc69826cacdf7b238bc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 24 Feb 2023 18:59:47 +0100 Subject: [PATCH 167/470] Fixed tests --- src/Analyzer/IQueryTreeNode.cpp | 7 +++++++ .../Passes/RewriteAggregateFunctionWithIfPass.cpp | 8 ++++++-- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++++++ src/Planner/Planner.cpp | 3 ++- src/Planner/PlannerJoinTree.cpp | 2 +- 5 files changed, 22 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 6b9b8b53d81..ba361af0007 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -263,6 +263,13 @@ QueryTreeNodePtr IQueryTreeNode::cloneAndReplace(const ReplacementMap & replacem const auto [node_to_clone, place_for_cloned_node] = nodes_to_clone.back(); nodes_to_clone.pop_back(); + auto already_cloned_node_it = old_pointer_to_new_pointer.find(node_to_clone); + if (already_cloned_node_it != old_pointer_to_new_pointer.end()) + { + *place_for_cloned_node = already_cloned_node_it->second; + continue; + } + auto it = replacement_map.find(node_to_clone); auto node_clone = it != replacement_map.end() ? it->second : node_to_clone->cloneImpl(); *place_for_cloned_node = node_clone; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index 4039e058b34..de264948d4c 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -17,7 +17,6 @@ namespace DB { - namespace { @@ -90,9 +89,14 @@ private: static inline void resolveAsAggregateFunctionWithIf(FunctionNode & function_node, const DataTypes & argument_types) { auto result_type = function_node.getResultType(); + + std::string suffix = "If"; + if (result_type->isNullable()) + suffix = "OrNullIf"; + AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get( - function_node.getFunctionName() + (result_type->isNullable() ? "IfOrNull" : "If"), + function_node.getFunctionName() + suffix, argument_types, function_node.getAggregateFunction()->getParameters(), properties); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index edc6cc92fd7..29b7a4db609 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1224,6 +1224,12 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) output_columns.push_back(output_column->cloneEmpty()); } } + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); } } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 49120c91db2..a65ee60edfa 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1440,7 +1440,8 @@ void Planner::buildPlanForQueryNode() } } - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + if (!select_query_options.only_analyze) + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); } void Planner::addStorageLimits(const StorageLimitsList & limits) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f8e4a3b7021..06424b041ba 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -360,7 +360,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl auto projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); Block source_header; for (auto & projection_column : projection_columns) - source_header.insert(ColumnWithTypeAndName(nullptr, projection_column.type, projection_column.name)); + source_header.insert(ColumnWithTypeAndName(projection_column.type, projection_column.name)); Pipe pipe(std::make_shared(source_header)); auto read_from_pipe = std::make_unique(std::move(pipe)); From d39be3ac9ca3d0c6402867f000d6e938a22580b1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 25 Feb 2023 20:16:51 +0100 Subject: [PATCH 168/470] Fixed tests --- src/Analyzer/ColumnNode.cpp | 11 +- src/Analyzer/IdentifierNode.cpp | 10 +- src/Analyzer/Passes/CrossToInnerJoinPass.cpp | 35 +-- .../Passes/CustomizeFunctionsPass.cpp | 77 ------- src/Analyzer/Passes/CustomizeFunctionsPass.h | 25 --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 200 +++++++++++------- src/Analyzer/Passes/QueryAnalysisPass.h | 1 + src/Analyzer/QueryTreePassManager.cpp | 3 - src/Analyzer/TableFunctionNode.cpp | 9 +- src/Analyzer/TableNode.cpp | 21 +- src/Analyzer/TableNode.h | 13 ++ src/Analyzer/Utils.cpp | 28 +++ src/Analyzer/Utils.h | 11 + src/Functions/FunctionsExternalDictionaries.h | 25 +-- src/Functions/formatRow.cpp | 25 ++- .../ExternalDictionariesLoader.cpp | 5 + src/Interpreters/ExternalDictionariesLoader.h | 2 + src/Parsers/ExpressionElementParsers.cpp | 4 +- src/Planner/PlannerJoinTree.cpp | 21 +- src/Storages/StorageDistributed.cpp | 4 +- ..._no_aggregates_and_constant_keys.reference | 8 +- ..._shard_no_aggregates_and_constant_keys.sql | 3 +- .../00313_const_totals_extremes.reference | 12 +- .../00313_const_totals_extremes.sh | 12 +- ...01013_totals_without_aggregation.reference | 6 +- .../01013_totals_without_aggregation.sql | 2 + .../01115_join_with_dictionary.sql | 4 +- .../01470_columns_transformers.sql | 8 +- .../01603_decimal_mult_float.reference | 2 +- .../0_stateless/01603_decimal_mult_float.sql | 2 + .../0_stateless/01774_tuple_null_in.reference | 1 - .../0_stateless/01774_tuple_null_in.sql | 1 - ...hree_parts_identifiers_in_wrong_places.sql | 8 +- ..._with_statement_non_exist_column.reference | 1 - .../02354_with_statement_non_exist_column.sql | 2 - ..._group_by_without_key_and_totals.reference | 8 +- .../00081_group_by_without_key_and_totals.sql | 2 + 37 files changed, 323 insertions(+), 289 deletions(-) delete mode 100644 src/Analyzer/Passes/CustomizeFunctionsPass.cpp delete mode 100644 src/Analyzer/Passes/CustomizeFunctionsPass.h diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index b9939df37bb..c07d7bab717 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -110,8 +110,15 @@ ASTPtr ColumnNode::toASTImpl() const } else if (auto * table_node = column_source->as()) { - const auto & table_storage_id = table_node->getStorageID(); - column_identifier_parts = {table_storage_id.getDatabaseName(), table_storage_id.getTableName()}; + if (!table_node->getTemporaryTableName().empty()) + { + column_identifier_parts = { table_node->getTemporaryTableName() }; + } + else + { + const auto & table_storage_id = table_node->getStorageID(); + column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; + } } } } diff --git a/src/Analyzer/IdentifierNode.cpp b/src/Analyzer/IdentifierNode.cpp index 4efc7f515ea..cb5d9609962 100644 --- a/src/Analyzer/IdentifierNode.cpp +++ b/src/Analyzer/IdentifierNode.cpp @@ -40,15 +40,7 @@ void IdentifierNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_sta bool IdentifierNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - - if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) - return false; - else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) - return false; - else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) - return false; - - return identifier == rhs_typed.identifier; + return identifier == rhs_typed.identifier && table_expression_modifiers == rhs_typed.table_expression_modifiers; } void IdentifierNode::updateTreeHashImpl(HashState & state) const diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index 4f62d6ef068..d4877d23f28 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include @@ -152,25 +154,34 @@ public: QueryTreeNodes other_conditions; exctractJoinConditions(where_condition, equi_conditions, other_conditions); bool can_convert_cross_to_inner = false; - for (auto & cond : equi_conditions) + for (auto & condition : equi_conditions) { - auto left_src = getExpressionSource(getEquiArgument(cond, 0)); - auto right_src = getExpressionSource(getEquiArgument(cond, 1)); - if (left_src.second && right_src.second && left_src.first && right_src.first) - { - bool can_join_on = (findInTableExpression(left_src.first, left_table) && findInTableExpression(right_src.first, right_table)) - || (findInTableExpression(left_src.first, right_table) && findInTableExpression(right_src.first, left_table)); + const auto & lhs_equi_argument = getEquiArgument(condition, 0); + const auto & rhs_equi_argument = getEquiArgument(condition, 1); - if (can_join_on) + DataTypes key_types = {lhs_equi_argument->getResultType(), rhs_equi_argument->getResultType()}; + DataTypePtr common_key_type = tryGetLeastSupertype(key_types); + + /// If there is common key type, we can join on this condition + if (common_key_type) + { + auto left_src = getExpressionSource(lhs_equi_argument); + auto right_src = getExpressionSource(rhs_equi_argument); + + if (left_src.second && right_src.second && left_src.first && right_src.first) { - can_convert_cross_to_inner = true; - continue; + if ((findInTableExpression(left_src.first, left_table) && findInTableExpression(right_src.first, right_table)) || + (findInTableExpression(left_src.first, right_table) && findInTableExpression(right_src.first, left_table))) + { + can_convert_cross_to_inner = true; + continue; + } } } /// Can't join on this condition, move it to other conditions - other_conditions.push_back(cond); - cond = nullptr; + other_conditions.push_back(condition); + condition = nullptr; } if (!can_convert_cross_to_inner) diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp deleted file mode 100644 index 55307fd39e7..00000000000 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ /dev/null @@ -1,77 +0,0 @@ -#include - -#include -#include - -#include - -#include - -#include -#include - -namespace DB -{ - -namespace -{ - -class CustomizeFunctionsVisitor : public InDepthQueryTreeVisitorWithContext -{ -public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; - - void visitImpl(QueryTreeNodePtr & node) const - { - auto * function_node = node->as(); - if (!function_node) - return; - - const auto & settings = getSettings(); - - /// After successful function replacement function name and function name lowercase must be recalculated - auto function_name = function_node->getFunctionName(); - auto function_name_lowercase = Poco::toLower(function_name); - - if (settings.transform_null_in) - { - auto function_result_type = function_node->getResultType(); - - static constexpr std::array, 4> in_function_to_replace_null_in_function_map = - {{ - {"in", "nullIn"}, - {"notin", "notNullIn"}, - {"globalin", "globalNullIn"}, - {"globalnotin", "globalNotNullIn"}, - }}; - - for (const auto & [in_function_name, in_function_name_to_replace] : in_function_to_replace_null_in_function_map) - { - if (function_name_lowercase == in_function_name) - { - resolveOrdinaryFunctionNode(*function_node, String(in_function_name_to_replace)); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - break; - } - } - } - } - - inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const - { - auto function = FunctionFactory::instance().get(function_name, getContext()); - function_node.resolveAsFunction(function->build(function_node.getArgumentColumns())); - } -}; - -} - -void CustomizeFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) -{ - CustomizeFunctionsVisitor visitor(std::move(context)); - visitor.visit(query_tree_node); -} - -} diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.h b/src/Analyzer/Passes/CustomizeFunctionsPass.h deleted file mode 100644 index 7145099ca4c..00000000000 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** Customize aggregate functions and `in` functions implementations. - * - * Example: SELECT countDistinct(); - * Result: SELECT countDistinctImplementation(); - * Function countDistinctImplementation is taken from settings.count_distinct_implementation. - */ -class CustomizeFunctionsPass final : public IQueryTreePass -{ -public: - String getName() override { return "CustomizeFunctions"; } - - String getDescription() override { return "Customize implementation of aggregate functions, and in functions."; } - - void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; - -}; - -} diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9d1a096e582..01b9494bfc3 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -37,14 +37,16 @@ #include #include +#include +#include #include #include -#include #include -#include #include -#include +#include +#include +#include #include @@ -92,6 +94,7 @@ namespace ErrorCodes extern const int CYCLIC_ALIASES; extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY; extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int TYPE_MISMATCH; extern const int AMBIGUOUS_IDENTIFIER; @@ -1971,26 +1974,10 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto constant_node = std::make_shared(std::move(constant_value), node); if (constant_node->getValue().isNull()) - { - std::string cast_type = constant_node->getResultType()->getName(); - std::string cast_function_name = "_CAST"; - - auto cast_type_constant_value = std::make_shared(std::move(cast_type), std::make_shared()); - auto cast_type_constant_node = std::make_shared(std::move(cast_type_constant_value)); - - auto cast_function_node = std::make_shared(cast_function_name); - cast_function_node->getArguments().getNodes().push_back(constant_node); - cast_function_node->getArguments().getNodes().push_back(std::move(cast_type_constant_node)); - - auto cast_function = FunctionFactory::instance().get(cast_function_name, context); - cast_function_node->resolveAsFunction(cast_function->build(cast_function_node->getArgumentColumns())); - - node = std::move(cast_function_node); - } + node = buildCastFunction(constant_node, constant_node->getResultType(), context); else - { node = std::move(constant_node); - } + return; } @@ -2333,6 +2320,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con StorageID storage_id(database_name, table_name); storage_id = context->resolveStorageID(storage_id); + bool is_temporary_table = storage_id.getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE; + auto storage = DatabaseCatalog::instance().tryGetTable(storage_id, context); if (!storage) return {}; @@ -2340,7 +2329,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); - return std::make_shared(std::move(storage), std::move(storage_lock), std::move(storage_snapshot)); + auto result = std::make_shared(std::move(storage), std::move(storage_lock), std::move(storage_snapshot)); + if (is_temporary_table) + result->setTemporaryTableName(table_name); + + return result; } /// Resolve identifier from compound expression @@ -3961,17 +3954,16 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I matched_expression_nodes_with_names = resolveUnqualifiedMatcher(matcher_node, scope); std::unordered_map> strict_transformer_to_used_column_names; - auto add_strict_transformer_column_name = [&](const IColumnTransformerNode * transformer, const std::string & column_name) + for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes()) { - auto it = strict_transformer_to_used_column_names.find(transformer); - if (it == strict_transformer_to_used_column_names.end()) - { - auto [inserted_it, _] = strict_transformer_to_used_column_names.emplace(transformer, std::unordered_set()); - it = inserted_it; - } + auto * except_transformer = transformer->as(); + auto * replace_transformer = transformer->as(); - it->second.insert(column_name); - }; + if (except_transformer && except_transformer->isStrict()) + strict_transformer_to_used_column_names.emplace(except_transformer, std::unordered_set()); + else if (replace_transformer && replace_transformer->isStrict()) + strict_transformer_to_used_column_names.emplace(replace_transformer, std::unordered_set()); + } ListNodePtr list = std::make_shared(); ProjectionNames result_projection_names; @@ -4026,12 +4018,12 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I else if (auto * except_transformer = transformer->as()) { if (apply_transformer_was_used || replace_transformer_was_used) - break; + continue; if (except_transformer->isColumnMatching(column_name)) { if (except_transformer->isStrict()) - add_strict_transformer_column_name(except_transformer, column_name); + strict_transformer_to_used_column_names[except_transformer].insert(column_name); node = {}; break; @@ -4040,7 +4032,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I else if (auto * replace_transformer = transformer->as()) { if (apply_transformer_was_used || replace_transformer_was_used) - break; + continue; replace_transformer_was_used = true; @@ -4049,7 +4041,7 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I continue; if (replace_transformer->isStrict()) - add_strict_transformer_column_name(replace_transformer, column_name); + strict_transformer_to_used_column_names[replace_transformer].insert(column_name); node = replace_expression->clone(); node_projection_names = resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -4140,22 +4132,12 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I non_matched_column_names.push_back(column_name); } - WriteBufferFromOwnString non_matched_column_names_buffer; - size_t non_matched_column_names_size = non_matched_column_names.size(); - for (size_t i = 0; i < non_matched_column_names_size; ++i) - { - const auto & column_name = non_matched_column_names[i]; - - non_matched_column_names_buffer << column_name; - if (i + 1 != non_matched_column_names_size) - non_matched_column_names_buffer << ", "; - } - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Strict {} column transformer {} expects following column(s) {}", + "Strict {} column transformer {} expects following column(s) : {}. In scope {}", toString(strict_transformer_type), strict_transformer->formatASTForErrorMessage(), - non_matched_column_names_buffer.str()); + fmt::join(non_matched_column_names, ", "), + scope.scope_node->formatASTForErrorMessage()); } matcher_node = std::move(list); @@ -4449,17 +4431,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } bool is_special_function_in = false; - bool is_special_function_dict_get_or_join_get = false; + bool is_special_function_dict_get = false; + bool is_special_function_join_get = false; bool is_special_function_exists = false; if (!lambda_expression_untyped) { is_special_function_in = isNameOfInFunction(function_name); - is_special_function_dict_get_or_join_get = functionIsJoinGet(function_name) || functionIsDictGet(function_name); + is_special_function_dict_get = functionIsDictGet(function_name); + is_special_function_join_get = functionIsJoinGet(function_name); is_special_function_exists = function_name == "exists"; - /// Handle SELECT count(*) FROM test_table - if (Poco::toLower(function_name) == "count" && function_node_ptr->getArguments().getNodes().size() == 1) + auto function_name_lowercase = Poco::toLower(function_name); + + /** Special handling for count and countState functions. + * + * Example: SELECT count(*) FROM test_table + * Example: SELECT countState(*) FROM test_table; + */ + if (function_node_ptr->getArguments().getNodes().size() == 1 && + (function_name_lowercase == "count" || function_name_lowercase == "countstate")) { auto * matcher_node = function_node_ptr->getArguments().getNodes().front()->as(); if (matcher_node && matcher_node->isUnqualified()) @@ -4476,19 +4467,56 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi * Otherwise replace identifier with identifier full name constant. * Validation that dictionary exists or table exists will be performed during function `getReturnType` method call. */ - if (is_special_function_dict_get_or_join_get && + if ((is_special_function_dict_get || is_special_function_join_get) && !function_node_ptr->getArguments().getNodes().empty() && function_node_ptr->getArguments().getNodes()[0]->getNodeType() == QueryTreeNodeType::IDENTIFIER) { auto & first_argument = function_node_ptr->getArguments().getNodes()[0]; - auto & identifier_node = first_argument->as(); - IdentifierLookup identifier_lookup{identifier_node.getIdentifier(), IdentifierLookupContext::EXPRESSION}; + auto & first_argument_identifier = first_argument->as(); + auto identifier = first_argument_identifier.getIdentifier(); + + IdentifierLookup identifier_lookup{identifier, IdentifierLookupContext::EXPRESSION}; auto resolve_result = tryResolveIdentifier(identifier_lookup, scope); if (resolve_result.isResolved()) + { first_argument = std::move(resolve_result.resolved_identifier); + } else - first_argument = std::make_shared(identifier_node.getIdentifier().getFullName()); + { + size_t parts_size = identifier.getPartsSize(); + if (parts_size < 1 || parts_size > 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected {} function first argument identifier to contain 1 or 2 parts. Actual '{}'. In scope {}", + function_name, + identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + if (is_special_function_dict_get) + { + scope.context->getExternalDictionariesLoader().assertDictionaryStructureExists(identifier.getFullName(), scope.context); + } + else + { + auto table_node = tryResolveTableIdentifierFromDatabaseCatalog(identifier, scope.context); + if (!table_node) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} first argument expected table identifier '{}'. In scope {}", + function_name, + identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + + auto & table_node_typed = table_node->as(); + if (!std::dynamic_pointer_cast(table_node_typed.getStorage())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function {} table '{}' should have engine StorageJoin. In scope {}", + function_name, + identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage()); + } + + first_argument = std::make_shared(identifier.getFullName()); + } } /// Resolve function arguments @@ -4526,6 +4554,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns if (is_special_function_in) { + if (scope.context->getSettingsRef().transform_null_in) + { + static constexpr std::array, 4> in_function_to_replace_null_in_function_map = + {{ + {"in", "nullIn"}, + {"notIn", "notNullIn"}, + {"globalIn", "globalNullIn"}, + {"globalNotIn", "globalNotNullIn"}, + }}; + + for (const auto & [in_function_name, in_function_name_to_replace] : in_function_to_replace_null_in_function_map) + { + if (function_name == in_function_name) + { + function_name = in_function_name_to_replace; + break; + } + } + } + auto & function_in_arguments_nodes = function_node.getArguments().getNodes(); if (function_in_arguments_nodes.size() != 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function '{}' expects 2 arguments", function_name); @@ -4586,6 +4634,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto & function_argument = function_arguments[function_argument_index]; ColumnWithTypeAndName argument_column; + argument_column.name = arguments_projection_names[function_argument_index]; /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction * where function argument types are initialized with empty array of lambda arguments size. @@ -4627,7 +4676,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } /// Calculate function projection name - ProjectionNames result_projection_names = {calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names)}; + ProjectionNames result_projection_names = { calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names) }; /** Try to resolve function as * 1. Lambda function in current scope. Example: WITH (x -> x + 1) AS lambda SELECT lambda(1); @@ -4939,7 +4988,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } /// Recalculate function projection name after lambda resolution - result_projection_names = {calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names)}; + result_projection_names = { calculateFunctionProjectionName(node, parameters_projection_names, arguments_projection_names) }; } /** Create SET column for special function IN to allow constant folding @@ -5765,10 +5814,19 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table if (table_node) { - const auto & table_storage_id = table_node->getStorageID(); - table_expression_data.database_name = table_storage_id.database_name; - table_expression_data.table_name = table_storage_id.table_name; - table_expression_data.table_expression_name = table_storage_id.getFullNameNotQuoted(); + if (!table_node->getTemporaryTableName().empty()) + { + table_expression_data.table_name = table_node->getTemporaryTableName(); + table_expression_data.table_expression_name = table_node->getTemporaryTableName(); + } + else + { + const auto & table_storage_id = table_node->getStorageID(); + table_expression_data.database_name = table_storage_id.database_name; + table_expression_data.table_name = table_storage_id.table_name; + table_expression_data.table_expression_name = table_storage_id.getFullNameNotQuoted(); + } + table_expression_data.table_expression_description = "table"; } else if (query_node || union_node) @@ -5814,7 +5872,9 @@ void QueryAnalyzer::initializeTableExpressionData(const QueryTreeNodePtr & table if (column_default && column_default->kind == ColumnDefaultKind::Alias) { - auto column_node = std::make_shared(column_name_and_type, buildQueryTree(column_default->expression, scope.context), table_expression_node); + auto alias_expression = buildQueryTree(column_default->expression, scope.context); + alias_expression = buildCastFunction(alias_expression, column_name_and_type.type, scope.context, false /*resolve*/); + auto column_node = std::make_shared(column_name_and_type, std::move(alias_expression), table_expression_node); column_name_to_column_node.emplace(column_name_and_type.name, column_node); alias_columns_to_resolve.emplace_back(column_name_and_type.name, column_node); } @@ -6181,13 +6241,6 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS IdentifierLookup identifier_lookup{identifier_node->getIdentifier(), IdentifierLookupContext::EXPRESSION}; auto result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node_typed.getLeftTableExpression(), scope); - if (!result_left_table_expression && identifier_node->hasAlias()) - { - std::vector alias_identifier_parts = {identifier_node->getAlias()}; - IdentifierLookup alias_identifier_lookup{Identifier(std::move(alias_identifier_parts)), IdentifierLookupContext::EXPRESSION}; - result_left_table_expression = tryResolveIdentifierFromJoinTreeNode(alias_identifier_lookup, join_node_typed.getLeftTableExpression(), scope); - } - if (!result_left_table_expression) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier '{}' cannot be resolved from left table expression. In scope {}", @@ -6203,13 +6256,6 @@ void QueryAnalyzer::resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveS scope.scope_node->formatASTForErrorMessage()); auto result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, join_node_typed.getRightTableExpression(), scope); - if (!result_right_table_expression && identifier_node->hasAlias()) - { - std::vector alias_identifier_parts = {identifier_node->getAlias()}; - IdentifierLookup alias_identifier_lookup{Identifier(std::move(alias_identifier_parts)), IdentifierLookupContext::EXPRESSION}; - result_right_table_expression = tryResolveIdentifierFromJoinTreeNode(alias_identifier_lookup, join_node_typed.getRightTableExpression(), scope); - } - if (!result_right_table_expression) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "JOIN {} using identifier '{}' cannot be resolved from right table expression. In scope {}", diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index 8b3a8a116e5..a4ee685812d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -54,6 +54,7 @@ namespace DB * Replace `countDistinct` and `countIfDistinct` aggregate functions using setting count_distinct_implementation. * Add -OrNull suffix to aggregate functions if setting aggregate_functions_null_for_empty is true. * Function `exists` is converted into `in`. + * Functions `in`, `notIn`, `globalIn`, `globalNotIn` converted into `nullIn`, `notNullIn`, `globalNullIn`, `globalNotNullIn` if setting transform_null_in is true. * * For function `grouping` arguments are resolved, but it is planner responsibility to initialize it with concrete grouping function * based on group by kind and group by keys positions. diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index dfac2f499dd..2cecf4f81a2 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -242,8 +241,6 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index c8cd05cf685..fb3a3af31e1 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -82,14 +82,7 @@ bool TableFunctionNode::isEqualImpl(const IQueryTreeNode & rhs) const if (storage && rhs_typed.storage) return storage_id == rhs_typed.storage_id; - if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) - return false; - else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) - return false; - else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) - return false; - - return true; + return table_expression_modifiers == rhs_typed.table_expression_modifiers; } void TableFunctionNode::updateTreeHashImpl(HashState & state) const diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 1018570c3d4..a746986be04 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -42,6 +42,9 @@ void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", table_name: " << storage_id.getFullNameNotQuoted(); + if (!temporary_table_name.empty()) + buffer << ", temporary_table_name: " << temporary_table_name; + if (table_expression_modifiers) { buffer << ", "; @@ -52,15 +55,8 @@ void TableNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s bool TableNode::isEqualImpl(const IQueryTreeNode & rhs) const { const auto & rhs_typed = assert_cast(rhs); - - if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) - return false; - else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) - return false; - else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) - return false; - - return storage_id == rhs_typed.storage_id; + return storage_id == rhs_typed.storage_id && table_expression_modifiers == rhs_typed.table_expression_modifiers && + temporary_table_name == rhs_typed.temporary_table_name; } void TableNode::updateTreeHashImpl(HashState & state) const @@ -69,6 +65,9 @@ void TableNode::updateTreeHashImpl(HashState & state) const state.update(full_name.size()); state.update(full_name); + state.update(temporary_table_name.size()); + state.update(temporary_table_name); + if (table_expression_modifiers) table_expression_modifiers->updateTreeHash(state); } @@ -77,12 +76,16 @@ QueryTreeNodePtr TableNode::cloneImpl() const { auto result_table_node = std::make_shared(storage, storage_id, storage_lock, storage_snapshot); result_table_node->table_expression_modifiers = table_expression_modifiers; + result_table_node->temporary_table_name = temporary_table_name; return result_table_node; } ASTPtr TableNode::toASTImpl() const { + if (!temporary_table_name.empty()) + return std::make_shared(temporary_table_name); + return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); } diff --git a/src/Analyzer/TableNode.h b/src/Analyzer/TableNode.h index 6d47f87c78b..1d5ec112ee0 100644 --- a/src/Analyzer/TableNode.h +++ b/src/Analyzer/TableNode.h @@ -56,6 +56,18 @@ public: return storage_lock; } + /// Get temporary table name + const std::string & getTemporaryTableName() const + { + return temporary_table_name; + } + + /// Set temporary table name + void setTemporaryTableName(std::string temporary_table_name_value) + { + temporary_table_name = std::move(temporary_table_name_value); + } + /// Return true if table node has table expression modifiers, false otherwise bool hasTableExpressionModifiers() const { @@ -102,6 +114,7 @@ private: TableLockHolder storage_lock; StorageSnapshotPtr storage_snapshot; std::optional table_expression_modifiers; + std::string temporary_table_name; static constexpr size_t children_size = 0; }; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index e746a8ff570..c5a5c042cbc 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -5,13 +5,18 @@ #include #include +#include #include #include #include +#include + +#include #include #include +#include #include #include #include @@ -74,6 +79,29 @@ bool isNameOfInFunction(const std::string & function_name) return is_special_function_in; } +QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, + const DataTypePtr & type, + const ContextPtr & context, + bool resolve) +{ + std::string cast_type = type->getName(); + auto cast_type_constant_value = std::make_shared(std::move(cast_type), std::make_shared()); + auto cast_type_constant_node = std::make_shared(std::move(cast_type_constant_value)); + + std::string cast_function_name = "_CAST"; + auto cast_function_node = std::make_shared(cast_function_name); + cast_function_node->getArguments().getNodes().push_back(expression); + cast_function_node->getArguments().getNodes().push_back(std::move(cast_type_constant_node)); + + if (resolve) + { + auto cast_function = FunctionFactory::instance().get(cast_function_name, context); + cast_function_node->resolveAsFunction(cast_function->build(cast_function_node->getArgumentColumns())); + } + + return cast_function_node; +} + static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node) { ASTPtr table_expression_node_ast; diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 0f54b5cadd5..3e2d95c6012 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -2,6 +2,8 @@ #include +#include + namespace DB { @@ -11,6 +13,15 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); /// Returns true if function name is name of IN function or its variations, false otherwise bool isNameOfInFunction(const std::string & function_name); +/** Build cast function that cast expression into type. + * If resolve = true, then result cast function is resolved during build, otherwise + * result cast function is not resolved during build. + */ +QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, + const DataTypePtr & type, + const ContextPtr & context, + bool resolve = true); + /** Add table expression in tables in select query children. * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. */ diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 48ecc13da39..570147e4c86 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -675,27 +675,28 @@ private: return impl.isInjective(sample_columns); } - DataTypePtr getReturnTypeImpl(const DataTypes &) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - DataTypePtr result; + DataTypePtr result_type; if constexpr (IsDataTypeDecimal) - result = std::make_shared(DataType::maxPrecision(), 0); + result_type = std::make_shared(DataType::maxPrecision(), 0); else - result = std::make_shared(); + result_type = std::make_shared(); - return result; + auto return_type = impl.getReturnTypeImpl(arguments); + if (!return_type->equals(*result_type)) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Function {} dictionary attribute has different type {} expected {}", + getName(), + return_type->getName(), + result_type->getName()); + + return result_type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - auto return_type = impl.getReturnTypeImpl(arguments); - - if (!return_type->equals(*result_type)) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Dictionary attribute has different type {} expected {}", - return_type->getName(), result_type->getName()); - - return impl.executeImpl(arguments, return_type, input_rows_count); + return impl.executeImpl(arguments, result_type, input_rows_count); } const FunctionDictGetNoType impl; diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 53fb5d61472..12a5fc2cc27 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -35,7 +35,10 @@ class FunctionFormatRow : public IFunction public: static constexpr auto name = no_newline ? "formatRowNoNewline" : "formatRow"; - FunctionFormatRow(const String & format_name_, ContextPtr context_) : format_name(format_name_), context(context_) + FunctionFormatRow(String format_name_, Names arguments_column_names_, ContextPtr context_) + : format_name(std::move(format_name_)) + , arguments_column_names(std::move(arguments_column_names_)) + , context(std::move(context_)) { if (!FormatFactory::instance().getAllFormats().contains(format_name)) throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown format {}", format_name); @@ -55,9 +58,17 @@ public: WriteBufferFromVector buffer(vec); ColumnString::Offsets & offsets = col_str->getOffsets(); offsets.resize(input_rows_count); + Block arg_columns; - for (auto i = 1u; i < arguments.size(); ++i) - arg_columns.insert(arguments[i]); + + size_t arguments_size = arguments.size(); + for (size_t i = 1; i < arguments_size; ++i) + { + auto argument_column = arguments[i]; + argument_column.name = arguments_column_names[i]; + arg_columns.insert(std::move(argument_column)); + } + materializeBlockInplace(arg_columns); auto format_settings = getFormatSettings(context); auto out = FormatFactory::instance().getOutputFormat(format_name, buffer, arg_columns, context, format_settings); @@ -93,6 +104,7 @@ public: private: String format_name; + Names arguments_column_names; ContextPtr context; }; @@ -115,9 +127,14 @@ public: throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least two arguments: the format name and its output expression(s)", getName()); + Names arguments_column_names; + arguments_column_names.reserve(arguments.size()); + for (const auto & argument : arguments) + arguments_column_names.push_back(argument.name); + if (const auto * name_col = checkAndGetColumnConst(arguments.at(0).column.get())) return std::make_unique( - std::make_shared>(name_col->getValue(), context), + std::make_shared>(name_col->getValue(), std::move(arguments_column_names), context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); else diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 2de0ff13e2b..080878c7d86 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -85,6 +85,11 @@ DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); } +void ExternalDictionariesLoader::assertDictionaryStructureExists(const std::string & dictionary_name, ContextPtr query_context) const +{ + getDictionaryStructure(dictionary_name, query_context); +} + QualifiedTableName ExternalDictionariesLoader::qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr query_context) const { auto qualified_name = QualifiedTableName::tryParseFromString(dictionary_name); diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index 421154a6d4f..659820d02ac 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -31,6 +31,8 @@ public: DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; + void assertDictionaryStructureExists(const std::string & dictionary_name, ContextPtr context) const; + static DictionaryStructure getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config = "dictionary"); static DictionaryStructure getDictionaryStructure(const ObjectConfig & config); diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index f2660e5a40c..72353a42a87 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1474,7 +1474,7 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e auto opos = pos; if (ParserExpression().parse(pos, lambda, expected)) { - if (const auto * func = lambda->as(); func && func->name == "lambda") + if (auto * func = lambda->as(); func && func->name == "lambda") { if (func->arguments->children.size() != 2) throw Exception(ErrorCodes::SYNTAX_ERROR, "lambda requires two arguments"); @@ -1491,6 +1491,8 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e lambda_arg = *opt_arg_name; else throw Exception(ErrorCodes::SYNTAX_ERROR, "lambda argument declarations must be identifiers"); + + func->is_lambda_function = true; } else { diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 06424b041ba..4d6181e3686 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -404,15 +404,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl auto expected_header = planner.getQueryPlan().getCurrentDataStream().header; materializeBlockInplace(expected_header); - auto rename_actions_dag = ActionsDAG::makeConvertingActions( - query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), - expected_header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position, - true /*ignore_constant_values*/); - auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); - std::string step_description = table_expression_data.isRemote() ? "Change remote column names to local column names" : "Change column names"; - rename_step->setStepDescription(std::move(step_description)); - query_plan.addStep(std::move(rename_step)); + if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, expected_header)) + { + auto rename_actions_dag = ActionsDAG::makeConvertingActions( + query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), + expected_header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position, + true /*ignore_constant_values*/); + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(rename_actions_dag)); + std::string step_description = table_expression_data.isRemote() ? "Change remote column names to local column names" : "Change column names"; + rename_step->setStepDescription(std::move(step_description)); + query_plan.addStep(std::move(rename_step)); + } } return {std::move(query_plan), from_stage}; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 286742586f2..5c0027b20f5 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -647,9 +647,7 @@ QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const Sele else { auto resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto storage = DatabaseCatalog::instance().tryGetTable(resolved_remote_storage_id, query_context); - if (!storage) - storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); + auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); replacement_table_expression = std::make_shared(std::move(storage), query_context); } diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index fc77ed8a241..c5e3f5b9cc9 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -5,16 +5,16 @@ 11 40 -40 +0 41 -41 +0 2 42 -2 42 +2 0 43 -43 +0 11 11 diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql index 93fa1f440b1..5cc06a41132 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.sql @@ -1,6 +1,7 @@ -- Tags: shard -set enable_positional_arguments=0; +set allow_experimental_analyzer = 1; +set enable_positional_arguments = 0; select 40 as z from (select * from system.numbers limit 3) group by z; select 41 as z from remote('127.0.0.{2,3}', system.one) group by z; diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.reference b/tests/queries/0_stateless/00313_const_totals_extremes.reference index fcb39b8080c..fd10e1f4387 100644 --- a/tests/queries/0_stateless/00313_const_totals_extremes.reference +++ b/tests/queries/0_stateless/00313_const_totals_extremes.reference @@ -1,6 +1,6 @@ 1 1 -1 1 +0 1 1 1 1 1 @@ -27,7 +27,7 @@ "totals": { - "k": "1234567890123", + "k": "0", "count()": "1" }, @@ -65,7 +65,7 @@ [1.23, "1"] ], - "totals": [1.23, "1"], + "totals": [0, "1"], "extremes": { @@ -77,7 +77,7 @@ } 2010-01-01 1 -2010-01-01 1 +1970-01-01 1 2010-01-01 1 2010-01-01 1 @@ -104,7 +104,7 @@ "totals": { - "k": "2010-01-01 01:02:03", + "k": "1970-01-01 01:00:00", "count()": "1" }, @@ -142,7 +142,7 @@ [1.1, "1"] ], - "totals": [1.1, "1"], + "totals": [0, "1"], "extremes": { diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.sh b/tests/queries/0_stateless/00313_const_totals_extremes.sh index 0c51d80abe8..5e880fe7aa0 100755 --- a/tests/queries/0_stateless/00313_const_totals_extremes.sh +++ b/tests/queries/0_stateless/00313_const_totals_extremes.sh @@ -4,10 +4,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS FORMAT JSON"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1234567890123 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03') AS k, count() GROUP BY k WITH TOTALS FORMAT JSON"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS FORMAT JSONCompact"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; diff --git a/tests/queries/0_stateless/01013_totals_without_aggregation.reference b/tests/queries/0_stateless/01013_totals_without_aggregation.reference index 6dddf22a467..a1f5c1aa914 100644 --- a/tests/queries/0_stateless/01013_totals_without_aggregation.reference +++ b/tests/queries/0_stateless/01013_totals_without_aggregation.reference @@ -1,7 +1,7 @@ 11 -11 -12 +0 12 +0 13 -13 +0 diff --git a/tests/queries/0_stateless/01013_totals_without_aggregation.sql b/tests/queries/0_stateless/01013_totals_without_aggregation.sql index 584a8994767..291f95c6bd6 100644 --- a/tests/queries/0_stateless/01013_totals_without_aggregation.sql +++ b/tests/queries/0_stateless/01013_totals_without_aggregation.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + SELECT 11 AS n GROUP BY n WITH TOTALS; SELECT 12 AS n GROUP BY n WITH ROLLUP; SELECT 13 AS n GROUP BY n WITH CUBE; diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql index cde1385eaae..83227aa555a 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.sql +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -88,11 +88,11 @@ SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s1 FULL JOIN dict_flat SELECT '-'; SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY INNER JOIN dict_flat d USING(key) ORDER BY s1.key; SELECT '-'; -SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY RIGHT JOIN dict_flat d USING(key) ORDER BY key; SELECT '-'; SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 SEMI RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key; SELECT '-'; -SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANTI RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key; +SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANTI RIGHT JOIN dict_flat d USING(key) ORDER BY key; SET join_use_nulls = 0; diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index 2da2f6e9c67..22c30ed36bf 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS columns_transformers; CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) Engine=TinyLog; @@ -17,15 +19,15 @@ SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; SELECT * EXCEPT STRICT i from columns_transformers; SELECT * EXCEPT STRICT (i, j) from columns_transformers; SELECT * EXCEPT STRICT i, j1 from columns_transformers; -- { serverError 47 } -SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 16 } +SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 36 } SELECT * REPLACE STRICT i + 1 AS i from columns_transformers; -SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError 16 } +SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError 36 } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 16 } +SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 36 } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.reference b/tests/queries/0_stateless/01603_decimal_mult_float.reference index c2917516e99..4c9d45423ee 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.reference +++ b/tests/queries/0_stateless/01603_decimal_mult_float.reference @@ -3,7 +3,7 @@ 0.00012000000000000002 150.16500000000002 7.775900000000001 -56.62269 +56.622689999999984 598.8376688440277 299.41883695311844 0.7485470860550345 diff --git a/tests/queries/0_stateless/01603_decimal_mult_float.sql b/tests/queries/0_stateless/01603_decimal_mult_float.sql index 99e4b775da7..799ab91d332 100644 --- a/tests/queries/0_stateless/01603_decimal_mult_float.sql +++ b/tests/queries/0_stateless/01603_decimal_mult_float.sql @@ -1,3 +1,5 @@ +SET optimize_arithmetic_operations_in_aggregate_functions = 0; + SELECT toDecimal32(2, 2) * 1.2; SELECT toDecimal64(0.5, 2) * 20.33; SELECT 0.00001 * toDecimal32(12, 2); diff --git a/tests/queries/0_stateless/01774_tuple_null_in.reference b/tests/queries/0_stateless/01774_tuple_null_in.reference index aa47d0d46d4..573541ac970 100644 --- a/tests/queries/0_stateless/01774_tuple_null_in.reference +++ b/tests/queries/0_stateless/01774_tuple_null_in.reference @@ -1,2 +1 @@ 0 -0 diff --git a/tests/queries/0_stateless/01774_tuple_null_in.sql b/tests/queries/0_stateless/01774_tuple_null_in.sql index a9cc39e8840..f08e1a067e5 100644 --- a/tests/queries/0_stateless/01774_tuple_null_in.sql +++ b/tests/queries/0_stateless/01774_tuple_null_in.sql @@ -1,2 +1 @@ -SELECT (NULL, NULL) = (8, 0) OR (NULL, NULL) = (3, 2) OR (NULL, NULL) = (0, 0) OR (NULL, NULL) = (3, 1); SELECT (NULL, NULL) IN ((NULL, 0), (3, 1), (3, 2), (8, 0), (NULL, NULL)); diff --git a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql index d2ca771edc5..83dd708c575 100644 --- a/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql +++ b/tests/queries/0_stateless/01936_three_parts_identifiers_in_wrong_places.sql @@ -1,7 +1,9 @@ -SELECT dictGet(t.nest.a, concat(currentDatabase(), '.dict.dict'), 's', number) FROM numbers(5); -- { serverError 47 } +SET allow_experimental_analyzer = 1; -SELECT dictGetFloat64(t.b.s, 'database_for_dict.dict1', dictGetFloat64('Ta\0', toUInt64('databas\0_for_dict.dict1databas\0_for_dict.dict1', dictGetFloat64('', '', toUInt64(1048577), toDate(NULL)), NULL), toDate(dictGetFloat64(257, 'database_for_dict.dict1database_for_dict.dict1', '', toUInt64(NULL), 2, toDate(NULL)), '2019-05-2\0')), NULL, toUInt64(dictGetFloat64('', '', toUInt64(-9223372036854775808), toDate(NULL)), NULL)); -- { serverError 47 } +SELECT dictGet(t.nest.a, concat(currentDatabase(), '.dict.dict'), 's', number) FROM numbers(5); -- { serverError 36 } -SELECT NULL AND (2147483648 AND NULL) AND -2147483647, toUUID(((1048576 AND NULL) AND (2147483647 AND 257 AND NULL AND -2147483649) AND NULL) IN (test_01103.t1_distr.id), '00000000-e1fe-11e\0-bb8f\0853d60c00749'), stringToH3('89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff'); -- { serverError 47 } +SELECT dictGetFloat64(t.b.s, 'database_for_dict.dict1', dictGetFloat64('Ta\0', toUInt64('databas\0_for_dict.dict1databas\0_for_dict.dict1', dictGetFloat64('', '', toUInt64(1048577), toDate(NULL)), NULL), toDate(dictGetFloat64(257, 'database_for_dict.dict1database_for_dict.dict1', '', toUInt64(NULL), 2, toDate(NULL)), '2019-05-2\0')), NULL, toUInt64(dictGetFloat64('', '', toUInt64(-9223372036854775808), toDate(NULL)), NULL)); -- { serverError 36 } + +SELECT NULL AND (2147483648 AND NULL) AND -2147483647, toUUID(((1048576 AND NULL) AND (2147483647 AND 257 AND NULL AND -2147483649) AND NULL) IN (test_01103.t1_distr.id), '00000000-e1fe-11e\0-bb8f\0853d60c00749'), stringToH3('89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff89184926cc3ffff'); -- { serverError 36 } SELECT 'still alive'; diff --git a/tests/queries/0_stateless/02354_with_statement_non_exist_column.reference b/tests/queries/0_stateless/02354_with_statement_non_exist_column.reference index d00491fd7e5..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02354_with_statement_non_exist_column.reference +++ b/tests/queries/0_stateless/02354_with_statement_non_exist_column.reference @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02354_with_statement_non_exist_column.sql b/tests/queries/0_stateless/02354_with_statement_non_exist_column.sql index 1a989c1d952..869c335b621 100644 --- a/tests/queries/0_stateless/02354_with_statement_non_exist_column.sql +++ b/tests/queries/0_stateless/02354_with_statement_non_exist_column.sql @@ -1,5 +1,3 @@ -WITH x AS y SELECT 1; - DROP TEMPORARY TABLE IF EXISTS t1; DROP TEMPORARY TABLE IF EXISTS t2; diff --git a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference index fc48d81d7bb..f7b0f272bd7 100644 --- a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference +++ b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference @@ -12,16 +12,16 @@ 523264 1 523264 -1 523264 +0 523264 1 523264 -1 523264 +0 523264 1 523264 -1 523264 +0 523264 1 523264 -1 523264 +0 523264 -1 86614 0 413533 1 9984 diff --git a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql index d9653f48552..03e9e0feb40 100644 --- a/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql +++ b/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; From 47ece846bc29fc16625fadfc4b419d861db4b430 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 26 Feb 2023 14:45:10 +0100 Subject: [PATCH 169/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 20 +++++++++---------- src/TableFunctions/TableFunctionMerge.cpp | 19 ++++++++++++++++++ src/TableFunctions/TableFunctionMerge.h | 1 + tests/queries/0_stateless/00203_full_join.sql | 10 +++++----- .../00744_join_not_found_column.sql | 20 +++++++++++++------ .../01165_lost_part_empty_partition.sql | 4 +++- .../01495_subqueries_in_with_statement_2.sql | 20 +++++++++---------- .../01720_join_implicit_cast.sql.j2 | 1 - .../01881_join_on_conditions_hash.sql.j2 | 7 ++++--- .../01881_join_on_conditions_merge.sql.j2 | 6 ++++-- .../0_stateless/02233_interpolate_1.sql | 2 +- 11 files changed, 69 insertions(+), 41 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 01b9494bfc3..8a69eb348d6 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4519,14 +4519,6 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } } - /// Resolve function arguments - - bool allow_table_expressions = is_special_function_in || is_special_function_exists; - auto arguments_projection_names = resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), - scope, - true /*allow_lambda_expression*/, - allow_table_expressions /*allow_table_expression*/); - if (is_special_function_exists) { /// Rewrite EXISTS (subquery) into 1 IN (SELECT 1 FROM (subquery) LIMIT 1). @@ -4535,20 +4527,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto constant_data_type = std::make_shared(); auto in_subquery = std::make_shared(Context::createCopy(scope.context)); + in_subquery->setIsSubquery(true); in_subquery->getProjection().getNodes().push_back(std::make_shared(1UL, constant_data_type)); in_subquery->getJoinTree() = exists_subquery_argument; in_subquery->getLimit() = std::make_shared(1UL, constant_data_type); - in_subquery->resolveProjectionColumns({NameAndTypePair("1", constant_data_type)}); - in_subquery->setIsSubquery(true); function_node_ptr = std::make_shared("in"); function_node_ptr->getArguments().getNodes() = {std::make_shared(1UL, constant_data_type), in_subquery}; node = function_node_ptr; function_name = "in"; - is_special_function_in = true; } + /// Resolve function arguments + + bool allow_table_expressions = is_special_function_in; + auto arguments_projection_names = resolveExpressionNodeList(function_node_ptr->getArgumentsNode(), + scope, + true /*allow_lambda_expression*/, + allow_table_expressions /*allow_table_expression*/); + auto & function_node = *function_node_ptr; /// Replace right IN function argument if it is table or table function with subquery that read ordinary columns diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 742cc7f366b..066caa8170d 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -36,6 +38,23 @@ namespace } } +std::vector TableFunctionMerge::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +{ + auto & table_function_node = query_node_table_function->as(); + auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); + size_t table_function_arguments_size = table_function_arguments_nodes.size(); + + std::vector result; + + for (size_t i = 0; i < table_function_arguments_size; ++i) + { + auto * function_node = table_function_arguments_nodes[i]->as(); + if (function_node && function_node->getFunctionName() == "REGEXP") + result.push_back(i); + } + + return {0}; +} void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr context) { diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index 10221c8c72c..3439056deda 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -24,6 +24,7 @@ private: using DBToTableSetMap = std::map; const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const; ColumnsDescription getActualTableStructure(ContextPtr context) const override; + std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context); diff --git a/tests/queries/0_stateless/00203_full_join.sql b/tests/queries/0_stateless/00203_full_join.sql index 9b07e9c84e7..43ce4c6da7d 100644 --- a/tests/queries/0_stateless/00203_full_join.sql +++ b/tests/queries/0_stateless/00203_full_join.sql @@ -17,17 +17,17 @@ DROP TABLE IF EXISTS t1_00203; DROP TABLE IF EXISTS t2_00203; CREATE TABLE t1_00203 (k1 UInt32, k2 UInt32, k3 UInt32, val_t1 String) ENGINE=TinyLog; -CREATE TABLE t2_00203 (val_t2 String, k3 UInt32, k2_alias UInt32, k1 UInt32) ENGINE=TinyLog; +CREATE TABLE t2_00203 (val_t2 String, k3 UInt32, k2 UInt32, k1 UInt32) ENGINE=TinyLog; INSERT INTO t1_00203 VALUES (1, 2, 3, 'aaa'), (2, 3, 4, 'bbb'); INSERT INTO t2_00203 VALUES ('ccc', 4, 3, 2), ('ddd', 7, 6, 5); -SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; +SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2) ORDER BY k1, k2, k3; +SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2) ORDER BY k1, k2, k3; SET any_join_distinct_right_table_keys = 0; -SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 } -SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; +SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2) ORDER BY k1, k2, k3; -- { serverError 48 } +SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2) ORDER BY k1, k2, k3; DROP TABLE t1_00203; DROP TABLE t2_00203; diff --git a/tests/queries/0_stateless/00744_join_not_found_column.sql b/tests/queries/0_stateless/00744_join_not_found_column.sql index a271347dcd9..72969e7937d 100644 --- a/tests/queries/0_stateless/00744_join_not_found_column.sql +++ b/tests/queries/0_stateless/00744_join_not_found_column.sql @@ -1,3 +1,5 @@ +SET joined_subquery_requires_alias = 0; + DROP TEMPORARY TABLE IF EXISTS test_00744; CREATE TEMPORARY TABLE test_00744 ( @@ -11,27 +13,33 @@ FROM ( SELECT x, - 1 - FROM test_00744 + `1` + FROM + ( + SELECT x, 1 FROM test_00744 + ) ALL INNER JOIN ( SELECT count(), 1 FROM test_00744 - ) jss2 USING (1) + ) jss2 USING (`1`) LIMIT 10 ); SELECT x, - 1 -FROM test_00744 + `1` +FROM +( + SELECT x, 1 FROM test_00744 +) ALL INNER JOIN ( SELECT count(), 1 FROM test_00744 -) js2 USING (1) +) js2 USING (`1`) LIMIT 10; diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index 9279de1a5b3..acda7d68391 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -1,11 +1,13 @@ -- Tags: zookeeper +SET allow_experimental_analyzer = 1; + create table rmt1 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '1') order by n partition by toYYYYMMDD(d); create table rmt2 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '2') order by n partition by toYYYYMMDD(d); system stop replicated sends rmt1; insert into rmt1 values (now(), arrayJoin([1, 2])); -- { clientError 36 } -insert into rmt1(n) select * from system.numbers limit arrayJoin([1, 2]); -- { serverError 36 } +insert into rmt1(n) select * from system.numbers limit arrayJoin([1, 2]); -- { serverError 440 } insert into rmt1 values (now(), rand()); drop table rmt1; diff --git a/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql index 7ec4eeaa778..43dff687d84 100644 --- a/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql +++ b/tests/queries/0_stateless/01495_subqueries_in_with_statement_2.sql @@ -1,29 +1,29 @@ -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)) SELECT * FROM x WHERE a in (SELECT a FROM y) ORDER BY a; -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)) SELECT * FROM x left JOIN y USING a ORDER BY a; -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)) SELECT * FROM x JOIN y USING a ORDER BY x.a; -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)), z AS (SELECT toUInt64(1) b) SELECT * FROM x JOIN y USING a WHERE a in (SELECT * FROM z); -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)), z AS (SELECT * FROM x WHERE a % 2), @@ -31,19 +31,17 @@ w AS (SELECT * FROM y WHERE a > 0) SELECT * FROM x JOIN y USING a WHERE a in (SELECT * FROM z) ORDER BY x.a; -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)), z AS (SELECT * FROM x WHERE a % 2), w AS (SELECT * FROM y WHERE a > 0) -SELECT max(a) FROM x JOIN y USING a WHERE a in (SELECT * FROM z) -HAVING a > (SELECT min(a) FROM w); +SELECT max(a) FROM x JOIN y USING a WHERE a in (SELECT * FROM z) AND a > (SELECT min(a) FROM w); -WITH +WITH x AS (SELECT number AS a FROM numbers(10)), y AS (SELECT number AS a FROM numbers(5)), z AS (SELECT * FROM x WHERE a % 2), w AS (SELECT * FROM y WHERE a > 0) -SELECT a FROM x JOIN y USING a WHERE a in (SELECT * FROM z) -HAVING a <= (SELECT max(a) FROM w) +SELECT a FROM x JOIN y USING a WHERE a in (SELECT * FROM z) AND a <= (SELECT max(a) FROM w) ORDER BY x.a; diff --git a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 index f7439c8c56c..b08bcd3330e 100644 --- a/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 +++ b/tests/queries/0_stateless/01720_join_implicit_cast.sql.j2 @@ -47,7 +47,6 @@ SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 RIGHT JOIN t_ab2 USING (a, b); SELECT any(toTypeName(a)) == 'Int32' AND any(toTypeName(b)) == 'Nullable(Int64)' FROM t_ab1 INNER JOIN t_ab2 USING (a, b); -SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 ON (t_ab1.a == t_ab2.a); -- { serverError 44 } SELECT * FROM ( SELECT a, b as "_CAST(a, Int32)" FROM t_ab1 ) t_ab1 FULL JOIN t_ab2 USING (a) FORMAT Null; {% endfor %} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index d2cc066a1b1..cda2b3e08ca 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2_nullable; @@ -31,7 +33,6 @@ SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '--'; SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT '--'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; @@ -71,8 +72,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index 1704fedb92b..c1e9bdb4dc9 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2_nullable; @@ -68,8 +70,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } diff --git a/tests/queries/0_stateless/02233_interpolate_1.sql b/tests/queries/0_stateless/02233_interpolate_1.sql index b08f4a23898..1924ae1fcad 100644 --- a/tests/queries/0_stateless/02233_interpolate_1.sql +++ b/tests/queries/0_stateless/02233_interpolate_1.sql @@ -38,7 +38,7 @@ SELECT n, source, inter FROM ( # Test INTERPOLATE with inconsistent column - should produce error SELECT n, source, inter FROM ( SELECT toFloat32(number % 10) AS n, 'original' AS source, number as inter FROM numbers(10) WHERE number % 3 = 1 -) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 6 } +) ORDER BY n WITH FILL FROM 0 TO 11.51 STEP 0.5 INTERPOLATE (inter AS source); -- { serverError 32 } # Test INTERPOLATE with aliased column SELECT n, source, inter + 1 AS inter_p FROM ( From 242552b019a6718855a7d7527369d06038fc5263 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 1 Mar 2023 18:39:48 +0100 Subject: [PATCH 170/470] Update threadPoolCallbackRunner.h --- src/Interpreters/threadPoolCallbackRunner.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 1d2f67e8e59..c79bc270f41 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -30,6 +30,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & /// Releasing has to use proper memory tracker which has been set here before callback [[maybe_unused]] auto tmp = std::move(callback); + thread_name.clear(); } if (thread_group) From 6537029cccd57d51a9125f8d56e5d0ac3c7cc5b6 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Thu, 2 Mar 2023 02:15:09 +0800 Subject: [PATCH 171/470] fix: keeper systemd service file include invalid inline comment --- packages/clickhouse-keeper.service | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/clickhouse-keeper.service b/packages/clickhouse-keeper.service index 2809074c93a..e4ec5bf4ede 100644 --- a/packages/clickhouse-keeper.service +++ b/packages/clickhouse-keeper.service @@ -14,7 +14,8 @@ User=clickhouse Group=clickhouse Restart=always RestartSec=30 -RuntimeDirectory=%p # %p is resolved to the systemd unit name +# %p is resolved to the systemd unit name +RuntimeDirectory=%p ExecStart=/usr/bin/clickhouse-keeper --config=/etc/clickhouse-keeper/keeper_config.xml --pid-file=%t/%p/%p.pid # Minus means that this file is optional. EnvironmentFile=-/etc/default/%p From f3e3b916c913fee844ab9b564112289bb12130f9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 Mar 2023 19:53:01 +0100 Subject: [PATCH 172/470] Allow nested custom disks --- src/Disks/getOrCreateDiskFromAST.cpp | 119 ++++++++++++------ src/Disks/getOrCreateDiskFromAST.h | 2 +- src/Parsers/FieldFromAST.cpp | 87 ++++++++----- src/Storages/MergeTree/MergeTreeSettings.cpp | 3 +- .../test_disk_configuration/test.py | 52 ++++++++ ...54_create_table_with_custom_disk.reference | 1 + .../02454_create_table_with_custom_disk.sql | 10 ++ 7 files changed, 204 insertions(+), 70 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 997bd2c853f..4479b532fb9 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -6,9 +6,14 @@ #include #include #include +#include +#include +#include #include #include #include +#include +#include namespace DB { @@ -18,48 +23,90 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) +namespace { - /// 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, true); - auto disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - - LOG_TRACE( - &Poco::Logger::get("getOrCreateDiskFromDiskAST"), - "Using disk name `{}` for custom disk {}", - disk_name, disk_setting_string); - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(disk_name, function_args, context); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isRemote()) + std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) { - static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); + /// 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, true); + auto disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - 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_disks_base_dir_in_config); + LOG_TRACE( + &Poco::Logger::get("getOrCreateDiskFromDiskAST"), + "Using disk name `{}` for custom disk {}", + disk_name, disk_setting_string); - 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); + auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(disk_name, function_args, context); + auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(); + return disk; + }); + + if (!result_disk->isRemote()) + { + static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_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_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; + }; + + 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); + ast = std::make_shared(disk_name); + } + } + }; + + /// Visits children first. + using FlattenDiskConfigurationVisitor = InDepthNodeVisitor; +} + + +std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context) +{ + if (!isDiskFunction(disk_function)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected a disk function"); + + auto ast = disk_function->clone(); + + FlattenDiskConfigurationVisitor::Data data{context}; + FlattenDiskConfigurationVisitor{data}.visit(ast); + + auto disk_name = assert_cast(*ast).value.get(); + LOG_TRACE(&Poco::Logger::get("getOrCreateDiskFromDiskAST"), "Result disk name: {}", disk_name); return disk_name; } diff --git a/src/Disks/getOrCreateDiskFromAST.h b/src/Disks/getOrCreateDiskFromAST.h index 7c64707b0bd..0195f575278 100644 --- a/src/Disks/getOrCreateDiskFromAST.h +++ b/src/Disks/getOrCreateDiskFromAST.h @@ -13,6 +13,6 @@ class ASTFunction; * add it to DiskSelector by a unique (but always the same for given configuration) disk name * and return this name. */ -std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context); +std::string getOrCreateDiskFromDiskAST(const ASTPtr & disk_function, ContextPtr context); } diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index 3cd10c1cf80..cdfd9c627bc 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -5,7 +5,10 @@ #include #include #include +#include +#include #include +#include namespace DB @@ -31,42 +34,64 @@ bool FieldFromASTImpl::isSecret() const return isDiskFunction(ast); } +class DiskConfigurationHider +{ +public: + struct Data {}; + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data &) + { + if (isDiskFunction(ast)) + { + const auto & disk_function = assert_cast(*ast); + const auto * disk_function_args_expr = assert_cast(disk_function.arguments.get()); + const auto & disk_function_args = disk_function_args_expr->children; + + auto is_secret_arg = [](const std::string & arg_name) + { + /// We allow to not hide type of the disk, e.g. disk(type = s3, ...) + /// and also nested disk, e.g. disk(type = cache, disk = disk(type = s3, ...)) + return arg_name != "type" && arg_name != "disk"; + }; + + for (const auto & arg : disk_function_args) + { + auto * setting_function = arg->as(); + if (!setting_function || setting_function->name != "equals") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected equals function"); + + auto * function_args_expr = assert_cast(setting_function->arguments.get()); + if (!function_args_expr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected arguments"); + + auto & function_args = function_args_expr->children; + if (function_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected non zero number of arguments"); + + auto * key_identifier = function_args[0]->as(); + if (!key_identifier) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected Identifier"); + + const std::string & key = key_identifier->name(); + if (is_secret_arg(key)) + function_args[1] = std::make_shared("[HIDDEN]"); + } + } + } +}; + +/// Visits children first. +using HideDiskConfigurationVisitor = InDepthNodeVisitor; + String FieldFromASTImpl::toString(bool show_secrets) const { if (!show_secrets && isDiskFunction(ast)) { auto hidden = ast->clone(); - const auto & disk_function = assert_cast(*hidden); - const auto * disk_function_args_expr = assert_cast(disk_function.arguments.get()); - const auto & disk_function_args = disk_function_args_expr->children; - - auto is_secret_arg = [](const std::string & arg_name) - { - return arg_name != "type"; - }; - - for (const auto & arg : disk_function_args) - { - auto * setting_function = arg->as(); - if (!setting_function || setting_function->name != "equals") - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected equals function"); - - auto * function_args_expr = assert_cast(setting_function->arguments.get()); - if (!function_args_expr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected arguments"); - - auto & function_args = function_args_expr->children; - if (function_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected non zero number of arguments"); - - auto * key_identifier = function_args[0]->as(); - if (!key_identifier) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad format: expected Identifier"); - - const std::string & key = key_identifier->name(); - if (is_secret_arg(key)) - function_args[1] = std::make_shared("[HIDDEN]"); - } + HideDiskConfigurationVisitor::Data data{}; + HideDiskConfigurationVisitor{data}.visit(hidden); return serializeAST(*hidden); } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index e951b8f54cf..479e50fdebb 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -64,8 +64,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte auto ast = dynamic_cast(custom.getImpl()).ast; if (ast && isDiskFunction(ast)) { - const auto & ast_function = assert_cast(*ast); - auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); + auto disk_name = getOrCreateDiskFromDiskAST(ast, context); LOG_TRACE(&Poco::Logger::get("MergeTreeSettings"), "Created custom disk {}", disk_name); value = disk_name; } diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index 34f8bea219f..96cdb0213bc 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -295,6 +295,58 @@ def test_merge_tree_custom_disk_setting(start_cluster): ) +def test_merge_tree_nested_custom_disk_setting(start_cluster): + node = cluster.instances["node1"] + + minio = cluster.minio_client + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): + minio.remove_object(cluster.minio_bucket, obj.object_name) + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == 0 + ) + + node.query( + """ + DROP TABLE IF EXISTS test; + CREATE TABLE test (a Int32) + ENGINE = MergeTree() order by tuple() + SETTINGS disk = disk( + type=cache, + max_size='1Gi', + path='/var/lib/clickhouse/custom_disk_cache/', + disk=disk( + type=s3, + endpoint='http://minio1:9001/root/data/', + access_key_id='minio', + secret_access_key='minio123')); + """ + ) + + node.query("INSERT INTO test SELECT number FROM numbers(100)") + node.query("SYSTEM DROP FILESYSTEM CACHE") + + # Check cache is filled + assert 0 == int(node.query("SELECT count() FROM system.filesystem_cache")) + assert 100 == int(node.query("SELECT count() FROM test")) + node.query("SELECT * FROM test") + assert 0 < int(node.query("SELECT count() FROM system.filesystem_cache")) + + # Check s3 is filled + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) > 0 + ) + + node.restart_clickhouse() + + assert 100 == int(node.query("SELECT count() FROM test")) + + expected = """ + SETTINGS disk = disk(type = cache, max_size = \\'[HIDDEN]\\', path = \\'[HIDDEN]\\', disk = disk(type = s3, endpoint = \\'[HIDDEN]\\' + """ + assert expected.strip() in node.query(f"SHOW CREATE TABLE test").strip() + + def test_merge_tree_setting_override(start_cluster): node = cluster.instances["node3"] assert ( diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference index 1d8610c59c9..a71d52b6f57 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -9,3 +9,4 @@ SETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\') CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'[HIDDEN]\'), index_granularity = 8192 a Int32 200 +CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = cache, max_size = \'[HIDDEN]\', path = \'[HIDDEN]\', disk = disk(type = local, path = \'[HIDDEN]\')), index_granularity = 8192 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 6cb1c0774aa..4909f91ccb1 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -26,3 +26,13 @@ DESCRIBE TABLE test; INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; + +DROP TABLE test; + +CREATE TABLE test (a Int32) +ENGINE = MergeTree() order by tuple() +SETTINGS disk = disk(type=cache, max_size='1Gi', path='/var/lib/clickhouse/custom_disk_cache/', disk=disk(type=local, path='/var/lib/clickhouse/disks/local/')); + +SHOW CREATE TABLE test; + +DROP TABLE test; From 13d50509c0b56fbef24132e4687122be633c6983 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 1 Mar 2023 20:16:15 +0100 Subject: [PATCH 173/470] Fix style check --- src/Disks/getOrCreateDiskFromAST.cpp | 1 - src/Parsers/FieldFromAST.cpp | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 4479b532fb9..9abc4a7d413 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index cdfd9c627bc..b2251599441 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -5,8 +5,6 @@ #include #include #include -#include -#include #include #include From 9f6b154ccfeb498ba18373c5fe66c6e1080a48ea Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 1 Mar 2023 20:51:23 +0100 Subject: [PATCH 174/470] Insignificant improvement --- tests/ci/terminate_runner_lambda/app.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index ee9aadc488d..d72e6384ea2 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -64,19 +64,25 @@ def list_runners(access_token: str) -> RunnerDescriptions: "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", } + per_page = 100 response = requests.get( - "https://api.github.com/orgs/ClickHouse/actions/runners?per_page=100", + f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}", headers=headers, ) response.raise_for_status() data = response.json() total_runners = data["total_count"] + print("Expected total runners", total_runners) runners = data["runners"] - total_pages = int(total_runners / 100 + 1) + # round to 0 for 0, 1 for 1..100, but to 2 for 101..200 + total_pages = (total_runners - 1) // per_page + 1 + + print("Total pages", total_pages) for i in range(2, total_pages + 1): response = requests.get( - f"https://api.github.com/orgs/ClickHouse/actions/runners?page={i}&per_page=100", + "https://api.github.com/orgs/ClickHouse/actions/runners" + f"?page={i}&per_page={per_page}", headers=headers, ) response.raise_for_status() @@ -95,6 +101,7 @@ def list_runners(access_token: str) -> RunnerDescriptions: busy=runner["busy"], ) result.append(desc) + return result From cda4ff0df73d9c4bcdd759c0e1dfe6ebe633b82e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Mar 2023 19:54:22 +0000 Subject: [PATCH 175/470] Remove trace logs --- tests/config/config.d/keeper_port.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 03f80bfbe62..cffd325e968 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -9,13 +9,11 @@ 10000 100000 10000 - false 240000 1000000000000000 100000 - trace 0 From 76451745bcaa4bfa4a224b9ea72c44eece5eab72 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 1 Mar 2023 20:56:38 +0100 Subject: [PATCH 176/470] Update threadPoolCallbackRunner.h --- src/Interpreters/threadPoolCallbackRunner.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index c79bc270f41..1d2f67e8e59 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -30,7 +30,6 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & /// Releasing has to use proper memory tracker which has been set here before callback [[maybe_unused]] auto tmp = std::move(callback); - thread_name.clear(); } if (thread_group) From e17c0b6bf1e7cfe03e0d3f42a5398aee4ec18edb Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 1 Mar 2023 20:19:51 +0000 Subject: [PATCH 177/470] Review fixes --- src/Planner/PlannerJoinTree.cpp | 30 ++++++++----------- .../02674_trivial_count_analyzer.reference | 5 ++-- .../02674_trivial_count_analyzer.sql | 5 ++-- 3 files changed, 16 insertions(+), 24 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 11944b4b71d..26fc2764a64 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1,6 +1,7 @@ #include #include +#include #include @@ -19,6 +20,8 @@ #include #include #include +#include +#include #include #include @@ -27,6 +30,7 @@ #include #include #include +#include #include #include @@ -41,12 +45,8 @@ #include #include -#include -#include #include #include -#include -#include namespace DB { @@ -174,23 +174,17 @@ bool applyTrivialCountIfPossible( if (!storage || storage->hasLightweightDeletedMask()) return false; - if (settings.max_parallel_replicas > 1 || // - settings.allow_experimental_query_deduplication || // - settings.empty_result_for_aggregation_by_empty_set) + if (settings.max_parallel_replicas > 1 || settings.allow_experimental_query_deduplication + || settings.empty_result_for_aggregation_by_empty_set) return false; QueryTreeNodes aggregates = collectAggregateFunctionNodes(query_tree); if (aggregates.size() != 1) return false; - const auto * function_node = typeid_cast(aggregates.front().get()); - if (!function_node) - return false; - - if (!function_node->getAggregateFunction()) - return false; - - const auto * count_func = typeid_cast(function_node->getAggregateFunction().get()); + const auto & function_node = aggregates.front().get()->as(); + chassert(function_node.getAggregateFunction() != nullptr); + const auto * count_func = typeid_cast(function_node.getAggregateFunction().get()); if (!count_func) return false; @@ -226,7 +220,7 @@ bool applyTrivialCountIfPossible( SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place)); agg_count.set(place, num_rows.value()); - auto column = ColumnAggregateFunction::create(function_node->getAggregateFunction()); + auto column = ColumnAggregateFunction::create(function_node.getAggregateFunction()); column->insertFrom(place); /// get count() argument type @@ -240,7 +234,7 @@ bool applyTrivialCountIfPossible( Block block_with_count{ {std::move(column), - std::make_shared(function_node->getAggregateFunction(), argument_types, Array{}), + std::make_shared(function_node.getAggregateFunction(), argument_types, Array{}), columns_names.front()}}; auto source = std::make_shared(block_with_count); @@ -416,7 +410,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } } - /// apply trivial_count optimization if possible + /// Apply trivial_count optimization if possible bool is_trivial_count_applied = is_single_table_expression && table_node && select_query_info.has_aggregates && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), columns_names); diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.reference b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference index 2a94fd59d7b..05feadb58a0 100644 --- a/tests/queries/0_stateless/02674_trivial_count_analyzer.reference +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.reference @@ -18,8 +18,7 @@ select count(b) from m3; 2 select count() + 1 from m3; 3 --- drop table m3; - +drop table m3; -- checking queries with FINAL create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b); SYSTEM STOP MERGES replacing_m3; @@ -39,10 +38,10 @@ select count(a) from replacing_m3; 4 select count(b) from replacing_m3; 4 -set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL select count() from replacing_m3 FINAL; 3 select count(a) from replacing_m3 FINAL; 3 select count(b) from replacing_m3 FINAL; 3 +drop table replacing_m3; diff --git a/tests/queries/0_stateless/02674_trivial_count_analyzer.sql b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql index d4a686e6eff..988d1b9ba92 100644 --- a/tests/queries/0_stateless/02674_trivial_count_analyzer.sql +++ b/tests/queries/0_stateless/02674_trivial_count_analyzer.sql @@ -19,7 +19,7 @@ select count(a) from m3; select count(b) from m3; select count() + 1 from m3; --- drop table m3; +drop table m3; -- checking queries with FINAL create table replacing_m3(a Int64, b UInt64) Engine=ReplacingMergeTree() order by (a, b); @@ -38,9 +38,8 @@ select count(*) from replacing_m3; select count(a) from replacing_m3; select count(b) from replacing_m3; -set optimize_trivial_count_query=0; -- FIXME: wrong result for queries with FINAL select count() from replacing_m3 FINAL; select count(a) from replacing_m3 FINAL; select count(b) from replacing_m3 FINAL; --- drop table replacing_m3; +drop table replacing_m3; From 169398ec07d0db3d7812526db00aa1c3d4481fb2 Mon Sep 17 00:00:00 2001 From: Alexey Korepanov Date: Wed, 1 Mar 2023 21:57:51 +0100 Subject: [PATCH 178/470] Add check for empty leading spaces --- programs/server/play.html | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/server/play.html b/programs/server/play.html index d90293fbd1b..8c3dd0c1f5b 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -687,6 +687,10 @@ const prev_lines = text_before_cursor.split('\n'); const prev_line = prev_lines.pop(); const lead_spaces = prev_line.match(/^\s*/)[0]; + if (!lead_spaces) { + return; + } + // Add leading spaces to the current line. elem.value = text_before_cursor + '\n' + lead_spaces + text_after_cursor; elem.selectionStart = cursor_pos + lead_spaces.length + 1; From 5ab5902f38906254af902d01cc71851f6cb78401 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 1 Mar 2023 21:27:46 +0000 Subject: [PATCH 179/470] Allow control compression in Parquet/ORC/Arrow output formats, support more compression for input formats --- contrib/arrow-cmake/CMakeLists.txt | 20 +++++++++- contrib/orc | 2 +- src/Core/Settings.h | 3 ++ src/Core/SettingsEnums.cpp | 23 +++++++++++- src/Core/SettingsEnums.h | 6 +++ src/Formats/FormatFactory.cpp | 3 ++ src/Formats/FormatSettings.h | 29 +++++++++++++++ .../Formats/Impl/ArrowBlockOutputFormat.cpp | 26 ++++++++++++- .../Formats/Impl/ORCBlockOutputFormat.cpp | 32 +++++++++++++++- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 37 +++++++++++++++++-- 10 files changed, 171 insertions(+), 10 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index ae6f270a768..4181f916d63 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -115,6 +115,13 @@ configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/A # ARROW_ORC + adapters/orc/CMakefiles set(ORC_SRCS + "${CMAKE_CURRENT_BINARY_DIR}/orc_proto.pb.h" + "${ORC_SOURCE_SRC_DIR}/sargs/ExpressionTree.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/Literal.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/PredicateLeaf.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/SargsApplier.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/SearchArgument.cc" + "${ORC_SOURCE_SRC_DIR}/sargs/TruthValue.cc" "${ORC_SOURCE_SRC_DIR}/Exceptions.cc" "${ORC_SOURCE_SRC_DIR}/OrcFile.cc" "${ORC_SOURCE_SRC_DIR}/Reader.cc" @@ -129,13 +136,20 @@ set(ORC_SRCS "${ORC_SOURCE_SRC_DIR}/MemoryPool.cc" "${ORC_SOURCE_SRC_DIR}/RLE.cc" "${ORC_SOURCE_SRC_DIR}/RLEv1.cc" - "${ORC_SOURCE_SRC_DIR}/RLEv2.cc" + "${ORC_SOURCE_SRC_DIR}/RleDecoderV2.cc" + "${ORC_SOURCE_SRC_DIR}/RleEncoderV2.cc" + "${ORC_SOURCE_SRC_DIR}/RLEV2Util.cc" "${ORC_SOURCE_SRC_DIR}/Statistics.cc" "${ORC_SOURCE_SRC_DIR}/StripeStream.cc" "${ORC_SOURCE_SRC_DIR}/Timezone.cc" "${ORC_SOURCE_SRC_DIR}/TypeImpl.cc" "${ORC_SOURCE_SRC_DIR}/Vector.cc" "${ORC_SOURCE_SRC_DIR}/Writer.cc" + "${ORC_SOURCE_SRC_DIR}/Adaptor.cc" + "${ORC_SOURCE_SRC_DIR}/BloomFilter.cc" + "${ORC_SOURCE_SRC_DIR}/Murmur3.cc" + "${ORC_SOURCE_SRC_DIR}/BlockBuffer.cc" + "${ORC_SOURCE_SRC_DIR}/wrap/orc-proto-wrapper.cc" "${ORC_SOURCE_SRC_DIR}/io/InputStream.cc" "${ORC_SOURCE_SRC_DIR}/io/OutputStream.cc" "${ORC_ADDITION_SOURCE_DIR}/orc_proto.pb.cc" @@ -358,6 +372,9 @@ SET(ARROW_SRCS "${LIBRARY_DIR}/util/compression_zlib.cc" ${ARROW_SRCS}) add_definitions(-DARROW_WITH_ZSTD) SET(ARROW_SRCS "${LIBRARY_DIR}/util/compression_zstd.cc" ${ARROW_SRCS}) +add_definitions(-DARROW_WITH_BROTLI) +SET(ARROW_SRCS "${LIBRARY_DIR}/util/compression_brotli.cc" ${ARROW_SRCS}) + add_library(_arrow ${ARROW_SRCS}) @@ -372,6 +389,7 @@ target_link_libraries(_arrow PRIVATE ch_contrib::snappy ch_contrib::zlib ch_contrib::zstd + ch_contrib::brotli ) target_link_libraries(_arrow PUBLIC _orc) diff --git a/contrib/orc b/contrib/orc index f9a393ed243..c5d7755ba0b 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit f9a393ed2433a60034795284f82d093b348f2102 +Subproject commit c5d7755ba0b9a95631c8daea4d094101f26ec761 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3908254b6f1..8d3e787f42c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -857,6 +857,7 @@ class IColumn; M(Bool, output_format_parquet_string_as_string, false, "Use Parquet String type instead of Binary for String columns.", 0) \ M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \ M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ + M(ParquetCompression, output_format_parquet_compression_method, "snappy", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ @@ -899,8 +900,10 @@ class IColumn; M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ M(Bool, output_format_arrow_string_as_string, false, "Use Arrow String type instead of Binary for String columns", 0) \ M(Bool, output_format_arrow_fixed_string_as_fixed_byte_array, true, "Use Arrow FIXED_SIZE_BINARY type instead of Binary for FixedString columns.", 0) \ + M(ArrowCompression, output_format_arrow_compression_method, "none", "Compression method for Arrow output format. Supported codecs: lz4_frame, zstd, none (uncompressed)", 0) \ \ M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ + M(ORCCompression, output_format_orc_compression_method, "none", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ \ M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 9e1ab585bb0..91572aa1b3f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -158,7 +158,7 @@ IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, {"XML", FormatSettings::EscapingRule::XML}, {"Raw", FormatSettings::EscapingRule::Raw}}) -IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation , ErrorCodes::BAD_ARGUMENTS, +IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation, ErrorCodes::BAD_ARGUMENTS, {{"bin", FormatSettings::MsgPackUUIDRepresentation::BIN}, {"str", FormatSettings::MsgPackUUIDRepresentation::STR}, {"ext", FormatSettings::MsgPackUUIDRepresentation::EXT}}) @@ -172,11 +172,30 @@ IMPLEMENT_SETTING_ENUM(LocalFSReadMethod, ErrorCodes::BAD_ARGUMENTS, {"pread", LocalFSReadMethod::pread}, {"read", LocalFSReadMethod::read}}) - IMPLEMENT_SETTING_ENUM_WITH_RENAME(ParquetVersion, ErrorCodes::BAD_ARGUMENTS, {{"1.0", FormatSettings::ParquetVersion::V1_0}, {"2.4", FormatSettings::ParquetVersion::V2_4}, {"2.6", FormatSettings::ParquetVersion::V2_6}, {"2.latest", FormatSettings::ParquetVersion::V2_LATEST}}) +IMPLEMENT_SETTING_ENUM(ParquetCompression, ErrorCodes::BAD_ARGUMENTS, + {{"none", FormatSettings::ParquetCompression::NONE}, + {"snappy", FormatSettings::ParquetCompression::SNAPPY}, + {"zstd", FormatSettings::ParquetCompression::ZSTD}, + {"gzip", FormatSettings::ParquetCompression::GZIP}, + {"lz4", FormatSettings::ParquetCompression::LZ4}, + {"brotli", FormatSettings::ParquetCompression::BROTLI}}) + +IMPLEMENT_SETTING_ENUM(ArrowCompression, ErrorCodes::BAD_ARGUMENTS, + {{"none", FormatSettings::ArrowCompression::NONE}, + {"lz4_frame", FormatSettings::ArrowCompression::LZ4_FRAME}, + {"zstd", FormatSettings::ArrowCompression::ZSTD}}) + +IMPLEMENT_SETTING_ENUM(ORCCompression, ErrorCodes::BAD_ARGUMENTS, + {{"none", FormatSettings::ORCCompression::NONE}, + {"snappy", FormatSettings::ORCCompression::SNAPPY}, + {"zstd", FormatSettings::ORCCompression::ZSTD}, + {"zlib", FormatSettings::ORCCompression::ZLIB}, + {"lz4", FormatSettings::ORCCompression::LZ4}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 139a04f3a5a..14e952bbd65 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -194,6 +194,12 @@ DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) DECLARE_SETTING_ENUM_WITH_RENAME(MsgPackUUIDRepresentation, FormatSettings::MsgPackUUIDRepresentation) +DECLARE_SETTING_ENUM_WITH_RENAME(ParquetCompression, FormatSettings::ParquetCompression) + +DECLARE_SETTING_ENUM_WITH_RENAME(ArrowCompression, FormatSettings::ArrowCompression) + +DECLARE_SETTING_ENUM_WITH_RENAME(ORCCompression, FormatSettings::ORCCompression) + enum class Dialect { clickhouse, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index aca3166a8c4..7f14810b260 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -118,6 +118,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; + format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -158,6 +159,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.case_insensitive_column_matching = settings.input_format_arrow_case_insensitive_column_matching; format_settings.arrow.output_string_as_string = settings.output_format_arrow_string_as_string; format_settings.arrow.output_fixed_string_as_fixed_byte_array = settings.output_format_arrow_fixed_string_as_fixed_byte_array; + format_settings.arrow.output_compression_method = settings.output_format_arrow_compression_method; format_settings.orc.import_nested = settings.input_format_orc_import_nested; format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; @@ -168,6 +170,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_orc_skip_columns_with_unsupported_types_in_schema_inference; format_settings.orc.case_insensitive_column_matching = settings.input_format_orc_case_insensitive_column_matching; format_settings.orc.output_string_as_string = settings.output_format_orc_string_as_string; + format_settings.orc.output_compression_method = settings.output_format_orc_compression_method; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d1755a35c5f..88a5adbc8df 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -86,6 +86,13 @@ struct FormatSettings UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; + enum class ArrowCompression + { + NONE, + LZ4_FRAME, + ZSTD + }; + struct { UInt64 row_group_size = 1000000; @@ -96,6 +103,7 @@ struct FormatSettings bool case_insensitive_column_matching = false; bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; + ArrowCompression output_compression_method = ArrowCompression::NONE; } arrow; struct @@ -183,6 +191,16 @@ struct FormatSettings V2_LATEST, }; + enum class ParquetCompression + { + NONE, + SNAPPY, + ZSTD, + LZ4, + GZIP, + BROTLI, + }; + struct { UInt64 row_group_size = 1000000; @@ -195,6 +213,7 @@ struct FormatSettings bool output_fixed_string_as_fixed_byte_array = true; UInt64 max_block_size = 8192; ParquetVersion output_version; + ParquetCompression output_compression_method = ParquetCompression::SNAPPY; } parquet; struct Pretty @@ -276,6 +295,15 @@ struct FormatSettings bool accurate_types_of_literals = true; } values; + enum class ORCCompression + { + NONE, + LZ4, + SNAPPY, + ZSTD, + ZLIB, + }; + struct { bool import_nested = false; @@ -285,6 +313,7 @@ struct FormatSettings bool case_insensitive_column_matching = false; std::unordered_set skip_stripes = {}; bool output_string_as_string = false; + ORCCompression output_compression_method = ORCCompression::NONE; } orc; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index bf0e2448082..ec35c52e37c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -8,6 +8,7 @@ #include #include "ArrowBufferedStreams.h" #include "CHColumnToArrowColumn.h" +#include "config.h" namespace DB @@ -17,6 +18,25 @@ namespace ErrorCodes extern const int UNKNOWN_EXCEPTION; } +namespace +{ + +arrow::Compression::type getArrowCompression(FormatSettings::ArrowCompression method) +{ + switch (method) + { + case FormatSettings::ArrowCompression::NONE: + return arrow::Compression::type::UNCOMPRESSED; + case FormatSettings::ArrowCompression::ZSTD: + return arrow::Compression::type::ZSTD; + case FormatSettings::ArrowCompression::LZ4_FRAME: + return arrow::Compression::type::LZ4_FRAME; + } +} + +} + + ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_) , stream{stream_} @@ -78,12 +98,14 @@ void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr { arrow_ostream = std::make_shared(out); arrow::Result> writer_status; + arrow::ipc::IpcWriteOptions options = arrow::ipc::IpcWriteOptions::Defaults(); + options.codec = *arrow::util::Codec::Create(getArrowCompression(format_settings.arrow.output_compression_method)); // TODO: should we use arrow::ipc::IpcOptions::alignment? if (stream) - writer_status = arrow::ipc::MakeStreamWriter(arrow_ostream.get(), schema); + writer_status = arrow::ipc::MakeStreamWriter(arrow_ostream.get(), schema, options); else - writer_status = arrow::ipc::MakeFileWriter(arrow_ostream.get(), schema); + writer_status = arrow::ipc::MakeFileWriter(arrow_ostream.get(), schema,options); if (!writer_status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 42c3e178436..ecb7c2fbc92 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -22,12 +22,42 @@ #include #include +#include "config.h" + namespace DB { namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; +} + +namespace +{ + +orc::CompressionKind getORCCompression(FormatSettings::ORCCompression method) +{ + if (method == FormatSettings::ORCCompression::NONE) + return orc::CompressionKind::CompressionKind_NONE; + +#if USE_SNAPPY + if (method == FormatSettings::ORCCompression::SNAPPY) + return orc::CompressionKind::CompressionKind_SNAPPY; +#endif + + if (method == FormatSettings::ORCCompression::ZSTD) + return orc::CompressionKind::CompressionKind_ZSTD; + + if (method == FormatSettings::ORCCompression::LZ4) + return orc::CompressionKind::CompressionKind_LZ4; + + if (method == FormatSettings::ORCCompression::ZLIB) + return orc::CompressionKind::CompressionKind_ZLIB; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); +} + } ORCOutputStream::ORCOutputStream(WriteBuffer & out_) : out(out_) {} @@ -529,7 +559,7 @@ void ORCBlockOutputFormat::prepareWriter() { const Block & header = getPort(PortKind::Main).getHeader(); schema = orc::createStructType(); - options.setCompression(orc::CompressionKind::CompressionKind_NONE); + options.setCompression(getORCCompression(format_settings.orc.output_compression_method)); size_t columns_count = header.columns(); for (size_t i = 0; i != columns_count; ++i) schema->addStructField(header.safeGetByPosition(i).name, getORCType(recursiveRemoveLowCardinality(data_types[i]))); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 18c81f8fd6a..742912df980 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -16,6 +16,9 @@ namespace ErrorCodes extern const int UNKNOWN_EXCEPTION; } +namespace +{ + static parquet::ParquetVersion::type getParquetVersion(const FormatSettings & settings) { switch (settings.parquet.output_version) @@ -31,6 +34,36 @@ static parquet::ParquetVersion::type getParquetVersion(const FormatSettings & se } } +parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompression method) +{ + if (method == FormatSettings::ParquetCompression::NONE) + return parquet::Compression::type::UNCOMPRESSED; + +#if USE_SNAPPY + if (method == FormatSettings::ParquetCompression::SNAPPY) + return parquet::Compression::type::SNAPPY; +#endif + +#if USE_BROTLI + if (method == FormatSettings::ParquetCompression::BROTLI) + return parquet::Compression::type::BROTLI; +#endif + + if (method == FormatSettings::ParquetCompression::ZSTD) + return parquet::Compression::type::ZSTD; + + if (method == FormatSettings::ParquetCompression::LZ4) + return parquet::Compression::type::LZ4; + + if (method == FormatSettings::ParquetCompression::GZIP) + return parquet::Compression::type::GZIP; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); +} + + +} + ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_), format_settings{format_settings_} { @@ -60,9 +93,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) parquet::WriterProperties::Builder builder; builder.version(getParquetVersion(format_settings)); -#if USE_SNAPPY - builder.compression(parquet::Compression::SNAPPY); -#endif + builder.compression(getParquetCompression(format_settings.parquet.output_compression_method)); auto props = builder.build(); auto status = parquet::arrow::FileWriter::Open( *arrow_table->schema(), From 4f33e95c96af0b4cf9e1ad9ab3b92d1cfebf72e4 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 1 Mar 2023 21:28:37 +0000 Subject: [PATCH 180/470] Add tests --- ...1_parquet_arrow_orc_compressions.reference | 14 +++++++++++ .../02581_parquet_arrow_orc_compressions.sh | 24 +++++++++++++++++++ 2 files changed, 38 insertions(+) create mode 100644 tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.reference create mode 100755 tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh diff --git a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.reference b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.reference new file mode 100644 index 00000000000..492b12dba56 --- /dev/null +++ b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.reference @@ -0,0 +1,14 @@ +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 diff --git a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh new file mode 100755 index 00000000000..10850e17bea --- /dev/null +++ b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='none'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='lz4'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='snappy'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='zstd'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='brotli'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='gzip'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" + +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format ORC settings output_format_orc_compression_method='none'" | $CLICKHOUSE_LOCAL --input-format=ORC -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format ORC settings output_format_orc_compression_method='lz4'" | $CLICKHOUSE_LOCAL --input-format=ORC -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format ORC settings output_format_orc_compression_method='zstd'" | $CLICKHOUSE_LOCAL --input-format=ORC -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format ORC settings output_format_orc_compression_method='zlib'" | $CLICKHOUSE_LOCAL --input-format=ORC -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format ORC settings output_format_orc_compression_method='snappy'" | $CLICKHOUSE_LOCAL --input-format=ORC -q "select count() from table" + + +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Arrow settings output_format_arrow_compression_method='none'" | $CLICKHOUSE_LOCAL --input-format=Arrow -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Arrow settings output_format_arrow_compression_method='lz4_frame'" | $CLICKHOUSE_LOCAL --input-format=Arrow -q "select count() from table" +$CLICKHOUSE_LOCAL -q "select * from numbers(10) format Arrow settings output_format_arrow_compression_method='zstd'" | $CLICKHOUSE_LOCAL --input-format=Arrow -q "select count() from table" + From 657afa849e8ab3140ebe3ce21ae7f05ef249a82a Mon Sep 17 00:00:00 2001 From: AndyB Date: Wed, 1 Mar 2023 23:35:09 +0000 Subject: [PATCH 181/470] logger level in test --- tests/integration/test_log_levels_update/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py index 176733cd7cb..842c7914eaa 100644 --- a/tests/integration/test_log_levels_update/test.py +++ b/tests/integration/test_log_levels_update/test.py @@ -10,7 +10,7 @@ node = cluster.add_instance( config = """ - information + debug /var/log/clickhouse-server/clickhouse-server.log """ From 0a3a60b7a1ae5fcae0cc9354f22ff1fd781cc657 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 00:56:11 +0100 Subject: [PATCH 182/470] Additionally search for inactive runners in the whole ASG --- tests/ci/terminate_runner_lambda/app.py | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index d72e6384ea2..61daa6af02a 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -156,8 +156,10 @@ def main( # so they will be cleaned out by ci_runners_metrics_lambda eventually instances_to_kill = [] + total_to_kill = 0 for zone, num_to_kill in to_kill_by_zone.items(): candidates = instances_by_zone[zone] + total_to_kill += num_to_kill if num_to_kill > len(candidates): raise Exception( f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}" @@ -195,8 +197,25 @@ def main( instances_to_kill += [runner.name for runner in delete_for_av] - print("Got instances to kill: ", ", ".join(instances_to_kill)) + if len(instances_to_kill) < total_to_kill: + print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}") + client = boto3.client("autoscaling") + as_groups = client.describe_auto_scaling_groups( + AutoScalingGroupNames=[event["AutoScalingGroupName"]] + ) + assert len(as_groups["AutoScalingGroups"]) == 1 + asg = as_groups["AutoScalingGroups"][0] + for instance in asg["Instances"]: + for runner in runners: + if runner.name == instance["InstanceId"] and not runner.busy: + print(f"Runner {runner.name} is not busy and can be deleted") + instances_to_kill.append(runner.name) + if total_to_kill <= len(instances_to_kill): + print("Got enough instances to kill") + break + + print("Got instances to kill: ", ", ".join(instances_to_kill)) response = {"InstanceIDs": instances_to_kill} print(response) return response From 036eaad9190f73f5fca4aee5e67a5293a6b64fdc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 01:29:40 +0100 Subject: [PATCH 183/470] Reduce lambda time by caching access_token --- tests/ci/terminate_runner_lambda/app.py | 58 +++++++++++++++++-------- 1 file changed, 41 insertions(+), 17 deletions(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 61daa6af02a..c9192417575 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -1,10 +1,11 @@ #!/usr/bin/env python3 import argparse -import sys import json +import sys import time from collections import namedtuple +from dataclasses import dataclass from typing import Any, Dict, List, Tuple import boto3 # type: ignore @@ -53,6 +54,32 @@ def get_access_token(jwt_token: str, installation_id: int) -> str: return data["token"] # type: ignore +@dataclass +class CachedToken: + time: int + value: str + + +cached_token = CachedToken(0, "") + + +def get_cached_access_token() -> str: + if time.time() - 500 < cached_token.time: + return cached_token.value + private_key, app_id = get_key_and_app_from_aws() + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": app_id, + } + + encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + cached_token.time = int(time.time()) + cached_token.value = get_access_token(encoded_jwt, installation_id) + return cached_token.value + + RunnerDescription = namedtuple( "RunnerDescription", ["id", "name", "tags", "offline", "busy"] ) @@ -132,23 +159,11 @@ def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: return instances_by_zone -def main( - github_secret_key: str, github_app_id: int, event: dict -) -> Dict[str, List[str]]: +def main(access_token: str, event: dict) -> Dict[str, List[str]]: print("Got event", json.dumps(event, sort_keys=True, indent=4)) to_kill_by_zone = how_many_instances_to_kill(event) instances_by_zone = get_candidates_to_be_killed(event) - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": github_app_id, - } - - encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") - installation_id = get_installation_id(encoded_jwt) - access_token = get_access_token(encoded_jwt, installation_id) - runners = list_runners(access_token) # We used to delete potential hosts to terminate from GitHub runners pool, # but the documentation states: @@ -222,8 +237,7 @@ def main( def handler(event: dict, context: Any) -> Dict[str, List[str]]: - private_key, app_id = get_key_and_app_from_aws() - return main(private_key, app_id, event) + return main(get_cached_access_token(), event) if __name__ == "__main__": @@ -300,4 +314,14 @@ if __name__ == "__main__": "Cause": "SCALE_IN", } - main(private_key, args.app_id, sample_event) + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": args.app_id, + } + + encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + access_token = get_access_token(encoded_jwt, args.app_id) + + main(access_token, sample_event) From 566a0e166fe0f51b21e11cb3977ac4f0e5aac6eb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 2 Mar 2023 00:42:02 +0000 Subject: [PATCH 184/470] preserve uid gid if running with sudo --- .../decompressor.cpp | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 37fbd043814..5f2a769dcdb 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -168,6 +168,24 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n return 0; } +bool getSudoIDs(uid_t &sudo_uid, uid_t &sudo_gid) +{ + sudo_uid = 0; + sudo_gid = 0; + + if (getuid() || geteuid() || getenv("SUDO_USER") == nullptr || getenv("SUDO_UID") == nullptr || getenv("SUDO_GID") == nullptr) + return false; + + char * str_end; + long id = strtol(getenv("SUDO_UID"), &str_end, 10); + if (*str_end == 0) + sudo_uid = static_cast(id); + id = strtol(getenv("SUDO_GID"), &str_end, 10); + if (*str_end == 0) + sudo_gid = static_cast(id); + + return true; +} /// Read data about files and decomrpess them. int decompressFiles(int input_fd, char * path, char * name, bool & have_compressed_analoge, bool & has_exec, char * decompressed_suffix, uint64_t * decompressed_umask) @@ -220,6 +238,10 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress return 1; } + uid_t sudo_uid = 0; + uid_t sudo_gid = 0; + getSudoIDs(sudo_uid, sudo_gid); + FileData file_info; /// Decompress files with appropriate file names for (size_t i = 0; i < le64toh(metadata.number_of_files); ++i) @@ -319,6 +341,9 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress perror("fsync"); if (0 != close(output_fd)) perror("close"); + + if (sudo_uid && sudo_gid) + chown(file_name, sudo_uid, sudo_gid); } if (0 != munmap(input, info_in.st_size)) @@ -532,6 +557,9 @@ int main(int/* argc*/, char* argv[]) return 1; } + if (uid_t sudo_uid = 0, sudo_gid = 0; getSudoIDs(sudo_uid, sudo_gid)) + chown(static_cast(self), sudo_uid, sudo_gid); + if (has_exec) { #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) From 088869b654f70f5f32d1edafac90bceeeafde5b7 Mon Sep 17 00:00:00 2001 From: SupunKavinda Date: Thu, 2 Mar 2023 01:47:00 +0100 Subject: [PATCH 185/470] adds hyvor/clickhouse-php to docs --- docs/en/interfaces/third-party/client-libraries.md | 1 + docs/ru/interfaces/third-party/client-libraries.md | 1 + docs/zh/interfaces/third-party/client-libraries.md | 1 + 3 files changed, 3 insertions(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 4114e47e982..0e065cb7179 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -26,6 +26,7 @@ ClickHouse Inc does **not** maintain the libraries listed below and hasn’t don - [one-ck](https://github.com/lizhichao/one-ck) - [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel) - [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php) + - [hyvor/clickhouse-php](https://github.com/hyvor/clickhouse-php) - Go - [clickhouse](https://github.com/kshvakov/clickhouse/) - [go-clickhouse](https://github.com/roistat/go-clickhouse) diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index 6c63fb4c730..f55bbe2a47d 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -24,6 +24,7 @@ sidebar_label: "Клиентские библиотеки от сторонни - [SeasClick C++ client](https://github.com/SeasX/SeasClick) - [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel) - [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php) + - [hyvor/clickhouse-php](https://github.com/hyvor/clickhouse-php) - Go - [clickhouse](https://github.com/kshvakov/clickhouse/) - [go-clickhouse](https://github.com/roistat/go-clickhouse) diff --git a/docs/zh/interfaces/third-party/client-libraries.md b/docs/zh/interfaces/third-party/client-libraries.md index 1d46c19ce8b..4ee74f1d2c3 100644 --- a/docs/zh/interfaces/third-party/client-libraries.md +++ b/docs/zh/interfaces/third-party/client-libraries.md @@ -24,6 +24,7 @@ Yandex**没有**维护下面列出的库,也没有做过任何广泛的测试 - [SeasClick C++ client](https://github.com/SeasX/SeasClick) - [one-ck](https://github.com/lizhichao/one-ck) - [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel) + - [hyvor/clickhouse-php](https://github.com/hyvor/clickhouse-php) - Go - [clickhouse](https://github.com/kshvakov/clickhouse/) - [go-clickhouse](https://github.com/roistat/go-clickhouse) From ad4a44df52b6b04ac5977d12aa35b099a792133c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Mar 2023 02:59:27 +0100 Subject: [PATCH 186/470] fix --- src/Interpreters/Session.cpp | 32 ++++++++++++++++++- src/Interpreters/Session.h | 3 ++ src/Server/HTTPHandler.cpp | 14 +++++++- .../02435_rollback_cancelled_queries.sh | 12 +++---- 4 files changed, 52 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 7411050aa2d..70d4c0e6ae0 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -140,6 +140,23 @@ public: scheduleCloseSession(session, lock); } + void closeSession(const UUID & user_id, const String & session_id) + { + std::unique_lock lock(mutex); + Key key{user_id, session_id}; + auto it = sessions.find(key); + if (it == sessions.end()) + { + LOG_INFO(log, "Session {} not found for user {}, probably it's already closed", session_id, user_id); + return; + } + + if (!it->second.unique()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot close session {} with refcount {}", session_id, it->second.use_count()); + + sessions.erase(it); + } + private: class SessionKeyHash { @@ -408,7 +425,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: std::shared_ptr new_named_session; bool new_named_session_created = false; std::tie(new_named_session, new_named_session_created) - = NamedSessionsStorage::instance().acquireSession(global_context, user_id.value_or(UUID{}), session_name_, timeout_, session_check_); + = NamedSessionsStorage::instance().acquireSession(global_context, *user_id, session_name_, timeout_, session_check_); auto new_session_context = new_named_session->context; new_session_context->makeSessionContext(); @@ -533,5 +550,18 @@ void Session::releaseSessionID() named_session = nullptr; } +void Session::closeSession(const String & session_id) +{ + if (!user_id) /// User was not authenticated + return; + + /// named_session may be not set due to an early exception + if (!named_session) + return; + + releaseSessionID(); + NamedSessionsStorage::instance().closeSession(*user_id, session_id); +} + } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 0f17c378915..443867806d6 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -77,6 +77,9 @@ public: /// Releases the currently used session ID so it becomes available for reuse by another session. void releaseSessionID(); + /// Closes and removes session + void closeSession(const String & session_id); + private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 702743ef1f0..f468167f782 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -678,7 +679,7 @@ void HTTPHandler::processQuery( std::unique_ptr in; static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version"}; + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"}; Names reserved_param_suffixes; @@ -957,6 +958,14 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse /// In case of exception, send stack trace to client. bool with_stacktrace = false; + /// Close http session (if any) after processing the request + bool close_session = false; + String session_id; + + SCOPE_EXIT_SAFE({ + if (close_session && !session_id.empty()) + session->closeSession(session_id); + }); OpenTelemetry::TracingContextHolderPtr thread_trace_context; SCOPE_EXIT({ @@ -1006,6 +1015,9 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse HTMLForm params(default_settings, request); with_stacktrace = params.getParsed("stacktrace", false); + close_session = params.getParsed("close_session", false); + if (close_session) + session_id = params.get("session_id"); /// FIXME: maybe this check is already unnecessary. /// Workaround. Poco does not detect 411 Length Required case. diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index b639cd5ef70..7c7ef037e02 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -16,22 +16,20 @@ $CLICKHOUSE_CLIENT -q 'create table dedup_test(A Int64) Engine = MergeTree order function insert_data { IMPLICIT=$(( RANDOM % 2 )) - SESSION_ID="${SESSION}_$RANDOM$RANDOM$RANDOM" - TXN_SETTINGS="session_id=$SESSION_ID&throw_on_unsupported_query_inside_transaction=0" + SESSION_ID="${SESSION}_$RANDOM.$RANDOM.$RANDOM" + TXN_SETTINGS="session_id=$SESSION_ID&throw_on_unsupported_query_inside_transaction=0&implicit_transaction=$IMPLICIT" BEGIN="" COMMIT="" SETTINGS="query_id=$ID&$TXN_SETTINGS&max_insert_block_size=110000&min_insert_block_size_rows=110000" if [[ "$IMPLICIT" -eq 0 ]]; then $CLICKHOUSE_CURL -sS -d 'begin transaction' "$CLICKHOUSE_URL&$TXN_SETTINGS" + SETTINGS="$SETTINGS&session_check=1" BEGIN="begin transaction;" COMMIT=$(echo -ne "\n\ncommit") - else - TXN_SETTINGS="$TXN_SETTINGS&implicit_transaction=1" fi - SETTINGS="query_id=$ID&$TXN_SETTINGS&max_insert_block_size=110000&min_insert_block_size_rows=110000" # max_block_size=10000, so external table will contain smaller blocks that will be squashed on insert-select (more chances to catch a bug on query cancellation) - TRASH_SETTINGS="query_id=$ID&$TXN_SETTINGS&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_insert_block_size=110000&max_block_size=10000&min_insert_block_size_bytes=0&min_insert_block_size_rows=110000&max_insert_block_size=110000" + TRASH_SETTINGS="$SETTINGS&input_format_parallel_parsing=0&max_threads=1&max_insert_threads=1&max_block_size=10000&min_insert_block_size_bytes=0" TYPE=$(( RANDOM % 6 )) if [[ "$TYPE" -eq 0 ]]; then @@ -49,7 +47,7 @@ function insert_data fi if [[ "$IMPLICIT" -eq 0 ]]; then - $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS" | grep -Faq "Transaction is not in RUNNING state" && $CLICKHOUSE_CURL -sS -d 'rollback' "$CLICKHOUSE_URL&$TXN_SETTINGS" + $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS&close_session=1" | grep -Faq "Transaction is not in RUNNING state" && $CLICKHOUSE_CURL -sS -d 'rollback' "$CLICKHOUSE_URL&$TXN_SETTINGS" fi } From 11fbb6076b5616287ef635c5eef63dec6919ca82 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 2 Mar 2023 14:20:35 +0800 Subject: [PATCH 187/470] fix fast test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) 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 e41249af54c..571a3c3afb5 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 @@ -421,6 +421,7 @@ mapContains mapContainsKeyLike mapExtractKeyLike mapFilter +mapFromArrays mapKeys mapPopulateSeries mapSubtract From a6cf2cdab8e59e8457def1a891fb3a3443918ca3 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Mar 2023 10:36:07 +0000 Subject: [PATCH 188/470] Fix style, add docs --- docs/en/interfaces/formats.md | 3 +++ .../en/operations/settings/settings-formats.md | 18 ++++++++++++++++++ .../Formats/Impl/ArrowBlockOutputFormat.cpp | 2 -- .../Formats/Impl/ORCBlockOutputFormat.cpp | 2 -- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../02581_parquet_arrow_orc_compressions.sh | 1 + 6 files changed, 23 insertions(+), 5 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index b2b2c6d5b1e..1b32de9723f 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1973,6 +1973,7 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`. - [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`. - [output_format_parquet_version](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_version) - The version of Parquet format used in output format. Default value - `2.latest`. +- [output_format_parquet_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_compression_method) - compression method used in output Parquet format. Default value - `snappy`. ## Arrow {#data-format-arrow} @@ -2041,6 +2042,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam - [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`. - [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`. - [output_format_arrow_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_fixed_string_as_fixed_byte_array) - use Arrow FIXED_SIZE_BINARY type instead of Binary/String for FixedString columns. Default value - `true`. +- [output_format_arrow_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_compression_method) - compression method used in output Arrow format. Default value - `none`. ## ArrowStream {#data-format-arrow-stream} @@ -2096,6 +2098,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT ORC" > {filename. ### Arrow format settings {#parquet-format-settings} - [output_format_arrow_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`. +- [output_format_orc_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_orc_compression_method) - compression method used in output ORC format. Default value - `none`. - [input_format_arrow_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`. - [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`. - [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3580d83f704..919ebaf562f 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1014,6 +1014,12 @@ Use Arrow FIXED_SIZE_BINARY type instead of Binary/String for FixedString column Enabled by default. +### output_format_arrow_compression_method {#output_format_arrow_compression_method} + +Compression method used in output Arrow format. Supported codecs: `lz4_frame`, `zstd`, `none` (uncompressed) + +Default value: `none`. + ## ORC format settings {#orc-format-settings} ### input_format_orc_import_nested {#input_format_orc_import_nested} @@ -1057,6 +1063,12 @@ Use ORC String type instead of Binary for String columns. Disabled by default. +### output_format_orc_compression_method {#output_format_orc_compression_method} + +Compression method used in output ORC format. Supported codecs: `lz4`, `snappy`, `zlib`, `zstd`, `none` (uncompressed) + +Default value: `none`. + ## Parquet format settings {#parquet-format-settings} ### input_format_parquet_import_nested {#input_format_parquet_import_nested} @@ -1112,6 +1124,12 @@ The version of Parquet format used in output format. Supported versions: `1.0`, Default value: `2.latest`. +### output_format_parquet_compression_method {#output_format_parquet_compression_method} + +Compression method used in output Parquet format. Supported codecs: `snappy`, `lz4`, `brotli`, `zstd`, `gzip`, `none` (uncompressed) + +Default value: `snappy`. + ## Hive format settings {#hive-format-settings} ### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter} diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index ec35c52e37c..c85c0342c8c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -8,7 +8,6 @@ #include #include "ArrowBufferedStreams.h" #include "CHColumnToArrowColumn.h" -#include "config.h" namespace DB @@ -36,7 +35,6 @@ arrow::Compression::type getArrowCompression(FormatSettings::ArrowCompression me } - ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_) , stream{stream_} diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index ecb7c2fbc92..39cacde94ed 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -22,8 +22,6 @@ #include #include -#include "config.h" - namespace DB { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 742912df980..cedd8a9c54c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; + extern const int NOT_IMPLEMENTED; } namespace @@ -61,7 +62,6 @@ parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompress throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); } - } ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) diff --git a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh index 10850e17bea..89b5147f026 100755 --- a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh +++ b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.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 5ddc9a9eedd1bd26881142e5ce5aacf020ed7983 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Mar 2023 10:37:47 +0000 Subject: [PATCH 189/470] Add binary compatibility check for Aarch64 --- .github/workflows/backport_branches.yml | 40 +++++++++++-- .github/workflows/master.yml | 40 +++++++++++-- .github/workflows/pull_request.yml | 40 +++++++++++-- .github/workflows/release_branches.yml | 40 +++++++++++-- tests/ci/ci_config.py | 5 +- tests/ci/compatibility_check.py | 80 ++++++++++++++++++------- 6 files changed, 207 insertions(+), 38 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 110c06631c7..b14a32127c4 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -79,7 +79,7 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json - CompatibilityCheck: + CompatibilityCheckX86: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: @@ -98,12 +98,43 @@ jobs: uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck + - name: CompatibilityCheckX86 run: | sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + CompatibilityCheckAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheckAarch64 + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc - name: Cleanup if: always() run: | @@ -741,7 +772,8 @@ jobs: - FunctionalStatefulTestDebug - StressTestTsan - IntegrationTestsRelease - - CompatibilityCheck + - CompatibilityCheckX86 + - CompatibilityCheckAarch64 runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7e045992dee..e224d6cf5c3 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -110,7 +110,7 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - CompatibilityCheck: + CompatibilityCheckX86: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: @@ -129,12 +129,43 @@ jobs: uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck + - name: CompatibilityCheckX86 run: | sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + CompatibilityCheckAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheckAarch64 + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc - name: Cleanup if: always() run: | @@ -3124,7 +3155,8 @@ jobs: - PerformanceComparisonX86-1 - PerformanceComparisonX86-2 - PerformanceComparisonX86-3 - - CompatibilityCheck + - CompatibilityCheckX86 + - CompatibilityCheckAarch64 - ASTFuzzerTestDebug - ASTFuzzerTestAsan - ASTFuzzerTestTsan diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7d410f833c5..ae6cb1d3a83 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -174,7 +174,7 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - CompatibilityCheck: + CompatibilityCheckX86: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: @@ -193,12 +193,43 @@ jobs: uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck + - name: CompatibilityCheckX86 run: | sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + CompatibilityCheckAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheckAarch64 + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc - name: Cleanup if: always() run: | @@ -4792,7 +4823,8 @@ jobs: - UnitTestsMsan - UnitTestsUBsan - UnitTestsReleaseClang - - CompatibilityCheck + - CompatibilityCheckX86 + - CompatibilityCheckAarch64 - IntegrationTestsFlakyCheck - SQLancerTestRelease - SQLancerTestDebug diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 4d2a99c2106..229532efdab 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -71,7 +71,7 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json - CompatibilityCheck: + CompatibilityCheckX86: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: @@ -90,12 +90,43 @@ jobs: uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck + - name: CompatibilityCheckX86 run: | sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + CompatibilityCheckAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheckAarch64 + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (aarch64)" --check-glibc - name: Cleanup if: always() run: | @@ -1947,7 +1978,8 @@ jobs: - IntegrationTestsTsan1 - IntegrationTestsTsan2 - IntegrationTestsTsan3 - - CompatibilityCheck + - CompatibilityCheckX86 + - CompatibilityCheckAarch64 runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 08cd2d466d0..9df198430d2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -316,9 +316,12 @@ CI_CONFIG = { "Integration tests flaky check (asan)": { "required_build": "package_asan", }, - "Compatibility check": { + "Compatibility check (x86)": { "required_build": "package_release", }, + "Compatibility check (aarch64)": { + "required_build": "package_aarch64", + }, "Unit tests (release-clang)": { "required_build": "binary_release", }, diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 0bdcf1ba3b4..9f3c1a1ca8b 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -2,6 +2,7 @@ from distutils.version import StrictVersion from typing import List, Tuple +import argparse import logging import os import subprocess @@ -28,9 +29,7 @@ from upload_result_helper import upload_results IMAGE_UBUNTU = "clickhouse/test-old-ubuntu" IMAGE_CENTOS = "clickhouse/test-old-centos" -MAX_GLIBC_VERSION = "2.4" DOWNLOAD_RETRIES_COUNT = 5 -CHECK_NAME = "Compatibility check" def process_os_check(log_path: str) -> TestResult: @@ -43,7 +42,7 @@ def process_os_check(log_path: str) -> TestResult: return TestResult(name, "OK") -def process_glibc_check(log_path: str) -> TestResults: +def process_glibc_check(log_path: str, max_glibc_version: str) -> TestResults: test_results = [] # type: TestResults with open(log_path, "r") as log: for line in log: @@ -53,7 +52,7 @@ def process_glibc_check(log_path: str) -> TestResults: _, version = symbol_with_glibc.split("@GLIBC_") if version == "PRIVATE": test_results.append(TestResult(symbol_with_glibc, "FAIL")) - elif StrictVersion(version) > MAX_GLIBC_VERSION: + elif StrictVersion(version) > max_glibc_version: test_results.append(TestResult(symbol_with_glibc, "FAIL")) if not test_results: test_results.append(TestResult("glibc check", "OK")) @@ -61,10 +60,10 @@ def process_glibc_check(log_path: str) -> TestResults: def process_result( - result_folder: str, server_log_folder: str + result_folder: str, server_log_folder: str, max_glibc_version: str ) -> Tuple[str, str, TestResults, List[str]]: glibc_log_path = os.path.join(result_folder, "glibc.log") - test_results = process_glibc_check(glibc_log_path) + test_results = process_glibc_check(glibc_log_path, max_glibc_version) status = "success" description = "Compatibility check passed" @@ -101,13 +100,18 @@ def process_result( return status, description, test_results, result_logs -def get_run_commands( - build_path, result_folder, server_log_folder, image_centos, image_ubuntu -): +def get_run_commands_glibc(build_path, result_folder): return [ f"readelf -s --wide {build_path}/usr/bin/clickhouse | grep '@GLIBC_' > {result_folder}/glibc.log", f"readelf -s --wide {build_path}/usr/bin/clickhouse-odbc-bridge | grep '@GLIBC_' >> {result_folder}/glibc.log", f"readelf -s --wide {build_path}/usr/bin/clickhouse-library-bridge | grep '@GLIBC_' >> {result_folder}/glibc.log", + ] + + +def get_run_commands_distributions( + build_path, result_folder, server_log_folder, image_centos, image_ubuntu +): + return [ f"docker run --network=host --volume={build_path}/usr/bin/clickhouse:/clickhouse " f"--volume={build_path}/etc/clickhouse-server:/config " f"--volume={server_log_folder}:/var/log/clickhouse-server {image_ubuntu} > {result_folder}/ubuntu:12.04", @@ -117,9 +121,21 @@ def get_run_commands( ] +def parse_args(): + parser = argparse.ArgumentParser("Check compatibility with old distributions") + parser.add_argument("--check-name", required=True) + parser.add_argument("--check-glibc", action="store_true") + parser.add_argument( + "--check-distributions", action="store_true" + ) # currently hardcoded to x86, don't enable for ARM + return parser.parse_args() + + def main(): logging.basicConfig(level=logging.INFO) + args = parse_args() + stopwatch = Stopwatch() temp_path = TEMP_PATH @@ -129,13 +145,11 @@ def main(): gh = Github(get_best_robot_token(), per_page=100) - rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) + rerun_helper = RerunHelper(gh, pr_info, args.check_name) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) - docker_images = get_images_with_versions(reports_path, [IMAGE_CENTOS, IMAGE_UBUNTU]) - packages_path = os.path.join(temp_path, "packages") if not os.path.exists(packages_path): os.makedirs(packages_path) @@ -145,7 +159,7 @@ def main(): "clickhouse-common-static_" in url or "clickhouse-server_" in url ) - download_builds_filter(CHECK_NAME, reports_path, packages_path, url_filter) + download_builds_filter(args.check_name, reports_path, packages_path, url_filter) for f in os.listdir(packages_path): if ".deb" in f: @@ -162,9 +176,24 @@ def main(): if not os.path.exists(result_path): os.makedirs(result_path) - run_commands = get_run_commands( - packages_path, result_path, server_log_path, docker_images[0], docker_images[1] - ) + run_commands = [] + + if args.check_glibc: + check_glibc_commands = get_run_commands_glibc(packages_path, result_path) + run_commands.extend(check_glibc_commands) + + if args.check_distributions: + docker_images = get_images_with_versions( + reports_path, [IMAGE_CENTOS, IMAGE_UBUNTU] + ) + check_distributions_commands = get_run_commands_distributions( + packages_path, + result_path, + server_log_path, + docker_images[0], + docker_images[1], + ) + run_commands.extend(check_distributions_commands) state = "success" for run_command in run_commands: @@ -177,13 +206,22 @@ def main(): subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + # See https://sourceware.org/glibc/wiki/Glibc%20Timeline + max_glibc_version = "" + if "x86" in args.check_name: + max_glibc_version = "2.4" + elif "aarch64" in args.check_nam: + max_glibc_version = "2.18" # because of build with newer sysroot? + else: + raise Exception("Can't determine max glibc version") + s3_helper = S3Helper() state, description, test_results, additional_logs = process_result( - result_path, server_log_path + result_path, server_log_path, max_glibc_version ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, CHECK_NAME, test_results) + mark_flaky_tests(ch_helper, args.check_name, test_results) report_url = upload_results( s3_helper, @@ -191,10 +229,10 @@ def main(): pr_info.sha, test_results, additional_logs, - CHECK_NAME, + args.check_name, ) print(f"::notice ::Report url: {report_url}") - post_commit_status(gh, pr_info.sha, CHECK_NAME, description, state, report_url) + post_commit_status(gh, pr_info.sha, args.check_name, description, state, report_url) prepared_events = prepare_tests_results_for_clickhouse( pr_info, @@ -203,7 +241,7 @@ def main(): stopwatch.duration_seconds, stopwatch.start_time_str, report_url, - CHECK_NAME, + args.check_name, ) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) From 33cafc6e891a692af3d1315706afb8cea95804e2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Mar 2023 11:22:32 +0000 Subject: [PATCH 190/470] Automatic style fix --- tests/ci/compatibility_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 9f3c1a1ca8b..398f3df3deb 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -211,7 +211,7 @@ def main(): if "x86" in args.check_name: max_glibc_version = "2.4" elif "aarch64" in args.check_nam: - max_glibc_version = "2.18" # because of build with newer sysroot? + max_glibc_version = "2.18" # because of build with newer sysroot? else: raise Exception("Can't determine max glibc version") From cdc8596f81cd4322fda3a188ffcbb9a4065e8882 Mon Sep 17 00:00:00 2001 From: KevinyhZou Date: Thu, 2 Mar 2023 20:08:39 +0800 Subject: [PATCH 191/470] enable int type in repeat function --- .../functions/string-functions.md | 2 +- src/Functions/repeat.cpp | 29 +++++++++++++------ .../01013_repeat_function.reference | 8 ++--- .../0_stateless/01013_repeat_function.sql | 8 ++--- 4 files changed, 29 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 845be6e04c7..f3c5b20f886 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -330,7 +330,7 @@ repeat(s, n) **Arguments** - `s` — The string to repeat. [String](../../sql-reference/data-types/string.md). -- `n` — The number of times to repeat the string. [UInt](../../sql-reference/data-types/int-uint.md). +- `n` — The number of times to repeat the string. [UInt or Int](../../sql-reference/data-types/int-uint.md). **Returned value** diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index dcd05f373fc..0c323c39969 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -39,13 +39,15 @@ struct RepeatImpl size, max_string_size); } + template static void vectorStrConstRepeat( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets, - UInt64 repeat_time) + T repeat_time) { + repeat_time = repeat_time < 0 ? 0 : repeat_time; checkRepeatTime(repeat_time); UInt64 data_size = 0; @@ -77,7 +79,8 @@ struct RepeatImpl res_offsets.assign(offsets); for (UInt64 i = 0; i < col_num.size(); ++i) { - size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * col_num[i] + 1; + T repeat_time = col_num[i] < 0 ? 0 : col_num[i]; + size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1; checkStringSize(repeated_size); data_size += repeated_size; res_offsets[i] = data_size; @@ -86,7 +89,7 @@ struct RepeatImpl for (UInt64 i = 0; i < col_num.size(); ++i) { - T repeat_time = col_num[i]; + T repeat_time = col_num[i] < 0 ? 0 : col_num[i]; checkRepeatTime(repeat_time); process(data.data() + offsets[i - 1], res_data.data() + res_offsets[i - 1], offsets[i] - offsets[i - 1], repeat_time); } @@ -105,7 +108,8 @@ struct RepeatImpl UInt64 col_size = col_num.size(); for (UInt64 i = 0; i < col_size; ++i) { - size_t repeated_size = str_size * col_num[i] + 1; + T repeat_time = col_num[i] < 0 ? 0 : col_num[i]; + size_t repeated_size = str_size * repeat_time + 1; checkStringSize(repeated_size); data_size += repeated_size; res_offsets[i] = data_size; @@ -113,7 +117,7 @@ struct RepeatImpl res_data.resize(data_size); for (UInt64 i = 0; i < col_size; ++i) { - T repeat_time = col_num[i]; + T repeat_time = col_num[i] < 0 ? 0 : col_num[i]; checkRepeatTime(repeat_time); process( reinterpret_cast(const_cast(copy_str.data())), @@ -168,7 +172,8 @@ class FunctionRepeat : public IFunction template static bool castType(const IDataType * type, F && f) { - return castTypeToEither(type, std::forward(f)); + return castTypeToEither(type, std::forward(f)); } public: @@ -186,7 +191,7 @@ public: if (!isString(arguments[0])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[0]->getName(), getName()); - if (!isUnsignedInteger(arguments[1])) + if (!isInteger(arguments[1])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", arguments[1]->getName(), getName()); return arguments[0]; @@ -204,9 +209,15 @@ public: { if (const ColumnConst * scale_column_num = checkAndGetColumn(numcolumn.get())) { - UInt64 repeat_time = scale_column_num->getValue(); auto col_res = ColumnString::create(); - RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); + castType(arguments[1].type.get(), [&](const auto & type) + { + using DataType = std::decay_t; + using T = typename DataType::FieldType; + T repeat_time = scale_column_num->getValue(); + RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); + return true; + }); return col_res; } else if (castType(arguments[1].type.get(), [&](const auto & type) diff --git a/tests/queries/0_stateless/01013_repeat_function.reference b/tests/queries/0_stateless/01013_repeat_function.reference index 46bb248a99a..ea0dadd524f 100644 --- a/tests/queries/0_stateless/01013_repeat_function.reference +++ b/tests/queries/0_stateless/01013_repeat_function.reference @@ -1,7 +1,7 @@ abcabcabcabcabcabcabcabcabcabc abcabcabc -sdfggsdfgg -xywq + + abcabcabcabcabcabcabcabcabcabcabcabc sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg @@ -20,8 +20,8 @@ sdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfggsdfgg xywqxywqxywqxywqxywqxywqxywqxywqxywqxywq plkfplkfplkfplkfplkfplkfplkfplkfplkfplkf abcabcabc -abcabc -abc + + abcabcabcabcabcabcabcabcabcabcabcabc abcabcabcabcabcabcabcabcabcabc diff --git a/tests/queries/0_stateless/01013_repeat_function.sql b/tests/queries/0_stateless/01013_repeat_function.sql index 85b0c16b4ab..b29cc032f28 100644 --- a/tests/queries/0_stateless/01013_repeat_function.sql +++ b/tests/queries/0_stateless/01013_repeat_function.sql @@ -3,20 +3,20 @@ DROP TABLE IF EXISTS defaults; CREATE TABLE defaults ( strings String, - u8 UInt8, + i8 Int8, u16 UInt16, u32 UInt32, u64 UInt64 )ENGINE = Memory(); -INSERT INTO defaults values ('abc', 3, 12, 4, 56) ('sdfgg', 2, 10, 21, 200) ('xywq', 1, 4, 9, 5) ('plkf', 0, 5, 7,77); +INSERT INTO defaults values ('abc', 3, 12, 4, 56) ('sdfgg', -2, 10, 21, 200) ('xywq', -1, 4, 9, 5) ('plkf', 0, 5, 7,77); -SELECT repeat(strings, u8) FROM defaults; +SELECT repeat(strings, i8) FROM defaults; SELECT repeat(strings, u16) FROM defaults; SELECT repeat(strings, u32) from defaults; SELECT repeat(strings, u64) FROM defaults; SELECT repeat(strings, 10) FROM defaults; -SELECT repeat('abc', u8) FROM defaults; +SELECT repeat('abc', i8) FROM defaults; SELECT repeat('abc', u16) FROM defaults; SELECT repeat('abc', u32) FROM defaults; SELECT repeat('abc', u64) FROM defaults; From eab29f4a4e38c15a6b7845208c02fb3dcb5e3210 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 2 Mar 2023 13:09:18 +0100 Subject: [PATCH 192/470] Fixed tests --- src/Functions/FunctionsExternalDictionaries.h | 21 ++++++++------- .../00313_const_totals_extremes.reference | 4 +-- .../00313_const_totals_extremes.sh | 2 +- ...62_pm_all_join_with_block_continuation.sql | 26 +++++++++---------- .../0_stateless/01508_explain_header.sql | 2 ++ 5 files changed, 29 insertions(+), 26 deletions(-) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 570147e4c86..d3be5b1200b 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -675,28 +675,29 @@ private: return impl.isInjective(sample_columns); } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + DataTypePtr getReturnTypeImpl(const DataTypes &) const override { - DataTypePtr result_type; + DataTypePtr result; if constexpr (IsDataTypeDecimal) - result_type = std::make_shared(DataType::maxPrecision(), 0); + result = std::make_shared(DataType::maxPrecision(), 0); else - result_type = std::make_shared(); + result = std::make_shared(); + return result; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { auto return_type = impl.getReturnTypeImpl(arguments); + if (!return_type->equals(*result_type)) throw Exception(ErrorCodes::TYPE_MISMATCH, "Function {} dictionary attribute has different type {} expected {}", getName(), return_type->getName(), result_type->getName()); - return result_type; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - return impl.executeImpl(arguments, result_type, input_rows_count); + return impl.executeImpl(arguments, return_type, input_rows_count); } const FunctionDictGetNoType impl; diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.reference b/tests/queries/0_stateless/00313_const_totals_extremes.reference index fd10e1f4387..b5abbfa3c4e 100644 --- a/tests/queries/0_stateless/00313_const_totals_extremes.reference +++ b/tests/queries/0_stateless/00313_const_totals_extremes.reference @@ -86,7 +86,7 @@ [ { "name": "k", - "type": "DateTime" + "type": "DateTime('UTC')" }, { "name": "count()", @@ -104,7 +104,7 @@ "totals": { - "k": "1970-01-01 01:00:00", + "k": "1970-01-01 00:00:00", "count()": "1" }, diff --git a/tests/queries/0_stateless/00313_const_totals_extremes.sh b/tests/queries/0_stateless/00313_const_totals_extremes.sh index 5e880fe7aa0..539a19817e2 100755 --- a/tests/queries/0_stateless/00313_const_totals_extremes.sh +++ b/tests/queries/0_stateless/00313_const_totals_extremes.sh @@ -9,5 +9,5 @@ ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statist ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toFloat32(1.23) AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDate('2010-01-01') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1"; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT toDateTime('2010-01-01 01:02:03', 'UTC') AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSON"; ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1&output_format_write_statistics=0" -d "SELECT 1.1 AS k, count() GROUP BY k WITH TOTALS SETTINGS allow_experimental_analyzer = 1 FORMAT JSONCompact"; diff --git a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql index 0732db73e38..15a28e92386 100644 --- a/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql +++ b/tests/queries/0_stateless/01062_pm_all_join_with_block_continuation.sql @@ -2,32 +2,32 @@ SET max_memory_usage = 50000000; SET join_algorithm = 'partial_merge'; SELECT 'defaults'; - + SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(10)) nums + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000)) nums + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000000)) nums + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); -- errors SET max_joined_block_size_rows = 0; - + SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); -- { serverError 241 } SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); -- { serverError 241 } @@ -35,17 +35,17 @@ SELECT 'max_joined_block_size_rows = 2000'; SET max_joined_block_size_rows = 2000; SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) as k FROM numbers(1000000)) nums + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); @@ -53,16 +53,16 @@ SELECT 'max_rows_in_join = 1000'; SET max_rows_in_join = 1000; SELECT count(1) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(10)) nums + SELECT materialize(1) as k, n FROM numbers(10) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(1000000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000)) nums + SELECT materialize(1) as k, n FROM numbers(1000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10000)) j USING k); SELECT count(1), uniqExact(n) FROM ( - SELECT k, n FROM (SELECT materialize(1) AS k FROM numbers(1000000)) nums + SELECT materialize(1) as k, n FROM numbers(1000000) nums JOIN (SELECT materialize(1) AS k, number n FROM numbers(10)) j USING k); diff --git a/tests/queries/0_stateless/01508_explain_header.sql b/tests/queries/0_stateless/01508_explain_header.sql index fb274c84d8a..a9f876068aa 100644 --- a/tests/queries/0_stateless/01508_explain_header.sql +++ b/tests/queries/0_stateless/01508_explain_header.sql @@ -1 +1,3 @@ +SET allow_experimental_analyzer = 1; + explain header = 1 select 1 as x; From f9e80b4d416e385e0135166f9a86bc756999b90a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 16:36:35 +0400 Subject: [PATCH 193/470] Typo in the `/docs/ru/sql-reference/statements/select/group-by.md` --- docs/ru/sql-reference/statements/select/group-by.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/group-by.md b/docs/ru/sql-reference/statements/select/group-by.md index 450543cb7a2..ea4f357d33c 100644 --- a/docs/ru/sql-reference/statements/select/group-by.md +++ b/docs/ru/sql-reference/statements/select/group-by.md @@ -108,7 +108,7 @@ SELECT year, month, day, count(*) FROM t GROUP BY year, month, day WITH ROLLUP; ## Модификатор WITH CUBE {#with-cube-modifier} -Модификатор `WITH CUBE` применятеся для расчета подытогов по всем комбинациям группировки ключевых выражений в списке `GROUP BY`. +Модификатор `WITH CUBE` применяется для расчета подытогов по всем комбинациям группировки ключевых выражений в списке `GROUP BY`. Строки с подытогами добавляются в конец результирующей таблицы. В колонках, по которым выполняется группировка, указывается значение `0` или пустая строка. From d55b800fc6b40699b5608c46de4658879d32d374 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 2 Mar 2023 13:43:03 +0100 Subject: [PATCH 194/470] Fixed code review issues --- src/Analyzer/ColumnTransformers.cpp | 2 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 10 +++++++-- src/Planner/Planner.cpp | 1 + src/Planner/PlannerJoinTree.cpp | 26 +++++++++++------------ 4 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/Analyzer/ColumnTransformers.cpp b/src/Analyzer/ColumnTransformers.cpp index cc037fb2a99..ee336a0e7f3 100644 --- a/src/Analyzer/ColumnTransformers.cpp +++ b/src/Analyzer/ColumnTransformers.cpp @@ -348,7 +348,7 @@ ASTPtr ReplaceColumnTransformerNode::toASTImpl() const auto replacement_ast = std::make_shared(); replacement_ast->name = replacements_names[i]; replacement_ast->children.push_back(replacement_expressions_nodes[i]->toAST()); - ast_replace_transformer->children.push_back(replacement_ast); + ast_replace_transformer->children.push_back(std::move(replacement_ast)); } return ast_replace_transformer; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8a69eb348d6..f8645f2b756 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4045,6 +4045,12 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I node = replace_expression->clone(); node_projection_names = resolveExpressionNode(node, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + /** If replace expression resolved as single node, we want to use replace column name as result projection name, instead + * of using replace expression projection name. + * + * Example: SELECT * REPLACE id + 5 AS id FROM test_table; + */ if (node_projection_names.size() == 1) node_projection_names[0] = column_name; @@ -5026,14 +5032,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto function_base = function->build(argument_columns); /// Do not constant fold get scalar functions - bool is_get_scalar_function = function_name == "__getScalar" || function_name == "shardNum" || + bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || function_name == "shardCount"; /** If function is suitable for constant folding try to convert it to constant. * Example: SELECT plus(1, 1); * Result: SELECT 2; */ - if (function_base->isSuitableForConstantFolding() && !is_get_scalar_function) + if (function_base->isSuitableForConstantFolding() && !disable_constant_folding) { auto result_type = function_base->getResultType(); auto executable_function = function_base->prepare(argument_columns); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index a65ee60edfa..2ce470d9ecf 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -594,6 +594,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, auto query_plan_columns = query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(); for (auto & query_plan_column : query_plan_columns) { + /// INTERPOLATE actions dag input columns must be non constant query_plan_column.column = nullptr; interpolate_actions_dag->addInput(query_plan_column); } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4d6181e3686..d6235208063 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -336,7 +336,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } else if (query_node || union_node) { - if (!select_query_options.only_analyze) + if (select_query_options.only_analyze) + { + auto projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + Block source_header; + for (auto & projection_column : projection_columns) + source_header.insert(ColumnWithTypeAndName(projection_column.type, projection_column.name)); + + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + query_plan.addStep(std::move(read_from_pipe)); + } + else { if (table_expression_data.getColumnNames().empty()) { @@ -355,18 +367,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl subquery_planner.buildQueryPlanIfNeeded(); query_plan = std::move(subquery_planner).extractQueryPlan(); } - else - { - auto projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); - Block source_header; - for (auto & projection_column : projection_columns) - source_header.insert(ColumnWithTypeAndName(projection_column.type, projection_column.name)); - - Pipe pipe(std::make_shared(source_header)); - auto read_from_pipe = std::make_unique(std::move(pipe)); - read_from_pipe->setStepDescription("Read from NullSource"); - query_plan.addStep(std::move(read_from_pipe)); - } } else { From 6aee72391847c86ca6007443828b6b4000daf645 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 16:50:28 +0400 Subject: [PATCH 195/470] Typos in the page `/docs/ru/sql-reference/statements/select/join.md` --- docs/ru/sql-reference/statements/select/join.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/statements/select/join.md b/docs/ru/sql-reference/statements/select/join.md index 6853a8c8c90..6be438f8c43 100644 --- a/docs/ru/sql-reference/statements/select/join.md +++ b/docs/ru/sql-reference/statements/select/join.md @@ -16,7 +16,7 @@ FROM (ON )|(USING ) ... ``` -Выражения из секции `ON` и столбцы из секции `USING` называется «ключами соединения». Если не указано иное, при присоединение создаётся [Декартово произведение](https://en.wikipedia.org/wiki/Cartesian_product) из строк с совпадающими значениями ключей соединения, что может привести к получению результатов с гораздо большим количеством строк, чем исходные таблицы. +Выражения из секции `ON` и столбцы из секции `USING` называются «ключами соединения». Если не указано иное, при присоединение создаётся [Декартово произведение](https://en.wikipedia.org/wiki/Cartesian_product) из строк с совпадающими значениями ключей соединения, что может привести к получению результатов с гораздо большим количеством строк, чем исходные таблицы. ## Поддерживаемые типы соединения {#select-join-types} @@ -28,7 +28,7 @@ FROM - `FULL OUTER JOIN`, не совпадающие строки из обеих таблиц возвращаются в дополнение к совпадающим строкам. - `CROSS JOIN`, производит декартово произведение таблиц целиком, ключи соединения не указываются. -Без указания типа `JOIN` подразумевается `INNER`. Ключевое слово `OUTER` можно опускать. Альтернативным синтаксисом для `CROSS JOIN` является ли указание нескольких таблиц, разделённых запятыми, в [секции FROM](from.md). +Без указания типа `JOIN` подразумевается `INNER`. Ключевое слово `OUTER` можно опускать. Альтернативным синтаксисом для `CROSS JOIN` является указание нескольких таблиц, разделённых запятыми, в [секции FROM](from.md). Дополнительные типы соединений, доступные в ClickHouse: @@ -62,7 +62,7 @@ FROM Строки объединяются только тогда, когда всё составное условие выполнено. Если оно не выполнено, то строки могут попасть в результат в зависимости от типа `JOIN`. Обратите внимание, что если то же самое условие поместить в секцию `WHERE`, то строки, для которых оно не выполняется, никогда не попаду в результат. -Оператор `OR` внутри секции `ON` работает, используя алгоритм хеш-соединения — на каждый агрумент `OR` с ключами соединений для `JOIN` создается отдельная хеш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`. +Оператор `OR` внутри секции `ON` работает, используя алгоритм хеш-соединения — на каждый аргумент `OR` с ключами соединений для `JOIN` создается отдельная хеш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`. :::note "Примечание" Если в условии использованы столбцы из разных таблиц, то пока поддерживается только оператор равенства (`=`). @@ -280,7 +280,7 @@ SELECT a, b, toTypeName(a), toTypeName(b) FROM t_1 FULL JOIN t_2 USING (a, b); Каждый раз для выполнения запроса с одинаковым `JOIN`, подзапрос выполняется заново — результат не кэшируется. Это можно избежать, используя специальный движок таблиц [Join](../../../engines/table-engines/special/join.md), представляющий собой подготовленное множество для соединения, которое всегда находится в оперативке. -В некоторых случаях это более эффективно использовать [IN](../../operators/in.md) вместо `JOIN`. +В некоторых случаях более эффективно использовать [IN](../../operators/in.md) вместо `JOIN`. Если `JOIN` необходим для соединения с таблицами измерений (dimension tables - сравнительно небольшие таблицы, которые содержат свойства измерений - например, имена для рекламных кампаний), то использование `JOIN` может быть не очень удобным из-за громоздкости синтаксиса, а также из-за того, что правая таблица читается заново при каждом запросе. Специально для таких случаев существует функциональность «Внешние словари», которую следует использовать вместо `JOIN`. Дополнительные сведения смотрите в разделе «Внешние словари». From bafd9773bc310b65f4a1f94201dcb72b193fb70e Mon Sep 17 00:00:00 2001 From: alex filatov <42028484+phil-88@users.noreply.github.com> Date: Thu, 2 Mar 2023 15:53:09 +0300 Subject: [PATCH 196/470] fix Unknown library method 'extDict_libClone' We have an issue when using external dictionary. Occasionally library bridge called with extDict_libClone and fails with Unknown library method 'extDict_libClone'. And it looks like it is because of at some point `else if (method == "extDict_libNew")` was changed to if (lib_new) with no handling for extDict_libClone inside this new if else statement and reporing an error that extDict_libClone is an unknown method. So there is a two-line fix to handle extDict_libClone properly. Error logs that we have: ``` 2022.12.16 14:17:44.285088 [ 393573 ] {} ExternalDictionaries: Could not update cache dictionary 'dict.vhash_s', next update is scheduled at 2022-12-16 14:18:00: Code: 86. DB::Exception: Received error from remote server /extdict_request?version=1&dictionary_id=be2b2cd1-ba57-4658-8d1b-35ef40ab005b&method=extDict_libClone&from_dictionary_id=c3537142-eaa9-4deb-9b65-47eb8ea1dee6. HTTP status code: 500 Internal Server Error, body: Unknown library method 'extDict_libClone' 2022.12.16 14:17:44.387049 [ 399133 ] {} ExternalDictionaries: Could not update cache dictionary 'dict.vhash_s', next update is scheduled at 2022-12-16 14:17:51: Code: 86. DB::Exception: Received error from remote server /extdict_request?version=1&dictionary_id=0df866ac-6c94-4974-a76c-3940522091b9&method=extDict_libClone&from_dictionary_id=c3537142-eaa9-4deb-9b65-47eb8ea1dee6. HTTP status code: 500 Internal Server Error, body: Unknown library method 'extDict_libClone' 2022.12.16 14:17:44.488468 [ 397769 ] {} ExternalDictionaries: Could not update cache dictionary 'dict.vhash_s', next update is scheduled at 2022-12-16 14:19:38: Code: 86. DB::Exception: Received error from remote server /extdict_request?version=1&dictionary_id=2d8af321-b669-4526-982b-42c0fabf0e8d&method=extDict_libClone&from_dictionary_id=c3537142-eaa9-4deb-9b65-47eb8ea1dee6. HTTP status code: 500 Internal Server Error, body: Unknown library method 'extDict_libClone' 2022.12.16 14:17:44.489935 [ 398226 ] {datamarts_v_dwh_node0032-241534:0x552da2_1_11} executeQuery: Code: 510. DB::Exception: Update failed for dictionary 'dict.vhash_s': Code: 510. DB::Exception: Update failed for dictionary dict.vhash_s : Code: 86. DB::Exception: Received error from remote server /extdict_request?version=1&dictionary_id=be2b2cd1-ba57-4658-8d1b-35ef40ab005b&method=extDict_libClone&from_dictionary_id=c3537142-eaa9-4deb-9b65-47eb8ea1dee6. HTTP status code: 500 Internal Server Error, body: Unknown library method 'extDict_libClone' ``` --- programs/library-bridge/LibraryBridgeHandlers.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/library-bridge/LibraryBridgeHandlers.cpp b/programs/library-bridge/LibraryBridgeHandlers.cpp index ab81472be88..9642dd7ee63 100644 --- a/programs/library-bridge/LibraryBridgeHandlers.cpp +++ b/programs/library-bridge/LibraryBridgeHandlers.cpp @@ -158,6 +158,8 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ if (cloned) { writeStringBinary("1", out); + out.finalize(); + return; } else { From 4a746f32a54a9f7b10bb029532c8b2369db9a5b0 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:12:15 +0400 Subject: [PATCH 197/470] Typos in the page `/docs/ru/sql-reference/statements/select/order-by.md` --- docs/ru/sql-reference/statements/select/order-by.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/sql-reference/statements/select/order-by.md b/docs/ru/sql-reference/statements/select/order-by.md index 40efb862250..a3390501f2c 100644 --- a/docs/ru/sql-reference/statements/select/order-by.md +++ b/docs/ru/sql-reference/statements/select/order-by.md @@ -67,7 +67,7 @@ sidebar_label: ORDER BY ## Примеры с использованием сравнения {#collation-examples} -Пример с значениями типа [String](../../../sql-reference/data-types/string.md): +Пример со значениями типа [String](../../../sql-reference/data-types/string.md): Входная таблица: @@ -241,13 +241,13 @@ SELECT * FROM collate_test ORDER BY s ASC COLLATE 'en'; └───┴─────────┘ ``` -## Деталь реализации {#implementation-details} +## Детали реализации {#implementation-details} Если кроме `ORDER BY` указан также не слишком большой [LIMIT](limit.md), то расходуется меньше оперативки. Иначе расходуется количество памяти, пропорциональное количеству данных для сортировки. При распределённой обработке запроса, если отсутствует [GROUP BY](group-by.md), сортировка частично делается на удалённых серверах, а на сервере-инициаторе запроса производится слияние результатов. Таким образом, при распределённой сортировке, может сортироваться объём данных, превышающий размер памяти на одном сервере. Существует возможность выполнять сортировку во внешней памяти (с созданием временных файлов на диске), если оперативной памяти не хватает. Для этого предназначена настройка `max_bytes_before_external_sort`. Если она выставлена в 0 (по умолчанию), то внешняя сортировка выключена. Если она включена, то при достижении объёмом данных для сортировки указанного количества байт, накопленные данные будут отсортированы и сброшены во временный файл. После того, как все данные будут прочитаны, будет произведено слияние всех сортированных файлов и выдача результата. Файлы записываются в директорию `/var/lib/clickhouse/tmp/` (по умолчанию, может быть изменено с помощью параметра `tmp_path`) в конфиге. -На выполнение запроса может расходоваться больше памяти, чем `max_bytes_before_external_sort`. Поэтому, значение этой настройки должно быть существенно меньше, чем `max_memory_usage`. Для примера, если на вашем сервере 128 GB оперативки, и вам нужно выполнить один запрос, то выставите `max_memory_usage` в 100 GB, а `max_bytes_before_external_sort` в 80 GB. +На выполнение запроса может расходоваться больше памяти, чем `max_bytes_before_external_sort`. Поэтому значение этой настройки должно быть существенно меньше, чем `max_memory_usage`. Для примера, если на вашем сервере 128 GB оперативки, и вам нужно выполнить один запрос, то выставьте `max_memory_usage` в 100 GB, а `max_bytes_before_external_sort` в 80 GB. Внешняя сортировка работает существенно менее эффективно, чем сортировка в оперативке. @@ -366,9 +366,9 @@ ORDER BY └────────────┴────────────┴──────────┘ ``` -Поле `d1` не заполняется и использует значение по умолчанию. Поскольку у нас нет повторяющихся значений для `d2`, мы не можем правильно рассчитать последователность заполнения для `d1`. +Поле `d1` не заполняется и использует значение по умолчанию. Поскольку у нас нет повторяющихся значений для `d2`, мы не можем правильно рассчитать последовательность заполнения для `d1`. -Cледующий запрос (с измененым порядком в ORDER BY): +Cледующий запрос (с измененным порядком в ORDER BY): ```sql SELECT toDate((number * 10) * 86400) AS d1, From f57bb3d71ebe16ad542bc5e5ea6b6722cd873690 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:14:52 +0400 Subject: [PATCH 198/470] Typo in the page `/docs/ru/sql-reference/statements/select/prewhere.md` --- docs/ru/sql-reference/statements/select/prewhere.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/prewhere.md b/docs/ru/sql-reference/statements/select/prewhere.md index e37aec814ec..092370d4b3a 100644 --- a/docs/ru/sql-reference/statements/select/prewhere.md +++ b/docs/ru/sql-reference/statements/select/prewhere.md @@ -13,7 +13,7 @@ Prewhere — это оптимизация для более эффективн `PREWHERE` имеет смысл использовать, если есть условия фильтрации, которые использует меньшинство столбцов из тех, что есть в запросе, но достаточно сильно фильтрует данные. Таким образом, сокращается количество читаемых данных. -В запросе может быть одновременно указаны и `PREWHERE`, и `WHERE`. В этом случае `PREWHERE` предшествует `WHERE`. +В запросе могут быть одновременно указаны и `PREWHERE`, и `WHERE`. В этом случае `PREWHERE` предшествует `WHERE`. Если значение параметра [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) равно 0, эвристика по автоматическому перемещению части выражений из `WHERE` к `PREWHERE` отключается. From 00d28893e8e7f1020f2fb08935db014b7f52da82 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:20:25 +0400 Subject: [PATCH 199/470] Typos in the page `/docs/ru/sql-reference/statements/select/sample.md` --- docs/ru/sql-reference/statements/select/sample.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/sample.md b/docs/ru/sql-reference/statements/select/sample.md index e63479d6c27..decef52d06f 100644 --- a/docs/ru/sql-reference/statements/select/sample.md +++ b/docs/ru/sql-reference/statements/select/sample.md @@ -10,7 +10,7 @@ sidebar_label: SAMPLE Сэмплирование имеет смысл, когда: 1. Точность результата не важна, например, для оценочных расчетов. -2. Возможности аппаратной части не позволяют соответствовать строгим критериям. Например, время ответа должно быть \<100 мс. При этом точность расчета имеет более низкий приоритет. +2. Возможности аппаратной части не позволяют соответствовать строгим критериям. Например, время ответа должно быть <100 мс. При этом точность расчета имеет более низкий приоритет. 3. Точность результата участвует в бизнес-модели сервиса. Например, пользователи с бесплатной подпиской на сервис могут получать отчеты с меньшей точностью, чем пользователи с премиум подпиской. :::note "Внимание" From 39aff35ac3f77c8f99b279e5224911dbb9d6f861 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:23:02 +0400 Subject: [PATCH 200/470] Typos in the page `/docs/ru/sql-reference/statements/select/union.md` --- docs/ru/sql-reference/statements/select/union.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/union.md b/docs/ru/sql-reference/statements/select/union.md index 4dad63c2c7c..05d20100452 100644 --- a/docs/ru/sql-reference/statements/select/union.md +++ b/docs/ru/sql-reference/statements/select/union.md @@ -26,7 +26,7 @@ SELECT CounterID, 2 AS table, sum(Sign) AS c Результирующие столбцы сопоставляются по их индексу (порядку внутри `SELECT`). Если имена столбцов не совпадают, то имена для конечного результата берутся из первого запроса. -При объединении выполняет приведение типов. Например, если два запроса имеют одно и то же поле с не-`Nullable` и `Nullable` совместимыми типами, полученные в результате `UNION` данные будут иметь `Nullable` тип. +При объединении выполняется приведение типов. Например, если два запроса имеют одно и то же поле с не-`Nullable` и `Nullable` совместимыми типами, полученные в результате `UNION` данные будут иметь `Nullable` тип. Запросы, которые являются частью `UNION`, могут быть заключены в круглые скобки. [ORDER BY](order-by.md) и [LIMIT](limit.md) применяются к отдельным запросам, а не к конечному результату. Если вам нужно применить преобразование к конечному результату, вы можете разместить все объединенные с помощью `UNION` запросы в подзапрос в секции [FROM](from.md). From d42a95a6c5ada969e69030c07e9ae1ee01b17e42 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:25:49 +0400 Subject: [PATCH 201/470] Type in the page `/docs/ru/sql-reference/statements/select/with.md` --- docs/ru/sql-reference/statements/select/with.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/with.md b/docs/ru/sql-reference/statements/select/with.md index 58f380b0a85..c4121a6569a 100644 --- a/docs/ru/sql-reference/statements/select/with.md +++ b/docs/ru/sql-reference/statements/select/with.md @@ -5,7 +5,7 @@ sidebar_label: WITH # Секция WITH {#with-clause} -Clickhouse поддерживает [Общие табличные выражения](https://ru.wikipedia.org/wiki/Иерархические_и_рекурсивные_запросы_в_SQL), то есть позволяет использовать результаты выражений из секции `WITH` в остальной части `SELECT` запроса. Именованные подзапросы могут быть включены в текущий и дочерний контекст запроса в тех местах, где разрешены табличные объекты. Рекурсия предотвращается путем скрытия общего табличного выражения текущего уровня из выражения `WITH`. +ClickHouse поддерживает [Общие табличные выражения](https://ru.wikipedia.org/wiki/Иерархические_и_рекурсивные_запросы_в_SQL), то есть позволяет использовать результаты выражений из секции `WITH` в остальной части `SELECT` запроса. Именованные подзапросы могут быть включены в текущий и дочерний контекст запроса в тех местах, где разрешены табличные объекты. Рекурсия предотвращается путем скрытия общего табличного выражения текущего уровня из выражения `WITH`. ## Синтаксис From 740aeaba1fc18ad8667a0bbf7ffefca12b5fc9e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 13:36:47 +0000 Subject: [PATCH 202/470] Apply some CTAD --- src/Common/CancelToken.cpp | 8 +++---- src/Common/ConcurrentBoundedQueue.h | 14 ++++++------ src/Common/DateLUT.cpp | 2 +- src/Common/StatusInfo.h | 4 ++-- src/Common/TLDListsHolder.cpp | 4 ++-- src/Common/mysqlxx/Pool.cpp | 6 ++--- src/Common/mysqlxx/PoolFactory.cpp | 4 ++-- src/Common/mysqlxx/PoolWithFailover.cpp | 2 +- src/Coordination/InMemoryLogStore.cpp | 22 +++++++++---------- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/SummingStateMachine.cpp | 10 ++++----- src/Databases/MySQL/DatabaseMySQL.cpp | 18 +++++++-------- .../PostgreSQL/DatabasePostgreSQL.cpp | 18 +++++++-------- src/Databases/SQLite/DatabaseSQLite.cpp | 10 ++++----- src/Interpreters/Context.cpp | 2 +- .../ExternalLoaderXMLConfigRepository.cpp | 4 ++-- src/Interpreters/GraceHashJoin.cpp | 2 +- src/Interpreters/JIT/CHJIT.cpp | 6 ++--- src/Loggers/OwnSplitChannel.cpp | 4 ++-- .../Impl/ParallelFormattingOutputFormat.cpp | 8 +++---- .../Formats/Impl/ParallelParsingInputFormat.h | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/Sources/ShellCommandSource.cpp | 4 ++-- src/Server/PrometheusMetricsWriter.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.cpp | 16 +++++++------- src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- src/Storages/Hive/StorageHive.cpp | 4 ++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 ++--- src/Storages/StorageDictionary.cpp | 8 +++---- src/Storages/StorageMergeTree.cpp | 4 ++-- 30 files changed, 101 insertions(+), 101 deletions(-) diff --git a/src/Common/CancelToken.cpp b/src/Common/CancelToken.cpp index 53f71fee144..077d60c7efa 100644 --- a/src/Common/CancelToken.cpp +++ b/src/Common/CancelToken.cpp @@ -35,26 +35,26 @@ namespace void CancelToken::Registry::insert(CancelToken * token) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); threads[token->thread_id] = token; } void CancelToken::Registry::remove(CancelToken * token) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); threads.erase(token->thread_id); } void CancelToken::Registry::signal(UInt64 tid) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (auto it = threads.find(tid); it != threads.end()) it->second->signalImpl(); } void CancelToken::Registry::signal(UInt64 tid, int code, const String & message) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (auto it = threads.find(tid); it != threads.end()) it->second->signalImpl(code, message); } diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index fd4a2d5790b..d3abf86c817 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -163,14 +163,14 @@ public: /// Returns size of queue size_t size() const { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); return queue.size(); } /// Returns if queue is empty bool empty() const { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); return queue.empty(); } @@ -184,7 +184,7 @@ public: bool was_finished_before = false; { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); if (is_finished) return true; @@ -202,14 +202,14 @@ public: /// Returns if queue is finished bool isFinished() const { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); return is_finished; } /// Returns if queue is finished and empty bool isFinishedAndEmpty() const { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); return is_finished && queue.empty(); } @@ -217,7 +217,7 @@ public: void clear() { { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); if (is_finished) return; @@ -233,7 +233,7 @@ public: void clearAndFinish() { { - std::lock_guard lock(queue_mutex); + std::lock_guard lock(queue_mutex); std::queue empty_queue; queue.swap(empty_queue); diff --git a/src/Common/DateLUT.cpp b/src/Common/DateLUT.cpp index ae6127670e5..a31ece00c0b 100644 --- a/src/Common/DateLUT.cpp +++ b/src/Common/DateLUT.cpp @@ -149,7 +149,7 @@ DateLUT::DateLUT() const DateLUTImpl & DateLUT::getImplementation(const std::string & time_zone) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); auto it = impls.emplace(time_zone, nullptr).first; if (!it->second) diff --git a/src/Common/StatusInfo.h b/src/Common/StatusInfo.h index 99d6a6f23f5..9aa185cd0c3 100644 --- a/src/Common/StatusInfo.h +++ b/src/Common/StatusInfo.h @@ -26,13 +26,13 @@ namespace CurrentStatusInfo inline void set(Status status, Key key, Int8 value) { - std::lock_guard lock(locks[status]); + std::lock_guard lock(locks[status]); values[status][key] = value; } inline void unset(Status status, Key key) { - std::lock_guard lock(locks[status]); + std::lock_guard lock(locks[status]); values[status].erase(key); } } diff --git a/src/Common/TLDListsHolder.cpp b/src/Common/TLDListsHolder.cpp index c493a88010b..623b88f83a5 100644 --- a/src/Common/TLDListsHolder.cpp +++ b/src/Common/TLDListsHolder.cpp @@ -109,14 +109,14 @@ size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::s } size_t tld_list_size = tld_list.size(); - std::lock_guard lock(tld_lists_map_mutex); + std::lock_guard lock(tld_lists_map_mutex); tld_lists_map.insert(std::make_pair(name, std::move(tld_list))); return tld_list_size; } const TLDList & TLDListsHolder::getTldList(const std::string & name) { - std::lock_guard lock(tld_lists_map_mutex); + std::lock_guard lock(tld_lists_map_mutex); auto it = tld_lists_map.find(name); if (it == tld_lists_map.end()) throw Exception(ErrorCodes::TLD_LIST_NOT_FOUND, "TLD list {} does not exist", name); diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index bee62a0af2e..ba74700d4bc 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -130,7 +130,7 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co Pool::~Pool() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); for (auto & connection : connections) delete static_cast(connection); @@ -187,7 +187,7 @@ Pool::Entry Pool::get(uint64_t wait_timeout) Pool::Entry Pool::tryGet() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); initialize(); @@ -229,7 +229,7 @@ void Pool::removeConnection(Connection* connection) { logger.trace("(%s): Removing connection.", getDescription()); - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (connection) { if (connection->ref_count > 0) diff --git a/src/Common/mysqlxx/PoolFactory.cpp b/src/Common/mysqlxx/PoolFactory.cpp index 1d2f9fc74c7..5fae934a400 100644 --- a/src/Common/mysqlxx/PoolFactory.cpp +++ b/src/Common/mysqlxx/PoolFactory.cpp @@ -76,7 +76,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries) { - std::lock_guard lock(impl->mutex); + std::lock_guard lock(impl->mutex); if (auto entry = impl->pools.find(config_name); entry != impl->pools.end()) { return *(entry->second); @@ -106,7 +106,7 @@ PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & conf void PoolFactory::reset() { - std::lock_guard lock(impl->mutex); + std::lock_guard lock(impl->mutex); impl->pools.clear(); impl->pools_by_ids.clear(); } diff --git a/src/Common/mysqlxx/PoolWithFailover.cpp b/src/Common/mysqlxx/PoolWithFailover.cpp index 0ed231cdf8d..cd9b9e6ec31 100644 --- a/src/Common/mysqlxx/PoolWithFailover.cpp +++ b/src/Common/mysqlxx/PoolWithFailover.cpp @@ -123,7 +123,7 @@ PoolWithFailover::PoolWithFailover(const PoolWithFailover & other) PoolWithFailover::Entry PoolWithFailover::get() { Poco::Util::Application & app = Poco::Util::Application::instance(); - std::lock_guard locker(mutex); + std::lock_guard locker(mutex); /// If we cannot connect to some replica due to pool overflow, than we will wait and connect. PoolPtr * full_pool = nullptr; diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index b7bccdf588e..ca240584a54 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -27,7 +27,7 @@ uint64_t InMemoryLogStore::start_index() const uint64_t InMemoryLogStore::next_slot() const { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); // Exclude the dummy entry. return start_idx + logs.size() - 1; } @@ -35,7 +35,7 @@ uint64_t InMemoryLogStore::next_slot() const nuraft::ptr InMemoryLogStore::last_entry() const { uint64_t next_idx = next_slot(); - std::lock_guard lock(logs_lock); + std::lock_guard lock(logs_lock); auto entry = logs.find(next_idx - 1); if (entry == logs.end()) entry = logs.find(0); @@ -47,7 +47,7 @@ uint64_t InMemoryLogStore::append(nuraft::ptr & entry) { ptr clone = makeClone(entry); - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); uint64_t idx = start_idx + logs.size() - 1; logs[idx] = clone; return idx; @@ -58,7 +58,7 @@ void InMemoryLogStore::write_at(uint64_t index, nuraft::ptr & nuraft::ptr clone = makeClone(entry); // Discard all logs equal to or greater than `index. - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); auto itr = logs.lower_bound(index); while (itr != logs.end()) itr = logs.erase(itr); @@ -76,7 +76,7 @@ nuraft::ptr>> InMemoryLogStore::log_e { nuraft::ptr src = nullptr; { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); auto entry = logs.find(i); if (entry == logs.end()) { @@ -94,7 +94,7 @@ nuraft::ptr InMemoryLogStore::entry_at(uint64_t index) { nuraft::ptr src = nullptr; { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); auto entry = logs.find(index); if (entry == logs.end()) entry = logs.find(0); @@ -107,7 +107,7 @@ uint64_t InMemoryLogStore::term_at(uint64_t index) { uint64_t term = 0; { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); auto entry = logs.find(index); if (entry == logs.end()) entry = logs.find(0); @@ -125,7 +125,7 @@ nuraft::ptr InMemoryLogStore::pack(uint64_t index, Int32 cnt) { ptr le = nullptr; { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); le = logs[ii]; } assert(le.get()); @@ -162,13 +162,13 @@ void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack) nuraft::ptr le = nuraft::log_entry::deserialize(*buf_local); { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); logs[cur_idx] = le; } } { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); auto entry = logs.upper_bound(0); if (entry != logs.end()) start_idx = entry->first; @@ -179,7 +179,7 @@ void InMemoryLogStore::apply_pack(uint64_t index, nuraft::buffer & pack) bool InMemoryLogStore::compact(uint64_t last_log_index) { - std::lock_guard l(logs_lock); + std::lock_guard l(logs_lock); for (uint64_t ii = start_idx; ii <= last_log_index; ++ii) { auto entry = logs.find(ii); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 68f6cfde01e..ccee058ba5c 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -341,7 +341,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession nuraft::ptr KeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. - std::lock_guard lock(snapshots_lock); + std::lock_guard lock(snapshots_lock); return latest_snapshot_meta; } diff --git a/src/Coordination/SummingStateMachine.cpp b/src/Coordination/SummingStateMachine.cpp index 4151b727744..4c21f1d8658 100644 --- a/src/Coordination/SummingStateMachine.cpp +++ b/src/Coordination/SummingStateMachine.cpp @@ -37,7 +37,7 @@ nuraft::ptr SummingStateMachine::commit(const uint64_t log_idx, bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s) { - std::lock_guard ll(snapshots_lock); + std::lock_guard ll(snapshots_lock); auto entry = snapshots.find(s.get_last_log_idx()); if (entry == snapshots.end()) return false; @@ -50,7 +50,7 @@ bool SummingStateMachine::apply_snapshot(nuraft::snapshot & s) nuraft::ptr SummingStateMachine::last_snapshot() { // Just return the latest snapshot. - std::lock_guard ll(snapshots_lock); + std::lock_guard ll(snapshots_lock); auto entry = snapshots.rbegin(); if (entry == snapshots.rend()) return nullptr; @@ -100,7 +100,7 @@ void SummingStateMachine::save_logical_snp_obj( nuraft::buffer_serializer bs(data); int64_t local_value = static_cast(bs.get_u64()); - std::lock_guard ll(snapshots_lock); + std::lock_guard ll(snapshots_lock); auto entry = snapshots.find(s.get_last_log_idx()); assert(entry != snapshots.end()); entry->second->value = local_value; @@ -118,7 +118,7 @@ int SummingStateMachine::read_logical_snp_obj( { nuraft::ptr ctx = nullptr; { - std::lock_guard ll(snapshots_lock); + std::lock_guard ll(snapshots_lock); auto entry = snapshots.find(s.get_last_log_idx()); if (entry == snapshots.end()) { @@ -155,7 +155,7 @@ void SummingStateMachine::create_snapshot( nuraft::async_result::handler_type & when_done) { { - std::lock_guard ll(snapshots_lock); + std::lock_guard ll(snapshots_lock); createSnapshotInternal(s); } nuraft::ptr except(nullptr); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 85944319999..cfb64ba6959 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -82,7 +82,7 @@ DatabaseMySQL::DatabaseMySQL( bool DatabaseMySQL::empty() const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); fetchTablesIntoLocalCache(getContext()); @@ -99,7 +99,7 @@ bool DatabaseMySQL::empty() const DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & filter_by_table_name) const { Tables tables; - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); fetchTablesIntoLocalCache(local_context); @@ -117,7 +117,7 @@ bool DatabaseMySQL::isTableExist(const String & name, ContextPtr local_context) StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, ContextPtr local_context) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); fetchTablesIntoLocalCache(local_context); @@ -129,7 +129,7 @@ StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, ContextPt ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); fetchTablesIntoLocalCache(local_context); @@ -175,7 +175,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_name) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); fetchTablesIntoLocalCache(getContext()); @@ -360,7 +360,7 @@ void DatabaseMySQL::cleanOutdatedTables() void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; if (!local_tables_cache.contains(table_name)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot attach table {}.{} because it does not exist.", @@ -383,7 +383,7 @@ void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_ StoragePtr DatabaseMySQL::detachTable(ContextPtr /* context */, const String & table_name) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; if (remove_or_detach_tables.contains(table_name)) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", @@ -405,7 +405,7 @@ String DatabaseMySQL::getMetadataPath() const void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; fs::directory_iterator iter(getMetadataPath()); for (fs::directory_iterator end; iter != end; ++iter) @@ -421,7 +421,7 @@ void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel void DatabaseMySQL::detachTablePermanently(ContextPtr, const String & table_name) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index f2d57968baa..8c267ea929b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -76,7 +76,7 @@ String DatabasePostgreSQL::formatTableName(const String & table_name, bool quote bool DatabasePostgreSQL::empty() const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); auto connection_holder = pool->get(); auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); @@ -91,7 +91,7 @@ bool DatabasePostgreSQL::empty() const DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local_context, const FilterByNameFunction & /* filter_by_table_name */) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); Tables tables; /// Do not allow to throw here, because this might be, for example, a query to system.tables. @@ -154,7 +154,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* context */) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (detached_or_dropped.contains(table_name)) return false; @@ -165,7 +165,7 @@ bool DatabasePostgreSQL::isTableExist(const String & table_name, ContextPtr /* c StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, ContextPtr local_context) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (!detached_or_dropped.contains(table_name)) return fetchTable(table_name, local_context, false); @@ -210,7 +210,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; if (!checkPostgresTable(table_name)) throw Exception(ErrorCodes::UNKNOWN_TABLE, @@ -235,7 +235,7 @@ void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & t StoragePtr DatabasePostgreSQL::detachTable(ContextPtr /* context_ */, const String & table_name) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; if (detached_or_dropped.contains(table_name)) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot detach table {}. It is already dropped/detached", getTableNameForLogs(table_name)); @@ -266,7 +266,7 @@ void DatabasePostgreSQL::createTable(ContextPtr local_context, const String & ta void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /* sync */) { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; if (!checkPostgresTable(table_name)) throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot drop table {} because it does not exist", getTableNameForLogs(table_name)); @@ -293,7 +293,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/) void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) { { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; fs::directory_iterator iter(getMetadataPath()); /// Check for previously dropped tables @@ -314,7 +314,7 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, Load void DatabasePostgreSQL::removeOutdatedTables() { - std::lock_guard lock{mutex}; + std::lock_guard lock{mutex}; auto connection_holder = pool->get(); auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 653e22487f0..1cba9d1dc26 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -40,14 +40,14 @@ DatabaseSQLite::DatabaseSQLite( bool DatabaseSQLite::empty() const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); return fetchTablesList().empty(); } DatabaseTablesIteratorPtr DatabaseSQLite::getTablesIterator(ContextPtr local_context, const IDatabase::FilterByNameFunction &) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); Tables tables; auto table_names = fetchTablesList(); @@ -120,14 +120,14 @@ bool DatabaseSQLite::checkSQLiteTable(const String & table_name) const bool DatabaseSQLite::isTableExist(const String & table_name, ContextPtr) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); return checkSQLiteTable(table_name); } StoragePtr DatabaseSQLite::tryGetTable(const String & table_name, ContextPtr local_context) const { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); return fetchTable(table_name, local_context, false); } @@ -175,7 +175,7 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex { StoragePtr storage; { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); storage = fetchTable(table_name, local_context, false); } if (!storage) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 33505e41789..d1b09707bca 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1289,7 +1289,7 @@ void Context::addQueryAccessInfo( if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); - std::lock_guard lock(query_access_info.mutex); + std::lock_guard lock(query_access_info.mutex); query_access_info.databases.emplace(quoted_database_name); query_access_info.tables.emplace(full_quoted_table_name); for (const auto & column_name : column_names) diff --git a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp index ba4d9beffe3..8b9584f1242 100644 --- a/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp +++ b/src/Interpreters/ExternalLoaderXMLConfigRepository.cpp @@ -34,7 +34,7 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti std::unordered_set patterns_copy; { - std::lock_guard lock(patterns_mutex); + std::lock_guard lock(patterns_mutex); patterns_copy = patterns; } @@ -71,7 +71,7 @@ std::set ExternalLoaderXMLConfigRepository::getAllLoadablesDefiniti void ExternalLoaderXMLConfigRepository::updatePatterns(const std::unordered_set & patterns_) { - std::lock_guard lock(patterns_mutex); + std::lock_guard lock(patterns_mutex); if (patterns == patterns_) return; diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 51d4c7d1f4b..79a825a752f 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -52,7 +52,7 @@ namespace Block read() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); if (eof) return {}; diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 96c02ad539e..63fe4f44f5f 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -352,7 +352,7 @@ CHJIT::~CHJIT() = default; CHJIT::CompiledModule CHJIT::compileModule(std::function compile_function) { - std::lock_guard lock(jit_lock); + std::lock_guard lock(jit_lock); auto module = createModuleForCompilation(); compile_function(*module); @@ -426,7 +426,7 @@ CHJIT::CompiledModule CHJIT::compileModule(std::unique_ptr module) void CHJIT::deleteCompiledModule(const CHJIT::CompiledModule & module) { - std::lock_guard lock(jit_lock); + std::lock_guard lock(jit_lock); auto module_it = module_identifier_to_memory_manager.find(module.identifier); if (module_it == module_identifier_to_memory_manager.end()) @@ -438,7 +438,7 @@ void CHJIT::deleteCompiledModule(const CHJIT::CompiledModule & module) void CHJIT::registerExternalSymbol(const std::string & symbol_name, void * address) { - std::lock_guard lock(jit_lock); + std::lock_guard lock(jit_lock); symbol_resolver->registerSymbol(symbol_name, address); } diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 7974d5212e1..047f6caffcd 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -137,7 +137,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) std::shared_ptr text_log_locked{}; { - std::lock_guard lock(text_log_mutex); + std::lock_guard lock(text_log_mutex); text_log_locked = text_log.lock(); } if (text_log_locked) @@ -155,7 +155,7 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std #ifndef WITHOUT_TEXT_LOG void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_priority) { - std::lock_guard lock(text_log_mutex); + std::lock_guard lock(text_log_mutex); text_log = log; text_log_max_priority.store(max_priority, std::memory_order_relaxed); } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index db4bb422cb1..35d4dd9ddcd 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -15,7 +15,7 @@ namespace DB collector_finished.wait(); { - std::lock_guard lock(collector_thread_mutex); + std::lock_guard lock(collector_thread_mutex); if (collector_thread.joinable()) collector_thread.join(); } @@ -80,7 +80,7 @@ namespace DB } { - std::lock_guard lock(collector_thread_mutex); + std::lock_guard lock(collector_thread_mutex); if (collector_thread.joinable()) collector_thread.join(); } @@ -137,7 +137,7 @@ namespace DB { /// Notify other threads. - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); unit.status = READY_TO_INSERT; writer_condvar.notify_all(); } @@ -227,7 +227,7 @@ namespace DB unit.actual_memory_size = out_buffer.getActualSize(); { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); unit.status = READY_TO_READ; collector_condvar.notify_all(); } diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 252b6ec3f81..03fb2d650dc 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -292,7 +292,7 @@ private: { /// Additionally notify condvars - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); segmentator_condvar.notify_all(); reader_condvar.notify_all(); } diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 86b64d1519c..53dcec9ef0a 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -62,7 +62,7 @@ public: { assert(!rhs_ports.first->isConnected() && !rhs_ports.second->isConnected()); - std::lock_guard lock(mux); + std::lock_guard lock(mux); if (input_port || output_port) { assert(input_port && output_port); diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 4983fa047dc..ed8f9b41e78 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -274,7 +274,7 @@ namespace } catch (...) { - std::lock_guard lock(send_data_lock); + std::lock_guard lock(send_data_lock); exception_during_send_data = std::current_exception(); } }); @@ -387,7 +387,7 @@ namespace void rethrowExceptionDuringSendDataIfNeeded() { - std::lock_guard lock(send_data_lock); + std::lock_guard lock(send_data_lock); if (exception_during_send_data) { command_is_invalid = true; diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index 843d1e64463..abf2a2c0b6b 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -125,7 +125,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const { for (size_t i = 0, end = CurrentStatusInfo::end(); i < end; ++i) { - std::lock_guard lock(CurrentStatusInfo::locks[static_cast(i)]); + std::lock_guard lock(CurrentStatusInfo::locks[static_cast(i)]); std::string metric_name{CurrentStatusInfo::getName(static_cast(i))}; std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast(i))}; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 192721f9f3c..3c5342a1e83 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -12,7 +12,7 @@ FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, Stor FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); Events res; res.swap(events); return res; @@ -20,7 +20,7 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() FileLogDirectoryWatcher::Error FileLogDirectoryWatcher::getErrorAndReset() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); Error old_error = error; error = {}; return old_error; @@ -33,7 +33,7 @@ const std::string & FileLogDirectoryWatcher::getPath() const void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent ev) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemAdded"}; std::string event_path = ev.path; @@ -51,7 +51,7 @@ void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent e void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemRemoved"}; std::string event_path = ev.path; @@ -74,7 +74,7 @@ void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent /// because it is equal to just record and handle one MODIY event void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEvent ev) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); auto event_path = ev.path; EventInfo info{ev.event, "onItemModified"}; @@ -97,7 +97,7 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedFrom"}; std::string event_path = ev.path; @@ -114,7 +114,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEve void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedTo"}; std::string event_path = ev.path; @@ -131,7 +131,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent void FileLogDirectoryWatcher::onError(Exception e) { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex); LOG_ERROR(log, "Error happened during watching directory: {}", error.error_msg); error.has_error = true; error.error_msg = e.message(); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7838db881e9..dae6f6a7ca9 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -312,7 +312,7 @@ Pipe StorageFileLog::read( if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageFileLog with attached materialized views"); - std::lock_guard lock(file_infos_mutex); + std::lock_guard lock(file_infos_mutex); if (running_streams) { throw Exception(ErrorCodes::CANNOT_SELECT, "Another select query is running on this table, need to wait it finish."); @@ -659,7 +659,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - std::lock_guard lock(file_infos_mutex); + std::lock_guard lock(file_infos_mutex); if (running_streams) { LOG_INFO(log, "Another select query is running on this table, need to wait it finish."); diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 13477a46bb5..85e6341eb5a 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -856,7 +856,7 @@ HiveFiles StorageHive::collectHiveFiles( = collectHiveFilesFromPartition(partition, query_info, hive_table_metadata, fs, context_, prune_level); if (!hive_files_in_partition.empty()) { - std::lock_guard lock(hive_files_mutex); + std::lock_guard lock(hive_files_mutex); hit_parttions_num += 1; if (hive_max_query_partitions > 0 && hit_parttions_num > hive_max_query_partitions) { @@ -882,7 +882,7 @@ HiveFiles StorageHive::collectHiveFiles( auto hive_file = getHiveFileIfNeeded(file_info, {}, query_info, hive_table_metadata, context_, prune_level); if (hive_file) { - std::lock_guard lock(hive_files_mutex); + std::lock_guard lock(hive_files_mutex); hive_files.push_back(hive_file); } }); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 125529e7b94..20dbaa73812 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2316,7 +2316,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( return false; } - std::lock_guard lock(queue.state_mutex); + std::lock_guard lock(queue.state_mutex); /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer /// and it is guaranteed that it will contain all merges assigned before this object is constructed. @@ -2334,7 +2334,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( bool ReplicatedMergeTreeMergePredicate::partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const { - std::lock_guard lock(queue.state_mutex); + std::lock_guard lock(queue.state_mutex); for (const auto & entry : queue.queue) { if (entry->type != ReplicatedMergeTreeLogEntry::REPLACE_RANGE) @@ -2457,7 +2457,7 @@ bool ReplicatedMergeTreeMergePredicate::isGoingToBeDropped(const MergeTreePartIn String ReplicatedMergeTreeMergePredicate::getCoveringVirtualPart(const String & part_name) const { - std::lock_guard lock(queue.state_mutex); + std::lock_guard lock(queue.state_mutex); return queue.virtual_parts.getContainingPart(MergeTreePartInfo::fromPartName(part_name, queue.format_version)); } diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 5d1f08771be..b0b8aba38c7 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -208,13 +208,13 @@ void StorageDictionary::removeDictionaryConfigurationFromRepository() Poco::Timestamp StorageDictionary::getUpdateTime() const { - std::lock_guard lock(dictionary_config_mutex); + std::lock_guard lock(dictionary_config_mutex); return update_time; } LoadablesConfigurationPtr StorageDictionary::getConfiguration() const { - std::lock_guard lock(dictionary_config_mutex); + std::lock_guard lock(dictionary_config_mutex); return configuration; } @@ -234,7 +234,7 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) assert(old_table_id.uuid == new_table_id.uuid || move_to_atomic || move_to_ordinary); { - std::lock_guard lock(dictionary_config_mutex); + std::lock_guard lock(dictionary_config_mutex); configuration->setString("dictionary.database", new_table_id.database_name); configuration->setString("dictionary.name", new_table_id.table_name); @@ -301,7 +301,7 @@ void StorageDictionary::alter(const AlterCommands & params, ContextPtr alter_con dictionary_non_const->setDictionaryComment(new_comment); } - std::lock_guard lock(dictionary_config_mutex); + std::lock_guard lock(dictionary_config_mutex); configuration->setString("dictionary.comment", new_comment); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8210ef93172..aadd7b8c20a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -732,7 +732,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) to_kill->removeFile(); LOG_TRACE(log, "Cancelled part mutations and removed mutation file {}", mutation_id); { - std::lock_guard lock(mutation_wait_mutex); + std::lock_guard lock(mutation_wait_mutex); mutation_wait_event.notify_all(); } @@ -1306,7 +1306,7 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) std::vector mutations_to_delete; { - std::lock_guard lock(currently_processing_in_background_mutex); + std::lock_guard lock(currently_processing_in_background_mutex); if (current_mutations_by_version.size() <= finished_mutations_to_keep) return 0; From 10610cbdf475d47af8f301bac415c436404c16dd Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 Mar 2023 14:46:20 +0100 Subject: [PATCH 203/470] Fix test --- src/Disks/getOrCreateDiskFromAST.cpp | 5 ----- src/Parsers/FieldFromAST.cpp | 6 +++--- tests/queries/0_stateless/02344_show_caches.reference | 1 + 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 9abc4a7d413..637acff7b95 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -33,11 +33,6 @@ namespace auto disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - LOG_TRACE( - &Poco::Logger::get("getOrCreateDiskFromDiskAST"), - "Using disk name `{}` for custom disk {}", - disk_name, disk_setting_string); - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { const auto * function_args_expr = assert_cast(function.arguments.get()); const auto & function_args = function_args_expr->children; diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index b2251599441..a81bf45a8be 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -32,7 +32,7 @@ bool FieldFromASTImpl::isSecret() const return isDiskFunction(ast); } -class DiskConfigurationHider +class DiskConfigurationMasker { public: struct Data {}; @@ -50,7 +50,7 @@ public: auto is_secret_arg = [](const std::string & arg_name) { /// We allow to not hide type of the disk, e.g. disk(type = s3, ...) - /// and also nested disk, e.g. disk(type = cache, disk = disk(type = s3, ...)) + /// and also nested disk, e.g. disk = 'disk_name' return arg_name != "type" && arg_name != "disk"; }; @@ -81,7 +81,7 @@ public: }; /// Visits children first. -using HideDiskConfigurationVisitor = InDepthNodeVisitor; +using HideDiskConfigurationVisitor = InDepthNodeVisitor; String FieldFromASTImpl::toString(bool show_secrets) const { diff --git a/tests/queries/0_stateless/02344_show_caches.reference b/tests/queries/0_stateless/02344_show_caches.reference index 2ee4f902ba1..b321319a309 100644 --- a/tests/queries/0_stateless/02344_show_caches.reference +++ b/tests/queries/0_stateless/02344_show_caches.reference @@ -10,5 +10,6 @@ local_cache s3_cache_6 s3_cache_small local_cache_2 +__tmp_internal_324081342946782869538999598488311137423 local_cache_3 s3_cache_multi_2 From 29f0f4e74517e43eb2d9d8b1d76be08efaa0ba6a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 2 Mar 2023 17:54:16 +0400 Subject: [PATCH 204/470] Typos in the page `/docs/ru/sql-reference/statements/create/table.md` --- docs/ru/sql-reference/statements/create/table.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/create/table.md b/docs/ru/sql-reference/statements/create/table.md index c29d575cac1..7a930b529ed 100644 --- a/docs/ru/sql-reference/statements/create/table.md +++ b/docs/ru/sql-reference/statements/create/table.md @@ -301,7 +301,7 @@ ClickHouse поддерживает временные таблицы со сл - Временные таблицы исчезают после завершения сессии, в том числе при обрыве соединения. - Временная таблица использует только модуль памяти. - Невозможно указать базу данных для временной таблицы. Она создается вне баз данных. -- Невозможно создать временную таблицу распределнным DDL запросом на всех серверах кластера (с опцией `ON CLUSTER`): такая таблица существует только в рамках существующей сессии. +- Невозможно создать временную таблицу распределённым DDL запросом на всех серверах кластера (с опцией `ON CLUSTER`): такая таблица существует только в рамках существующей сессии. - Если временная таблица имеет то же имя, что и некоторая другая, то, при упоминании в запросе без указания БД, будет использована временная таблица. - При распределённой обработке запроса, используемые в запросе временные таблицы, передаются на удалённые серверы. @@ -344,7 +344,9 @@ REPLACE TABLE myOldTable SELECT * FROM myOldTable WHERE CounterID <12345; ### Синтаксис +```sql {CREATE [OR REPLACE]|REPLACE} TABLE [db.]table_name +``` Для данного запроса можно использовать любые варианты синтаксиса запроса `CREATE`. Запрос `REPLACE` для несуществующей таблицы вызовет ошибку. From 9722d8a4030f3aca302ae495cd6a2750e8eb5248 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 2 Mar 2023 14:17:48 +0000 Subject: [PATCH 205/470] Rename variable --- src/IO/ReadWriteBufferFromHTTP.h | 70 +++++++++++++++++++------------- 1 file changed, 42 insertions(+), 28 deletions(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index edf7b3f5f95..5c5285c5260 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -147,7 +147,7 @@ namespace detail size_t getOffset() const { return getRangeBegin() + offset_from_begin_pos; } - template + template std::istream * callImpl(UpdatableSessionPtr & current_session, Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) { // With empty path poco will send "POST HTTP/1.1" its bug. @@ -164,7 +164,7 @@ namespace detail request.set(header, value); std::optional range; - if constexpr (use_initial_range) + if constexpr (for_object_info) { if (withPartialContent(initial_read_range)) range = initial_read_range; @@ -202,7 +202,9 @@ namespace detail auto result_istr = receiveResponse(*sess, request, response, true); response.getCookies(cookies); - if constexpr (!use_initial_range) + /// we can fetch object info while the request is being processed + /// and we don't want to override any context used by it + if constexpr (!for_object_info) content_encoding = response.get("Content-Encoding", ""); return result_istr; @@ -222,7 +224,7 @@ namespace detail return *file_size; Poco::Net::HTTPResponse response; - getHeadResponse(response); + getHeadResponse(response); if (response.hasContentLength()) { @@ -251,14 +253,13 @@ namespace detail InitializeError initialization_error = InitializeError::NONE; private: - template void getHeadResponse(Poco::Net::HTTPResponse & response) { for (size_t i = 0; i < settings.http_max_tries; ++i) { try { - callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD); + callWithRedirects(response, Poco::Net::HTTPRequest::HTTP_HEAD); break; } catch (const Poco::Exception & e) @@ -377,17 +378,19 @@ namespace detail return location_uri; } - template + template void callWithRedirects(Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { UpdatableSessionPtr current_session = nullptr; - if (use_initial_range) + /// we can fetch object info while the request is being processed + /// and we don't want to override any context used by it + if constexpr (for_object_info) current_session = session->clone(uri); else current_session = session; - call(current_session, response, method_, throw_on_all_errors); + call(current_session, response, method_, throw_on_all_errors); Poco::URI prev_uri = uri; while (isRedirect(response.getStatus())) @@ -399,42 +402,53 @@ namespace detail current_session->updateSession(uri_redirect); - auto result_istr = callImpl(current_session, uri_redirect, response, method); - if (!use_initial_range) + /// we can fetch object info while the request is being processed + /// and we don't want to override any context used by it + auto result_istr = callImpl(current_session, uri_redirect, response, method); + if constexpr (!for_object_info) istr = result_istr; } } - template + template void call(UpdatableSessionPtr & current_session, Poco::Net::HTTPResponse & response, const String & method_, bool throw_on_all_errors = false) { try { - auto result_istr = callImpl(current_session, saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); - if (!use_initial_range) + /// we can fetch object info while the request is being processed + /// and we don't want to override any context used by it + auto result_istr = callImpl(current_session, saved_uri_redirect ? *saved_uri_redirect : uri, response, method_); + if constexpr (!for_object_info) istr = result_istr; } catch (...) { - if (throw_on_all_errors) + /// we can fetch object info while the request is being processed + /// and we don't want to override any context used by it + if constexpr (for_object_info) { throw; } - - auto http_status = response.getStatus(); - - if (http_status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND && http_skip_not_found_url) - { - initialization_error = InitializeError::SKIP_NOT_FOUND_URL; - } - else if (!isRetriableError(http_status)) - { - initialization_error = InitializeError::NON_RETRYABLE_ERROR; - exception = std::current_exception(); - } else { - throw; + if (throw_on_all_errors) + throw; + + auto http_status = response.getStatus(); + + if (http_status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND && http_skip_not_found_url) + { + initialization_error = InitializeError::SKIP_NOT_FOUND_URL; + } + else if (!isRetriableError(http_status)) + { + initialization_error = InitializeError::NON_RETRYABLE_ERROR; + exception = std::current_exception(); + } + else + { + throw; + } } } } From cf245538b91923a97c617a997e2bc68d01b51d75 Mon Sep 17 00:00:00 2001 From: AndyB Date: Thu, 2 Mar 2023 14:31:38 +0000 Subject: [PATCH 206/470] checker fix --- tests/integration/test_log_levels_update/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py index 842c7914eaa..4b83b6431fc 100644 --- a/tests/integration/test_log_levels_update/test.py +++ b/tests/integration/test_log_levels_update/test.py @@ -63,4 +63,4 @@ def test_log_levels_update(start_cluster): log = get_log(node) assert len(log) > 0 - assert not re.search("(|)", log) + assert not re.search("", log) From fdf8a418f517eeb739aa22e6afb0d7392a212a17 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Mar 2023 20:37:20 +0000 Subject: [PATCH 207/470] better interface --- src/DataTypes/DataTypeTuple.cpp | 14 -------------- src/DataTypes/DataTypeTuple.h | 1 - src/DataTypes/IDataType.cpp | 9 --------- src/DataTypes/IDataType.h | 1 - src/DataTypes/Serializations/SerializationInfo.cpp | 9 +++++++++ src/DataTypes/Serializations/SerializationInfo.h | 2 ++ .../Serializations/SerializationInfoTuple.cpp | 14 ++++++++++++++ .../Serializations/SerializationInfoTuple.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- 9 files changed, 27 insertions(+), 27 deletions(-) diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 1e28af3ee54..768f87fe3d4 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -312,20 +312,6 @@ MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const Seriali return std::make_shared(std::move(infos), names, settings); } -MutableSerializationInfoPtr DataTypeTuple::cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const -{ - const auto & old_info_tuple = assert_cast(old_info); - assert(old_info_tuple.getNumElements() == elems.size()); - - MutableSerializationInfos infos; - infos.reserve(elems.size()); - for (size_t i = 0; i < elems.size(); ++i) - infos.push_back(elems[i]->cloneSerializationInfo(*old_info_tuple.getElementInfo(i), settings)); - - return std::make_shared(std::move(infos), names, settings); -} - - SerializationInfoPtr DataTypeTuple::getSerializationInfo(const IColumn & column) const { if (const auto * column_const = checkAndGetColumn(&column)) diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 9f3860f78db..152f21015f5 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -57,7 +57,6 @@ public: SerializationPtr doGetDefaultSerialization() const override; SerializationPtr getSerialization(const SerializationInfo & info) const override; MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override; - MutableSerializationInfoPtr cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const override; SerializationInfoPtr getSerializationInfo(const IColumn & column) const override; const DataTypePtr & getElement(size_t i) const { return elems[i]; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 5cd0b6f659c..e0612fbbf36 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -191,15 +191,6 @@ MutableSerializationInfoPtr IDataType::createSerializationInfo(const Serializati return std::make_shared(ISerialization::Kind::DEFAULT, settings); } -MutableSerializationInfoPtr IDataType::cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const -{ - auto new_kind = old_info.getKind(); - if (new_kind == ISerialization::Kind::SPARSE && !supportsSparseSerialization()) - new_kind = ISerialization::Kind::DEFAULT; - - return std::make_shared(new_kind, settings); -} - SerializationInfoPtr IDataType::getSerializationInfo(const IColumn & column) const { if (const auto * column_const = checkAndGetColumn(&column)) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 5bd12f84d04..e5bdbeca69e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -104,7 +104,6 @@ public: Names getSubcolumnNames() const; virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const; - virtual MutableSerializationInfoPtr cloneSerializationInfo(const SerializationInfo & old_info, const SerializationInfo::Settings & settings) const; virtual SerializationInfoPtr getSerializationInfo(const IColumn & column) const; /// TODO: support more types. diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 313fa1fa235..af3330d867d 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -97,6 +97,15 @@ MutableSerializationInfoPtr SerializationInfo::clone() const return std::make_shared(kind, settings, data); } +std::shared_ptr SerializationInfo::createWithType(const IDataType & type, const Settings & new_settings) const +{ + auto new_kind = kind; + if (new_kind == ISerialization::Kind::SPARSE && !type.supportsSparseSerialization()) + new_kind = ISerialization::Kind::DEFAULT; + + return std::make_shared(new_kind, new_settings); +} + void SerializationInfo::serialializeKindBinary(WriteBuffer & out) const { writeBinary(static_cast(kind), out); diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 7f73d053f1b..560156980db 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -8,6 +8,7 @@ namespace DB { +class ReadBuffer; class ReadBuffer; class WriteBuffer; class NamesAndTypesList; @@ -59,6 +60,7 @@ public: virtual void replaceData(const SerializationInfo & other); virtual std::shared_ptr clone() const; + virtual std::shared_ptr createWithType(const IDataType & type, const Settings & new_settings) const; virtual void serialializeKindBinary(WriteBuffer & out) const; virtual void deserializeFromKindsBinary(ReadBuffer & in); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 1a9639a1566..5724dd7a5c3 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -97,6 +97,20 @@ MutableSerializationInfoPtr SerializationInfoTuple::clone() const return std::make_shared(std::move(elems_cloned), names, settings); } +MutableSerializationInfoPtr SerializationInfoTuple::createWithType(const IDataType & type, const Settings & new_settings) const +{ + const auto & type_tuple = assert_cast(type); + const auto & tuple_elements = type_tuple.getElements(); + assert(elems.size() == tuple_elements.size()); + + MutableSerializationInfos infos; + infos.reserve(elems.size()); + for (size_t i = 0; i < elems.size(); ++i) + infos.push_back(elems[i]->createWithType(*tuple_elements[i], new_settings)); + + return std::make_shared(std::move(infos), names, new_settings); +} + void SerializationInfoTuple::serialializeKindBinary(WriteBuffer & out) const { SerializationInfo::serialializeKindBinary(out); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.h b/src/DataTypes/Serializations/SerializationInfoTuple.h index ef288948b9a..4bcd14d34cb 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.h +++ b/src/DataTypes/Serializations/SerializationInfoTuple.h @@ -19,6 +19,7 @@ public: void replaceData(const SerializationInfo & other) override; MutableSerializationInfoPtr clone() const override; + MutableSerializationInfoPtr createWithType(const IDataType & type, const Settings & new_settings) const override; void serialializeKindBinary(WriteBuffer & out) const override; void deserializeFromKindsBinary(ReadBuffer & in) override; @@ -26,7 +27,6 @@ public: Poco::JSON::Object toJSON() const override; void fromJSON(const Poco::JSON::Object & object) override; - size_t getNumElements() const { return elems.size(); } const MutableSerializationInfoPtr & getElementInfo(size_t i) const { return elems[i]; } ISerialization::Kind getElementKind(size_t i) const { return elems[i]->getKind(); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9420b7bebaf..1239befdd67 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -244,7 +244,7 @@ getColumnsForNewDataPart( continue; } - new_info = new_type->cloneSerializationInfo(*old_info, settings); + new_info = old_info->createWithType(*new_type, settings); new_serialization_infos.emplace(new_name, std::move(new_info)); } From df41a83ddb1f83618ce1ad666545cf6138444df3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 2 Mar 2023 14:49:49 +0000 Subject: [PATCH 208/470] set uid gid to file's original --- .../decompressor.cpp | 30 +++++-------------- 1 file changed, 7 insertions(+), 23 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 5f2a769dcdb..39fe552dfac 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -168,23 +168,9 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n return 0; } -bool getSudoIDs(uid_t &sudo_uid, uid_t &sudo_gid) +bool isSudo() { - sudo_uid = 0; - sudo_gid = 0; - - if (getuid() || geteuid() || getenv("SUDO_USER") == nullptr || getenv("SUDO_UID") == nullptr || getenv("SUDO_GID") == nullptr) - return false; - - char * str_end; - long id = strtol(getenv("SUDO_UID"), &str_end, 10); - if (*str_end == 0) - sudo_uid = static_cast(id); - id = strtol(getenv("SUDO_GID"), &str_end, 10); - if (*str_end == 0) - sudo_gid = static_cast(id); - - return true; + return getuid() == 0 && geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); } /// Read data about files and decomrpess them. @@ -238,9 +224,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress return 1; } - uid_t sudo_uid = 0; - uid_t sudo_gid = 0; - getSudoIDs(sudo_uid, sudo_gid); + bool is_sudo = isSudo(); FileData file_info; /// Decompress files with appropriate file names @@ -342,8 +326,8 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress if (0 != close(output_fd)) perror("close"); - if (sudo_uid && sudo_gid) - chown(file_name, sudo_uid, sudo_gid); + if (is_sudo) + chown(file_name, info_in.st_uid, info_in.st_gid); } if (0 != munmap(input, info_in.st_size)) @@ -557,8 +541,8 @@ int main(int/* argc*/, char* argv[]) return 1; } - if (uid_t sudo_uid = 0, sudo_gid = 0; getSudoIDs(sudo_uid, sudo_gid)) - chown(static_cast(self), sudo_uid, sudo_gid); + if (isSudo()) + chown(static_cast(self), input_info.st_uid, input_info.st_gid); if (has_exec) { From 95292ce5c79a8386ec6736e1b3fd934c3a38e02a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 16:26:46 +0000 Subject: [PATCH 209/470] Make 01710_projections more stable. --- tests/queries/0_stateless/01710_projections.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projections.sql b/tests/queries/0_stateless/01710_projections.sql index cbabd3ec598..146b46aed38 100644 --- a/tests/queries/0_stateless/01710_projections.sql +++ b/tests/queries/0_stateless/01710_projections.sql @@ -1,6 +1,6 @@ drop table if exists projection_test; -create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain)) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain); +create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain)) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain) settings index_granularity_bytes = 10000000; insert into projection_test with rowNumberInAllBlocks() as id select 1, toDateTime('2020-10-24 00:00:00') + (id / 20), toString(id % 100), * from generateRandom('x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64)', 10, 10, 1) limit 1000 settings max_threads = 1; From 48fc545e68fc86cb903a2272f1a62640429bfde2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 17:27:57 +0100 Subject: [PATCH 210/470] Update 01710_projections.sql --- tests/queries/0_stateless/01710_projections.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projections.sql b/tests/queries/0_stateless/01710_projections.sql index 146b46aed38..5097a88c8fa 100644 --- a/tests/queries/0_stateless/01710_projections.sql +++ b/tests/queries/0_stateless/01710_projections.sql @@ -1,6 +1,6 @@ drop table if exists projection_test; -create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain)) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain) settings index_granularity_bytes = 10000000; +create table projection_test (`sum(block_count)` UInt64, domain_alias UInt64 alias length(domain), datetime DateTime, domain LowCardinality(String), x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64), projection p (select toStartOfMinute(datetime) dt_m, countIf(first_time = 0) / count(), avg((kbytes * 8) / duration), count(), sum(block_count) / sum(duration), avg(block_count / duration), sum(buffer_time) / sum(duration), avg(buffer_time / duration), sum(valid_bytes) / sum(total_bytes), sum(completed_bytes) / sum(total_bytes), sum(fixed_bytes) / sum(total_bytes), sum(force_bytes) / sum(total_bytes), sum(valid_bytes) / sum(total_bytes), sum(retry_count) / sum(duration), avg(retry_count / duration), countIf(block_count > 0) / count(), countIf(first_time = 0) / count(), uniqHLL12(x_id), uniqHLL12(y_id) group by dt_m, domain)) engine MergeTree partition by toDate(datetime) order by (toStartOfTenMinutes(datetime), domain) settings index_granularity_bytes = 10000000; insert into projection_test with rowNumberInAllBlocks() as id select 1, toDateTime('2020-10-24 00:00:00') + (id / 20), toString(id % 100), * from generateRandom('x_id String, y_id String, block_count Int64, retry_count Int64, duration Int64, kbytes Int64, buffer_time Int64, first_time Int64, total_bytes Nullable(UInt64), valid_bytes Nullable(UInt64), completed_bytes Nullable(UInt64), fixed_bytes Nullable(UInt64), force_bytes Nullable(UInt64)', 10, 10, 1) limit 1000 settings max_threads = 1; From 73a6c920d3dd595d6771e6801e91c5fa0bbf9446 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 2 Mar 2023 11:31:22 -0500 Subject: [PATCH 211/470] Update date-time-functions.md closes #42758 --- .../functions/date-time-functions.md | 39 +++++++++++++++++-- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f6af8abcbaf..a6043d2507e 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1126,15 +1126,48 @@ Rounds the time to the half hour. ## toYYYYMM -Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 100 + MM). +Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 100 + MM). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. + +### example +```sql +SELECT + toYYYYMM(now(), 'US/Eastern') +``` +```response +┌─toYYYYMM(now(), 'US/Eastern')─┐ +│ 202303 │ +└───────────────────────────────┘ +``` ## toYYYYMMDD -Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 10000 + MM \* 100 + DD). +Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 10000 + MM \* 100 + DD). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. + +### example +```sql +SELECT + toYYYYMMDD(now(), 'US/Eastern') +``` +```response +┌─toYYYYMMDD(now(), 'US/Eastern')─┐ +│ 20230302 │ +└─────────────────────────────────┘ +``` ## toYYYYMMDDhhmmss -Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). +Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. + +### example +```sql +SELECT + toYYYYMMDDhhmmss(now(), 'US/Eastern') +``` +```response +┌─toYYYYMMDDhhmmss(now(), 'US/Eastern')─┐ +│ 20230302112209 │ +└───────────────────────────────────────┘ +``` ## addYears, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addQuarters From 63eeec1785c7fa6553e93f5cbe0d94ce0647d292 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 2 Mar 2023 17:44:48 +0100 Subject: [PATCH 212/470] Mark 01771_bloom_filter_not_has as no-parallel and long This test can take long enough time in debug build > 10min, and because clickhouse-test does not wait the test in this case and simply call DROP DATABASE, this will eventually lead to error during DROP since the table is still in use. CI: https://s3.amazonaws.com/clickhouse-test-reports/45491/99329d868232d9377d7f808763e951e6f15fd71c/stateless_tests__debug__%5B5/5%5D.html Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/01771_bloom_filter_not_has.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01771_bloom_filter_not_has.sql b/tests/queries/0_stateless/01771_bloom_filter_not_has.sql index ab0e3d308f9..f945cbde56b 100644 --- a/tests/queries/0_stateless/01771_bloom_filter_not_has.sql +++ b/tests/queries/0_stateless/01771_bloom_filter_not_has.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel, long DROP TABLE IF EXISTS bloom_filter_null_array; CREATE TABLE bloom_filter_null_array (v Array(Int32), INDEX idx v TYPE bloom_filter GRANULARITY 3) ENGINE = MergeTree() ORDER BY v; INSERT INTO bloom_filter_null_array SELECT [number] FROM numbers(10000000); From d378ae9c7698685cb187deff8e7e490058d5ccd5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 17:05:11 +0000 Subject: [PATCH 213/470] Docs: Switch two rows for consistency --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f6af8abcbaf..49b0bc25edd 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1231,8 +1231,8 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %e | day of the month, space-padded (1-31) |   2 | | %f | fractional second from the fractional part of DateTime64 | 1234560 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | -| %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | +| %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | | %h | hour in 12h format (01-12) | 09 | | %H | hour in 24h format (00-23) | 22 | | %i | minute (00-59) | 33 | From 14351922e3f250c4c92383462f701fcaef198054 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Mar 2023 17:20:29 +0000 Subject: [PATCH 214/470] Fix possible deadlock in QueryStatus --- src/Interpreters/ProcessList.cpp | 58 ++++++++++++++++++++++++++++---- src/Interpreters/ProcessList.h | 16 ++++++++- 2 files changed, 66 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 49d7989ac5e..52674dc1c77 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -373,6 +373,19 @@ QueryStatus::~QueryStatus() } } +void QueryStatus::ExecutorHolder::cancel() +{ + std::lock_guard lock(mutex); + if (executor) + executor->cancel(); +} + +void QueryStatus::ExecutorHolder::remove() +{ + std::lock_guard lock(mutex); + executor = nullptr; +} + CancellationCode QueryStatus::cancelQuery(bool) { if (is_killed.load()) @@ -380,9 +393,37 @@ CancellationCode QueryStatus::cancelQuery(bool) is_killed.store(true); - std::lock_guard lock(executors_mutex); - for (auto * e : executors) - e->cancel(); + std::unique_lock lock(executors_mutex); + + /// Track all cancelled executors. + std::unordered_set cancelled; + /// We cancel executors from the left to the right, so if the last executor + /// was cancelled, then all executors were cancelled. + while (!cancelled.contains(executors.back().get())) + { + size_t size = executors.size(); + /// We should create a copy of executor holder, because it can be + /// removed from vector in removePipelineExecutor from another thread + /// and reference will be invalid. + for (auto e : executors) + { + if (cancelled.contains(e.get())) + continue; + /// We should call cancel() with unlocked executors_mutex, because + /// cancel() can try to lock some internal mutex that is already locked by query executing + /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, + /// which will lead to deadlock. + lock.unlock(); + e->cancel(); + lock.lock(); + cancelled.insert(e.get()); + /// While executors_mutex was unlocked, removePipelineExecutor could be called and + /// the size of executors could have changed. In this case we should start iterating + /// over it again to avoid using invalid iterators. + if (executors.size() != size) + break; + } + } return CancellationCode::CancelSent; } @@ -396,15 +437,18 @@ void QueryStatus::addPipelineExecutor(PipelineExecutor * e) throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query was cancelled"); std::lock_guard lock(executors_mutex); - assert(std::find(executors.begin(), executors.end(), e) == executors.end()); - executors.push_back(e); + assert(std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }) == executors.end()); + executors.push_back(std::make_shared(e)); } void QueryStatus::removePipelineExecutor(PipelineExecutor * e) { std::lock_guard lock(executors_mutex); - assert(std::find(executors.begin(), executors.end(), e) != executors.end()); - std::erase_if(executors, [e](PipelineExecutor * x) { return x == e; }); + auto it = std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }); + assert(it != executors.end()); + /// Invalidate executor pointer inside holder. + (*it)->remove(); + executors.erase(it); } bool QueryStatus::checkTimeLimit() diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index d5c136ab62a..30bfde4e218 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -119,8 +119,22 @@ protected: mutable std::mutex executors_mutex; + struct ExecutorHolder + { + ExecutorHolder(PipelineExecutor * e) : executor(e) {} + + void cancel(); + + void remove(); + + PipelineExecutor * executor; + std::mutex mutex; + }; + + using ExecutorHolderPtr = std::shared_ptr; + /// Array of PipelineExecutors to be cancelled when a cancelQuery is received - std::vector executors; + std::vector executors; enum QueryStreamsStatus { From fc8ea01c2b2a198d2170f8c4f9ca775f5d3a37b4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Mar 2023 18:23:31 +0100 Subject: [PATCH 215/470] use unique names and paths in test_replicated_database --- .../test_replicated_database/test.py | 249 +++++++++--------- 1 file changed, 126 insertions(+), 123 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ead9a762b1b..c71c76244ce 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -80,15 +80,15 @@ def started_cluster(): def test_create_replicated_table(started_cluster): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica' || '1');" + "CREATE DATABASE create_replicated_table ENGINE = Replicated('/test/create_replicated_table', 'shard1', 'replica' || '1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE create_replicated_table ENGINE = Replicated('/test/create_replicated_table', 'shard1', 'replica2');" ) assert ( "Explicit zookeeper_path and replica_name are specified" in main_node.query_and_get_error( - "CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "CREATE TABLE create_replicated_table.replicated_table (d Date, k UInt64, i32 Int32) " "ENGINE=ReplicatedMergeTree('/test/tmp', 'r') ORDER BY k PARTITION BY toYYYYMM(d);" ) ) @@ -96,7 +96,7 @@ def test_create_replicated_table(started_cluster): assert ( "Explicit zookeeper_path and replica_name are specified" in main_node.query_and_get_error( - "CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "CREATE TABLE create_replicated_table.replicated_table (d Date, k UInt64, i32 Int32) " "ENGINE=ReplicatedMergeTree('/test/tmp', 'r') ORDER BY k PARTITION BY toYYYYMM(d);" ) ) @@ -104,39 +104,39 @@ def test_create_replicated_table(started_cluster): assert ( "This syntax for *MergeTree engine is deprecated" in main_node.query_and_get_error( - "CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "CREATE TABLE create_replicated_table.replicated_table (d Date, k UInt64, i32 Int32) " "ENGINE=ReplicatedMergeTree('/test/tmp/{shard}', '{replica}', d, k, 8192);" ) ) main_node.query( - "CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);" + "CREATE TABLE create_replicated_table.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);" ) expected = ( - "CREATE TABLE testdb.replicated_table\\n(\\n `d` Date,\\n `k` UInt64,\\n `i32` Int32\\n)\\n" + "CREATE TABLE create_replicated_table.replicated_table\\n(\\n `d` Date,\\n `k` UInt64,\\n `i32` Int32\\n)\\n" "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\n" "PARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, dummy_node], "testdb.replicated_table", expected) + assert_create_query([main_node, dummy_node], "create_replicated_table.replicated_table", expected) # assert without replacing uuid - assert main_node.query("show create testdb.replicated_table") == dummy_node.query( - "show create testdb.replicated_table" + assert main_node.query("show create create_replicated_table.replicated_table") == dummy_node.query( + "show create create_replicated_table.replicated_table" ) - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE create_replicated_table SYNC") + dummy_node.query("DROP DATABASE create_replicated_table SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_simple_alter_table(started_cluster, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica2');" ) # test_simple_alter_table - name = "testdb.alter_test_{}".format(engine) + name = "test_simple_alter_table.alter_test_{}".format(engine) main_node.query( "CREATE TABLE {} " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " @@ -184,10 +184,10 @@ def test_simple_alter_table(started_cluster, engine): # test_create_replica_after_delay competing_node.query( - "CREATE DATABASE IF NOT EXISTS testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');" + "CREATE DATABASE IF NOT EXISTS test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica3');" ) - name = "testdb.alter_test_{}".format(engine) + name = "test_simple_alter_table.alter_test_{}".format(engine) main_node.query("ALTER TABLE {} ADD COLUMN Added3 UInt32;".format(name)) main_node.query("ALTER TABLE {} DROP COLUMN AddedNested1;".format(name)) main_node.query("ALTER TABLE {} RENAME COLUMN Added1 TO AddedNested1;".format(name)) @@ -207,21 +207,21 @@ def test_simple_alter_table(started_cluster, engine): ) assert_create_query([main_node, dummy_node, competing_node], name, expected) - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") - competing_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE test_simple_alter_table SYNC") + dummy_node.query("DROP DATABASE test_simple_alter_table SYNC") + competing_node.query("DROP DATABASE test_simple_alter_table SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_delete_from_table(started_cluster, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard2', 'replica1');" + "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard2', 'replica1');" ) - name = "testdb.delete_test_{}".format(engine) + name = "delete_from_table.delete_test_{}".format(engine) main_node.query( "CREATE TABLE {} " "(id UInt64, value String) " @@ -238,7 +238,7 @@ def test_delete_from_table(started_cluster, engine): table_for_select = name if not "Replicated" in engine: - table_for_select = "cluster('testdb', {})".format(name) + table_for_select = "cluster('delete_from_table', {})".format(name) for node in [main_node, dummy_node]: assert_eq_with_retry( node, @@ -246,8 +246,8 @@ def test_delete_from_table(started_cluster, engine): expected, ) - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE delete_from_table SYNC") + dummy_node.query("DROP DATABASE delete_from_table SYNC") def get_table_uuid(database, name): @@ -276,17 +276,17 @@ def fixture_attachable_part(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_attach(started_cluster, attachable_part, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica2');" ) name = "alter_attach_test_{}".format(engine) main_node.query( - f"CREATE TABLE testdb.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE alter_attach.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - table_uuid = get_table_uuid("testdb", name) + table_uuid = get_table_uuid("alter_attach", name) # Provide and attach a part to the main node shutil.copytree( attachable_part, @@ -295,113 +295,113 @@ def test_alter_attach(started_cluster, attachable_part, engine): f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0", ), ) - main_node.query(f"ALTER TABLE testdb.{name} ATTACH PART 'all_1_1_0'") + main_node.query(f"ALTER TABLE alter_attach.{name} ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + assert main_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" else: - assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "" - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "" + main_node.query("DROP DATABASE alter_attach SYNC") + dummy_node.query("DROP DATABASE alter_attach SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_part(started_cluster, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica2');" ) table = f"alter_drop_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE alter_drop_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + main_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") - main_node.query(f"ALTER TABLE testdb.{table} DROP PART '{part_name}'") - assert main_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + dummy_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (456)") + main_node.query(f"ALTER TABLE alter_drop_part.{table} DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" else: - assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "456\n" - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "456\n" + main_node.query("DROP DATABASE alter_drop_part SYNC") + dummy_node.query("DROP DATABASE alter_drop_part SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_detach_part(started_cluster, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica2');" ) table = f"alter_detach_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE alter_detach_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + main_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") - main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + dummy_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (456)") + main_node.query(f"ALTER TABLE alter_detach_part.{table} DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_detach_part' AND table='{table}'" assert main_node.query(detached_parts_query) == f"{part_name}\n" if engine == "ReplicatedMergeTree": # The detach operation is still replicated at the table engine level assert dummy_node.query(detached_parts_query) == f"{part_name}\n" else: assert dummy_node.query(detached_parts_query) == "" - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE alter_detach_part SYNC") + dummy_node.query("DROP DATABASE alter_detach_part SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_detached_part(started_cluster, engine): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica2');" ) table = f"alter_drop_detached_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE alter_drop_detached_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") - main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + main_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (123)") + main_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") - dummy_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") - main_node.query(f"ALTER TABLE testdb.{table} DROP DETACHED PART '{part_name}'") - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + dummy_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (456)") + dummy_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'") + main_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_drop_detached_part' AND table='{table}'" assert main_node.query(detached_parts_query) == "" assert dummy_node.query(detached_parts_query) == f"{part_name}\n" - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE alter_drop_detached_part SYNC") + dummy_node.query("DROP DATABASE alter_drop_detached_part SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_partition(started_cluster, engine): main_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/clickhouse/databases/test_alter_drop_partition', 'shard1', 'replica1');" + "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/clickhouse/databases/test_alter_drop_partition', 'shard1', 'replica2');" + "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica2');" ) snapshotting_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/clickhouse/databases/test_alter_drop_partition', 'shard2', 'replica1');" + "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard2', 'replica1');" ) table = f"alter_drop_partition.alter_drop_{engine}" @@ -430,52 +430,52 @@ def test_alter_drop_partition(started_cluster, engine): def test_alter_fetch(started_cluster): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alter_fetch ENGINE = Replicated('/test/alter_fetch', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alter_fetch ENGINE = Replicated('/test/alter_fetch', 'shard1', 'replica2');" ) main_node.query( - "CREATE TABLE testdb.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)" + "CREATE TABLE alter_fetch.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)" ) main_node.query( - "CREATE TABLE testdb.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)" + "CREATE TABLE alter_fetch.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)" ) - main_node.query("INSERT INTO testdb.fetch_source VALUES (123)") - table_uuid = get_table_uuid("testdb", "fetch_source") + main_node.query("INSERT INTO alter_fetch.fetch_source VALUES (123)") + table_uuid = get_table_uuid("alter_fetch", "fetch_source") main_node.query( - f"ALTER TABLE testdb.fetch_target FETCH PART 'all_0_0_0' FROM '/clickhouse/tables/{table_uuid}/{{shard}}' " + f"ALTER TABLE alter_fetch.fetch_target FETCH PART 'all_0_0_0' FROM '/clickhouse/tables/{table_uuid}/{{shard}}' " ) - detached_parts_query = "SELECT name FROM system.detached_parts WHERE database='testdb' AND table='fetch_target'" + detached_parts_query = "SELECT name FROM system.detached_parts WHERE database='alter_fetch' AND table='fetch_target'" assert main_node.query(detached_parts_query) == "all_0_0_0\n" assert dummy_node.query(detached_parts_query) == "" - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE alter_fetch SYNC") + dummy_node.query("DROP DATABASE alter_fetch SYNC") def test_alters_from_different_replicas(started_cluster): main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard1', 'replica2');" ) # test_alters_from_different_replicas competing_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard1', 'replica3');" ) main_node.query( - "CREATE TABLE testdb.concurrent_test " + "CREATE TABLE alters_from_different_replicas.concurrent_test " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " "ENGINE = MergeTree PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);" ) main_node.query( - "CREATE TABLE testdb.dist AS testdb.concurrent_test ENGINE = Distributed(testdb, testdb, concurrent_test, CounterID)" + "CREATE TABLE alters_from_different_replicas.dist AS alters_from_different_replicas.concurrent_test ENGINE = Distributed(alters_from_different_replicas, alters_from_different_replicas, concurrent_test, CounterID)" ) dummy_node.stop_clickhouse(kill=True) @@ -484,7 +484,7 @@ def test_alters_from_different_replicas(started_cluster): assert ( "There are 1 unfinished hosts (0 of them are currently active)" in competing_node.query_and_get_error( - "ALTER TABLE testdb.concurrent_test ADD COLUMN Added0 UInt32;", + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", settings=settings, ) ) @@ -493,7 +493,7 @@ def test_alters_from_different_replicas(started_cluster): "distributed_ddl_output_mode": "null_status_on_timeout", } assert "shard1\treplica2\tQUEUED\t" in main_node.query( - "ALTER TABLE testdb.concurrent_test ADD COLUMN Added2 UInt32;", + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added2 UInt32;", settings=settings, ) settings = { @@ -501,22 +501,22 @@ def test_alters_from_different_replicas(started_cluster): "distributed_ddl_output_mode": "never_throw", } assert "shard1\treplica2\tQUEUED\t" in competing_node.query( - "ALTER TABLE testdb.concurrent_test ADD COLUMN Added1 UInt32 AFTER Added0;", + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added1 UInt32 AFTER Added0;", settings=settings, ) dummy_node.start_clickhouse() main_node.query( - "ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;" + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;" ) competing_node.query( - "ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;" + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;" ) main_node.query( - "ALTER TABLE testdb.concurrent_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;" + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;" ) expected = ( - "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" + "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" " `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32,\\n" " `Added0` UInt32,\\n `Added1` UInt32,\\n `Added2` UInt32,\\n `AddedNested1.A` Array(UInt32),\\n" " `AddedNested1.B` Array(UInt64),\\n `AddedNested1.C` Array(String),\\n `AddedNested2.A` Array(UInt32),\\n" @@ -524,51 +524,51 @@ def test_alters_from_different_replicas(started_cluster): "ENGINE = MergeTree\\nPARTITION BY toYYYYMM(StartDate)\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected) + assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) # test_create_replica_after_delay - main_node.query("DROP TABLE testdb.concurrent_test SYNC") + main_node.query("DROP TABLE alters_from_different_replicas.concurrent_test SYNC") main_node.query( - "CREATE TABLE testdb.concurrent_test " + "CREATE TABLE alters_from_different_replicas.concurrent_test " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " "ENGINE = ReplicatedMergeTree ORDER BY CounterID;" ) expected = ( - "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" + "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" " `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected) + assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) main_node.query( - "INSERT INTO testdb.dist (CounterID, StartDate, UserID) SELECT number, addDays(toDate('2020-02-02'), number), intHash32(number) FROM numbers(10)" + "INSERT INTO alters_from_different_replicas.dist (CounterID, StartDate, UserID) SELECT number, addDays(toDate('2020-02-02'), number), intHash32(number) FROM numbers(10)" ) # test_replica_restart main_node.restart_clickhouse() expected = ( - "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" + "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" " `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192" ) # test_snapshot_and_snapshot_recover snapshotting_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard2', 'replica1');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard2', 'replica1');" ) snapshot_recovering_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard2', 'replica2');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard2', 'replica2');" ) - assert_create_query(all_nodes, "testdb.concurrent_test", expected) + assert_create_query(all_nodes, "alters_from_different_replicas.concurrent_test", expected) - main_node.query("SYSTEM FLUSH DISTRIBUTED testdb.dist") + main_node.query("SYSTEM FLUSH DISTRIBUTED alters_from_different_replicas.dist") main_node.query( - "ALTER TABLE testdb.concurrent_test UPDATE StartDate = addYears(StartDate, 1) WHERE 1" + "ALTER TABLE alters_from_different_replicas.concurrent_test UPDATE StartDate = addYears(StartDate, 1) WHERE 1" ) - res = main_node.query("ALTER TABLE testdb.concurrent_test DELETE WHERE UserID % 2") + res = main_node.query("ALTER TABLE alters_from_different_replicas.concurrent_test DELETE WHERE UserID % 2") assert ( "shard1\treplica1\tOK" in res and "shard1\treplica2\tOK" in res @@ -585,28 +585,28 @@ def test_alters_from_different_replicas(started_cluster): ) assert ( main_node.query( - "SELECT shard_num, replica_num, host_name FROM system.clusters WHERE cluster='testdb'" + "SELECT shard_num, replica_num, host_name FROM system.clusters WHERE cluster='alters_from_different_replicas'" ) == expected ) # test_drop_and_create_replica - main_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE alters_from_different_replicas SYNC") main_node.query( - "CREATE DATABASE testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard1', 'replica1');" ) expected = ( - "CREATE TABLE testdb.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" + "CREATE TABLE alters_from_different_replicas.concurrent_test\\n(\\n `CounterID` UInt32,\\n `StartDate` Date,\\n `UserID` UInt32,\\n" " `VisitID` UInt32,\\n `NestedColumn.A` Array(UInt8),\\n `NestedColumn.S` Array(String),\\n `ToDrop` UInt32\\n)\\n" "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "testdb.concurrent_test", expected) - assert_create_query(all_nodes, "testdb.concurrent_test", expected) + assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) + assert_create_query(all_nodes, "alters_from_different_replicas.concurrent_test", expected) for node in all_nodes: - node.query("SYSTEM SYNC REPLICA testdb.concurrent_test") + node.query("SYSTEM SYNC REPLICA alters_from_different_replicas.concurrent_test") expected = ( "0\t2021-02-02\t4249604106\n" @@ -618,14 +618,14 @@ def test_alters_from_different_replicas(started_cluster): assert_eq_with_retry( dummy_node, - "SELECT CounterID, StartDate, UserID FROM testdb.dist ORDER BY CounterID", + "SELECT CounterID, StartDate, UserID FROM alters_from_different_replicas.dist ORDER BY CounterID", expected, ) - main_node.query("DROP DATABASE testdb SYNC") - dummy_node.query("DROP DATABASE testdb SYNC") - competing_node.query("DROP DATABASE testdb SYNC") - snapshotting_node.query("DROP DATABASE testdb SYNC") - snapshot_recovering_node.query("DROP DATABASE testdb SYNC") + main_node.query("DROP DATABASE alters_from_different_replicas SYNC") + dummy_node.query("DROP DATABASE alters_from_different_replicas SYNC") + competing_node.query("DROP DATABASE alters_from_different_replicas SYNC") + snapshotting_node.query("DROP DATABASE alters_from_different_replicas SYNC") + snapshot_recovering_node.query("DROP DATABASE alters_from_different_replicas SYNC") def create_some_tables(db): @@ -1063,10 +1063,10 @@ def test_server_uuid(started_cluster): def test_sync_replica(started_cluster): main_node.query( - "CREATE DATABASE test_sync_database ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica1');" + "CREATE DATABASE test_sync_database ENGINE = Replicated('/test/sync_replica', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE test_sync_database ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica2');" + "CREATE DATABASE test_sync_database ENGINE = Replicated('/test/sync_replica', 'shard1', 'replica2');" ) number_of_tables = 1000 @@ -1113,17 +1113,20 @@ def test_sync_replica(started_cluster): ) lp1 = main_node.query( - "select value from system.zookeeper where path='/clickhouse/databases/test1/replicas/shard1|replica1' and name='log_ptr'" + "select value from system.zookeeper where path='/test/sync_replica/replicas/shard1|replica1' and name='log_ptr'" ) lp2 = main_node.query( - "select value from system.zookeeper where path='/clickhouse/databases/test1/replicas/shard1|replica2' and name='log_ptr'" + "select value from system.zookeeper where path='/test/sync_replica/replicas/shard1|replica2' and name='log_ptr'" ) max_lp = main_node.query( - "select value from system.zookeeper where path='/clickhouse/databases/test1/' and name='max_log_ptr'" + "select value from system.zookeeper where path='/test/sync_replica/' and name='max_log_ptr'" ) assert lp1 == max_lp assert lp2 == max_lp + main_node.query("DROP DATABASE test_sync_database SYNC") + dummy_node.query("DROP DATABASE test_sync_database SYNC") + def test_force_synchronous_settings(started_cluster): main_node.query( From 4457ebab48ab60d8cae6838a40e73f6e72c8b08e Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 2 Mar 2023 12:31:56 -0500 Subject: [PATCH 216/470] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index a6043d2507e..9dd6ef63ec1 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1156,7 +1156,7 @@ SELECT ## toYYYYMMDDhhmmss -Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. +Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. ### example ```sql From fdcbec4fee7df1bb5a51249cef96964b5245c822 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Mar 2023 17:35:56 +0000 Subject: [PATCH 217/470] Automatic style fix --- .../test_replicated_database/test.py | 57 ++++++++++++++----- 1 file changed, 43 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index c71c76244ce..2ab2fe499ff 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -118,11 +118,13 @@ def test_create_replicated_table(started_cluster): "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\n" "PARTITION BY toYYYYMM(d)\\nORDER BY k\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, dummy_node], "create_replicated_table.replicated_table", expected) - # assert without replacing uuid - assert main_node.query("show create create_replicated_table.replicated_table") == dummy_node.query( - "show create create_replicated_table.replicated_table" + assert_create_query( + [main_node, dummy_node], "create_replicated_table.replicated_table", expected ) + # assert without replacing uuid + assert main_node.query( + "show create create_replicated_table.replicated_table" + ) == dummy_node.query("show create create_replicated_table.replicated_table") main_node.query("DROP DATABASE create_replicated_table SYNC") dummy_node.query("DROP DATABASE create_replicated_table SYNC") @@ -330,7 +332,10 @@ def test_alter_drop_part(started_cluster, engine): # The DROP operation is still replicated at the table engine level assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" else: - assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "456\n" + assert ( + dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") + == "456\n" + ) main_node.query("DROP DATABASE alter_drop_part SYNC") dummy_node.query("DROP DATABASE alter_drop_part SYNC") @@ -379,11 +384,17 @@ def test_alter_drop_detached_part(started_cluster, engine): f"CREATE TABLE alter_drop_detached_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) main_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (123)") - main_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'") + main_node.query( + f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + ) if engine == "MergeTree": dummy_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (456)") - dummy_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'") - main_node.query(f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'") + dummy_node.query( + f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + ) + main_node.query( + f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'" + ) detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_drop_detached_part' AND table='{table}'" assert main_node.query(detached_parts_query) == "" assert dummy_node.query(detached_parts_query) == f"{part_name}\n" @@ -524,7 +535,11 @@ def test_alters_from_different_replicas(started_cluster): "ENGINE = MergeTree\\nPARTITION BY toYYYYMM(StartDate)\\nORDER BY (CounterID, StartDate, intHash32(UserID), VisitID)\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) + assert_create_query( + [main_node, competing_node], + "alters_from_different_replicas.concurrent_test", + expected, + ) # test_create_replica_after_delay main_node.query("DROP TABLE alters_from_different_replicas.concurrent_test SYNC") @@ -540,7 +555,11 @@ def test_alters_from_different_replicas(started_cluster): "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) + assert_create_query( + [main_node, competing_node], + "alters_from_different_replicas.concurrent_test", + expected, + ) main_node.query( "INSERT INTO alters_from_different_replicas.dist (CounterID, StartDate, UserID) SELECT number, addDays(toDate('2020-02-02'), number), intHash32(number) FROM numbers(10)" @@ -562,13 +581,17 @@ def test_alters_from_different_replicas(started_cluster): snapshot_recovering_node.query( "CREATE DATABASE alters_from_different_replicas ENGINE = Replicated('/test/alters_from_different_replicas', 'shard2', 'replica2');" ) - assert_create_query(all_nodes, "alters_from_different_replicas.concurrent_test", expected) + assert_create_query( + all_nodes, "alters_from_different_replicas.concurrent_test", expected + ) main_node.query("SYSTEM FLUSH DISTRIBUTED alters_from_different_replicas.dist") main_node.query( "ALTER TABLE alters_from_different_replicas.concurrent_test UPDATE StartDate = addYears(StartDate, 1) WHERE 1" ) - res = main_node.query("ALTER TABLE alters_from_different_replicas.concurrent_test DELETE WHERE UserID % 2") + res = main_node.query( + "ALTER TABLE alters_from_different_replicas.concurrent_test DELETE WHERE UserID % 2" + ) assert ( "shard1\treplica1\tOK" in res and "shard1\treplica2\tOK" in res @@ -602,8 +625,14 @@ def test_alters_from_different_replicas(started_cluster): "ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\')\\nORDER BY CounterID\\nSETTINGS index_granularity = 8192" ) - assert_create_query([main_node, competing_node], "alters_from_different_replicas.concurrent_test", expected) - assert_create_query(all_nodes, "alters_from_different_replicas.concurrent_test", expected) + assert_create_query( + [main_node, competing_node], + "alters_from_different_replicas.concurrent_test", + expected, + ) + assert_create_query( + all_nodes, "alters_from_different_replicas.concurrent_test", expected + ) for node in all_nodes: node.query("SYSTEM SYNC REPLICA alters_from_different_replicas.concurrent_test") From 167e4903a8f5b175d587efaed15c154ee0837991 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 2 Mar 2023 18:45:59 +0100 Subject: [PATCH 218/470] Fix concrete columns PREWHERE support This is the fix for the IStorage::supportedPrewhereColumns() API. Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- .../MergeTree/MergeTreeWhereOptimizer.cpp | 16 ++++++++++++++++ src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 3 +++ ...rge_prewhere_different_default_kind.reference | 9 +++++---- ...575_merge_prewhere_different_default_kind.sql | 11 +++++++---- 5 files changed, 32 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 318ea5fdf42..a4eed606de1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -593,8 +593,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); - if (supported_prewhere_columns.has_value()) - std::erase_if(queried_columns, [&](const auto & name) { return !supported_prewhere_columns->contains(name); }); MergeTreeWhereOptimizer{ current_info, @@ -602,6 +600,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::move(column_compressed_sizes), metadata_snapshot, queried_columns, + supported_prewhere_columns, log}; } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3a866cc8934..fdddc29048b 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -32,10 +32,12 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, + const std::optional & supported_columns_, Poco::Logger * log_) : table_columns{collections::map( metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })} , queried_columns{queried_columns_} + , supported_columns{supported_columns_} , sorting_key_names{NameSet( metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())} , block_with_constants{KeyCondition::getBlockWithConstants(query_info.query->clone(), query_info.syntax_analyzer_result, context)} @@ -195,6 +197,8 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node, && (!is_final || isExpressionOverSortingKey(node)) /// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded. && isSubsetOfTableColumns(cond.identifiers) + /// Some identifiers can unable to support PREWHERE (usually because of different types in Merge engine) + && identifiersSupportsPrewhere(cond.identifiers) /// Do not move conditions involving all queried columns. && cond.identifiers.size() < queried_columns.size(); @@ -321,6 +325,18 @@ UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identif return size; } +bool MergeTreeWhereOptimizer::identifiersSupportsPrewhere(const NameSet & identifiers) const +{ + if (!supported_columns.has_value()) + return true; + + for (const auto & identifier : identifiers) + if (!supported_columns->contains(identifier)) + return false; + + return true; +} + bool MergeTreeWhereOptimizer::isExpressionOverSortingKey(const ASTPtr & ast) const { if (const auto * func = ast->as()) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index f37255bdbee..8953923542e 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -39,6 +39,7 @@ public: std::unordered_map column_sizes_, const StorageMetadataPtr & metadata_snapshot, const Names & queried_columns_, + const std::optional & supported_columns_, Poco::Logger * log_); private: @@ -82,6 +83,7 @@ private: void optimizeArbitrary(ASTSelectQuery & select) const; UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const; + bool identifiersSupportsPrewhere(const NameSet & identifiers) const; bool isExpressionOverSortingKey(const ASTPtr & ast) const; @@ -105,6 +107,7 @@ private: const StringSet table_columns; const Names queried_columns; + const std::optional supported_columns; const NameSet sorting_key_names; const Block block_with_constants; Poco::Logger * log; diff --git a/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.reference b/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.reference index 32db2512eab..c17e235ddad 100644 --- a/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.reference +++ b/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.reference @@ -1,12 +1,13 @@ -- { echoOn } -- for pure PREWHERE it is not addressed yet. SELECT * FROM m PREWHERE a = 'OK'; -OK 0 +OK 1970-01-01 0 SELECT * FROM m PREWHERE f = 0; -- { serverError ILLEGAL_PREWHERE } SELECT * FROM m WHERE f = 0 SETTINGS optimize_move_to_prewhere=0; -OK 0 +OK 1970-01-01 0 SELECT * FROM m WHERE f = 0 SETTINGS optimize_move_to_prewhere=1; -OK 0 +OK 1970-01-01 0 -- { echoOn } SELECT * FROM m WHERE f = 0 SETTINGS optimize_move_to_prewhere=1; -OK 0 +OK 1970-01-01 0 +OK 1970-01-01 0 diff --git a/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.sql b/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.sql index 0f1d582a26e..88c7923a570 100644 --- a/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.sql +++ b/tests/queries/0_stateless/02575_merge_prewhere_different_default_kind.sql @@ -6,20 +6,22 @@ DROP TABLE IF EXISTS t2; CREATE TABLE m ( - `a` String, - `f` UInt8 + a String, + date Date, + f UInt8 ) ENGINE = Merge(currentDatabase(), '^(t1|t2)$'); CREATE TABLE t1 ( a String, + date Date, f UInt8 ALIAS 0 ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; -INSERT INTO t1 VALUES ('OK'); +INSERT INTO t1 (a) VALUES ('OK'); -- { echoOn } -- for pure PREWHERE it is not addressed yet. @@ -32,12 +34,13 @@ SELECT * FROM m WHERE f = 0 SETTINGS optimize_move_to_prewhere=1; CREATE TABLE t2 ( a String, + date Date, f UInt8, ) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; -INSERT INTO t2 VALUES ('OK', 1); +INSERT INTO t2 (a) VALUES ('OK'); -- { echoOn } SELECT * FROM m WHERE f = 0 SETTINGS optimize_move_to_prewhere=1; From a0a40fb0575582ade22514dd9d95f20db887ae8b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 17:51:43 +0000 Subject: [PATCH 219/470] nam --> name --- tests/ci/compatibility_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 398f3df3deb..688fe883c1e 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -210,7 +210,7 @@ def main(): max_glibc_version = "" if "x86" in args.check_name: max_glibc_version = "2.4" - elif "aarch64" in args.check_nam: + elif "aarch64" in args.check_name: max_glibc_version = "2.18" # because of build with newer sysroot? else: raise Exception("Can't determine max glibc version") From 2db7df61537e1eb25f356d7d7625f2907721d257 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 2 Mar 2023 12:55:04 -0500 Subject: [PATCH 220/470] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 41e50977bba..1b75ceaa9b4 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1126,7 +1126,7 @@ Rounds the time to the half hour. ## toYYYYMM -Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 100 + MM). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. +Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 100 + MM). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. ### example ```sql From 9cb70723d1afd51b57e8010720dd3f47e88e23bc Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 2 Mar 2023 12:55:16 -0500 Subject: [PATCH 221/470] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 1b75ceaa9b4..0892f0d7ef7 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1141,7 +1141,7 @@ SELECT ## toYYYYMMDD -Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 10000 + MM \* 100 + DD). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. +Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 10000 + MM \* 100 + DD). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. ### example ```sql From fbabba78c905cca5b1c24ae4fdd2503f5870a146 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 2 Mar 2023 12:55:23 -0500 Subject: [PATCH 222/470] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0892f0d7ef7..ef0475027dd 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1156,7 +1156,7 @@ SELECT ## toYYYYMMDDhhmmss -Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. +Converts a date or date with time to a UInt64 number containing the year and month number (YYYY \* 10000000000 + MM \* 100000000 + DD \* 1000000 + hh \* 10000 + mm \* 100 + ss). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. ### example ```sql From a6c2473518bf78d4ccfa2f06f2621a09d3f4d092 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 1 Mar 2023 18:16:03 +0100 Subject: [PATCH 223/470] do flushUntrackedMemory when context swith --- src/Common/CurrentThread.cpp | 3 +-- src/Common/ThreadStatus.cpp | 6 ++++++ src/Common/ThreadStatus.h | 2 ++ src/Interpreters/ThreadStatusExt.cpp | 4 +++- 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 526e28c043d..90483171304 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -129,8 +129,7 @@ void CurrentThread::flushUntrackedMemory() if (current_thread->untracked_memory == 0) return; - current_thread->memory_tracker.adjustWithUntrackedMemory(current_thread->untracked_memory); - current_thread->untracked_memory = 0; + current_thread->flushUntrackedMemory(); } } diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 46c171b5cb6..da723f5c89d 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -144,6 +144,12 @@ ThreadStatus::ThreadStatus() #endif } +void ThreadStatus::flushUntrackedMemory() +{ + memory_tracker.adjustWithUntrackedMemory(untracked_memory); + untracked_memory = 0; +} + ThreadStatus::~ThreadStatus() { memory_tracker.adjustWithUntrackedMemory(untracked_memory); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 20550a63312..e620413c8eb 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -290,6 +290,8 @@ public: void logToQueryViewsLog(const ViewRuntimeData & vinfo); + void flushUntrackedMemory(); + protected: void applyQuerySettings(); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 84400fc3711..b22748e84ae 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -352,8 +352,10 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) thread_group->threads.erase(this); } performance_counters.setParent(&ProfileEvents::global_counters); - memory_tracker.reset(); + flushUntrackedMemory(); + + memory_tracker.reset(); memory_tracker.setParent(thread_group->memory_tracker.getParent()); query_id.clear(); From 7a9f4aab8ed435008e35b2cb73f634da50fb0630 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 2 Mar 2023 19:16:42 +0100 Subject: [PATCH 224/470] do not call adjustWithUntrackedMemory when untracked_memory is 0 --- src/Common/CurrentThread.cpp | 3 --- src/Common/ThreadStatus.cpp | 5 ++++- src/Storages/MergeTree/MergeList.cpp | 7 +++++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 90483171304..c10d5dbc68c 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -126,9 +126,6 @@ void CurrentThread::flushUntrackedMemory() { if (unlikely(!current_thread)) return; - if (current_thread->untracked_memory == 0) - return; - current_thread->flushUntrackedMemory(); } diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index da723f5c89d..407343aa268 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -146,13 +146,16 @@ ThreadStatus::ThreadStatus() void ThreadStatus::flushUntrackedMemory() { + if (untracked_memory == 0) + return; + memory_tracker.adjustWithUntrackedMemory(untracked_memory); untracked_memory = 0; } ThreadStatus::~ThreadStatus() { - memory_tracker.adjustWithUntrackedMemory(untracked_memory); + flushUntrackedMemory(); if (thread_group) { diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 76d69cc6b7d..fa1887a02e6 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -144,8 +144,11 @@ MergeInfo MergeListElement::getInfo() const MergeListElement::~MergeListElement() { - CurrentThread::getMemoryTracker()->adjustWithUntrackedMemory(untracked_memory); - untracked_memory = 0; + if (untracked_memory != 0) + { + CurrentThread::getMemoryTracker()->adjustWithUntrackedMemory(untracked_memory); + untracked_memory = 0; + } } From edd238273c4d6d0a6462c6f2bc0e7d7656169bf1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Mar 2023 19:51:05 +0100 Subject: [PATCH 225/470] fix another bug in client --- src/Client/ClientBase.cpp | 4 ++-- src/Server/TCPHandler.cpp | 3 +++ .../0_stateless/02232_dist_insert_send_logs_level_hung.sh | 2 +- .../0_stateless/02434_cancel_insert_when_client_dies.sh | 3 ++- .../queries/0_stateless/02435_rollback_cancelled_queries.sh | 5 +++-- 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 96aff9aa304..53eb5080130 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1360,7 +1360,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des throw; } - if (have_data_in_stdin) + if (have_data_in_stdin && !cancelled) sendDataFromStdin(sample, columns_description_for_query, parsed_query); } else if (parsed_insert_query->data) @@ -1370,7 +1370,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des try { sendDataFrom(data_in, sample, columns_description_for_query, parsed_query, have_data_in_stdin); - if (have_data_in_stdin) + if (have_data_in_stdin && !cancelled) sendDataFromStdin(sample, columns_description_for_query, parsed_query); } catch (Exception & e) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a1ef7a98992..768f16a09e0 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1335,6 +1335,7 @@ bool TCPHandler::receivePacket() std::this_thread::sleep_for(ms); } + LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the query"); state.is_cancelled = true; return false; @@ -1378,6 +1379,7 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() { if (packet_type == Protocol::Client::Cancel) { + LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the read task"); state.is_cancelled = true; /// For testing connection collector. if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) @@ -1411,6 +1413,7 @@ std::optional TCPHandler::receivePartitionMergeTreeReadTas { if (packet_type == Protocol::Client::Cancel) { + LOG_INFO(log, "Received 'Cancel' packet from the client, canceling the MergeTree read task"); state.is_cancelled = true; /// For testing connection collector. if (unlikely(sleep_in_receive_cancel.totalMilliseconds())) diff --git a/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh b/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh index 322e7e73991..5ed94148bc1 100755 --- a/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh +++ b/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh @@ -49,7 +49,7 @@ insert_client_opts=( timeout 250s $CLICKHOUSE_CLIENT "${client_opts[@]}" "${insert_client_opts[@]}" -q "insert into function remote('127.2', currentDatabase(), in_02232) select * from numbers(1e6)" # Kill underlying query of remote() to make KILL faster -timeout 30s $CLICKHOUSE_CLIENT "${client_opts[@]}" -q "KILL QUERY WHERE Settings['log_comment'] = '$CLICKHOUSE_LOG_COMMENT' SYNC" --format Null +timeout 60s $CLICKHOUSE_CLIENT "${client_opts[@]}" -q "KILL QUERY WHERE Settings['log_comment'] = '$CLICKHOUSE_LOG_COMMENT' SYNC" --format Null echo $? $CLICKHOUSE_CLIENT "${client_opts[@]}" -nm -q " diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index c4d7367ba14..85aa992d98c 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -93,5 +93,6 @@ $CLICKHOUSE_CLIENT -q 'system flush logs' # Ensure that thread_cancel actually did something $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( - message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or + message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', + 'Query was cancelled or a client has unexpectedly dropped the connection') or message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 7c7ef037e02..408743d395b 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -47,7 +47,7 @@ function insert_data fi if [[ "$IMPLICIT" -eq 0 ]]; then - $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS&close_session=1" | grep -Faq "Transaction is not in RUNNING state" && $CLICKHOUSE_CURL -sS -d 'rollback' "$CLICKHOUSE_URL&$TXN_SETTINGS" + $CLICKHOUSE_CURL -sS -d 'commit' "$CLICKHOUSE_URL&$TXN_SETTINGS&close_session=1" fi } @@ -109,7 +109,8 @@ $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 # Ensure that thread_cancel actually did something $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( - message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes') or + message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', + 'Query was cancelled or a client has unexpectedly dropped the connection') or message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" wait_for_queries_to_finish 30 From 547319eb371a7853a5703fcb780aec4e894dfc89 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Mar 2023 19:53:47 +0100 Subject: [PATCH 226/470] fix --- tests/queries/0_stateless/02435_rollback_cancelled_queries.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 408743d395b..e4aec6503a4 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -62,7 +62,7 @@ function thread_insert # supress "Killed" messages from bash while true; do export ID="$TEST_MARK$RANDOM" - bash -c insert_data 2>&1| grep -Fav "Killed" + bash -c insert_data 2>&1| grep -Fav "Killed" | grep -Fav "SESSION_IS_LOCKED" | grep -Fav "SESSION_NOT_FOUND" done } From e91cdb4f8df2f9340856a71b255d5b11b095d98e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Mar 2023 19:03:17 +0000 Subject: [PATCH 227/470] Add stupid retries in clickhouse-test health check. --- tests/clickhouse-test | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 124eed01723..7da125bcbfa 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -94,7 +94,7 @@ class HTTPError(Exception): # Helpers to execute queries via HTTP interface. def clickhouse_execute_http( - base_args, query, timeout=30, settings=None, default_format=None, max_http_retries=5 + base_args, query, timeout=30, settings=None, default_format=None, max_http_retries=5, retry_error_codes=False ): if args.secure: client = http.client.HTTPSConnection( @@ -131,7 +131,8 @@ def clickhouse_execute_http( ) res = client.getresponse() data = res.read() - break + if res.status == 200 or (not retry_error_codes): + break except Exception as ex: if i == max_http_retries - 1: raise ex @@ -143,8 +144,8 @@ def clickhouse_execute_http( return data -def clickhouse_execute(base_args, query, timeout=30, settings=None, max_http_retries=5): - return clickhouse_execute_http(base_args, query, timeout, settings, max_http_retries=max_http_retries).strip() +def clickhouse_execute(base_args, query, timeout=30, settings=None, max_http_retries=5, retry_error_codes=False): + return clickhouse_execute_http(base_args, query, timeout, settings, max_http_retries=max_http_retries, retry_error_codes=retry_error_codes).strip() def clickhouse_execute_json(base_args, query, timeout=60, settings=None, max_http_retries=5): @@ -1078,7 +1079,7 @@ class TestCase: @staticmethod def send_test_name_failed(suite: str, case: str): pid = os.getpid() - clickhouse_execute(args, f"SELECT 'Running test {suite}/{case} from pid={pid}'") + clickhouse_execute(args, f"SELECT 'Running test {suite}/{case} from pid={pid}'", retry_error_codes=True) def run_single_test( self, server_logs_level, client_options From 31bc032e0b71e5373b14338622e7d7b2dd7b6d18 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 19:23:00 +0000 Subject: [PATCH 228/470] Docs: Cleanup default value docs --- .../sql-reference/statements/create/table.md | 33 +++++++++---------- src/Interpreters/InterpreterInsertQuery.cpp | 4 +-- 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 54977e1b0ab..4bb6430eff6 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -112,23 +112,21 @@ See also [data_type_default_nullable](../../../operations/settings/settings.md#d ## Default Values -The column description can specify an expression for a default value, in one of the following ways: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. +The column description can specify a default value expression in the form of `DEFAULT expr`, `MATERIALIZED expr`, or `ALIAS expr`. Example: `URLDomain String DEFAULT domain(URL)`. -Example: `URLDomain String DEFAULT domain(URL)`. +The expression `expr` is optional. If it is omitted, the column type must be specified explicitly and the default value will be `0` for numeric columns, `''` (the empty string) for string columns, `[]` (the empty array) for array columns, `1970-01-01` for date columns, or `NULL` for nullable columns. -If an expression for the default value is not defined, the default values will be set to zeros for numbers, empty strings for strings, empty arrays for arrays, and `1970-01-01` for dates or zero unix timestamp for DateTime, NULL for Nullable. +The column type of a default value column can be omitted in which case it is infered from `expr`'s type. For example the type of column `EventDate DEFAULT toDate(EventTime)` will be date. -If the default expression is defined, the column type is optional. If there isn’t an explicitly defined type, the default expression type is used. Example: `EventDate DEFAULT toDate(EventTime)` – the ‘Date’ type will be used for the ‘EventDate’ column. +If both a data type and a default value expression are specified, an implicit type casting function inserted which converts the expression to the specified type. Example: `Hits UInt32 DEFAULT 0` is internally represented as `Hits UInt32 DEFAULT toUInt32(0)`. -If the data type and default expression are defined explicitly, this expression will be cast to the specified type using type casting functions. Example: `Hits UInt32 DEFAULT 0` means the same thing as `Hits UInt32 DEFAULT toUInt32(0)`. - -Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions do not contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. +A default value expression `expr` may reference arbitrary table columns and constants. ClickHouse checks that changes of the table structure do not introduce loops in the expression calculation. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. ### DEFAULT `DEFAULT expr` -Normal default value. If the INSERT query does not specify the corresponding column, it will be filled in by computing the corresponding expression. +Normal default value. If the value of such a column is not specified in an INSERT query, it is computed from `expr`. Example: @@ -154,9 +152,9 @@ SELECT * FROM test; `MATERIALIZED expr` -Materialized expression. Such a column can’t be specified for INSERT, because it is always calculated. -For an INSERT without a list of columns, these columns are not considered. -In addition, this column is not substituted when using an asterisk in a SELECT query. This is to preserve the invariant that the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns. +Materialized expression. Values of such columns are always calculated, they cannot be specified in INSERT queries. + +Also, default value columns of this type are not included in the result of `SELECT *`. This is to preserve the invariant that the result of a `SELECT *` can always be inserted back into the table using `INSERT`. This behavior can be disabled with setting `asterisk_include_materialized_columns`. Example: @@ -192,8 +190,9 @@ SELECT * FROM test SETTINGS asterisk_include_materialized_columns=1; `EPHEMERAL [expr]` -Ephemeral column. Such a column isn't stored in the table and cannot be SELECTed, but can be referenced in the defaults of CREATE statement. If `expr` is omitted type for column is required. -INSERT without list of columns will skip such column, so SELECT/INSERT invariant is preserved - the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns. +Ephemeral column. Columns of this type are not stored in the table and it is not possible to SELECT from them. The only purpose of ephemeral columns is to build default value expressions of other columns from them. + +An insert without explicitly specified columns will skip columns of this type. This is to preserve the invariant that the result of a `SELECT *` can always be inserted back into the table using `INSERT`. Example: @@ -205,7 +204,7 @@ CREATE OR REPLACE TABLE test hexed FixedString(4) DEFAULT unhex(unhexed) ) ENGINE = MergeTree -ORDER BY id +ORDER BY id; INSERT INTO test (id, unhexed) Values (1, '5a90b714'); @@ -227,9 +226,9 @@ hex(hexed): 5A90B714 `ALIAS expr` -Synonym. Such a column isn’t stored in the table at all. -Its values can’t be inserted in a table, and it is not substituted when using an asterisk in a SELECT query. -It can be used in SELECTs if the alias is expanded during query parsing. +Calculated columns (synonym). Column of this type are not stored in the table and it is not possible to INSERT values into them. + +When SELECT queries explicitly reference columns of this type, the value is computed at query time from `expr`. By default, `SELECT *` excludes ALIAS columns. This behavior can be disabled with setting `asteriks_include_alias_columns`. When using the ALTER query to add new columns, old data for these columns is not written. Instead, when reading old data that does not have values for the new columns, expressions are computed on the fly by default. However, if running the expressions requires different columns that are not indicated in the query, these columns will additionally be read, but only for the blocks of data that need it. diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b4a19ea7403..3cae219fa60 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -161,7 +161,7 @@ Block InterpreterInsertQuery::getSampleBlock( if (table_sample_physical.has(current_name)) { if (!allow_materialized) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is a MATERIALIZED column", current_name); res.insert(ColumnWithTypeAndName(table_sample_physical.getByName(current_name).type, current_name)); } @@ -527,7 +527,7 @@ BlockIO InterpreterInsertQuery::execute() { for (const auto & column : metadata_snapshot->getColumns()) if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is a MATERIALIZED column", column.name); } res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); From c848052c7cb00e776aaec7087f735c28ac8b878e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Mar 2023 19:25:50 +0000 Subject: [PATCH 229/470] Add test, rename some tests to fix order --- ...reference => 02581_width_bucket.reference} | 0 ...idth_bucket.sql => 02581_width_bucket.sql} | 0 ...join_subquery_empty_column_list.reference} | 0 ...lyzer_join_subquery_empty_column_list.sql} | 0 ..._async_reading_with_small_limit.reference} | 0 ... 02582_async_reading_with_small_limit.sql} | 0 ...rence => 02583_map_literal_cast.reference} | 0 ...al_cast.sql => 02583_map_literal_cast.sql} | 0 ...4.reference => 02584_range_ipv4.reference} | 0 ...74_range_ipv4.sql => 02584_range_ipv4.sql} | 0 .../02585_query_status_deadlock.reference | 0 .../02585_query_status_deadlock.sh | 24 +++++++++++++++++++ 12 files changed, 24 insertions(+) rename tests/queries/0_stateless/{25337_width_bucket.reference => 02581_width_bucket.reference} (100%) rename tests/queries/0_stateless/{25337_width_bucket.sql => 02581_width_bucket.sql} (100%) rename tests/queries/0_stateless/{25339_analyzer_join_subquery_empty_column_list.reference => 02582_analyzer_join_subquery_empty_column_list.reference} (100%) rename tests/queries/0_stateless/{25339_analyzer_join_subquery_empty_column_list.sql => 02582_analyzer_join_subquery_empty_column_list.sql} (100%) rename tests/queries/0_stateless/{02664_async_reading_with_small_limit.reference => 02582_async_reading_with_small_limit.reference} (100%) rename tests/queries/0_stateless/{02664_async_reading_with_small_limit.sql => 02582_async_reading_with_small_limit.sql} (100%) rename tests/queries/0_stateless/{02670_map_literal_cast.reference => 02583_map_literal_cast.reference} (100%) rename tests/queries/0_stateless/{02670_map_literal_cast.sql => 02583_map_literal_cast.sql} (100%) rename tests/queries/0_stateless/{02674_range_ipv4.reference => 02584_range_ipv4.reference} (100%) rename tests/queries/0_stateless/{02674_range_ipv4.sql => 02584_range_ipv4.sql} (100%) create mode 100644 tests/queries/0_stateless/02585_query_status_deadlock.reference create mode 100755 tests/queries/0_stateless/02585_query_status_deadlock.sh diff --git a/tests/queries/0_stateless/25337_width_bucket.reference b/tests/queries/0_stateless/02581_width_bucket.reference similarity index 100% rename from tests/queries/0_stateless/25337_width_bucket.reference rename to tests/queries/0_stateless/02581_width_bucket.reference diff --git a/tests/queries/0_stateless/25337_width_bucket.sql b/tests/queries/0_stateless/02581_width_bucket.sql similarity index 100% rename from tests/queries/0_stateless/25337_width_bucket.sql rename to tests/queries/0_stateless/02581_width_bucket.sql diff --git a/tests/queries/0_stateless/25339_analyzer_join_subquery_empty_column_list.reference b/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.reference similarity index 100% rename from tests/queries/0_stateless/25339_analyzer_join_subquery_empty_column_list.reference rename to tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.reference diff --git a/tests/queries/0_stateless/25339_analyzer_join_subquery_empty_column_list.sql b/tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql similarity index 100% rename from tests/queries/0_stateless/25339_analyzer_join_subquery_empty_column_list.sql rename to tests/queries/0_stateless/02582_analyzer_join_subquery_empty_column_list.sql diff --git a/tests/queries/0_stateless/02664_async_reading_with_small_limit.reference b/tests/queries/0_stateless/02582_async_reading_with_small_limit.reference similarity index 100% rename from tests/queries/0_stateless/02664_async_reading_with_small_limit.reference rename to tests/queries/0_stateless/02582_async_reading_with_small_limit.reference diff --git a/tests/queries/0_stateless/02664_async_reading_with_small_limit.sql b/tests/queries/0_stateless/02582_async_reading_with_small_limit.sql similarity index 100% rename from tests/queries/0_stateless/02664_async_reading_with_small_limit.sql rename to tests/queries/0_stateless/02582_async_reading_with_small_limit.sql diff --git a/tests/queries/0_stateless/02670_map_literal_cast.reference b/tests/queries/0_stateless/02583_map_literal_cast.reference similarity index 100% rename from tests/queries/0_stateless/02670_map_literal_cast.reference rename to tests/queries/0_stateless/02583_map_literal_cast.reference diff --git a/tests/queries/0_stateless/02670_map_literal_cast.sql b/tests/queries/0_stateless/02583_map_literal_cast.sql similarity index 100% rename from tests/queries/0_stateless/02670_map_literal_cast.sql rename to tests/queries/0_stateless/02583_map_literal_cast.sql diff --git a/tests/queries/0_stateless/02674_range_ipv4.reference b/tests/queries/0_stateless/02584_range_ipv4.reference similarity index 100% rename from tests/queries/0_stateless/02674_range_ipv4.reference rename to tests/queries/0_stateless/02584_range_ipv4.reference diff --git a/tests/queries/0_stateless/02674_range_ipv4.sql b/tests/queries/0_stateless/02584_range_ipv4.sql similarity index 100% rename from tests/queries/0_stateless/02674_range_ipv4.sql rename to tests/queries/0_stateless/02584_range_ipv4.sql diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.reference b/tests/queries/0_stateless/02585_query_status_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.sh b/tests/queries/0_stateless/02585_query_status_deadlock.sh new file mode 100755 index 00000000000..92dd05ef46c --- /dev/null +++ b/tests/queries/0_stateless/02585_query_status_deadlock.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +QUERY_ID="${CLICKHOUSE_DATABASE}_test_02585_query_to_kill_id_1" + +$CLICKHOUSE_CLIENT --query_id="$QUERY_ID" -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 & + + +while true +do + res=$($CLICKHOUSE_CLIENT -q "select query, event_time from system.query_log where query_id = '$QUERY_ID' and query like 'select%' limit 1") + if [ -n "$res" ]; then + break + fi + sleep 1 +done + +$CLICKHOUSE_CLIENT -q "kill query where query_id = '$QUERY_ID' sync" &> /dev/null + From 81b30021db077be76e9244ea92e5ec87a223a515 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 2 Mar 2023 20:48:13 +0100 Subject: [PATCH 230/470] ActionsDAG: do not change result of and() during optimization - part 2 (#47028) --- src/Interpreters/ActionsDAG.cpp | 6 +++--- .../02674_and_consistency.reference | 4 ++++ .../0_stateless/02674_and_consistency.sql | 21 +++++++++++++++++++ 3 files changed, 28 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02674_and_consistency.reference create mode 100644 tests/queries/0_stateless/02674_and_consistency.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index bfb010b6105..daba4c1608d 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1946,7 +1946,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( } auto conjunction = getConjunctionNodes(predicate, allowed_nodes); - if (conjunction.rejected.size() == 1 && WhichDataType{conjunction.rejected.front()->result_type}.isFloat()) + if (conjunction.rejected.size() == 1 && WhichDataType{removeNullable(conjunction.rejected.front()->result_type)}.isFloat()) return nullptr; auto actions = cloneActionsForConjunction(conjunction.allowed, all_inputs); @@ -2014,7 +2014,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( node.children.swap(new_children); *predicate = std::move(node); } - else if (!WhichDataType{new_children.front()->result_type}.isFloat()) + else if (!WhichDataType{removeNullable(new_children.front()->result_type)}.isFloat()) { /// If type is different, cast column. /// This case is possible, cause AND can use any numeric type as argument. @@ -2045,7 +2045,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown( /// remove the AND. /// Just update children and rebuild it. predicate->children.swap(new_children); - if (WhichDataType{predicate->children.front()->result_type}.isFloat()) + if (WhichDataType{removeNullable(predicate->children.front()->result_type)}.isFloat()) { Node node; node.type = ActionType::COLUMN; diff --git a/tests/queries/0_stateless/02674_and_consistency.reference b/tests/queries/0_stateless/02674_and_consistency.reference new file mode 100644 index 00000000000..e74ab1928c5 --- /dev/null +++ b/tests/queries/0_stateless/02674_and_consistency.reference @@ -0,0 +1,4 @@ +10 +#45218 +10 += diff --git a/tests/queries/0_stateless/02674_and_consistency.sql b/tests/queries/0_stateless/02674_and_consistency.sql new file mode 100644 index 00000000000..5988832ba68 --- /dev/null +++ b/tests/queries/0_stateless/02674_and_consistency.sql @@ -0,0 +1,21 @@ +SELECT SUM(number) +FROM +( + SELECT 10 AS number +) +GROUP BY number +HAVING 1 AND sin(SUMOrNull(number)) +SETTINGS enable_optimize_predicate_expression = 0; + +select '#45218'; + +SELECT SUM(number) +FROM +( + SELECT 10 AS number +) +GROUP BY cos(min2(number, number) % number) - number +HAVING ((-sign(-233841197)) IS NOT NULL) AND sin(lcm(SUM(number), SUM(number)) >= ('372497213' IS NOT NULL)) +SETTINGS aggregate_functions_null_for_empty = 1, enable_optimize_predicate_expression = 0; + +select '='; From d4c5ab9dcdcbbbbd4b0d3754385369147f2ede1a Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Fri, 3 Mar 2023 04:01:52 +0800 Subject: [PATCH 231/470] Optimize one nullable key aggregate performance (#45772) --- src/Common/ColumnsHashing.h | 82 +++++++++++------ src/Common/ColumnsHashingImpl.h | 49 +++++++++- src/Common/HashTable/TwoLevelStringHashMap.h | 2 +- src/Interpreters/Aggregator.cpp | 61 +++++++++--- src/Interpreters/Aggregator.h | 92 ++++++++++++++++--- src/Interpreters/JIT/compileFunction.cpp | 8 +- src/Interpreters/JIT/compileFunction.h | 2 +- tests/performance/groupby_onekey_nullable.xml | 49 ++++++++++ 8 files changed, 285 insertions(+), 60 deletions(-) create mode 100644 tests/performance/groupby_onekey_nullable.xml diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index 50db3977519..5ac6ca15812 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -16,7 +16,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -29,26 +28,42 @@ namespace ColumnsHashing /// For the case when there is one numeric key. /// UInt8/16/32/64 for any type with corresponding bit width. -template +template struct HashMethodOneNumber - : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache, need_offset> + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache, need_offset, nullable> { - using Self = HashMethodOneNumber; - using Base = columns_hashing_impl::HashMethodBase; + using Self = HashMethodOneNumber; + using Base = columns_hashing_impl::HashMethodBase; static constexpr bool has_cheap_key_calculation = true; const char * vec; /// If the keys of a fixed length then key_sizes contains their lengths, empty otherwise. - HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + HashMethodOneNumber(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) : Base(key_columns[0]) { - vec = key_columns[0]->getRawData().data(); + if constexpr (nullable) + { + const auto * null_column = checkAndGetColumn(key_columns[0]); + vec = null_column->getNestedColumnPtr()->getRawData().data(); + } + else + { + vec = key_columns[0]->getRawData().data(); + } } - explicit HashMethodOneNumber(const IColumn * column) + explicit HashMethodOneNumber(const IColumn * column) : Base(column) { - vec = column->getRawData().data(); + if constexpr (nullable) + { + const auto * null_column = checkAndGetColumn(column); + vec = null_column->getNestedColumnPtr()->getRawData().data(); + } + else + { + vec = column->getRawData().data(); + } } /// Creates context. Method is called once and result context is used in all threads. @@ -73,22 +88,30 @@ struct HashMethodOneNumber /// For the case when there is one string key. -template +template struct HashMethodString - : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache, need_offset> + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache, need_offset, nullable> { - using Self = HashMethodString; - using Base = columns_hashing_impl::HashMethodBase; + using Self = HashMethodString; + using Base = columns_hashing_impl::HashMethodBase; static constexpr bool has_cheap_key_calculation = false; const IColumn::Offset * offsets; const UInt8 * chars; - HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) : Base(key_columns[0]) { - const IColumn & column = *key_columns[0]; - const ColumnString & column_string = assert_cast(column); + const IColumn * column; + if constexpr (nullable) + { + column = checkAndGetColumn(key_columns[0])->getNestedColumnPtr().get(); + } + else + { + column = key_columns[0]; + } + const ColumnString & column_string = assert_cast(*column); offsets = column_string.getOffsets().data(); chars = column_string.getChars().data(); } @@ -108,28 +131,35 @@ struct HashMethodString } protected: - friend class columns_hashing_impl::HashMethodBase; + friend class columns_hashing_impl::HashMethodBase; }; /// For the case when there is one fixed-length string key. -template +template struct HashMethodFixedString - : public columns_hashing_impl:: - HashMethodBase, Value, Mapped, use_cache, need_offset> + : public columns_hashing_impl::HashMethodBase, Value, Mapped, use_cache, need_offset, nullable> { - using Self = HashMethodFixedString; - using Base = columns_hashing_impl::HashMethodBase; + using Self = HashMethodFixedString; + using Base = columns_hashing_impl::HashMethodBase; static constexpr bool has_cheap_key_calculation = false; size_t n; const ColumnFixedString::Chars * chars; - HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) + HashMethodFixedString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) : Base(key_columns[0]) { - const IColumn & column = *key_columns[0]; - const ColumnFixedString & column_string = assert_cast(column); + const IColumn * column; + if constexpr (nullable) + { + column = checkAndGetColumn(key_columns[0])->getNestedColumnPtr().get(); + } + else + { + column = key_columns[0]; + } + const ColumnFixedString & column_string = assert_cast(*column); n = column_string.getN(); chars = &column_string.getChars(); } @@ -149,7 +179,7 @@ struct HashMethodFixedString } protected: - friend class columns_hashing_impl::HashMethodBase; + friend class columns_hashing_impl::HashMethodBase; }; diff --git a/src/Common/ColumnsHashingImpl.h b/src/Common/ColumnsHashingImpl.h index 03e7b0436e9..3240510ea9b 100644 --- a/src/Common/ColumnsHashingImpl.h +++ b/src/Common/ColumnsHashingImpl.h @@ -6,7 +6,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -140,7 +139,7 @@ public: FindResultImpl(bool found_, size_t off) : FindResultImplBase(found_), FindResultImplOffsetBase(off) {} }; -template +template class HashMethodBase { public: @@ -154,6 +153,19 @@ public: template ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool) { + if constexpr (nullable) + { + if (isNullAt(row)) + { + bool has_null_key = data.hasNullKeyData(); + data.hasNullKeyData() = true; + + if constexpr (has_mapped) + return EmplaceResult(data.getNullKeyData(), data.getNullKeyData(), !has_null_key); + else + return EmplaceResult(!has_null_key); + } + } auto key_holder = static_cast(*this).getKeyHolder(row, pool); return emplaceImpl(key_holder, data); } @@ -161,6 +173,16 @@ public: template ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool) { + if constexpr (nullable) + { + if (isNullAt(row)) + { + if constexpr (has_mapped) + return FindResult(&data.getNullKeyData(), data.hasNullKeyData(), 0); + else + return FindResult(data.hasNullKeyData(), 0); + } + } auto key_holder = static_cast(*this).getKeyHolder(row, pool); return findKeyImpl(keyHolderGetKey(key_holder), data); } @@ -172,10 +194,25 @@ public: return data.hash(keyHolderGetKey(key_holder)); } + ALWAYS_INLINE bool isNullAt(size_t row) const + { + if constexpr (nullable) + { + return null_map->getBool(row); + } + else + { + return false; + } + } + protected: Cache cache; + const IColumn * null_map = nullptr; + bool has_null_data = false; - HashMethodBase() + /// column argument only for nullable column + explicit HashMethodBase(const IColumn * column = nullptr) { if constexpr (consecutive_keys_optimization) { @@ -188,6 +225,11 @@ protected: else cache.value = Value(); } + if constexpr (nullable) + { + + null_map = &checkAndGetColumn(column)->getNullMapColumn(); + } } template @@ -293,7 +335,6 @@ protected: } }; - template struct MappedCache : public PaddedPODArray {}; diff --git a/src/Common/HashTable/TwoLevelStringHashMap.h b/src/Common/HashTable/TwoLevelStringHashMap.h index 6bd8f74dbd6..31b98a952c8 100644 --- a/src/Common/HashTable/TwoLevelStringHashMap.h +++ b/src/Common/HashTable/TwoLevelStringHashMap.h @@ -9,7 +9,7 @@ class TwoLevelStringHashMap : public TwoLevelStringHashTable, StringHashMap>; + using Base = TwoLevelStringHashTable, ImplTable>; using LookupResult = typename Base::LookupResult; using Base::Base; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 55e49eb8d19..229cb818735 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -775,6 +775,31 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod() if (has_nullable_key) { + /// Optimization for one key + if (params.keys_size == 1 && !has_low_cardinality) + { + if (types_removed_nullable[0]->isValueRepresentedByNumber()) + { + size_t size_of_field = types_removed_nullable[0]->getSizeOfValueInMemory(); + if (size_of_field == 1) + return AggregatedDataVariants::Type::nullable_key8; + if (size_of_field == 2) + return AggregatedDataVariants::Type::nullable_key16; + if (size_of_field == 4) + return AggregatedDataVariants::Type::nullable_key32; + if (size_of_field == 8) + return AggregatedDataVariants::Type::nullable_key64; + } + if (isFixedString(types_removed_nullable[0])) + { + return AggregatedDataVariants::Type::nullable_key_fixed_string; + } + if (isString(types_removed_nullable[0])) + { + return AggregatedDataVariants::Type::nullable_key_string; + } + } + if (params.keys_size == num_fixed_contiguous_keys && !has_low_cardinality) { /// Pack if possible all the keys along with information about which key values are nulls @@ -1889,7 +1914,7 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu template -Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const +Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool has_null_key_data [[maybe_unused]]) const { std::exception_ptr exception; size_t aggregate_functions_destroy_index = 0; @@ -1912,8 +1937,12 @@ Block Aggregator::insertResultsIntoColumns(PaddedPODArray & pl continue; auto & final_aggregate_column = out_cols.final_aggregate_columns[i]; - final_aggregate_column = final_aggregate_column->cloneResized(places.size()); - columns_data.emplace_back(getColumnData(final_aggregate_column.get())); + /** + * In convertToBlockImplFinal, additional data with a key of null may be written, + * and additional memory for null data needs to be allocated when using the compiled function + */ + final_aggregate_column = final_aggregate_column->cloneResized(places.size() + (has_null_key_data ? 1 : 0)); + columns_data.emplace_back(getColumnData(final_aggregate_column.get(), (has_null_key_data ? 1 : 0))); } auto insert_aggregates_into_columns_function = compiled_functions.insert_aggregates_into_columns_function; @@ -1982,15 +2011,21 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena std::optional out_cols; std::optional shuffled_key_sizes; PaddedPODArray places; + bool has_null_key_data = false; auto init_out_cols = [&]() { out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) { + /** + * When one_key_nullable_optimization is enabled, null data will be written to the key column and result column in advance. + * And in insertResultsIntoColumns need to allocate memory for null data. + */ if (data.hasNullKeyData()) { + has_null_key_data = Method::one_key_nullable_optimization; out_cols->key_columns[0]->insertDefault(); insertAggregatesIntoColumns(data.getNullKeyData(), out_cols->final_aggregate_columns, arena); data.hasNullKeyData() = false; @@ -2022,7 +2057,7 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena { if (places.size() >= max_block_size) { - res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data)); places.clear(); out_cols.reset(); } @@ -2031,12 +2066,12 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena if constexpr (return_single_block) { - return insertResultsIntoColumns(places, std::move(out_cols.value()), arena); + return insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data); } else { if (out_cols.has_value()) - res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena)); + res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data)); return res; } } @@ -2056,7 +2091,7 @@ Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & a { out_cols = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), aggregates_pools, final, max_block_size); - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) { if (data.hasNullKeyData()) { @@ -2397,7 +2432,7 @@ void NO_INLINE Aggregator::mergeDataNullKey( Table & table_src, Arena * arena) const { - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) { if (table_src.hasNullKeyData()) { @@ -2429,7 +2464,7 @@ void NO_INLINE Aggregator::mergeDataNullKey( template void NO_INLINE Aggregator::mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena) const { - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) mergeDataNullKey(table_dst, table_src, arena); auto merge = [&](AggregateDataPtr & __restrict dst, AggregateDataPtr & __restrict src, bool inserted) @@ -2490,7 +2525,7 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl( Arena * arena) const { /// Note : will create data for NULL key if not exist - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) mergeDataNullKey(table_dst, table_src, arena); table_src.mergeToViaFind(table_dst, [&](AggregateDataPtr dst, AggregateDataPtr & src, bool found) @@ -2518,7 +2553,7 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl( Arena * arena) const { /// Note : will create data for NULL key if not exist - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) mergeDataNullKey(table_dst, table_src, arena); table_src.mergeToViaFind(table_dst, @@ -3194,7 +3229,7 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl( /// For every row. for (size_t i = 0; i < rows; ++i) { - if constexpr (Method::low_cardinality_optimization) + if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) { if (state.isNullAt(i)) { diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 6e776fb9fa8..e2f76f31466 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -159,10 +159,17 @@ using StringHashTableWithNullKey = AggregationDataWithNullKey; using AggregatedDataWithNullableUInt16Key = AggregationDataWithNullKey; +using AggregatedDataWithNullableUInt32Key = AggregationDataWithNullKey; + using AggregatedDataWithNullableUInt64Key = AggregationDataWithNullKey; using AggregatedDataWithNullableStringKey = AggregationDataWithNullKey; +using AggregatedDataWithNullableShortStringKey = AggregationDataWithNullKey; + +using AggregatedDataWithNullableUInt32KeyTwoLevel = AggregationDataWithNullKeyTwoLevel< + TwoLevelHashMap, + TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; using AggregatedDataWithNullableUInt64KeyTwoLevel = AggregationDataWithNullKeyTwoLevel< TwoLevelHashMap, TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; @@ -174,11 +181,10 @@ using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTw TwoLevelHashMapWithSavedHash, TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>; - /// For the case where there is one numeric key. /// FieldType is UInt8/16/32/64 for any type with corresponding bit width. template + bool consecutive_keys_optimization = true, bool nullable = false> struct AggregationMethodOneNumber { using Data = TData; @@ -198,10 +204,11 @@ struct AggregationMethodOneNumber /// To use one `Method` in different threads, use different `State`. using State = ColumnsHashing::HashMethodOneNumber; + Mapped, FieldType, consecutive_keys_optimization, false, nullable>; /// Use optimization for low cardinality. static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = nullable; /// Shuffle key columns before `insertKeyIntoColumns` call if needed. std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } @@ -209,9 +216,20 @@ struct AggregationMethodOneNumber // Insert the key from the hash table into columns. static void insertKeyIntoColumns(const Key & key, std::vector & key_columns, const Sizes & /*key_sizes*/) { + ColumnVectorHelper * column; + if constexpr (nullable) + { + ColumnNullable & nullable_col = assert_cast(*key_columns[0]); + ColumnUInt8 * null_map = assert_cast(&nullable_col.getNullMapColumn()); + null_map->insertDefault(); + column = static_cast(&nullable_col.getNestedColumn()); + } + else + { + column = static_cast(key_columns[0]); + } static_assert(sizeof(FieldType) <= sizeof(Key)); const auto * key_holder = reinterpret_cast(&key); - auto * column = static_cast(key_columns[0]); if constexpr (sizeof(FieldType) < sizeof(Key) && std::endian::native == std::endian::big) column->insertRawData(key_holder + (sizeof(Key) - sizeof(FieldType))); else @@ -242,6 +260,7 @@ struct AggregationMethodString using State = ColumnsHashing::HashMethodString; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = false; std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } @@ -253,7 +272,7 @@ struct AggregationMethodString /// Same as above but without cache -template +template struct AggregationMethodStringNoCache { using Data = TData; @@ -271,15 +290,23 @@ struct AggregationMethodStringNoCache { } - using State = ColumnsHashing::HashMethodString; + using State = ColumnsHashing::HashMethodString; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = nullable; std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { - static_cast(key_columns[0])->insertData(key.data, key.size); + if constexpr (nullable) + { + static_cast(key_columns[0])->insertData(key.data, key.size); + } + else + { + static_cast(key_columns[0])->insertData(key.data, key.size); + } } }; @@ -306,6 +333,7 @@ struct AggregationMethodFixedString using State = ColumnsHashing::HashMethodFixedString; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = false; std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } @@ -316,7 +344,7 @@ struct AggregationMethodFixedString }; /// Same as above but without cache -template +template struct AggregationMethodFixedStringNoCache { using Data = TData; @@ -334,15 +362,23 @@ struct AggregationMethodFixedStringNoCache { } - using State = ColumnsHashing::HashMethodFixedString; + using State = ColumnsHashing::HashMethodFixedString; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = nullable; std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } static void insertKeyIntoColumns(StringRef key, std::vector & key_columns, const Sizes &) { - static_cast(key_columns[0])->insertData(key.data, key.size); + if constexpr (nullable) + { + static_cast(key_columns[0])->insertData(key.data, key.size); + } + else + { + static_cast(key_columns[0])->insertData(key.data, key.size); + } } }; @@ -418,6 +454,7 @@ struct AggregationMethodKeysFixed use_cache>; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = false; std::optional shuffleKeyColumns(std::vector & key_columns, const Sizes & key_sizes) { @@ -505,6 +542,7 @@ struct AggregationMethodSerialized using State = ColumnsHashing::HashMethodSerialized; static const bool low_cardinality_optimization = false; + static const bool one_key_nullable_optimization = false; std::optional shuffleKeyColumns(std::vector &, const Sizes &) { return {}; } @@ -588,6 +626,18 @@ struct AggregatedDataVariants : private boost::noncopyable std::unique_ptr> serialized_hash64; /// Support for nullable keys. + std::unique_ptr> nullable_key8; + std::unique_ptr> nullable_key16; + std::unique_ptr> nullable_key32; + std::unique_ptr> nullable_key64; + std::unique_ptr> nullable_key32_two_level; + std::unique_ptr> nullable_key64_two_level; + + std::unique_ptr> nullable_key_string; + std::unique_ptr> nullable_key_fixed_string; + std::unique_ptr> nullable_key_string_two_level; + std::unique_ptr> nullable_key_fixed_string_two_level; + std::unique_ptr> nullable_keys128; std::unique_ptr> nullable_keys256; std::unique_ptr> nullable_keys128_two_level; @@ -640,6 +690,16 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128_hash64, false) \ M(keys256_hash64, false) \ M(serialized_hash64, false) \ + M(nullable_key8, false) \ + M(nullable_key16, false) \ + M(nullable_key32, false) \ + M(nullable_key64, false) \ + M(nullable_key32_two_level, true) \ + M(nullable_key64_two_level, true) \ + M(nullable_key_string, false) \ + M(nullable_key_fixed_string, false) \ + M(nullable_key_string_two_level, true) \ + M(nullable_key_fixed_string_two_level, true) \ M(nullable_keys128, false) \ M(nullable_keys256, false) \ M(nullable_keys128_two_level, true) \ @@ -754,6 +814,10 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128) \ M(keys256) \ M(serialized) \ + M(nullable_key32) \ + M(nullable_key64) \ + M(nullable_key_string) \ + M(nullable_key_fixed_string) \ M(nullable_keys128) \ M(nullable_keys256) \ M(low_cardinality_key32) \ @@ -767,6 +831,8 @@ struct AggregatedDataVariants : private boost::noncopyable #define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \ M(key8) \ M(key16) \ + M(nullable_key8) \ + M(nullable_key16) \ M(keys16) \ M(key64_hash64) \ M(key_string_hash64)\ @@ -810,6 +876,10 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128_two_level) \ M(keys256_two_level) \ M(serialized_two_level) \ + M(nullable_key32_two_level) \ + M(nullable_key64_two_level) \ + M(nullable_key_string_two_level) \ + M(nullable_key_fixed_string_two_level) \ M(nullable_keys128_two_level) \ M(nullable_keys256_two_level) \ M(low_cardinality_key32_two_level) \ @@ -1290,7 +1360,7 @@ private: Arena * arena) const; template - Block insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena) const; + Block insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool has_null_key_data) const; template ConvertToBlockRes diff --git a/src/Interpreters/JIT/compileFunction.cpp b/src/Interpreters/JIT/compileFunction.cpp index 8bf0eb25b60..a7233433861 100644 --- a/src/Interpreters/JIT/compileFunction.cpp +++ b/src/Interpreters/JIT/compileFunction.cpp @@ -41,7 +41,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -ColumnData getColumnData(const IColumn * column) +ColumnData getColumnData(const IColumn * column, size_t skip_rows) { const bool is_const = isColumnConst(*column); @@ -52,11 +52,11 @@ ColumnData getColumnData(const IColumn * column) if (const auto * nullable = typeid_cast(column)) { - result.null_data = nullable->getNullMapColumn().getRawData().data(); + result.null_data = nullable->getNullMapColumn().getDataAt(skip_rows).data; column = &nullable->getNestedColumn(); } - - result.data = column->getRawData().data(); + /// skip null key data for one nullable key optimization + result.data = column->getDataAt(skip_rows).data; return result; } diff --git a/src/Interpreters/JIT/compileFunction.h b/src/Interpreters/JIT/compileFunction.h index 0e0a1106698..fe5abe1988c 100644 --- a/src/Interpreters/JIT/compileFunction.h +++ b/src/Interpreters/JIT/compileFunction.h @@ -26,7 +26,7 @@ struct ColumnData /** Returns ColumnData for column. * If constant column is passed, LOGICAL_ERROR will be thrown. */ -ColumnData getColumnData(const IColumn * column); +ColumnData getColumnData(const IColumn * column, size_t skip_rows = 0); using ColumnDataRowsOffset = size_t; using ColumnDataRowsSize = size_t; diff --git a/tests/performance/groupby_onekey_nullable.xml b/tests/performance/groupby_onekey_nullable.xml new file mode 100644 index 00000000000..2c03cb0861f --- /dev/null +++ b/tests/performance/groupby_onekey_nullable.xml @@ -0,0 +1,49 @@ + + + 8 + 0 + + + + + key_type + + key_string + key_string_two_level + key_fixed_string + key_int64 + key_int64_two_level + key_int32 + key_int32_two_level + Key_int16 + key_int8 + + + + + CREATE TABLE t_nullable + ( + key_string Nullable(String), + key_string_two_level Nullable(String), + key_fixed_string Nullable(FixedString(3)), + key_int64 Nullable(Int64), + key_int64_two_level Nullable(Int64), + key_int32 Nullable(Int32), + key_int32_two_level Nullable(Int32), + Key_int16 Nullable(Int16), + key_int8 Nullable(Int8), + i1 Nullable(Int64), + i2 Nullable(Int64), + ) + ENGINE = MergeTree + ORDER BY tuple() + + insert into t_nullable select ['aa','bb','cc','dd'][number % 10000 + 1], ['aa','bb','cc','dd'][number % 200000 + 1], ['aa','bb','cc','dd'][number % 4 + 1], toInt64(number%10000), toInt64(number%200000), toInt32(number%10000), toInt32(number%200000), toInt16(number%10000), toInt8(number%100), number%6000+1, number%5000+2 from numbers_mt(20000000) + insert into t_nullable select ['aa','bb','cc','dd'][number % 10000 + 1], ['aa','bb','cc','dd'][number % 200000 + 1], ['aa','bb','cc','dd'][number % 4 + 1], toInt64(number%10000), toInt64(number%200000), toInt32(number%10000), toInt32(number%200000), toInt16(number%10000), toInt8(number%100), number%6000+1, number%5000+2 from numbers_mt(20000000) + insert into t_nullable select ['aa','bb','cc','dd'][number % 10000 + 1], ['aa','bb','cc','dd'][number % 200000 + 1], ['aa','bb','cc','dd'][number % 4 + 1], toInt64(number%10000), toInt64(number%200000), toInt32(number%10000), toInt32(number%200000), toInt16(number%10000), toInt8(number%100), number%6000+1, number%5000+2 from numbers_mt(20000000) + insert into t_nullable select ['aa','bb','cc','dd'][number % 10000 + 1], ['aa','bb','cc','dd'][number % 200000 + 1], ['aa','bb','cc','dd'][number % 4 + 1], toInt64(number%10000), toInt64(number%200000), toInt32(number%10000), toInt32(number%200000), toInt16(number%10000), toInt8(number%100), number%6000+1, number%5000+2 from numbers_mt(20000000) + OPTIMIZE TABLE t_nullable FINAL + select min(i1), min(i2) from t_nullable group by {key_type} format Null + + drop table if exists t_nullable + \ No newline at end of file From 93861e04ae941d2b20b9b6f79b73e19bacec74ab Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 20:48:25 +0000 Subject: [PATCH 232/470] Fix fix fix --- tests/ci/compatibility_check.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 688fe883c1e..a06f4baf359 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -60,18 +60,20 @@ def process_glibc_check(log_path: str, max_glibc_version: str) -> TestResults: def process_result( - result_folder: str, server_log_folder: str, max_glibc_version: str + result_folder: str, server_log_folder: str, check_glibc: bool, check_distributions: bool, max_glibc_version: str ) -> Tuple[str, str, TestResults, List[str]]: glibc_log_path = os.path.join(result_folder, "glibc.log") test_results = process_glibc_check(glibc_log_path, max_glibc_version) status = "success" description = "Compatibility check passed" - if len(test_results) > 1 or test_results[0].status != "OK": - status = "failure" - description = "glibc check failed" - if status == "success": + if check_glibc: + if len(test_results) > 1 or test_results[0].status != "OK": + status = "failure" + description = "glibc check failed" + + if status == "success" and check_distributions: for operating_system in ("ubuntu:12.04", "centos:5"): test_result = process_os_check( os.path.join(result_folder, operating_system) @@ -217,7 +219,7 @@ def main(): s3_helper = S3Helper() state, description, test_results, additional_logs = process_result( - result_path, server_log_path, max_glibc_version + result_path, server_log_path, args.check_glibc, args.check_distributions, max_glibc_version ) ch_helper = ClickHouseHelper() From 6eb2877e532aeb083357dcd4a0d7477e4de357b8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 2 Mar 2023 20:54:45 +0000 Subject: [PATCH 233/470] Automatic style fix --- tests/ci/compatibility_check.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index a06f4baf359..8841bedbb18 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -60,7 +60,11 @@ def process_glibc_check(log_path: str, max_glibc_version: str) -> TestResults: def process_result( - result_folder: str, server_log_folder: str, check_glibc: bool, check_distributions: bool, max_glibc_version: str + result_folder: str, + server_log_folder: str, + check_glibc: bool, + check_distributions: bool, + max_glibc_version: str, ) -> Tuple[str, str, TestResults, List[str]]: glibc_log_path = os.path.join(result_folder, "glibc.log") test_results = process_glibc_check(glibc_log_path, max_glibc_version) @@ -219,7 +223,11 @@ def main(): s3_helper = S3Helper() state, description, test_results, additional_logs = process_result( - result_path, server_log_path, args.check_glibc, args.check_distributions, max_glibc_version + result_path, + server_log_path, + args.check_glibc, + args.check_distributions, + max_glibc_version, ) ch_helper = ClickHouseHelper() From fc227d94bcaf9cd4d3583b1cb5c91c177b150cb8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 2 Mar 2023 21:20:26 +0000 Subject: [PATCH 234/470] 02346_full_text_search.sql: Add result separators to simplify analysis --- .../02346_full_text_search.reference | 9 +++++++++ .../0_stateless/02346_full_text_search.sql | 19 ++++++++++--------- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02346_full_text_search.reference b/tests/queries/0_stateless/02346_full_text_search.reference index f1e21e511d0..9cd09110608 100644 --- a/tests/queries/0_stateless/02346_full_text_search.reference +++ b/tests/queries/0_stateless/02346_full_text_search.reference @@ -1,3 +1,4 @@ +Test inverted(2) af inverted 1 101 Alick a01 @@ -10,6 +11,7 @@ af inverted 113 Click b03 118 Click b08 1 +Test inverted() af inverted 101 Alick a01 106 Alick a06 @@ -21,9 +23,11 @@ af inverted 101 Alick a01 111 Alick b01 1 +Test on array columns af inverted 3 ['Click a03','Click b03'] 1 +Test on map columns af inverted 103 {'Click':'Click a03'} 108 {'Click':'Click a08'} @@ -32,20 +36,25 @@ af inverted 1 103 {'Click':'Click a03'} 1 +Test inverted(2) on a column with two parts af inverted 101 Alick a01 111 Alick b01 201 rick c01 1 +Test inverted(2) on UTF-8 data af inverted 102 clickhouse你好 1 +Test max_digestion_size_per_segment af inverted BC614E,05397FB1,6969696969898240,CF3304 1 +Test density==1 af inverted 1 1 +Test density==0.1 af inverted 1 1 diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index 2b10800e78f..ed086861f1f 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -2,7 +2,7 @@ SET allow_experimental_inverted_index = 1; SET log_queries = 1; ---------------------------------------------------- --- Test inverted(2) +SELECT 'Test inverted(2)'; DROP TABLE IF EXISTS tab; @@ -58,7 +58,7 @@ SELECT read_rows==8 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test inverted() +SELECT 'Test inverted()'; DROP TABLE IF EXISTS tab_x; @@ -111,7 +111,7 @@ SELECT read_rows==4 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test on array columns +SELECT 'Test on array columns'; DROP TABLE IF EXISTS tab; @@ -138,7 +138,7 @@ SELECT read_rows==2 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test on map columns +SELECT 'Test on map columns'; DROP TABLE IF EXISTS tab; @@ -178,7 +178,8 @@ SELECT read_rows==8 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test inverted(2) on a column with two parts +SELECT 'Test inverted(2) on a column with two parts'; + DROP TABLE IF EXISTS tab; @@ -206,7 +207,7 @@ SELECT read_rows==6 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test inverted(2) on UTF-8 data +SELECT 'Test inverted(2) on UTF-8 data'; DROP TABLE IF EXISTS tab; @@ -234,7 +235,7 @@ SELECT read_rows==2 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test max_digestion_size_per_segment +SELECT 'Test max_digestion_size_per_segment'; DROP TABLE IF EXISTS tab; @@ -265,7 +266,7 @@ SELECT read_rows==256 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test density==1 +SELECT 'Test density==1'; DROP TABLE IF EXISTS tab; @@ -294,7 +295,7 @@ SELECT read_rows==0 from system.query_log LIMIT 1; ---------------------------------------------------- --- Test density==0.1 +SELECT 'Test density==0.1'; DROP TABLE IF EXISTS tab; From 9c01c6cc252627d109809696bfa3acb584ef3979 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Mar 2023 22:30:52 +0100 Subject: [PATCH 235/470] Add Cloud --- programs/server/play.html | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/programs/server/play.html b/programs/server/play.html index c511d13cf91..e4fe719ddcb 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -398,15 +398,30 @@ fill: var(--logo-color); } + #cloud-logo + { + color: var(--background-color); + text-shadow: 0rem 0rem 2rem var(--logo-color); + font-size: 10vw; + display: block; + } + #logo:hover { fill: var(--logo-color-active); + color: var(--logo-color-active); + } + + #cloud-logo:hover + { + filter: brightness(150%); } #logo-container { text-align: center; margin-top: 5em; + line-height: 0.75; } #chart @@ -487,6 +502,7 @@ +

From 6c68e5c4c61c991802ef0c2ebb0b77573e45813d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 Mar 2023 01:04:10 +0300 Subject: [PATCH 236/470] Revert "Give users option of overwriting" --- docs/_includes/install/universal.sh | 33 +++++++++++------------------ 1 file changed, 12 insertions(+), 21 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index 403aab6f4e6..de34897a6f6 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -60,21 +60,12 @@ fi clickhouse_download_filename_prefix="clickhouse" clickhouse="$clickhouse_download_filename_prefix" -if [ -f "$clickhouse" ] -then - read -p "ClickHouse binary ${clickhouse} already exists. Overwrite? [y/N] " answer - if [ "$answer" = "y" -o "$answer" = "Y" ] - then - rm -f "$clickhouse" - else - i=0 - while [ -f "$clickhouse" ] - do - clickhouse="${clickhouse_download_filename_prefix}.${i}" - i=$(($i+1)) - done - fi -fi +i=0 +while [ -f "$clickhouse" ] +do + clickhouse="${clickhouse_download_filename_prefix}.${i}" + i=$(($i+1)) +done URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" echo @@ -85,9 +76,9 @@ echo echo "Successfully downloaded the ClickHouse binary, you can run it as: ./${clickhouse}" -#if [ "${OS}" = "Linux" ] -#then - #echo - #echo "You can also install it: - #sudo ./${clickhouse} install" -#fi +if [ "${OS}" = "Linux" ] +then + echo + echo "You can also install it: + sudo ./${clickhouse} install" +fi From 1f4141a35d314a7c5cf663cbb6b80229c9dc2220 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Sat, 25 Feb 2023 11:05:18 -0500 Subject: [PATCH 237/470] Give users option of overwriting --- docs/_includes/install/universal.sh | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index de34897a6f6..1699be138c8 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -60,12 +60,21 @@ fi clickhouse_download_filename_prefix="clickhouse" clickhouse="$clickhouse_download_filename_prefix" -i=0 -while [ -f "$clickhouse" ] -do - clickhouse="${clickhouse_download_filename_prefix}.${i}" - i=$(($i+1)) -done +if [ -f "$clickhouse" ] +then + read -p "ClickHouse binary ${clickhouse} already exists. Overwrite? [y/N] " answer + if [ "$answer" = "y" -o "$answer" = "Y" ] + then + rm -f "$clickhouse" + else + i=0 + while [ -f "$clickhouse" ] + do + clickhouse="${clickhouse_download_filename_prefix}.${i}" + i=$(($i+1)) + done + fi +fi URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" echo From 619b28206034120c7eb51b75bc6c919e8feed795 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 3 Mar 2023 01:22:04 +0100 Subject: [PATCH 238/470] more correct handling of fatal errors --- src/Common/CurrentThread.cpp | 4 ++-- src/Common/CurrentThread.h | 4 ++-- src/Common/ThreadStatus.cpp | 12 +++++++----- src/Daemon/BaseDaemon.cpp | 6 +++--- src/Interpreters/ThreadStatusExt.cpp | 7 +++++-- src/Server/GRPCServer.cpp | 3 +-- src/Server/TCPHandler.cpp | 13 +++++-------- 7 files changed, 25 insertions(+), 24 deletions(-) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 526e28c043d..cf4bd97490e 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -67,8 +67,8 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr callback) { - if (unlikely(!current_thread)) - return; + /// It does not make sense to set a callback for sending logs to a client if there's no thread status + chassert(current_thread); current_thread->setFatalErrorCallback(callback); } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index f36b92e319d..ffc00c77504 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -100,8 +100,8 @@ public: /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope : private boost::noncopyable { - explicit QueryScope(ContextMutablePtr query_context); - explicit QueryScope(ContextPtr query_context); + explicit QueryScope(ContextMutablePtr query_context, std::function fatal_error_callback = {}); + explicit QueryScope(ContextPtr query_context, std::function fatal_error_callback = {}); ~QueryScope(); void logPeakMemoryUsage(); diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 46c171b5cb6..18b33cd412a 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -226,17 +226,19 @@ void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQ void ThreadStatus::setFatalErrorCallback(std::function callback) { - fatal_error_callback = std::move(callback); - - if (!thread_group) - return; - + /// It does not make sense to set a callback for sending logs to a client if there's no thread group + chassert(thread_group); std::lock_guard lock(thread_group->mutex); + fatal_error_callback = std::move(callback); thread_group->fatal_error_callback = fatal_error_callback; } void ThreadStatus::onFatalError() { + /// No thread group - no callback + if (!thread_group) + return; + std::lock_guard lock(thread_group->mutex); if (fatal_error_callback) fatal_error_callback(); diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 60179fd5317..fbfa9e68774 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -134,7 +134,7 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) } -static std::atomic fatal_error_printed{false}; +static std::atomic_flag fatal_error_printed; /** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. */ @@ -165,7 +165,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) for (size_t i = 0; i < 300; ++i) { /// We will synchronize with the thread printing the messages with an atomic variable to finish earlier. - if (fatal_error_printed) + if (fatal_error_printed.test()) break; /// This coarse method of synchronization is perfectly ok for fatal signals. @@ -421,7 +421,7 @@ private: if (thread_ptr) thread_ptr->onFatalError(); - fatal_error_printed = true; + fatal_error_printed.test_and_set(); } }; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 84400fc3711..b42f2671a5e 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -550,15 +550,16 @@ void CurrentThread::detachQueryIfNotDetached() } -CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context) +CurrentThread::QueryScope::QueryScope(ContextMutablePtr query_context, std::function fatal_error_callback) { CurrentThread::initializeQuery(); CurrentThread::attachQueryContext(query_context); if (!query_context->hasQueryContext()) query_context->makeQueryContext(); + setFatalErrorCallback(fatal_error_callback); } -CurrentThread::QueryScope::QueryScope(ContextPtr query_context) +CurrentThread::QueryScope::QueryScope(ContextPtr query_context, std::function fatal_error_callback) { if (!query_context->hasQueryContext()) throw Exception( @@ -566,6 +567,7 @@ CurrentThread::QueryScope::QueryScope(ContextPtr query_context) CurrentThread::initializeQuery(); CurrentThread::attachQueryContext(query_context); + setFatalErrorCallback(fatal_error_callback); } void CurrentThread::QueryScope::logPeakMemoryUsage() @@ -585,6 +587,7 @@ CurrentThread::QueryScope::~QueryScope() if (log_peak_memory_usage_in_destructor) logPeakMemoryUsage(); + setFatalErrorCallback({}); CurrentThread::detachQueryIfNotDetached(); } catch (...) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 595f5a8c2b7..533c3d0959b 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -836,7 +836,7 @@ namespace query_context->applySettingsChanges(settings_changes); query_context->setCurrentQueryId(query_info.query_id()); - query_scope.emplace(query_context); + query_scope.emplace(query_context, /* fatal_error_callback */ [this]{ onFatalError(); }); /// Set up tracing context for this query on current thread thread_trace_context = std::make_unique("GRPCServer", @@ -854,7 +854,6 @@ namespace logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); logs_queue->setSourceRegexp(settings.send_logs_source_regexp); CurrentThread::attachInternalTextLogsQueue(logs_queue, client_logs_level); - CurrentThread::setFatalErrorCallback([this]{ onFatalError(); }); } /// Set the current database if specified. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..cac7160b070 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -277,7 +277,11 @@ void TCPHandler::runImpl() query_context->getSettingsRef(), query_context->getOpenTelemetrySpanLog()); - query_scope.emplace(query_context); + query_scope.emplace(query_context, /* fatal_error_callback */ [this] + { + std::lock_guard lock(fatal_error_mutex); + sendLogs(); + }); /// If query received, then settings in query_context has been updated. /// So it's better to update the connection settings for flexibility. @@ -298,11 +302,6 @@ void TCPHandler::runImpl() state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); state.logs_queue->setSourceRegexp(query_context->getSettingsRef().send_logs_source_regexp); CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level); - CurrentThread::setFatalErrorCallback([this] - { - std::lock_guard lock(fatal_error_mutex); - sendLogs(); - }); } if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) { @@ -611,8 +610,6 @@ void TCPHandler::runImpl() /// It is important to destroy query context here. We do not want it to live arbitrarily longer than the query. query_context.reset(); - CurrentThread::setFatalErrorCallback({}); - if (is_interserver_mode) { /// We don't really have session in interserver mode, new one is created for each query. It's better to reset it now. From 29e396cc30e219815e574105bb170505443a09bb Mon Sep 17 00:00:00 2001 From: houbaron Date: Fri, 3 Mar 2023 11:57:03 +0800 Subject: [PATCH 239/470] Update skipping-indexes.md --- docs/zh/guides/improving-query-performance/skipping-indexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/guides/improving-query-performance/skipping-indexes.md b/docs/zh/guides/improving-query-performance/skipping-indexes.md index 2f9ce09d77f..f5889898c2c 100644 --- a/docs/zh/guides/improving-query-performance/skipping-indexes.md +++ b/docs/zh/guides/improving-query-performance/skipping-indexes.md @@ -1,6 +1,6 @@ --- slug: /zh/guides/improving-query-performance/skipping-indexes -sidebar_label: Data Skipping Indexes +sidebar_label: 跳数索引 sidebar_position: 2 --- From 1f7a0c70f0442feb1ea3ac1593a4d699893de6bc Mon Sep 17 00:00:00 2001 From: houbaron Date: Fri, 3 Mar 2023 12:00:42 +0800 Subject: [PATCH 240/470] Update sparse-primary-indexes.md --- .../improving-query-performance/sparse-primary-indexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md index e773a02fbc3..18b23a79f86 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -1,6 +1,6 @@ --- slug: /zh/guides/improving-query-performance/sparse-primary-indexes -sidebar_label: Sparse Primary Indexes +sidebar_label: 主键稀疏索引 sidebar_position: 20 --- From df8d36470830f0f140c01a3d13c203b8eb7eeddf Mon Sep 17 00:00:00 2001 From: houbaron Date: Fri, 3 Mar 2023 14:24:36 +0800 Subject: [PATCH 241/470] Update backup.md --- docs/zh/operations/backup.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/backup.md b/docs/zh/operations/backup.md index e0c5174a9a4..04bb09c60a6 100644 --- a/docs/zh/operations/backup.md +++ b/docs/zh/operations/backup.md @@ -12,8 +12,9 @@ sidebar_label: "\u6570\u636E\u5907\u4EFD" 不同公司有不同的可用资源和业务需求,因此不存在一个通用的解决方案可以应对各种情况下的ClickHouse备份和恢复。 适用于 1GB 数据的方案可能并不适用于几十 PB 数据的情况。 有多种具备各自优缺点的可能方法,将在下面对其进行讨论。最好使用几种方法而不是仅仅使用一种方法来弥补它们的各种缺点。。 -!!! note "注" - 需要注意的是,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时可能无法正常恢复(或者至少需要的时间比业务能够容忍的时间更长)。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上演练。 +:::node +需要注意的是,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时可能无法正常恢复(或者至少需要的时间比业务能够容忍的时间更长)。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上演练。 +::: ## 将源数据复制到其它地方 {#duplicating-source-data-somewhere-else} From fda64b503b8af400b1e1f6b30aaf81bb28bca090 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 10:47:29 +0400 Subject: [PATCH 242/470] Fixed a typo at the page `/docs/ru/sql-reference/statements/alter/view.md` --- docs/ru/sql-reference/statements/alter/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/view.md b/docs/ru/sql-reference/statements/alter/view.md index 2d4823bba3a..e6f6730ff99 100644 --- a/docs/ru/sql-reference/statements/alter/view.md +++ b/docs/ru/sql-reference/statements/alter/view.md @@ -6,7 +6,7 @@ sidebar_label: VIEW # Выражение ALTER TABLE … MODIFY QUERY {#alter-modify-query} -Вы можеие изменить запрос `SELECT`, который был задан при создании [материализованного представления](../create/view.md#materialized), с помощью запроса 'ALTER TABLE … MODIFY QUERY'. Используйте его если при создании материализованного представления не использовалась секция `TO [db.]name`. Настройка `allow_experimental_alter_materialized_view_structure` должна быть включена. +Вы можете изменить запрос `SELECT`, который был задан при создании [материализованного представления](../create/view.md#materialized), с помощью запроса 'ALTER TABLE … MODIFY QUERY'. Используйте его если при создании материализованного представления не использовалась секция `TO [db.]name`. Настройка `allow_experimental_alter_materialized_view_structure` должна быть включена. Если при создании материализованного представления использовалась конструкция `TO [db.]name`, то для изменения отсоедините представление с помощью [DETACH](../detach.md), измените таблицу с помощью [ALTER TABLE](index.md), а затем снова присоедините запрос с помощью [ATTACH](../attach.md). From aa4f72728eb7289be11101312f8a3b607c0cb3e2 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 11:21:26 +0400 Subject: [PATCH 243/470] Markup fixed at the page `/docs/ru/sql-reference/statements/optimize.md` --- docs/ru/sql-reference/statements/optimize.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/statements/optimize.md b/docs/ru/sql-reference/statements/optimize.md index b70bba2d765..26993183232 100644 --- a/docs/ru/sql-reference/statements/optimize.md +++ b/docs/ru/sql-reference/statements/optimize.md @@ -10,6 +10,7 @@ sidebar_label: OPTIMIZE :::danger "Внимание" `OPTIMIZE` не устраняет причину появления ошибки `Too many parts`. +::: **Синтаксис** From 9aef18abadda4f9395e1bcb2c732571a22a9bc0c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 3 Mar 2023 08:35:21 +0000 Subject: [PATCH 244/470] Doc update for mapFromArrays() --- .../functions/tuple-map-functions.md | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 28f2f930d90..34c865e7752 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -68,7 +68,9 @@ Result: ## mapFromArrays -mapFromArrays merges an array of keys and an array of values into a Map. +Merges an [Array](../../sql-reference/data-types/array.md) of keys and an [Array](../../sql-reference/data-types/array.md) of values into a [Map(key, value)](../../sql-reference/data-types/map.md). + +The function is a more convenient alternative to `CAST((key_array, value_array), 'Map(key_type, value_type)')`. For example, instead of writing `CAST((['aa', 'bb'], [4, 5]), 'Map(String, UInt32)')`, you can write `mapFromArrays(['aa', 'bb'], [4, 5])`. **Syntax** @@ -78,7 +80,7 @@ mapFromArrays(keys, values) Alias: `MAP_FROM_ARRAYS(keys, values)` -**Parameters** +**Arguments** - `keys` — Given key array to create a map from. The nested type of array must be: [String](../../sql-reference/data-types/string.md), [Integer](../../sql-reference/data-types/int-uint.md), [LowCardinality](../../sql-reference/data-types/lowcardinality.md), [FixedString](../../sql-reference/data-types/fixedstring.md), [UUID](../../sql-reference/data-types/uuid.md), [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), [Date32](../../sql-reference/data-types/date32.md), [Enum](../../sql-reference/data-types/enum.md) - `values` - Given value array to create a map from. @@ -269,7 +271,7 @@ Determines whether the `map` contains the `key` parameter. mapContains(map, key) ``` -**Parameters** +**Arguments** - `map` — Map. [Map](../../sql-reference/data-types/map.md). - `key` — Key. Type matches the type of keys of `map` parameter. @@ -314,7 +316,7 @@ Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operat mapKeys(map) ``` -**Parameters** +**Arguments** - `map` — Map. [Map](../../sql-reference/data-types/map.md). @@ -357,7 +359,7 @@ Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operat mapValues(map) ``` -**Parameters** +**Arguments** - `map` — Map. [Map](../../sql-reference/data-types/map.md). @@ -396,7 +398,7 @@ Result: mapContainsKeyLike(map, pattern) ``` -**Parameters** +**Arguments** - `map` — Map. [Map](../../sql-reference/data-types/map.md). - `pattern` - String pattern to match. @@ -434,7 +436,7 @@ Result: mapExtractKeyLike(map, pattern) ``` -**Parameters** +**Arguments** - `map` — Map. [Map](../../sql-reference/data-types/map.md). - `pattern` - String pattern to match. @@ -463,8 +465,6 @@ Result: │ {} │ └────────────────────────────┘ ``` - - ## mapApply @@ -474,7 +474,7 @@ Result: mapApply(func, map) ``` -**Parameters** +**Arguments** - `func` - [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). - `map` — [Map](../../sql-reference/data-types/map.md). @@ -514,7 +514,7 @@ Result: mapFilter(func, map) ``` -**Parameters** +**Arguments** - `func` - [Lambda function](../../sql-reference/functions/index.md#higher-order-functions---operator-and-lambdaparams-expr-function). - `map` — [Map](../../sql-reference/data-types/map.md). @@ -556,7 +556,7 @@ Result: mapUpdate(map1, map2) ``` -**Parameters** +**Arguments** - `map1` [Map](../../sql-reference/data-types/map.md). - `map2` [Map](../../sql-reference/data-types/map.md). From e631a4d9b8e0c95a3c4e64fc830ad4d2be5b20a6 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 12:50:13 +0400 Subject: [PATCH 245/470] Typo at the page `/docs/ru/sql-reference/functions/introspection.md` --- docs/ru/sql-reference/functions/introspection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/introspection.md b/docs/ru/sql-reference/functions/introspection.md index 7d04dff6b72..26497ef21d3 100644 --- a/docs/ru/sql-reference/functions/introspection.md +++ b/docs/ru/sql-reference/functions/introspection.md @@ -22,7 +22,7 @@ sidebar_label: "Функции интроспекции" ClickHouse сохраняет отчеты профилировщика в [журнал трассировки](../../operations/system-tables/trace_log.md#system_tables-trace_log) в системной таблице. Убедитесь, что таблица и профилировщик настроены правильно. -## addresssToLine {#addresstoline} +## addressToLine {#addresstoline} Преобразует адрес виртуальной памяти внутри процесса сервера ClickHouse в имя файла и номер строки в исходном коде ClickHouse. From a1d99c602b50877d322b5efa411c641317f9a7c2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 3 Mar 2023 10:52:45 +0100 Subject: [PATCH 246/470] Update docs/zh/operations/backup.md --- docs/zh/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/backup.md b/docs/zh/operations/backup.md index 04bb09c60a6..6d491f9c2f7 100644 --- a/docs/zh/operations/backup.md +++ b/docs/zh/operations/backup.md @@ -12,7 +12,7 @@ sidebar_label: "\u6570\u636E\u5907\u4EFD" 不同公司有不同的可用资源和业务需求,因此不存在一个通用的解决方案可以应对各种情况下的ClickHouse备份和恢复。 适用于 1GB 数据的方案可能并不适用于几十 PB 数据的情况。 有多种具备各自优缺点的可能方法,将在下面对其进行讨论。最好使用几种方法而不是仅仅使用一种方法来弥补它们的各种缺点。。 -:::node +:::note 需要注意的是,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时可能无法正常恢复(或者至少需要的时间比业务能够容忍的时间更长)。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上演练。 ::: From 38d157a850769ff88a040b1f802258f8a9d532c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 2 Mar 2023 17:50:26 +0000 Subject: [PATCH 247/470] Fix race in grace hash join with limit --- src/Interpreters/GraceHashJoin.cpp | 4 +++- src/Interpreters/GraceHashJoin.h | 2 +- .../25340_grace_hash_limit_race.reference | 0 .../0_stateless/25340_grace_hash_limit_race.sql | 15 +++++++++++++++ 4 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/25340_grace_hash_limit_race.reference create mode 100644 tests/queries/0_stateless/25340_grace_hash_limit_race.sql diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 79a825a752f..1b62939ac7b 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -410,6 +410,8 @@ void GraceHashJoin::initialize(const Block & sample_block) void GraceHashJoin::joinBlock(Block & block, std::shared_ptr & not_processed) { + std::shared_lock current_bucket_lock(current_bucket_mutex); + if (block.rows() == 0) { hash_join->joinBlock(block, not_processed); @@ -549,7 +551,7 @@ public: IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() { - std::lock_guard current_bucket_lock(current_bucket_mutex); + std::unique_lock current_bucket_lock(current_bucket_mutex); if (current_bucket == nullptr) return nullptr; diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 4f7694e2f07..3a0b45370ea 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -139,7 +139,7 @@ private: mutable SharedMutex rehash_mutex; FileBucket * current_bucket = nullptr; - mutable std::mutex current_bucket_mutex; + mutable SharedMutex current_bucket_mutex; InMemoryJoinPtr hash_join; Block hash_join_sample_block; diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.reference b/tests/queries/0_stateless/25340_grace_hash_limit_race.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql new file mode 100644 index 00000000000..322c808c94d --- /dev/null +++ b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_grace_hash; + +CREATE TABLE test_grace_hash (id UInt32, value UInt64) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_grace_hash SELECT number, number % 100 = 0 FROM numbers(100000); + +SET join_algorithm = 'grace_hash'; + +SELECT * FROM ( + SELECT f.id FROM test_grace_hash AS f + LEFT JOIN test_grace_hash AS d + ON f.id = d.id + LIMIT 1000 +) FORMAT Null; + From f06910da98cfc4aafe3bc86876975a86ee3cc760 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 3 Mar 2023 00:47:57 +0300 Subject: [PATCH 248/470] Update 25340_grace_hash_limit_race.sql --- tests/queries/0_stateless/25340_grace_hash_limit_race.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql index 322c808c94d..b12dd7bdbba 100644 --- a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql +++ b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql @@ -13,3 +13,4 @@ SELECT * FROM ( LIMIT 1000 ) FORMAT Null; +DROP TABLE test_grace_hash; From db2ffc109ccae0118679896cf7c0e9a1f90aad35 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 3 Mar 2023 10:17:34 +0000 Subject: [PATCH 249/470] Add comment to GrachHashJoin::current_bucket_mutex --- src/Interpreters/GraceHashJoin.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 3a0b45370ea..0d6c4741b95 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -139,6 +139,12 @@ private: mutable SharedMutex rehash_mutex; FileBucket * current_bucket = nullptr; + + /* Function `getDelayedBlocks` should be a critical section. + * Also some `joinBlock` calls may be in progress and we need to wait for them to finish, + * because they may may use `hash_join`, but `getDelayedBlocks` switches it to another bucket. + * So, `joinBlock` acquires shared lock and getDelayedBlocks acquires exclusive lock. + */ mutable SharedMutex current_bucket_mutex; InMemoryJoinPtr hash_join; From b96682ce32ef307c66172e70fb1f7c5093691e0c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 3 Mar 2023 10:35:07 +0000 Subject: [PATCH 250/470] Fix markup in docs --- .../zh/engines/database-engines/replicated.md | 27 +++++++++--------- .../mergetree-family/mergetree.md | 14 ++++++---- .../table-engines/special/distributed.md | 28 ++++++++++--------- docs/zh/getting-started/tutorial.md | 5 ++-- docs/zh/interfaces/formats.md | 12 ++++---- docs/zh/interfaces/http.md | 10 ++++--- docs/zh/interfaces/third-party/index.md | 3 +- docs/zh/operations/access-rights.md | 6 ++-- docs/zh/operations/backup.md | 5 ++-- .../settings.md | 15 ++++++---- .../operations/settings/settings-profiles.md | 5 ++-- docs/zh/operations/settings/settings-users.md | 5 ++-- docs/zh/operations/settings/settings.md | 5 ++-- docs/zh/operations/system-tables/parts.md | 5 ++-- docs/zh/operations/system-tables/query_log.md | 5 ++-- .../aggregate-functions/reference/corr.md | 5 ++-- .../aggregate-functions/reference/covarpop.md | 5 ++-- .../reference/covarsamp.md | 5 ++-- .../reference/quantiletiming.md | 10 ++++--- .../reference/quantiletimingweighted.md | 10 ++++--- .../reference/stddevpop.md | 5 ++-- .../reference/stddevsamp.md | 5 ++-- .../reference/uniqcombined.md | 5 ++-- .../aggregate-functions/reference/varpop.md | 5 ++-- .../aggregate-functions/reference/varsamp.md | 5 ++-- docs/zh/sql-reference/ansi.md | 5 ++-- .../data-types/simpleaggregatefunction.md | 8 +++--- .../functions/string-search-functions.md | 25 ++++++++++------- .../sql-reference/statements/alter/delete.md | 5 ++-- .../sql-reference/statements/alter/index.md | 5 ++-- .../statements/alter/order-by.md | 5 ++-- .../sql-reference/statements/alter/setting.md | 5 ++-- .../sql-reference/statements/alter/update.md | 5 ++-- docs/zh/sql-reference/statements/exchange.md | 3 +- docs/zh/sql-reference/statements/rename.md | 3 +- .../statements/select/group-by.md | 5 ++-- .../sql-reference/statements/select/join.md | 10 ++++--- .../statements/select/limit-by.md | 5 ++-- .../sql-reference/statements/select/sample.md | 5 ++-- .../sql-reference/statements/select/where.md | 7 +++-- docs/zh/sql-reference/statements/system.md | 7 ++--- .../zh/sql-reference/table-functions/mysql.md | 5 ++-- 42 files changed, 186 insertions(+), 137 deletions(-) diff --git a/docs/zh/engines/database-engines/replicated.md b/docs/zh/engines/database-engines/replicated.md index df5872e9b19..843d7a3edd5 100644 --- a/docs/zh/engines/database-engines/replicated.md +++ b/docs/zh/engines/database-engines/replicated.md @@ -19,7 +19,6 @@ CREATE DATABASE testdb ENGINE = Replicated('zoo_path', 'shard_name', 'replica_na - `shard_name` — 分片的名字。数据库副本按`shard_name`分组到分片中。 - `replica_name` — 副本的名字。同一分片的所有副本的副本名称必须不同。 -!!! note "警告" 对于[ReplicatedMergeTree](../table-engines/mergetree-family/replication.md#table_engines-replication)表,如果没有提供参数,则使用默认参数:`/clickhouse/tables/{uuid}/{shard}`和`{replica}`。这些可以在服务器设置[default_replica_path](../../operations/server-configuration-parameters/settings.md#default_replica_path)和[default_replica_name](../../operations/server-configuration-parameters/settings.md#default_replica_name)中更改。宏`{uuid}`被展开到表的uuid, `{shard}`和`{replica}`被展开到服务器配置的值,而不是数据库引擎参数。但是在将来,可以使用Replicated数据库的`shard_name`和`replica_name`。 ## 使用方式 {#specifics-and-recommendations} @@ -52,8 +51,8 @@ CREATE TABLE r.rmt (n UInt64) ENGINE=ReplicatedMergeTree ORDER BY n; ``` ``` text -┌─────hosts────────────┬──status─┬─error─┬─num_hosts_remaining─┬─num_hosts_active─┐ -│ shard1|replica1 │ 0 │ │ 2 │ 0 │ +┌─────hosts────────────┬──status─┬─error─┬─num_hosts_remaining─┬─num_hosts_active─┐ +│ shard1|replica1 │ 0 │ │ 2 │ 0 │ │ shard1|other_replica │ 0 │ │ 1 │ 0 │ │ other_shard|r1 │ 0 │ │ 0 │ 0 │ └──────────────────────┴─────────┴───────┴─────────────────────┴──────────────────┘ @@ -62,13 +61,13 @@ CREATE TABLE r.rmt (n UInt64) ENGINE=ReplicatedMergeTree ORDER BY n; 显示系统表: ``` sql -SELECT cluster, shard_num, replica_num, host_name, host_address, port, is_local +SELECT cluster, shard_num, replica_num, host_name, host_address, port, is_local FROM system.clusters WHERE cluster='r'; ``` ``` text -┌─cluster─┬─shard_num─┬─replica_num─┬─host_name─┬─host_address─┬─port─┬─is_local─┐ -│ r │ 1 │ 1 │ node3 │ 127.0.0.1 │ 9002 │ 0 │ +┌─cluster─┬─shard_num─┬─replica_num─┬─host_name─┬─host_address─┬─port─┬─is_local─┐ +│ r │ 1 │ 1 │ node3 │ 127.0.0.1 │ 9002 │ 0 │ │ r │ 2 │ 1 │ node2 │ 127.0.0.1 │ 9001 │ 0 │ │ r │ 2 │ 2 │ node1 │ 127.0.0.1 │ 9000 │ 1 │ └─────────┴───────────┴─────────────┴───────────┴──────────────┴──────┴──────────┘ @@ -83,9 +82,9 @@ node1 :) SELECT materialize(hostName()) AS host, groupArray(n) FROM r.d GROUP BY ``` ``` text -┌─hosts─┬─groupArray(n)─┐ -│ node1 │ [1,3,5,7,9] │ -│ node2 │ [0,2,4,6,8] │ +┌─hosts─┬─groupArray(n)─┐ +│ node1 │ [1,3,5,7,9] │ +│ node2 │ [0,2,4,6,8] │ └───────┴───────────────┘ ``` @@ -98,8 +97,8 @@ node4 :) CREATE DATABASE r ENGINE=Replicated('some/path/r','other_shard','r2'); 集群配置如下所示: ``` text -┌─cluster─┬─shard_num─┬─replica_num─┬─host_name─┬─host_address─┬─port─┬─is_local─┐ -│ r │ 1 │ 1 │ node3 │ 127.0.0.1 │ 9002 │ 0 │ +┌─cluster─┬─shard_num─┬─replica_num─┬─host_name─┬─host_address─┬─port─┬─is_local─┐ +│ r │ 1 │ 1 │ node3 │ 127.0.0.1 │ 9002 │ 0 │ │ r │ 1 │ 2 │ node4 │ 127.0.0.1 │ 9003 │ 0 │ │ r │ 2 │ 1 │ node2 │ 127.0.0.1 │ 9001 │ 0 │ │ r │ 2 │ 2 │ node1 │ 127.0.0.1 │ 9000 │ 1 │ @@ -113,8 +112,8 @@ node2 :) SELECT materialize(hostName()) AS host, groupArray(n) FROM r.d GROUP BY ``` ```text -┌─hosts─┬─groupArray(n)─┐ -│ node2 │ [1,3,5,7,9] │ -│ node4 │ [0,2,4,6,8] │ +┌─hosts─┬─groupArray(n)─┐ +│ node2 │ [1,3,5,7,9] │ +│ node4 │ [0,2,4,6,8] │ └───────┴───────────────┘ ``` diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index 40aa764e2d3..1fcf64fcd25 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -1,6 +1,6 @@ ---- -slug: /zh/engines/table-engines/mergetree-family/mergetree ---- +--- +slug: /zh/engines/table-engines/mergetree-family/mergetree +--- # MergeTree {#table_engines-mergetree} Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及该系列(`*MergeTree`)中的其他引擎。 @@ -25,8 +25,9 @@ Clickhouse 中最强大的表引擎当属 `MergeTree` (合并树)引擎及 需要的话,您可以给表设置一个采样方法。 -!!! note "注意" - [合并](../special/merge.md#merge) 引擎并不属于 `*MergeTree` 系列。 +:::info +[合并](../special/merge.md#merge) 引擎并不属于 `*MergeTree` 系列。 +::: ## 建表 {#table_engine-mergetree-creating-a-table} @@ -364,7 +365,7 @@ WHERE 子句中的条件可以包含对某列数据进行运算的函数表达 常量参数小于 ngram 大小的函数不能使用 `ngrambf_v1` 进行查询优化。 -!!! note "注意" +:::note 布隆过滤器可能会包含不符合条件的匹配,所以 `ngrambf_v1`, `tokenbf_v1` 和 `bloom_filter` 索引不能用于结果返回为假的函数,例如: - 可以用来优化的场景 @@ -379,6 +380,7 @@ WHERE 子句中的条件可以包含对某列数据进行运算的函数表达 - `NOT s = 1` - `s != 1` - `NOT startsWith(s, 'test')` +::: ## 并发数据访问 {#concurrent-data-access} diff --git a/docs/zh/engines/table-engines/special/distributed.md b/docs/zh/engines/table-engines/special/distributed.md index c832e9e19ba..3c8a7a833d0 100644 --- a/docs/zh/engines/table-engines/special/distributed.md +++ b/docs/zh/engines/table-engines/special/distributed.md @@ -45,7 +45,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2 - [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) 设置 - [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) 查看示例 - + **分布式设置** - `fsync_after_insert` - 对异步插入到分布式的文件数据执行`fsync`。确保操作系统将所有插入的数据刷新到启动节点**磁盘上的一个文件**中。 @@ -66,19 +66,20 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2 - `monitor_max_sleep_time_ms` - 等同于 [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) -!!! note "备注" +::note +**稳定性设置** (`fsync_...`): - **稳定性设置** (`fsync_...`): +- 只影响异步插入(例如:`insert_distributed_sync=false`), 当数据首先存储在启动节点磁盘上,然后再异步发送到shard。 +— 可能会显著降低`insert`的性能 +- 影响将存储在分布式表文件夹中的数据写入 **接受您插入的节点** 。如果你需要保证写入数据到底层的MergeTree表中,请参阅 `system.merge_tree_settings` 中的持久性设置(`...fsync...`) - - 只影响异步插入(例如:`insert_distributed_sync=false`), 当数据首先存储在启动节点磁盘上,然后再异步发送到shard。 - — 可能会显著降低`insert`的性能 - - 影响将存储在分布式表文件夹中的数据写入 **接受您插入的节点** 。如果你需要保证写入数据到底层的MergeTree表中,请参阅 `system.merge_tree_settings` 中的持久性设置(`...fsync...`) +**插入限制设置** (`..._insert`) 请见: - **插入限制设置** (`..._insert`) 请见: +- [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) 设置 +- [prefer_localhost_replica](../../../operations/settings/settings.md#settings-prefer-localhost-replica) 设置 +- `bytes_to_throw_insert` 在 `bytes_to_delay_insert` 之前处理,所以你不应该设置它的值小于 `bytes_to_delay_insert` +::: - - [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) 设置 - - [prefer_localhost_replica](../../../operations/settings/settings.md#settings-prefer-localhost-replica) 设置 - - `bytes_to_throw_insert` 在 `bytes_to_delay_insert` 之前处理,所以你不应该设置它的值小于 `bytes_to_delay_insert` **示例** ``` sql @@ -214,7 +215,7 @@ SELECT 查询会被发送到所有分片,并且无论数据在分片中如何 ## 读取数据 {#distributed-reading-data} - + 当查询一个`Distributed`表时,`SELECT`查询被发送到所有的分片,不管数据是如何分布在分片上的(它们可以完全随机分布)。当您添加一个新分片时,您不必将旧数据传输到它。相反,您可以使用更重的权重向其写入新数据——数据的分布会稍微不均匀,但查询将正确有效地工作。 当启用`max_parallel_replicas`选项时,查询处理将在单个分片中的所有副本之间并行化。更多信息,请参见[max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas)。 @@ -225,8 +226,9 @@ SELECT 查询会被发送到所有分片,并且无论数据在分片中如何 - `_shard_num` — 表`system.clusters` 中的 `shard_num` 值 . 数据类型: [UInt32](../../../sql-reference/data-types/int-uint.md). -!!! note "备注" - 因为 [remote](../../../sql-reference/table-functions/remote.md) 和 [cluster](../../../sql-reference/table-functions/cluster.mdx) 表方法内部创建了分布式表, `_shard_num` 对他们都有效. +:::note +因为 [remote](../../../sql-reference/table-functions/remote.md) 和 [cluster](../../../sql-reference/table-functions/cluster.mdx) 表方法内部创建了分布式表, `_shard_num` 对他们都有效. +::: **详见** - [虚拟列](../../../engines/table-engines/index.md#table_engines-virtual_columns) 描述 diff --git a/docs/zh/getting-started/tutorial.md b/docs/zh/getting-started/tutorial.md index bef3ecee35f..989cf5f57d8 100644 --- a/docs/zh/getting-started/tutorial.md +++ b/docs/zh/getting-started/tutorial.md @@ -617,8 +617,9 @@ INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1; 启用本机复制[Zookeeper](http://zookeeper.apache.org/)是必需的。 ClickHouse负责所有副本的数据一致性,并在失败后自动运行恢复过程。建议将ZooKeeper集群部署在单独的服务器上(其中没有其他进程,包括运行的ClickHouse)。 -!!! note "注意" - ZooKeeper不是一个严格的要求:在某些简单的情况下,您可以通过将数据写入应用程序代码中的所有副本来复制数据。 这种方法是**不**建议的,在这种情况下,ClickHouse将无法保证所有副本上的数据一致性。 因此需要由您的应用来保证这一点。 +:::note +ZooKeeper不是一个严格的要求:在某些简单的情况下,您可以通过将数据写入应用程序代码中的所有副本来复制数据。 这种方法是**不**建议的,在这种情况下,ClickHouse将无法保证所有副本上的数据一致性。 因此需要由您的应用来保证这一点。 +::: ZooKeeper位置在配置文件中指定: diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index 852b327366b..fd3cf743818 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -685,8 +685,9 @@ CREATE TABLE IF NOT EXISTS example_table - 如果`input_format_defaults_for_omitted_fields = 0`, 那么`x`和`a`的默认值等于`0`(作为`UInt32`数据类型的默认值)。 - 如果`input_format_defaults_for_omitted_fields = 1`, 那么`x`的默认值为`0`,但`a`的默认值为`x * 2`。 -!!! note "注意" +:::warning 当使用`input_format_defaults_for_omitted_fields = 1`插入数据时,与使用`input_format_defaults_for_omitted_fields = 0`相比,ClickHouse消耗更多的计算资源。 +::: ### Selecting Data {#selecting-data} @@ -708,8 +709,9 @@ CREATE TABLE IF NOT EXISTS example_table 与[JSON](#json)格式不同,没有替换无效的UTF-8序列。值以与`JSON`相同的方式转义。 -!!! note "提示" +:::info 字符串中可以输出任意一组字节。如果您确信表中的数据可以被格式化为JSON而不会丢失任何信息,那么就使用`JSONEachRow`格式。 +::: ### Nested Structures {#jsoneachrow-nested} @@ -1216,9 +1218,9 @@ SET format_avro_schema_registry_url = 'http://schema-registry'; SELECT * FROM topic1_stream; ``` -!!! note "警告" - 设置 `format_avro_schema_registry_url` 需要写入配置文件`users.xml`以在Clickhouse重启后,该设置仍为您的设定值。您也可以在使用Kafka引擎的时候指定该设置。 - +:::warning +设置 `format_avro_schema_registry_url` 需要写入配置文件`users.xml`以在Clickhouse重启后,该设置仍为您的设定值。您也可以在使用Kafka引擎的时候指定该设置。 +::: ## Parquet {#data-format-parquet} diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index 11754ed3e2f..e0c12193a6a 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -188,8 +188,9 @@ $ curl -vsS "http://localhost:8123/?enable_http_compression=1" -d 'SELECT number $ echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' ``` -!!! note "警告" - 一些HTTP客户端可能会在默认情况下从服务器解压数据(使用`gzip`和`deflate`),即使您未正确地使用了压缩设置,您也可能会得到解压数据。 +:::warning +一些HTTP客户端可能会在默认情况下从服务器解压数据(使用`gzip`和`deflate`),即使您未正确地使用了压缩设置,您也可能会得到解压数据。 +::: 您可以使用`database`URL参数或`X-ClickHouse-Database`头来指定默认数据库。 @@ -447,8 +448,9 @@ $ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost: max_final_threads 2 ``` -!!! note "警告" - 在一个`predefined_query_handler`中,只支持insert类型的一个`查询`。 +:::warning +在一个`predefined_query_handler`中,只支持insert类型的一个`查询`。 +::: ### 动态查询 {#dynamic_query_handler} diff --git a/docs/zh/interfaces/third-party/index.md b/docs/zh/interfaces/third-party/index.md index 0c6b194d04a..64c0b30a9c0 100644 --- a/docs/zh/interfaces/third-party/index.md +++ b/docs/zh/interfaces/third-party/index.md @@ -13,5 +13,6 @@ sidebar_position: 24 - [GUI](../../interfaces/third-party/gui.md) - [Proxies](../../interfaces/third-party/proxy.md) -!!! note "注意" +:::note 支持通用API的通用工具[ODBC](../../interfaces/odbc.md)或[JDBC](../../interfaces/jdbc.md),通常也适用于ClickHouse,但这里没有列出,因为它们实在太多了。 +::: diff --git a/docs/zh/operations/access-rights.md b/docs/zh/operations/access-rights.md index 713cf991f47..8d640ef32c0 100644 --- a/docs/zh/operations/access-rights.md +++ b/docs/zh/operations/access-rights.md @@ -24,9 +24,9 @@ ClickHouse权限实体包括: 我们建议你使用SQL工作流的方式。当然配置的方式也可以同时起作用, 所以如果你正在用服务端配置的方式来管理权限和账户,你可以平滑的切换到SQL驱动的工作流方式。 -!!! note "警告" - 你无法同时使用两个配置的方式来管理同一个权限实体。 - +:::warning +你无法同时使用两个配置的方式来管理同一个权限实体。 +::: ## 用法 {#access-control-usage} diff --git a/docs/zh/operations/backup.md b/docs/zh/operations/backup.md index e0c5174a9a4..6d491f9c2f7 100644 --- a/docs/zh/operations/backup.md +++ b/docs/zh/operations/backup.md @@ -12,8 +12,9 @@ sidebar_label: "\u6570\u636E\u5907\u4EFD" 不同公司有不同的可用资源和业务需求,因此不存在一个通用的解决方案可以应对各种情况下的ClickHouse备份和恢复。 适用于 1GB 数据的方案可能并不适用于几十 PB 数据的情况。 有多种具备各自优缺点的可能方法,将在下面对其进行讨论。最好使用几种方法而不是仅仅使用一种方法来弥补它们的各种缺点。。 -!!! note "注" - 需要注意的是,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时可能无法正常恢复(或者至少需要的时间比业务能够容忍的时间更长)。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上演练。 +:::note +需要注意的是,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时可能无法正常恢复(或者至少需要的时间比业务能够容忍的时间更长)。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上演练。 +::: ## 将源数据复制到其它地方 {#duplicating-source-data-somewhere-else} diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 89ae411b9e0..2fd04b1260a 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -528,8 +528,9 @@ SSL客户端/服务器配置。 包含数据的目录的路径。 -!!! note "注" - 尾部斜杠是强制性的。 +:::note +尾部斜杠是强制性的。 +::: **示例** @@ -714,8 +715,9 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv 用于处理大型查询的临时数据的路径。 -!!! note "注" - 尾部斜杠是强制性的。 +:::note +尾部斜杠是强制性的。 +::: **示例** @@ -728,11 +730,12 @@ TCP端口,用于与客户端进行安全通信。 使用它与 [OpenSSL](#serv 从政策 [`storage_configuration`](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) 存储临时文件。 如果没有设置 [`tmp_path`](#server-settings-tmp_path) 被使用,否则被忽略。 -!!! note "注" - - `move_factor` 被忽略 +:::note +- `move_factor` 被忽略 - `keep_free_space_bytes` 被忽略 - `max_data_part_size_bytes` 被忽略 -您必须在该政策中只有一个卷 +::: ## uncompressed_cache_size {#server-settings-uncompressed_cache_size} diff --git a/docs/zh/operations/settings/settings-profiles.md b/docs/zh/operations/settings/settings-profiles.md index 1ad394950bf..5051276607f 100644 --- a/docs/zh/operations/settings/settings-profiles.md +++ b/docs/zh/operations/settings/settings-profiles.md @@ -8,8 +8,9 @@ sidebar_label: "\u8BBE\u7F6E\u914D\u7F6E" 设置配置是设置的集合,并按照相同的名称进行分组。 -!!! note "信息" - ClickHouse 还支持用 [SQL驱动的工作流](../../operations/access-rights.md#access-control) 管理设置配置。我们建议使用它。 +:::info +ClickHouse 还支持用 [SQL驱动的工作流](../../operations/access-rights.md#access-control) 管理设置配置。我们建议使用它。 +::: 设置配置可以任意命名。你可以为不同的用户指定相同的设置配置。您可以在设置配置中写入的最重要的内容是 `readonly=1`,这将确保只读访问。 diff --git a/docs/zh/operations/settings/settings-users.md b/docs/zh/operations/settings/settings-users.md index de4aa27df69..3fb97bbddb2 100644 --- a/docs/zh/operations/settings/settings-users.md +++ b/docs/zh/operations/settings/settings-users.md @@ -10,8 +10,9 @@ sidebar_label: "\u7528\u6237\u8BBE\u7F6E" `user.xml` 中的 `users` 配置段包含了用户配置 -!!! note "提示" - ClickHouse还支持 [SQL驱动的工作流](../access-rights.md#access-control) 用于管理用户。 我们建议使用它。 +:::note +ClickHouse还支持 [SQL驱动的工作流](../access-rights.md#access-control) 用于管理用户。 我们建议使用它。 +::: `users` 配置段的结构: diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 4107a499463..457b208602f 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -266,8 +266,9 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( 执行时 `INSERT` 查询时,将省略的输入列值替换为相应列的默认值。 此选项仅适用于 [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) 和 [TabSeparated](../../interfaces/formats.md#tabseparated) 格式。 -!!! note "注" - 启用此选项后,扩展表元数据将从服务器发送到客户端。 它会消耗服务器上的额外计算资源,并可能降低性能。 +:::note +启用此选项后,扩展表元数据将从服务器发送到客户端。 它会消耗服务器上的额外计算资源,并可能降低性能。 +::: 可能的值: diff --git a/docs/zh/operations/system-tables/parts.md b/docs/zh/operations/system-tables/parts.md index 0bd728f543f..0ebac3944ff 100644 --- a/docs/zh/operations/system-tables/parts.md +++ b/docs/zh/operations/system-tables/parts.md @@ -99,8 +99,9 @@ slug: /zh/operations/system-tables/parts - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 表达式的数组。 每个表达式定义一个 [TTL MOVE 规则](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). - !!! note "警告" - 保留 `move_ttl_info.expression` 数组主要是为了向后兼容,现在检查 `TTL MOVE` 规则最简单的方法是使用 `move_ttl_info.min` 和 `move_ttl_info.max` 字段。 +:::warning +保留 `move_ttl_info.expression` 数组主要是为了向后兼容,现在检查 `TTL MOVE` 规则最简单的方法是使用 `move_ttl_info.min` 和 `move_ttl_info.max` 字段。 +::: - `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — 日期值和时间值的数组。数组中的每个元素都描述了一个 [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) 的最小键值。 diff --git a/docs/zh/operations/system-tables/query_log.md b/docs/zh/operations/system-tables/query_log.md index 93e5771d4b5..7149282dfcc 100644 --- a/docs/zh/operations/system-tables/query_log.md +++ b/docs/zh/operations/system-tables/query_log.md @@ -8,8 +8,9 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 包含已执行查询的相关信息,例如:开始时间、处理持续时间、错误消息。 -!!! note "注" - 此表不包含以下内容的摄取数据 `INSERT` 查询。 +:::note +此表不包含以下内容的摄取数据 `INSERT` 查询。 +::: 您可以更改query_log的设置,在服务器配置的 [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) 部分。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/corr.md b/docs/zh/sql-reference/aggregate-functions/reference/corr.md index 01a89e428ab..48b5bf904f5 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/corr.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/corr.md @@ -12,5 +12,6 @@ sidebar_position: 107 计算Pearson相关系数: `Σ((x - x̅)(y - y̅)) / sqrt(Σ((x - x̅)^2) * Σ((y - y̅)^2))`。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `corrStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `corrStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md b/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md index 93bfee15684..e98270dc896 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/covarpop.md @@ -12,5 +12,6 @@ covarPop(x, y) 计算 `Σ((x - x̅)(y - y̅)) / n` 的值。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarPopStable` 函数。 它的工作速度较慢,但提供了较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarPopStable` 函数。 它的工作速度较慢,但提供了较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md index 7c8565211b1..1b8dfc4a60f 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/covarsamp.md @@ -14,5 +14,6 @@ covarSamp(x, y) 返回Float64。 当 `n <= 1`, 返回 +∞。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `covarSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md b/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md index 5e14ce6a11c..af7ad77c717 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/quantiletiming.md @@ -37,8 +37,9 @@ quantileTiming(level)(expr) 否则,计算结果将四舍五入到16毫秒的最接近倍数。 -!!! note "注" - 对于计算页面加载时间分位数, 此函数比[quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile)更有效和准确。 +:::note +对于计算页面加载时间分位数, 此函数比[quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile)更有效和准确。 +::: **返回值** @@ -46,8 +47,9 @@ quantileTiming(level)(expr) 类型: `Float32`。 -!!! note "注" - 如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 +:::note +如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 +::: **示例** diff --git a/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md b/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md index 2c28583343a..b520a0f96af 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/quantiletimingweighted.md @@ -39,8 +39,9 @@ quantileTimingWeighted(level)(expr, weight) 否则,计算结果将四舍五入到16毫秒的最接近倍数。 -!!! note "注" - 对于计算页面加载时间分位数, 此函数比[quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile)更有效和准确。 +:::note +对于计算页面加载时间分位数, 此函数比[quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile)更有效和准确。 +::: **返回值** @@ -48,8 +49,9 @@ quantileTimingWeighted(level)(expr, weight) 类型: `Float32`。 -!!! note "注" - 如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 +:::note +如果没有值传递给函数(当使用 `quantileTimingIf`), [NaN](../../../sql-reference/data-types/float.md#data_type-float-nan-inf)被返回。 这样做的目的是将这些案例与导致零的案例区分开来。 参见 [ORDER BY clause](../../../sql-reference/statements/select/order-by.md#select-order-by) 对于 `NaN` 值排序注意事项。 +::: **示例** diff --git a/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md b/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md index ea82e21e46f..a113084cdee 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/stddevpop.md @@ -7,5 +7,6 @@ sidebar_position: 30 结果等于 [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md)的平方根。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md index efeafb71072..d242f4e3401 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/stddevsamp.md @@ -7,5 +7,6 @@ sidebar_position: 31 结果等于 [varSamp] (../../../sql-reference/aggregate-functions/reference/varsamp.md)的平方根。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `stddevSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: \ No newline at end of file diff --git a/docs/zh/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/zh/sql-reference/aggregate-functions/reference/uniqcombined.md index edc790ec00f..7b4a78d662e 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -36,8 +36,9 @@ uniqCombined(HLL_precision)(x[, ...]) - 确定性地提供结果(它不依赖于查询处理顺序)。 -!!! note "注" - 由于它对非 `String` 类型使用32位哈希,对于基数显著大于`UINT_MAX` ,结果将有非常高的误差(误差将在几百亿不同值之后迅速提高), 因此这种情况,你应该使用 [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) +:::note +由于它对非 `String` 类型使用32位哈希,对于基数显著大于`UINT_MAX` ,结果将有非常高的误差(误差将在几百亿不同值之后迅速提高), 因此这种情况,你应该使用 [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) +::: 相比于 [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) 函数, 该 `uniqCombined`: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/varpop.md b/docs/zh/sql-reference/aggregate-functions/reference/varpop.md index eb17955210b..6d6b0acc615 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/varpop.md @@ -9,5 +9,6 @@ sidebar_position: 32 换句话说,计算一组数据的离差。 返回 `Float64`。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varPopStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md b/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md index 9b9d0ced92d..508e35445a7 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/varsamp.md @@ -11,5 +11,6 @@ sidebar_position: 33 返回 `Float64`。 当 `n <= 1`,返回 `+∞`。 -!!! note "注" - 该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +:::note +该函数使用数值不稳定的算法。 如果你需要 [数值稳定性](https://en.wikipedia.org/wiki/Numerical_stability) 在计算中,使用 `varSampStable` 函数。 它的工作速度较慢,但提供较低的计算错误。 +::: diff --git a/docs/zh/sql-reference/ansi.md b/docs/zh/sql-reference/ansi.md index cdccee0084f..74f13256eba 100644 --- a/docs/zh/sql-reference/ansi.md +++ b/docs/zh/sql-reference/ansi.md @@ -6,8 +6,9 @@ sidebar_label: "ANSI\u517C\u5BB9\u6027" # ClickHouse SQL方言 与ANSI SQL的兼容性{#ansi-sql-compatibility-of-clickhouse-sql-dialect} -!!! note "注" - 本文参考Annex G所著的[ISO/IEC CD 9075-2:2011](https://www.iso.org/obp/ui/#iso:std:iso-iec:9075:-2:ed-4:v1:en:sec:8)标准. +:::note +本文参考Annex G所著的[ISO/IEC CD 9075-2:2011](https://www.iso.org/obp/ui/#iso:std:iso-iec:9075:-2:ed-4:v1:en:sec:8)标准. +::: ## 行为差异 {#differences-in-behaviour} diff --git a/docs/zh/sql-reference/data-types/simpleaggregatefunction.md b/docs/zh/sql-reference/data-types/simpleaggregatefunction.md index b26994a775e..601cb602a78 100644 --- a/docs/zh/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/zh/sql-reference/data-types/simpleaggregatefunction.md @@ -25,10 +25,10 @@ slug: /zh/sql-reference/data-types/simpleaggregatefunction - [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md) -!!! note "注" - `SimpleAggregateFunction(func, Type)` 的值外观和存储方式于 `Type` 相同, 所以你不需要应用带有 `-Merge`/`-State` 后缀的函数。 - - `SimpleAggregateFunction` 的性能优于具有相同聚合函数的 `AggregateFunction` 。 +:::note +`SimpleAggregateFunction(func, Type)` 的值外观和存储方式于 `Type` 相同, 所以你不需要应用带有 `-Merge`/`-State` 后缀的函数。 +`SimpleAggregateFunction` 的性能优于具有相同聚合函数的 `AggregateFunction` 。 +::: **参数** diff --git a/docs/zh/sql-reference/functions/string-search-functions.md b/docs/zh/sql-reference/functions/string-search-functions.md index 756ac7c16c7..e4167127424 100644 --- a/docs/zh/sql-reference/functions/string-search-functions.md +++ b/docs/zh/sql-reference/functions/string-search-functions.md @@ -42,8 +42,9 @@ slug: /zh/sql-reference/functions/string-search-functions 对于不区分大小写的搜索或/和UTF-8格式,使用函数`multiSearchAnyCaseInsensitive,multiSearchAnyUTF8,multiSearchAnyCaseInsensitiveUTF8`。 -!!! note "注意" - 在所有`multiSearch*`函数中,由于实现规范,needles的数量应小于28。 +:::note +在所有`multiSearch*`函数中,由于实现规范,needles的数量应小于28。 +::: ## 匹配(大海捞针,模式) {#matchhaystack-pattern} @@ -60,8 +61,9 @@ slug: /zh/sql-reference/functions/string-search-functions 与`match`相同,但如果所有正则表达式都不匹配,则返回0;如果任何模式匹配,则返回1。它使用[超扫描](https://github.com/intel/hyperscan)库。对于在字符串中搜索子字符串的模式,最好使用«multisearchany»,因为它更高效。 -!!! note "注意" - 任何`haystack`字符串的长度必须小于232\字节,否则抛出异常。这种限制是因为hyperscan API而产生的。 +:::note +任何`haystack`字符串的长度必须小于232\字节,否则抛出异常。这种限制是因为hyperscan API而产生的。 +::: ## multiMatchAnyIndex(大海捞针,\[模式1,模式2, …, patternn\]) {#multimatchanyindexhaystack-pattern1-pattern2-patternn} @@ -75,11 +77,13 @@ slug: /zh/sql-reference/functions/string-search-functions 与`multiFuzzyMatchAny`相同,但返回匹配项的匹配能容的索引位置。 -!!! note "注意" - `multiFuzzyMatch*`函数不支持UTF-8正则表达式,由于hyperscan限制,这些表达式被按字节解析。 +:::note +`multiFuzzyMatch*`函数不支持UTF-8正则表达式,由于hyperscan限制,这些表达式被按字节解析。 +::: -!!! note "注意" - 如要关闭所有hyperscan函数的使用,请设置`SET allow_hyperscan = 0;`。 +:::note +如要关闭所有hyperscan函数的使用,请设置`SET allow_hyperscan = 0;`。 +::: ## 提取(大海捞针,图案) {#extracthaystack-pattern} @@ -119,5 +123,6 @@ slug: /zh/sql-reference/functions/string-search-functions 对于不区分大小写的搜索或/和UTF-8格式,使用函数`ngramSearchCaseInsensitive,ngramSearchUTF8,ngramSearchCaseInsensitiveUTF8`。 -!!! note "注意" - 对于UTF-8,我们使用3-gram。所有这些都不是完全公平的n-gram距离。我们使用2字节哈希来散列n-gram,然后计算这些哈希表之间的(非)对称差异 - 可能会发生冲突。对于UTF-8不区分大小写的格式,我们不使用公平的`tolower`函数 - 我们将每个Unicode字符字节的第5位(从零开始)和字节的第一位归零 - 这适用于拉丁语,主要用于所有西里尔字母。 +:::note +对于UTF-8,我们使用3-gram。所有这些都不是完全公平的n-gram距离。我们使用2字节哈希来散列n-gram,然后计算这些哈希表之间的(非)对称差异 - 可能会发生冲突。对于UTF-8不区分大小写的格式,我们不使用公平的`tolower`函数 - 我们将每个Unicode字符字节的第5位(从零开始)和字节的第一位归零 - 这适用于拉丁语,主要用于所有西里尔字母。 +::: diff --git a/docs/zh/sql-reference/statements/alter/delete.md b/docs/zh/sql-reference/statements/alter/delete.md index 85d3d3077a7..5eb77c35a93 100644 --- a/docs/zh/sql-reference/statements/alter/delete.md +++ b/docs/zh/sql-reference/statements/alter/delete.md @@ -12,8 +12,9 @@ ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr 删除匹配指定过滤表达式的数据。实现为[突变](../../../sql-reference/statements/alter/index.md#mutations). -!!! note "备注" - `ALTER TABLE`前缀使得这个语法不同于大多数其他支持SQL的系统。它的目的是表示,与OLTP数据库中的类似查询不同,这是一个不为经常使用而设计的繁重操作。 +:::note +`ALTER TABLE`前缀使得这个语法不同于大多数其他支持SQL的系统。它的目的是表示,与OLTP数据库中的类似查询不同,这是一个不为经常使用而设计的繁重操作。 +::: `filter_expr` 的类型必须是`UInt8`。该查询删除表中该表达式接受非零值的行。 diff --git a/docs/zh/sql-reference/statements/alter/index.md b/docs/zh/sql-reference/statements/alter/index.md index b0f0fc21cbe..8320b207725 100644 --- a/docs/zh/sql-reference/statements/alter/index.md +++ b/docs/zh/sql-reference/statements/alter/index.md @@ -17,8 +17,9 @@ sidebar_label: ALTER - [CONSTRAINT](../../../sql-reference/statements/alter/constraint.md) - [TTL](../../../sql-reference/statements/alter/ttl.md) -!!! note "备注" - 大多数 `ALTER TABLE` 查询只支持[\*MergeTree](../../../engines/table-engines/mergetree-family/index.md)表,以及[Merge](../../../engines/table-engines/special/merge.md)和[Distributed](../../../engines/table-engines/special/distributed.md)。 +:::note +大多数 `ALTER TABLE` 查询只支持[\*MergeTree](../../../engines/table-engines/mergetree-family/index.md)表,以及[Merge](../../../engines/table-engines/special/merge.md)和[Distributed](../../../engines/table-engines/special/distributed.md)。 +::: 这些 `ALTER` 语句操作视图: diff --git a/docs/zh/sql-reference/statements/alter/order-by.md b/docs/zh/sql-reference/statements/alter/order-by.md index e70a8b59c85..e50c4e6e805 100644 --- a/docs/zh/sql-reference/statements/alter/order-by.md +++ b/docs/zh/sql-reference/statements/alter/order-by.md @@ -14,5 +14,6 @@ ALTER TABLE [db].name [ON CLUSTER cluster] MODIFY ORDER BY new_expression 从某种意义上说,该命令是轻量级的,它只更改元数据。要保持数据部分行按排序键表达式排序的属性,您不能向排序键添加包含现有列的表达式(仅在相同的`ALTER`查询中由`ADD COLUMN`命令添加的列,没有默认的列值)。 -!!! note "备注" - 它只适用于[`MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md)表族(包括[replicated](../../../engines/table-engines/mergetree-family/replication.md)表)。 +:::note +它只适用于[`MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md)表族(包括[replicated](../../../engines/table-engines/mergetree-family/replication.md)表)。 +::: diff --git a/docs/zh/sql-reference/statements/alter/setting.md b/docs/zh/sql-reference/statements/alter/setting.md index e2d597554e7..2e1e97db331 100644 --- a/docs/zh/sql-reference/statements/alter/setting.md +++ b/docs/zh/sql-reference/statements/alter/setting.md @@ -14,8 +14,9 @@ sidebar_label: SETTING ALTER TABLE [db].name [ON CLUSTER cluster] MODIFY|RESET SETTING ... ``` -!!! note "注意" - 这些查询只能应用于 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) 表。 +:::note +这些查询只能应用于 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) 表。 +::: ## 修改设置 {#alter_modify_setting} diff --git a/docs/zh/sql-reference/statements/alter/update.md b/docs/zh/sql-reference/statements/alter/update.md index 522eb0a705b..97b2b43d889 100644 --- a/docs/zh/sql-reference/statements/alter/update.md +++ b/docs/zh/sql-reference/statements/alter/update.md @@ -12,8 +12,9 @@ ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr 操作与指定过滤表达式相匹配的数据。作为一个[变更 mutation](../../../sql-reference/statements/alter/index.md#mutations)来实现. -!!! note "Note" - `ALTER TABLE` 的前缀使这个语法与其他大多数支持SQL的系统不同。它的目的是表明,与OLTP数据库中的类似查询不同,这是一个繁重的操作,不是为频繁使用而设计。 +:::note +`ALTER TABLE` 的前缀使这个语法与其他大多数支持SQL的系统不同。它的目的是表明,与OLTP数据库中的类似查询不同,这是一个繁重的操作,不是为频繁使用而设计。 +::: `filter_expr`必须是`UInt8`类型。这个查询将指定列的值更新为行中相应表达式的值,对于这些行,`filter_expr`取值为非零。使用`CAST`操作符将数值映射到列的类型上。不支持更新用于计算主键或分区键的列。 diff --git a/docs/zh/sql-reference/statements/exchange.md b/docs/zh/sql-reference/statements/exchange.md index e6ac1dbf1dc..47cefa0d2e6 100644 --- a/docs/zh/sql-reference/statements/exchange.md +++ b/docs/zh/sql-reference/statements/exchange.md @@ -9,8 +9,9 @@ sidebar_label: EXCHANGE 以原子方式交换两个表或字典的名称。 此任务也可以通过使用[RENAME](./rename.md)来完成,但在这种情况下操作不是原子的。 -!!! note "注意" +:::note `EXCHANGE`仅支持[Atomic](../../engines/database-engines/atomic.md)数据库引擎. +::: **语法** diff --git a/docs/zh/sql-reference/statements/rename.md b/docs/zh/sql-reference/statements/rename.md index c26dce306cc..156306fbd3e 100644 --- a/docs/zh/sql-reference/statements/rename.md +++ b/docs/zh/sql-reference/statements/rename.md @@ -9,8 +9,9 @@ sidebar_label: RENAME 重命名数据库、表或字典。 可以在单个查询中重命名多个实体。 请注意,具有多个实体的`RENAME`查询是非原子操作。 要以原子方式交换实体名称,请使用[EXCHANGE](./exchange.md)语法. -!!! note "注意" +:::note `RENAME`仅支持[Atomic](../../engines/database-engines/atomic.md)数据库引擎. +::: **语法** diff --git a/docs/zh/sql-reference/statements/select/group-by.md b/docs/zh/sql-reference/statements/select/group-by.md index 31c1649bc30..29c72ce7e45 100644 --- a/docs/zh/sql-reference/statements/select/group-by.md +++ b/docs/zh/sql-reference/statements/select/group-by.md @@ -11,8 +11,9 @@ sidebar_label: GROUP BY - 在所有的表达式在 [SELECT](../../../sql-reference/statements/select/index.md), [HAVING](../../../sql-reference/statements/select/having),和 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句中 **必须** 基于键表达式进行计算 **或** 上 [聚合函数](../../../sql-reference/aggregate-functions/index.md) 在非键表达式(包括纯列)上。 换句话说,从表中选择的每个列必须用于键表达式或聚合函数内,但不能同时使用。 - 聚合结果 `SELECT` 查询将包含尽可能多的行,因为有唯一值 “grouping key” 在源表中。 通常这会显着减少行数,通常是数量级,但不一定:如果所有行数保持不变 “grouping key” 值是不同的。 -!!! note "注" - 还有一种额外的方法可以在表上运行聚合。 如果查询仅在聚合函数中包含表列,则 `GROUP BY` 可以省略,并且通过一个空的键集合来假定聚合。 这样的查询总是只返回一行。 +:::note +还有一种额外的方法可以在表上运行聚合。 如果查询仅在聚合函数中包含表列,则 `GROUP BY` 可以省略,并且通过一个空的键集合来假定聚合。 这样的查询总是只返回一行。 +::: ## 空处理 {#null-processing} diff --git a/docs/zh/sql-reference/statements/select/join.md b/docs/zh/sql-reference/statements/select/join.md index 08290a02de5..a2686aa5e53 100644 --- a/docs/zh/sql-reference/statements/select/join.md +++ b/docs/zh/sql-reference/statements/select/join.md @@ -39,8 +39,9 @@ ClickHouse中提供的其他联接类型: ## 严格 {#join-settings} -!!! note "注" - 可以使用以下方式复盖默认的严格性值 [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) 设置。 +:::note +可以使用以下方式复盖默认的严格性值 [join_default_strictness](../../../operations/settings/settings.md#settings-join_default_strictness) 设置。 +::: Also the behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys) setting. @@ -91,8 +92,9 @@ USING (equi_column1, ... equi_columnN, asof_column) `ASOF JOIN`会从 `table_2` 中的用户事件时间戳找出和 `table_1` 中用户事件时间戳中最近的一个时间戳,来满足最接近匹配的条件。如果有得话,则相等的时间戳值是最接近的值。在此例中,`user_id` 列可用于条件匹配,`ev_time` 列可用于最接近匹配。在此例中,`event_1_1` 可以 JOIN `event_2_1`,`event_1_2` 可以JOIN `event_2_3`,但是 `event_2_2` 不能被JOIN。 -!!! note "注" - `ASOF JOIN`在 [JOIN](../../../engines/table-engines/special/join.md) 表引擎中 **不受** 支持。 +:::note +`ASOF JOIN`在 [JOIN](../../../engines/table-engines/special/join.md) 表引擎中 **不受** 支持。 +::: ## 分布式联接 {#global-join} diff --git a/docs/zh/sql-reference/statements/select/limit-by.md b/docs/zh/sql-reference/statements/select/limit-by.md index 50e3505b7fb..68b88bf8d7a 100644 --- a/docs/zh/sql-reference/statements/select/limit-by.md +++ b/docs/zh/sql-reference/statements/select/limit-by.md @@ -14,8 +14,9 @@ ClickHouse支持以下语法变体: 处理查询时,ClickHouse首先选择经由排序键排序过后的数据。排序键可以显式地使用[ORDER BY](order-by.md#select-order-by)从句指定,或隐式地使用表引擎使用的排序键(数据的顺序仅在使用[ORDER BY](order-by.md#select-order-by)时才可以保证,否则由于多线程处理,数据顺序会随机化)。然后ClickHouse执行`LIMIT n BY expressions`从句,将每一行按 `expressions` 的值进行分组,并对每一分组返回前`n`行。如果指定了`OFFSET`,那么对于每一分组,ClickHouse会跳过前`offset_value`行,接着返回前`n`行。如果`offset_value`大于某一分组的行数,ClickHouse会从分组返回0行。 -!!! note "注" - `LIMIT BY`与[LIMIT](../../../sql-reference/statements/select/limit.md)没有关系。它们可以在同一个查询中使用。 +:::note +`LIMIT BY`与[LIMIT](../../../sql-reference/statements/select/limit.md)没有关系。它们可以在同一个查询中使用。 +::: ## 例 {#examples} diff --git a/docs/zh/sql-reference/statements/select/sample.md b/docs/zh/sql-reference/statements/select/sample.md index f701bd3b805..0993958b029 100644 --- a/docs/zh/sql-reference/statements/select/sample.md +++ b/docs/zh/sql-reference/statements/select/sample.md @@ -15,8 +15,9 @@ sidebar_label: SAMPLE - 当您的原始数据不准确时,所以近似不会明显降低质量。 - 业务需求的目标是近似结果(为了成本效益,或者向高级用户推销确切结果)。 -!!! note "注" - 您只能使用采样中的表 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) 族,并且只有在表创建过程中指定了采样表达式(请参阅 [MergeTree引擎](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table)). +:::note +您只能使用采样中的表 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) 族,并且只有在表创建过程中指定了采样表达式(请参阅 [MergeTree引擎](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table)). +::: 下面列出了数据采样的功能: diff --git a/docs/zh/sql-reference/statements/select/where.md b/docs/zh/sql-reference/statements/select/where.md index fe165e8ad16..6c7183840c7 100644 --- a/docs/zh/sql-reference/statements/select/where.md +++ b/docs/zh/sql-reference/statements/select/where.md @@ -11,9 +11,10 @@ sidebar_label: WHERE 如果基础表引擎支持,`WHERE`表达式会使用索引和分区进行剪枝。 -!!! note "注" - 有一个叫做过滤优化 [prewhere](../../../sql-reference/statements/select/prewhere.md) 的东西. - +:::note +有一个叫做过滤优化 [prewhere](../../../sql-reference/statements/select/prewhere.md) 的东西. +::: + 如果需要测试一个 [NULL](../../../sql-reference/syntax.md#null-literal) 值,请使用 [IS NULL](../../operators/index.md#operator-is-null) and [IS NOT NULL](../../operators/index.md#is-not-null) 运算符或 [isNull](../../../sql-reference/functions/functions-for-nulls.md#isnull) 和 [isNotNull](../../../sql-reference/functions/functions-for-nulls.md#isnotnull) 函数。否则带有 NULL 的表达式永远不会通过。 **示例** diff --git a/docs/zh/sql-reference/statements/system.md b/docs/zh/sql-reference/statements/system.md index d8d60c28af5..1942d6fb79a 100644 --- a/docs/zh/sql-reference/statements/system.md +++ b/docs/zh/sql-reference/statements/system.md @@ -124,10 +124,9 @@ ClickHouse可以管理 [MergeTree](../../engines/table-engines/mergetree-family/ SYSTEM STOP MERGES [[db.]merge_tree_family_table_name] ``` - -!!! note "Note" - `DETACH / ATTACH` 表操作会在后台进行表的merge操作,甚至当所有MergeTree表的合并操作已经停止的情况下。 - +:::note +`DETACH / ATTACH` 表操作会在后台进行表的merge操作,甚至当所有MergeTree表的合并操作已经停止的情况下。 +::: ### START MERGES {#query_language-system-start-merges} diff --git a/docs/zh/sql-reference/table-functions/mysql.md b/docs/zh/sql-reference/table-functions/mysql.md index 6c9753b9b12..4efee2e616b 100644 --- a/docs/zh/sql-reference/table-functions/mysql.md +++ b/docs/zh/sql-reference/table-functions/mysql.md @@ -49,8 +49,9 @@ SELECT name FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'mysql_database', 与原始MySQL表具有相同列的表对象。 -!!! note "注意" - 在`INSERT`查询中为了区分`mysql(...)`与带有列名列表的表名的表函数,你必须使用关键字`FUNCTION`或`TABLE FUNCTION`。查看如下示例。 +:::note +在`INSERT`查询中为了区分`mysql(...)`与带有列名列表的表名的表函数,你必须使用关键字`FUNCTION`或`TABLE FUNCTION`。查看如下示例。 +::: ## 用法示例 {#usage-example} From 380e076e842fee30e4bd6b9729a36edbe623e61f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 3 Mar 2023 11:19:05 +0000 Subject: [PATCH 251/470] x86 --> amd64 --- .github/workflows/backport_branches.yml | 2 +- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release_branches.yml | 2 +- tests/ci/ci_config.py | 2 +- tests/ci/compatibility_check.py | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index b14a32127c4..a324d20abc9 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -103,7 +103,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions - name: Cleanup if: always() run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e224d6cf5c3..b70fe256833 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -134,7 +134,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions - name: Cleanup if: always() run: | diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ae6cb1d3a83..ff98739db00 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -198,7 +198,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions - name: Cleanup if: always() run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 229532efdab..74ec1163cc9 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -95,7 +95,7 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (x86)" --check-glibc --check-distributions + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py --check-name "Compatibility check (amd64)" --check-glibc --check-distributions - name: Cleanup if: always() run: | diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9df198430d2..2f35b337cb3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -316,7 +316,7 @@ CI_CONFIG = { "Integration tests flaky check (asan)": { "required_build": "package_asan", }, - "Compatibility check (x86)": { + "Compatibility check (amd64)": { "required_build": "package_release", }, "Compatibility check (aarch64)": { diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 8841bedbb18..432e9ec7c01 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -214,7 +214,7 @@ def main(): # See https://sourceware.org/glibc/wiki/Glibc%20Timeline max_glibc_version = "" - if "x86" in args.check_name: + if "amd64" in args.check_name: max_glibc_version = "2.4" elif "aarch64" in args.check_name: max_glibc_version = "2.18" # because of build with newer sysroot? From d34f00f4370fd548a739bda02c4d1902c464e945 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 3 Mar 2023 15:02:05 +0300 Subject: [PATCH 252/470] Update run.sh --- docker/test/upgrade/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index df32e2833e7..ce8a56c777e 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -162,7 +162,7 @@ rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Cannot flush" \ -e "Container already exists" \ /var/log/clickhouse-server/clickhouse-server.upgrade.log | zgrep -Fa "" > /test_output/upgrade_error_messages.txt \ - && echo -e "Error message in clickhouse-server.log (see upgrade_error_messages.txt)$FAIL$(head_escaped /test_output/bc_check_error_messages.txt)" \ + && echo -e "Error message in clickhouse-server.log (see upgrade_error_messages.txt)$FAIL$(head_escaped /test_output/upgrade_error_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "No Error messages after server upgrade$OK" >> /test_output/test_results.tsv From 56f02374f6dbbb2195279fdf9415a3275a5697c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Mar 2023 12:14:05 +0100 Subject: [PATCH 253/470] Remove stateless test because it makes unfixably 02344_show_caches flaky --- tests/queries/0_stateless/02344_show_caches.reference | 1 - .../02454_create_table_with_custom_disk.reference | 1 - .../02454_create_table_with_custom_disk.sql | 10 ---------- 3 files changed, 12 deletions(-) diff --git a/tests/queries/0_stateless/02344_show_caches.reference b/tests/queries/0_stateless/02344_show_caches.reference index b321319a309..2ee4f902ba1 100644 --- a/tests/queries/0_stateless/02344_show_caches.reference +++ b/tests/queries/0_stateless/02344_show_caches.reference @@ -10,6 +10,5 @@ local_cache s3_cache_6 s3_cache_small local_cache_2 -__tmp_internal_324081342946782869538999598488311137423 local_cache_3 s3_cache_multi_2 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference index a71d52b6f57..1d8610c59c9 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -9,4 +9,3 @@ SETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\') CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'[HIDDEN]\'), index_granularity = 8192 a Int32 200 -CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = cache, max_size = \'[HIDDEN]\', path = \'[HIDDEN]\', disk = disk(type = local, path = \'[HIDDEN]\')), index_granularity = 8192 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 4909f91ccb1..6cb1c0774aa 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -26,13 +26,3 @@ DESCRIBE TABLE test; INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; - -DROP TABLE test; - -CREATE TABLE test (a Int32) -ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=cache, max_size='1Gi', path='/var/lib/clickhouse/custom_disk_cache/', disk=disk(type=local, path='/var/lib/clickhouse/disks/local/')); - -SHOW CREATE TABLE test; - -DROP TABLE test; From bc8b34f74ba8176020ba65c6d8aed44f8f53afdd Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 3 Mar 2023 13:16:46 +0100 Subject: [PATCH 254/470] adjust test test_seekable_formats --- tests/integration/test_storage_s3/test.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 54944b56919..8b20727a7b5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1056,13 +1056,13 @@ def test_seekable_formats(started_cluster): table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", ) result = instance.query( - f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='50M'" + f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='60M'" ) - assert int(result) == 1000000 + assert int(result) == 1500000 instance.query(f"SELECT * FROM {table_function} FORMAT Null") @@ -1073,7 +1073,7 @@ def test_seekable_formats(started_cluster): result = result.strip() assert result.endswith("MiB") result = result[: result.index(".")] - assert int(result) > 80 + assert int(result) > 150 def test_seekable_formats_url(started_cluster): @@ -1083,23 +1083,23 @@ def test_seekable_formats_url(started_cluster): table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", ) result = instance.query(f"SELECT count() FROM {table_function}") - assert int(result) == 1000000 + assert int(result) == 1500000 table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" exec_query_with_retry( instance, - f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1000000) settings s3_truncate_on_insert=1", + f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(1500000) settings s3_truncate_on_insert=1", ) table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" result = instance.query( - f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='50M'" + f"SELECT count() FROM {table_function} SETTINGS max_memory_usage='60M'" ) - assert int(result) == 1000000 + assert int(result) == 1500000 def test_empty_file(started_cluster): From e2278ca70c8945b17dce8ff9698905ca33f6f847 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 1 Mar 2023 20:59:56 +0100 Subject: [PATCH 255/470] Add code for autoscaling lambda --- tests/ci/autoscale_runners_lambda/app.py | 226 ++++++++++++++++++ .../build_and_deploy_archive.sh | 1 + .../autoscale_runners_lambda/requirements.txt | 1 + 3 files changed, 228 insertions(+) create mode 100644 tests/ci/autoscale_runners_lambda/app.py create mode 120000 tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh create mode 100644 tests/ci/autoscale_runners_lambda/requirements.txt diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py new file mode 100644 index 00000000000..2777d5cf9c1 --- /dev/null +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -0,0 +1,226 @@ +#!/usr/bin/env python3 + +"""The lambda to decrease/increase ASG desired capacity based on current queue""" + +import json +import logging +import time +from dataclasses import dataclass +from pprint import pformat +from typing import Any, List, Literal, Optional + +import boto3 # type: ignore +import requests # type: ignore + +RUNNER_TYPE_LABELS = [ + "builder", + "func-tester", + "func-tester-aarch64", + "fuzzer-unit-tester", + "stress-tester", + "style-checker", + "style-checker-aarch64", +] +QUEUE_QUERY = f"""SELECT + last_status AS status, + toUInt32(count()) AS length, + labels +FROM +( + SELECT + arraySort(groupArray(status))[-1] AS last_status, + labels, + id, + html_url + FROM default.workflow_jobs + WHERE has(labels, 'self-hosted') + AND hasAny({RUNNER_TYPE_LABELS}, labels) + AND started_at > now() - INTERVAL 2 DAY + GROUP BY ALL + HAVING last_status IN ('in_progress', 'queued') +) +GROUP BY ALL +ORDER BY labels, last_status""" + + +@dataclass +class Queue: + status: Literal["in_progress", "queued"] + lentgh: int + label: str + + +### VENDORING +def get_parameter_from_ssm(name, decrypt=True, client=None): + if not client: + client = boto3.client("ssm", region_name="us-east-1") + return client.get_parameter(Name=name, WithDecryption=decrypt)["Parameter"]["Value"] + + +class CHException(Exception): + pass + + +class ClickHouseHelper: + def __init__( + self, + url: Optional[str] = None, + user: Optional[str] = None, + password: Optional[str] = None, + ): + self.url = url + self.auth = {} + if user: + self.auth["X-ClickHouse-User"] = user + if password: + self.auth["X-ClickHouse-Key"] = password + + def _select_and_get_json_each_row(self, db, query): + params = { + "database": db, + "query": query, + "default_format": "JSONEachRow", + } + for i in range(5): + response = None + try: + response = requests.get(self.url, params=params, headers=self.auth) + response.raise_for_status() + return response.text + except Exception as ex: + logging.warning("Cannot fetch data with exception %s", str(ex)) + if response: + logging.warning("Reponse text %s", response.text) + time.sleep(0.1 * i) + + raise CHException("Cannot fetch data from clickhouse") + + def select_json_each_row(self, db, query): + text = self._select_and_get_json_each_row(db, query) + result = [] + for line in text.split("\n"): + if line: + result.append(json.loads(line)) + return result + + +CH_CLIENT = ClickHouseHelper(get_parameter_from_ssm("clickhouse-test-stat-url"), "play") + + +def set_capacity( + runner_type: str, queues: List[Queue], client: Any, dry_run: bool = True +) -> None: + assert len(queues) in (1, 2) + assert all(q.label == runner_type for q in queues) + as_groups = client.describe_auto_scaling_groups( + Filters=[ + {"Name": "tag-key", "Values": ["github:runner-type"]}, + {"Name": "tag-value", "Values": [runner_type]}, + ] + )["AutoScalingGroups"] + assert len(as_groups) == 1 + asg = as_groups[0] + running = 0 + queued = 0 + for q in queues: + if q.status == "in_progress": + running = q.lentgh + continue + if q.status == "queued": + queued = q.lentgh + continue + raise ValueError("Queue status is not in ['in_progress', 'queued']") + + capacity_reserve = max(0, asg["DesiredCapacity"] - running) + stop = False + if queued: + # This part is about scaling up + # First, let's check if there's enough runners to cover the queue + stop = stop or (asg["DesiredCapacity"] - running - queued) > 0 + + stop = stop or asg["MaxSize"] <= asg["DesiredCapacity"] + # Let's calculate a new desired capacity. Here the scale is used to not + # scale up and down too quickly + desired_capacity = asg["DesiredCapacity"] + ((queued - capacity_reserve) // 5) + desired_capacity = max(desired_capacity, asg["MinSize"]) + desired_capacity = min(desired_capacity, asg["MaxSize"]) + # Finally, should the capacity be even changed + stop = stop or asg["DesiredCapacity"] == desired_capacity + if stop: + return + logging.info( + "The ASG %s capacity will be increased to %s, current capacity=%s, " + "maximum capacity=%s, running jobs=%s, queue size=%s", + asg["AutoScalingGroupName"], + desired_capacity, + asg["DesiredCapacity"], + asg["MaxSize"], + running, + queued, + ) + if not dry_run: + client.set_desired_capacity( + AutoScalingGroupName=asg["AutoScalingGroupName"], + DesiredCapacity=desired_capacity, + ) + return + + # Now we will calculate if we need to scale down + stop = stop or asg["DesiredCapacity"] <= asg["MinSize"] + stop = stop or asg["DesiredCapacity"] <= running + # Scale down quicker than scale up + desired_capacity = asg["DesiredCapacity"] - (capacity_reserve // 3) + desired_capacity = max(desired_capacity, asg["MinSize"]) + desired_capacity = min(desired_capacity, asg["MaxSize"]) + stop = stop or asg["DesiredCapacity"] == desired_capacity + if stop: + return + + logging.info( + "The ASG %s capacity will be decreased to %s, current capacity=%s, " + "minimum capacity=%s, running jobs=%s, queue size=%s", + asg["AutoScalingGroupName"], + desired_capacity, + asg["DesiredCapacity"], + asg["MinSize"], + running, + queued, + ) + if not dry_run: + client.set_desired_capacity( + AutoScalingGroupName=asg["AutoScalingGroupName"], + DesiredCapacity=desired_capacity, + ) + + +def main(dry_run: bool = True) -> None: + logging.getLogger().setLevel(logging.INFO) + asg_client = boto3.client("autoscaling") + try: + global CH_CLIENT + queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) + except CHException as ex: + logging.exception( + "Got an exception on insert, tryuing to update the client " + "credentials and repeat", + exc_info=ex, + ) + CH_CLIENT = ClickHouseHelper( + get_parameter_from_ssm("clickhouse-test-stat-url"), "play" + ) + queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) + + logging.info("Received queue data:\n%s", pformat(queues, width=120)) + for runner_type in RUNNER_TYPE_LABELS: + runner_queues = [ + Queue(queue["status"], queue["length"], runner_type) + for queue in queues + if runner_type in queue["labels"] + ] + set_capacity(runner_type, runner_queues, asg_client, dry_run) + + +def handler(event: dict, context: Any) -> None: + _ = event + _ = context + return main(False) diff --git a/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh b/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh new file mode 120000 index 00000000000..96ba3fa024e --- /dev/null +++ b/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh @@ -0,0 +1 @@ +../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/autoscale_runners_lambda/requirements.txt b/tests/ci/autoscale_runners_lambda/requirements.txt new file mode 100644 index 00000000000..f2293605cf1 --- /dev/null +++ b/tests/ci/autoscale_runners_lambda/requirements.txt @@ -0,0 +1 @@ +requests From 0483de0e04af22ae9a51b1feceef86810efd2cda Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 13:53:56 +0100 Subject: [PATCH 256/470] Reorder `status` in default.workflow_jobs --- tests/ci/workflow_jobs_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py index f6589576806..9436e01ad53 100644 --- a/tests/ci/workflow_jobs_lambda/app.py +++ b/tests/ci/workflow_jobs_lambda/app.py @@ -213,7 +213,7 @@ def send_event_workflow_job(workflow_job: WorkflowJob) -> None: # `head_sha` String, # `url` String, # `html_url` String, - # `status` Enum8('queued' = 1, 'in_progress' = 2, 'completed' = 3, 'waiting' = 4), + # `status` Enum8('waiting' = 1, 'queued' = 2, 'in_progress' = 3, 'completed' = 4), # `conclusion` LowCardinality(String), # `started_at` DateTime, # `completed_at` DateTime, From a5869e25f644551b00c034bb25efa56d4ed111d3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 20:21:40 +0100 Subject: [PATCH 257/470] Add tests for autoscale_runners_lambda --- tests/ci/autoscale_runners_lambda_test.py | 133 ++++++++++++++++++++++ 1 file changed, 133 insertions(+) create mode 100644 tests/ci/autoscale_runners_lambda_test.py diff --git a/tests/ci/autoscale_runners_lambda_test.py b/tests/ci/autoscale_runners_lambda_test.py new file mode 100644 index 00000000000..e6ac157ee4a --- /dev/null +++ b/tests/ci/autoscale_runners_lambda_test.py @@ -0,0 +1,133 @@ +#!/usr/bin/env python + +import unittest +from dataclasses import dataclass +from typing import Any, List + +from autoscale_runners_lambda.app import set_capacity, Queue + + +@dataclass +class TestCase: + name: str + min_size: int + desired_capacity: int + max_size: int + queues: List[Queue] + expected_capacity: int + + +class TestSetCapacity(unittest.TestCase): + class FakeClient: + def __init__(self): + self._expected_data = {} # type: dict + self._expected_capacity = -1 + + @property + def expected_data(self) -> dict: + """a one-time property""" + data, self._expected_data = self._expected_data, {} + return data + + @expected_data.setter + def expected_data(self, value: dict) -> None: + self._expected_data = value + + @property + def expected_capacity(self) -> int: + """one-time property""" + capacity, self._expected_capacity = self._expected_capacity, -1 + return capacity + + def describe_auto_scaling_groups(self, **kwargs: Any) -> dict: + _ = kwargs + return self.expected_data + + def set_desired_capacity(self, **kwargs: Any) -> None: + self._expected_capacity = kwargs["DesiredCapacity"] + + def data_helper( + self, name: str, min_size: int, desired_capacity: int, max_size: int + ) -> None: + self.expected_data = { + "AutoScalingGroups": [ + { + "AutoScalingGroupName": name, + "DesiredCapacity": desired_capacity, + "MinSize": min_size, + "MaxSize": max_size, + } + ] + } + + def setUp(self): + self.client = self.FakeClient() + + def test_normal_cases(self): + test_cases = ( + # Do not change capacity + TestCase("noqueue", 1, 13, 20, [Queue("in_progress", 155, "noqueue")], -1), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 17, "w/reserve")], -1), + TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), + TestCase("less-min", 10, 3, 20, [Queue("queued", 1, "less-min")], -1), + # Increase capacity + TestCase("increase", 1, 13, 20, [Queue("queued", 23, "increase")], 15), + TestCase("increase", 1, 13, 20, [Queue("queued", 18, "increase")], 14), + TestCase("increase", 1, 13, 20, [Queue("queued", 183, "increase")], 20), + TestCase( + "increase-w/o reserve", + 1, + 13, + 20, + [ + Queue("in_progress", 11, "increase-w/o reserve"), + Queue("queued", 12, "increase-w/o reserve"), + ], + 15, + ), + TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), + # Decrease capacity + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 10), + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 5, "decrease")], 11), + ) + for t in test_cases: + self.client.data_helper(t.name, t.min_size, t.desired_capacity, t.max_size) + set_capacity(t.name, t.queues, self.client, False) + self.assertEqual(t.expected_capacity, self.client.expected_capacity, t.name) + + def test_exceptions(self): + test_cases = ( + ( + TestCase( + "different names", + 1, + 1, + 1, + [Queue("queued", 5, "another name")], + -1, + ), + AssertionError, + ), + (TestCase("wrong queue len", 1, 1, 1, [], -1), AssertionError), + ( + TestCase( + "wrong queue", 1, 1, 1, [Queue("wrong", 1, "wrong queue")], -1 # type: ignore + ), + ValueError, + ), + ) + for t, error in test_cases: + with self.assertRaises(error): + self.client.data_helper( + t.name, t.min_size, t.desired_capacity, t.max_size + ) + set_capacity(t.name, t.queues, self.client, False) + + with self.assertRaises(AssertionError): + self.client.expected_data = {"AutoScalingGroups": [1, 2]} + set_capacity( + "wrong number of ASGs", + [Queue("queued", 1, "wrong number of ASGs")], + self.client, + ) From 4e9c2462648c265646cd79055837a73ecd6b7631 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 20:28:27 +0100 Subject: [PATCH 258/470] Check CI python code always --- .github/workflows/pull_request.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7d410f833c5..1efa6147c9a 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -37,7 +37,6 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py PythonUnitTests: - needs: CheckLabels runs-on: [self-hosted, style-checker] steps: - name: Check out repository code From c1774cabd01cea086c96547e7fffd35a3a2282b2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 2 Mar 2023 20:29:34 +0100 Subject: [PATCH 259/470] Do not install GitPython on runners, it is preinstalled --- .github/workflows/cherry_pick.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/cherry_pick.yml b/.github/workflows/cherry_pick.yml index 065e584182b..8d1e2055978 100644 --- a/.github/workflows/cherry_pick.yml +++ b/.github/workflows/cherry_pick.yml @@ -35,7 +35,6 @@ jobs: fetch-depth: 0 - name: Cherry pick run: | - sudo pip install GitPython cd "$GITHUB_WORKSPACE/tests/ci" python3 cherry_pick.py - name: Cleanup From 6a746beff8586ebbe496089b8da4dff044b5657a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Mar 2023 09:53:21 +0100 Subject: [PATCH 260/470] Tune style-checker scaling up and down by modifiers --- tests/ci/autoscale_runners_lambda/app.py | 26 ++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 2777d5cf9c1..25c6c2eb9a8 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -7,7 +7,7 @@ import logging import time from dataclasses import dataclass from pprint import pformat -from typing import Any, List, Literal, Optional +from typing import Any, List, Literal, Optional, Tuple import boto3 # type: ignore import requests # type: ignore @@ -50,6 +50,19 @@ class Queue: label: str +def get_scales(runner_type: str) -> Tuple[int, int]: + "returns the multipliers for scaling down and up ASG by types" + # Scaling down is quicker on the lack of running jobs than scaling up on + # queue + scale_down = 3 + scale_up = 5 + if runner_type == "style-checker": + # the style checkers have so many noise, so it scales up too quickly + scale_down = 2 + scale_up = 10 + return scale_down, scale_up + + ### VENDORING def get_parameter_from_ssm(name, decrypt=True, client=None): if not client: @@ -131,6 +144,7 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") + scale_down, scale_up = get_scales(runner_type) capacity_reserve = max(0, asg["DesiredCapacity"] - running) stop = False if queued: @@ -139,9 +153,10 @@ def set_capacity( stop = stop or (asg["DesiredCapacity"] - running - queued) > 0 stop = stop or asg["MaxSize"] <= asg["DesiredCapacity"] - # Let's calculate a new desired capacity. Here the scale is used to not - # scale up and down too quickly - desired_capacity = asg["DesiredCapacity"] + ((queued - capacity_reserve) // 5) + # Let's calculate a new desired capacity + desired_capacity = asg["DesiredCapacity"] + ( + (queued - capacity_reserve) // scale_up + ) desired_capacity = max(desired_capacity, asg["MinSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"]) # Finally, should the capacity be even changed @@ -168,8 +183,7 @@ def set_capacity( # Now we will calculate if we need to scale down stop = stop or asg["DesiredCapacity"] <= asg["MinSize"] stop = stop or asg["DesiredCapacity"] <= running - # Scale down quicker than scale up - desired_capacity = asg["DesiredCapacity"] - (capacity_reserve // 3) + desired_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) desired_capacity = max(desired_capacity, asg["MinSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"]) stop = stop or asg["DesiredCapacity"] == desired_capacity From a5764643604dff138150ad7719b7c0781ea75593 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Mar 2023 12:01:05 +0100 Subject: [PATCH 261/470] Reduce statisctics interval, process empty response for runners --- tests/ci/autoscale_runners_lambda/app.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 25c6c2eb9a8..ca173be2eab 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -35,7 +35,7 @@ FROM FROM default.workflow_jobs WHERE has(labels, 'self-hosted') AND hasAny({RUNNER_TYPE_LABELS}, labels) - AND started_at > now() - INTERVAL 2 DAY + AND started_at > now() - INTERVAL 1 DAY GROUP BY ALL HAVING last_status IN ('in_progress', 'queued') ) @@ -231,6 +231,7 @@ def main(dry_run: bool = True) -> None: for queue in queues if runner_type in queue["labels"] ] + runner_queues = runner_queues or [Queue("in_progress", 0, runner_type)] set_capacity(runner_type, runner_queues, asg_client, dry_run) From 602933ce1edd9f02ec35b47ef428a49685fa123c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Mar 2023 12:09:09 +0100 Subject: [PATCH 262/470] Make smarter scaler based on reserve/deficit --- tests/ci/autoscale_runners_lambda/app.py | 19 +++++++++---------- tests/ci/autoscale_runners_lambda_test.py | 5 ++--- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index ca173be2eab..596e675ee24 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -145,18 +145,18 @@ def set_capacity( raise ValueError("Queue status is not in ['in_progress', 'queued']") scale_down, scale_up = get_scales(runner_type) - capacity_reserve = max(0, asg["DesiredCapacity"] - running) + # How much nodes are free (positive) or need to be added (negative) + capacity_reserve = asg["DesiredCapacity"] - running - queued stop = False - if queued: + if capacity_reserve < 0: # This part is about scaling up - # First, let's check if there's enough runners to cover the queue - stop = stop or (asg["DesiredCapacity"] - running - queued) > 0 - + capacity_deficit = -capacity_reserve + # It looks that we are still OK, since no queued jobs exist + stop = stop or queued == 0 + # Are we already at the capacity limits stop = stop or asg["MaxSize"] <= asg["DesiredCapacity"] # Let's calculate a new desired capacity - desired_capacity = asg["DesiredCapacity"] + ( - (queued - capacity_reserve) // scale_up - ) + desired_capacity = asg["DesiredCapacity"] + (capacity_deficit // scale_up) desired_capacity = max(desired_capacity, asg["MinSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"]) # Finally, should the capacity be even changed @@ -181,8 +181,7 @@ def set_capacity( return # Now we will calculate if we need to scale down - stop = stop or asg["DesiredCapacity"] <= asg["MinSize"] - stop = stop or asg["DesiredCapacity"] <= running + stop = stop or asg["DesiredCapacity"] == asg["MinSize"] desired_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) desired_capacity = max(desired_capacity, asg["MinSize"]) desired_capacity = min(desired_capacity, asg["MaxSize"]) diff --git a/tests/ci/autoscale_runners_lambda_test.py b/tests/ci/autoscale_runners_lambda_test.py index e6ac157ee4a..7efa0004745 100644 --- a/tests/ci/autoscale_runners_lambda_test.py +++ b/tests/ci/autoscale_runners_lambda_test.py @@ -67,10 +67,7 @@ class TestSetCapacity(unittest.TestCase): test_cases = ( # Do not change capacity TestCase("noqueue", 1, 13, 20, [Queue("in_progress", 155, "noqueue")], -1), - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), TestCase("w/reserve", 1, 13, 20, [Queue("queued", 17, "w/reserve")], -1), - TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), - TestCase("less-min", 10, 3, 20, [Queue("queued", 1, "less-min")], -1), # Increase capacity TestCase("increase", 1, 13, 20, [Queue("queued", 23, "increase")], 15), TestCase("increase", 1, 13, 20, [Queue("queued", 18, "increase")], 14), @@ -88,6 +85,8 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), # Decrease capacity + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 11), + TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 20), TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 10), TestCase("decrease", 1, 13, 20, [Queue("in_progress", 5, "decrease")], 11), ) From dfb3feb686a4ebb7c32c682e6657c4a5c3470188 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 16:48:09 +0400 Subject: [PATCH 263/470] Typo at the page `/docs/ru/engines/table-engines/mergetree-family/collapsingmergetree.md` --- .../table-engines/mergetree-family/collapsingmergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/ru/engines/table-engines/mergetree-family/collapsingmergetree.md index dac490468d0..e3b4238a200 100644 --- a/docs/ru/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -89,7 +89,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] └─────────────────────┴───────────┴──────────┴──────┘ ``` -Первая строка отменяет предыдущее состояние объекта (пользователя). Она должен повторять все поля из ключа сортировки для отменённого состояния за исключением `Sign`. +Первая строка отменяет предыдущее состояние объекта (пользователя). Она должна повторять все поля из ключа сортировки для отменённого состояния за исключением `Sign`. Вторая строка содержит текущее состояние. From 5d1f3dbf4e2fec3bdf190b47d02aacee83fb6e21 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 16:50:14 +0400 Subject: [PATCH 264/470] Typo at the page `/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md` --- .../dictionaries/external-dictionaries/external-dicts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 314fefab5eb..a262a354889 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -14,7 +14,7 @@ ClickHouse: - Периодически обновляет их и динамически подгружает отсутствующие значения. - Позволяет создавать внешние словари с помощью xml-файлов или [DDL-запросов](../../statements/create/dictionary.md#create-dictionary-query). -Конфигурация внешних словарей может находится в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config). +Конфигурация внешних словарей может находиться в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config). Словари могут загружаться при старте сервера или при первом использовании, в зависимости от настройки [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load). From 1bbf5acd47b86e19f6767619d372466bca28bec2 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 3 Mar 2023 15:55:52 +0300 Subject: [PATCH 265/470] Pass headers from StorageURL to WriteBufferFromHTTP (#46996) * Pass headers from StorageURL to WriteBufferFromHTTP * Add a test * Lint * `time.sleep(1)` * Start echo server earlier * Add proper handling for mock server start * Automatic style fix --------- Co-authored-by: robot-clickhouse --- src/IO/WriteBufferFromHTTP.cpp | 4 ++ src/IO/WriteBufferFromHTTP.h | 2 + src/Storages/StorageURL.cpp | 11 +++- src/Storages/StorageURL.h | 1 + .../test_storage_url_http_headers/__init__.py | 0 .../http_headers_echo_server.py | 31 +++++++++ .../test_storage_url_http_headers/test.py | 66 +++++++++++++++++++ 7 files changed, 113 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_storage_url_http_headers/__init__.py create mode 100644 tests/integration/test_storage_url_http_headers/http_headers_echo_server.py create mode 100644 tests/integration/test_storage_url_http_headers/test.py diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index f7456ad6b6c..355c42a23c9 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -11,6 +11,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const std::string & method, const std::string & content_type, const std::string & content_encoding, + const HTTPHeaderEntries & additional_headers, const ConnectionTimeouts & timeouts, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) @@ -28,6 +29,9 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( if (!content_encoding.empty()) request.set("Content-Encoding", content_encoding); + for (const auto & header: additional_headers) + request.add(header.name, header.value); + LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); ostr = &session->sendRequest(request); diff --git a/src/IO/WriteBufferFromHTTP.h b/src/IO/WriteBufferFromHTTP.h index 6966bc8a5c5..ce5020dfa78 100644 --- a/src/IO/WriteBufferFromHTTP.h +++ b/src/IO/WriteBufferFromHTTP.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ public: const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only const std::string & content_type = "", const std::string & content_encoding = "", + const HTTPHeaderEntries & additional_headers = {}, const ConnectionTimeouts & timeouts = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 152dda8f360..c0ddb0bc48a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -458,6 +459,7 @@ StorageURLSink::StorageURLSink( ContextPtr context, const ConnectionTimeouts & timeouts, const CompressionMethod compression_method, + const HTTPHeaderEntries & headers, const String & http_method) : SinkToStorage(sample_block) { @@ -465,7 +467,7 @@ StorageURLSink::StorageURLSink( std::string content_encoding = toContentEncodingName(compression_method); write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, timeouts), + std::make_unique(Poco::URI(uri), http_method, content_type, content_encoding, headers, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, format_settings); @@ -530,6 +532,7 @@ public: ContextPtr context_, const ConnectionTimeouts & timeouts_, const CompressionMethod compression_method_, + const HTTPHeaderEntries & headers_, const String & http_method_) : PartitionedSink(partition_by, context_, sample_block_) , uri(uri_) @@ -539,6 +542,7 @@ public: , context(context_) , timeouts(timeouts_) , compression_method(compression_method_) + , headers(headers_) , http_method(http_method_) { } @@ -548,7 +552,7 @@ public: auto partition_path = PartitionedSink::replaceWildcards(uri, partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared( - partition_path, format, format_settings, sample_block, context, timeouts, compression_method, http_method); + partition_path, format, format_settings, sample_block, context, timeouts, compression_method, headers, http_method); } private: @@ -560,6 +564,7 @@ private: const ConnectionTimeouts timeouts; const CompressionMethod compression_method; + const HTTPHeaderEntries headers; const String http_method; }; @@ -821,6 +826,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad context, getHTTPTimeouts(context), compression_method, + headers, http_method); } else @@ -833,6 +839,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad context, getHTTPTimeouts(context), compression_method, + headers, http_method); } } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index acf49f3cb71..1cfffc3e73a 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -137,6 +137,7 @@ public: ContextPtr context, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, + const HTTPHeaderEntries & headers = {}, const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } diff --git a/tests/integration/test_storage_url_http_headers/__init__.py b/tests/integration/test_storage_url_http_headers/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_url_http_headers/http_headers_echo_server.py b/tests/integration/test_storage_url_http_headers/http_headers_echo_server.py new file mode 100644 index 00000000000..b1a3f6777b1 --- /dev/null +++ b/tests/integration/test_storage_url_http_headers/http_headers_echo_server.py @@ -0,0 +1,31 @@ +import http.server + +RESULT_PATH = "/headers.txt" + + +class RequestHandler(http.server.BaseHTTPRequestHandler): + def log_message(self, *args): + with open(RESULT_PATH, "w") as f: + f.write(self.headers.as_string()) + + def do_POST(self): + self.rfile.read1() + self.send_response(200) + self.end_headers() + self.wfile.write(b'{"status":"ok"}') + + +if __name__ == "__main__": + with open(RESULT_PATH, "w") as f: + f.write("") + httpd = http.server.HTTPServer( + ( + "localhost", + 8000, + ), + RequestHandler, + ) + try: + httpd.serve_forever() + finally: + httpd.server_close() diff --git a/tests/integration/test_storage_url_http_headers/test.py b/tests/integration/test_storage_url_http_headers/test.py new file mode 100644 index 00000000000..3bbf5ec81c9 --- /dev/null +++ b/tests/integration/test_storage_url_http_headers/test.py @@ -0,0 +1,66 @@ +import pytest +import os +import time + +from . import http_headers_echo_server + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +server = cluster.add_instance("node") + + +def run_echo_server(): + script_dir = os.path.dirname(os.path.realpath(__file__)) + + server.copy_file_to_container( + os.path.join(script_dir, "http_headers_echo_server.py"), + "/http_headers_echo_server.py", + ) + + server.exec_in_container( + [ + "bash", + "-c", + "python3 /http_headers_echo_server.py > /http_headers_echo.server.log 2>&1", + ], + detach=True, + user="root", + ) + + for _ in range(0, 10): + ping_response = server.exec_in_container( + ["curl", "-s", f"http://localhost:8000/"], + nothrow=True, + ) + + if "html" in ping_response: + return + + print(ping_response) + + raise Exception("Echo server is not responding") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + run_echo_server() + yield cluster + finally: + cluster.shutdown() + + +def test_storage_url_http_headers(started_cluster): + query = "INSERT INTO TABLE FUNCTION url('http://localhost:8000/', JSON, 'a UInt64', headers('X-My-Custom-Header'='test-header')) VALUES (1)" + + server.query(query) + + result = server.exec_in_container( + ["cat", http_headers_echo_server.RESULT_PATH], user="root" + ) + + print(result) + + assert "X-My-Custom-Header: test-header" in result From 6c9225063674f01285f98a05382e9a80aee74f97 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 3 Mar 2023 08:21:02 -0500 Subject: [PATCH 266/470] add compression types for file function --- docs/en/sql-reference/table-functions/file.md | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index d2ef66dde73..594c328c3ff 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -6,21 +6,22 @@ sidebar_label: file # file -Creates a table from a file. This table function is similar to [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md) ones. +Creates a table from a file. This table function is similar to [url](/docs/en/sql-reference/table-functions/url.md) and [hdfs](/docs/en/sql-reference/table-functions/hdfs.md) ones. -`file` function can be used in `SELECT` and `INSERT` queries on data in [File](../../engines/table-engines/special/file.md) tables. +`file` function can be used in `SELECT` and `INSERT` queries on data in [File](/docs/en/engines/table-engines/special/file.md) tables. **Syntax** ``` sql -file(path [,format] [,structure]) +file(path [,format] [,structure] [,compression]) ``` **Parameters** -- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. -- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. +- `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. The supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. **Returned value** @@ -53,7 +54,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U └─────────┴─────────┴─────────┘ ``` -Getting the first 10 lines of a table that contains 3 columns of [UInt32](../../sql-reference/data-types/int-uint.md) type from a CSV file: +Getting the first 10 lines of a table that contains 3 columns of [UInt32](/docs/en/sql-reference/data-types/int-uint.md) type from a CSV file: ``` sql SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10; @@ -143,4 +144,4 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 **See Also** -- [Virtual columns](../../engines/table-engines/index.md#table_engines-virtual_columns) +- [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) From 302e076ddd93f838605bcbf71ca12bb47b17edc6 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 17:21:52 +0400 Subject: [PATCH 267/470] Typos at the page `/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md` --- .../external-dictionaries/external-dicts-dict-polygon.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 64637edc4a4..24f29d3bf53 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -61,7 +61,7 @@ LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) - Мультиполигон. Представляет из себя массив полигонов. Каждый полигон задается двумерным массивом точек — первый элемент этого массива задает внешнюю границу полигона, последующие элементы могут задавать дырки, вырезаемые из него. -Точки могут задаваться массивом или кортежем из своих координат. В текущей реализации поддерживается только двумерные точки. +Точки могут задаваться массивом или кортежем из своих координат. В текущей реализации поддерживаются только двумерные точки. Пользователь может [загружать свои собственные данные](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) во всех поддерживаемых ClickHouse форматах. @@ -80,7 +80,7 @@ LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) - `POLYGON`. Синоним к `POLYGON_INDEX_CELL`. Запросы к словарю осуществляются с помощью стандартных [функций](../../../sql-reference/functions/ext-dict-functions.md) для работы со внешними словарями. -Важным отличием является то, что здесь ключами будут являются точки, для которых хочется найти содержащий их полигон. +Важным отличием является то, что здесь ключами являются точки, для которых хочется найти содержащий их полигон. **Пример** From 221ac4a1e57fa124f81a677a3b8630518bcbe4bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 3 Mar 2023 13:23:44 +0000 Subject: [PATCH 268/470] Better --- src/Interpreters/ProcessList.cpp | 49 ++++++++++++-------------------- 1 file changed, 18 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 52674dc1c77..6f5553f58fd 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -362,7 +362,9 @@ QueryStatus::QueryStatus( QueryStatus::~QueryStatus() { - assert(executors.empty()); + /// Check that all executors were invalidated. + for (const auto & e : executors) + assert(!e->executor); if (auto * memory_tracker = getMemoryTracker()) { @@ -394,35 +396,21 @@ CancellationCode QueryStatus::cancelQuery(bool) is_killed.store(true); std::unique_lock lock(executors_mutex); - - /// Track all cancelled executors. - std::unordered_set cancelled; - /// We cancel executors from the left to the right, so if the last executor - /// was cancelled, then all executors were cancelled. - while (!cancelled.contains(executors.back().get())) + for (const auto & e : executors) { - size_t size = executors.size(); - /// We should create a copy of executor holder, because it can be - /// removed from vector in removePipelineExecutor from another thread - /// and reference will be invalid. - for (auto e : executors) - { - if (cancelled.contains(e.get())) - continue; - /// We should call cancel() with unlocked executors_mutex, because - /// cancel() can try to lock some internal mutex that is already locked by query executing - /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, - /// which will lead to deadlock. - lock.unlock(); - e->cancel(); - lock.lock(); - cancelled.insert(e.get()); - /// While executors_mutex was unlocked, removePipelineExecutor could be called and - /// the size of executors could have changed. In this case we should start iterating - /// over it again to avoid using invalid iterators. - if (executors.size() != size) - break; - } + /// We should call cancel() with unlocked executors_mutex, because + /// cancel() can try to lock some internal mutex that is already locked by query executing + /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, + /// which will lead to deadlock. + /// Note that the size and the content of executors cannot be changed while + /// executors_mutex is unlocked, because: + /// 1) We don't allow adding new executors while cancelling query in addPipelineExecutor + /// 2) We don't actually remove executor holder from executors in removePipelineExecutor, + /// just mark that executor is invalid. + /// So, it's safe to continue iteration over executors after subsequent mutex locking. + lock.unlock(); + e->cancel(); + lock.lock(); } return CancellationCode::CancelSent; @@ -446,9 +434,8 @@ void QueryStatus::removePipelineExecutor(PipelineExecutor * e) std::lock_guard lock(executors_mutex); auto it = std::find_if(executors.begin(), executors.end(), [e](const ExecutorHolderPtr & x){ return x->executor == e; }); assert(it != executors.end()); - /// Invalidate executor pointer inside holder. + /// Invalidate executor pointer inside holder, but don't remove holder from the executors (to avoid race with cancelQuery) (*it)->remove(); - executors.erase(it); } bool QueryStatus::checkTimeLimit() From e06bcf21ec9f543d9a79620ac2565c05bed62635 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 18:11:27 +0400 Subject: [PATCH 269/470] Markup fixed at the page `/docs/ru/sql-reference/data-types/aggregatefunction.md` --- docs/ru/sql-reference/data-types/aggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index 21b452acb1d..e42b467e4af 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -6,7 +6,7 @@ sidebar_label: AggregateFunction # AggregateFunction {#data-type-aggregatefunction} -Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления] (../../sql-reference/statements/create/view.md). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. +Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления](../../sql-reference/statements/create/view.md). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. `AggregateFunction(name, types_of_arguments…)` — параметрический тип данных. From 420c79a2521d2a8358525e92df6e37861a6d2786 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 18:16:57 +0400 Subject: [PATCH 270/470] Typo at the page `/docs/ru/sql-reference/data-types/tuple.md` --- docs/ru/sql-reference/data-types/tuple.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/tuple.md b/docs/ru/sql-reference/data-types/tuple.md index 76370d01c0d..8953134d154 100644 --- a/docs/ru/sql-reference/data-types/tuple.md +++ b/docs/ru/sql-reference/data-types/tuple.md @@ -34,7 +34,7 @@ SELECT tuple(1,'a') AS x, toTypeName(x) ## Особенности работы с типами данных {#osobennosti-raboty-s-tipami-dannykh} -При создании кортежа «на лету» ClickHouse автоматически определяет тип каждого аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../../sql-reference/data-types/tuple.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). +При создании кортежа «на лету» ClickHouse автоматически определяет тип всех аргументов как минимальный из типов, который может сохранить значение аргумента. Если аргумент — [NULL](../../sql-reference/data-types/tuple.md#null-literal), то тип элемента кортежа — [Nullable](nullable.md). Пример автоматического определения типа данных: From e44e5281182c2cda7aef51bdabff529d337d1f3a Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Fri, 3 Mar 2023 06:20:18 -0800 Subject: [PATCH 271/470] remove unnecessary reverse memcpy --- src/Common/formatIPv6.h | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index bc8f70f047c..7b88f93750b 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -82,11 +82,7 @@ inline bool parseIPv4(T * &src, EOFfunction eof, unsigned char * dst, int32_t fi break; } - if constexpr (std::endian::native == std::endian::little) - memcpy(dst, &result, sizeof(result)); - else - reverseMemcpy(dst, &result, sizeof(result)); - + memcpy(dst, &result, sizeof(result)); return true; } From dfb9258c2862af4196fa17eb213dc8cf217ea1cb Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 18:22:02 +0400 Subject: [PATCH 272/470] Markup fixed a the page `/docs/ru/sql-reference/data-types/special-data-types/interval.md` --- docs/ru/sql-reference/data-types/special-data-types/interval.md | 1 + 1 file changed, 1 insertion(+) 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 856275ed8f2..109ceee7852 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 @@ -10,6 +10,7 @@ sidebar_label: Interval :::danger "Внимание" Нельзя использовать типы данных `Interval` для хранения данных в таблице. +::: Структура: From a8ceab136635a9238a431dea5d651c9731a10e6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 3 Mar 2023 09:24:58 -0500 Subject: [PATCH 273/470] NOLINT for getenv --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 39fe552dfac..8db0d95e70d 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -170,7 +170,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n bool isSudo() { - return getuid() == 0 && geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); + return getuid() == 0 && geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); // NOLINT(concurrency-mt-unsafe) } /// Read data about files and decomrpess them. From d55ebe74f67a6a03eec9b2034c775bd3d7d8c08c Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 18:27:27 +0400 Subject: [PATCH 274/470] Markup fixed at the page `/docs/ru/sql-reference/data-types/geo.md` --- docs/ru/sql-reference/data-types/geo.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/data-types/geo.md b/docs/ru/sql-reference/data-types/geo.md index a7c5f79b0be..24d981195f5 100644 --- a/docs/ru/sql-reference/data-types/geo.md +++ b/docs/ru/sql-reference/data-types/geo.md @@ -10,6 +10,7 @@ ClickHouse поддерживает типы данных для отображ :::danger "Предупреждение" Сейчас использование типов данных для работы с географическими структурами является экспериментальной возможностью. Чтобы использовать эти типы данных, включите настройку `allow_experimental_geo_types = 1`. +::: **См. также** - [Хранение географических структур данных](https://ru.wikipedia.org/wiki/GeoJSON). From 737cf8e14934f7b4ce4869241fb67e461143bc45 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 Mar 2023 15:14:49 +0000 Subject: [PATCH 275/470] Better --- src/Client/HedgedConnections.cpp | 4 +- src/Client/MultiplexedConnections.cpp | 4 +- src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 5 - src/Core/SettingsEnums.h | 9 - src/Interpreters/Cluster.cpp | 2 +- src/Interpreters/Context.cpp | 23 ++- src/Interpreters/Context.h | 9 + src/Interpreters/InterpreterSelectQuery.cpp | 45 +++-- .../getCustomKeyFilterForParallelReplicas.cpp | 49 ++--- .../getCustomKeyFilterForParallelReplicas.h | 7 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +- src/Storages/SelectQueryInfo.h | 2 + src/Storages/StorageDistributed.cpp | 55 +----- .../test_parallel_replicas_custom_key/test.py | 74 ++------ ...4_shard_distributed_with_many_replicas.sql | 2 +- ...here_max_parallel_replicas_distributed.sql | 5 +- .../01034_sample_final_distributed.sql | 5 +- .../01557_max_parallel_replicas_no_sample.sql | 3 +- .../02221_parallel_replicas_bug.sh | 2 +- ...arallel_reading_from_replicas_benchmark.sh | 1 - .../02404_memory_bound_merging.sql | 3 +- ...max_parallel_replicas_custom_key.reference | 175 +++++++++++------- .../02535_max_parallel_replicas_custom_key.sh | 12 +- 24 files changed, 222 insertions(+), 282 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index b1fa6886e84..b97f9454fa5 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -175,9 +175,7 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 - && (settings.parallel_replicas_mode != ParallelReplicasMode::READ_TASKS - || !settings.allow_experimental_parallel_reading_from_replicas); + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) { diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index ec6788105ca..cc260353339 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -141,9 +141,7 @@ void MultiplexedConnections::sendQuery( } } - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 - && (settings.parallel_replicas_mode != ParallelReplicasMode::READ_TASKS - || !settings.allow_experimental_parallel_reading_from_replicas); + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; size_t num_replicas = replica_states.size(); if (num_replicas > 1) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 83aa6104996..797f57f1c04 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -147,9 +147,8 @@ class IColumn; M(UInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ M(UInt64, parallel_replicas_count, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the number of parallel replicas participating in query processing.", 0) \ M(UInt64, parallel_replica_offset, 0, "This is internal setting that should not be used directly and represents an implementation detail of the 'parallel replicas' mode. This setting will be automatically set up by the initiator server for distributed queries to the index of the replica participating in query processing among parallel replicas.", 0) \ - M(Map, parallel_replicas_custom_key, "", "Custom key for parallel replicas using modulo operation on the key for assigning work to replicas.", 0) \ + M(String, parallel_replicas_custom_key, "", "Custom key assigning work to replicas when parallel replicas are used.", 0) \ M(ParallelReplicasCustomKeyFilterType, parallel_replicas_custom_key_filter_type, ParallelReplicasCustomKeyFilterType::DEFAULT, "Type of filter to use with custom key for parallel replicas. default - use modulo operation on the custom key, range - use range filter on custom key using all possible values for the value type of custom key.", 0) \ - M(ParallelReplicasMode, parallel_replicas_mode, ParallelReplicasMode::SAMPLE_KEY, "How to process query using multiple replicas.", 0) \ \ M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \ M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index d5bd6cb2ae6..b04757ecaa2 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -167,11 +167,6 @@ IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, {"kusto", Dialect::kusto}}) -IMPLEMENT_SETTING_ENUM(ParallelReplicasMode, ErrorCodes::BAD_ARGUMENTS, - {{"sample_key", ParallelReplicasMode::SAMPLE_KEY}, - {"custom_key", ParallelReplicasMode::CUSTOM_KEY}, - {"read_tasks", ParallelReplicasMode::READ_TASKS}}) - IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, {{"default", ParallelReplicasCustomKeyFilterType::DEFAULT}, {"range", ParallelReplicasCustomKeyFilterType::RANGE}}) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 5bb36cec60d..ae9456cc6d7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -203,15 +203,6 @@ enum class Dialect DECLARE_SETTING_ENUM(Dialect) -enum class ParallelReplicasMode : uint8_t -{ - SAMPLE_KEY, - CUSTOM_KEY, - READ_TASKS, -}; - -DECLARE_SETTING_ENUM(ParallelReplicasMode) - enum class ParallelReplicasCustomKeyFilterType : uint8_t { DEFAULT, diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index 1975fa29686..b419dacd523 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -752,7 +752,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti auto shuffled_replicas = replicas; // shuffle replicas so we don't always pick the same subset shuffleReplicas(shuffled_replicas, settings); - create_shards_from_replicas(std::span{shuffled_replicas.begin(), shuffled_replicas.begin() + max_replicas_from_shard}); + create_shards_from_replicas(std::span{shuffled_replicas.begin(), max_replicas_from_shard}); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 08a3cf206c5..90f9c3bbac4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4056,23 +4056,34 @@ std::shared_ptr Context::getAsyncReadCounters() const return async_read_counters; } +Context::ParallelReplicasMode Context::getParallelReplicasMode() const +{ + const auto & settings = getSettingsRef(); + + using enum Context::ParallelReplicasMode; + if (!settings.parallel_replicas_custom_key.value.empty()) + return CUSTOM_KEY; + + if (settings.allow_experimental_parallel_reading_from_replicas + && !settings.use_hedged_requests) + return READ_TASKS; + + return SAMPLE_KEY; +} + bool Context::canUseParallelReplicasOnInitiator() const { const auto & settings = getSettingsRef(); - return settings.allow_experimental_parallel_reading_from_replicas - && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1 - && !settings.use_hedged_requests && !getClientInfo().collaborate_with_initiator; } bool Context::canUseParallelReplicasOnFollower() const { const auto & settings = getSettingsRef(); - return settings.allow_experimental_parallel_reading_from_replicas - && settings.parallel_replicas_mode == ParallelReplicasMode::READ_TASKS + return getParallelReplicasMode() == ParallelReplicasMode::READ_TASKS && settings.max_parallel_replicas > 1 - && !settings.use_hedged_requests && getClientInfo().collaborate_with_initiator; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 19bb6868331..67594a41459 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1123,6 +1123,15 @@ public: bool canUseParallelReplicasOnInitiator() const; bool canUseParallelReplicasOnFollower() const; + enum class ParallelReplicasMode : uint8_t + { + SAMPLE_KEY, + CUSTOM_KEY, + READ_TASKS, + }; + + ParallelReplicasMode getParallelReplicasMode() const; + private: std::unique_lock getLock() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e84df90c449..c53734d1e9b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -517,29 +517,38 @@ InterpreterSelectQuery::InterpreterSelectQuery( settings.additional_table_filters, joined_tables.tablesWithColumns().front().table, *context); ASTPtr parallel_replicas_custom_filter_ast = nullptr; - if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) + if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY && !joined_tables.tablesWithColumns().empty()) { - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, joined_tables.tablesWithColumns().front().table, *context)) + if (settings.parallel_replicas_count > 1) { - LOG_INFO(log, "Processing query on a replica using custom_key"); - if (!storage) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context)) + { + LOG_INFO(log, "Processing query on a replica using custom_key"); + if (!storage) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); - parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( - settings.parallel_replicas_count, - settings.parallel_replica_offset, - std::move(custom_key_ast), - settings.parallel_replicas_custom_key_filter_type, - *storage, - context); + parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica( + settings.parallel_replicas_count, + settings.parallel_replica_offset, + std::move(custom_key_ast), + settings.parallel_replicas_custom_key_filter_type, + *storage, + context); + } + else if (settings.parallel_replica_offset > 0) + { + LOG_DEBUG( + log, + "Will use no data on this replica because parallel replicas processing with custom_key has been requested" + " (setting 'max_parallel_replicas') but the table does not have custom_key defined for it or it's invalid (settings `parallel_replicas_custom_key`)"); + parallel_replicas_custom_filter_ast = std::make_shared(false); + } } - else if (settings.parallel_replica_offset > 0) + else if (auto * distributed = dynamic_cast(storage.get()); + distributed && canUseCustomKey(settings, *distributed->getCluster(), *context)) { - LOG_DEBUG( - log, - "Will use no data on this replica because parallel replicas processing with custom_key has been requested" - " (setting 'max_parallel_replicas') but the table does not have custom_key defined for it (settings `parallel_replicas_custom_key`)"); - parallel_replicas_custom_filter_ast = std::make_shared(false); + query_info.use_custom_key = true; + context->setSetting("distributed_group_by_no_merge", 2); } } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp index 229668ceff4..2a32d450497 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.cpp @@ -20,6 +20,12 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; } +bool canUseCustomKey(const Settings & settings, const Cluster & cluster, const Context & context) +{ + return settings.max_parallel_replicas > 1 && context.getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY + && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; +} + ASTPtr getCustomKeyFilterForParallelReplica( size_t replicas_count, size_t replica_num, @@ -115,43 +121,14 @@ ASTPtr getCustomKeyFilterForParallelReplica( return makeASTFunction("and", std::move(lower_function), std::move(upper_function)); } -ASTPtr parseCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context) +ASTPtr parseCustomKeyForTable(const String & custom_key, const Context & context) { - for (size_t i = 0; i < custom_keys.size(); ++i) - { - const auto & tuple = custom_keys[i].safeGet(); - auto & table = tuple.at(0).safeGet(); - auto & filter = tuple.at(1).safeGet(); - - if (table == target.alias || - (table == target.table && context.getCurrentDatabase() == target.database) || - (table == target.database + '.' + target.table)) - { - /// Try to parse expression - ParserExpression parser; - const auto & settings = context.getSettingsRef(); - return parseQuery( - parser, filter.data(), filter.data() + filter.size(), - "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); - } - } - - return nullptr; -} - -bool containsCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context) -{ - for (size_t i = 0; i < custom_keys.size(); ++i) - { - const auto & tuple = custom_keys[i].safeGet(); - auto & table = tuple.at(0).safeGet(); - - if (table == target.alias || - (table == target.table && context.getCurrentDatabase() == target.database) || - (table == target.database + '.' + target.table)) - return true; - } - return false; + /// Try to parse expression + ParserExpression parser; + const auto & settings = context.getSettingsRef(); + return parseQuery( + parser, custom_key.data(), custom_key.data() + custom_key.size(), + "parallel replicas custom key", settings.max_query_size, settings.max_parser_depth); } } diff --git a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h index fc515132487..543f1889b32 100644 --- a/src/Interpreters/getCustomKeyFilterForParallelReplicas.h +++ b/src/Interpreters/getCustomKeyFilterForParallelReplicas.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -9,6 +10,8 @@ namespace DB { +bool canUseCustomKey(const Settings & settings, const Cluster & cluster, const Context & context); + /// Get AST for filter created from custom_key /// replica_num is the number of the replica for which we are generating filter starting from 0 ASTPtr getCustomKeyFilterForParallelReplica( @@ -19,8 +22,6 @@ ASTPtr getCustomKeyFilterForParallelReplica( const IStorage & storage, const ContextPtr & context); -ASTPtr parseCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context); - -bool containsCustomKeyForTable(const Map & custom_keys, const DatabaseAndTableWithAlias & target, const Context & context); +ASTPtr parseCustomKeyForTable(const String & custom_keys, const Context & context); } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7e914486326..07da66e4378 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -588,9 +588,10 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ + auto parallel_replicas_mode = context->getParallelReplicasMode(); /// Parallel replicas has been requested but there is no way to sample data. /// Select all data from first replica and no data from other replicas. - if (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY + if (settings.parallel_replicas_count > 1 && parallel_replicas_mode == Context::ParallelReplicasMode::SAMPLE_KEY && !data.supportsSampling() && settings.parallel_replica_offset > 0) { LOG_DEBUG( @@ -602,7 +603,7 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( } sampling.use_sampling = relative_sample_size > 0 - || (settings.parallel_replicas_count > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::SAMPLE_KEY + || (settings.parallel_replicas_count > 1 && parallel_replicas_mode == Context::ParallelReplicasMode::SAMPLE_KEY && data.supportsSampling()); bool no_data = false; /// There is nothing left after sampling. diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 7a3452f87d4..e3996950e79 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -207,6 +207,8 @@ struct SelectQueryInfo /// /// Configured in StorageDistributed::getQueryProcessingStage() ClusterPtr optimized_cluster; + /// should we use custom key with the cluster + bool use_custom_key = false; mutable ParallelReplicasReadingCoordinatorPtr coordinator; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b088a008b46..259caaf394a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -264,12 +264,6 @@ size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & clus return (num_remote_shards + num_local_shards) * settings.max_parallel_replicas; } -bool canUseCustomKey(const Settings & settings, const Cluster & cluster) -{ - return settings.max_parallel_replicas > 1 && settings.parallel_replicas_mode == ParallelReplicasMode::CUSTOM_KEY - && cluster.getShardCount() == 1 && cluster.getShardsInfo()[0].getAllNodeCount() > 1; -} - } /// For destruction of std::unique_ptr of type that is incomplete in class definition. @@ -412,36 +406,9 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( size_t nodes = getClusterQueriedNodes(settings, cluster); - const auto use_virtual_shards = [&] - { - if (!canUseCustomKey(settings, *cluster)) - return false; - - auto distributed_table = DatabaseAndTableWithAlias( - *getTableExpression(query_info.query->as(), 0), local_context->getCurrentDatabase()); - - if (containsCustomKeyForTable(settings.parallel_replicas_custom_key, distributed_table, *local_context)) - { - LOG_INFO(log, "Found custom_key for {}", distributed_table.getQualifiedNamePrefix(false)); - return true; - } - - DatabaseAndTableWithAlias remote_table_info; - remote_table_info.database = remote_database; - remote_table_info.table = remote_table; - if (containsCustomKeyForTable(settings.parallel_replicas_custom_key, remote_table_info, *local_context)) - { - LOG_INFO(log, "Found custom_key for {}", remote_table_info.getQualifiedNamePrefix(false)); - return true; - } - - return false; - }; - - if (use_virtual_shards()) + if (query_info.use_custom_key) { LOG_INFO(log, "Single shard cluster used with custom_key, transforming replicas into virtual shards"); - query_info.cluster = cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); } else @@ -816,25 +783,9 @@ void StorageDistributed::read( auto settings = local_context->getSettingsRef(); ClusterProxy::AdditionalShardFilterGenerator additional_shard_filter_generator; - if (canUseCustomKey(settings, *getCluster())) + if (query_info.use_custom_key) { - const auto get_custom_key_ast = [&]() -> ASTPtr - { - auto distributed_table = DatabaseAndTableWithAlias( - *getTableExpression(query_info.query->as(), 0), local_context->getCurrentDatabase()); - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, distributed_table, *local_context)) - return custom_key_ast; - - DatabaseAndTableWithAlias remote_table_info; - remote_table_info.database = remote_database; - remote_table_info.table = remote_table; - if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, remote_table_info, *local_context)) - return custom_key_ast; - - return nullptr; - }; - - if (auto custom_key_ast = get_custom_key_ast()) + if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *local_context)) { if (query_info.getCluster()->getShardCount() == 1) { diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 9222f417a94..342abdcb088 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -44,11 +44,11 @@ def create_tables(cluster): def insert_data(cluster, row_num): create_tables(cluster) n1 = nodes[0] - n1.query(f"INSERT INTO dist_table SELECT number, number FROM numbers({row_num})") + n1.query(f"INSERT INTO dist_table SELECT number % 4, number FROM numbers({row_num})") n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") -@pytest.mark.parametrize("custom_key", ["sipHash64(value)", "key"]) +@pytest.mark.parametrize("custom_key", ["sipHash64(key)", "key"]) @pytest.mark.parametrize("filter_type", ["default", "range"]) @pytest.mark.parametrize( "cluster", @@ -61,21 +61,22 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter row_num = 1000 insert_data(cluster, row_num) + expected_result = "" + for i in range(4): + expected_result += f"{i}\t250\n" + n1 = nodes[0] assert ( - int( - n1.query( - "SELECT count() FROM dist_table", - settings={ - "prefer_localhost_replica": 0, - "max_parallel_replicas": 4, - "parallel_replicas_mode": "custom_key", - "parallel_replicas_custom_key": f"{{'test_table': '{custom_key}'}}", - "parallel_replicas_custom_key_filter_type": filter_type, - }, - ) + n1.query( + "SELECT key, count() FROM dist_table GROUP BY key ORDER BY key", + settings={ + "prefer_localhost_replica": 0, + "max_parallel_replicas": 4, + "parallel_replicas_custom_key": custom_key, + "parallel_replicas_custom_key_filter_type": filter_type, + }, ) - == row_num + == expected_result ) if cluster == "test_multiple_shards_multiple_replicas": @@ -89,48 +90,3 @@ def test_parallel_replicas_custom_key(start_cluster, cluster, custom_key, filter assert n1.contains_in_log( "Single shard cluster used with custom_key, transforming replicas into virtual shards" ) - - -def test_custom_key_different_table_names(start_cluster): - def run(table_source, table_name_for_custom_key, should_use_virtual_shard=True): - for node in nodes: - node.rotate_logs() - - row_num = 1000 - insert_data("test_single_shard_multiple_replicas", row_num) - - n1 = nodes[0] - assert ( - int( - n1.query( - f"SELECT count() FROM {table_source}", - settings={ - "prefer_localhost_replica": 0, - "max_parallel_replicas": 4, - "parallel_replicas_mode": "custom_key", - "parallel_replicas_custom_key": f"{{'{table_name_for_custom_key}': 'sipHash64(value)'}}", - }, - ) - ) - == row_num - ) - - # we first transform all replicas into shards and then append for each shard filter - assert not should_use_virtual_shard or n1.contains_in_log( - "Single shard cluster used with custom_key, transforming replicas into virtual shards" - ) - - run("dist_table", "dist_table") - run("dist_table as d", "d") - run("dist_table as d", "dist_table") - run("dist_table as d", "test_table") - run( - "cluster('test_single_shard_multiple_replicas', default.test_table)", - "test_table", - ) - run("cluster('test_single_shard_multiple_replicas', default.test_table) as d", "d") - run( - "cluster('test_single_shard_multiple_replicas', default.test_table) as d", - "test_table", - ) - run("dist_table as d", "dist", should_use_virtual_shard=False) diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index 795551e5dfa..901b818cbc0 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,7 +1,7 @@ -- Tags: replica, distributed +SET allow_experimental_parallel_reading_from_replicas = 0; SET max_parallel_replicas = 2; -SET parallel_replicas_mode = 'sample_key'; DROP TABLE IF EXISTS report; set allow_deprecated_syntax_for_merge_tree=1; diff --git a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql index 96d52d7e60e..86c84427297 100644 --- a/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql +++ b/tests/queries/0_stateless/01034_prewhere_max_parallel_replicas_distributed.sql @@ -1,5 +1,7 @@ -- Tags: replica, distributed +set allow_experimental_parallel_reading_from_replicas=0; + drop table if exists test_max_parallel_replicas_lr; -- If you wonder why the table is named with "_lr" suffix in this test. @@ -9,9 +11,6 @@ CREATE TABLE test_max_parallel_replicas_lr (timestamp UInt64) ENGINE = MergeTree INSERT INTO test_max_parallel_replicas_lr select number as timestamp from system.numbers limit 100; SET max_parallel_replicas = 2; -SET parallel_replicas_mode = 'sample_key'; -SET allow_experimental_parallel_reading_from_replicas = 0; - select count() FROM remote('127.0.0.{2|3}', currentDatabase(), test_max_parallel_replicas_lr) PREWHERE timestamp > 0; drop table test_max_parallel_replicas_lr; diff --git a/tests/queries/0_stateless/01034_sample_final_distributed.sql b/tests/queries/0_stateless/01034_sample_final_distributed.sql index 1b1832f7a0d..a81fef645db 100644 --- a/tests/queries/0_stateless/01034_sample_final_distributed.sql +++ b/tests/queries/0_stateless/01034_sample_final_distributed.sql @@ -1,5 +1,7 @@ -- Tags: distributed +set allow_experimental_parallel_reading_from_replicas = 0; + drop table if exists sample_final; create table sample_final (CounterID UInt32, EventDate Date, EventTime DateTime, UserID UInt64, Sign Int8) engine = CollapsingMergeTree(Sign) order by (CounterID, EventDate, intHash32(UserID), EventTime) sample by intHash32(UserID); insert into sample_final select number / (8192 * 4), toDate('2019-01-01'), toDateTime('2019-01-01 00:00:01') + number, number / (8192 * 2), number % 3 = 1 ? -1 : 1 from numbers(1000000); @@ -13,10 +15,7 @@ select count() from sample_final sample 1/2; select 'count sample final'; select count() from sample_final final sample 1/2; select 'count final max_parallel_replicas'; - -set allow_experimental_parallel_reading_from_replicas = 0; set max_parallel_replicas=2; -set parallel_replicas_mode='sample_key'; select count() from remote('127.0.0.{2|3}', currentDatabase(), sample_final) final; drop table if exists sample_final; diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql index c44c335700f..04777f5b31c 100644 --- a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -1,10 +1,11 @@ -- Tags: replica +SET allow_experimental_parallel_reading_from_replicas=0; + DROP TABLE IF EXISTS t; CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; INSERT INTO t VALUES ('Hello'); -SET parallel_replicas_mode = 'sample_key'; SET max_parallel_replicas = 3; SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); diff --git a/tests/queries/0_stateless/02221_parallel_replicas_bug.sh b/tests/queries/0_stateless/02221_parallel_replicas_bug.sh index 1a703b20d2a..cce32bf8272 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_mode='read_tasks' -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null +${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null 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 283c51cf0a2..2a163746e20 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 @@ -15,7 +15,6 @@ as select * from numbers(1); # Logical error: 'Coordinator for parallel reading from replicas is not initialized'. opts=( --allow_experimental_parallel_reading_from_replicas 1 - --parallel_replicas_mode 'read_tasks' --max_parallel_replicas 3 --iterations 1 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index 6e6bc5a3b6c..b6299de9aae 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, long, no-random-merge-tree-settings +-- Tags: no-parallel, no-random-merge-tree-settings drop table if exists pr_t; drop table if exists dist_t_different_dbs; @@ -61,7 +61,6 @@ create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a; insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); set allow_experimental_parallel_reading_from_replicas = 1; -set parallel_replicas_mode = 'read_tasks'; set max_parallel_replicas = 3; set use_hedged_requests = 0; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference index 21ed26b7579..9e4e24235aa 100644 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference @@ -23,103 +23,150 @@ filter_type='range' max_replicas=2 prefer_localhost_replica=1 Hello filter_type='range' max_replicas=3 prefer_localhost_replica=1 Hello -query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x)' +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='y' filter_type='default' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=1 -1000 -query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='y' +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='cityHash64(y)' filter_type='default' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=1 -1000 -query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + y' +0 334 +1 333 +2 333 +query='SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y' with custom_key='cityHash64(y) + 1' filter_type='default' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=0 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='default' max_replicas=3 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=1 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=2 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 filter_type='range' max_replicas=3 prefer_localhost_replica=1 -1000 -query='SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)' with custom_key='cityHash64(x) + 1' -filter_type='default' max_replicas=1 prefer_localhost_replica=0 -1000 -filter_type='default' max_replicas=2 prefer_localhost_replica=0 -1000 -filter_type='default' max_replicas=3 prefer_localhost_replica=0 -1000 -filter_type='range' max_replicas=1 prefer_localhost_replica=0 -1000 -filter_type='range' max_replicas=2 prefer_localhost_replica=0 -1000 -filter_type='range' max_replicas=3 prefer_localhost_replica=0 -1000 -filter_type='default' max_replicas=1 prefer_localhost_replica=1 -1000 -filter_type='default' max_replicas=2 prefer_localhost_replica=1 -1000 -filter_type='default' max_replicas=3 prefer_localhost_replica=1 -1000 -filter_type='range' max_replicas=1 prefer_localhost_replica=1 -1000 -filter_type='range' max_replicas=2 prefer_localhost_replica=1 -1000 -filter_type='range' max_replicas=3 prefer_localhost_replica=1 -1000 +0 334 +1 333 +2 333 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index b6e5c853dc2..314251be874 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -12,8 +12,7 @@ function run_with_custom_key { for max_replicas in {1..3}; do echo "filter_type='$filter_type' max_replicas=$max_replicas prefer_localhost_replica=$prefer_localhost_replica" query="$1 SETTINGS max_parallel_replicas=$max_replicas\ - , parallel_replicas_mode='custom_key'\ - , parallel_replicas_custom_key={'02535_custom_key': '$2'}\ + , parallel_replicas_custom_key='$2'\ , parallel_replicas_custom_key_filter_type='$filter_type'\ , prefer_localhost_replica=$prefer_localhost_replica" $CLICKHOUSE_CLIENT --query="$query" @@ -32,15 +31,14 @@ run_with_custom_key "SELECT * FROM cluster(test_cluster_one_shard_three_replicas $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" $CLICKHOUSE_CLIENT --query="CREATE TABLE 02535_custom_key (x String, y Int32) ENGINE = MergeTree ORDER BY cityHash64(x)" -$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number FROM numbers(1000)" +$CLICKHOUSE_CLIENT --query="INSERT INTO 02535_custom_key SELECT toString(number), number % 3 FROM numbers(1000)" function run_count_with_custom_key { - run_with_custom_key "SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key)" "$1" + run_with_custom_key "SELECT y, count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) GROUP BY y ORDER BY y" "$1" } -run_count_with_custom_key "cityHash64(x)" run_count_with_custom_key "y" -run_count_with_custom_key "cityHash64(x) + y" -run_count_with_custom_key "cityHash64(x) + 1" +run_count_with_custom_key "cityHash64(y)" +run_count_with_custom_key "cityHash64(y) + 1" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" From 46667d43d88d2b30acdae30b9316df385ec20369 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 3 Mar 2023 15:25:25 +0000 Subject: [PATCH 276/470] Automatic style fix --- tests/integration/test_parallel_replicas_custom_key/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_custom_key/test.py b/tests/integration/test_parallel_replicas_custom_key/test.py index 342abdcb088..baac2661506 100644 --- a/tests/integration/test_parallel_replicas_custom_key/test.py +++ b/tests/integration/test_parallel_replicas_custom_key/test.py @@ -44,7 +44,9 @@ def create_tables(cluster): def insert_data(cluster, row_num): create_tables(cluster) n1 = nodes[0] - n1.query(f"INSERT INTO dist_table SELECT number % 4, number FROM numbers({row_num})") + n1.query( + f"INSERT INTO dist_table SELECT number % 4, number FROM numbers({row_num})" + ) n1.query("SYSTEM FLUSH DISTRIBUTED dist_table") From 0f2ae721411afc2b3150bde3a4a12c5426118f6f Mon Sep 17 00:00:00 2001 From: AVMusorin Date: Thu, 2 Mar 2023 13:51:21 +0100 Subject: [PATCH 277/470] prohibit DEFAULT/EPHEMERAL/ALIAS in KafkaEngine --- .../table-engines/integrations/kafka.md | 4 ++ .../sql-reference/statements/create/table.md | 4 ++ src/Storages/ColumnsDescription.cpp | 9 ++++ src/Storages/ColumnsDescription.h | 3 ++ src/Storages/Kafka/StorageKafka.cpp | 5 ++ tests/integration/test_storage_kafka/test.py | 50 +++++++++++++++++++ 6 files changed, 75 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index ef422632d3e..e6134043b8e 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -125,6 +125,10 @@ Groups are flexible and synced on the cluster. For instance, if you have 10 topi 2. Create a table with the desired structure. 3. Create a materialized view that converts data from the engine and puts it into a previously created table. +:::info +Kafka Engine doesn't support columns with default value of type `DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS`. If you need columns with any default type, they can be added at `MATERIALIZED VIEW` level. +::: + When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. This allows you to continually receive messages from Kafka and convert them to the required format using `SELECT`. One kafka table can have as many materialized views as you like, they do not read data from the kafka table directly, but receive new records (in blocks), this way you can write to several tables with different detail level (with grouping - aggregation and without). diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 9e66afba613..ba495b0eed5 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -121,6 +121,10 @@ If the data type and default expression are defined explicitly, this expression Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions do not contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. +:::info +Kafka Engine doesn't support columns with default value of type `DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS`. If you need columns with any default type, they can be added at `MATERIALIZED VIEW` level, see [Kafka Engine](../../../engines/table-engines/integrations/kafka.md#description). +::: + ### DEFAULT `DEFAULT expr` diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index d401840eec7..fa39e304925 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -383,6 +383,15 @@ NamesAndTypesList ColumnsDescription::getEphemeral() const return ret; } +NamesAndTypesList ColumnsDescription::getWithDefaultExpression() const +{ + NamesAndTypesList ret; + for (const auto & col : columns) + if (col.default_desc.expression) + ret.emplace_back(col.name, col.type); + return ret; +} + NamesAndTypesList ColumnsDescription::getAll() const { NamesAndTypesList ret; diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 4f874f4b850..36109392ab6 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -132,6 +132,9 @@ public: NamesAndTypesList getInsertable() const; /// ordinary + ephemeral NamesAndTypesList getAliases() const; NamesAndTypesList getEphemeral() const; + // Columns with preset default expression. + // For example from `CREATE TABLE` statement + NamesAndTypesList getWithDefaultExpression() const; NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. NamesAndTypesList getAll() const; /// ordinary + materialized + aliases + ephemeral /// Returns .size0/.null/... diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7b97273d8af..2afdc0dda8a 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -959,6 +959,11 @@ void registerStorageKafka(StorageFactory & factory) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "kafka_poll_max_batch_size can not be lower than 1"); } + if (args.columns.getOrdinary() != args.columns.getAll() || !args.columns.getWithDefaultExpression().empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS expressions for columns. " + "See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration"); + } return std::make_shared(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name); }; diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9f617369859..51952ac1eb7 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -285,6 +285,56 @@ def avro_confluent_message(schema_registry_client, value): # Tests +def test_kafka_prohibited_column_types(kafka_cluster): + def assert_returned_exception(e): + assert e.value.returncode == 36 + assert ( + "KafkaEngine doesn't support DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS expressions for columns." + in str(e.value) + ) + + # check column with DEFAULT expression + with pytest.raises(QueryRuntimeException) as exception: + instance.query( + """ + CREATE TABLE test.kafka (a Int, b Int DEFAULT 0) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') + """ + ) + assert_returned_exception(exception) + + # check EPHEMERAL + with pytest.raises(QueryRuntimeException) as exception: + instance.query( + """ + CREATE TABLE test.kafka (a Int, b Int EPHEMERAL) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') + """ + ) + assert_returned_exception(exception) + + # check ALIAS + with pytest.raises(QueryRuntimeException) as exception: + instance.query( + """ + CREATE TABLE test.kafka (a Int, b String Alias toString(a)) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') + """ + ) + assert_returned_exception(exception) + + # check MATERIALIZED + # check ALIAS + with pytest.raises(QueryRuntimeException) as exception: + instance.query( + """ + CREATE TABLE test.kafka (a Int, b String MATERIALIZED toString(a)) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_new}', '{kafka_group_name_new}', '{kafka_format_json_each_row}', '\\n') + """ + ) + assert_returned_exception(exception) + + def test_kafka_settings_old_syntax(kafka_cluster): assert TSV( instance.query( From 5403360924ef984e8e8d7cb517f71ab2e6ea0f5a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Mar 2023 16:25:42 +0100 Subject: [PATCH 278/470] Add real client (initiator server) address into the logs for interserver mode It is useful to understand what is going on, in some obscure cases, for instance if someone will copy configuration from the production to some docker env, and then you will see docker's private network addresses in the logs. Signed-off-by: Azat Khuzhin --- src/Server/TCPHandler.cpp | 29 +++++++++++++++++++---------- src/Server/TCPHandler.h | 2 ++ 2 files changed, 21 insertions(+), 10 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a307b472a64..f2b4284613a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1208,14 +1208,7 @@ void TCPHandler::receiveHello() session = makeSession(); auto & client_info = session->getClientInfo(); - - /// Extract the last entry from comma separated list of forwarded_for addresses. - /// Only the last proxy can be trusted (if any). - String forwarded_address = client_info.getLastForwardedFor(); - if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) - session->authenticate(user, password, Poco::Net::SocketAddress(forwarded_address, socket().peerAddress().port())); - else - session->authenticate(user, password, socket().peerAddress()); + session->authenticate(user, password, getClientAddress(client_info)); } void TCPHandler::receiveAddendum() @@ -1501,11 +1494,16 @@ void TCPHandler::receiveQuery() /// so we should not rely on that. However, in this particular case we got client_info from other clickhouse-server, so it's ok. if (client_info.initial_user.empty()) { - LOG_DEBUG(log, "User (no user, interserver mode)"); + LOG_DEBUG(log, "User (no user, interserver mode) (client: {})", getClientAddress(client_info).toString()); } else { - LOG_DEBUG(log, "User (initial, interserver mode): {}", client_info.initial_user); + LOG_DEBUG(log, "User (initial, interserver mode): {} (client: {})", client_info.initial_user, getClientAddress(client_info).toString()); + /// In case of inter-server mode authorization is done with the + /// initial address of the client, not the real address from which + /// the query was come, since the real address is the address of + /// the initiator server, while we are interested in client's + /// address. session->authenticate(AlwaysAllowCredentials{client_info.initial_user}, client_info.initial_address); } #else @@ -1991,4 +1989,15 @@ void TCPHandler::run() } } +Poco::Net::SocketAddress TCPHandler::getClientAddress(const ClientInfo & client_info) +{ + /// Extract the last entry from comma separated list of forwarded_for addresses. + /// Only the last proxy can be trusted (if any). + String forwarded_address = client_info.getLastForwardedFor(); + if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) + return Poco::Net::SocketAddress(forwarded_address, socket().peerAddress().port()); + else + return socket().peerAddress(); +} + } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index f06b0b060b3..e3673b213d5 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -273,6 +273,8 @@ private: /// This function is called from different threads. void updateProgress(const Progress & value); + + Poco::Net::SocketAddress getClientAddress(const ClientInfo & client_info); }; } From b5771016f6d53f460b635f10538df60d575dac23 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 3 Mar 2023 15:46:41 +0000 Subject: [PATCH 279/470] Fix 01019_alter_materialized_view_consistent --- .../01019_alter_materialized_view_consistent.sh | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh index e90085f4e8e..496215ba492 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh @@ -50,12 +50,19 @@ function insert_thread() { function alter_thread() { trap 'exit' INT - ALTER[0]="ALTER TABLE mv MODIFY QUERY SELECT v == 1 as test, v as case FROM src_a;" - ALTER[1]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + # Generate random ALTERs, but make sure that at least one of them is for each source table. + for i in {0..5}; do + ALTER[$i]="ALTER TABLE mv MODIFY QUERY SELECT v == 1 as test, v as case FROM src_a;" + done + ALTER[$RANDOM % 3]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" + i=0 while true; do - $CLICKHOUSE_CLIENT --allow_experimental_alter_materialized_view_structure=1 \ - -q "${ALTER[$RANDOM % 2]}" + $CLICKHOUSE_CLIENT --allow_experimental_alter_materialized_view_structure=1 -q "${ALTER[$i % 6]}" + ((i=i+1)) + sleep "0.0$RANDOM" is_done=$($CLICKHOUSE_CLIENT -q "SELECT countIf(case = 1) > 0 AND countIf(case = 2) > 0 FROM mv;") From 3f3091cca5536e30b955bb9d40a053ff9f9abe76 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 3 Mar 2023 19:55:44 +0400 Subject: [PATCH 280/470] Markup fixed at the page `/docs/ru/interfaces/third-party/client-libraries.md` --- docs/ru/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index f55bbe2a47d..a4659e9ac4e 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -8,6 +8,7 @@ sidebar_label: "Клиентские библиотеки от сторонни :::danger "Disclaimer" Яндекс не поддерживает перечисленные ниже библиотеки и не проводит тщательного тестирования для проверки их качества. +::: - Python: - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) From a0f5756751a6cec5c03a56a5f1552113ffb57cfd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 3 Mar 2023 16:28:41 +0000 Subject: [PATCH 281/470] Fix DelayedJoinedBlocksWorkerTransform for grace hash join --- src/Interpreters/GraceHashJoin.cpp | 5 +--- src/Interpreters/GraceHashJoin.h | 8 ++---- .../Transforms/JoiningTransform.cpp | 27 +++++++++++++++++-- .../25340_grace_hash_limit_race.reference | 1 + .../25340_grace_hash_limit_race.sql | 4 +-- 5 files changed, 31 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 1b62939ac7b..7795061072c 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -410,8 +409,6 @@ void GraceHashJoin::initialize(const Block & sample_block) void GraceHashJoin::joinBlock(Block & block, std::shared_ptr & not_processed) { - std::shared_lock current_bucket_lock(current_bucket_mutex); - if (block.rows() == 0) { hash_join->joinBlock(block, not_processed); @@ -551,7 +548,7 @@ public: IBlocksStreamPtr GraceHashJoin::getDelayedBlocks() { - std::unique_lock current_bucket_lock(current_bucket_mutex); + std::lock_guard current_bucket_lock(current_bucket_mutex); if (current_bucket == nullptr) return nullptr; diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index 0d6c4741b95..cbc0f2f3266 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -140,12 +140,8 @@ private: FileBucket * current_bucket = nullptr; - /* Function `getDelayedBlocks` should be a critical section. - * Also some `joinBlock` calls may be in progress and we need to wait for them to finish, - * because they may may use `hash_join`, but `getDelayedBlocks` switches it to another bucket. - * So, `joinBlock` acquires shared lock and getDelayedBlocks acquires exclusive lock. - */ - mutable SharedMutex current_bucket_mutex; + /// Function `getDelayedBlocks` is a critical section, we process only one bucket at a time. + mutable std::mutex current_bucket_mutex; InMemoryJoinPtr hash_join; Block hash_join_sample_block; diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index c28a84e9d5d..bf80de67b42 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -318,12 +318,22 @@ DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block out IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() { + auto & output = outputs.front(); + auto & input = inputs.front(); + + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + + if (!output.canPush()) + return Status::PortFull; + if (inputs.size() != 1 && outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have exactly one input port"); - auto & output = outputs.front(); - auto & input = inputs.front(); if (output_chunk) { @@ -397,6 +407,9 @@ DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, J void DelayedJoinedBlocksTransform::work() { + if (finished) + return; + delayed_blocks = join->getDelayedBlocks(); finished = finished || delayed_blocks == nullptr; } @@ -406,6 +419,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() { for (auto & output : outputs) { + if (output.isFinished()) + { + /// If at least one output is finished, then we have read all data from buckets. + /// Some workers can still can busy with joining the last chunk of data in memory, + /// but after that they also will finish when they will try to get next chunk. + finished = true; + continue; + } if (!output.canPush()) return Status::PortFull; } @@ -414,6 +435,8 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() { for (auto & output : outputs) { + if (output.isFinished()) + continue; Chunk chunk; chunk.setChunkInfo(std::make_shared()); output.push(std::move(chunk)); diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.reference b/tests/queries/0_stateless/25340_grace_hash_limit_race.reference index e69de29bb2d..83b33d238da 100644 --- a/tests/queries/0_stateless/25340_grace_hash_limit_race.reference +++ b/tests/queries/0_stateless/25340_grace_hash_limit_race.reference @@ -0,0 +1 @@ +1000 diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql index b12dd7bdbba..55262ab2455 100644 --- a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql +++ b/tests/queries/0_stateless/25340_grace_hash_limit_race.sql @@ -6,11 +6,11 @@ INSERT INTO test_grace_hash SELECT number, number % 100 = 0 FROM numbers(100000) SET join_algorithm = 'grace_hash'; -SELECT * FROM ( +SELECT count() FROM ( SELECT f.id FROM test_grace_hash AS f LEFT JOIN test_grace_hash AS d ON f.id = d.id LIMIT 1000 -) FORMAT Null; +); DROP TABLE test_grace_hash; From 9389cc29da406533ff589fde058a2f22fc1e9aa6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 3 Mar 2023 16:48:47 +0000 Subject: [PATCH 282/470] better mutations of sparse columns --- .../Serializations/SerializationInfo.cpp | 32 +++++++++++++++++-- .../Serializations/SerializationInfo.h | 6 +++- .../Serializations/SerializationInfoTuple.cpp | 18 ++++++++--- .../Serializations/SerializationInfoTuple.h | 6 +++- src/Storages/MergeTree/MutateTask.cpp | 16 +++------- ...02662_sparse_columns_mutations_4.reference | 2 ++ .../02662_sparse_columns_mutations_4.sql | 21 ++++++++++++ ...02662_sparse_columns_mutations_5.reference | 2 ++ .../02662_sparse_columns_mutations_5.sql | 21 ++++++++++++ 9 files changed, 103 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_4.reference create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_4.sql create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_5.reference create mode 100644 tests/queries/0_stateless/02662_sparse_columns_mutations_5.sql diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index af3330d867d..4e5790ad58d 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -97,11 +97,37 @@ MutableSerializationInfoPtr SerializationInfo::clone() const return std::make_shared(kind, settings, data); } -std::shared_ptr SerializationInfo::createWithType(const IDataType & type, const Settings & new_settings) const +/// Returns true if all rows with default values of type 'lhs' +/// are mapped to default values of type 'rhs' after conversion. +static bool preserveDefaultsAfterConversion(const IDataType & lhs, const IDataType & rhs) +{ + if (lhs.equals(rhs)) + return true; + + bool lhs_is_columned_as_numeric = isColumnedAsNumber(lhs) || isColumnedAsDecimal(lhs); + bool rhs_is_columned_as_numeric = isColumnedAsNumber(rhs) || isColumnedAsDecimal(rhs); + + if (lhs_is_columned_as_numeric && rhs_is_columned_as_numeric) + return true; + + if (isStringOrFixedString(lhs) && isStringOrFixedString(rhs)) + return true; + + return false; +} + +std::shared_ptr SerializationInfo::createWithType( + const IDataType & old_type, + const IDataType & new_type, + const Settings & new_settings) const { auto new_kind = kind; - if (new_kind == ISerialization::Kind::SPARSE && !type.supportsSparseSerialization()) - new_kind = ISerialization::Kind::DEFAULT; + if (new_kind == ISerialization::Kind::SPARSE) + { + if (!new_type.supportsSparseSerialization() + || !preserveDefaultsAfterConversion(old_type, new_type)) + new_kind = ISerialization::Kind::DEFAULT; + } return std::make_shared(new_kind, new_settings); } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 560156980db..5b802b379e1 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -60,7 +60,11 @@ public: virtual void replaceData(const SerializationInfo & other); virtual std::shared_ptr clone() const; - virtual std::shared_ptr createWithType(const IDataType & type, const Settings & new_settings) const; + + virtual std::shared_ptr createWithType( + const IDataType & old_type, + const IDataType & new_type, + const Settings & new_settings) const; virtual void serialializeKindBinary(WriteBuffer & out) const; virtual void deserializeFromKindsBinary(ReadBuffer & in); diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 5724dd7a5c3..d36668f03b6 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -97,16 +97,24 @@ MutableSerializationInfoPtr SerializationInfoTuple::clone() const return std::make_shared(std::move(elems_cloned), names, settings); } -MutableSerializationInfoPtr SerializationInfoTuple::createWithType(const IDataType & type, const Settings & new_settings) const +MutableSerializationInfoPtr SerializationInfoTuple::createWithType( + const IDataType & old_type, + const IDataType & new_type, + const Settings & new_settings) const { - const auto & type_tuple = assert_cast(type); - const auto & tuple_elements = type_tuple.getElements(); - assert(elems.size() == tuple_elements.size()); + const auto & old_tuple = assert_cast(old_type); + const auto & new_tuple = assert_cast(new_type); + + const auto & old_elements = old_tuple.getElements(); + const auto & new_elements = new_tuple.getElements(); + + assert(elems.size() == old_elements.size()); + assert(elems.size() == new_elements.size()); MutableSerializationInfos infos; infos.reserve(elems.size()); for (size_t i = 0; i < elems.size(); ++i) - infos.push_back(elems[i]->createWithType(*tuple_elements[i], new_settings)); + infos.push_back(elems[i]->createWithType(*old_elements[i], *new_elements[i], new_settings)); return std::make_shared(std::move(infos), names, new_settings); } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.h b/src/DataTypes/Serializations/SerializationInfoTuple.h index 4bcd14d34cb..a9f3bdb6c6e 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.h +++ b/src/DataTypes/Serializations/SerializationInfoTuple.h @@ -19,7 +19,11 @@ public: void replaceData(const SerializationInfo & other) override; MutableSerializationInfoPtr clone() const override; - MutableSerializationInfoPtr createWithType(const IDataType & type, const Settings & new_settings) const override; + + MutableSerializationInfoPtr createWithType( + const IDataType & old_type, + const IDataType & new_type, + const Settings & new_settings) const override; void serialializeKindBinary(WriteBuffer & out) const override; void deserializeFromKindsBinary(ReadBuffer & in) override; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 1239befdd67..94715822e52 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -219,18 +219,18 @@ getColumnsForNewDataPart( continue; } - auto old_type = part_columns.getPhysical(name); + auto old_type = part_columns.getPhysical(name).type; auto new_type = updated_header.getByName(new_name).type; - if (!new_type->supportsSparseSerialization()) - continue; - SerializationInfo::Settings settings { .ratio_of_defaults_for_sparse = source_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization, .choose_kind = false }; + if (!new_type->supportsSparseSerialization() || settings.isAlwaysDefault()) + continue; + auto new_info = new_type->createSerializationInfo(settings); if (!old_info->structureEquals(*new_info)) { @@ -238,13 +238,7 @@ getColumnsForNewDataPart( continue; } - if (!old_info->hasCustomSerialization()) - { - new_serialization_infos.emplace(new_name, old_info); - continue; - } - - new_info = old_info->createWithType(*new_type, settings); + new_info = old_info->createWithType(*old_type, *new_type, settings); new_serialization_infos.emplace(new_name, std::move(new_info)); } diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_4.reference b/tests/queries/0_stateless/02662_sparse_columns_mutations_4.reference new file mode 100644 index 00000000000..2e24ab44f9a --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_4.reference @@ -0,0 +1,2 @@ +UInt64 Sparse +String Default diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_4.sql b/tests/queries/0_stateless/02662_sparse_columns_mutations_4.sql new file mode 100644 index 00000000000..039af658489 --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_4.sql @@ -0,0 +1,21 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_mutations_4; + +CREATE TABLE t_sparse_mutations_4 (k UInt64, v UInt64) +ENGINE = MergeTree ORDER BY k +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_mutations_4 SELECT number, 0 FROM numbers(10000); + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_4' AND column = 'v' AND active +ORDER BY name; + +ALTER TABLE t_sparse_mutations_4 MODIFY COLUMN v String; + +SELECT type, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_4' AND column = 'v' AND active +ORDER BY name; + +DROP TABLE t_sparse_mutations_4; diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_5.reference b/tests/queries/0_stateless/02662_sparse_columns_mutations_5.reference new file mode 100644 index 00000000000..698d61cbb24 --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_5.reference @@ -0,0 +1,2 @@ +Tuple(UInt64, UInt64) Default ['1','2'] ['UInt64','UInt64'] ['Sparse','Sparse'] +Tuple(UInt64, String) Default ['1','2'] ['UInt64','String'] ['Sparse','Default'] diff --git a/tests/queries/0_stateless/02662_sparse_columns_mutations_5.sql b/tests/queries/0_stateless/02662_sparse_columns_mutations_5.sql new file mode 100644 index 00000000000..79bac836bdc --- /dev/null +++ b/tests/queries/0_stateless/02662_sparse_columns_mutations_5.sql @@ -0,0 +1,21 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_mutations_5; + +CREATE TABLE t_sparse_mutations_5 (k UInt64, t Tuple(UInt64, UInt64)) +ENGINE = MergeTree ORDER BY k +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_mutations_5 SELECT number, (0, 0) FROM numbers(10000); + +SELECT type, serialization_kind, subcolumns.names, subcolumns.types, subcolumns.serializations FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_5' AND column = 't' AND active +ORDER BY name; + +ALTER TABLE t_sparse_mutations_5 MODIFY COLUMN t Tuple(UInt64, String); + +SELECT type, serialization_kind, subcolumns.names, subcolumns.types, subcolumns.serializations FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_mutations_5' AND column = 't' AND active +ORDER BY name; + +DROP TABLE t_sparse_mutations_5; From bbbe529e249adb3847f45571b83086fc805c6071 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Fri, 3 Mar 2023 09:29:35 -0800 Subject: [PATCH 283/470] Fix farmhash for s390x --- contrib/libfarmhash/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/libfarmhash/CMakeLists.txt b/contrib/libfarmhash/CMakeLists.txt index a0533a93f17..436bc3d0108 100644 --- a/contrib/libfarmhash/CMakeLists.txt +++ b/contrib/libfarmhash/CMakeLists.txt @@ -6,6 +6,10 @@ if (MSVC) target_compile_definitions (_farmhash PRIVATE FARMHASH_NO_BUILTIN_EXPECT=1) endif () +if (ARCH_S390X) + add_compile_definitions(WORDS_BIGENDIAN) +endif () + target_include_directories (_farmhash BEFORE PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}) add_library(ch_contrib::farmhash ALIAS _farmhash) From 6c929737eb4d5c62377927889988d5e5dda9ba25 Mon Sep 17 00:00:00 2001 From: Dave Lahn Date: Fri, 3 Mar 2023 18:42:10 +0000 Subject: [PATCH 284/470] Add missing policy name element The S3 cache example had a list of policies, but was missing the policy name. This caused this configuration to fail. --- docs/en/operations/storing-data.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 203fe4e42d2..cba666f73da 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -135,11 +135,13 @@ Example of configuration for versions later or equal to 22.8: - -
- cache -
-
+ + +
+ cache +
+
+
``` From 7997df5739e509132a596ec4782b39e6e389970c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Mar 2023 19:50:21 +0100 Subject: [PATCH 285/470] Update storing-data.md --- docs/en/operations/storing-data.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index cba666f73da..3f9a0f67187 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -161,11 +161,13 @@ Example of configuration for versions earlier than 22.8:
- -
- s3 -
-
+ + +
+ s3 +
+
+
``` From 5c4da5aa4a08ea2fe1cfa279fc1ec8c4e6e7f68f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 3 Mar 2023 20:05:42 +0100 Subject: [PATCH 286/470] Use separate thread pool for IO operations for backups (#47174) --- programs/server/Server.cpp | 6 ++++++ src/Backups/BackupIO_S3.cpp | 8 ++++---- src/Core/ServerSettings.h | 4 +++- src/IO/BackupIOThreadPool.cpp | 34 ++++++++++++++++++++++++++++++++++ src/IO/BackupsIOThreadPool.h | 20 ++++++++++++++++++++ 5 files changed, 67 insertions(+), 5 deletions(-) create mode 100644 src/IO/BackupIOThreadPool.cpp create mode 100644 src/IO/BackupsIOThreadPool.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1486a51c710..57d361886d2 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -773,6 +774,11 @@ try server_settings.max_io_thread_pool_free_size, server_settings.io_thread_pool_queue_size); + BackupsIOThreadPool::initialize( + server_settings.max_backups_io_thread_pool_size, + server_settings.max_backups_io_thread_pool_free_size, + server_settings.backups_io_thread_pool_queue_size); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index a303a0bc2d5..2f315e8d488 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -167,7 +167,7 @@ void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_na std::string src_bucket = object_storage->getObjectsNamespace(); auto file_path = fs::path(s3_uri.key) / dest_file_name; copyS3File(client, src_bucket, objects[0].absolute_path, src_offset, src_size, s3_uri.bucket, file_path, request_settings, {}, - threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); } } @@ -175,7 +175,7 @@ void BackupWriterS3::copyDataToFile( const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name) { copyDataToS3File(create_read_buffer, offset, size, client, s3_uri.bucket, fs::path(s3_uri.key) / dest_file_name, request_settings, {}, - threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); } BackupWriterS3::~BackupWriterS3() = default; @@ -222,7 +222,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) request_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, - threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); + threadPoolCallbackRunner(BackupsIOThreadPool::get(), "BackupWriterS3")); } void BackupWriterS3::removeFile(const String & file_name) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1e884266c17..abc5b4d14d8 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -21,6 +21,9 @@ namespace DB M(UInt64, max_io_thread_pool_size, 100, "The maximum number of threads that would be used for IO operations", 0) \ M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ + M(UInt64, max_backups_io_thread_pool_size, 1000, "The maximum number of threads that would be used for IO operations for BACKUP queries", 0) \ + M(UInt64, max_backups_io_thread_pool_free_size, 0, "Max free size for backups IO thread pool.", 0) \ + M(UInt64, backups_io_thread_pool_queue_size, 0, "Queue size for backups IO thread pool.", 0) \ M(Int32, max_connections, 1024, "Max server connections.", 0) \ M(UInt32, asynchronous_metrics_update_period_s, 1, "Period in seconds for updating asynchronous metrics.", 0) \ M(UInt32, asynchronous_heavy_metrics_update_period_s, 120, "Period in seconds for updating asynchronous metrics.", 0) \ @@ -75,4 +78,3 @@ struct ServerSettings : public BaseSettings }; } - diff --git a/src/IO/BackupIOThreadPool.cpp b/src/IO/BackupIOThreadPool.cpp new file mode 100644 index 00000000000..067fc54b1ae --- /dev/null +++ b/src/IO/BackupIOThreadPool.cpp @@ -0,0 +1,34 @@ +#include +#include "Core/Field.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +std::unique_ptr BackupsIOThreadPool::instance; + +void BackupsIOThreadPool::initialize(size_t max_threads, size_t max_free_threads, size_t queue_size) +{ + if (instance) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is initialized twice"); + } + + instance = std::make_unique(max_threads, max_free_threads, queue_size, false /*shutdown_on_exception*/); +} + +ThreadPool & BackupsIOThreadPool::get() +{ + if (!instance) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "The BackupsIO thread pool is not initialized"); + } + + return *instance; +} + +} diff --git a/src/IO/BackupsIOThreadPool.h b/src/IO/BackupsIOThreadPool.h new file mode 100644 index 00000000000..f5aae5741a8 --- /dev/null +++ b/src/IO/BackupsIOThreadPool.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +/* + * ThreadPool used for the Backup IO. + */ +class BackupsIOThreadPool +{ + static std::unique_ptr instance; + +public: + static void initialize(size_t max_threads, size_t max_free_threads, size_t queue_size); + static ThreadPool & get(); +}; + +} From ea8ae193d3ae40215493e450e34dc4a188a981b4 Mon Sep 17 00:00:00 2001 From: liding1992 Date: Thu, 2 Mar 2023 17:07:10 +0800 Subject: [PATCH 287/470] Fix the problem that the 'ReplicatedMergeTree' table failed to insert two similar data when the 'part_type' is configured as 'InMemory' mode. --- src/Columns/ColumnString.h | 4 ++-- ...cated_merge_tree_insert_zookeeper_long.reference | 1 + ..._replicated_merge_tree_insert_zookeeper_long.sql | 13 +++++++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.reference create mode 100644 tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index aa251b1fda0..fc2ac2d2385 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -187,8 +187,8 @@ public: void updateHashFast(SipHash & hash) const override { - hash.update(reinterpret_cast(offsets.data()), size() * sizeof(offsets[0])); - hash.update(reinterpret_cast(chars.data()), size() * sizeof(chars[0])); + hash.update(reinterpret_cast(offsets.data()), offsets.size() * sizeof(offsets[0])); + hash.update(reinterpret_cast(chars.data()), chars.size() * sizeof(chars[0])); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.reference b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql new file mode 100644 index 00000000000..df07d2e2faf --- /dev/null +++ b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS inmemory_test; + +CREATE TABLE inmemory_test (d Date, id String) +ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/inmemory_test', 'r1') +PARTITION BY toYYYYMMDD(d) ORDER BY (d, id) +SETTINGS min_rows_for_compact_part = 10, index_granularity = 8192; + +INSERT INTO inmemory_test(d, id) VALUES('2023-01-01', 'abcdefghijklmnopqrstuvwxyz'); +INSERT INTO inmemory_test(d, id) VALUES('2023-01-01', 'a1234567890123456789012345'); + +SELECT COUNT(1) FROM inmemory_test; + +DROP TABLE inmemory_test; From b1e8b74b9cd81f6c8296b63515f91c893530b353 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Fri, 3 Mar 2023 15:54:41 +0100 Subject: [PATCH 288/470] Update 02675_replicated_merge_tree_insert_zookeeper_long.sql --- .../02675_replicated_merge_tree_insert_zookeeper_long.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql index df07d2e2faf..194ea9bfcc1 100644 --- a/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql +++ b/tests/queries/0_stateless/02675_replicated_merge_tree_insert_zookeeper_long.sql @@ -1,3 +1,5 @@ +-- Tags: no-s3-storage + DROP TABLE IF EXISTS inmemory_test; CREATE TABLE inmemory_test (d Date, id String) From a70789c0b3ecf5384f0cd22fb214b24c9f6d5038 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 4 Mar 2023 09:15:33 +0100 Subject: [PATCH 289/470] Whitespace --- src/Backups/BackupCoordinationRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 8c696057755..2ca06b76dc1 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -604,7 +604,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) return false; auto zk = getZooKeeper(); - std::string backup_stage_path = zookeeper_path +"/stage"; + std::string backup_stage_path = zookeeper_path + "/stage"; if (!zk->exists(root_zookeeper_path)) zk->createAncestors(root_zookeeper_path); From 694f5eb0a830bd96a8f95e51a042607e418df330 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 4 Mar 2023 12:10:27 +0300 Subject: [PATCH 290/470] Update ProcessList.cpp --- src/Interpreters/ProcessList.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 6f5553f58fd..37cb9ee599f 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -362,9 +362,11 @@ QueryStatus::QueryStatus( QueryStatus::~QueryStatus() { +#if !defined(NDEBUG) /// Check that all executors were invalidated. for (const auto & e : executors) assert(!e->executor); +#endif if (auto * memory_tracker = getMemoryTracker()) { From 7c2ccef1b9f59c64d70e5cca30deb94f0e90807b Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sat, 4 Mar 2023 17:00:18 +0400 Subject: [PATCH 291/470] Markup fixed at the page `/docs/ru/sql-reference/operators/index.md` --- docs/ru/sql-reference/operators/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 57c426cb5ad..b5fec3cb38c 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -263,6 +263,7 @@ SELECT toDateTime('2014-10-26 00:00:00', 'Europe/Moscow') AS time, time + 60 * 6 │ 2014-10-26 00:00:00 │ 2014-10-26 23:00:00 │ 2014-10-27 00:00:00 │ └─────────────────────┴─────────────────────┴─────────────────────┘ ``` +::: **Смотрите также** From 1d7bb4e83b405063ab74e6375aed4126a5edd690 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sat, 4 Mar 2023 17:02:03 +0400 Subject: [PATCH 292/470] Markup fixed at the page `/docs/ru/sql-reference/operators/exists.md` --- docs/ru/sql-reference/operators/exists.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/operators/exists.md b/docs/ru/sql-reference/operators/exists.md index 3fc085fe021..38855abbcf3 100644 --- a/docs/ru/sql-reference/operators/exists.md +++ b/docs/ru/sql-reference/operators/exists.md @@ -8,7 +8,8 @@ slug: /ru/sql-reference/operators/exists `EXISTS` может быть использован в секции [WHERE](../../sql-reference/statements/select/where.md). :::danger "Предупреждение" - Ссылки на таблицы или столбцы основного запроса не поддерживаются в подзапросе. + Ссылки на таблицы или столбцы основного запроса не поддерживаются в подзапросе. +::: **Синтаксис** From 97cbc7e7fe52cf11d6a431a9311ea94a7e1a90fe Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sat, 4 Mar 2023 17:20:54 +0400 Subject: [PATCH 293/470] Typos fixed at the page `/docs/ru/sql-reference/operators/in.md` --- docs/ru/sql-reference/operators/in.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index fa679b890a7..60400fb2b31 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -38,9 +38,9 @@ SELECT '1' IN (SELECT 1); └──────────────────────┘ ``` -Если в качестве правой части оператора указано имя таблицы (например, `UserID IN users`), то это эквивалентно подзапросу `UserID IN (SELECT * FROM users)`. Это используется при работе с внешними данными, отправляемым вместе с запросом. Например, вместе с запросом может быть отправлено множество идентификаторов посетителей, загруженное во временную таблицу users, по которому следует выполнить фильтрацию. +Если в качестве правой части оператора указано имя таблицы (например, `UserID IN users`), то это эквивалентно подзапросу `UserID IN (SELECT * FROM users)`. Это используется при работе с внешними данными, отправляемыми вместе с запросом. Например, вместе с запросом может быть отправлено множество идентификаторов посетителей, загруженное во временную таблицу users, по которому следует выполнить фильтрацию. -Если в качестве правой части оператора, указано имя таблицы, имеющий движок Set (подготовленное множество, постоянно находящееся в оперативке), то множество не будет создаваться заново при каждом запросе. +Если в качестве правой части оператора, указано имя таблицы, имеющей движок Set (подготовленное множество, постоянно находящееся в оперативке), то множество не будет создаваться заново при каждом запросе. В подзапросе может быть указано более одного столбца для фильтрации кортежей. Пример: @@ -49,9 +49,9 @@ SELECT '1' IN (SELECT 1); SELECT (CounterID, UserID) IN (SELECT CounterID, UserID FROM ...) FROM ... ``` -Типы столбцов слева и справа оператора IN, должны совпадать. +Типы столбцов слева и справа оператора IN должны совпадать. -Оператор IN и подзапрос могут встречаться в любой части запроса, в том числе в агрегатных и лямбда функциях. +Оператор IN и подзапрос могут встречаться в любой части запроса, в том числе в агрегатных и лямбда-функциях. Пример: ``` sql @@ -122,7 +122,7 @@ FROM t_null Существует два варианта IN-ов с подзапросами (аналогично для JOIN-ов): обычный `IN` / `JOIN` и `GLOBAL IN` / `GLOBAL JOIN`. Они отличаются способом выполнения при распределённой обработке запроса. -:::note "Attention" +:::note "Внимание" Помните, что алгоритмы, описанные ниже, могут работать иначе в зависимости от [настройки](../../operations/settings/settings.md) `distributed_product_mode`. ::: При использовании обычного IN-а, запрос отправляется на удалённые серверы, и на каждом из них выполняются подзапросы в секциях `IN` / `JOIN`. @@ -228,7 +228,7 @@ SELECT CounterID, count() FROM distributed_table_1 WHERE UserID IN (SELECT UserI SETTINGS max_parallel_replicas=3 ``` -преобразуются на каждом сервере в +преобразуется на каждом сервере в ```sql SELECT CounterID, count() FROM local_table_1 WHERE UserID IN (SELECT UserID FROM local_table_2 WHERE CounterID < 100) From 849fac672ade1bfc668d785ff3875b5a9ad09156 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 4 Mar 2023 16:31:18 +0300 Subject: [PATCH 294/470] Update 02585_query_status_deadlock.sh --- tests/queries/0_stateless/02585_query_status_deadlock.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02585_query_status_deadlock.sh b/tests/queries/0_stateless/02585_query_status_deadlock.sh index 92dd05ef46c..227ecb1c1b2 100755 --- a/tests/queries/0_stateless/02585_query_status_deadlock.sh +++ b/tests/queries/0_stateless/02585_query_status_deadlock.sh @@ -10,6 +10,7 @@ $CLICKHOUSE_CLIENT --query_id="$QUERY_ID" -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 & +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" while true do From 11930ad68503c30f8c6a91e5894c205e364b9414 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 4 Mar 2023 17:27:47 +0000 Subject: [PATCH 295/470] Docs: Update index support of has(), hasAny(), hasAll() --- .../mergetree-family/mergetree.md | 45 ++++++++++--------- .../MergeTree/MergeTreeIndexFullText.cpp | 4 +- 2 files changed, 25 insertions(+), 24 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7c0416d3ea4..fc8060077b0 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -450,29 +450,32 @@ INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY Conditions in the `WHERE` clause contains calls of the functions that operate with columns. If the column is a part of an index, ClickHouse tries to use this index when performing the functions. ClickHouse supports different subsets of functions for using indexes. -The `set` index can be used with all functions. Function subsets for other indexes are shown in the table below. +Indexes of type `set` can be utilized by all functions. The other index types are supported as follows: | Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | -|------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| -| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | -| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenCaseInsensitive | ✗ | ✗ | ✗ | ✔ | ✗ | -| hasTokenCaseInsensitiveOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | +|------------------------------------------------------------------------------------------------------------|-------------|--------|------------|------------|--------------| +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | +| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [has](/docs/en/sql-reference/functions/array-functions#function-has) | ✗ | ✗ | ✔ | ✔ | ✔ | +| [hasAny](/docs/en/sql-reference/functions/array-functions#function-hasAny) | ✗ | ✗ | ✗ | ✗ | ✔ | +| [hasAll](/docs/en/sql-reference/functions/array-functions#function-hasAll) | ✗ | ✗ | ✗ | ✗ | ✔ | +| hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasTokenOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasTokenCaseInsensitive | ✗ | ✗ | ✗ | ✔ | ✗ | +| hasTokenCaseInsensitiveOrNull | ✗ | ✗ | ✗ | ✔ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 80c4c201c3f..fa1bd36f863 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -748,9 +748,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) if (!data_type.isString() && !data_type.isFixedString()) throw Exception(ErrorCodes::INCORRECT_QUERY, - "Bloom filter index can be used only with `String`, `FixedString`, " - "`LowCardinality(String)`, `LowCardinality(FixedString)` column " - "or Array with `String` or `FixedString` values column."); + "Ngram and token bloom filter indexes can only be used with column types `String`, `FixedString`, `LowCardinality(String)`, `LowCardinality(FixedString)`, `Array(String)` or `Array(FixedString)`"); } if (index.type == NgramTokenExtractor::getName()) From 08ce7b031e95a6c61e965e07deb26c23174e7a95 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 4 Mar 2023 16:48:04 +0100 Subject: [PATCH 296/470] release shared ptrs after finishing a transaction --- src/Interpreters/MergeTreeTransaction.cpp | 14 ++++++++++++++ src/Interpreters/MergeTreeTransaction.h | 4 ++++ src/Interpreters/TransactionLog.cpp | 6 ++++-- src/Interpreters/TransactionLog.h | 2 +- 4 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index f16ece46530..50ecb061752 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -315,6 +315,15 @@ bool MergeTreeTransaction::rollback() noexcept return true; } +void MergeTreeTransaction::afterFinalize() +{ + std::lock_guard lock{mutex}; + /// Release shared pointers just in case + storages.clear(); + mutations.clear(); + finalized = true; +} + void MergeTreeTransaction::onException() { TransactionLog::instance().rollbackTransaction(shared_from_this()); @@ -331,6 +340,11 @@ String MergeTreeTransaction::dumpDescription() const } std::lock_guard lock{mutex}; + if (finalized) + { + res += ", cannot dump detailed description, transaction is finalized"; + return res; + } res += fmt::format(", affects {} tables:", storages.size()); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index f2d8d29d244..e5a80e03e18 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -65,6 +65,8 @@ private: scope_guard beforeCommit(); void afterCommit(CSN assigned_csn) noexcept; bool rollback() noexcept; + void afterFinalize(); + void checkIsNotCancelled() const; mutable std::mutex mutex; @@ -74,6 +76,8 @@ private: std::atomic snapshot; const std::list::iterator snapshot_in_use_it; + bool finalized TSA_GUARDED_BY(mutex) = false; + /// Lists of changes made by transaction std::unordered_set storages TSA_GUARDED_BY(mutex); DataPartsVector creating_parts TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 827957b8749..6257e617d4a 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -350,7 +350,7 @@ void TransactionLog::tryFinalizeUnknownStateTransactions() /// CSNs must be already loaded, only need to check if the corresponding mapping exists. if (auto csn = getCSN(txn->tid)) { - finalizeCommittedTransaction(txn, csn, state_guard); + finalizeCommittedTransaction(txn.get(), csn, state_guard); } else { @@ -431,7 +431,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool /// The only thing we can do is to postpone its finalization. { std::lock_guard lock{running_list_mutex}; - unknown_state_list.emplace_back(txn.get(), std::move(state_guard)); + unknown_state_list.emplace_back(txn, std::move(state_guard)); } log_updated_event->set(); if (throw_on_unknown_status) @@ -487,6 +487,7 @@ CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN } } + txn->afterFinalize(); return allocated_csn; } @@ -523,6 +524,7 @@ void TransactionLog::rollbackTransaction(const MergeTreeTransactionPtr & txn) no } tryWriteEventToSystemLog(log, global_context, TransactionsInfoLogElement::ROLLBACK, txn->tid); + txn->afterFinalize(); } MergeTreeTransactionPtr TransactionLog::tryGetRunningTransaction(const TIDHash & tid) diff --git a/src/Interpreters/TransactionLog.h b/src/Interpreters/TransactionLog.h index 64d02ad3ac5..6e8777d8519 100644 --- a/src/Interpreters/TransactionLog.h +++ b/src/Interpreters/TransactionLog.h @@ -177,7 +177,7 @@ private: /// Transactions that are currently processed TransactionsList running_list TSA_GUARDED_BY(running_list_mutex); /// If we lost connection on attempt to create csn- node then we don't know transaction's state. - using UnknownStateList = std::vector>; + using UnknownStateList = std::vector>; UnknownStateList unknown_state_list TSA_GUARDED_BY(running_list_mutex); UnknownStateList unknown_state_list_loaded TSA_GUARDED_BY(running_list_mutex); /// Ordered list of snapshots that are currently used by some transactions. Needed for background cleanup. From b8d33f4d38b688ddcaffc372a50148db84eb6d25 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 5 Mar 2023 13:32:04 +0100 Subject: [PATCH 297/470] Fix test --- .../test_disk_configuration/test.py | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index 96cdb0213bc..6ebe994dc68 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -294,6 +294,12 @@ def test_merge_tree_custom_disk_setting(start_cluster): ).strip() ) + node1.query(f"DROP TABLE {TABLE_NAME} SYNC") + node1.query(f"DROP TABLE {TABLE_NAME}_2 SYNC") + node1.query(f"DROP TABLE {TABLE_NAME}_3 SYNC") + node1.query(f"DROP TABLE {TABLE_NAME}_4 SYNC") + node2.query(f"DROP TABLE {TABLE_NAME}_4 SYNC") + def test_merge_tree_nested_custom_disk_setting(start_cluster): node = cluster.instances["node1"] @@ -307,9 +313,9 @@ def test_merge_tree_nested_custom_disk_setting(start_cluster): ) node.query( - """ - DROP TABLE IF EXISTS test; - CREATE TABLE test (a Int32) + f""" + DROP TABLE IF EXISTS {TABLE_NAME} SYNC; + CREATE TABLE {TABLE_NAME} (a Int32) ENGINE = MergeTree() order by tuple() SETTINGS disk = disk( type=cache, @@ -323,13 +329,13 @@ def test_merge_tree_nested_custom_disk_setting(start_cluster): """ ) - node.query("INSERT INTO test SELECT number FROM numbers(100)") + node.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") node.query("SYSTEM DROP FILESYSTEM CACHE") # Check cache is filled assert 0 == int(node.query("SELECT count() FROM system.filesystem_cache")) - assert 100 == int(node.query("SELECT count() FROM test")) - node.query("SELECT * FROM test") + assert 100 == int(node.query(f"SELECT count() FROM {TABLE_NAME}")) + node.query(f"SELECT * FROM {TABLE_NAME}") assert 0 < int(node.query("SELECT count() FROM system.filesystem_cache")) # Check s3 is filled @@ -339,12 +345,13 @@ def test_merge_tree_nested_custom_disk_setting(start_cluster): node.restart_clickhouse() - assert 100 == int(node.query("SELECT count() FROM test")) + assert 100 == int(node.query(f"SELECT count() FROM {TABLE_NAME}")) expected = """ SETTINGS disk = disk(type = cache, max_size = \\'[HIDDEN]\\', path = \\'[HIDDEN]\\', disk = disk(type = s3, endpoint = \\'[HIDDEN]\\' """ - assert expected.strip() in node.query(f"SHOW CREATE TABLE test").strip() + assert expected.strip() in node.query(f"SHOW CREATE TABLE {TABLE_NAME}").strip() + node.query(f"DROP TABLE {TABLE_NAME} SYNC") def test_merge_tree_setting_override(start_cluster): @@ -419,3 +426,4 @@ def test_merge_tree_setting_override(start_cluster): assert ( len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) > 0 ) + node.query(f"DROP TABLE {TABLE_NAME} SYNC") From 099013831a8f50fbb0f9cce97bde5db90251fceb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 5 Mar 2023 16:15:03 +0100 Subject: [PATCH 298/470] Added batching for reads and retries for the most heavy function in backups (#47243) --- src/Backups/BackupCoordinationRemote.cpp | 150 +++++++++++++++++++++-- src/Backups/BackupCoordinationRemote.h | 18 ++- src/Backups/BackupsWorker.cpp | 27 +++- src/Backups/IBackupCoordination.h | 20 ++- src/Core/Settings.h | 4 + 5 files changed, 199 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 2ca06b76dc1..029a27d2712 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -166,13 +166,25 @@ namespace } BackupCoordinationRemote::BackupCoordinationRemote( - const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_) - : root_zookeeper_path(root_zookeeper_path_) + const BackupKeeperSettings & keeper_settings_, + const String & root_zookeeper_path_, + const String & backup_uuid_, + zkutil::GetZooKeeper get_zookeeper_, + bool is_internal_) + : keeper_settings(keeper_settings_) + , root_zookeeper_path(root_zookeeper_path_) , zookeeper_path(root_zookeeper_path_ + "/backup-" + backup_uuid_) , backup_uuid(backup_uuid_) , get_zookeeper(get_zookeeper_) , is_internal(is_internal_) { + zookeeper_retries_info = ZooKeeperRetriesInfo( + "BackupCoordinationRemote", + &Poco::Logger::get("BackupCoordinationRemote"), + keeper_settings.keeper_max_retries, + keeper_settings.keeper_retry_initial_backoff_ms, + keeper_settings.keeper_retry_max_backoff_ms); + createRootNodes(); stage_sync.emplace( zookeeper_path + "/stage", [this] { return getZooKeeper(); }, &Poco::Logger::get("BackupCoordination")); @@ -486,19 +498,131 @@ void BackupCoordinationRemote::updateFileInfo(const FileInfo & file_info) std::vector BackupCoordinationRemote::getAllFileInfos() const { - auto zk = getZooKeeper(); - std::vector file_infos; - Strings escaped_names = zk->getChildren(zookeeper_path + "/file_names"); - for (const String & escaped_name : escaped_names) + /// There could be tons of files inside /file_names or /file_infos + /// Thus we use MultiRead requests for processing them + /// We also use [Zoo]Keeper retries and it should be safe, because + /// this function is called at the end after the actual copying is finished. + + auto split_vector = [](Strings && vec, size_t max_batch_size) -> std::vector { - String size_and_checksum = zk->get(zookeeper_path + "/file_names/" + escaped_name); - UInt64 size = deserializeSizeAndChecksum(size_and_checksum).first; - FileInfo file_info; - if (size) /// we don't keep FileInfos for empty files - file_info = deserializeFileInfo(zk->get(zookeeper_path + "/file_infos/" + size_and_checksum)); - file_info.file_name = unescapeForFileName(escaped_name); - file_infos.emplace_back(std::move(file_info)); + std::vector result; + size_t left_border = 0; + + auto move_to_result = [&](auto && begin, auto && end) + { + auto batch = Strings(); + batch.reserve(max_batch_size); + std::move(begin, end, std::back_inserter(batch)); + result.push_back(std::move(batch)); + }; + + if (max_batch_size == 0) + { + move_to_result(vec.begin(), vec.end()); + return result; + } + + for (size_t pos = 0; pos < vec.size(); ++pos) + { + if (pos >= left_border + max_batch_size) + { + move_to_result(vec.begin() + left_border, vec.begin() + pos); + left_border = pos; + } + } + + if (vec.begin() + left_border != vec.end()) + move_to_result(vec.begin() + left_border, vec.end()); + + return result; + }; + + std::vector batched_escaped_names; + { + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getChildren", zookeeper_retries_info); + retries_ctl.retryLoop([&]() + { + auto zk = getZooKeeper(); + batched_escaped_names = split_vector(zk->getChildren(zookeeper_path + "/file_names"), keeper_settings.batch_size_for_keeper_multiread); + }); } + + std::vector file_infos; + file_infos.reserve(batched_escaped_names.size()); + + for (auto & batch : batched_escaped_names) + { + std::optional sizes_and_checksums; + { + Strings file_names_paths; + file_names_paths.reserve(batch.size()); + for (const String & escaped_name : batch) + file_names_paths.emplace_back(zookeeper_path + "/file_names/" + escaped_name); + + + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getSizesAndChecksums", zookeeper_retries_info); + retries_ctl.retryLoop([&]() + { + auto zk = getZooKeeper(); + sizes_and_checksums = zk->get(file_names_paths); + }); + } + + Strings non_empty_file_names; + Strings non_empty_file_infos_paths; + std::vector non_empty_files_infos; + + /// Process all files and understand whether there are some empty files + /// Save non empty file names for further batch processing + { + std::vector empty_files_infos; + for (size_t i = 0; i < batch.size(); ++i) + { + auto file_name = batch[i]; + if (sizes_and_checksums.value()[i].error != Coordination::Error::ZOK) + throw zkutil::KeeperException(sizes_and_checksums.value()[i].error); + auto size_and_checksum = sizes_and_checksums.value()[i].data; + auto size = deserializeSizeAndChecksum(size_and_checksum).first; + + if (size) + { + /// Save it later for batch processing + non_empty_file_names.emplace_back(file_name); + non_empty_file_infos_paths.emplace_back(zookeeper_path + "/file_infos/" + size_and_checksum); + continue; + } + + /// File is empty + FileInfo empty_file_info; + empty_file_info.file_name = unescapeForFileName(file_name); + empty_files_infos.emplace_back(std::move(empty_file_info)); + } + + std::move(empty_files_infos.begin(), empty_files_infos.end(), std::back_inserter(file_infos)); + } + + std::optional non_empty_file_infos_serialized; + ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getFileInfos", zookeeper_retries_info); + retries_ctl.retryLoop([&]() + { + auto zk = getZooKeeper(); + non_empty_file_infos_serialized = zk->get(non_empty_file_infos_paths); + }); + + /// Process non empty files + for (size_t i = 0; i < non_empty_file_names.size(); ++i) + { + FileInfo file_info; + if (non_empty_file_infos_serialized.value()[i].error != Coordination::Error::ZOK) + throw zkutil::KeeperException(non_empty_file_infos_serialized.value()[i].error); + file_info = deserializeFileInfo(non_empty_file_infos_serialized.value()[i].data); + file_info.file_name = unescapeForFileName(non_empty_file_names[i]); + non_empty_files_infos.emplace_back(std::move(file_info)); + } + + std::move(non_empty_files_infos.begin(), non_empty_files_infos.end(), std::back_inserter(file_infos)); + } + return file_infos; } diff --git a/src/Backups/BackupCoordinationRemote.h b/src/Backups/BackupCoordinationRemote.h index c7260bcd237..23c76f5be47 100644 --- a/src/Backups/BackupCoordinationRemote.h +++ b/src/Backups/BackupCoordinationRemote.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -16,7 +17,20 @@ constexpr size_t MAX_ZOOKEEPER_ATTEMPTS = 10; class BackupCoordinationRemote : public IBackupCoordination { public: - BackupCoordinationRemote(const String & root_zookeeper_path_, const String & backup_uuid_, zkutil::GetZooKeeper get_zookeeper_, bool is_internal_); + struct BackupKeeperSettings + { + UInt64 keeper_max_retries; + UInt64 keeper_retry_initial_backoff_ms; + UInt64 keeper_retry_max_backoff_ms; + UInt64 batch_size_for_keeper_multiread; + }; + + BackupCoordinationRemote( + const BackupKeeperSettings & keeper_settings_, + const String & root_zookeeper_path_, + const String & backup_uuid_, + zkutil::GetZooKeeper get_zookeeper_, + bool is_internal_); ~BackupCoordinationRemote() override; void setStage(const String & current_host, const String & new_stage, const String & message) override; @@ -68,12 +82,14 @@ private: void prepareReplicatedTables() const; void prepareReplicatedAccess() const; + const BackupKeeperSettings keeper_settings; const String root_zookeeper_path; const String zookeeper_path; const String backup_uuid; const zkutil::GetZooKeeper get_zookeeper; const bool is_internal; + mutable ZooKeeperRetriesInfo zookeeper_retries_info; std::optional stage_sync; mutable std::mutex mutex; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 865151cc9ec..bdcff249e7d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -38,12 +38,14 @@ namespace Stage = BackupCoordinationStage; namespace { - std::shared_ptr makeBackupCoordination(const String & root_zk_path, const String & backup_uuid, const ContextPtr & context, bool is_internal_backup) + std::shared_ptr makeBackupCoordination(std::optional keeper_settings, String & root_zk_path, const String & backup_uuid, const ContextPtr & context, bool is_internal_backup) { if (!root_zk_path.empty()) { + if (!keeper_settings.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parameter keeper_settings is empty while root_zk_path is not. This is bug"); auto get_zookeeper = [global_context = context->getGlobalContext()] { return global_context->getZooKeeper(); }; - return std::make_shared(root_zk_path, backup_uuid, get_zookeeper, is_internal_backup); + return std::make_shared(*keeper_settings, root_zk_path, backup_uuid, get_zookeeper, is_internal_backup); } else { @@ -169,7 +171,15 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context /// if it's not created here. However to handle errors better it's better to make a coordination here because this way /// if an exception will be thrown in startMakingBackup() other hosts will know about that. root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); - backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); + + BackupCoordinationRemote::BackupKeeperSettings keeper_settings + { + .keeper_max_retries = context->getSettingsRef().backup_keeper_max_retries, + .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_keeper_retry_initial_backoff_ms, + .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_keeper_retry_max_backoff_ms, + .batch_size_for_keeper_multiread = context->getSettingsRef().backup_batch_size_for_keeper_multiread, + }; + backup_coordination = makeBackupCoordination(keeper_settings, root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); } auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); @@ -265,10 +275,17 @@ void BackupsWorker::doBackup( context->checkAccess(required_access); String root_zk_path; - + std::optional keeper_settings; ClusterPtr cluster; if (on_cluster) { + keeper_settings = BackupCoordinationRemote::BackupKeeperSettings + { + .keeper_max_retries = context->getSettingsRef().backup_keeper_max_retries, + .keeper_retry_initial_backoff_ms = context->getSettingsRef().backup_keeper_retry_initial_backoff_ms, + .keeper_retry_max_backoff_ms = context->getSettingsRef().backup_keeper_retry_max_backoff_ms, + .batch_size_for_keeper_multiread = context->getSettingsRef().backup_batch_size_for_keeper_multiread, + }; root_zk_path = context->getConfigRef().getString("backups.zookeeper_path", "/clickhouse/backups"); backup_query->cluster = context->getMacros()->expand(backup_query->cluster); cluster = context->getCluster(backup_query->cluster); @@ -277,7 +294,7 @@ void BackupsWorker::doBackup( /// Make a backup coordination. if (!backup_coordination) - backup_coordination = makeBackupCoordination(root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); + backup_coordination = makeBackupCoordination(keeper_settings, root_zk_path, toString(*backup_settings.backup_uuid), context, backup_settings.internal); if (!allow_concurrent_backups && backup_coordination->hasConcurrentBackups(std::ref(num_active_backups))) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Concurrent backups not supported, turn on setting 'allow_concurrent_backups'"); diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index f5fa01a1530..b75d856b50f 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -1,7 +1,9 @@ #pragma once -#include #include +#include +#include +#include namespace DB @@ -85,6 +87,22 @@ public: /// Position in the archive. UInt64 pos_in_archive = static_cast(-1); + + /// Note: this format doesn't allow to parse data back + /// It is useful only for debugging purposes + [[ maybe_unused ]] String describe() + { + String result; + result += fmt::format("file_name: {};\n", file_name); + result += fmt::format("size: {};\n", size); + result += fmt::format("checksum: {};\n", getHexUIntLowercase(checksum)); + result += fmt::format("base_size: {};\n", base_size); + result += fmt::format("base_checksum: {};\n", getHexUIntLowercase(checksum)); + result += fmt::format("data_file_name: {};\n", data_file_name); + result += fmt::format("archive_suffix: {};\n", archive_suffix); + result += fmt::format("pos_in_archive: {};\n", pos_in_archive); + return result; + } }; /// Adds file information. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a65f2ccb60f..aeb6b5b1130 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -414,6 +414,10 @@ class IColumn; \ M(UInt64, backup_threads, 16, "The maximum number of threads to execute BACKUP requests.", 0) \ M(UInt64, restore_threads, 16, "The maximum number of threads to execute RESTORE requests.", 0) \ + M(UInt64, backup_keeper_max_retries, 20, "Max retries for keeper operations during backup", 0) \ + M(UInt64, backup_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for [Zoo]Keeper operations during backup", 0) \ + M(UInt64, backup_keeper_retry_max_backoff_ms, 5000, "Max backoff timeout for [Zoo]Keeper operations during backup", 0) \ + M(UInt64, backup_batch_size_for_keeper_multiread, 10000, "Maximum size of batch for multiread request to [Zoo]Keeper during backup", 0) \ \ M(Bool, log_profile_events, true, "Log query performance statistics into the query_log, query_thread_log and query_views_log.", 0) \ M(Bool, log_query_settings, true, "Log query settings into the query_log.", 0) \ From c38d820d0f9b71156a01a1a20d16b8260c000c03 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 5 Mar 2023 16:45:17 +0000 Subject: [PATCH 299/470] Add 4LW for cleaning resource --- programs/keeper/CMakeLists.txt | 1 + src/Coordination/CoordinationSettings.cpp | 2 +- src/Coordination/FourLetterCommand.cpp | 9 ++++++++ src/Coordination/FourLetterCommand.h | 13 ++++++++++- src/Coordination/KeeperDispatcher.cpp | 27 ++++++++++++++++++++++- src/Coordination/KeeperDispatcher.h | 2 ++ 6 files changed, 51 insertions(+), 3 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 9b01e6920a4..761335fb707 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -128,6 +128,7 @@ if (BUILD_STANDALONE_KEEPER) ch_contrib::lz4 ch_contrib::zstd ch_contrib::cityhash + ch_contrib::jemalloc common ch_contrib::double_conversion ch_contrib::dragonbox_to_chars pcg_random diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index f5c79d3be7a..5e1ac1e2d7f 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -36,7 +36,7 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco } -const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc"; +const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "conf,cons,crst,envi,ruok,srst,srvr,stat,wchs,dirs,mntr,isro,rcvr,apiv,csnp,lgif,rqld,rclc,clrs"; KeeperConfigurationAndSettings::KeeperConfigurationAndSettings() : server_id(NOT_EXIST) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 6157daad1cd..8a7fdb82fb7 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -148,6 +148,9 @@ void FourLetterCommandFactory::registerCommands(KeeperDispatcher & keeper_dispat FourLetterCommandPtr recalculate_command = std::make_shared(keeper_dispatcher); factory.registerCommand(recalculate_command); + FourLetterCommandPtr clean_resources_command = std::make_shared(keeper_dispatcher); + factory.registerCommand(clean_resources_command); + factory.initializeAllowList(keeper_dispatcher); factory.setInitialize(true); } @@ -524,4 +527,10 @@ String RecalculateCommand::run() return "ok"; } +String CleanResourcesCommand::run() +{ + keeper_dispatcher.cleanResources(); + return "ok"; +} + } diff --git a/src/Coordination/FourLetterCommand.h b/src/Coordination/FourLetterCommand.h index e1fe0333081..c1a91303c05 100644 --- a/src/Coordination/FourLetterCommand.h +++ b/src/Coordination/FourLetterCommand.h @@ -377,7 +377,6 @@ struct RequestLeaderCommand : public IFourLetterCommand ~RequestLeaderCommand() override = default; }; -/// Request to be leader. struct RecalculateCommand : public IFourLetterCommand { explicit RecalculateCommand(KeeperDispatcher & keeper_dispatcher_) @@ -390,4 +389,16 @@ struct RecalculateCommand : public IFourLetterCommand ~RecalculateCommand() override = default; }; +struct CleanResourcesCommand : public IFourLetterCommand +{ + explicit CleanResourcesCommand(KeeperDispatcher & keeper_dispatcher_) + : IFourLetterCommand(keeper_dispatcher_) + { + } + + String name() override { return "clrs"; } + String run() override; + ~CleanResourcesCommand() override = default; +}; + } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 06c693e45be..a6d16334924 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -9,7 +9,7 @@ #include #include #include - +#include #include #include @@ -17,12 +17,26 @@ #include #include +#if USE_JEMALLOC +# include + +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + +#endif + namespace CurrentMetrics { extern const Metric KeeperAliveConnections; extern const Metric KeeperOutstandingRequets; } +namespace ProfileEvents +{ + extern const Event MemoryAllocatorPurge; + extern const Event MemoryAllocatorPurgeTimeMicroseconds; +} + namespace fs = std::filesystem; namespace DB @@ -753,4 +767,15 @@ Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const return result; } +void KeeperDispatcher::cleanResources() +{ +#if USE_JEMALLOC + LOG_TRACE(&Poco::Logger::get("KeeperDispatcher"), "Purging unused memory"); + Stopwatch watch; + mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); +#endif +} + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 90965d0934e..9371d2fbbac 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -230,6 +230,8 @@ public: { return server->recalculateStorageStats(); } + + static void cleanResources(); }; } From 3a184880c6488a5027416315ec97ee06a93f1917 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 5 Mar 2023 17:18:53 +0000 Subject: [PATCH 300/470] Add test for clrs --- .../test_keeper_four_word_command/test.py | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 04f6800b92b..d3fcfcc3014 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -679,3 +679,41 @@ def test_cmd_rqld(started_cluster): + " does not become leader after 30s, maybe there is something wrong." ) assert keeper_utils.is_leader(cluster, node) + + +def test_cmd_clrs(started_cluster): + def get_memory_purges(): + return node1.query( + "SELECT value FROM system.events WHERE event = 'MemoryAllocatorPurge' SETTINGS system_events_show_zero_values = 1" + ) + + zk = None + try: + wait_nodes() + + zk = get_fake_zk(node1.name, timeout=30.0) + + paths = [f"/clrs_{i}" for i in range(10000)] + + # we only count the events because we cannot reliably test memory usage of Keeper + # but let's create and delete nodes so the first purge needs to release some memory + create_transaction = zk.transaction() + for path in paths: + create_transaction.create(path) + create_transaction.commit() + + delete_transaction = zk.transaction() + for path in paths: + delete_transaction.delete(path) + delete_transaction.commit() + + # repeat multiple times to make sure MemoryAllocatorPurge isn't increased because of other reasons + for _ in range(5): + prev_purges = int(get_memory_purges()) + keeper_utils.send_4lw_cmd(cluster, node1, cmd="clrs") + current_purges = int(get_memory_purges()) + assert current_purges > prev_purges + prev_purges = current_purges + + finally: + destroy_zk_client(zk) From 4f85b733f13f0b3780705e55f4e36dc910d9021b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 5 Mar 2023 20:19:33 +0300 Subject: [PATCH 301/470] Use string concatenation for XML serialization (#47251) --- src/Backups/BackupImpl.cpp | 66 ++++++++++++++++++++------------------ src/IO/Operators.h | 8 +++++ 2 files changed, 42 insertions(+), 32 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index e4c85bec496..b5f48a1a277 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -16,11 +16,11 @@ #include #include #include -#include #include #include #include #include +#include #include #include #include @@ -317,11 +317,19 @@ void BackupImpl::writeBackupMetadata() { assert(!is_internal_backup); - Poco::AutoPtr config{new Poco::Util::XMLConfiguration()}; - config->setInt("version", CURRENT_BACKUP_VERSION); - config->setBool("deduplicate_files", deduplicate_files); - config->setString("timestamp", toString(LocalDateTime{timestamp})); - config->setString("uuid", toString(*uuid)); + checkLockFile(true); + + std::unique_ptr out; + if (use_archives) + out = getArchiveWriter("")->writeFile(".backup"); + else + out = writer->writeFile(".backup"); + + *out << ""; + *out << "" << CURRENT_BACKUP_VERSION << ""; + *out << "" << deduplicate_files << ""; + *out << "" << toString(LocalDateTime{timestamp}) << ""; + *out << "" << toString(*uuid) << ""; auto all_file_infos = coordination->getAllFileInfos(); @@ -336,8 +344,8 @@ void BackupImpl::writeBackupMetadata() if (base_backup_in_use) { - config->setString("base_backup", base_backup_info->toString()); - config->setString("base_backup_uuid", toString(*base_backup_uuid)); + *out << "" << xml << base_backup_info->toString() << ""; + *out << "" << toString(*base_backup_uuid) << ""; } } @@ -346,31 +354,32 @@ void BackupImpl::writeBackupMetadata() num_entries = 0; size_of_entries = 0; - for (size_t i = 0; i != all_file_infos.size(); ++i) + *out << ""; + for (const auto & info : all_file_infos) { - const auto & info = all_file_infos[i]; - String prefix = i ? "contents.file[" + std::to_string(i) + "]." : "contents.file."; - config->setString(prefix + "name", info.file_name); - config->setUInt64(prefix + "size", info.size); + *out << ""; + + *out << "" << xml << info.file_name << ""; + *out << "" << info.size << ""; if (info.size) { - config->setString(prefix + "checksum", hexChecksum(info.checksum)); + *out << "" << hexChecksum(info.checksum) << ""; if (info.base_size) { - config->setBool(prefix + "use_base", true); + *out << "true"; if (info.base_size != info.size) { - config->setUInt64(prefix + "base_size", info.base_size); - config->setString(prefix + "base_checksum", hexChecksum(info.base_checksum)); + *out << "" << info.base_size << ""; + *out << "" << hexChecksum(info.base_checksum) << ""; } } if (!info.data_file_name.empty() && (info.data_file_name != info.file_name)) - config->setString(prefix + "data_file", info.data_file_name); + *out << "" << xml << info.data_file_name << ""; if (!info.archive_suffix.empty()) - config->setString(prefix + "archive_suffix", info.archive_suffix); + *out << "" << xml << info.archive_suffix << ""; if (info.pos_in_archive != static_cast(-1)) - config->setUInt64(prefix + "pos_in_archive", info.pos_in_archive); + *out << "" << info.pos_in_archive << ""; } total_size += info.size; @@ -380,23 +389,16 @@ void BackupImpl::writeBackupMetadata() ++num_entries; size_of_entries += info.size - info.base_size; } + + *out << ""; } + *out << ""; - std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - config->save(stream); - String str = stream.str(); + *out << ""; - checkLockFile(true); - - std::unique_ptr out; - if (use_archives) - out = getArchiveWriter("")->writeFile(".backup"); - else - out = writer->writeFile(".backup"); - out->write(str.data(), str.size()); out->finalize(); - uncompressed_size = size_of_entries + str.size(); + uncompressed_size = size_of_entries + out->count(); } diff --git a/src/IO/Operators.h b/src/IO/Operators.h index 06ff20c43e8..185745e8415 100644 --- a/src/IO/Operators.h +++ b/src/IO/Operators.h @@ -30,11 +30,13 @@ enum EscapeManip { escape }; /// For strings - escape special c enum QuoteManip { quote }; /// For strings, dates, datetimes - enclose in single quotes with escaping. In the rest, as usual. enum DoubleQuoteManip { double_quote }; /// For strings, dates, datetimes - enclose in double quotes with escaping. In the rest, as usual. enum BinaryManip { binary }; /// Output in binary format. +enum XMLManip { xml }; /// Output strings with XML escaping. struct EscapeManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; struct QuoteManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; struct DoubleQuoteManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; struct BinaryManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; +struct XMLManipWriteBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; struct EscapeManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; struct QuoteManipReadBuffer : std::reference_wrapper { using std::reference_wrapper::reference_wrapper; }; @@ -48,11 +50,13 @@ inline EscapeManipWriteBuffer operator<< (WriteBuffer & buf, EscapeManip) inline QuoteManipWriteBuffer operator<< (WriteBuffer & buf, QuoteManip) { return buf; } inline DoubleQuoteManipWriteBuffer operator<< (WriteBuffer & buf, DoubleQuoteManip) { return buf; } inline BinaryManipWriteBuffer operator<< (WriteBuffer & buf, BinaryManip) { return buf; } +inline XMLManipWriteBuffer operator<< (WriteBuffer & buf, XMLManip) { return buf; } template WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const T & x) { writeText(x, buf.get()); return buf; } template WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const T & x) { writeQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; } template WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; } +template WriteBuffer & operator<< (XMLManipWriteBuffer buf, const T & x) { writeText(x, buf.get()); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; } inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; } @@ -63,6 +67,10 @@ inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x inline WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'"'>(x, x + strlen(x), buf.get()); return buf; } inline WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const char * x) { writeStringBinary(x, buf.get()); return buf; } +inline WriteBuffer & operator<< (XMLManipWriteBuffer buf, std::string_view x) { writeXMLStringForTextElementOrAttributeValue(x, buf); return buf; } +inline WriteBuffer & operator<< (XMLManipWriteBuffer buf, StringRef x) { writeXMLStringForTextElementOrAttributeValue(x.toView(), buf); return buf; } +inline WriteBuffer & operator<< (XMLManipWriteBuffer buf, const char * x) { writeXMLStringForTextElementOrAttributeValue(std::string_view(x), buf); return buf; } + /// The manipulator calls the WriteBuffer method `next` - this makes the buffer reset. For nested buffers, the reset is not recursive. enum FlushManip { flush }; From 2aecaa7e46ef7890224aafbfa657f1ff9ca9d479 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Mon, 6 Mar 2023 10:51:51 +0400 Subject: [PATCH 302/470] Markup fixed at the page `/docs/ru/operations/server-configuration-parameters/settings.md` --- docs/ru/operations/server-configuration-parameters/settings.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index bffa3c39a60..e29b9def9d4 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -26,6 +26,7 @@ ClickHouse перезагружает встроенные словари с з :::danger "Внимание" Лучше не использовать, если вы только начали работать с ClickHouse. +::: Общий вид конфигурации: @@ -1064,6 +1065,7 @@ ClickHouse использует потоки из глобального пул :::danger "Обратите внимание" Завершающий слеш обязателен. +::: **Пример** @@ -1330,6 +1332,7 @@ TCP порт для защищённого обмена данными с кли :::danger "Обратите внимание" Завершающий слеш обязателен. +::: **Пример** From 7dc685f74c4f209c4856f369f3f73668ab33760b Mon Sep 17 00:00:00 2001 From: Dmitry Date: Mon, 6 Mar 2023 11:00:27 +0400 Subject: [PATCH 303/470] Markup fixed at the page `/docs/ru/operations/opentelemetry.md` --- docs/ru/operations/opentelemetry.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/operations/opentelemetry.md b/docs/ru/operations/opentelemetry.md index b6c5e89bcc6..4e127e9e0f0 100644 --- a/docs/ru/operations/opentelemetry.md +++ b/docs/ru/operations/opentelemetry.md @@ -10,6 +10,7 @@ ClickHouse поддерживает [OpenTelemetry](https://opentelemetry.io/) :::danger "Предупреждение" Поддержка стандарта экспериментальная и будет со временем меняться. +::: ## Обеспечение поддержки контекста трассировки в ClickHouse From 4d75b0a0454174195f97ad0ac61db9eafee4c94e Mon Sep 17 00:00:00 2001 From: Dmitry Date: Mon, 6 Mar 2023 11:05:07 +0400 Subject: [PATCH 304/470] Typos at the page `/docs/ru/operations/clickhouse-keeper.md` --- docs/ru/operations/clickhouse-keeper.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 67be83e13b2..3a931529b32 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -325,21 +325,21 @@ clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 -- Например, для кластера из 3 нод, алгоритм кворума продолжает работать при отказе не более чем одной ноды. Конфигурация кластера может быть изменена динамически с некоторыми ограничениями. -Переконфигурация также использует Raft, поэтому для добавление новой ноды кластера или исключения старой ноды из него требуется достижения кворума в рамках текущей конфигурации кластера. +Переконфигурация также использует Raft, поэтому для добавления новой ноды кластера или исключения старой ноды требуется достижение кворума в рамках текущей конфигурации кластера. Если в вашем кластере произошел отказ большего числа нод, чем допускает Raft для вашей текущей конфигурации и у вас нет возможности восстановить их работоспособность, Raft перестанет работать и не позволит изменить конфигурацию стандартным механизмом. -Тем не менее ClickHousr Keeper имеет возможность запуститься в режиме восстановления, который позволяет переконфигурировать класте используя только одну ноду кластера. +Тем не менее ClickHouse Keeper имеет возможность запуститься в режиме восстановления, который позволяет переконфигурировать кластер используя только одну ноду кластера. Этот механизм может использоваться только как крайняя мера, когда вы не можете восстановить существующие ноды кластера или запустить новый сервер с тем же идентификатором. Важно: - Удостоверьтесь, что отказавшие ноды не смогут в дальнейшем подключиться к кластеру в будущем. -- Не запускайте новые ноды, пока не завешите процедуру ниже. +- Не запускайте новые ноды, пока не завершите процедуру ниже. После того, как выполнили действия выше выполните следующие шаги. -1. Выберете одну ноду Keeper, которая станет новым лидером. Учтите, что данные которые с этой ноды будут испольщзованы всем кластером, поэтому рекомендуется выбрать ноду с наиболее актуальным состоянием. +1. Выберете одну ноду Keeper, которая станет новым лидером. Учтите, что данные с этой ноды будут использованы всем кластером, поэтому рекомендуется выбрать ноду с наиболее актуальным состоянием. 2. Перед дальнейшими действиям сделайте резервную копию данных из директорий `log_storage_path` и `snapshot_storage_path`. 3. Измените настройки на всех нодах кластера, которые вы собираетесь использовать. -4. Отправьте команду `rcvr` на ноду, которую вы выбрали или остановите ее и запустите заново с аргументом `--force-recovery`. Это переведет ноду в режим восстановления. +4. Отправьте команду `rcvr` на ноду, которую вы выбрали, или остановите ее и запустите заново с аргументом `--force-recovery`. Это переведет ноду в режим восстановления. 5. Запускайте остальные ноды кластера по одной и проверяйте, что команда `mntr` возвращает `follower` в выводе состояния `zk_server_state` перед тем, как запустить следующую ноду. -6. Пока нода работает в режиме восстановления, лидер будет возвращать ошибку на запрос `mntr` пока кворум не будет достигнут с помощью новых нод. Любые запросы от клиентов и постедователей будут возвращать ошибку. +6. Пока нода работает в режиме восстановления, лидер будет возвращать ошибку на запрос `mntr` пока кворум не будет достигнут с помощью новых нод. Любые запросы от клиентов и последователей будут возвращать ошибку. 7. После достижения кворума лидер перейдет в нормальный режим работы и станет обрабатывать все запросы через Raft. Удостоверьтесь, что запрос `mntr` возвращает `leader` в выводе состояния `zk_server_state`. From c1d6204a8c5b5b760e32c9f2cad6729aed3fefa0 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Mon, 6 Mar 2023 12:02:42 +0400 Subject: [PATCH 305/470] Misspell fixed at the page `/docs/ru/operations/storing-data.md` --- docs/ru/operations/storing-data.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/storing-data.md b/docs/ru/operations/storing-data.md index 2f5c9c95ea4..56081c82bc9 100644 --- a/docs/ru/operations/storing-data.md +++ b/docs/ru/operations/storing-data.md @@ -82,7 +82,7 @@ sidebar_label: "Хранение данных на внешних дисках" - `type` — `encrypted`. Иначе зашифрованный диск создан не будет. - `disk` — тип диска для хранения данных. -- `key` — ключ для шифрования и расшифровки. Тип: [Uint64](../sql-reference/data-types/int-uint.md). Вы можете использовать параметр `key_hex` для шифрования в шестнадцатеричной форме. +- `key` — ключ для шифрования и расшифровки. Тип: [UInt64](../sql-reference/data-types/int-uint.md). Вы можете использовать параметр `key_hex` для шифрования в шестнадцатеричной форме. Вы можете указать несколько ключей, используя атрибут `id` (смотрите пример выше). Необязательные параметры: From 64a169014ce9b39cff3fea6c92d3d34c8982a11f Mon Sep 17 00:00:00 2001 From: Dmitry Date: Mon, 6 Mar 2023 12:07:57 +0400 Subject: [PATCH 306/470] Markup fixed at the page `/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md` --- .../external-dictionaries/external-dicts-dict-structure.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 8c01b8295bf..a711287ae8e 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -59,6 +59,7 @@ ClickHouse поддерживает следующие виды ключей: :::danger "Обратите внимание" Ключ не надо дополнительно описывать в атрибутах. +::: ### Числовой ключ {#ext_dict-numeric-key} From d4f1ac60bc2fb3a8ffb20491705c3ffa823e7b62 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 6 Mar 2023 10:30:51 +0100 Subject: [PATCH 307/470] Update comment in alter_materialized_view_consistent --- .../0_stateless/01019_alter_materialized_view_consistent.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh index 496215ba492..3a2eac1f38f 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_consistent.sh @@ -54,6 +54,7 @@ function alter_thread() { for i in {0..5}; do ALTER[$i]="ALTER TABLE mv MODIFY QUERY SELECT v == 1 as test, v as case FROM src_a;" done + # Insert 3 ALTERs to src_b, one in the first half of the array and two in arbitrary positions. ALTER[$RANDOM % 3]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" ALTER[$RANDOM % 6]="ALTER TABLE mv MODIFY QUERY SELECT v == 2 as test, v as case FROM src_b;" From 48f644a95859af19efa3999369214755e3f81edb Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 6 Mar 2023 10:38:21 +0100 Subject: [PATCH 308/470] Fix style in JoiningTransform.cpp --- src/Processors/Transforms/JoiningTransform.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index bf80de67b42..120ff51cad1 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -333,8 +333,6 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (inputs.size() != 1 && outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have exactly one input port"); - - if (output_chunk) { input.setNotNeeded(); @@ -414,7 +412,6 @@ void DelayedJoinedBlocksTransform::work() finished = finished || delayed_blocks == nullptr; } - IProcessor::Status DelayedJoinedBlocksTransform::prepare() { for (auto & output : outputs) From 56e8547f7e1573a1f60188e6cc7223327c2c3755 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 6 Mar 2023 10:47:32 +0000 Subject: [PATCH 309/470] Do not apply the optimization when plan is built only for analysis --- src/Planner/PlannerJoinTree.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 06dc5b70bc1..6f818e2c8f7 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -410,13 +410,12 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } /// Apply trivial_count optimization if possible - bool is_trivial_count_applied = is_single_table_expression && table_node && select_query_info.has_aggregates + bool is_trivial_count_applied = !select_query_options.only_analyze && is_single_table_expression && table_node && select_query_info.has_aggregates && applyTrivialCountIfPossible(query_plan, *table_node, select_query_info.query_tree, planner_context->getQueryContext(), columns_names); if (is_trivial_count_applied) { - if (!select_query_options.only_analyze) - from_stage = QueryProcessingStage::WithMergeableState; + from_stage = QueryProcessingStage::WithMergeableState; } else { From eb2ed1b123f76be4d23e61a97e6a958cecee2e36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 Mar 2023 13:40:16 +0100 Subject: [PATCH 310/470] Add support for different expected errors --- src/Client/ClientBase.cpp | 43 +++++++------- src/Client/TestHint.cpp | 45 +++++++++++---- src/Client/TestHint.h | 57 +++++++++++++++++-- .../01470_columns_transformers.sql | 8 +-- 4 files changed, 111 insertions(+), 42 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 53eb5080130..b5c662b4a80 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1834,7 +1834,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) { /// disable logs if expects errors TestHint test_hint(all_queries_text); - if (test_hint.clientError() || test_hint.serverError()) + if (!test_hint.clientErrors().empty() || !test_hint.serverErrors().empty()) processTextAsSingleQuery("SET send_logs_level = 'fatal'"); } @@ -1876,17 +1876,18 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // the query ends because we failed to parse it, so we consume // the entire line. TestHint hint(String(this_query_begin, this_query_end - this_query_begin)); - if (hint.serverError()) + if (!hint.serverErrors().empty()) { // Syntax errors are considered as client errors - current_exception->addMessage("\nExpected server error '{}'.", hint.serverError()); + current_exception->addMessage("\nExpected server error: {}.", hint.serverErrors()); current_exception->rethrow(); } - if (hint.clientError() != current_exception->code()) + if (std::find(hint.clientErrors().begin(), hint.clientErrors().end(), current_exception->code()) + == hint.clientErrors().end()) { - if (hint.clientError()) - current_exception->addMessage("\nExpected client error: " + std::to_string(hint.clientError())); + if (!hint.clientErrors().empty()) + current_exception->addMessage("\nExpected client error: {}.", hint.clientErrors()); current_exception->rethrow(); } @@ -1935,37 +1936,41 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) bool error_matches_hint = true; if (have_error) { - if (test_hint.serverError()) + if (!test_hint.serverErrors().empty()) { if (!server_exception) { error_matches_hint = false; fmt::print(stderr, "Expected server error code '{}' but got no server error (query: {}).\n", - test_hint.serverError(), full_query); + test_hint.serverErrors(), full_query); } - else if (server_exception->code() != test_hint.serverError()) + else if ( + std::find(test_hint.serverErrors().begin(), test_hint.serverErrors().end(), server_exception->code()) + == test_hint.serverErrors().end()) { error_matches_hint = false; fmt::print(stderr, "Expected server error code: {} but got: {} (query: {}).\n", - test_hint.serverError(), server_exception->code(), full_query); + test_hint.serverErrors(), server_exception->code(), full_query); } } - if (test_hint.clientError()) + if (!test_hint.clientErrors().empty()) { if (!client_exception) { error_matches_hint = false; fmt::print(stderr, "Expected client error code '{}' but got no client error (query: {}).\n", - test_hint.clientError(), full_query); + test_hint.clientErrors(), full_query); } - else if (client_exception->code() != test_hint.clientError()) + else if ( + std::find(test_hint.clientErrors().begin(), test_hint.clientErrors().end(), client_exception->code()) + == test_hint.clientErrors().end()) { error_matches_hint = false; fmt::print(stderr, "Expected client error code '{}' but got '{}' (query: {}).\n", - test_hint.clientError(), client_exception->code(), full_query); + test_hint.clientErrors(), client_exception->code(), full_query); } } - if (!test_hint.clientError() && !test_hint.serverError()) + if (test_hint.clientErrors().empty() && test_hint.serverErrors().empty()) { // No error was expected but it still occurred. This is the // default case without test hint, doesn't need additional @@ -1975,19 +1980,19 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) } else { - if (test_hint.clientError()) + if (!test_hint.clientErrors().empty()) { error_matches_hint = false; fmt::print(stderr, "The query succeeded but the client error '{}' was expected (query: {}).\n", - test_hint.clientError(), full_query); + test_hint.clientErrors(), full_query); } - if (test_hint.serverError()) + if (!test_hint.serverErrors().empty()) { error_matches_hint = false; fmt::print(stderr, "The query succeeded but the server error '{}' was expected (query: {}).\n", - test_hint.serverError(), full_query); + test_hint.serverErrors(), full_query); } } diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index f6d1e5d73c3..adaae5fe5ee 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -6,25 +6,46 @@ #include #include +namespace DB::ErrorCodes +{ +extern const int CANNOT_PARSE_TEXT; +} + namespace { /// Parse error as number or as a string (name of the error code const) -int parseErrorCode(DB::ReadBufferFromString & in) +DB::TestHint::error_vector parseErrorCode(DB::ReadBufferFromString & in) { - int code = -1; - String code_name; + DB::TestHint::error_vector error_codes{}; - auto * pos = in.position(); - tryReadText(code, in); - if (pos != in.position()) + while (!in.eof()) { - return code; + int code = -1; + String code_name; + auto * pos = in.position(); + + tryReadText(code, in); + if (pos == in.position()) + { + readStringUntilWhitespace(code_name, in); + code = DB::ErrorCodes::getErrorCodeByName(code_name); + } + error_codes.push_back(code); + + if (in.eof()) + break; + skipWhitespaceIfAny(in); + if (in.eof()) + break; + char c; + in.readStrict(c); + if (c != '|') + throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected separator '|'. Got '{}'", c); + skipWhitespaceIfAny(in); } - /// Try parse as string - readStringUntilWhitespace(code_name, in); - return DB::ErrorCodes::getErrorCodeByName(code_name); + return error_codes; } } @@ -85,9 +106,9 @@ void TestHint::parse(const String & hint, bool is_leading_hint) if (!is_leading_hint) { if (item == "serverError") - server_error = parseErrorCode(in); + server_errors = parseErrorCode(in); else if (item == "clientError") - client_error = parseErrorCode(in); + client_errors = parseErrorCode(in); } if (item == "echo") diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index 7fa4e86c025..30b3cacd3cb 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -1,6 +1,10 @@ #pragma once #include +#include + +#include + #include @@ -12,10 +16,13 @@ namespace DB /// The following comment hints are supported: /// /// - "-- { serverError 60 }" -- in case of you are expecting server error. +/// - "-- { serverError 16 | 36 }" -- in case of you are expecting one of the 2 errors /// /// - "-- { clientError 20 }" -- in case of you are expecting client error. +/// - "-- { clientError 20 | 60 | 92 }" -- It's expected that the client will return one of the 3 errors. /// /// - "-- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. +/// - "-- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS }" -- by error name. /// /// - "-- { clientError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. /// @@ -43,29 +50,67 @@ namespace DB class TestHint { public: + using error_vector = std::vector; TestHint(const String & query_); - int serverError() const { return server_error; } - int clientError() const { return client_error; } + const auto & serverErrors() const { return server_errors; } + const auto & clientErrors() const { return client_errors; } std::optional echoQueries() const { return echo; } private: const String & query; - int server_error = 0; - int client_error = 0; + error_vector server_errors{}; + error_vector client_errors{}; std::optional echo; void parse(const String & hint, bool is_leading_hint); bool allErrorsExpected(int actual_server_error, int actual_client_error) const { - return (server_error || client_error) && (server_error == actual_server_error) && (client_error == actual_client_error); + if (actual_server_error && std::find(server_errors.begin(), server_errors.end(), actual_server_error) == server_errors.end()) + return false; + if (!actual_server_error && server_errors.size()) + return false; + + if (actual_client_error && std::find(client_errors.begin(), client_errors.end(), actual_client_error) == client_errors.end()) + return false; + if (!actual_client_error && client_errors.size()) + return false; + + return true; } bool lostExpectedError(int actual_server_error, int actual_client_error) const { - return (server_error && !actual_server_error) || (client_error && !actual_client_error); + return (server_errors.size() && !actual_server_error) || (client_errors.size() && !actual_client_error); } }; } + +template <> +struct fmt::formatter +{ + static constexpr auto parse(format_parse_context & ctx) + { + const auto * it = ctx.begin(); + const auto * end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("Invalid format"); + + return it; + } + + template + auto format(const DB::TestHint::error_vector & error_vector, FormatContext & ctx) + { + if (error_vector.empty()) + return format_to(ctx.out(), "{}", 0); + else if (error_vector.size() == 1) + return format_to(ctx.out(), "{}", error_vector[0]); + else + return format_to(ctx.out(), "One of [{}]", fmt::join(error_vector, ", ")); + } +}; diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index 22c30ed36bf..7cff1920a4e 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -1,5 +1,3 @@ -SET allow_experimental_analyzer = 1; - DROP TABLE IF EXISTS columns_transformers; CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) Engine=TinyLog; @@ -19,15 +17,15 @@ SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; SELECT * EXCEPT STRICT i from columns_transformers; SELECT * EXCEPT STRICT (i, j) from columns_transformers; SELECT * EXCEPT STRICT i, j1 from columns_transformers; -- { serverError 47 } -SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError 36 } +SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } SELECT * REPLACE STRICT i + 1 AS i from columns_transformers; -SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError 36 } +SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError 36 } +SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; From 6c43781d116fb48512df6a966ca5bf9b3859639b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 Mar 2023 15:13:07 +0100 Subject: [PATCH 311/470] Try manually fixing tests --- .../0_stateless/00002_system_numbers.sql | 2 +- .../0_stateless/00386_has_column_in_table.sql | 10 +++++----- .../0_stateless/00718_format_datetime.sql | 18 +++++++++--------- .../queries/0_stateless/00975_values_list.sql | 4 ++-- .../0_stateless/01056_create_table_as.sql | 6 +++--- ...mize_skip_unused_shards_const_expr_eval.sql | 16 ++++++++-------- ...timize_skip_unused_shards_type_mismatch.sql | 2 +- .../01225_drop_dictionary_as_table.sql | 2 +- ...01225_show_create_table_from_dictionary.sql | 2 +- .../0_stateless/01231_log_queries_min_type.sql | 4 ++-- ...t_block_size_rows_for_materialized_views.sh | 2 +- tests/queries/0_stateless/01284_port.sql.j2 | 6 +++--- ...ctive_elimination_dictGet_BAD_ARGUMENTS.sql | 2 +- ..._GROUP_BY_injective_elimination_dictGet.sql | 2 +- .../01402_cast_nullable_string_to_enum.sql | 8 ++++---- .../01404_roundUpToPowerOfTwoOrZero_safety.sql | 2 +- .../0_stateless/01407_lambda_arrayJoin.sql | 2 +- .../0_stateless/01408_range_overflow.sql | 2 +- ..._trivial_count_with_partition_predicate.sql | 12 ++++++------ ...timize_aggregation_in_order_memory_long.sql | 4 ++-- .../01516_create_table_primary_key.sql | 2 +- ...terministic_optimize_skip_unused_shards.sql | 2 +- .../01530_drop_database_atomic_sync.sql | 2 +- .../01555_system_distribution_queue_mask.sql | 2 +- .../0_stateless/01592_toUnixTimestamp_Date.sql | 2 +- .../queries/0_stateless/01595_countMatches.sql | 4 ++-- .../01709_inactive_parts_to_throw_insert.sql | 2 +- .../01710_projection_with_mixed_pipeline.sql | 2 +- .../0_stateless/01888_read_int_safe.sql | 16 ++++++++-------- .../02008_tuple_to_name_value_pairs.sql | 8 ++++---- 30 files changed, 75 insertions(+), 75 deletions(-) diff --git a/tests/queries/0_stateless/00002_system_numbers.sql b/tests/queries/0_stateless/00002_system_numbers.sql index 95f75573201..d5934c7d387 100644 --- a/tests/queries/0_stateless/00002_system_numbers.sql +++ b/tests/queries/0_stateless/00002_system_numbers.sql @@ -6,7 +6,7 @@ SELECT number FROM system.numbers WHERE number >= 5 LIMIT 2; SELECT * FROM system.numbers WHERE number == 7 LIMIT 1; SELECT number AS n FROM system.numbers WHERE number IN(8, 9) LIMIT 2; select number from system.numbers limit 0; -select x from system.numbers limit 1; -- { clientError 0 serverError 47 } +select x from system.numbers limit 1; -- { serverError UNKNOWN_IDENTIFIER } SELECT x, number FROM system.numbers LIMIT 1; -- { serverError 47 } SELECT * FROM system.number LIMIT 1; -- { serverError 60 } SELECT * FROM system LIMIT 1; -- { serverError 60 } diff --git a/tests/queries/0_stateless/00386_has_column_in_table.sql b/tests/queries/0_stateless/00386_has_column_in_table.sql index d543bb42ca7..7347293e05b 100644 --- a/tests/queries/0_stateless/00386_has_column_in_table.sql +++ b/tests/queries/0_stateless/00386_has_column_in_table.sql @@ -21,11 +21,11 @@ SELECT hasColumnInTable('localhost', currentDatabase(), 'has_column_in_table', ' SELECT hasColumnInTable('system', 'one', ''); /* bad queries */ -SELECT hasColumnInTable('', '', ''); -- { serverError 60; } -SELECT hasColumnInTable('', 't', 'c'); -- { serverError 81; } -SELECT hasColumnInTable(currentDatabase(), '', 'c'); -- { serverError 60; } -SELECT hasColumnInTable('d', 't', 's'); -- { serverError 81; } -SELECT hasColumnInTable(currentDatabase(), 't', 's'); -- { serverError 60; } +SELECT hasColumnInTable('', '', ''); -- { serverError 60 } +SELECT hasColumnInTable('', 't', 'c'); -- { serverError 81 } +SELECT hasColumnInTable(currentDatabase(), '', 'c'); -- { serverError 60 } +SELECT hasColumnInTable('d', 't', 's'); -- { serverError 81 } +SELECT hasColumnInTable(currentDatabase(), 't', 's'); -- { serverError 60 } DROP TABLE has_column_in_table; diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index 74ec03d83d3..3f8c927dfe7 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -1,14 +1,14 @@ SET send_logs_level = 'fatal'; -SELECT formatDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH (42) } -SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT (43) } -SELECT formatDateTime(now(), now()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT (43) } -SELECT formatDateTime(now(), 'good format pattern', now()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT (43) } -SELECT formatDateTime(now(), 'unescaped %'); -- { serverError BAD_ARGUMENTS (36) } -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%U'); -- { serverError NOT_IMPLEMENTED (48) } -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%v'); -- { serverError NOT_IMPLEMENTED (48) } -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%x'); -- { serverError NOT_IMPLEMENTED (48) } -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%X'); -- { serverError NOT_IMPLEMENTED (48) } +SELECT formatDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT formatDateTime(now(), now()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT formatDateTime(now(), 'good format pattern', now()); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT formatDateTime(now(), 'unescaped %'); -- { serverError BAD_ARGUMENTS } +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%U'); -- { serverError NOT_IMPLEMENTED } +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%v'); -- { serverError NOT_IMPLEMENTED } +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%x'); -- { serverError NOT_IMPLEMENTED } +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%X'); -- { serverError NOT_IMPLEMENTED } SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%a'), formatDateTime(toDate32('2018-01-02'), '%a'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%b'), formatDateTime(toDate32('2018-01-02'), '%b'); diff --git a/tests/queries/0_stateless/00975_values_list.sql b/tests/queries/0_stateless/00975_values_list.sql index 40c86898966..35afc99e93e 100644 --- a/tests/queries/0_stateless/00975_values_list.sql +++ b/tests/queries/0_stateless/00975_values_list.sql @@ -12,8 +12,8 @@ SELECT * FROM VALUES('n UInt64, s String, ss String', (1 + 22, '23', toString(23 SELECT * FROM VALUES('a Decimal(4, 4), b String, c String', (divide(toDecimal32(5, 3), 3), 'a', 'b')); -SELECT * FROM VALUES('x Float64', toUInt64(-1)); -- { serverError 69; } -SELECT * FROM VALUES('x Float64', NULL); -- { serverError 53; } +SELECT * FROM VALUES('x Float64', toUInt64(-1)); -- { serverError 69 } +SELECT * FROM VALUES('x Float64', NULL); -- { serverError 53 } SELECT * FROM VALUES('x Nullable(Float64)', NULL); DROP TABLE values_list; diff --git a/tests/queries/0_stateless/01056_create_table_as.sql b/tests/queries/0_stateless/01056_create_table_as.sql index 62db8282ac0..2e146d67ca9 100644 --- a/tests/queries/0_stateless/01056_create_table_as.sql +++ b/tests/queries/0_stateless/01056_create_table_as.sql @@ -19,12 +19,12 @@ DROP TABLE t3; -- live view SET allow_experimental_live_view=1; CREATE LIVE VIEW lv AS SELECT * FROM t1; -CREATE TABLE t3 AS lv; -- { serverError 80; } +CREATE TABLE t3 AS lv; -- { serverError 80 } DROP TABLE lv; -- view CREATE VIEW v AS SELECT * FROM t1; -CREATE TABLE t3 AS v; -- { serverError 80; } +CREATE TABLE t3 AS v; -- { serverError 80 } DROP TABLE v; -- dictionary @@ -43,7 +43,7 @@ SOURCE(CLICKHOUSE( TABLE 'dict_data' DB 'test_01056_dict_data' USER 'default' PASSWORD '')) LIFETIME(MIN 0 MAX 0) LAYOUT(SPARSE_HASHED()); -CREATE TABLE t3 AS dict; -- { serverError 80; } +CREATE TABLE t3 AS dict; -- { serverError 80 } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t3; diff --git a/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql b/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql index 85c239765bc..24eaaacb8bd 100644 --- a/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql +++ b/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql @@ -16,16 +16,16 @@ select * from dist_01072 where key=toInt32OrZero(toString(xxHash64(0))); select * from dist_01072 where key=toInt32(xxHash32(0)); select * from dist_01072 where key=toInt32(toInt32(xxHash32(0))); select * from dist_01072 where key=toInt32(toInt32(toInt32(xxHash32(0)))); -select * from dist_01072 where key=value; -- { serverError 507; } -select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value; -- { serverError 507 } +select * from dist_01072 where key=toInt32(value); -- { serverError 507 } select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; drop table dist_01072; create table dist_01072 (key Int, value Nullable(Int), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); select * from dist_01072 where key=toInt32(xxHash32(0)); -select * from dist_01072 where key=value; -- { serverError 507; } -select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value; -- { serverError 507 } +select * from dist_01072 where key=toInt32(value); -- { serverError 507 } select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; @@ -34,16 +34,16 @@ set allow_suspicious_low_cardinality_types=1; drop table dist_01072; create table dist_01072 (key Int, value LowCardinality(Int), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); select * from dist_01072 where key=toInt32(xxHash32(0)); -select * from dist_01072 where key=value; -- { serverError 507; } -select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value; -- { serverError 507 } +select * from dist_01072 where key=toInt32(value); -- { serverError 507 } select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; drop table dist_01072; create table dist_01072 (key Int, value LowCardinality(Nullable(Int)), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); select * from dist_01072 where key=toInt32(xxHash32(0)); -select * from dist_01072 where key=value; -- { serverError 507; } -select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value; -- { serverError 507 } +select * from dist_01072 where key=toInt32(value); -- { serverError 507 } select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; diff --git a/tests/queries/0_stateless/01211_optimize_skip_unused_shards_type_mismatch.sql b/tests/queries/0_stateless/01211_optimize_skip_unused_shards_type_mismatch.sql index 65adaf3ad71..de41132df62 100644 --- a/tests/queries/0_stateless/01211_optimize_skip_unused_shards_type_mismatch.sql +++ b/tests/queries/0_stateless/01211_optimize_skip_unused_shards_type_mismatch.sql @@ -9,7 +9,7 @@ create table data_02000 (key Int) Engine=Null(); create table dist_02000 as data_02000 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_02000, key); select * from data_02000 where key = 0xdeadbeafdeadbeaf; -select * from dist_02000 where key = 0xdeadbeafdeadbeaf settings force_optimize_skip_unused_shards=2; -- { serverError 507; } +select * from dist_02000 where key = 0xdeadbeafdeadbeaf settings force_optimize_skip_unused_shards=2; -- { serverError 507 } select * from dist_02000 where key = 0xdeadbeafdeadbeaf; drop table data_02000; diff --git a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql index 513ecbd4ed4..be2f7b2a9bf 100644 --- a/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql +++ b/tests/queries/0_stateless/01225_drop_dictionary_as_table.sql @@ -16,7 +16,7 @@ LAYOUT(FLAT()); SYSTEM RELOAD DICTIONARY dict_db_01225.dict; -DROP TABLE dict_db_01225.dict; -- { serverError 520; } +DROP TABLE dict_db_01225.dict; -- { serverError 520 } DROP DICTIONARY dict_db_01225.dict; DROP DATABASE dict_db_01225; diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 09cde642ed2..bc733a0c546 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -18,7 +18,7 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.dict` FORMAT TSVRaw; -SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 487; } +SHOW CREATE TABLE dict_db_01225_dictionary.`dict_db_01225.no_such_dict`; -- { serverError 487 } DROP DATABASE dict_db_01225; DROP DATABASE dict_db_01225_dictionary; diff --git a/tests/queries/0_stateless/01231_log_queries_min_type.sql b/tests/queries/0_stateless/01231_log_queries_min_type.sql index c2470bb9a56..0ed5e3e605c 100644 --- a/tests/queries/0_stateless/01231_log_queries_min_type.sql +++ b/tests/queries/0_stateless/01231_log_queries_min_type.sql @@ -15,7 +15,7 @@ select count() from system.query_log where current_database = currentDatabase() set max_rows_to_read='100K'; set log_queries_min_type='EXCEPTION_WHILE_PROCESSING'; -select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } +select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158 } set max_rows_to_read=0; system flush logs; select count() from system.query_log where current_database = currentDatabase() @@ -23,7 +23,7 @@ select count() from system.query_log where current_database = currentDatabase() and event_date >= yesterday() and type = 'ExceptionWhileProcessing'; set max_rows_to_read='100K'; -select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; } +select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158 } system flush logs; set max_rows_to_read=0; select count() from system.query_log where 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 0e258bbbb09..08cc97c84bf 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 @@ -76,7 +76,7 @@ insert into data_01278 select reinterpretAsString(number), // s6 reinterpretAsString(number), // s7 reinterpretAsString(number) // s8 -from numbers(100000); -- { serverError 241; }" > /dev/null 2>&1 +from numbers(100000); -- { serverError 241 }" > /dev/null 2>&1 local ret_code=$? if [[ $ret_code -eq 0 ]]; then diff --git a/tests/queries/0_stateless/01284_port.sql.j2 b/tests/queries/0_stateless/01284_port.sql.j2 index 6f78b3b8e3b..50e096c6deb 100644 --- a/tests/queries/0_stateless/01284_port.sql.j2 +++ b/tests/queries/0_stateless/01284_port.sql.j2 @@ -19,9 +19,9 @@ select port{{ suffix }}('http://127.0.0.1/', toUInt16(80)); select port{{ suffix }}('http://foobar.com/', toUInt16(80)); -- unsupported -/* ILLEGAL_TYPE_OF_ARGUMENT */ select port(toFixedString('', 1)); -- { serverError 43; } -/* ILLEGAL_TYPE_OF_ARGUMENT */ select port{{ suffix }}('', 1); -- { serverError 43; } -/* NUMBER_OF_ARGUMENTS_DOESNT_MATCH */ select port{{ suffix }}('', 1, 1); -- { serverError 42; } +/* ILLEGAL_TYPE_OF_ARGUMENT */ select port(toFixedString('', 1)); -- { serverError 43 } +/* ILLEGAL_TYPE_OF_ARGUMENT */ select port{{ suffix }}('', 1); -- { serverError 43 } +/* NUMBER_OF_ARGUMENTS_DOESNT_MATCH */ select port{{ suffix }}('', 1, 1); -- { serverError 42 } -- -- Known limitations of domain() (getURLHost()) diff --git a/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql b/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql index 88a2b25c2db..8ff9cd2b9f2 100644 --- a/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql +++ b/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql @@ -1 +1 @@ -SELECT dictGetString(concat('default', '.countryId'), 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36; } +SELECT dictGetString(concat('default', '.countryId'), 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36 } diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql index 258d96829a5..29ffcb46fbf 100644 --- a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql @@ -1,7 +1,7 @@ -- Tags: no-parallel -- https://github.com/ClickHouse/ClickHouse/issues/11469 -SELECT dictGet('default.countryId', 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36; } +SELECT dictGet('default.countryId', 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36 } -- with real dictionary diff --git a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql index 3b53e593095..b8b5370515a 100644 --- a/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql +++ b/tests/queries/0_stateless/01402_cast_nullable_string_to_enum.sql @@ -5,9 +5,9 @@ SELECT CAST(CAST(NULL AS Nullable(String)) AS Nullable(Enum8('Hello' = 1))); SELECT CAST(CAST(NULL AS Nullable(FixedString(1))) AS Nullable(Enum8('Hello' = 1))); -- empty string still not acceptable -SELECT CAST(CAST('' AS Nullable(String)) AS Nullable(Enum8('Hello' = 1))); -- { serverError 36; } -SELECT CAST(CAST('' AS Nullable(FixedString(1))) AS Nullable(Enum8('Hello' = 1))); -- { serverError 36; } +SELECT CAST(CAST('' AS Nullable(String)) AS Nullable(Enum8('Hello' = 1))); -- { serverError 36 } +SELECT CAST(CAST('' AS Nullable(FixedString(1))) AS Nullable(Enum8('Hello' = 1))); -- { serverError 36 } -- non-Nullable Enum() still not acceptable -SELECT CAST(CAST(NULL AS Nullable(String)) AS Enum8('Hello' = 1)); -- { serverError 349; } -SELECT CAST(CAST(NULL AS Nullable(FixedString(1))) AS Enum8('Hello' = 1)); -- { serverError 349; } +SELECT CAST(CAST(NULL AS Nullable(String)) AS Enum8('Hello' = 1)); -- { serverError 349 } +SELECT CAST(CAST(NULL AS Nullable(FixedString(1))) AS Enum8('Hello' = 1)); -- { serverError 349 } diff --git a/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql b/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql index 4ee6e1fa5e4..d61a35c9999 100644 --- a/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql +++ b/tests/queries/0_stateless/01404_roundUpToPowerOfTwoOrZero_safety.sql @@ -1,4 +1,4 @@ -- repeat() with this length and this number of rows will allocation huge enough region (MSB set), -- which will cause roundUpToPowerOfTwoOrZero() returns 0 for such allocation (before the fix), -- and later repeat() will try to use this memory and will got SIGSEGV. -SELECT repeat('0.0001048576', number * (number * (number * 255))) FROM numbers(65535); -- { serverError 131; } +SELECT repeat('0.0001048576', number * (number * (number * 255))) FROM numbers(65535); -- { serverError 131 } diff --git a/tests/queries/0_stateless/01407_lambda_arrayJoin.sql b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql index 363b1d92dbb..e1b8c1d5a76 100644 --- a/tests/queries/0_stateless/01407_lambda_arrayJoin.sql +++ b/tests/queries/0_stateless/01407_lambda_arrayJoin.sql @@ -1,5 +1,5 @@ SELECT arrayFilter((a) -> ((a, arrayJoin([])) IN (Null, [Null])), []); SELECT arrayFilter((a) -> ((a, arrayJoin([[]])) IN (Null, [Null])), []); -SELECT * FROM system.one ARRAY JOIN arrayFilter((a) -> ((a, arrayJoin([])) IN (NULL)), []) AS arr_x; -- { serverError 43; } +SELECT * FROM system.one ARRAY JOIN arrayFilter((a) -> ((a, arrayJoin([])) IN (NULL)), []) AS arr_x; -- { serverError 43 } SELECT * FROM numbers(1) LEFT ARRAY JOIN arrayFilter((x_0, x_1) -> (arrayJoin([]) IN (NULL)), [], []) AS arr_x; diff --git a/tests/queries/0_stateless/01408_range_overflow.sql b/tests/queries/0_stateless/01408_range_overflow.sql index 2107e8c3f36..d26507f8358 100644 --- a/tests/queries/0_stateless/01408_range_overflow.sql +++ b/tests/queries/0_stateless/01408_range_overflow.sql @@ -1,7 +1,7 @@ -- executeGeneric() SELECT range(1025, 1048576 + 9223372036854775807, 9223372036854775807); SELECT range(1025, 1048576 + (9223372036854775807 AS i), i); -SELECT range(1025, 18446744073709551615, 1); -- { serverError 69; } +SELECT range(1025, 18446744073709551615, 1); -- { serverError 69 } -- executeConstStep() SELECT range(number, 1048576 + 9223372036854775807, 9223372036854775807) FROM system.numbers LIMIT 1 OFFSET 1025; diff --git a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql index e4e2e3dd76a..e8643a4468c 100644 --- a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql +++ b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql @@ -7,16 +7,16 @@ insert into test1 values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), set max_rows_to_read = 1; -- non-optimized -select count() from test1 settings max_parallel_replicas = 3; -- { serverError 158; } +select count() from test1 settings max_parallel_replicas = 3; -- { serverError 158 } -- optimized (toYear is monotonic and we provide the partition expr as is) select count() from test1 where toYear(toDate(p)) = 1999; -- non-optimized (toDate(DateTime) is always monotonic, but we cannot relaxing the predicates to do trivial count()) -select count() from test1 where p > toDateTime('2020-09-01 10:00:00'); -- { serverError 158; } +select count() from test1 where p > toDateTime('2020-09-01 10:00:00'); -- { serverError 158 } -- optimized (partition expr wrapped with non-monotonic functions) select count() FROM test1 where toDate(p) = '2020-09-01' and sipHash64(toString(toDate(p))) % 2 = 1; select count() FROM test1 where toDate(p) = '2020-09-01' and sipHash64(toString(toDate(p))) % 2 = 0; -- non-optimized (some predicate depends on non-partition_expr columns) -select count() FROM test1 where toDate(p) = '2020-09-01' and k = 2; -- { serverError 158; } +select count() FROM test1 where toDate(p) = '2020-09-01' and k = 2; -- { serverError 158 } -- optimized select count() from test1 where toDate(p) > '2020-09-01'; -- non-optimized @@ -35,10 +35,10 @@ select count() from test_tuple where i > 2; -- optimized select count() from test_tuple where i < 1; -- non-optimized -select count() from test_tuple array join [p,p] as c where toDate(p) = '2020-09-01'; -- { serverError 158; } +select count() from test_tuple array join [p,p] as c where toDate(p) = '2020-09-01'; -- { serverError 158 } select count() from test_tuple array join [1,2] as c where toDate(p) = '2020-09-01' settings max_rows_to_read = 4; -- non-optimized -select count() from test_tuple array join [1,2,3] as c where toDate(p) = '2020-09-01'; -- { serverError 158; } +select count() from test_tuple array join [1,2,3] as c where toDate(p) = '2020-09-01'; -- { serverError 158 } select count() from test_tuple array join [1,2,3] as c where toDate(p) = '2020-09-01' settings max_rows_to_read = 6; create table test_two_args(i int, j int, k int) engine MergeTree partition by i + j order by k settings index_granularity = 1; @@ -48,7 +48,7 @@ insert into test_two_args values (1, 2, 3), (2, 1, 3), (0, 3, 4); -- optimized select count() from test_two_args where i + j = 3; -- non-optimized -select count() from test_two_args where i = 1; -- { serverError 158; } +select count() from test_two_args where i = 1; -- { serverError 158 } drop table test1; drop table test_tuple; diff --git a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql index 228e4d73167..3d57518d0f4 100644 --- a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql +++ b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory_long.sql @@ -13,9 +13,9 @@ set max_memory_usage='500M'; set max_threads=1; set max_block_size=500; -select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null settings optimize_aggregation_in_order=0; -- { serverError 241; } +select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null settings optimize_aggregation_in_order=0; -- { serverError 241 } select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null settings optimize_aggregation_in_order=1; -- for WITH TOTALS previous groups should be kept. -select key, groupArray(repeat('a', 200)), count() from data_01513 group by key with totals format Null settings optimize_aggregation_in_order=1; -- { serverError 241; } +select key, groupArray(repeat('a', 200)), count() from data_01513 group by key with totals format Null settings optimize_aggregation_in_order=1; -- { serverError 241 } drop table data_01513; diff --git a/tests/queries/0_stateless/01516_create_table_primary_key.sql b/tests/queries/0_stateless/01516_create_table_primary_key.sql index b2b9f288eab..630c573c2cc 100644 --- a/tests/queries/0_stateless/01516_create_table_primary_key.sql +++ b/tests/queries/0_stateless/01516_create_table_primary_key.sql @@ -35,7 +35,7 @@ ATTACH TABLE primary_key_test(v1 Int32, v2 Int32) ENGINE=ReplacingMergeTree ORDE SELECT * FROM primary_key_test FINAL; DROP TABLE primary_key_test; -CREATE TABLE primary_key_test(v1 Int64, v2 Int32, v3 String, PRIMARY KEY(v1, gcd(v1, v2))) ENGINE=ReplacingMergeTree ORDER BY v1; -- { serverError 36; } +CREATE TABLE primary_key_test(v1 Int64, v2 Int32, v3 String, PRIMARY KEY(v1, gcd(v1, v2))) ENGINE=ReplacingMergeTree ORDER BY v1; -- { serverError 36 } CREATE TABLE primary_key_test(v1 Int64, v2 Int32, v3 String, PRIMARY KEY(v1, gcd(v1, v2))) ENGINE=ReplacingMergeTree ORDER BY (v1, gcd(v1, v2)); diff --git a/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql b/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql index 08fba7480d1..ac04178e585 100644 --- a/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql +++ b/tests/queries/0_stateless/01528_allow_nondeterministic_optimize_skip_unused_shards.sql @@ -5,7 +5,7 @@ create table dist_01528 as system.one engine=Distributed('test_cluster_two_shard set optimize_skip_unused_shards=1; set force_optimize_skip_unused_shards=1; -select * from dist_01528 where dummy = 2; -- { serverError 507; } +select * from dist_01528 where dummy = 2; -- { serverError 507 } select * from dist_01528 where dummy = 2 settings allow_nondeterministic_optimize_skip_unused_shards=1; drop table dist_01528; diff --git a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql index 7a2e64742cf..13b4a4e331b 100644 --- a/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql +++ b/tests/queries/0_stateless/01530_drop_database_atomic_sync.sql @@ -30,7 +30,7 @@ create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickho drop database db_01530_atomic; create database db_01530_atomic Engine=Atomic; -create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; -- { serverError 253; } +create table db_01530_atomic.data (key Int) Engine=ReplicatedMergeTree('/clickhouse/tables/{database}/db_01530_atomic/data', 'test') order by key; -- { serverError 253 } set database_atomic_wait_for_drop_and_detach_synchronously=1; diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index fea75e1439f..f19c77c68a3 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -17,7 +17,7 @@ create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect insert into dist_01555 values (1)(2); -- since test_cluster_with_incorrect_pw contains incorrect password ignore error -system flush distributed dist_01555; -- { serverError 516; } +system flush distributed dist_01555; -- { serverError 516 } select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1'), extract(last_exception, 'AUTHENTICATION_FAILED'), dateDiff('s', last_exception_time, now()) < 5 from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; drop table dist_01555; diff --git a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql index 5dc87e31f75..e8411484d71 100644 --- a/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql +++ b/tests/queries/0_stateless/01592_toUnixTimestamp_Date.sql @@ -1 +1 @@ -select toUnixTimestamp(today()); -- { serverError 44; } +select toUnixTimestamp(today()); -- { serverError 44 } diff --git a/tests/queries/0_stateless/01595_countMatches.sql b/tests/queries/0_stateless/01595_countMatches.sql index 6374fe7bc5b..0b170945d44 100644 --- a/tests/queries/0_stateless/01595_countMatches.sql +++ b/tests/queries/0_stateless/01595_countMatches.sql @@ -25,5 +25,5 @@ select countMatchesCaseInsensitive('foo.com BAR.COM baz.com bam.com', '([^. ]+)\ select countMatchesCaseInsensitive('foo.com@foo.com bar.com@foo.com BAZ.com@foo.com bam.com@foo.com', '([^. ]+)\.([^. ]+)@([^. ]+)\.([^. ]+)'); select 'errors'; -select countMatches(1, 'foo') from numbers(1); -- { serverError 43; } -select countMatches('foobarfoo', toString(number)) from numbers(1); -- { serverError 44; } +select countMatches(1, 'foo') from numbers(1); -- { serverError 43 } +select countMatches('foobarfoo', toString(number)) from numbers(1); -- { serverError 44 } diff --git a/tests/queries/0_stateless/01709_inactive_parts_to_throw_insert.sql b/tests/queries/0_stateless/01709_inactive_parts_to_throw_insert.sql index 6de0d4f4e0c..2bb92aec713 100644 --- a/tests/queries/0_stateless/01709_inactive_parts_to_throw_insert.sql +++ b/tests/queries/0_stateless/01709_inactive_parts_to_throw_insert.sql @@ -7,6 +7,6 @@ insert into data_01709 values (2); optimize table data_01709 final; -insert into data_01709 values (3); -- { serverError 252; } +insert into data_01709 values (3); -- { serverError 252 } drop table data_01709; diff --git a/tests/queries/0_stateless/01710_projection_with_mixed_pipeline.sql b/tests/queries/0_stateless/01710_projection_with_mixed_pipeline.sql index 734aa659146..5169c667b81 100644 --- a/tests/queries/0_stateless/01710_projection_with_mixed_pipeline.sql +++ b/tests/queries/0_stateless/01710_projection_with_mixed_pipeline.sql @@ -4,6 +4,6 @@ create table t (x UInt32) engine = MergeTree order by tuple() settings index_gra insert into t select number from numbers(100); alter table t add projection p (select uniqHLL12(x)); insert into t select number + 100 from numbers(100); -select uniqHLL12(x) from t settings allow_experimental_projection_optimization = 1, max_bytes_to_read=400, max_block_size=8; -- { serverError 307; } +select uniqHLL12(x) from t settings allow_experimental_projection_optimization = 1, max_bytes_to_read=400, max_block_size=8; -- { serverError 307 } drop table if exists t; diff --git a/tests/queries/0_stateless/01888_read_int_safe.sql b/tests/queries/0_stateless/01888_read_int_safe.sql index 3aea8e38ab0..197338775c4 100644 --- a/tests/queries/0_stateless/01888_read_int_safe.sql +++ b/tests/queries/0_stateless/01888_read_int_safe.sql @@ -1,10 +1,10 @@ -select toInt64('--1'); -- { serverError 72; } -select toInt64('+-1'); -- { serverError 72; } -select toInt64('++1'); -- { serverError 72; } -select toInt64('++'); -- { serverError 72; } -select toInt64('+'); -- { serverError 72; } -select toInt64('1+1'); -- { serverError 6; } -select toInt64('1-1'); -- { serverError 6; } -select toInt64(''); -- { serverError 32; } +select toInt64('--1'); -- { serverError 72 } +select toInt64('+-1'); -- { serverError 72 } +select toInt64('++1'); -- { serverError 72 } +select toInt64('++'); -- { serverError 72 } +select toInt64('+'); -- { serverError 72 } +select toInt64('1+1'); -- { serverError 6 } +select toInt64('1-1'); -- { serverError 6 } +select toInt64(''); -- { serverError 32 } select toInt64('1'); select toInt64('-1'); diff --git a/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql index 59987a86590..1f6026bb61e 100644 --- a/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql +++ b/tests/queries/0_stateless/02008_tuple_to_name_value_pairs.sql @@ -19,7 +19,7 @@ INSERT INTO test02008 VALUES (tuple(3.3, 5.5, 6.6)); SELECT untuple(arrayJoin(tupleToNameValuePairs(col))) from test02008; DROP TABLE IF EXISTS test02008; -SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43; } -SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43; } -SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43; } -SELECT tupleToNameValuePairs(33); -- { serverError 43; } +SELECT tupleToNameValuePairs(tuple(1, 1.3)); -- { serverError 43 } +SELECT tupleToNameValuePairs(tuple(1, [1,2])); -- { serverError 43 } +SELECT tupleToNameValuePairs(tuple(1, 'a')); -- { serverError 43 } +SELECT tupleToNameValuePairs(33); -- { serverError 43 } From cb3dd3c2009ac2c5ab9e397dbf91675472f0611c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 Mar 2023 13:06:31 +0100 Subject: [PATCH 312/470] Fix flaky test --- tests/integration/test_filesystem_layout/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 898bbc40eb9..2be478f95d0 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -44,8 +44,6 @@ def test_file_path_escaping(started_cluster): ] ) - -def test_file_path_escaping_atomic_db(started_cluster): node.query("CREATE DATABASE IF NOT EXISTS `test 2` ENGINE = Atomic") node.query( """ From aa776d00fec38fc80c4ed584fe46e9d0a82ac3fd Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Mar 2023 13:24:36 +0100 Subject: [PATCH 313/470] Use executors snapshot --- src/Interpreters/ProcessList.cpp | 33 +++++++++++++++++--------------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 37cb9ee599f..bf452775d27 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -397,23 +397,26 @@ CancellationCode QueryStatus::cancelQuery(bool) is_killed.store(true); - std::unique_lock lock(executors_mutex); - for (const auto & e : executors) + std::vector executors_snapshot; + { - /// We should call cancel() with unlocked executors_mutex, because - /// cancel() can try to lock some internal mutex that is already locked by query executing - /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, - /// which will lead to deadlock. - /// Note that the size and the content of executors cannot be changed while - /// executors_mutex is unlocked, because: - /// 1) We don't allow adding new executors while cancelling query in addPipelineExecutor - /// 2) We don't actually remove executor holder from executors in removePipelineExecutor, - /// just mark that executor is invalid. - /// So, it's safe to continue iteration over executors after subsequent mutex locking. - lock.unlock(); - e->cancel(); - lock.lock(); + /// Create a snapshot of executors under a mutex. + std::lock_guard lock(executors_mutex); + executors_snapshot = executors; } + + /// We should call cancel() for each executor with unlocked executors_mutex, because + /// cancel() can try to lock some internal mutex that is already locked by query executing + /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, + /// which will lead to deadlock. + /// Note that the size and the content of executors cannot be changed while + /// executors_mutex is unlocked, because: + /// 1) We don't allow adding new executors while cancelling query in addPipelineExecutor + /// 2) We don't actually remove executor holder from executors in removePipelineExecutor, + /// just mark that executor is invalid. + /// So, it's ok to use a snapshot created above under a mutex, it won't be any differ from actual executors. + for (const auto & e : executors_shapshot) + e->cancel(); return CancellationCode::CancelSent; } From 9117c7491dde2fbe4e01a90f44cd433f5521c9cc Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 Mar 2023 12:55:02 +0000 Subject: [PATCH 314/470] Join threads if exception happened in constructor --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 1fbdd857379..b637bdea835 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -358,12 +358,27 @@ ZooKeeper::ZooKeeper( if (!args.auth_scheme.empty()) sendAuth(args.auth_scheme, args.identity); - send_thread = ThreadFromGlobalPool([this] { sendThread(); }); - receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); + try + { + send_thread = ThreadFromGlobalPool([this] { sendThread(); }); + receive_thread = ThreadFromGlobalPool([this] { receiveThread(); }); - initApiVersion(); + initApiVersion(); - ProfileEvents::increment(ProfileEvents::ZooKeeperInit); + ProfileEvents::increment(ProfileEvents::ZooKeeperInit); + } + catch (...) + { + tryLogCurrentException(log, "Failed to connect to ZooKeeper"); + + if (send_thread.joinable()) + send_thread.join(); + + if (receive_thread.joinable()) + receive_thread.join(); + + throw; + } } From 13bda10470f9e710bf7b6a287f33b8700f34f4a5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Mar 2023 14:14:03 +0100 Subject: [PATCH 315/470] Reimplement hints using the parser --- src/Client/TestHint.cpp | 139 +++++++++--------- src/Client/TestHint.h | 4 +- .../01470_columns_transformers.sql | 6 +- 3 files changed, 78 insertions(+), 71 deletions(-) diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index adaae5fe5ee..c9e845f2039 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -1,55 +1,17 @@ -#include "TestHint.h" +#include +#include + +#include -#include -#include -#include -#include #include +#include +#include namespace DB::ErrorCodes { extern const int CANNOT_PARSE_TEXT; } -namespace -{ - -/// Parse error as number or as a string (name of the error code const) -DB::TestHint::error_vector parseErrorCode(DB::ReadBufferFromString & in) -{ - DB::TestHint::error_vector error_codes{}; - - while (!in.eof()) - { - int code = -1; - String code_name; - auto * pos = in.position(); - - tryReadText(code, in); - if (pos == in.position()) - { - readStringUntilWhitespace(code_name, in); - code = DB::ErrorCodes::getErrorCodeByName(code_name); - } - error_codes.push_back(code); - - if (in.eof()) - break; - skipWhitespaceIfAny(in); - if (in.eof()) - break; - char c; - in.readStrict(c); - if (c != '|') - throw DB::Exception(DB::ErrorCodes::CANNOT_PARSE_TEXT, "Expected separator '|'. Got '{}'", c); - skipWhitespaceIfAny(in); - } - - return error_codes; -} - -} - namespace DB { @@ -81,8 +43,8 @@ TestHint::TestHint(const String & query_) size_t pos_end = comment.find('}', pos_start); if (pos_end != String::npos) { - String hint(comment.begin() + pos_start + 1, comment.begin() + pos_end); - parse(hint, is_leading_hint); + Lexer comment_lexer(comment.c_str() + pos_start + 1, comment.c_str() + pos_end, 0); + parse(comment_lexer, is_leading_hint); } } } @@ -90,33 +52,76 @@ TestHint::TestHint(const String & query_) } } -void TestHint::parse(const String & hint, bool is_leading_hint) +void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) { - ReadBufferFromString in(hint); - String item; + std::unordered_set commands{"echo", "echoOn", "echoOff"}; - while (!in.eof()) + std::unordered_set command_errors{ + "serverError", + "clientError", + }; + + for (Token token = comment_lexer.nextToken(); !token.isEnd(); token = comment_lexer.nextToken()) { - readStringUntilWhitespace(item, in); - if (in.eof()) - break; - - skipWhitespaceIfAny(in); - - if (!is_leading_hint) + String item = String(token.begin, token.end); + if (token.type == TokenType::BareWord && commands.contains(item)) { - if (item == "serverError") - server_errors = parseErrorCode(in); - else if (item == "clientError") - client_errors = parseErrorCode(in); + if (item == "echo") + echo.emplace(true); + if (item == "echoOn") + echo.emplace(true); + if (item == "echoOff") + echo.emplace(false); } + else if (!is_leading_hint && token.type == TokenType::BareWord && command_errors.contains(item)) + { + /// Everything after this must be a list of errors separated by comma + error_vector error_codes; + while (!token.isEnd()) + { + token = comment_lexer.nextToken(); + if (token.type == TokenType::Whitespace) + continue; + if (token.type == TokenType::Number) + { + int code; + auto [p, ec] = std::from_chars(token.begin, token.end, code); + if (p == token.begin) + throw DB::Exception( + DB::ErrorCodes::CANNOT_PARSE_TEXT, + "Could not parse integer number for errorcode: {}", + std::string_view(token.begin, token.end)); + error_codes.push_back(code); + } + else if (token.type == TokenType::BareWord) + { + int code = code = DB::ErrorCodes::getErrorCodeByName(std::string_view(token.begin, token.end)); + error_codes.push_back(code); + } + else + throw DB::Exception( + DB::ErrorCodes::CANNOT_PARSE_TEXT, + "Could not parse error code in {}: {}", + getTokenName(token.type), + std::string_view(token.begin, token.end)); + do + { + token = comment_lexer.nextToken(); + } while (!token.isEnd() && token.type == TokenType::Whitespace); - if (item == "echo") - echo.emplace(true); - if (item == "echoOn") - echo.emplace(true); - if (item == "echoOff") - echo.emplace(false); + if (!token.isEnd() && token.type != TokenType::Comma) + throw DB::Exception( + DB::ErrorCodes::CANNOT_PARSE_TEXT, + "Could not parse error code. Expected ','. Got '{}'", + std::string_view(token.begin, token.end)); + } + + if (item == "serverError") + server_errors = error_codes; + else + client_errors = error_codes; + break; + } } } diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index 30b3cacd3cb..fff82502078 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -11,6 +11,8 @@ namespace DB { +class Lexer; + /// Checks expected server and client error codes. /// /// The following comment hints are supported: @@ -63,7 +65,7 @@ private: error_vector client_errors{}; std::optional echo; - void parse(const String & hint, bool is_leading_hint); + void parse(Lexer & comment_lexer, bool is_leading_hint); bool allErrorsExpected(int actual_server_error, int actual_client_error) const { diff --git a/tests/queries/0_stateless/01470_columns_transformers.sql b/tests/queries/0_stateless/01470_columns_transformers.sql index 7cff1920a4e..8840ce3f3b5 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.sql +++ b/tests/queries/0_stateless/01470_columns_transformers.sql @@ -17,15 +17,15 @@ SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a; SELECT * EXCEPT STRICT i from columns_transformers; SELECT * EXCEPT STRICT (i, j) from columns_transformers; SELECT * EXCEPT STRICT i, j1 from columns_transformers; -- { serverError 47 } -SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } +SELECT * EXCEPT STRICT(i, j1) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE , BAD_ARGUMENTS } SELECT * REPLACE STRICT i + 1 AS i from columns_transformers; -SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } +SELECT * REPLACE STRICT(i + 1 AS col) from columns_transformers; -- { serverError NO_SUCH_COLUMN_IN_TABLE, BAD_ARGUMENTS } SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers; SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 } -- REPLACE after APPLY will not match anything SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a; -SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS } +SELECT a.* APPLY(toDate) REPLACE STRICT(i + 1 AS i) APPLY(any) from columns_transformers a; -- { serverError NO_SUCH_COLUMN_IN_TABLE, BAD_ARGUMENTS } EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers; EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers; From 12525f768cd2375d6605a4cf9e2ebdcdda16dfd0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 Mar 2023 14:18:01 +0100 Subject: [PATCH 316/470] Add default constructor for `MultiReadResponse` (#47254) * Add default constructor for MultiReadResponse * Remove optional * Fix style --- src/Backups/BackupCoordinationRemote.cpp | 18 +++++++-------- src/Common/ZooKeeper/ZooKeeper.h | 28 +++++++++++++++++++++--- 2 files changed, 34 insertions(+), 12 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 029a27d2712..c0fb4d5e066 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -552,7 +552,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const for (auto & batch : batched_escaped_names) { - std::optional sizes_and_checksums; + zkutil::ZooKeeper::MultiGetResponse sizes_and_checksums; { Strings file_names_paths; file_names_paths.reserve(batch.size()); @@ -561,7 +561,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getSizesAndChecksums", zookeeper_retries_info); - retries_ctl.retryLoop([&]() + retries_ctl.retryLoop([&] { auto zk = getZooKeeper(); sizes_and_checksums = zk->get(file_names_paths); @@ -579,9 +579,9 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const for (size_t i = 0; i < batch.size(); ++i) { auto file_name = batch[i]; - if (sizes_and_checksums.value()[i].error != Coordination::Error::ZOK) - throw zkutil::KeeperException(sizes_and_checksums.value()[i].error); - auto size_and_checksum = sizes_and_checksums.value()[i].data; + if (sizes_and_checksums[i].error != Coordination::Error::ZOK) + throw zkutil::KeeperException(sizes_and_checksums[i].error); + const auto & size_and_checksum = sizes_and_checksums[i].data; auto size = deserializeSizeAndChecksum(size_and_checksum).first; if (size) @@ -601,7 +601,7 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const std::move(empty_files_infos.begin(), empty_files_infos.end(), std::back_inserter(file_infos)); } - std::optional non_empty_file_infos_serialized; + zkutil::ZooKeeper::MultiGetResponse non_empty_file_infos_serialized; ZooKeeperRetriesControl retries_ctl("getAllFileInfos::getFileInfos", zookeeper_retries_info); retries_ctl.retryLoop([&]() { @@ -613,9 +613,9 @@ std::vector BackupCoordinationRemote::getAllFileInfos() const for (size_t i = 0; i < non_empty_file_names.size(); ++i) { FileInfo file_info; - if (non_empty_file_infos_serialized.value()[i].error != Coordination::Error::ZOK) - throw zkutil::KeeperException(non_empty_file_infos_serialized.value()[i].error); - file_info = deserializeFileInfo(non_empty_file_infos_serialized.value()[i].data); + if (non_empty_file_infos_serialized[i].error != Coordination::Error::ZOK) + throw zkutil::KeeperException(non_empty_file_infos_serialized[i].error); + file_info = deserializeFileInfo(non_empty_file_infos_serialized[i].data); file_info.file_name = unescapeForFileName(non_empty_file_names[i]); non_empty_files_infos.emplace_back(std::move(file_info)); } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 9de8241cfbe..d20d036f04e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -33,6 +33,12 @@ namespace CurrentMetrics namespace DB { class ZooKeeperLog; + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + } namespace zkutil @@ -79,13 +85,23 @@ concept ZooKeeperResponse = std::derived_from; template struct MultiReadResponses { + MultiReadResponses() = default; + template explicit MultiReadResponses(TResponses responses_) : responses(std::move(responses_)) {} size_t size() const { - return std::visit([](auto && resp) { return resp.size(); }, responses); + return std::visit( + [&](const TResponses & resp) -> size_t + { + if constexpr (std::same_as) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "No responses set for MultiRead"); + else + return resp.size(); + }, + responses); } ResponseType & operator[](size_t index) @@ -94,8 +110,10 @@ struct MultiReadResponses [&](TResponses & resp) -> ResponseType & { if constexpr (std::same_as) + { return dynamic_cast(*resp[index]); - else + } + else if constexpr (std::same_as) { if constexpr (try_multi) { @@ -107,6 +125,10 @@ struct MultiReadResponses } return resp[index]; } + else + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "No responses set for MultiRead"); + } }, responses); } @@ -137,7 +159,7 @@ private: size_t size() const { return future_responses.size(); } }; - std::variant responses; + std::variant responses; }; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. From 7c14059f3f7dd2634e4dcd9336cb5fed9111262e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Mar 2023 14:20:45 +0100 Subject: [PATCH 317/470] Update comments --- src/Client/TestHint.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index fff82502078..8d6b6dfc4e0 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -18,13 +18,13 @@ class Lexer; /// The following comment hints are supported: /// /// - "-- { serverError 60 }" -- in case of you are expecting server error. -/// - "-- { serverError 16 | 36 }" -- in case of you are expecting one of the 2 errors +/// - "-- { serverError 16, 36 }" -- in case of you are expecting one of the 2 errors /// /// - "-- { clientError 20 }" -- in case of you are expecting client error. -/// - "-- { clientError 20 | 60 | 92 }" -- It's expected that the client will return one of the 3 errors. +/// - "-- { clientError 20, 60, 92 }" -- It's expected that the client will return one of the 3 errors. /// /// - "-- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. -/// - "-- { serverError NO_SUCH_COLUMN_IN_TABLE | BAD_ARGUMENTS }" -- by error name. +/// - "-- { serverError NO_SUCH_COLUMN_IN_TABLE, BAD_ARGUMENTS }" -- by error name. /// /// - "-- { clientError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. /// From c13638904af33b71d76e4eae0b91b894588879b9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 6 Mar 2023 14:38:54 +0100 Subject: [PATCH 318/470] Adjust the aggregation interval --- tests/ci/autoscale_runners_lambda/app.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 596e675ee24..7e3af3f6779 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -21,6 +21,11 @@ RUNNER_TYPE_LABELS = [ "style-checker", "style-checker-aarch64", ] + +# 4 HOUR - is a balance to get the most precise values +# - Our longest possible running check is around 5h on the worst scenario +# - The long queue won't be wiped out and replaced, so the measurmenet is fine +# - If the data is spoiled by something, we are from the bills perspective QUEUE_QUERY = f"""SELECT last_status AS status, toUInt32(count()) AS length, @@ -35,7 +40,7 @@ FROM FROM default.workflow_jobs WHERE has(labels, 'self-hosted') AND hasAny({RUNNER_TYPE_LABELS}, labels) - AND started_at > now() - INTERVAL 1 DAY + AND started_at > now() - INTERVAL 4 HOUR GROUP BY ALL HAVING last_status IN ('in_progress', 'queued') ) From 43da841e1e03552e908277cdfb5465f6addbfe62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Mar 2023 14:53:32 +0100 Subject: [PATCH 319/470] Style --- src/Client/TestHint.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index 8d6b6dfc4e0..d9650b9d4d7 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -24,7 +24,7 @@ class Lexer; /// - "-- { clientError 20, 60, 92 }" -- It's expected that the client will return one of the 3 errors. /// /// - "-- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. -/// - "-- { serverError NO_SUCH_COLUMN_IN_TABLE, BAD_ARGUMENTS }" -- by error name. +/// - "-- { serverError NO_SUCH_COLUMN_IN_TABLE, BAD_ARGUMENTS }" -- by error name. /// /// - "-- { clientError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }" -- by error name. /// From e4751f95e1860b52652f2c16d930ab6280d346c6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Mar 2023 14:56:05 +0100 Subject: [PATCH 320/470] Fix typo --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index bf452775d27..a26844ae73c 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -415,7 +415,7 @@ CancellationCode QueryStatus::cancelQuery(bool) /// 2) We don't actually remove executor holder from executors in removePipelineExecutor, /// just mark that executor is invalid. /// So, it's ok to use a snapshot created above under a mutex, it won't be any differ from actual executors. - for (const auto & e : executors_shapshot) + for (const auto & e : executors_snapshot) e->cancel(); return CancellationCode::CancelSent; From 9e3794f7d73b983a7cf4afb1bad9f01b650bfb71 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 6 Mar 2023 14:36:53 +0000 Subject: [PATCH 321/470] Skip for sanitizer builds --- tests/integration/test_keeper_four_word_command/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index d3fcfcc3014..412780c8f0f 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -682,6 +682,9 @@ def test_cmd_rqld(started_cluster): def test_cmd_clrs(started_cluster): + if node1.is_built_with_sanitizer(): + return + def get_memory_purges(): return node1.query( "SELECT value FROM system.events WHERE event = 'MemoryAllocatorPurge' SETTINGS system_events_show_zero_values = 1" From d8e5fb519557041dc03c64026775329720da3d2a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 6 Mar 2023 15:12:15 +0000 Subject: [PATCH 322/470] Use switch, use lz4 by default --- src/Core/Settings.h | 6 ++-- src/Core/SettingsChangesHistory.h | 5 ++- .../Formats/Impl/ORCBlockOutputFormat.cpp | 32 ++++++++--------- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 36 +++++++++---------- 4 files changed, 39 insertions(+), 40 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8eaecff9708..ae5d5326031 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -858,7 +858,7 @@ class IColumn; M(Bool, output_format_parquet_string_as_string, false, "Use Parquet String type instead of Binary for String columns.", 0) \ M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \ M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ - M(ParquetCompression, output_format_parquet_compression_method, "snappy", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ + M(ParquetCompression, output_format_parquet_compression_method, "lz4", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ @@ -901,10 +901,10 @@ class IColumn; M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ M(Bool, output_format_arrow_string_as_string, false, "Use Arrow String type instead of Binary for String columns", 0) \ M(Bool, output_format_arrow_fixed_string_as_fixed_byte_array, true, "Use Arrow FIXED_SIZE_BINARY type instead of Binary for FixedString columns.", 0) \ - M(ArrowCompression, output_format_arrow_compression_method, "none", "Compression method for Arrow output format. Supported codecs: lz4_frame, zstd, none (uncompressed)", 0) \ + M(ArrowCompression, output_format_arrow_compression_method, "lz4_frame", "Compression method for Arrow output format. Supported codecs: lz4_frame, zstd, none (uncompressed)", 0) \ \ M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ - M(ORCCompression, output_format_orc_compression_method, "none", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ + M(ORCCompression, output_format_orc_compression_method, "lz4", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ \ M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 04f328bb665..91b3dff3141 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -81,7 +81,10 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"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_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, + {"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"}}}, {"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"}, diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 39cacde94ed..4264e9da4d6 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -23,7 +23,7 @@ #include namespace DB -{ +{7 namespace ErrorCodes { @@ -36,24 +36,22 @@ namespace orc::CompressionKind getORCCompression(FormatSettings::ORCCompression method) { - if (method == FormatSettings::ORCCompression::NONE) - return orc::CompressionKind::CompressionKind_NONE; - + switch (method) + { + case FormatSettings::ORCCompression::NONE: + return orc::CompressionKind::CompressionKind_NONE; + case FormatSettings::ORCCompression::SNAPPY: #if USE_SNAPPY - if (method == FormatSettings::ORCCompression::SNAPPY) - return orc::CompressionKind::CompressionKind_SNAPPY; + return orc::CompressionKind::CompressionKind_SNAPPY; #endif - - if (method == FormatSettings::ORCCompression::ZSTD) - return orc::CompressionKind::CompressionKind_ZSTD; - - if (method == FormatSettings::ORCCompression::LZ4) - return orc::CompressionKind::CompressionKind_LZ4; - - if (method == FormatSettings::ORCCompression::ZLIB) - return orc::CompressionKind::CompressionKind_ZLIB; - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Snappy compression method is not supported"); + case FormatSettings::ORCCompression::ZSTD: + return orc::CompressionKind::CompressionKind_ZSTD; + case FormatSettings::ORCCompression::LZ4: + return orc::CompressionKind::CompressionKind_LZ4; + case FormatSettings::ORCCompression::ZLIB: + return orc::CompressionKind::CompressionKind_ZLIB; + } } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index cedd8a9c54c..3695bb9d110 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -37,29 +37,27 @@ static parquet::ParquetVersion::type getParquetVersion(const FormatSettings & se parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompression method) { - if (method == FormatSettings::ParquetCompression::NONE) - return parquet::Compression::type::UNCOMPRESSED; - + switch (method) + { + case FormatSettings::ParquetCompression::NONE: + return parquet::Compression::type::UNCOMPRESSED; + case FormatSettings::ParquetCompression::SNAPPY: #if USE_SNAPPY - if (method == FormatSettings::ParquetCompression::SNAPPY) - return parquet::Compression::type::SNAPPY; + return parquet::Compression::type::SNAPPY; #endif - + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Snappy compression method is not supported"); + case FormatSettings::ParquetCompression::BROTLI: #if USE_BROTLI - if (method == FormatSettings::ParquetCompression::BROTLI) - return parquet::Compression::type::BROTLI; + return parquet::Compression::type::BROTLI; #endif - - if (method == FormatSettings::ParquetCompression::ZSTD) - return parquet::Compression::type::ZSTD; - - if (method == FormatSettings::ParquetCompression::LZ4) - return parquet::Compression::type::LZ4; - - if (method == FormatSettings::ParquetCompression::GZIP) - return parquet::Compression::type::GZIP; - - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Brotli compression method is not supported"); + case FormatSettings::ParquetCompression::ZSTD: + return parquet::Compression::type::ZSTD; + case FormatSettings::ParquetCompression::LZ4: + return parquet::Compression::type::LZ4; + case FormatSettings::ParquetCompression::GZIP: + return parquet::Compression::type::GZIP; + } } } From 1b0d0c61082574733c7281a5511184dc98a6a59d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 6 Mar 2023 15:29:13 +0000 Subject: [PATCH 323/470] fix skip indexes --- src/Storages/IndicesDescription.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index 591a9082f7a..a93ac248c98 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -96,8 +96,11 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true); result.sample_block = result.expression->getSampleBlock(); - for (const auto & elem : result.sample_block) + for (auto & elem : result.sample_block) { + if (!elem.column) + elem.column = elem.type->createColumn(); + result.column_names.push_back(elem.name); result.data_types.push_back(elem.type); } From 3076e929802c56fa9e38770f56a02e0622233e02 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 6 Mar 2023 16:56:02 +0100 Subject: [PATCH 324/470] Apply suggestions from code review Co-authored-by: Igor Nikonov <954088+devcrafter@users.noreply.github.com> --- src/Interpreters/GraceHashJoin.h | 1 - src/Processors/Transforms/JoiningTransform.cpp | 5 ++++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index cbc0f2f3266..b8d83f4cad0 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -140,7 +140,6 @@ private: FileBucket * current_bucket = nullptr; - /// Function `getDelayedBlocks` is a critical section, we process only one bucket at a time. mutable std::mutex current_bucket_mutex; InMemoryJoinPtr hash_join; diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 120ff51cad1..bba8ec6fa16 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -328,7 +328,10 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() } if (!output.canPush()) + { + input.setNotNeeded(); return Status::PortFull; + } if (inputs.size() != 1 && outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have exactly one input port"); @@ -419,7 +422,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (output.isFinished()) { /// If at least one output is finished, then we have read all data from buckets. - /// Some workers can still can busy with joining the last chunk of data in memory, + /// Some workers can still be busy with joining the last chunk of data in memory, /// but after that they also will finish when they will try to get next chunk. finished = true; continue; From 8664a20a9e1bcca40616aba9fc7aff3d938a585c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 6 Mar 2023 17:31:27 +0100 Subject: [PATCH 325/470] Fix fast tests --- tests/queries/0_stateless/02480_tlp_nan.reference | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02480_tlp_nan.reference b/tests/queries/0_stateless/02480_tlp_nan.reference index ea4aa44fa89..befd1f66564 100644 --- a/tests/queries/0_stateless/02480_tlp_nan.reference +++ b/tests/queries/0_stateless/02480_tlp_nan.reference @@ -1,10 +1,21 @@ +-- {echo} +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; nan 0 1 0 +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; nan 0 1 0 +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; -inf 0 1 0 +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; -inf 0 1 0 +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; \N \N \N 1 +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; \N \N \N 1 +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; inf 0 1 0 +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; inf 0 1 0 +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; nan 0 1 0 +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; nan 0 1 0 From 4bc443f3c757a7c3c3e61b9699b4cc57c468c178 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 3 Mar 2023 01:27:54 +0300 Subject: [PATCH 326/470] initial solution --- base/base/TypeList.h | 3 +- base/base/TypePair.h | 4 - src/Common/StackTrace.cpp | 516 ++++++++---------- src/Common/StackTrace.h | 13 +- src/Daemon/BaseDaemon.cpp | 2 +- src/DataTypes/DataTypeLowCardinality.cpp | 2 +- src/Functions/FunctionsEmbeddedDictionaries.h | 38 +- src/Functions/array/arrayIntersect.cpp | 8 +- src/Interpreters/CrashLog.cpp | 2 +- 9 files changed, 270 insertions(+), 318 deletions(-) delete mode 100644 base/base/TypePair.h diff --git a/base/base/TypeList.h b/base/base/TypeList.h index 244403b1c6b..310f0c0c586 100644 --- a/base/base/TypeList.h +++ b/base/base/TypeList.h @@ -4,7 +4,6 @@ #include #include #include "defines.h" -#include "TypePair.h" /// General-purpose typelist. Easy on compilation times as it does not use recursion. template @@ -28,7 +27,7 @@ namespace TypeListUtils /// In some contexts it's more handy to use functions in constexpr Root changeRoot(TypeList) { return {}; } template - constexpr void forEach(TypeList, F && f) { (std::forward(f)(Id{}), ...); } + constexpr void forEach(TypeList, F && f) { (std::forward(f)(TypeList{}), ...); } } template diff --git a/base/base/TypePair.h b/base/base/TypePair.h deleted file mode 100644 index 8c2f380618c..00000000000 --- a/base/base/TypePair.h +++ /dev/null @@ -1,4 +0,0 @@ -#pragma once - -template struct TypePair {}; -template struct Id {}; diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 0d47d3dcb92..eb6696e8c05 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -1,18 +1,21 @@ -#include +#include "StackTrace.h" + +#include +#include +#include #include #include -#include #include -#include +#include #include -#include #include +#include #include #include #include -#include +#include #include "config.h" @@ -20,24 +23,23 @@ # include #endif - namespace { - /// Currently this variable is set up once on server startup. - /// But we use atomic just in case, so it is possible to be modified at runtime. - std::atomic show_addresses = true; +/// Currently this variable is set up once on server startup. +/// But we use atomic just in case, so it is possible to be modified at runtime. +std::atomic show_addresses = true; - bool shouldShowAddress(const void * addr) - { - /// If the address is less than 4096, most likely it is a nullptr dereference with offset, - /// and showing this offset is secure nevertheless. - /// NOTE: 4096 is the page size on x86 and it can be different on other systems, - /// but for the purpose of this branch, it does not matter. - if (reinterpret_cast(addr) < 4096) - return true; +bool shouldShowAddress(const void * addr) +{ + /// If the address is less than 4096, most likely it is a nullptr dereference with offset, + /// and showing this offset is secure nevertheless. + /// NOTE: 4096 is the page size on x86 and it can be different on other systems, + /// but for the purpose of this branch, it does not matter. + if (reinterpret_cast(addr) < 4096) + return true; - return show_addresses.load(std::memory_order_relaxed); - } + return show_addresses.load(std::memory_order_relaxed); +} } void StackTrace::setShowAddresses(bool show) @@ -45,155 +47,129 @@ void StackTrace::setShowAddresses(bool show) show_addresses.store(show, std::memory_order_relaxed); } +std::string SigsegvErrorString(const siginfo_t & info, [[maybe_unused]] const ucontext_t & context) +{ + using namespace std::string_literals; + std::string address + = info.si_addr == nullptr ? "NULL pointer"s : (shouldShowAddress(info.si_addr) ? fmt::format("{}", info.si_addr) : ""s); + + const std::string_view access = +#if defined(__x86_64__) && !defined(OS_FREEBSD) && !defined(OS_DARWIN) && !defined(__arm__) && !defined(__powerpc__) + (context.uc_mcontext.gregs[REG_ERR] & 0x02) ? "write" : "read"; +#else + ""; +#endif + + std::string_view message; + + switch (info.si_code) + { + case SEGV_ACCERR: + message = "Attempted access has violated the permissions assigned to the memory area"; + break; + case SEGV_MAPERR: + message = "Address not mapped to object"; + break; + default: + message = "Unknown si_code"; + break; + } + + return fmt::format("Address: {}. Access: {}. {}.", std::move(address), access, message); +} + +constexpr std::string_view SigbusErrorString(int si_code) +{ + switch (si_code) + { + case BUS_ADRALN: + return "Invalid address alignment."; + case BUS_ADRERR: + return "Non-existent physical address."; + case BUS_OBJERR: + return "Object specific hardware error."; + + // Linux specific +#if defined(BUS_MCEERR_AR) + case BUS_MCEERR_AR: + return "Hardware memory error: action required."; +#endif +#if defined(BUS_MCEERR_AO) + case BUS_MCEERR_AO: + return "Hardware memory error: action optional."; +#endif + default: + return "Unknown si_code."; + } +} + +constexpr std::string_view SigfpeErrorString(int si_code) +{ + switch (si_code) + { + case FPE_INTDIV: + return "Integer divide by zero."; + case FPE_INTOVF: + return "Integer overflow."; + case FPE_FLTDIV: + return "Floating point divide by zero."; + case FPE_FLTOVF: + return "Floating point overflow."; + case FPE_FLTUND: + return "Floating point underflow."; + case FPE_FLTRES: + return "Floating point inexact result."; + case FPE_FLTINV: + return "Floating point invalid operation."; + case FPE_FLTSUB: + return "Subscript out of range."; + default: + return "Unknown si_code."; + } +} + +constexpr std::string_view SigillErrorString(int si_code) +{ + switch (si_code) + { + case ILL_ILLOPC: + return "Illegal opcode."; + case ILL_ILLOPN: + return "Illegal operand."; + case ILL_ILLADR: + return "Illegal addressing mode."; + case ILL_ILLTRP: + return "Illegal trap."; + case ILL_PRVOPC: + return "Privileged opcode."; + case ILL_PRVREG: + return "Privileged register."; + case ILL_COPROC: + return "Coprocessor error."; + case ILL_BADSTK: + return "Internal stack error."; + default: + return "Unknown si_code."; + } +} std::string signalToErrorMessage(int sig, const siginfo_t & info, [[maybe_unused]] const ucontext_t & context) { - std::stringstream error; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - error.exceptions(std::ios::failbit); switch (sig) { case SIGSEGV: - { - /// Print info about address and reason. - if (nullptr == info.si_addr) - error << "Address: NULL pointer."; - else if (shouldShowAddress(info.si_addr)) - error << "Address: " << info.si_addr; - -#if defined(__x86_64__) && !defined(OS_FREEBSD) && !defined(OS_DARWIN) && !defined(__arm__) && !defined(__powerpc__) - auto err_mask = context.uc_mcontext.gregs[REG_ERR]; - if ((err_mask & 0x02)) - error << " Access: write."; - else - error << " Access: read."; -#endif - - switch (info.si_code) - { - case SEGV_ACCERR: - error << " Attempted access has violated the permissions assigned to the memory area."; - break; - case SEGV_MAPERR: - error << " Address not mapped to object."; - break; - default: - error << " Unknown si_code."; - break; - } - break; - } - + return SigsegvErrorString(info, context); case SIGBUS: - { - switch (info.si_code) - { - case BUS_ADRALN: - error << "Invalid address alignment."; - break; - case BUS_ADRERR: - error << "Non-existent physical address."; - break; - case BUS_OBJERR: - error << "Object specific hardware error."; - break; - - // Linux specific -#if defined(BUS_MCEERR_AR) - case BUS_MCEERR_AR: - error << "Hardware memory error: action required."; - break; -#endif -#if defined(BUS_MCEERR_AO) - case BUS_MCEERR_AO: - error << "Hardware memory error: action optional."; - break; -#endif - - default: - error << "Unknown si_code."; - break; - } - break; - } - + return std::string{SigbusErrorString(info.si_code)}; case SIGILL: - { - switch (info.si_code) - { - case ILL_ILLOPC: - error << "Illegal opcode."; - break; - case ILL_ILLOPN: - error << "Illegal operand."; - break; - case ILL_ILLADR: - error << "Illegal addressing mode."; - break; - case ILL_ILLTRP: - error << "Illegal trap."; - break; - case ILL_PRVOPC: - error << "Privileged opcode."; - break; - case ILL_PRVREG: - error << "Privileged register."; - break; - case ILL_COPROC: - error << "Coprocessor error."; - break; - case ILL_BADSTK: - error << "Internal stack error."; - break; - default: - error << "Unknown si_code."; - break; - } - break; - } - + return std::string{SigillErrorString(info.si_code)}; case SIGFPE: - { - switch (info.si_code) - { - case FPE_INTDIV: - error << "Integer divide by zero."; - break; - case FPE_INTOVF: - error << "Integer overflow."; - break; - case FPE_FLTDIV: - error << "Floating point divide by zero."; - break; - case FPE_FLTOVF: - error << "Floating point overflow."; - break; - case FPE_FLTUND: - error << "Floating point underflow."; - break; - case FPE_FLTRES: - error << "Floating point inexact result."; - break; - case FPE_FLTINV: - error << "Floating point invalid operation."; - break; - case FPE_FLTSUB: - error << "Subscript out of range."; - break; - default: - error << "Unknown si_code."; - break; - } - break; - } - + return std::string{SigfpeErrorString(info.si_code)}; case SIGTSTP: - { - error << "This is a signal used for debugging purposes by the user."; - break; - } + return "This is a signal used for debugging purposes by the user."; + default: + return ""; } - - return error.str(); } static void * getCallerAddress(const ucontext_t & context) @@ -207,10 +183,8 @@ static void * getCallerAddress(const ucontext_t & context) # else return reinterpret_cast(context.uc_mcontext.gregs[REG_RIP]); # endif - #elif defined(OS_DARWIN) && defined(__aarch64__) return reinterpret_cast(context.uc_mcontext->__ss.__pc); - #elif defined(OS_FREEBSD) && defined(__aarch64__) return reinterpret_cast(context.uc_mcontext.mc_gpregs.gp_elr); #elif defined(__aarch64__) @@ -228,20 +202,17 @@ static void * getCallerAddress(const ucontext_t & context) #endif } +// FIXME: looks like this is used only for Sentry but duplicates the whole algo, maybe replace? void StackTrace::symbolize( - const StackTrace::FramePointers & frame_pointers, [[maybe_unused]] size_t offset, - size_t size, StackTrace::Frames & frames) + const StackTrace::FramePointers & frame_pointers, [[maybe_unused]] size_t offset, size_t size, StackTrace::Frames & frames) { #if defined(__ELF__) && !defined(OS_FREEBSD) - auto symbol_index_ptr = DB::SymbolIndex::instance(); const DB::SymbolIndex & symbol_index = *symbol_index_ptr; std::unordered_map dwarfs; for (size_t i = 0; i < offset; ++i) - { frames[i].virtual_addr = frame_pointers[i]; - } for (size_t i = offset; i < size; ++i) { @@ -254,7 +225,7 @@ void StackTrace::symbolize( if (object) { current_frame.object = object->name; - if (std::filesystem::exists(current_frame.object.value())) + if (std::error_code ec; std::filesystem::exists(current_frame.object.value(), ec) && !ec) { auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; @@ -269,34 +240,19 @@ void StackTrace::symbolize( } } else - { current_frame.object = "?"; - } - const auto * symbol = symbol_index.findSymbol(current_frame.virtual_addr); - if (symbol) - { - int status = 0; - current_frame.symbol = demangle(symbol->name, status); - } + if (const auto * symbol = symbol_index.findSymbol(current_frame.virtual_addr)) + current_frame.symbol = demangle(symbol->name); else - { current_frame.symbol = "?"; - } } #else for (size_t i = 0; i < size; ++i) - { frames[i].virtual_addr = frame_pointers[i]; - } #endif } -StackTrace::StackTrace() -{ - tryCapture(); -} - StackTrace::StackTrace(const ucontext_t & signal_context) { tryCapture(); @@ -325,81 +281,97 @@ StackTrace::StackTrace(const ucontext_t & signal_context) } } -StackTrace::StackTrace(NoCapture) -{ -} - void StackTrace::tryCapture() { - size = 0; #if USE_UNWIND size = unw_backtrace(frame_pointers.data(), capacity); __msan_unpoison(frame_pointers.data(), size * sizeof(frame_pointers[0])); +#else + size = 0; #endif } -size_t StackTrace::getSize() const +// Clickhouse uses bundled libc++ so type names will be same on every system thus it's save to hardcode them +constexpr std::pair replacements[] + = {{"::__1", ""}, {"std::basic_string, std::allocator>", "String"}}; + +String collapseNames(String && haystack) { - return size; + // TODO: surely there is a written version already for better in place search&replace + for (auto [needle, to] : replacements) + { + size_t pos = 0; + while ((pos = haystack.find(needle, pos)) != std::string::npos) + { + haystack.replace(pos, needle.length(), to); + pos += to.length(); + } + } + + return haystack; } -size_t StackTrace::getOffset() const +struct StackTraceRefTriple { - return offset; + const StackTrace::FramePointers & pointers; + size_t offset; + size_t size; +}; + +struct StackTraceTriple +{ + StackTrace::FramePointers pointers; + size_t offset; + size_t size; +}; + +template +concept MaybeRef = std::is_same_v || std::is_same_v; + +constexpr bool operator<(const MaybeRef auto & left, const MaybeRef auto & right) +{ + return std::tuple{left.pointers, left.size, left.offset} < std::tuple{right.pointers, right.size, right.offset}; } -const StackTrace::FramePointers & StackTrace::getFramePointers() const +static void +toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & stack_trace, Fn auto && callback) { - return frame_pointers; -} - -static void toStringEveryLineImpl( - [[maybe_unused]] bool fatal, - const StackTrace::FramePointers & frame_pointers, - size_t offset, - size_t size, - std::function callback) -{ - if (size == 0) + if (stack_trace.size == 0) return callback(""); #if defined(__ELF__) && !defined(OS_FREEBSD) - auto symbol_index_ptr = DB::SymbolIndex::instance(); - const DB::SymbolIndex & symbol_index = *symbol_index_ptr; - std::unordered_map dwarfs; - - std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out.exceptions(std::ios::failbit); - for (size_t i = offset; i < size; ++i) + using enum DB::Dwarf::LocationInfoMode; + const auto mode = fatal ? FULL_WITH_INLINE : FAST; + + auto symbol_index_ptr = DB::SymbolIndex::instance(); + const DB::SymbolIndex & symbol_index = *symbol_index_ptr; + std::unordered_map dwarfs; + + for (size_t i = stack_trace.offset; i < stack_trace.size; ++i) { std::vector inline_frames; - const void * virtual_addr = frame_pointers[i]; + const void * virtual_addr = stack_trace.pointers[i]; const auto * object = symbol_index.findObject(virtual_addr); uintptr_t virtual_offset = object ? uintptr_t(object->address_begin) : 0; const void * physical_addr = reinterpret_cast(uintptr_t(virtual_addr) - virtual_offset); out << i << ". "; - if (object) + if (std::error_code ec; object && std::filesystem::exists(object->name, ec) && !ec) { - if (std::filesystem::exists(object->name)) - { - auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; + auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; - DB::Dwarf::LocationInfo location; - auto mode = fatal ? DB::Dwarf::LocationInfoMode::FULL_WITH_INLINE : DB::Dwarf::LocationInfoMode::FAST; - if (dwarf_it->second.findAddress(uintptr_t(physical_addr), location, mode, inline_frames)) - out << location.file.toString() << ":" << location.line << ": "; - } + DB::Dwarf::LocationInfo location; + + if (dwarf_it->second.findAddress(uintptr_t(physical_addr), location, mode, inline_frames)) + out << location.file.toString() << ":" << location.line << ": "; } - const auto * symbol = symbol_index.findSymbol(virtual_addr); - if (symbol) - { - int status = 0; - out << demangle(symbol->name, status); - } + if (const auto * const symbol = symbol_index.findSymbol(virtual_addr)) + out << collapseNames(demangle(symbol->name)); else out << "?"; @@ -411,64 +383,31 @@ static void toStringEveryLineImpl( for (size_t j = 0; j < inline_frames.size(); ++j) { const auto & frame = inline_frames[j]; - int status = 0; - callback(fmt::format("{}.{}. inlined from {}:{}: {}", - i, j+1, frame.location.file.toString(), frame.location.line, demangle(frame.name, status))); + callback(fmt::format( + "{}.{}. inlined from {}:{}: {}", + i, + j + 1, + frame.location.file.toString(), + frame.location.line, + collapseNames(demangle(frame.name)))); } callback(out.str()); out.str({}); } #else - std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - out.exceptions(std::ios::failbit); - - for (size_t i = offset; i < size; ++i) - { - const void * addr = frame_pointers[i]; - if (shouldShowAddress(addr)) - { - out << i << ". " << addr; - - callback(out.str()); - out.str({}); - } - } + for (size_t i = stack_trace.offset; i < stack_trace.size; ++i) + if (const void * const addr = stack_trace.pointers[i]; shouldShowAddress(addr)) + callback(fmt::format("{}. {}", i, addr)); #endif } -static std::string toStringImpl(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size) +void StackTrace::toStringEveryLine(std::function callback) const { - std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - out.exceptions(std::ios::failbit); - toStringEveryLineImpl(false, frame_pointers, offset, size, [&](const std::string & str) { out << str << '\n'; }); - return out.str(); + toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback)); } -void StackTrace::toStringEveryLine(std::function callback) const -{ - toStringEveryLineImpl(true, frame_pointers, offset, size, std::move(callback)); -} - - -std::string StackTrace::toString() const -{ - return toStringStatic(frame_pointers, offset, size); -} - -std::string StackTrace::toString(void ** frame_pointers_, size_t offset, size_t size) -{ - __msan_unpoison(frame_pointers_, size * sizeof(*frame_pointers_)); - - StackTrace::FramePointers frame_pointers_copy{}; - for (size_t i = 0; i < size; ++i) - frame_pointers_copy[i] = frame_pointers_[i]; - - return toStringStatic(frame_pointers_copy, offset, size); -} - -using StackTraceRepresentation = std::tuple; -using StackTraceCache = std::map; +using StackTraceCache = std::map>; static StackTraceCache & cacheInstance() { @@ -478,21 +417,40 @@ static StackTraceCache & cacheInstance() static std::mutex stacktrace_cache_mutex; -std::string StackTrace::toStringStatic(const StackTrace::FramePointers & frame_pointers, size_t offset, size_t size) +String toStringCached(const StackTrace::FramePointers & pointers, size_t offset, size_t size) { /// Calculation of stack trace text is extremely slow. /// We use simple cache because otherwise the server could be overloaded by trash queries. /// Note that this cache can grow unconditionally, but practically it should be small. std::lock_guard lock{stacktrace_cache_mutex}; - StackTraceRepresentation key{frame_pointers, offset, size}; - auto & cache = cacheInstance(); - if (cache.contains(key)) - return cache[key]; + StackTraceCache & cache = cacheInstance(); - auto result = toStringImpl(frame_pointers, offset, size); - cache[key] = result; - return result; + if (auto it = cache.find(StackTraceRefTriple{pointers, offset, size}); it != cache.end()) + return it->second; + else + { + std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + out.exceptions(std::ios::failbit); + toStringEveryLineImpl(false, {pointers, offset, size}, [&](std::string_view str) { out << str << '\n'; }); + + return cache.emplace(StackTraceTriple{pointers, offset, size}, out.str()).first->second; + } +} + +std::string StackTrace::toString() const +{ + return toStringCached(frame_pointers, offset, size); +} + +std::string StackTrace::toString(void ** frame_pointers_raw, size_t offset, size_t size) +{ + __msan_unpoison(frame_pointers_raw, size * sizeof(*frame_pointers_raw)); + + StackTrace::FramePointers frame_pointers; + std::copy_n(frame_pointers_raw, size, frame_pointers.begin()); + + return toStringCached(frame_pointers, offset, size); } void StackTrace::dropCache() diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index f07c05107ee..3940c880c5b 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -46,26 +46,25 @@ public: using Frames = std::array; /// Tries to capture stack trace - StackTrace(); + inline StackTrace() { tryCapture(); } /// Tries to capture stack trace. Fallbacks on parsing caller address from /// signal context if no stack trace could be captured explicit StackTrace(const ucontext_t & signal_context); /// Creates empty object for deferred initialization - explicit StackTrace(NoCapture); + explicit inline StackTrace(NoCapture) {} - size_t getSize() const; - size_t getOffset() const; - const FramePointers & getFramePointers() const; + constexpr size_t getSize() const { return size; } + constexpr size_t getOffset() const { return offset; } + const FramePointers & getFramePointers() const { return frame_pointers; } std::string toString() const; static std::string toString(void ** frame_pointers, size_t offset, size_t size); - static std::string toStringStatic(const FramePointers & frame_pointers, size_t offset, size_t size); static void dropCache(); static void symbolize(const FramePointers & frame_pointers, size_t offset, size_t size, StackTrace::Frames & frames); - void toStringEveryLine(std::function callback) const; + void toStringEveryLine(std::function callback) const; /// Displaying the addresses can be disabled for security reasons. /// If you turn off addresses, it will be more secure, but we will be unable to help you with debugging. diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index fbfa9e68774..18c4c0d97a0 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -374,7 +374,7 @@ private: } /// Write symbolized stack trace line by line for better grep-ability. - stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, fmt::runtime(s)); }); + stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index 5e3a1cd3a0e..8293455cabc 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -55,7 +55,7 @@ namespace } template - void operator()(Id) + void operator()(TypeList) { if (typeid_cast *>(&keys_type)) column = creator(static_cast *>(nullptr)); diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index e54ab0277d5..af040c6ab93 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -166,12 +166,12 @@ public: if (arguments[0]->getName() != TypeName) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {} (must be {})", - arguments[0]->getName(), getName(), String(TypeName)); + arguments[0]->getName(), getName(), TypeName); - if (arguments.size() == 2 && arguments[1]->getName() != TypeName) + if (arguments.size() == 2 && arguments[1]->getName() != "String") throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second ('point of view') argument of function {} (must be {})", - arguments[1]->getName(), getName(), String(TypeName)); + "Illegal type {} of the second ('point of view') argument of function {} (must be String)", + arguments[1]->getName(), getName()); return arguments[0]; } @@ -257,16 +257,16 @@ public: if (arguments[0]->getName() != TypeName) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {} (must be {})", - arguments[0]->getName(), getName(), String(TypeName)); + arguments[0]->getName(), getName(), TypeName); if (arguments[1]->getName() != TypeName) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {} (must be {})", - arguments[1]->getName(), getName(), String(TypeName)); + arguments[1]->getName(), getName(), TypeName); - if (arguments.size() == 3 && arguments[2]->getName() != TypeName) + if (arguments.size() == 3 && arguments[2]->getName() != "String") throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the third ('point of view') argument of function {} (must be {})", - arguments[2]->getName(), getName(), String(TypeName)); + "Illegal type {} of the third ('point of view') argument of function {} (must be String)", + arguments[2]->getName(), getName()); return std::make_shared(); } @@ -390,12 +390,12 @@ public: if (arguments[0]->getName() != TypeName) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {} (must be {})", - arguments[0]->getName(), getName(), String(TypeName)); + arguments[0]->getName(), getName(), TypeName); - if (arguments.size() == 2 && arguments[1]->getName() != TypeName) + if (arguments.size() == 2 && arguments[1]->getName() != "String") throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second ('point of view') argument of function {} (must be {})", - arguments[1]->getName(), getName(), String(TypeName)); + "Illegal type {} of the second ('point of view') argument of function {} (must be String)", + arguments[1]->getName(), getName()); return std::make_shared(arguments[0]); } @@ -591,15 +591,15 @@ public: "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2.", getName(), arguments.size()); - if (arguments[0]->getName() != TypeName) + if (arguments[0]->getName() != "UInt32") throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the first argument of function {} (must be {})", - arguments[0]->getName(), getName(), String(TypeName)); + "Illegal type {} of the first argument of function {} (must be UInt32)", + arguments[0]->getName(), getName()); - if (arguments.size() == 2 && arguments[1]->getName() != TypeName) + if (arguments.size() == 2 && arguments[1]->getName() != "String") throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the second argument of function {} (must be {})", - arguments[0]->getName(), getName(), String(TypeName)); + "Illegal type {} of the second argument of function {} (must be String)", + arguments[0]->getName(), getName()); return std::make_shared(); } diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 1a718595a3b..c6f0a5afa62 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -107,7 +107,7 @@ private: : arrays(arrays_), data_type(data_type_), result(result_) {} template - void operator()(Id); + void operator()(TypeList); }; struct DecimalExecutor @@ -120,7 +120,7 @@ private: : arrays(arrays_), data_type(data_type_), result(result_) {} template - void operator()(Id); + void operator()(TypeList); }; }; @@ -446,7 +446,7 @@ ColumnPtr FunctionArrayIntersect::executeImpl(const ColumnsWithTypeAndName & arg } template -void FunctionArrayIntersect::NumberExecutor::operator()(Id) +void FunctionArrayIntersect::NumberExecutor::operator()(TypeList) { using Container = ClearableHashMapWithStackMemory, INITIAL_SIZE_DEGREE>; @@ -456,7 +456,7 @@ void FunctionArrayIntersect::NumberExecutor::operator()(Id) } template -void FunctionArrayIntersect::DecimalExecutor::operator()(Id) +void FunctionArrayIntersect::DecimalExecutor::operator()(TypeList) { using Container = ClearableHashMapWithStackMemory, INITIAL_SIZE_DEGREE>; diff --git a/src/Interpreters/CrashLog.cpp b/src/Interpreters/CrashLog.cpp index 47a9d62fba6..f1f0ffb6f60 100644 --- a/src/Interpreters/CrashLog.cpp +++ b/src/Interpreters/CrashLog.cpp @@ -80,7 +80,7 @@ void collectCrashLog(Int32 signal, UInt64 thread_id, const String & query_id, co for (size_t i = stack_trace_offset; i < stack_trace_size; ++i) trace.push_back(reinterpret_cast(stack_trace.getFramePointers()[i])); - stack_trace.toStringEveryLine([&trace_full](const std::string & line) { trace_full.push_back(line); }); + stack_trace.toStringEveryLine([&trace_full](std::string_view line) { trace_full.push_back(line); }); CrashLogElement element{static_cast(time / 1000000000), time, signal, thread_id, query_id, trace, trace_full}; crash_log_owned->add(element); From 07a9e486e6aafa184d7a3ea244db0758137128c7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Sat, 4 Mar 2023 21:37:28 +0300 Subject: [PATCH 327/470] memset for frame pointers' --- src/Common/StackTrace.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index eb6696e8c05..0aebaf9801c 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -425,14 +425,15 @@ String toStringCached(const StackTrace::FramePointers & pointers, size_t offset, std::lock_guard lock{stacktrace_cache_mutex}; StackTraceCache & cache = cacheInstance(); + const StackTraceRefTriple key{pointers, offset, size}; - if (auto it = cache.find(StackTraceRefTriple{pointers, offset, size}); it != cache.end()) + if (auto it = cache.find(key); it != cache.end()) return it->second; else { - std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::ostringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM out.exceptions(std::ios::failbit); - toStringEveryLineImpl(false, {pointers, offset, size}, [&](std::string_view str) { out << str << '\n'; }); + toStringEveryLineImpl(false, key, [&](std::string_view str) { out << str << '\n'; }); return cache.emplace(StackTraceTriple{pointers, offset, size}, out.str()).first->second; } @@ -447,7 +448,7 @@ std::string StackTrace::toString(void ** frame_pointers_raw, size_t offset, size { __msan_unpoison(frame_pointers_raw, size * sizeof(*frame_pointers_raw)); - StackTrace::FramePointers frame_pointers; + StackTrace::FramePointers frame_pointers{}; std::copy_n(frame_pointers_raw, size, frame_pointers.begin()); return toStringCached(frame_pointers, offset, size); From e50fd6e1ef67eaf7c69e173b9a4486c90e598f1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 6 Mar 2023 16:35:17 +0000 Subject: [PATCH 328/470] Update version_date.tsv and changelogs after v23.2.3.17-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.2.3.17-stable.md | 23 +++++++++++++++++++++++ utils/list-versions/version_date.tsv | 5 +++++ 5 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.2.3.17-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 09395befdad..500249b5bd6 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.3.17" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 472f25eed2d..5f613eea4d0 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.3.17" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 5dbb244c298..3e99bca8bc2 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.2.1.2537" +ARG VERSION="23.2.3.17" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.2.3.17-stable.md b/docs/changelogs/v23.2.3.17-stable.md new file mode 100644 index 00000000000..fb2c4e394dc --- /dev/null +++ b/docs/changelogs/v23.2.3.17-stable.md @@ -0,0 +1,23 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.3.17-stable (dec18bf7281) FIXME as compared to v23.2.2.20-stable (f6c269c8df2) + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#46907](https://github.com/ClickHouse/ClickHouse/issues/46907): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#47091](https://github.com/ClickHouse/ClickHouse/issues/47091): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#46885](https://github.com/ClickHouse/ClickHouse/issues/46885): Fix MSan report in the `maxIntersections` function. This closes [#43126](https://github.com/ClickHouse/ClickHouse/issues/43126). [#46847](https://github.com/ClickHouse/ClickHouse/pull/46847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#47067](https://github.com/ClickHouse/ClickHouse/issues/47067): Fix typo in systemd service, which causes the systemd service start to fail. [#47051](https://github.com/ClickHouse/ClickHouse/pull/47051) ([Palash Goel](https://github.com/palash-goel)). +* Backported in [#47259](https://github.com/ClickHouse/ClickHouse/issues/47259): Fix concrete columns PREWHERE support. [#47154](https://github.com/ClickHouse/ClickHouse/pull/47154) ([Azat Khuzhin](https://github.com/azat)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* do flushUntrackedMemory when context switches [#47102](https://github.com/ClickHouse/ClickHouse/pull/47102) ([Sema Checherinda](https://github.com/CheSema)). +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3814e94bf24..34bc3f646fc 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,7 +1,11 @@ +v23.2.3.17-stable 2023-03-06 +v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 +v22.12.4.76-stable 2023-03-01 v22.12.3.5-stable 2023-01-10 v22.12.2.25-stable 2023-01-06 v22.12.1.1752-stable 2022-12-15 @@ -25,6 +29,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 v22.8.11.15-lts 2022-12-08 From df34ab240d19d68226b230413d0ae023d98686f7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Mar 2023 18:03:17 +0100 Subject: [PATCH 329/470] Don't use switch --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 4264e9da4d6..7fae2c8f480 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -36,22 +36,24 @@ namespace orc::CompressionKind getORCCompression(FormatSettings::ORCCompression method) { - switch (method) - { - case FormatSettings::ORCCompression::NONE: - return orc::CompressionKind::CompressionKind_NONE; - case FormatSettings::ORCCompression::SNAPPY: + if (method == FormatSettings::ORCCompression::NONE) + return orc::CompressionKind::CompressionKind_NONE; + #if USE_SNAPPY - return orc::CompressionKind::CompressionKind_SNAPPY; + if (method == FormatSettings::ORCCompression::SNAPPY) + return orc::CompressionKind::CompressionKind_SNAPPY; #endif - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Snappy compression method is not supported"); - case FormatSettings::ORCCompression::ZSTD: - return orc::CompressionKind::CompressionKind_ZSTD; - case FormatSettings::ORCCompression::LZ4: - return orc::CompressionKind::CompressionKind_LZ4; - case FormatSettings::ORCCompression::ZLIB: - return orc::CompressionKind::CompressionKind_ZLIB; - } + + if (method == FormatSettings::ORCCompression::ZSTD) + return orc::CompressionKind::CompressionKind_ZSTD; + + if (method == FormatSettings::ORCCompression::LZ4) + return orc::CompressionKind::CompressionKind_LZ4; + + if (method == FormatSettings::ORCCompression::ZLIB) + return orc::CompressionKind::CompressionKind_ZLIB; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); } } From fed645a4945efab6aa9a2b98ac2861de483be158 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Mar 2023 18:03:54 +0100 Subject: [PATCH 330/470] Don't use switch --- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 3695bb9d110..cedd8a9c54c 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -37,27 +37,29 @@ static parquet::ParquetVersion::type getParquetVersion(const FormatSettings & se parquet::Compression::type getParquetCompression(FormatSettings::ParquetCompression method) { - switch (method) - { - case FormatSettings::ParquetCompression::NONE: - return parquet::Compression::type::UNCOMPRESSED; - case FormatSettings::ParquetCompression::SNAPPY: + if (method == FormatSettings::ParquetCompression::NONE) + return parquet::Compression::type::UNCOMPRESSED; + #if USE_SNAPPY - return parquet::Compression::type::SNAPPY; + if (method == FormatSettings::ParquetCompression::SNAPPY) + return parquet::Compression::type::SNAPPY; #endif - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Snappy compression method is not supported"); - case FormatSettings::ParquetCompression::BROTLI: + #if USE_BROTLI - return parquet::Compression::type::BROTLI; + if (method == FormatSettings::ParquetCompression::BROTLI) + return parquet::Compression::type::BROTLI; #endif - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Brotli compression method is not supported"); - case FormatSettings::ParquetCompression::ZSTD: - return parquet::Compression::type::ZSTD; - case FormatSettings::ParquetCompression::LZ4: - return parquet::Compression::type::LZ4; - case FormatSettings::ParquetCompression::GZIP: - return parquet::Compression::type::GZIP; - } + + if (method == FormatSettings::ParquetCompression::ZSTD) + return parquet::Compression::type::ZSTD; + + if (method == FormatSettings::ParquetCompression::LZ4) + return parquet::Compression::type::LZ4; + + if (method == FormatSettings::ParquetCompression::GZIP) + return parquet::Compression::type::GZIP; + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); } } From b5dffe7417527bb11ef5979133e092ec1ca222db Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 6 Mar 2023 19:57:09 +0100 Subject: [PATCH 331/470] Fix bug in zero copy replica which can lead to dataloss --- src/Storages/StorageReplicatedMergeTree.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 54ae8aa5a7b..9e79a715610 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8434,7 +8434,11 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else if (error_code == Coordination::Error::ZNONODE) { - LOG_TRACE(logger, "Node with parent zookeeper lock {} for part {} doesn't exist (part was unlocked before)", zookeeper_part_uniq_node, part_name); + /// We don't know what to do, because this part can be mutation part + /// with hardlinked columns. Since we don't have this information (about blobs not to remove) + /// we refuce to remove blobs. + LOG_WARNING(logger, "Node with parent zookeeper lock {} for part {} doesn't exist (part was unlocked before), refuse to remove blobs", zookeeper_part_uniq_node, part_name); + return std::make_pair{false, {}}; } else { From b298af2640f28a0ae440d36a59e72d2dedaa77b0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 15:56:21 -0400 Subject: [PATCH 332/470] Update exponentialmovingaverage.md --- .../reference/exponentialmovingaverage.md | 36 +++++++++++++++++-- 1 file changed, 33 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index 2587bc5533f..e99addf187c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -11,15 +11,15 @@ sidebar_title: exponentialMovingAverage **Syntax** ```sql -exponentialMovingAverage(x)(value, timestamp) +exponentialMovingAverage(x)(value, timeunit) ``` -Each `value` corresponds to the determinate `timestamp`. The half-life `x` is the time lag at which the exponential weights decay by one-half. The function returns a weighted average: the older the time point, the less weight the corresponding value is considered to be. +Each `value` corresponds to the determinate `timeunit`. The half-life `x` is the time lag at which the exponential weights decay by one-half. The function returns a weighted average: the older the time point, the less weight the corresponding value is considered to be. **Arguments** - `value` — Value. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). -- `timestamp` — Timestamp. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). +- `timeunit` — Timeunit. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). Timeunit is not timestamp (seconds), it's -- an index of the time interval. Can be calculated using [intDiv](../../unctions/arithmetic-functions/#intdiva-b). **Parameters** @@ -148,3 +148,33 @@ Result: │ 1 │ 49 │ 0.825 │ █████████████████████████████████████████▎│ └───────┴──────┴──────────────────────┴────────────────────────────────────────────┘ ``` + +```sql +CREATE TABLE data +ENGINE = Memory AS +SELECT + 10 AS value, + toDateTime('2020-01-01') + (3600 * number) AS time +FROM numbers_mt(10); + + +SELECT + value, + time, + exponentialMovingAverage(1)(value, intDiv(toUInt32(time),3600)) OVER (ORDER BY time ASC) res +FROM data +ORDER BY time; + +┌─value─┬────────────────time─┬─────────res─┐ +│ 10 │ 2020-01-01 00:00:00 │ 5 │ +│ 10 │ 2020-01-01 01:00:00 │ 7.5 │ +│ 10 │ 2020-01-01 02:00:00 │ 8.75 │ +│ 10 │ 2020-01-01 03:00:00 │ 9.375 │ +│ 10 │ 2020-01-01 04:00:00 │ 9.6875 │ +│ 10 │ 2020-01-01 05:00:00 │ 9.84375 │ +│ 10 │ 2020-01-01 06:00:00 │ 9.921875 │ +│ 10 │ 2020-01-01 07:00:00 │ 9.9609375 │ +│ 10 │ 2020-01-01 08:00:00 │ 9.98046875 │ +│ 10 │ 2020-01-01 09:00:00 │ 9.990234375 │ +└───────┴─────────────────────┴─────────────┘ +``` From 19dfd2eb17f9aa162801835304a77983e42f8120 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 15:59:24 -0400 Subject: [PATCH 333/470] Update exponentialmovingaverage.md --- .../reference/exponentialmovingaverage.md | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index e99addf187c..a789a149d1e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -177,4 +177,25 @@ ORDER BY time; │ 10 │ 2020-01-01 08:00:00 │ 9.98046875 │ │ 10 │ 2020-01-01 09:00:00 │ 9.990234375 │ └───────┴─────────────────────┴─────────────┘ + + +SELECT + value, + time, + round(exponentialMovingAverage(2)(value, toRelativeHourNum(time)) OVER (ORDER BY time ASC), 2) AS res +FROM data +ORDER BY time ASC + +┌─value─┬────────────────time─┬──res─┐ +│ 10 │ 2020-01-01 00:00:00 │ 2.93 │ +│ 10 │ 2020-01-01 01:00:00 │ 5 │ +│ 10 │ 2020-01-01 02:00:00 │ 6.46 │ +│ 10 │ 2020-01-01 03:00:00 │ 7.5 │ +│ 10 │ 2020-01-01 04:00:00 │ 8.23 │ +│ 10 │ 2020-01-01 05:00:00 │ 8.75 │ +│ 10 │ 2020-01-01 06:00:00 │ 9.12 │ +│ 10 │ 2020-01-01 07:00:00 │ 9.37 │ +│ 10 │ 2020-01-01 08:00:00 │ 9.56 │ +│ 10 │ 2020-01-01 09:00:00 │ 9.69 │ +└───────┴─────────────────────┴──────┘ ``` From 683da4897ca5bd094089588ac7d8e61319d4e9aa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 16:04:36 -0400 Subject: [PATCH 334/470] Update exponentialmovingaverage.md --- .../reference/exponentialmovingaverage.md | 60 ++++++++++--------- 1 file changed, 32 insertions(+), 28 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index a789a149d1e..26507bedb31 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -158,44 +158,48 @@ SELECT FROM numbers_mt(10); +-- Calculate timeunit using intDiv SELECT value, time, - exponentialMovingAverage(1)(value, intDiv(toUInt32(time),3600)) OVER (ORDER BY time ASC) res + exponentialMovingAverage(1)(value, intDiv(toUInt32(time), 3600)) OVER (ORDER BY time ASC) AS res, + intDiv(toUInt32(time), 3600) AS timeunit FROM data -ORDER BY time; +ORDER BY time ASC; -┌─value─┬────────────────time─┬─────────res─┐ -│ 10 │ 2020-01-01 00:00:00 │ 5 │ -│ 10 │ 2020-01-01 01:00:00 │ 7.5 │ -│ 10 │ 2020-01-01 02:00:00 │ 8.75 │ -│ 10 │ 2020-01-01 03:00:00 │ 9.375 │ -│ 10 │ 2020-01-01 04:00:00 │ 9.6875 │ -│ 10 │ 2020-01-01 05:00:00 │ 9.84375 │ -│ 10 │ 2020-01-01 06:00:00 │ 9.921875 │ -│ 10 │ 2020-01-01 07:00:00 │ 9.9609375 │ -│ 10 │ 2020-01-01 08:00:00 │ 9.98046875 │ -│ 10 │ 2020-01-01 09:00:00 │ 9.990234375 │ -└───────┴─────────────────────┴─────────────┘ +┌─value─┬────────────────time─┬─────────res─┬─timeunit─┐ +│ 10 │ 2020-01-01 00:00:00 │ 5 │ 438288 │ +│ 10 │ 2020-01-01 01:00:00 │ 7.5 │ 438289 │ +│ 10 │ 2020-01-01 02:00:00 │ 8.75 │ 438290 │ +│ 10 │ 2020-01-01 03:00:00 │ 9.375 │ 438291 │ +│ 10 │ 2020-01-01 04:00:00 │ 9.6875 │ 438292 │ +│ 10 │ 2020-01-01 05:00:00 │ 9.84375 │ 438293 │ +│ 10 │ 2020-01-01 06:00:00 │ 9.921875 │ 438294 │ +│ 10 │ 2020-01-01 07:00:00 │ 9.9609375 │ 438295 │ +│ 10 │ 2020-01-01 08:00:00 │ 9.98046875 │ 438296 │ +│ 10 │ 2020-01-01 09:00:00 │ 9.990234375 │ 438297 │ +└───────┴─────────────────────┴─────────────┴──────────┘ +-- Calculate timeunit timeunit using toRelativeHourNum SELECT value, time, - round(exponentialMovingAverage(2)(value, toRelativeHourNum(time)) OVER (ORDER BY time ASC), 2) AS res + exponentialMovingAverage(1)(value, toRelativeHourNum(time)) OVER (ORDER BY time ASC) AS res, + toRelativeHourNum(time) AS timeunit FROM data -ORDER BY time ASC +ORDER BY time ASC; -┌─value─┬────────────────time─┬──res─┐ -│ 10 │ 2020-01-01 00:00:00 │ 2.93 │ -│ 10 │ 2020-01-01 01:00:00 │ 5 │ -│ 10 │ 2020-01-01 02:00:00 │ 6.46 │ -│ 10 │ 2020-01-01 03:00:00 │ 7.5 │ -│ 10 │ 2020-01-01 04:00:00 │ 8.23 │ -│ 10 │ 2020-01-01 05:00:00 │ 8.75 │ -│ 10 │ 2020-01-01 06:00:00 │ 9.12 │ -│ 10 │ 2020-01-01 07:00:00 │ 9.37 │ -│ 10 │ 2020-01-01 08:00:00 │ 9.56 │ -│ 10 │ 2020-01-01 09:00:00 │ 9.69 │ -└───────┴─────────────────────┴──────┘ +┌─value─┬────────────────time─┬─────────res─┬─timeunit─┐ +│ 10 │ 2020-01-01 00:00:00 │ 5 │ 438288 │ +│ 10 │ 2020-01-01 01:00:00 │ 7.5 │ 438289 │ +│ 10 │ 2020-01-01 02:00:00 │ 8.75 │ 438290 │ +│ 10 │ 2020-01-01 03:00:00 │ 9.375 │ 438291 │ +│ 10 │ 2020-01-01 04:00:00 │ 9.6875 │ 438292 │ +│ 10 │ 2020-01-01 05:00:00 │ 9.84375 │ 438293 │ +│ 10 │ 2020-01-01 06:00:00 │ 9.921875 │ 438294 │ +│ 10 │ 2020-01-01 07:00:00 │ 9.9609375 │ 438295 │ +│ 10 │ 2020-01-01 08:00:00 │ 9.98046875 │ 438296 │ +│ 10 │ 2020-01-01 09:00:00 │ 9.990234375 │ 438297 │ +└───────┴─────────────────────┴─────────────┴──────────┘ ``` From 0b46de317b8b6217b9e920f494509b656f102c21 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 16:33:28 -0400 Subject: [PATCH 335/470] Update ip-address-functions.md --- docs/en/sql-reference/functions/ip-address-functions.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 3843ca0fc36..cbb93ee15d7 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -286,6 +286,14 @@ Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns 0. Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns null. +## toIPv6OrDefault(string) + +Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns 0. + +## toIPv6OrNull(string) + +Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns null. + ## toIPv6 Converts a string form of IPv6 address to [IPv6](../../sql-reference/data-types/domains/ipv6.md) type. If the IPv6 address has an invalid format, returns an empty value. From d8029b7b2d092c218668826abcfdfc1f3c2f6061 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 16:36:04 -0400 Subject: [PATCH 336/470] Update ip-address-functions.md --- docs/en/sql-reference/functions/ip-address-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index cbb93ee15d7..47058a28d12 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -280,7 +280,7 @@ SELECT ## toIPv4OrDefault(string) -Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns 0. +Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns `0.0.0.0` (0 IPv4). ## toIPv4OrNull(string) @@ -288,7 +288,7 @@ Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns null ## toIPv6OrDefault(string) -Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns 0. +Same as `toIPv6`, but if the IPv6 address has an invalid format, it returns `::` (0 IPv6). ## toIPv6OrNull(string) From 93ba0f4bdf45384bcbcf1296226275fe4a5ea7e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 Mar 2023 01:30:44 +0300 Subject: [PATCH 337/470] Update StackTrace.cpp --- 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 0aebaf9801c..3c0bcd2f808 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -291,7 +291,7 @@ void StackTrace::tryCapture() #endif } -// Clickhouse uses bundled libc++ so type names will be same on every system thus it's save to hardcode them +/// ClickHouse uses bundled libc++ so type names will be the same on every system thus it's safe to hardcode them constexpr std::pair replacements[] = {{"::__1", ""}, {"std::basic_string, std::allocator>", "String"}}; From 990af0fb7977ab41b291df5c563c062ab26557e6 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 6 Mar 2023 23:44:41 +0100 Subject: [PATCH 338/470] Fix typo --- src/Databases/DatabaseAtomic.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 34c4fd3d5d8..eaeeb793b23 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -273,7 +273,7 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_ else renameNoReplace(old_metadata_path, new_metadata_path); - /// After metadata was successfully moved, the following methods should not throw (if them do, it's a logical error) + /// After metadata was successfully moved, the following methods should not throw (if they do, it's a logical error) table_data_path = detach(*this, table_name, table->storesDataOnDisk()); if (exchange) other_table_data_path = detach(other_db, to_table_name, other_table->storesDataOnDisk()); From 1e11348a03113a2096b1103dc0dcd651a960f5bd Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 6 Mar 2023 20:42:17 -0400 Subject: [PATCH 339/470] Update exponentialmovingaverage.md --- .../aggregate-functions/reference/exponentialmovingaverage.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index 26507bedb31..296aae41daa 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -19,7 +19,7 @@ Each `value` corresponds to the determinate `timeunit`. The half-life `x` is the **Arguments** - `value` — Value. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). -- `timeunit` — Timeunit. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). Timeunit is not timestamp (seconds), it's -- an index of the time interval. Can be calculated using [intDiv](../../unctions/arithmetic-functions/#intdiva-b). +- `timeunit` — Timeunit. [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) or [Decimal](../../../sql-reference/data-types/decimal.md). Timeunit is not timestamp (seconds), it's -- an index of the time interval. Can be calculated using [intDiv](../../functions/arithmetic-functions/#intdiva-b). **Parameters** From aacd05e34bcc6abd44eb67613b68551b18fd4d99 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 6 Mar 2023 21:24:04 -0500 Subject: [PATCH 340/470] bugfix + review suggestion --- .../self-extracting-executable/decompressor.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 8db0d95e70d..1374c269dc1 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -170,7 +170,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n bool isSudo() { - return getuid() == 0 && geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); // NOLINT(concurrency-mt-unsafe) + return geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); // NOLINT(concurrency-mt-unsafe) } /// Read data about files and decomrpess them. @@ -423,6 +423,13 @@ int main(int/* argc*/, char* argv[]) else name = file_path; + struct stat input_info; + if (0 != stat(self, &input_info)) + { + perror("stat"); + return 1; + } + #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); @@ -450,13 +457,6 @@ int main(int/* argc*/, char* argv[]) return 1; } - struct stat input_info; - if (0 != stat(self, &input_info)) - { - perror("stat"); - return 1; - } - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to /// 32bit conversion of input_info.st_ino if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) From 951d65f2fdea0ff7027f71eecd4b5bb821c76403 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 7 Mar 2023 08:36:11 +0100 Subject: [PATCH 341/470] Fix debug build --- src/Client/TestHint.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index c9e845f2039..1500a8baf26 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -95,7 +95,7 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) } else if (token.type == TokenType::BareWord) { - int code = code = DB::ErrorCodes::getErrorCodeByName(std::string_view(token.begin, token.end)); + int code = DB::ErrorCodes::getErrorCodeByName(std::string_view(token.begin, token.end)); error_codes.push_back(code); } else From 946dff2d04e7f7af8d3541ed7de11f195b521b6c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 7 Mar 2023 10:51:06 +0100 Subject: [PATCH 342/470] Fix test 02566_ipv4_ipv6_binary_formats --- .../0_stateless/02566_ipv4_ipv6_binary_formats.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference index e228d911715..a3d8a33f757 100644 --- a/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference +++ b/tests/queries/0_stateless/02566_ipv4_ipv6_binary_formats.reference @@ -6,7 +6,7 @@ Arrow 2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 Parquet ipv6 Nullable(FixedString(16)) -ipv4 Nullable(Int64) +ipv4 Nullable(UInt32) 2001:db8:11a3:9d7:1f34:8a2e:7a0:765d 127.0.0.1 ORC ipv6 Nullable(String) From 0d778e693bbeb90d7a8d15c9724684e1b3766913 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 Mar 2023 11:31:43 +0100 Subject: [PATCH 343/470] Update docs/en/sql-reference/statements/create/view.md --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 6e412c13291..a9e7e34dcbb 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -67,7 +67,7 @@ Materialized views in ClickHouse use **column names** instead of column order du Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. -Materialized views in ClickHouse does not have deterministic behaviour in case of errors. This means that blocks that had been already written will be preserved in the destination table, but all blocks after error will not. +Materialized views in ClickHouse do not have deterministic behaviour in case of errors. This means that blocks that had been already written will be preserved in the destination table, but all blocks after error will not. By default if pushing to one of views fails, then the INSERT query will fail too, and some blocks may not be written to the destination table. This can be changed using `materialized_views_ignore_errors` setting (you should set it for `INSERT` query), if you will set `materialized_views_ignore_errors=true`, then any errors while pushing to views will be ignored and all blocks will be written to the destination table. From 785128cb55712b342f5ff2d5c9324f71870cad10 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Tue, 7 Mar 2023 11:44:51 +0100 Subject: [PATCH 344/470] Fix index_granularity for test --- tests/queries/0_stateless/00636_partition_key_parts_pruning.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh index fdaecd87f53..7ec4d99f028 100755 --- a/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh +++ b/tests/queries/0_stateless/00636_partition_key_parts_pruning.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --query="SELECT '*** Single column partition key ***'" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS single_col_partition_key" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE single_col_partition_key(x UInt32) ENGINE MergeTree ORDER BY x PARTITION BY intDiv(x, 10)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE single_col_partition_key(x UInt32) ENGINE MergeTree ORDER BY x PARTITION BY intDiv(x, 10) SETTINGS index_granularity=4" ${CLICKHOUSE_CLIENT} --query="INSERT INTO single_col_partition_key VALUES (1), (2), (3), (4), (11), (12), (20)" From 20925406ba0b1d51b338dd386e28a2fef5633573 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 7 Mar 2023 11:48:28 +0100 Subject: [PATCH 345/470] Docs for `BackupsIOThreadPool` 1/2 --- docs/en/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index 50b338844df..5c867c2c6b9 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -172,7 +172,7 @@ Global thread pool is `GlobalThreadPool` singleton class. To allocate thread fro Global pool is universal and all pools described below are implemented on top of it. This can be thought of as a hierarchy of pools. Any specialized pool takes its threads from the global pool using `ThreadPool` class. So the main purpose of any specialized pool is to apply limit on the number of simultaneous jobs and do job scheduling. If there are more jobs scheduled than threads in a pool, `ThreadPool` accumulates jobs in a queue with priorities. Each job has an integer priority. Default priority is zero. All jobs with higher priority values are started before any job with lower priority value. But there is no difference between already executing jobs, thus priority matters only when the pool in overloaded. -IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU. +IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU. Backup to S3 does significant amount of IO operations and to avoid impact on interactive queries there is separate `BackupsIOThreadPool` configured with `max_backups_io_thread_pool_size`, `max_backups_io_thread_pool_free_size` and `backups_io_thread_pool_queue_size` settings. For periodic task execution there is `BackgroundSchedulePool` class. You can register tasks using `BackgroundSchedulePool::TaskHolder` objects and the pool ensures that no task runs two jobs at the same time. It also allows you to postpone task execution to a specific instant in the future or temporarily deactivate task. Global `Context` provides a few instances of this class for different purposes. For general purpose tasks `Context::getSchedulePool()` is used. From 330aa8c35303d82477ba28f37b1f6cac5c989194 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Tue, 7 Mar 2023 14:49:21 +0400 Subject: [PATCH 346/470] Typo fixed at the page `/docs/ru/engines/table-engines/mergetree-family/mergetree.md` --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 7269cc023e4..24e0f8dbbb8 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -584,7 +584,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); Данные с истекшим `TTL` удаляются, когда ClickHouse мёржит куски данных. -Когда ClickHouse видит, что некоторые данные устарели, он выполняет внеплановые мёржи. Для управление частотой подобных мёржей, можно задать настройку `merge_with_ttl_timeout`. Если её значение слишком низкое, придется выполнять много внеплановых мёржей, которые могут начать потреблять значительную долю ресурсов сервера. +Когда ClickHouse видит, что некоторые данные устарели, он выполняет внеплановые мёржи. Для управления частотой подобных мёржей, можно задать настройку `merge_with_ttl_timeout`. Если её значение слишком низкое, придется выполнять много внеплановых мёржей, которые могут начать потреблять значительную долю ресурсов сервера. Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../../engines/table-engines/mergetree-family/mergetree.md#misc_operations-optimize) перед `SELECT`. @@ -679,7 +679,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); - `policy_name_N` — название политики. Названия политик должны быть уникальны. - `volume_name_N` — название тома. Названия томов должны быть уникальны. - `disk` — диск, находящийся внутри тома. -- `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том. +- `max_data_part_size_bytes` — максимальный размер куска данных, который может находиться на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том. - `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). Для перемещения куски сортируются по размеру от большего к меньшему (по убыванию) и выбираются куски, совокупный размер которых достаточен для соблюдения условия `move_factor`, если совокупный размер всех партов недостаточен, будут перемещены все парты. - `prefer_not_to_merge` — Отключает слияние кусков данных, хранящихся на данном томе. Если данная настройка включена, то слияние данных, хранящихся на данном томе, не допускается. Это позволяет контролировать работу ClickHouse с медленными дисками. @@ -730,7 +730,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); В приведенном примере, политика `hdd_in_order` реализует прицип [round-robin](https://ru.wikipedia.org/wiki/Round-robin_(%D0%B0%D0%BB%D0%B3%D0%BE%D1%80%D0%B8%D1%82%D0%BC)). Так как в политике есть всего один том (`single`), то все записи производятся на его диски по круговому циклу. Такая политика может быть полезна при наличии в системе нескольких похожих дисков, но при этом не сконфигурирован RAID. Учтите, что каждый отдельный диск ненадёжен и чтобы не потерять важные данные это необходимо скомпенсировать за счет хранения данных в трёх копиях. -Если система содержит диски различных типов, то может пригодиться политика `moving_from_ssd_to_hdd`. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также, при заполнении диска `fast_ssd` более чем на 80% данные будут переносится на диск `disk1` фоновым процессом. +Если система содержит диски различных типов, то может пригодиться политика `moving_from_ssd_to_hdd`. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также при заполнении диска `fast_ssd` более чем на 80% данные будут переноситься на диск `disk1` фоновым процессом. Порядок томов в политиках хранения важен, при достижении условий на переполнение тома данные переносятся на следующий. Порядок дисков в томах так же важен, данные пишутся по очереди на каждый из них. From ace08ad6b26d8f1d5a48667bca270056549e95c4 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 7 Mar 2023 11:49:38 +0100 Subject: [PATCH 347/470] Update docs/en/development/architecture.md --- docs/en/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index 5c867c2c6b9..dd46b294ddd 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -172,7 +172,7 @@ Global thread pool is `GlobalThreadPool` singleton class. To allocate thread fro Global pool is universal and all pools described below are implemented on top of it. This can be thought of as a hierarchy of pools. Any specialized pool takes its threads from the global pool using `ThreadPool` class. So the main purpose of any specialized pool is to apply limit on the number of simultaneous jobs and do job scheduling. If there are more jobs scheduled than threads in a pool, `ThreadPool` accumulates jobs in a queue with priorities. Each job has an integer priority. Default priority is zero. All jobs with higher priority values are started before any job with lower priority value. But there is no difference between already executing jobs, thus priority matters only when the pool in overloaded. -IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU. Backup to S3 does significant amount of IO operations and to avoid impact on interactive queries there is separate `BackupsIOThreadPool` configured with `max_backups_io_thread_pool_size`, `max_backups_io_thread_pool_free_size` and `backups_io_thread_pool_queue_size` settings. +IO thread pool is implemented as a plain `ThreadPool` accessible via `IOThreadPool::get()` method. It is configured in the same way as global pool with `max_io_thread_pool_size`, `max_io_thread_pool_free_size` and `io_thread_pool_queue_size` settings. The main purpose of IO thread pool is to avoid exhaustion of the global pool with IO jobs, which could prevent queries from fully utilizing CPU. Backup to S3 does significant amount of IO operations and to avoid impact on interactive queries there is a separate `BackupsIOThreadPool` configured with `max_backups_io_thread_pool_size`, `max_backups_io_thread_pool_free_size` and `backups_io_thread_pool_queue_size` settings. For periodic task execution there is `BackgroundSchedulePool` class. You can register tasks using `BackgroundSchedulePool::TaskHolder` objects and the pool ensures that no task runs two jobs at the same time. It also allows you to postpone task execution to a specific instant in the future or temporarily deactivate task. Global `Context` provides a few instances of this class for different purposes. For general purpose tasks `Context::getSchedulePool()` is used. From 2ae0bceb2b427e0e1a1d8ae3e6d30107448fad3e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 Mar 2023 10:56:11 +0000 Subject: [PATCH 348/470] Address PR comments --- docs/en/operations/settings/settings.md | 53 +++++++-------------- docs/en/sql-reference/operators/in.md | 6 +-- src/Interpreters/Cluster.cpp | 26 ++++++---- src/Interpreters/InterpreterSelectQuery.cpp | 12 ++--- 4 files changed, 43 insertions(+), 54 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 809fdb17d99..94dcf159ca9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1248,8 +1248,8 @@ Possible values: Default value: 1. :::warning -Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) with [parallel_replicas_mode](#settings-parallel_replicas_mode) set to `sample_key` or `read_tasks`. -If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `custom_key`, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. +Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) without [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key). +If [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. ::: @@ -1275,11 +1275,14 @@ Default value: `1`. **Additional Info** -This options will produce different results depending on the value of [parallel_replicas_mode](#settings-parallel_replicas_mode). +This options will produce different results depending on the settings used. -### `sample_key` +:::warning +This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. +::: + +### Parallel processing using `SAMPLE` key -If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `sample_key`, this setting is useful for replicated tables with a sampling key. A query may be processed faster if it is executed on several servers in parallel. But the query performance may degrade in the following cases: - The position of the sampling key in the partitioning key does not allow efficient range scans. @@ -1287,45 +1290,21 @@ A query may be processed faster if it is executed on several servers in parallel - The sampling key is an expression that is expensive to calculate. - The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. -### `custom_key` +### Parallel processing using [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) -If [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `custom_key`, this setting is useful for any replicated table. -A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) -and [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type). - -Use `default` for [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type) unless the data is split across the entire integer space (e.g. column contains hash values), -then `range` should be used. -Simple expressions using primary keys are preferred. - -If the `custom_key` mode is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. -Otherwise, it will behave same as `sample_key` mode, it will use multiple replicas of each shard. - -:::warning -This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. -::: - -## parallel_replicas_mode {#settings-parallel_replicas_mode} - -Mode of splitting work between replicas. - -Possible values: - -- `sample_key` — Use `SAMPLE` key defined in the `SAMPLE BY` clause to split the work between replicas. -- `custom_key` — Define an arbitrary integer expression to use for splitting work between replicas. -- `read_tasks` — Split tasks for reading physical parts between replicas. - -Default value: `sample_key`. +This setting is useful for any replicated table. ## parallel_replicas_custom_key {#settings-parallel_replicas_custom_key} -Map of arbitrary integer expression that can be used to split work between replicas for a specific table. -If it's used with `cluster` function, the key can be name of the local table defined inside the `cluster` function. -If it's used with `Distributed` engine, the key can be name of the distributed table, alias or the local table for which the `Distributed` engine is created. +An arbitrary integer expression that can be used to split work between replicas for a specific table. The value can be any integer expression. +A query may be processed faster if it is executed on several servers in parallel but it depends on the used [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) +and [parallel_replicas_custom_key_filter_type](#settings-parallel_replicas_custom_key_filter_type). -Used only if `parallel_replicas_mode` is set to `custom_key`. +Simple expressions using primary keys are preferred. -Default value: `{}`. +If the setting is used on a cluster that consists of a single shard with multiple replicas, those replicas will be converted into virtual shards. +Otherwise, it will behave same as for `SAMPLE` key, it will use multiple replicas of each shard. ## parallel_replicas_custom_key_filter_type {#settings-parallel_replicas_custom_key_filter_type} diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index e1e4118524a..0599a50c0a4 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -235,10 +235,10 @@ If `some_predicate` is not selective enough, it will return large amount of data When [max_parallel_replicas](#settings-max_parallel_replicas) is greater than 1, distributed queries are further transformed. -For example, if [parallel_replicas_mode](#settings-parallel_replicas_mode) is set to `sample_key`, the following: +For example, the following: ```sql SELECT CounterID, count() FROM distributed_table_1 WHERE UserID IN (SELECT UserID FROM local_table_2 WHERE CounterID < 100) -SETTINGS max_parallel_replicas=3, parallel_replicas_mode='sample_key' +SETTINGS max_parallel_replicas=3 ``` is transformed on each server into @@ -256,4 +256,4 @@ Therefore adding the [max_parallel_replicas](#settings-max_parallel_replicas) se One workaround if local_table_2 does not meet the requirements, is to use `GLOBAL IN` or `GLOBAL JOIN`. -If a table doesn't have a sampling key, more flexible options for [parallel_replicas_mode](#settings-parallel_replicas_mode) can be used that can produce different and more optimal behaviour. +If a table doesn't have a sampling key, more flexible options for [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) can be used that can produce different and more optimal behaviour. diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b419dacd523..0add0e427f9 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -671,7 +671,7 @@ std::unique_ptr Cluster::getClusterWithMultipleShards(const std::vector namespace { -void shuffleReplicas(auto & replicas, const Settings & settings) +void shuffleReplicas(std::vector & replicas, const Settings & settings, size_t replicas_needed) { std::random_device rd; std::mt19937 gen{rd()}; @@ -679,15 +679,25 @@ void shuffleReplicas(auto & replicas, const Settings & settings) if (settings.prefer_localhost_replica) { // force for local replica to always be included - auto local_replica = std::find_if(replicas.begin(), replicas.end(), [](const auto & replica) { return replica.is_local; }); - if (local_replica != replicas.end()) - { - if (local_replica != replicas.begin()) - std::swap(*replicas.begin(), *local_replica); + auto first_non_local_replica = std::partition(replicas.begin(), replicas.end(), [](const auto & replica) { return replica.is_local; }); + size_t local_replicas_count = first_non_local_replica - replicas.begin(); - std::shuffle(replicas.begin() + 1, replicas.end(), gen); + if (local_replicas_count == replicas_needed) + { + /// we have exact amount of local replicas as needed, no need to do anything return; } + + if (local_replicas_count > replicas_needed) + { + /// we can use only local replicas, shuffle them + std::shuffle(replicas.begin(), first_non_local_replica, gen); + return; + } + + /// shuffle just non local replicas + std::shuffle(first_non_local_replica, replicas.end(), gen); + return; } std::shuffle(replicas.begin(), replicas.end(), gen); @@ -751,7 +761,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti { auto shuffled_replicas = replicas; // shuffle replicas so we don't always pick the same subset - shuffleReplicas(shuffled_replicas, settings); + shuffleReplicas(shuffled_replicas, settings, max_replicas_from_shard); create_shards_from_replicas(std::span{shuffled_replicas.begin(), max_replicas_from_shard}); } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d016922e89e..33d362556aa 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -523,7 +523,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { if (auto custom_key_ast = parseCustomKeyForTable(settings.parallel_replicas_custom_key, *context)) { - LOG_INFO(log, "Processing query on a replica using custom_key"); + LOG_TRACE(log, "Processing query on a replica using custom_key '{}'", settings.parallel_replicas_custom_key.value); if (!storage) throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Storage is unknown when trying to parse custom key for parallel replica"); @@ -537,11 +537,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( } else if (settings.parallel_replica_offset > 0) { - LOG_DEBUG( - log, - "Will use no data on this replica because parallel replicas processing with custom_key has been requested" - " (setting 'max_parallel_replicas') but the table does not have custom_key defined for it or it's invalid (settings `parallel_replicas_custom_key`)"); - parallel_replicas_custom_filter_ast = std::make_shared(false); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Parallel replicas processing with custom_key has been requested " + "(setting 'max_parallel_replicas') but the table does not have custom_key defined for it " + "or it's invalid (settings `parallel_replicas_custom_key`)"); } } else if (auto * distributed = dynamic_cast(storage.get()); From 3dcdf4cd4ef72b3a8043e7533c1119efbbbfa090 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 7 Mar 2023 12:12:29 +0100 Subject: [PATCH 349/470] Docs for `BackupsIOThreadPool` 2/2 --- .../settings.md | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 17d03dfa4ec..92a8270ac0f 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -967,6 +967,7 @@ The maximum number of jobs that can be scheduled on the Global Thread pool. Incr Possible values: - Positive integer. +- 0 — No limit. Default value: `10000`. @@ -976,6 +977,69 @@ Default value: `10000`. 12000 ``` +## max_io_thread_pool_size {#max-io-thread-pool-size} + +ClickHouse uses threads from the IO Thread pool to do some IO operations (e.g. to interact with S3). `max_io_thread_pool_size` limits the maximum number of threads in the pool. + +Possible values: + +- Positive integer. + +Default value: `100`. + +## max_io_thread_pool_free_size {#max-io-thread-pool-free-size} + +If the number of **idle** threads in the IO Thread pool is greater than `max_io_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. + +Possible values: + +- Positive integer. + +Default value: `0`. + +## io_thread_pool_queue_size {#io-thread-pool-queue-size} + +The maximum number of jobs that can be scheduled on the IO Thread pool. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `10000`. + +## max_backups_io_thread_pool_size {#max-backups-io-thread-pool-size} + +ClickHouse uses threads from the Backups IO Thread pool to do S3 backup IO operations. `max_backups_io_thread_pool_size` limits the maximum number of threads in the pool. + +Possible values: + +- Positive integer. + +Default value: `1000`. + +## max_backups_io_thread_pool_free_size {#max-backups-io-thread-pool-free-size} + +If the number of **idle** threads in the Backups IO Thread pool is greater than `max_backup_io_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. + +Possible values: + +- Positive integer. +- Zero. + +Default value: `0`. + +## backups_io_thread_pool_queue_size {#backups-io-thread-pool-queue-size} + +The maximum number of jobs that can be scheduled on the Backups IO Thread pool. It is recommended to keep this queue unlimited due to the current S3 backup logic. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + ## background_pool_size {#background_pool_size} Sets the number of threads performing background merges and mutations for tables with MergeTree engines. This setting is also could be applied at server startup from the `default` profile configuration for backward compatibility at the ClickHouse server start. You can only increase the number of threads at runtime. To lower the number of threads you have to restart the server. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. From f408f750ebc9a4f09224de2e027be2b0f84e2e0a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 Mar 2023 11:24:46 +0000 Subject: [PATCH 350/470] Disable parallel replicas for joins --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- .../02535_max_parallel_replicas_custom_key.reference | 1 + .../0_stateless/02535_max_parallel_replicas_custom_key.sh | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 33d362556aa..2f579244b9a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -455,10 +455,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - if (joined_tables.tablesCount() > 1 && settings.allow_experimental_parallel_reading_from_replicas) + if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas)) { LOG_WARNING(log, "Joins are not supported with parallel replicas. Query will be executed without using them."); context->setSetting("allow_experimental_parallel_reading_from_replicas", false); + context->setSetting("parallel_replicas_custom_key", String{""}); } /// Rewrite JOINs diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference index 9e4e24235aa..8d0f56ba185 100644 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.reference @@ -170,3 +170,4 @@ filter_type='range' max_replicas=3 prefer_localhost_replica=1 0 334 1 333 2 333 +1 diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 314251be874..3035a191c8f 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,4 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "Joins are not supported with parallel replicas" + $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" From 1db6b9414e8b71f27bd1040f696d780f020ca1e0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 Mar 2023 12:39:17 +0100 Subject: [PATCH 351/470] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- 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 9e79a715610..61eedc18736 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8436,7 +8436,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( { /// We don't know what to do, because this part can be mutation part /// with hardlinked columns. Since we don't have this information (about blobs not to remove) - /// we refuce to remove blobs. + /// we refuse to remove blobs. LOG_WARNING(logger, "Node with parent zookeeper lock {} for part {} doesn't exist (part was unlocked before), refuse to remove blobs", zookeeper_part_uniq_node, part_name); return std::make_pair{false, {}}; } From c10cb436f41979e92679752d74dc5129637f932c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 Mar 2023 12:42:42 +0100 Subject: [PATCH 352/470] Fix build --- 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 61eedc18736..6c6ff30fd04 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8438,7 +8438,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( /// with hardlinked columns. Since we don't have this information (about blobs not to remove) /// we refuse to remove blobs. LOG_WARNING(logger, "Node with parent zookeeper lock {} for part {} doesn't exist (part was unlocked before), refuse to remove blobs", zookeeper_part_uniq_node, part_name); - return std::make_pair{false, {}}; + return {false, {}}; } else { From ced97cefbb5dcbdb74782543281c71ccba387efc Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 Mar 2023 09:36:41 +0000 Subject: [PATCH 353/470] Address review comments --- .../Passes/LogicalExpressionOptimizer.cpp | 21 +++++++++++++++---- .../Passes/LogicalExpressionOptimizerPass.h | 12 +++++------ 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp index 85b44de74da..73585a4cd23 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp @@ -102,7 +102,12 @@ private: if (and_operands.size() == 1) { - assert(!function_node.getResultType()->isNullable()); + /// AND operator can have UInt8 or bool as its type. + /// bool is used if a bool constant is at least one operand. + /// Because we reduce the number of operands here by eliminating the same equality checks, + /// the only situation we can end up here is we had AND check where all the equality checks are the same so we know the type is UInt8. + /// Otherwise, we will have > 1 operands and we don't have to do anything. + assert(!function_node.getResultType()->isNullable() && and_operands[0]->getResultType()->equals(*function_node.getResultType())); node = std::move(and_operands[0]); return; } @@ -204,9 +209,17 @@ private: if (or_operands.size() == 1) { - assert(!function_node.getResultType()->isNullable()); - node = std::move(or_operands[0]); - return; + /// if the result type of operand is the same as the result type of OR + /// we can replace OR with the operand + if (or_operands[0]->getResultType()->equals(*function_node.getResultType())) + { + assert(!function_node.getResultType()->isNullable()); + node = std::move(or_operands[0]); + return; + } + + /// otherwise add a stub 0 to make OR correct + or_operands.push_back(std::make_shared(static_cast(0))); } auto or_function_resolver = FunctionFactory::instance().get("or", getContext()); diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h index b436c94fe4c..05c10ddc685 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.h @@ -6,9 +6,9 @@ namespace DB { /** - * This pass tries to do optimizations on logical expression + * This pass tries to do optimizations on logical expression: * - * Replaces chains of equality functions inside an OR with a single IN operator. + * 1. Replaces chains of equality functions inside an OR with a single IN operator. * The replacement is done if: * - one of the operands of the equality function is a constant * - length of chain is at least 'optimize_min_equality_disjunction_chain_length' long OR the expression has type of LowCardinality @@ -26,7 +26,7 @@ namespace DB * WHERE b = 'test' OR a IN (1, 2); * ------------------------------- * - * Removes duplicate OR checks + * 2. Removes duplicate OR checks * ------------------------------- * SELECT * * FROM table @@ -39,7 +39,7 @@ namespace DB * WHERE a = 1 OR b = 'test'; * ------------------------------- * - * Replaces AND chains with a single constant. + * 3. Replaces AND chains with a single constant. * The replacement is done if: * - one of the operands of the equality function is a constant * - constants are different for same expression @@ -55,7 +55,7 @@ namespace DB * WHERE 0; * ------------------------------- * - * Removes duplicate AND checks + * 4. Removes duplicate AND checks * ------------------------------- * SELECT * * FROM table @@ -74,7 +74,7 @@ class LogicalExpressionOptimizerPass final : public IQueryTreePass public: String getName() override { return "LogicalExpressionOptimizer"; } - String getDescription() override { return "Transform all the 'or's with equality check to a single IN function"; } + String getDescription() override { return "Transform equality chain to a single IN function or a constant if possible"; } void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; From 37b09ce46addcfabdfae531c93e1108a74ed0746 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 Mar 2023 12:16:01 +0000 Subject: [PATCH 354/470] add back trace log --- tests/config/config.d/keeper_port.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index cffd325e968..6c60c8936a9 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -14,6 +14,7 @@ 1000000000000000 100000 + trace 0 From 7ce20f5cd2f702df04b2ccbc58117cee9cd974fb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 7 Mar 2023 00:53:37 +0100 Subject: [PATCH 355/470] fix tests --- src/Interpreters/InterpreterKillQueryQuery.cpp | 4 ++++ .../Executors/PushingAsyncPipelineExecutor.cpp | 2 +- tests/integration/test_grpc_protocol/test.py | 2 -- .../02232_dist_insert_send_logs_level_hung.sh | 11 ++++++++++- 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 40698386ccb..3330159aff5 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -161,6 +161,8 @@ public: if (curr_process.processed) continue; + LOG_DEBUG(&Poco::Logger::get("KillQuery"), "Will kill query {} (synchronously)", curr_process.query_id); + auto code = process_list.sendCancelToQuery(curr_process.query_id, curr_process.user, true); if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent) @@ -226,6 +228,8 @@ BlockIO InterpreterKillQueryQuery::execute() MutableColumns res_columns = header.cloneEmptyColumns(); for (const auto & query_desc : queries_to_stop) { + if (!query.test) + LOG_DEBUG(&Poco::Logger::get("KillQuery"), "Will kill query {} (asynchronously)", query_desc.query_id); auto code = (query.test) ? CancellationCode::Unknown : process_list.sendCancelToQuery(query_desc.query_id, query_desc.user, true); insertResultRow(query_desc.source_num, code, processes_block, header, res_columns); } diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 70815bb8b3b..4478f1548a4 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -187,7 +187,7 @@ void PushingAsyncPipelineExecutor::push(Chunk chunk) if (!is_pushed) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); + "Pipeline for PushingAsyncPipelineExecutor was finished before all data was inserted"); } void PushingAsyncPipelineExecutor::push(Block block) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index a1bc0d42a46..137d585f7d1 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -594,8 +594,6 @@ def test_cancel_while_processing_input(): stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQueryWithStreamInput(send_query_info()) assert result.cancelled == True - assert result.progress.written_rows == 6 - assert query("SELECT a FROM t ORDER BY a") == "1\n2\n3\n4\n5\n6\n" def test_cancel_while_generating_output(): diff --git a/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh b/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh index 5ed94148bc1..734cef06214 100755 --- a/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh +++ b/tests/queries/0_stateless/02232_dist_insert_send_logs_level_hung.sh @@ -49,7 +49,16 @@ insert_client_opts=( timeout 250s $CLICKHOUSE_CLIENT "${client_opts[@]}" "${insert_client_opts[@]}" -q "insert into function remote('127.2', currentDatabase(), in_02232) select * from numbers(1e6)" # Kill underlying query of remote() to make KILL faster -timeout 60s $CLICKHOUSE_CLIENT "${client_opts[@]}" -q "KILL QUERY WHERE Settings['log_comment'] = '$CLICKHOUSE_LOG_COMMENT' SYNC" --format Null +# This test is reproducing very interesting bahaviour. +# The block size is 1, so the secondary query creates InterpreterSelectQuery for each row due to pushing to the MV. +# It works extremely slow, and the initial query produces new blocks and writes them to the socket much faster +# then the secondary query can read and process them. Therefore, it fills network buffers in the kernel. +# Once a buffer in the kernel is full, send(...) blocks until the secondary query will finish processing data +# that it already has in ReadBufferFromPocoSocket and call recv. +# Or until the kernel will decide to resize the buffer (seems like it has non-trivial rules for that). +# Anyway, it may look like the initial query got stuck, but actually it did not. +# Moreover, the initial query cannot be killed at that point, so KILL QUERY ... SYNC will get "stuck" as well. +timeout 30s $CLICKHOUSE_CLIENT "${client_opts[@]}" -q "KILL QUERY WHERE query like '%INSERT INTO $CLICKHOUSE_DATABASE.in_02232%' SYNC" --format Null echo $? $CLICKHOUSE_CLIENT "${client_opts[@]}" -nm -q " From 0da310c8dcf51331bc435f7cf49c69e84145286e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Mar 2023 15:30:40 +0100 Subject: [PATCH 356/470] Update ProcessList.cpp --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index a26844ae73c..b792ea538ae 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -404,7 +404,7 @@ CancellationCode QueryStatus::cancelQuery(bool) std::lock_guard lock(executors_mutex); executors_snapshot = executors; } - + /// We should call cancel() for each executor with unlocked executors_mutex, because /// cancel() can try to lock some internal mutex that is already locked by query executing /// thread, and query executing thread can call removePipelineExecutor and lock executors_mutex, From 27e4b09f944ae450e394b2886dd557ac64ae141b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Mar 2023 15:13:13 +0000 Subject: [PATCH 357/470] Fix 02570_fallback_from_async_insert --- tests/queries/0_stateless/02570_fallback_from_async_insert.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02570_fallback_from_async_insert.sh b/tests/queries/0_stateless/02570_fallback_from_async_insert.sh index 9c158d6241b..d7c8944b89d 100755 --- a/tests/queries/0_stateless/02570_fallback_from_async_insert.sh +++ b/tests/queries/0_stateless/02570_fallback_from_async_insert.sh @@ -47,6 +47,7 @@ $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT --query " SELECT 'id_' || splitByChar('_', query_id)[1] AS id FROM system.text_log WHERE query_id LIKE '%$query_id_suffix' AND message LIKE '%$message%' + ORDER BY id " $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t_async_insert_fallback" From d9611dde0b12d2c7c5b5c633f028d7c625ae4d78 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Mar 2023 15:23:27 +0000 Subject: [PATCH 358/470] fix test --- .../01705_normalize_create_alter_function_names.reference | 4 ++-- .../01705_normalize_create_alter_function_names.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference index b6f5fe99ca1..b5b93c34c00 100644 --- a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference +++ b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference @@ -1,2 +1,2 @@ -CREATE TABLE default.x\n(\n `i` Int32,\n INDEX mm rand() TYPE minmax GRANULARITY 1,\n INDEX nn rand() TYPE minmax GRANULARITY 1,\n PROJECTION p\n (\n SELECT max(i)\n ),\n PROJECTION p2\n (\n SELECT min(i)\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/x\', \'r\')\nORDER BY i\nSETTINGS index_granularity = 8192 -metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 8192\nmode: 0\nsign column: \nprimary key: i\ndata format version: 1\npartition key: \nindices: mm rand() TYPE minmax GRANULARITY 1, nn rand() TYPE minmax GRANULARITY 1\nprojections: p (SELECT max(i)), p2 (SELECT min(i))\ngranularity bytes: 10485760\n +CREATE TABLE default.x\n(\n `i` Int32,\n INDEX mm log2(i) TYPE minmax GRANULARITY 1,\n INDEX nn log2(i) TYPE minmax GRANULARITY 1,\n PROJECTION p\n (\n SELECT max(i)\n ),\n PROJECTION p2\n (\n SELECT min(i)\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/x\', \'r\')\nORDER BY i\nSETTINGS index_granularity = 8192 +metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 8192\nmode: 0\nsign column: \nprimary key: i\ndata format version: 1\npartition key: \nindices: mm log2(i) TYPE minmax GRANULARITY 1, nn log2(i) TYPE minmax GRANULARITY 1\nprojections: p (SELECT max(i)), p2 (SELECT min(i))\ngranularity bytes: 10485760\n diff --git a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql index 683bd271405..be0f7e8b710 100644 --- a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql +++ b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql @@ -2,9 +2,9 @@ drop table if exists x; -create table x(i int, index mm RAND() type minmax granularity 1, projection p (select MAX(i))) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x', 'r') order by i; +create table x(i int, index mm LOG2(i) type minmax granularity 1, projection p (select MAX(i))) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x', 'r') order by i; -alter table x add index nn RAND() type minmax granularity 1, add projection p2 (select MIN(i)); +alter table x add index nn LOG2(i) type minmax granularity 1, add projection p2 (select MIN(i)); show create x; From af23c8bed07f665c0cf074b64bb10d7ad49cbb66 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 6 Mar 2023 17:46:46 +0100 Subject: [PATCH 359/470] Get the release logs w/o delay from curl --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 2ef05fe989b..4b4c7e6c70e 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -15,7 +15,7 @@ jobs: - name: Deploy packages and assets run: | GITHUB_TAG="${GITHUB_REF#refs/tags/}" - curl --silent --data '' \ + curl --silent --data '' --no-buffer \ '${{ secrets.PACKAGES_RELEASE_URL }}/release/'"${GITHUB_TAG}"'?binary=binary_darwin&binary=binary_darwin_aarch64&sync=true' ############################################################################################ ##################################### Docker images ####################################### From c63be68a8fd9a18c881df656d56f17ea899280fc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 7 Mar 2023 13:35:00 +0100 Subject: [PATCH 360/470] Use explicit docker tags and paths --- .github/workflows/backport_branches.yml | 3 ++- .github/workflows/master.yml | 3 ++- .github/workflows/pull_request.yml | 3 ++- .github/workflows/release.yml | 3 ++- .github/workflows/release_branches.yml | 3 ++- 5 files changed, 10 insertions(+), 5 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index a324d20abc9..7cdf11fec0f 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -452,7 +452,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push \ + --image-repo clickhouse/clickhouse-server --image-path docker/server python3 docker_server.py --release-type head --no-push --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index b70fe256833..7c5e477ab60 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -860,7 +860,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head + python3 docker_server.py --release-type head \ + --image-repo clickhouse/clickhouse-server --image-path docker/server python3 docker_server.py --release-type head --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ff98739db00..e7303c7b69f 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -917,7 +917,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push \ + --image-repo clickhouse/clickhouse-server --image-path docker/server python3 docker_server.py --release-type head --no-push --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 4b4c7e6c70e..e69d189c80e 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -31,7 +31,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type auto --version "${{ github.ref }}" + python3 docker_server.py --release-type auto --version "${{ github.ref }}" \ + --image-repo clickhouse/clickhouse-server --image-path docker/server python3 docker_server.py --release-type auto --version "${{ github.ref }}" --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 74ec1163cc9..e56a1fb58fc 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -525,7 +525,8 @@ jobs: - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push \ + --image-repo clickhouse/clickhouse-server --image-path docker/server python3 docker_server.py --release-type head --no-push --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup From dedd242f350d9ddfb8a43eac36103f4841b57941 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 7 Mar 2023 13:42:18 +0100 Subject: [PATCH 361/470] Add a manual trigger for release workflow --- .github/workflows/release.yml | 28 +++++++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index e69d189c80e..73246af6dfc 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -7,14 +7,27 @@ on: # yamllint disable-line rule:truthy release: types: - published + workflow_dispatch: + inputs: + tag: + description: 'Release tag' + required: true + type: string jobs: ReleasePublish: runs-on: [self-hosted, style-checker] steps: + - name: Set tag from input + if: github.event_name == 'workflow_dispatch' + run: | + echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" + - name: Set tag from REF + if: github.event_name == 'release' + run: | + echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - name: Deploy packages and assets run: | - GITHUB_TAG="${GITHUB_REF#refs/tags/}" curl --silent --data '' --no-buffer \ '${{ secrets.PACKAGES_RELEASE_URL }}/release/'"${GITHUB_TAG}"'?binary=binary_darwin&binary=binary_darwin_aarch64&sync=true' ############################################################################################ @@ -23,17 +36,26 @@ jobs: DockerServerImages: runs-on: [self-hosted, style-checker] steps: + - name: Set tag from input + if: github.event_name == 'workflow_dispatch' + run: | + echo "GITHUB_TAG=${{ github.event.inputs.tag }}" >> "$GITHUB_ENV" + - name: Set tag from REF + if: github.event_name == 'release' + run: | + echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - name: Check out repository code uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # otherwise we will have no version info + ref: ${{ env.GITHUB_TAG }} - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type auto --version "${{ github.ref }}" \ + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" \ --image-repo clickhouse/clickhouse-server --image-path docker/server - python3 docker_server.py --release-type auto --version "${{ github.ref }}" --no-ubuntu \ + python3 docker_server.py --release-type auto --version "$GITHUB_TAG" --no-ubuntu \ --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - name: Cleanup if: always() From 86410a7a3f8a9241413f8a36d3bda04e003e17da Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Mar 2023 16:16:51 +0000 Subject: [PATCH 362/470] Fix MergeTreeTransaction::isReadOnly --- src/Interpreters/MergeTreeTransaction.cpp | 7 +++++++ src/Interpreters/MergeTreeTransaction.h | 3 +++ 2 files changed, 10 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 50ecb061752..bfdda354c9b 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -168,6 +168,8 @@ void MergeTreeTransaction::addMutation(const StoragePtr & table, const String & bool MergeTreeTransaction::isReadOnly() const { std::lock_guard lock{mutex}; + if (finalized) + return is_read_only; chassert((creating_parts.empty() && removing_parts.empty() && mutations.empty()) == storages.empty()); return storages.empty(); } @@ -318,6 +320,11 @@ bool MergeTreeTransaction::rollback() noexcept void MergeTreeTransaction::afterFinalize() { std::lock_guard lock{mutex}; + chassert((creating_parts.empty() && removing_parts.empty() && mutations.empty()) == storages.empty()); + + /// Remember if it was read-only transaction before we clear storages + is_read_only = storages.empty(); + /// Release shared pointers just in case storages.clear(); mutations.clear(); diff --git a/src/Interpreters/MergeTreeTransaction.h b/src/Interpreters/MergeTreeTransaction.h index e5a80e03e18..4ca36cf64ad 100644 --- a/src/Interpreters/MergeTreeTransaction.h +++ b/src/Interpreters/MergeTreeTransaction.h @@ -78,6 +78,9 @@ private: bool finalized TSA_GUARDED_BY(mutex) = false; + /// Indicates if transaction was read-only before `afterFinalize` + bool is_read_only TSA_GUARDED_BY(mutex) = false; + /// Lists of changes made by transaction std::unordered_set storages TSA_GUARDED_BY(mutex); DataPartsVector creating_parts TSA_GUARDED_BY(mutex); From 4e79df448f151a471d049c6938c06e2e690e9904 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 7 Mar 2023 16:17:35 +0000 Subject: [PATCH 363/470] Catch exceptions in LiveViewPeriodicRefreshTask --- src/Storages/LiveView/StorageLiveView.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 547becf3837..3467628d081 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -241,7 +241,18 @@ StorageLiveView::StorageLiveView( blocks_metadata_ptr = std::make_shared(); active_ptr = std::make_shared(true); - periodic_refresh_task = getContext()->getSchedulePool().createTask("LiveViewPeriodicRefreshTask", [this]{ periodicRefreshTaskFunc(); }); + periodic_refresh_task = getContext()->getSchedulePool().createTask("LiveViewPeriodicRefreshTask", + [this] + { + try + { + periodicRefreshTaskFunc(); + } + catch (...) + { + tryLogCurrentException(log, "Exception in LiveView periodic refresh task in BackgroundSchedulePool"); + } + }); periodic_refresh_task->deactivate(); } From ba0cc954415275380c93f51c145b6ed479388aa3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 7 Mar 2023 20:04:52 +0100 Subject: [PATCH 364/470] Fix typo --- src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 7fae2c8f480..39cacde94ed 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -23,7 +23,7 @@ #include namespace DB -{7 +{ namespace ErrorCodes { From ff8fc0634ac6c27b74b7182e3307212b8b25f5bd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Mar 2023 19:50:42 +0000 Subject: [PATCH 365/470] Docs: Small cleanups after Kafka fix #47138 --- docs/en/engines/table-engines/integrations/kafka.md | 12 ++++++------ docs/en/sql-reference/statements/create/table.md | 8 ++------ src/Storages/ColumnsDescription.h | 4 +--- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index e6134043b8e..255ba06f056 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -19,8 +19,8 @@ Kafka lets you: ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + name1 [type1], + name2 [type2], ... ) ENGINE = Kafka() SETTINGS @@ -113,6 +113,10 @@ Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format +:::info +The Kafka table engine doesn't support columns with [default value](../../../sql-reference/statements/create/table.md#default_value). If you need columns with default value, you can add them at materialized view level (see below). +::: + ## Description {#description} The delivered messages are tracked automatically, so each message in a group is only counted once. If you want to get the data twice, then create a copy of the table with another group name. @@ -125,10 +129,6 @@ Groups are flexible and synced on the cluster. For instance, if you have 10 topi 2. Create a table with the desired structure. 3. Create a materialized view that converts data from the engine and puts it into a previously created table. -:::info -Kafka Engine doesn't support columns with default value of type `DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS`. If you need columns with any default type, they can be added at `MATERIALIZED VIEW` level. -::: - When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. This allows you to continually receive messages from Kafka and convert them to the required format using `SELECT`. One kafka table can have as many materialized views as you like, they do not read data from the kafka table directly, but receive new records (in blocks), this way you can write to several tables with different detail level (with grouping - aggregation and without). diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index b3d5060e13c..50e74920e4b 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -110,7 +110,7 @@ If the type is not `Nullable` and if `NULL` is specified, it will be treated as See also [data_type_default_nullable](../../../operations/settings/settings.md#data_type_default_nullable) setting. -## Default Values +## Default Values {#default_values} The column description can specify an expression for a default value, in one of the following ways: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. @@ -124,10 +124,6 @@ If the data type and default expression are defined explicitly, this expression Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions do not contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. -:::info -Kafka Engine doesn't support columns with default value of type `DEFAULT/MATERIALIZED/EPHEMERAL/ALIAS`. If you need columns with any default type, they can be added at `MATERIALIZED VIEW` level, see [Kafka Engine](../../../engines/table-engines/integrations/kafka.md#description). -::: - ### DEFAULT `DEFAULT expr` @@ -580,7 +576,7 @@ SELECT * FROM base.t1; You can add a comment to the table when you creating it. :::note -The comment is supported for all table engines except [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) and [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md). +The comment clause is supported by all table engines except [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) and [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md). ::: diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 36109392ab6..5551fdea2e3 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -132,9 +132,7 @@ public: NamesAndTypesList getInsertable() const; /// ordinary + ephemeral NamesAndTypesList getAliases() const; NamesAndTypesList getEphemeral() const; - // Columns with preset default expression. - // For example from `CREATE TABLE` statement - NamesAndTypesList getWithDefaultExpression() const; + NamesAndTypesList getWithDefaultExpression() const; // columns with default expression, for example set by `CREATE TABLE` statement NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. NamesAndTypesList getAll() const; /// ordinary + materialized + aliases + ephemeral /// Returns .size0/.null/... From 9792b93a1171f8fc95479c616b236141c70af667 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 23 Feb 2023 13:48:36 +0100 Subject: [PATCH 366/470] Fix changing a role which is already expired in RoleCache. --- src/Access/RoleCache.cpp | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 308b771243e..7a8c13636d2 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -163,11 +163,15 @@ void RoleCache::roleChanged(const UUID & role_id, const RolePtr & changed_role) std::lock_guard lock{mutex}; auto role_from_cache = cache.get(role_id); - if (!role_from_cache) - return; - role_from_cache->first = changed_role; - cache.update(role_id, role_from_cache); - collectEnabledRoles(¬ifications); + if (role_from_cache) + { + /// We update the role stored in a cache entry only if that entry has not expired yet. + role_from_cache->first = changed_role; + cache.update(role_id, role_from_cache); + } + + /// An enabled role for some users has been changed, we need to recalculate the access rights. + collectEnabledRoles(¬ifications); /// collectEnabledRoles() must be called with the `mutex` locked. } @@ -177,8 +181,12 @@ void RoleCache::roleRemoved(const UUID & role_id) scope_guard notifications; std::lock_guard lock{mutex}; + + /// If a cache entry with the role has expired already, that remove() will do nothing. cache.remove(role_id); - collectEnabledRoles(¬ifications); + + /// An enabled role for some users has been removed, we need to recalculate the access rights. + collectEnabledRoles(¬ifications); /// collectEnabledRoles() must be called with the `mutex` locked. } } From f8eb1fad91f0ea389e1a6f7b61270b07aa606f76 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 7 Mar 2023 23:06:14 +0100 Subject: [PATCH 367/470] Disable parallel format in health check --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e30e709f363..c1103da1552 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -118,6 +118,7 @@ def clickhouse_execute_http( "http_connection_timeout": timeout, "http_receive_timeout": timeout, "http_send_timeout": timeout, + "output_format_parallel_formatting": 0, } if settings is not None: params.update(settings) From 9920a52c51d177bd18ffb9a5788b29c8af9a5ac1 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 6 Mar 2023 17:53:58 +0300 Subject: [PATCH 368/470] use std::lerp, constexpr hex.h --- base/base/hex.h | 214 ++++++++++++++++++ base/base/interpolate.h | 13 ++ programs/install/Install.cpp | 2 +- src/Backups/BackupCoordinationRemote.cpp | 2 +- src/Backups/BackupImpl.cpp | 2 +- src/Backups/IBackupCoordination.h | 2 +- src/Columns/tests/gtest_weak_hash_32.cpp | 2 +- src/Common/BinStringDecodeHelper.h | 2 +- src/Common/OpenTelemetryTraceContext.cpp | 2 +- src/Common/SymbolIndex.cpp | 2 +- src/Common/escapeForFileName.cpp | 2 +- src/Common/formatIPv6.cpp | 2 +- src/Common/formatIPv6.h | 2 +- src/Common/getHashOfLoadedBinary.cpp | 2 +- src/Common/getMappedArea.cpp | 2 +- src/Common/hex.cpp | 92 -------- src/Common/hex.h | 145 ------------ src/Common/interpolate.h | 20 -- src/Compression/CompressedReadBufferBase.cpp | 2 +- src/Compression/CompressionCodecMultiple.cpp | 2 +- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Daemon/SentryWriter.cpp | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Formats/BSONTypes.cpp | 2 +- src/Formats/verbosePrintString.cpp | 2 +- src/Functions/FunctionsCodingIP.cpp | 2 +- src/Functions/FunctionsCodingUUID.cpp | 2 +- src/Functions/URL/decodeURLComponent.cpp | 2 +- src/Functions/bitShiftRight.cpp | 2 +- src/Functions/decodeXMLComponent.cpp | 2 +- src/IO/HTTPChunkedReadBuffer.cpp | 2 +- src/IO/ReadHelpers.cpp | 2 +- src/IO/WriteHelpers.cpp | 2 +- src/IO/tests/gtest_hadoop_snappy_decoder.cpp | 2 +- src/Interpreters/Cache/FileCacheKey.h | 2 +- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/OpenTelemetrySpanLog.cpp | 2 +- .../tests/gtest_lru_file_cache.cpp | 2 +- src/Parsers/Access/ParserCreateUserQuery.cpp | 2 +- src/Server/KeeperTCPHandler.cpp | 2 +- .../Cache/ExternalDataSourceCache.cpp | 2 +- .../DistributedAsyncInsertDirectoryQueue.cpp | 2 +- src/Storages/LiveView/LiveViewSink.h | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 2 +- .../PartMetadataManagerWithCache.cpp | 2 +- .../MergeTree/SimpleMergeSelector.cpp | 10 +- .../MaterializedPostgreSQLConsumer.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/System/StorageSystemParts.cpp | 2 +- .../System/StorageSystemProjectionParts.cpp | 2 +- utils/checksum-for-compressed-block/main.cpp | 2 +- utils/wikistat-loader/main.cpp | 2 +- 58 files changed, 284 insertions(+), 314 deletions(-) create mode 100644 base/base/hex.h create mode 100644 base/base/interpolate.h delete mode 100644 src/Common/hex.cpp delete mode 100644 src/Common/hex.h delete mode 100644 src/Common/interpolate.h diff --git a/base/base/hex.h b/base/base/hex.h new file mode 100644 index 00000000000..e0c57f9dd42 --- /dev/null +++ b/base/base/hex.h @@ -0,0 +1,214 @@ +#pragma once + +#include +#include "types.h" + +/// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly. + +constexpr inline std::string_view hex_digit_to_char_uppercase_table = "0123456789ABCDEF"; +constexpr inline std::string_view hex_digit_to_char_lowercase_table = "0123456789abcdef"; + +constexpr char hexDigitUppercase(unsigned char c) +{ + return hex_digit_to_char_uppercase_table[c]; +} +constexpr char hexDigitLowercase(unsigned char c) +{ + return hex_digit_to_char_lowercase_table[c]; +} + +/// Maps 0..255 to 00..FF or 00..ff correspondingly + +constexpr inline std::string_view hex_byte_to_char_uppercase_table = // + "000102030405060708090A0B0C0D0E0F" + "101112131415161718191A1B1C1D1E1F" + "202122232425262728292A2B2C2D2E2F" + "303132333435363738393A3B3C3D3E3F" + "404142434445464748494A4B4C4D4E4F" + "505152535455565758595A5B5C5D5E5F" + "606162636465666768696A6B6C6D6E6F" + "707172737475767778797A7B7C7D7E7F" + "808182838485868788898A8B8C8D8E8F" + "909192939495969798999A9B9C9D9E9F" + "A0A1A2A3A4A5A6A7A8A9AAABACADAEAF" + "B0B1B2B3B4B5B6B7B8B9BABBBCBDBEBF" + "C0C1C2C3C4C5C6C7C8C9CACBCCCDCECF" + "D0D1D2D3D4D5D6D7D8D9DADBDCDDDEDF" + "E0E1E2E3E4E5E6E7E8E9EAEBECEDEEEF" + "F0F1F2F3F4F5F6F7F8F9FAFBFCFDFEFF"; + +constexpr inline std::string_view hex_byte_to_char_lowercase_table = // + "000102030405060708090a0b0c0d0e0f" + "101112131415161718191a1b1c1d1e1f" + "202122232425262728292a2b2c2d2e2f" + "303132333435363738393a3b3c3d3e3f" + "404142434445464748494a4b4c4d4e4f" + "505152535455565758595a5b5c5d5e5f" + "606162636465666768696a6b6c6d6e6f" + "707172737475767778797a7b7c7d7e7f" + "808182838485868788898a8b8c8d8e8f" + "909192939495969798999a9b9c9d9e9f" + "a0a1a2a3a4a5a6a7a8a9aaabacadaeaf" + "b0b1b2b3b4b5b6b7b8b9babbbcbdbebf" + "c0c1c2c3c4c5c6c7c8c9cacbcccdcecf" + "d0d1d2d3d4d5d6d7d8d9dadbdcdddedf" + "e0e1e2e3e4e5e6e7e8e9eaebecedeeef" + "f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff"; + +inline void writeHexByteUppercase(UInt8 byte, void * out) +{ + memcpy(out, &hex_byte_to_char_uppercase_table[static_cast(byte) * 2], 2); +} + +inline void writeHexByteLowercase(UInt8 byte, void * out) +{ + memcpy(out, &hex_byte_to_char_lowercase_table[static_cast(byte) * 2], 2); +} + +constexpr inline std::string_view bin_byte_to_char_table = // + "0000000000000001000000100000001100000100000001010000011000000111" + "0000100000001001000010100000101100001100000011010000111000001111" + "0001000000010001000100100001001100010100000101010001011000010111" + "0001100000011001000110100001101100011100000111010001111000011111" + "0010000000100001001000100010001100100100001001010010011000100111" + "0010100000101001001010100010101100101100001011010010111000101111" + "0011000000110001001100100011001100110100001101010011011000110111" + "0011100000111001001110100011101100111100001111010011111000111111" + "0100000001000001010000100100001101000100010001010100011001000111" + "0100100001001001010010100100101101001100010011010100111001001111" + "0101000001010001010100100101001101010100010101010101011001010111" + "0101100001011001010110100101101101011100010111010101111001011111" + "0110000001100001011000100110001101100100011001010110011001100111" + "0110100001101001011010100110101101101100011011010110111001101111" + "0111000001110001011100100111001101110100011101010111011001110111" + "0111100001111001011110100111101101111100011111010111111001111111" + "1000000010000001100000101000001110000100100001011000011010000111" + "1000100010001001100010101000101110001100100011011000111010001111" + "1001000010010001100100101001001110010100100101011001011010010111" + "1001100010011001100110101001101110011100100111011001111010011111" + "1010000010100001101000101010001110100100101001011010011010100111" + "1010100010101001101010101010101110101100101011011010111010101111" + "1011000010110001101100101011001110110100101101011011011010110111" + "1011100010111001101110101011101110111100101111011011111010111111" + "1100000011000001110000101100001111000100110001011100011011000111" + "1100100011001001110010101100101111001100110011011100111011001111" + "1101000011010001110100101101001111010100110101011101011011010111" + "1101100011011001110110101101101111011100110111011101111011011111" + "1110000011100001111000101110001111100100111001011110011011100111" + "1110100011101001111010101110101111101100111011011110111011101111" + "1111000011110001111100101111001111110100111101011111011011110111" + "1111100011111001111110101111101111111100111111011111111011111111"; + +inline void writeBinByte(UInt8 byte, void * out) +{ + memcpy(out, &bin_byte_to_char_table[static_cast(byte) * 8], 8); +} + +/// Produces hex representation of an unsigned int with leading zeros (for checksums) +template +inline void writeHexUIntImpl(TUInt uint_, char * out, std::string_view table) +{ + union + { + TUInt value; + UInt8 uint8[sizeof(TUInt)]; + }; + + value = uint_; + + for (size_t i = 0; i < sizeof(TUInt); ++i) + { + if constexpr (std::endian::native == std::endian::little) + memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); + else + memcpy(out + i * 2, &table[static_cast(uint8[i]) * 2], 2); + } +} + +template +inline void writeHexUIntUppercase(TUInt uint_, char * out) +{ + writeHexUIntImpl(uint_, out, hex_byte_to_char_uppercase_table); +} + +template +inline void writeHexUIntLowercase(TUInt uint_, char * out) +{ + writeHexUIntImpl(uint_, out, hex_byte_to_char_lowercase_table); +} + +template +std::string getHexUIntUppercase(TUInt uint_) +{ + std::string res(sizeof(TUInt) * 2, '\0'); + writeHexUIntUppercase(uint_, res.data()); + return res; +} + +template +std::string getHexUIntLowercase(TUInt uint_) +{ + std::string res(sizeof(TUInt) * 2, '\0'); + writeHexUIntLowercase(uint_, res.data()); + return res; +} + +/// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value. + +constexpr inline std::string_view hex_char_to_digit_table + = {"\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\x00\x01\x02\x03\x04\x05\x06\x07\x08\x09\xff\xff\xff\xff\xff\xff" //0-9 + "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //A-Z + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //a-z + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" + "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff", + 256}; + +constexpr UInt8 unhex(char c) +{ + return hex_char_to_digit_table[static_cast(c)]; +} + +constexpr UInt8 unhex2(const char * data) +{ + return static_cast(unhex(data[0])) * 0x10 + static_cast(unhex(data[1])); +} + +constexpr UInt16 unhex4(const char * data) +{ + return static_cast(unhex(data[0])) * 0x1000 + static_cast(unhex(data[1])) * 0x100 + + static_cast(unhex(data[2])) * 0x10 + static_cast(unhex(data[3])); +} + +template +constexpr TUInt unhexUInt(const char * data) +{ + TUInt res = 0; + if constexpr ((sizeof(TUInt) <= 8) || ((sizeof(TUInt) % 8) != 0)) + { + for (size_t i = 0; i < sizeof(TUInt) * 2; ++i, ++data) + { + res <<= 4; + res += unhex(*data); + } + } + else + { + for (size_t i = 0; i < sizeof(TUInt) / 8; ++i, data += 16) + { + res <<= 64; + res += unhexUInt(data); + } + } + return res; +} diff --git a/base/base/interpolate.h b/base/base/interpolate.h new file mode 100644 index 00000000000..1d4fc0b6257 --- /dev/null +++ b/base/base/interpolate.h @@ -0,0 +1,13 @@ +#pragma once +#include +#include + +/** Linear interpolation in logarithmic coordinates. + * Exponential interpolation is related to linear interpolation + * exactly in same way as geometric mean is related to arithmetic mean. + */ +constexpr double interpolateExponential(double min, double max, double ratio) +{ + assert(min > 0 && ratio >= 0 && ratio <= 1); + return min * std::pow(max / min, ratio); +} diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d568012bb26..80f3b0bbc63 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index c0fb4d5e066..8e43676f59c 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index b5f48a1a277..fb8abee814a 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Backups/IBackupCoordination.h b/src/Backups/IBackupCoordination.h index b75d856b50f..588a20d9eeb 100644 --- a/src/Backups/IBackupCoordination.h +++ b/src/Backups/IBackupCoordination.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index 5755cc3af72..cbf47790b9f 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include diff --git a/src/Common/BinStringDecodeHelper.h b/src/Common/BinStringDecodeHelper.h index 513a4196b6f..df3e014cfad 100644 --- a/src/Common/BinStringDecodeHelper.h +++ b/src/Common/BinStringDecodeHelper.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/Common/OpenTelemetryTraceContext.cpp b/src/Common/OpenTelemetryTraceContext.cpp index b62822ceda2..df4ee6a34bf 100644 --- a/src/Common/OpenTelemetryTraceContext.cpp +++ b/src/Common/OpenTelemetryTraceContext.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 6f31009b1d2..f1cace5017c 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -1,7 +1,7 @@ #if defined(__ELF__) && !defined(OS_FREEBSD) #include -#include +#include #include #include diff --git a/src/Common/escapeForFileName.cpp b/src/Common/escapeForFileName.cpp index bcca04706dc..a1f9bff28d0 100644 --- a/src/Common/escapeForFileName.cpp +++ b/src/Common/escapeForFileName.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Common/formatIPv6.cpp b/src/Common/formatIPv6.cpp index 7c027a23b4d..86e33beb7c3 100644 --- a/src/Common/formatIPv6.cpp +++ b/src/Common/formatIPv6.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index 7b88f93750b..be4dfc7391e 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include constexpr size_t IPV4_BINARY_LENGTH = 4; diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index da053750036..cc0ad0d2143 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include static int callback(dl_phdr_info * info, size_t, void * data) diff --git a/src/Common/getMappedArea.cpp b/src/Common/getMappedArea.cpp index 573d3194f3d..4f40c604c6a 100644 --- a/src/Common/getMappedArea.cpp +++ b/src/Common/getMappedArea.cpp @@ -4,7 +4,7 @@ #if defined(OS_LINUX) #include -#include +#include #include #include diff --git a/src/Common/hex.cpp b/src/Common/hex.cpp deleted file mode 100644 index e8f9b981062..00000000000 --- a/src/Common/hex.cpp +++ /dev/null @@ -1,92 +0,0 @@ -#include - -const char * const hex_digit_to_char_uppercase_table = "0123456789ABCDEF"; -const char * const hex_digit_to_char_lowercase_table = "0123456789abcdef"; - -const char * const hex_byte_to_char_uppercase_table = - "000102030405060708090A0B0C0D0E0F" - "101112131415161718191A1B1C1D1E1F" - "202122232425262728292A2B2C2D2E2F" - "303132333435363738393A3B3C3D3E3F" - "404142434445464748494A4B4C4D4E4F" - "505152535455565758595A5B5C5D5E5F" - "606162636465666768696A6B6C6D6E6F" - "707172737475767778797A7B7C7D7E7F" - "808182838485868788898A8B8C8D8E8F" - "909192939495969798999A9B9C9D9E9F" - "A0A1A2A3A4A5A6A7A8A9AAABACADAEAF" - "B0B1B2B3B4B5B6B7B8B9BABBBCBDBEBF" - "C0C1C2C3C4C5C6C7C8C9CACBCCCDCECF" - "D0D1D2D3D4D5D6D7D8D9DADBDCDDDEDF" - "E0E1E2E3E4E5E6E7E8E9EAEBECEDEEEF" - "F0F1F2F3F4F5F6F7F8F9FAFBFCFDFEFF"; - -const char * const hex_byte_to_char_lowercase_table = - "000102030405060708090a0b0c0d0e0f" - "101112131415161718191a1b1c1d1e1f" - "202122232425262728292a2b2c2d2e2f" - "303132333435363738393a3b3c3d3e3f" - "404142434445464748494a4b4c4d4e4f" - "505152535455565758595a5b5c5d5e5f" - "606162636465666768696a6b6c6d6e6f" - "707172737475767778797a7b7c7d7e7f" - "808182838485868788898a8b8c8d8e8f" - "909192939495969798999a9b9c9d9e9f" - "a0a1a2a3a4a5a6a7a8a9aaabacadaeaf" - "b0b1b2b3b4b5b6b7b8b9babbbcbdbebf" - "c0c1c2c3c4c5c6c7c8c9cacbcccdcecf" - "d0d1d2d3d4d5d6d7d8d9dadbdcdddedf" - "e0e1e2e3e4e5e6e7e8e9eaebecedeeef" - "f0f1f2f3f4f5f6f7f8f9fafbfcfdfeff"; - -const char * const hex_char_to_digit_table = - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\x00\x01\x02\x03\x04\x05\x06\x07\x08\x09\xff\xff\xff\xff\xff\xff" //0-9 - "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //A-Z - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\x0a\x0b\x0c\x0d\x0e\x0f\xff\xff\xff\xff\xff\xff\xff\xff\xff" //a-z - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff" - "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff"; - -const char * const bin_byte_to_char_table = - "0000000000000001000000100000001100000100000001010000011000000111" - "0000100000001001000010100000101100001100000011010000111000001111" - "0001000000010001000100100001001100010100000101010001011000010111" - "0001100000011001000110100001101100011100000111010001111000011111" - "0010000000100001001000100010001100100100001001010010011000100111" - "0010100000101001001010100010101100101100001011010010111000101111" - "0011000000110001001100100011001100110100001101010011011000110111" - "0011100000111001001110100011101100111100001111010011111000111111" - "0100000001000001010000100100001101000100010001010100011001000111" - "0100100001001001010010100100101101001100010011010100111001001111" - "0101000001010001010100100101001101010100010101010101011001010111" - "0101100001011001010110100101101101011100010111010101111001011111" - "0110000001100001011000100110001101100100011001010110011001100111" - "0110100001101001011010100110101101101100011011010110111001101111" - "0111000001110001011100100111001101110100011101010111011001110111" - "0111100001111001011110100111101101111100011111010111111001111111" - "1000000010000001100000101000001110000100100001011000011010000111" - "1000100010001001100010101000101110001100100011011000111010001111" - "1001000010010001100100101001001110010100100101011001011010010111" - "1001100010011001100110101001101110011100100111011001111010011111" - "1010000010100001101000101010001110100100101001011010011010100111" - "1010100010101001101010101010101110101100101011011010111010101111" - "1011000010110001101100101011001110110100101101011011011010110111" - "1011100010111001101110101011101110111100101111011011111010111111" - "1100000011000001110000101100001111000100110001011100011011000111" - "1100100011001001110010101100101111001100110011011100111011001111" - "1101000011010001110100101101001111010100110101011101011011010111" - "1101100011011001110110101101101111011100110111011101111011011111" - "1110000011100001111000101110001111100100111001011110011011100111" - "1110100011101001111010101110101111101100111011011110111011101111" - "1111000011110001111100101111001111110100111101011111011011110111" - "1111100011111001111110101111101111111100111111011111111011111111"; diff --git a/src/Common/hex.h b/src/Common/hex.h deleted file mode 100644 index 062a6c27f76..00000000000 --- a/src/Common/hex.h +++ /dev/null @@ -1,145 +0,0 @@ -#pragma once -#include - - -/// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly. - -extern const char * const hex_digit_to_char_uppercase_table; -extern const char * const hex_digit_to_char_lowercase_table; - -inline char hexDigitUppercase(unsigned char c) -{ - return hex_digit_to_char_uppercase_table[c]; -} - -inline char hexDigitLowercase(unsigned char c) -{ - return hex_digit_to_char_lowercase_table[c]; -} - - -#include -#include - -#include - - -/// Maps 0..255 to 00..FF or 00..ff correspondingly - -extern const char * const hex_byte_to_char_uppercase_table; -extern const char * const hex_byte_to_char_lowercase_table; - -inline void writeHexByteUppercase(UInt8 byte, void * out) -{ - memcpy(out, &hex_byte_to_char_uppercase_table[static_cast(byte) * 2], 2); -} - -inline void writeHexByteLowercase(UInt8 byte, void * out) -{ - memcpy(out, &hex_byte_to_char_lowercase_table[static_cast(byte) * 2], 2); -} - -extern const char * const bin_byte_to_char_table; - -inline void writeBinByte(UInt8 byte, void * out) -{ - memcpy(out, &bin_byte_to_char_table[static_cast(byte) * 8], 8); -} - -/// Produces hex representation of an unsigned int with leading zeros (for checksums) -template -inline void writeHexUIntImpl(TUInt uint_, char * out, const char * const table) -{ - union - { - TUInt value; - UInt8 uint8[sizeof(TUInt)]; - }; - - value = uint_; - - for (size_t i = 0; i < sizeof(TUInt); ++i) - { - if constexpr (std::endian::native == std::endian::little) - memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); - else - memcpy(out + i * 2, &table[static_cast(uint8[i]) * 2], 2); - } -} - -template -inline void writeHexUIntUppercase(TUInt uint_, char * out) -{ - writeHexUIntImpl(uint_, out, hex_byte_to_char_uppercase_table); -} - -template -inline void writeHexUIntLowercase(TUInt uint_, char * out) -{ - writeHexUIntImpl(uint_, out, hex_byte_to_char_lowercase_table); -} - -template -std::string getHexUIntUppercase(TUInt uint_) -{ - std::string res(sizeof(TUInt) * 2, '\0'); - writeHexUIntUppercase(uint_, res.data()); - return res; -} - -template -std::string getHexUIntLowercase(TUInt uint_) -{ - std::string res(sizeof(TUInt) * 2, '\0'); - writeHexUIntLowercase(uint_, res.data()); - return res; -} - - -/// Maps 0..9, A..F, a..f to 0..15. Other chars are mapped to implementation specific value. - -extern const char * const hex_char_to_digit_table; - -inline UInt8 unhex(char c) -{ - return hex_char_to_digit_table[static_cast(c)]; -} - -inline UInt8 unhex2(const char * data) -{ - return - static_cast(unhex(data[0])) * 0x10 - + static_cast(unhex(data[1])); -} - -inline UInt16 unhex4(const char * data) -{ - return - static_cast(unhex(data[0])) * 0x1000 - + static_cast(unhex(data[1])) * 0x100 - + static_cast(unhex(data[2])) * 0x10 - + static_cast(unhex(data[3])); -} - -template -TUInt unhexUInt(const char * data) -{ - TUInt res = 0; - if constexpr ((sizeof(TUInt) <= 8) || ((sizeof(TUInt) % 8) != 0)) - { - for (size_t i = 0; i < sizeof(TUInt) * 2; ++i, ++data) - { - res <<= 4; - res += unhex(*data); - } - } - else - { - for (size_t i = 0; i < sizeof(TUInt) / 8; ++i, data += 16) - { - res <<= 64; - res += unhexUInt(data); - } - } - return res; -} diff --git a/src/Common/interpolate.h b/src/Common/interpolate.h deleted file mode 100644 index 05900563b80..00000000000 --- a/src/Common/interpolate.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include - - -inline double interpolateLinear(double min, double max, double ratio) -{ - return min + (max - min) * ratio; -} - - -/** It is linear interpolation in logarithmic coordinates. - * Exponential interpolation is related to linear interpolation - * exactly in same way as geometric mean is related to arithmetic mean. - * 'min' must be greater than zero, 'ratio' must be from 0 to 1. - */ -inline double interpolateExponential(double min, double max, double ratio) -{ - return min * std::pow(max / min, ratio); -} diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index ab856cc9801..3111f649b26 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index 5203e349317..dba67749e4d 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a6d16334924..2aa11dd9eed 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 33b2a91d8bf..41a6af54204 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index 9f4f18e64d1..3c62e54b117 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include "config.h" #include "config_version.h" diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 44a719c82bb..72346787cfb 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 43b6544acb0..8450e740ab5 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Formats/BSONTypes.cpp b/src/Formats/BSONTypes.cpp index 813c155325a..88396fd2aba 100644 --- a/src/Formats/BSONTypes.cpp +++ b/src/Formats/BSONTypes.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Formats/verbosePrintString.cpp b/src/Formats/verbosePrintString.cpp index 2f3e09ed75f..5c6111c2929 100644 --- a/src/Formats/verbosePrintString.cpp +++ b/src/Formats/verbosePrintString.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index a941092b7d6..4784368db9b 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/FunctionsCodingUUID.cpp b/src/Functions/FunctionsCodingUUID.cpp index dade406c801..dd9170e44ad 100644 --- a/src/Functions/FunctionsCodingUUID.cpp +++ b/src/Functions/FunctionsCodingUUID.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Functions/URL/decodeURLComponent.cpp b/src/Functions/URL/decodeURLComponent.cpp index 9e516e73e3c..7d98ccd63a0 100644 --- a/src/Functions/URL/decodeURLComponent.cpp +++ b/src/Functions/URL/decodeURLComponent.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 108847f13ed..21a0f7584aa 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 8b84bb1194e..a25e67e0e37 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/src/IO/HTTPChunkedReadBuffer.cpp b/src/IO/HTTPChunkedReadBuffer.cpp index b9c42088c41..65ccad4aab7 100644 --- a/src/IO/HTTPChunkedReadBuffer.cpp +++ b/src/IO/HTTPChunkedReadBuffer.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 86a2b9c650e..e14b3ae9129 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index caeea0a82a3..a0eceddc6f6 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/IO/tests/gtest_hadoop_snappy_decoder.cpp b/src/IO/tests/gtest_hadoop_snappy_decoder.cpp index 4db0deac08e..2847c730735 100644 --- a/src/IO/tests/gtest_hadoop_snappy_decoder.cpp +++ b/src/IO/tests/gtest_hadoop_snappy_decoder.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include using namespace DB; TEST(HadoopSnappyDecoder, repeatNeedMoreInput) { diff --git a/src/Interpreters/Cache/FileCacheKey.h b/src/Interpreters/Cache/FileCacheKey.h index fed4c7f47e0..67e1466e2d4 100644 --- a/src/Interpreters/Cache/FileCacheKey.h +++ b/src/Interpreters/Cache/FileCacheKey.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index f4d7b2612a5..bd4554c6532 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 29b7a4db609..c352280b7ed 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index 57d5c11ad97..63b8ae406a6 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 93faafb5cea..62aef2441d6 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index de83c5760c1..c1d0691d305 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -17,7 +17,7 @@ #include #include #include "config.h" -#include +#include #if USE_SSL # include # include diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 0853c6ee62b..f9e11062906 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #ifdef POCO_HAVE_FD_EPOLL diff --git a/src/Storages/Cache/ExternalDataSourceCache.cpp b/src/Storages/Cache/ExternalDataSourceCache.cpp index 56b2e661836..1fc68a2d774 100644 --- a/src/Storages/Cache/ExternalDataSourceCache.cpp +++ b/src/Storages/Cache/ExternalDataSourceCache.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index c6f675533c6..b47028b883a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index 1d90e35618f..e163400f2af 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 547becf3837..b24f5fb2c63 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -27,7 +27,7 @@ limitations under the License. */ #include #include #include -#include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0d123623f05..cf009a10c27 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -31,7 +31,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 8f4d066baa3..7a0b1d03e79 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -1,6 +1,6 @@ #include "MergeTreeDataPartChecksum.h" #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 3a6908ef32d..3b28012e7d6 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp index b6260d5edb6..3a53cf25745 100644 --- a/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp +++ b/src/Storages/MergeTree/PartMetadataManagerWithCache.cpp @@ -1,7 +1,7 @@ #include "PartMetadataManagerWithCache.h" #if USE_ROCKSDB -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/SimpleMergeSelector.cpp b/src/Storages/MergeTree/SimpleMergeSelector.cpp index 15291622a2a..af3373fd175 100644 --- a/src/Storages/MergeTree/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/SimpleMergeSelector.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include @@ -28,7 +28,7 @@ struct Estimator { double difference = std::abs(log2(static_cast(sum_size) / size_prev_at_left)); if (difference < settings.heuristic_to_align_parts_max_absolute_difference_in_powers_of_two) - current_score *= interpolateLinear(settings.heuristic_to_align_parts_max_score_adjustment, 1, + current_score *= std::lerp(settings.heuristic_to_align_parts_max_score_adjustment, 1, difference / settings.heuristic_to_align_parts_max_absolute_difference_in_powers_of_two); } @@ -115,8 +115,8 @@ bool allow( // std::cerr << "size_normalized: " << size_normalized << "\n"; /// Calculate boundaries for age - double min_age_to_lower_base = interpolateLinear(settings.min_age_to_lower_base_at_min_size, settings.min_age_to_lower_base_at_max_size, size_normalized); - double max_age_to_lower_base = interpolateLinear(settings.max_age_to_lower_base_at_min_size, settings.max_age_to_lower_base_at_max_size, size_normalized); + double min_age_to_lower_base = std::lerp(settings.min_age_to_lower_base_at_min_size, settings.min_age_to_lower_base_at_max_size, size_normalized); + double max_age_to_lower_base = std::lerp(settings.max_age_to_lower_base_at_min_size, settings.max_age_to_lower_base_at_max_size, size_normalized); // std::cerr << "min_age_to_lower_base: " << min_age_to_lower_base << "\n"; // std::cerr << "max_age_to_lower_base: " << max_age_to_lower_base << "\n"; @@ -137,7 +137,7 @@ bool allow( // std::cerr << "combined_ratio: " << combined_ratio << "\n"; - double lowered_base = interpolateLinear(settings.base, 2.0, combined_ratio); + double lowered_base = std::lerp(settings.base, 2.0, combined_ratio); // std::cerr << "------- lowered_base: " << lowered_base << "\n"; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 9c6eeceb605..d048c94ac75 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -2,7 +2,7 @@ #include "StorageMaterializedPostgreSQL.h" #include -#include +#include #include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 54ae8aa5a7b..4c0aced0b48 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index f6854e7d5d0..86ecb336b51 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 37c62ba5eb0..d2c6c3ef287 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/utils/checksum-for-compressed-block/main.cpp b/utils/checksum-for-compressed-block/main.cpp index 27a2154340e..4f9923e7638 100644 --- a/utils/checksum-for-compressed-block/main.cpp +++ b/utils/checksum-for-compressed-block/main.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include /** A tool to easily prove if "Checksum doesn't match: corrupted data" diff --git a/utils/wikistat-loader/main.cpp b/utils/wikistat-loader/main.cpp index 31ade014c74..493f1df05da 100644 --- a/utils/wikistat-loader/main.cpp +++ b/utils/wikistat-loader/main.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include From 429696b575a1eaa4d2206a7b0cd7d925060cc6c6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Mar 2023 22:57:47 +0000 Subject: [PATCH 369/470] Update reference file --- .../00173_group_by_use_nulls.reference | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/tests/queries/1_stateful/00173_group_by_use_nulls.reference b/tests/queries/1_stateful/00173_group_by_use_nulls.reference index 02723bf14dd..e82b996ad3c 100644 --- a/tests/queries/1_stateful/00173_group_by_use_nulls.reference +++ b/tests/queries/1_stateful/00173_group_by_use_nulls.reference @@ -8,3 +8,25 @@ 59183 1336 33010362 1336 800784 1336 +-- { echoOn } +set allow_experimental_analyzer = 1; +SELECT + CounterID AS k, + quantileBFloat16(0.5)(ResolutionWidth) +FROM remote('127.0.0.{1,2}', test, hits) +GROUP BY k +ORDER BY + count() DESC, + CounterID ASC +LIMIT 10 +SETTINGS group_by_use_nulls = 1; +1704509 1384 +732797 1336 +598875 1384 +792887 1336 +3807842 1336 +25703952 1336 +716829 1384 +59183 1336 +33010362 1336 +800784 1336 From f5cf039190d547f52d718b2a16403ce8eecad8ab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 7 Mar 2023 21:27:27 +0100 Subject: [PATCH 370/470] check if a symlink exists more carefully --- src/Common/filesystemHelpers.cpp | 8 ++++++++ src/Common/filesystemHelpers.h | 1 + src/Databases/DatabaseAtomic.cpp | 3 +++ 3 files changed, 12 insertions(+) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 6e1b5573bef..eabc7bdafbb 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -383,6 +383,14 @@ bool isSymlink(const fs::path & path) return fs::is_symlink(path); /// STYLE_CHECK_ALLOW_STD_FS_SYMLINK } +bool isSymlinkNoThrow(const fs::path & path) +{ + std::error_code dummy; + if (path.filename().empty()) + return fs::is_symlink(path.parent_path(), dummy); /// STYLE_CHECK_ALLOW_STD_FS_SYMLINK + return fs::is_symlink(path, dummy); /// STYLE_CHECK_ALLOW_STD_FS_SYMLINK +} + fs::path readSymlink(const fs::path & path) { /// See the comment for isSymlink diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 14ee5f54322..8591cd6cf92 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -95,6 +95,7 @@ void setModificationTime(const std::string & path, time_t time); time_t getChangeTime(const std::string & path); bool isSymlink(const fs::path & path); +bool isSymlinkNoThrow(const fs::path & path); fs::path readSymlink(const fs::path & path); } diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 34c4fd3d5d8..d8e746f01f9 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -509,6 +509,9 @@ void DatabaseAtomic::tryCreateMetadataSymlink() { try { + /// fs::exists could return false for broken symlink + if (FS::isSymlinkNoThrow(metadata_symlink)) + fs::remove(metadata_symlink); fs::create_directory_symlink(metadata_path, path_to_metadata_symlink); } catch (...) From 51edcf232cf2962d51d57f808ab6cea42e28327d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 8 Mar 2023 00:52:58 +0000 Subject: [PATCH 371/470] Add support for big int types to runningDifference() function --- src/Functions/runningDifference.h | 10 +++++++- .../queries/0_stateless/00534_filimonov.data | 8 +++++++ .../00653_running_difference.reference | 24 +++++++++++++++++++ .../0_stateless/00653_running_difference.sql | 8 +++++++ 4 files changed, 49 insertions(+), 1 deletion(-) diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index 154370d4cd9..f1ec4f9e523 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -70,7 +70,7 @@ private: if (!has_prev_value) { - dst[i] = is_first_line_zero ? 0 : src[i]; + dst[i] = is_first_line_zero ? static_cast(0) : static_cast(src[i]); prev = src[i]; has_prev_value = true; } @@ -102,6 +102,10 @@ private: f(UInt32()); else if (which.isUInt64()) f(UInt64()); + else if (which.isUInt128()) + f(UInt128()); + else if (which.isUInt256()) + f(UInt256()); else if (which.isInt8()) f(Int8()); else if (which.isInt16()) @@ -110,6 +114,10 @@ private: f(Int32()); else if (which.isInt64()) f(Int64()); + else if (which.isInt128()) + f(Int128()); + else if (which.isInt256()) + f(Int256()); else if (which.isFloat32()) f(Float32()); else if (which.isFloat64()) diff --git a/tests/queries/0_stateless/00534_filimonov.data b/tests/queries/0_stateless/00534_filimonov.data index 911a8e4d1f3..eb4500877e5 100644 --- a/tests/queries/0_stateless/00534_filimonov.data +++ b/tests/queries/0_stateless/00534_filimonov.data @@ -276,10 +276,14 @@ SELECT runningDifference(CAST( 0 AS Nullable(Int8))); SELECT runningDifference(CAST( 0 AS Nullable(Int16))); SELECT runningDifference(CAST( 0 AS Nullable(Int32))); SELECT runningDifference(CAST( 0 AS Nullable(Int64))); +SELECT runningDifference(CAST( 0 AS Nullable(Int128))); +SELECT runningDifference(CAST( 0 AS Nullable(Int256))); SELECT runningDifference(CAST( 0 AS Nullable(UInt8))); SELECT runningDifference(CAST( 0 AS Nullable(UInt16))); SELECT runningDifference(CAST( 0 AS Nullable(UInt32))); SELECT runningDifference(CAST( 0 AS Nullable(UInt64))); +SELECT runningDifference(CAST( 0 AS Nullable(UInt128))); +SELECT runningDifference(CAST( 0 AS Nullable(UInt256))); SELECT runningDifference(CAST( 0 AS Nullable(Float32))); SELECT runningDifference(CAST( 0 AS Nullable(Float64))); SELECT runningDifference(CAST( 0 AS Nullable(Date))); @@ -288,10 +292,14 @@ SELECT runningDifference(CAST(NULL AS Nullable(Int8))); SELECT runningDifference(CAST(NULL AS Nullable(Int16))); SELECT runningDifference(CAST(NULL AS Nullable(Int32))); SELECT runningDifference(CAST(NULL AS Nullable(Int64))); +SELECT runningDifference(CAST(NULL AS Nullable(Int128))); +SELECT runningDifference(CAST(NULL AS Nullable(Int256))); SELECT runningDifference(CAST(NULL AS Nullable(UInt8))); SELECT runningDifference(CAST(NULL AS Nullable(UInt16))); SELECT runningDifference(CAST(NULL AS Nullable(UInt32))); SELECT runningDifference(CAST(NULL AS Nullable(UInt64))); +SELECT runningDifference(CAST(NULL AS Nullable(UInt128))); +SELECT runningDifference(CAST(NULL AS Nullable(UInt256)); SELECT runningDifference(CAST(NULL AS Nullable(Float32))); SELECT runningDifference(CAST(NULL AS Nullable(Float64))); SELECT runningDifference(CAST(NULL AS Nullable(Date))); diff --git a/tests/queries/0_stateless/00653_running_difference.reference b/tests/queries/0_stateless/00653_running_difference.reference index e2833e0bb3e..624ce92ce0f 100644 --- a/tests/queries/0_stateless/00653_running_difference.reference +++ b/tests/queries/0_stateless/00653_running_difference.reference @@ -19,6 +19,30 @@ \N \N 2 +- +0 +1 +4 +5 +170141183460469231731687303715884105717 +- +0 +1 +4 +5 +170141183460469231731687303715884105718 +- +0 +1 +4 +5 +170141183460469231731687303715884105717 +- +0 +1 +4 +5 +170141183460469231731687303715884105718 --Date Difference-- \N \N diff --git a/tests/queries/0_stateless/00653_running_difference.sql b/tests/queries/0_stateless/00653_running_difference.sql index f2b4a7300b2..1f18cfc42a7 100644 --- a/tests/queries/0_stateless/00653_running_difference.sql +++ b/tests/queries/0_stateless/00653_running_difference.sql @@ -5,6 +5,14 @@ select '-'; select runningDifference(x) from (select arrayJoin([Null, 1]) as x); select '-'; select runningDifference(x) from (select arrayJoin([Null, Null, 1, 3, Null, Null, 5]) as x); +select '-'; +select runningDifference(x) from (select arrayJoin([0, 1, 5, 10, 170141183460469231731687303715884105727]::Array(UInt128)) as x); +select '-'; +select runningDifference(x) from (select arrayJoin([0, 1, 5, 10, 170141183460469231731687303715884105728]::Array(UInt256)) as x); +select '-'; +select runningDifference(x) from (select arrayJoin([0, 1, 5, 10, 170141183460469231731687303715884105727]::Array(Int128)) as x); +select '-'; +select runningDifference(x) from (select arrayJoin([0, 1, 5, 10, 170141183460469231731687303715884105728]::Array(Int256)) as x); select '--Date Difference--'; select runningDifference(x) from (select arrayJoin([Null, Null, toDate('1970-1-1'), toDate('1970-12-31'), Null, Null, toDate('2010-8-9')]) as x); select '-'; From 927efc0f2c0e164a2884c0d2ec087fa7dfbae7c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=5B=EB=8D=B0=EC=9D=B4=ED=84=B0=ED=94=8C=EB=9E=AB=ED=8F=BC?= =?UTF-8?q?=ED=8C=80=5D=20=EC=9D=B4=ED=98=B8=EC=84=A0?= Date: Wed, 8 Mar 2023 14:01:38 +0900 Subject: [PATCH 372/470] Fix git clone command. Insert space in front of shallow option. --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index b46cc10f99d..ace5ab79bb4 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -67,7 +67,7 @@ It generally means that the SSH keys for connecting to GitHub are missing. These You can also clone the repository via https protocol: - git clone --recursive--shallow-submodules https://github.com/ClickHouse/ClickHouse.git + git clone --recursive --shallow-submodules https://github.com/ClickHouse/ClickHouse.git This, however, will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command. From ce809e61fb18eb6b05105ffe2fef03124996ba75 Mon Sep 17 00:00:00 2001 From: Hosun Lee Date: Wed, 8 Mar 2023 14:09:18 +0900 Subject: [PATCH 373/470] Revert commit. --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index ace5ab79bb4..94834f41999 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -39,7 +39,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --recursive--shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Note: please, substitute *your_github_username* with what is appropriate! From aae82a73aa7ae838cdbecda5f1a305186de2fd47 Mon Sep 17 00:00:00 2001 From: Hosun Lee Date: Wed, 8 Mar 2023 14:10:12 +0900 Subject: [PATCH 374/470] Fix git clone command. Insert space in front of shallow option. --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 94834f41999..ace5ab79bb4 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -39,7 +39,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone --recursive--shallow-submodules git@github.com:your_github_username/ClickHouse.git + git clone --recursive --shallow-submodules git@github.com:your_github_username/ClickHouse.git cd ClickHouse Note: please, substitute *your_github_username* with what is appropriate! From 7966c114bdbbc97f98430a44f83c41c7bfcc0d9f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 8 Mar 2023 05:40:10 +0000 Subject: [PATCH 375/470] analyzer - fix combine logic for limit expression and limit setting --- src/Analyzer/QueryTreeBuilder.cpp | 53 +++++++++++++++++++++++++------ 1 file changed, 44 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index c7b9f9aae08..ed956071d5a 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -355,21 +355,56 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q if (select_limit_by) current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by, current_context); - /// Combine limit expression with limit setting + /// Combine limit expression with limit and offset settings into final limit expression + /// select_limit - limit expression + /// limit - limit setting + /// offset - offset setting + /// + /// if select_limit + /// -- if offset >= select_limit (expr 0) + /// then (0) (0 rows) + /// -- else if limit > 0 (expr 1) + /// then min(select_limit - offset, limit) (expr 2) + /// -- else + /// then (select_limit - offset) (expr 3) + /// else if limit > 0 + /// then limit + /// + /// offset = offset + of_expr auto select_limit = select_query_typed.limitLength(); - if (select_limit && limit) + if (select_limit) { - auto function_node = std::make_shared("least"); - function_node->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); - function_node->getArguments().getNodes().push_back(std::make_shared(limit)); + /// expr 3 + auto expr_3 = std::make_shared("minus"); + expr_3->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); + expr_3->getArguments().getNodes().push_back(std::make_shared(offset)); + + /// expr 2 + auto expr_2 = std::make_shared("least"); + expr_2->getArguments().getNodes().push_back(expr_3->clone()); + expr_2->getArguments().getNodes().push_back(std::make_shared(limit)); + + /// expr 0 + auto expr_0 = std::make_shared("greaterOrEquals"); + expr_0->getArguments().getNodes().push_back(std::make_shared(offset)); + expr_0->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); + + /// expr 1 + auto expr_1 = std::make_shared(limit > 0); + + auto function_node = std::make_shared("multiIf"); + function_node->getArguments().getNodes().push_back(expr_0); + function_node->getArguments().getNodes().push_back(std::make_shared(0)); + function_node->getArguments().getNodes().push_back(expr_1); + function_node->getArguments().getNodes().push_back(expr_2); + function_node->getArguments().getNodes().push_back(expr_3); + current_query_tree->getLimit() = std::move(function_node); } - else if (limit) + else if (limit > 0) current_query_tree->getLimit() = std::make_shared(limit); - else if (select_limit) - current_query_tree->getLimit() = buildExpression(select_limit, current_context); - /// Combine offset expression with offset setting + /// Combine offset expression with offset setting into final offset expression auto select_offset = select_query_typed.limitOffset(); if (select_offset && offset) { From 036fb0b9efb6d8aef2bc2837fc6696f8ca39c4e2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 8 Mar 2023 05:52:04 +0000 Subject: [PATCH 376/470] analyzer - add test --- .../02676_analyzer_limit_offset.reference | 63 +++++++++++++++++++ .../02676_analyzer_limit_offset.sql | 34 ++++++++++ 2 files changed, 97 insertions(+) create mode 100644 tests/queries/0_stateless/02676_analyzer_limit_offset.reference create mode 100644 tests/queries/0_stateless/02676_analyzer_limit_offset.sql diff --git a/tests/queries/0_stateless/02676_analyzer_limit_offset.reference b/tests/queries/0_stateless/02676_analyzer_limit_offset.reference new file mode 100644 index 00000000000..96483268d43 --- /dev/null +++ b/tests/queries/0_stateless/02676_analyzer_limit_offset.reference @@ -0,0 +1,63 @@ +0 +1 +2 +3 +4 +15 +15 +16 +16 +17 +30 +30 +31 +31 +32 +102 +103 +104 +105 +105 +106 +107 +108 +109 +105 +106 +107 +108 +109 +60 +60 +61 +61 +62 +62 +63 +63 +64 +64 +60 +35 +35 +36 +36 +37 +37 +38 +38 +39 +39 +105 +106 +107 +108 +109 +12 +13 +13 +14 +14 +15 +15 +16 diff --git a/tests/queries/0_stateless/02676_analyzer_limit_offset.sql b/tests/queries/0_stateless/02676_analyzer_limit_offset.sql new file mode 100644 index 00000000000..39c6b85f088 --- /dev/null +++ b/tests/queries/0_stateless/02676_analyzer_limit_offset.sql @@ -0,0 +1,34 @@ +set allow_experimental_analyzer=1; + +DROP TABLE IF EXISTS test; +CREATE TABLE test (i UInt64) Engine = MergeTree() order by i; +INSERT INTO test SELECT number FROM numbers(100); +INSERT INTO test SELECT number FROM numbers(10,100); +OPTIMIZE TABLE test FINAL; + +-- Only set limit +SET limit = 5; +SELECT * FROM test; -- 5 rows +SELECT * FROM test OFFSET 20; -- 5 rows +SELECT * FROM (SELECT i FROM test LIMIT 10 OFFSET 50) TMP; -- 5 rows +SELECT * FROM test LIMIT 4 OFFSET 192; -- 4 rows +SELECT * FROM test LIMIT 10 OFFSET 195; -- 5 rows + +-- Only set offset +SET limit = 0; +SET offset = 195; +SELECT * FROM test; -- 5 rows +SELECT * FROM test OFFSET 20; -- no result +SELECT * FROM test LIMIT 100; -- no result +SET offset = 10; +SELECT * FROM test LIMIT 20 OFFSET 100; -- 10 rows +SELECT * FROM test LIMIT 11 OFFSET 100; -- 1 rows + +-- offset and limit together +SET limit = 10; +SELECT * FROM test LIMIT 50 OFFSET 50; -- 10 rows +SELECT * FROM test LIMIT 50 OFFSET 190; -- 0 rows +SELECT * FROM test LIMIT 50 OFFSET 185; -- 5 rows +SELECT * FROM test LIMIT 18 OFFSET 5; -- 8 rows + +DROP TABLE test; From 31b7e6edc64689d1d682506530a426c8f06ae1b6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 7 Mar 2023 21:21:59 +0100 Subject: [PATCH 377/470] Keep subscriptions for all enabled roles. --- src/Access/EnabledRoles.h | 5 +-- src/Access/RoleCache.cpp | 71 +++++++++++++++++++++++++-------------- src/Access/RoleCache.h | 24 ++++++++++--- 3 files changed, 67 insertions(+), 33 deletions(-) diff --git a/src/Access/EnabledRoles.h b/src/Access/EnabledRoles.h index e0d773db343..5de76abe409 100644 --- a/src/Access/EnabledRoles.h +++ b/src/Access/EnabledRoles.h @@ -44,10 +44,11 @@ private: friend class RoleCache; explicit EnabledRoles(const Params & params_); - void setRolesInfo(const std::shared_ptr & info_, scope_guard * notifications); - const Params params; + /// Called by RoleCache to store `EnabledRolesInfo` in this `EnabledRoles` after the calculation is done. + void setRolesInfo(const std::shared_ptr & info_, scope_guard * notifications); + std::shared_ptr info; mutable std::mutex info_mutex; diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index 7a8c13636d2..bfc6200929d 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -57,7 +57,9 @@ namespace RoleCache::RoleCache(const AccessControl & access_control_) - : access_control(access_control_), cache(600000 /* 10 minutes */) {} + : access_control(access_control_), cache(600000 /* 10 minutes */) +{ +} RoleCache::~RoleCache() = default; @@ -70,18 +72,18 @@ RoleCache::getEnabledRoles(const std::vector & roles, const std::vectorsecond.lock(); - if (from_cache) - return from_cache; - enabled_roles.erase(it); + if (auto enabled_roles = it->second.enabled_roles.lock()) + return enabled_roles; + enabled_roles_by_params.erase(it); } auto res = std::shared_ptr(new EnabledRoles(params)); - collectEnabledRoles(*res, nullptr); - enabled_roles.emplace(std::move(params), res); + SubscriptionsOnRoles subscriptions_on_roles; + collectEnabledRoles(*res, subscriptions_on_roles, nullptr); + enabled_roles_by_params.emplace(std::move(params), EnabledRolesWithSubscriptions{res, std::move(subscriptions_on_roles)}); return res; } @@ -90,21 +92,23 @@ void RoleCache::collectEnabledRoles(scope_guard * notifications) { /// `mutex` is already locked. - for (auto i = enabled_roles.begin(), e = enabled_roles.end(); i != e;) + for (auto i = enabled_roles_by_params.begin(), e = enabled_roles_by_params.end(); i != e;) { - auto elem = i->second.lock(); - if (!elem) - i = enabled_roles.erase(i); + auto & item = i->second; + if (auto enabled_roles = item.enabled_roles.lock()) + { + collectEnabledRoles(*enabled_roles, item.subscriptions_on_roles, notifications); + ++i; + } else { - collectEnabledRoles(*elem, notifications); - ++i; + i = enabled_roles_by_params.erase(i); } } } -void RoleCache::collectEnabledRoles(EnabledRoles & enabled, scope_guard * notifications) +void RoleCache::collectEnabledRoles(EnabledRoles & enabled_roles, SubscriptionsOnRoles & subscriptions_on_roles, scope_guard * notifications) { /// `mutex` is already locked. @@ -112,43 +116,57 @@ void RoleCache::collectEnabledRoles(EnabledRoles & enabled, scope_guard * notifi auto new_info = std::make_shared(); boost::container::flat_set skip_ids; - auto get_role_function = [this](const UUID & id) { return getRole(id); }; + /// We need to collect and keep not only enabled roles but also subscriptions for them to be able to recalculate EnabledRolesInfo when some of the roles change. + SubscriptionsOnRoles new_subscriptions_on_roles; + new_subscriptions_on_roles.reserve(subscriptions_on_roles.size()); - for (const auto & current_role : enabled.params.current_roles) + auto get_role_function = [this, &subscriptions_on_roles](const UUID & id) TSA_NO_THREAD_SAFETY_ANALYSIS { return getRole(id, 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); - for (const auto & current_role : enabled.params.current_roles_with_admin_option) + for (const auto & current_role : enabled_roles.params.current_roles_with_admin_option) collectRoles(*new_info, skip_ids, get_role_function, current_role, true, true); + /// Remove duplicates from `subscriptions_on_roles`. + std::sort(new_subscriptions_on_roles.begin(), new_subscriptions_on_roles.end()); + new_subscriptions_on_roles.erase(std::unique(new_subscriptions_on_roles.begin(), new_subscriptions_on_roles.end()), new_subscriptions_on_roles.end()); + subscriptions_on_roles = std::move(new_subscriptions_on_roles); + /// Collect data from the collected roles. - enabled.setRolesInfo(new_info, notifications); + enabled_roles.setRolesInfo(new_info, notifications); } -RolePtr RoleCache::getRole(const UUID & role_id) +RolePtr RoleCache::getRole(const UUID & role_id, SubscriptionsOnRoles & subscriptions_on_roles) { /// `mutex` is already locked. auto role_from_cache = cache.get(role_id); if (role_from_cache) + { + subscriptions_on_roles.emplace_back(role_from_cache->second); return role_from_cache->first; + } - auto subscription = access_control.subscribeForChanges(role_id, - [this, role_id](const UUID &, const AccessEntityPtr & entity) + auto on_role_changed_or_removed = [this, role_id](const UUID &, const AccessEntityPtr & entity) { auto changed_role = entity ? typeid_cast(entity) : nullptr; if (changed_role) roleChanged(role_id, changed_role); else roleRemoved(role_id); - }); + }; + + auto subscription_on_role = std::make_shared(access_control.subscribeForChanges(role_id, on_role_changed_or_removed)); auto role = access_control.tryRead(role_id); if (role) { - auto cache_value = Poco::SharedPtr>( - new std::pair{role, std::move(subscription)}); + auto cache_value = Poco::SharedPtr>>( + new std::pair>{role, subscription_on_role}); cache.add(role_id, cache_value); + subscriptions_on_roles.emplace_back(subscription_on_role); return role; } @@ -162,6 +180,7 @@ void RoleCache::roleChanged(const UUID & role_id, const RolePtr & changed_role) scope_guard notifications; std::lock_guard lock{mutex}; + auto role_from_cache = cache.get(role_id); if (role_from_cache) { diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index 51c415d4d1d..24f19cb9d94 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -24,15 +24,29 @@ public: const std::vector & current_roles_with_admin_option); private: - void collectEnabledRoles(scope_guard * notifications); - void collectEnabledRoles(EnabledRoles & enabled, scope_guard * notifications); - RolePtr getRole(const UUID & role_id); + using SubscriptionsOnRoles = std::vector>; + + void collectEnabledRoles(scope_guard * notifications) TSA_REQUIRES(mutex); + void collectEnabledRoles(EnabledRoles & enabled_roles, SubscriptionsOnRoles & subscriptions_on_roles, scope_guard * notifications) TSA_REQUIRES(mutex); + RolePtr getRole(const UUID & role_id, SubscriptionsOnRoles & subscriptions_on_roles) TSA_REQUIRES(mutex); void roleChanged(const UUID & role_id, const RolePtr & changed_role); void roleRemoved(const UUID & role_id); const AccessControl & access_control; - Poco::AccessExpireCache> cache; - std::map> enabled_roles; + + Poco::AccessExpireCache>> TSA_GUARDED_BY(mutex) cache; + + struct EnabledRolesWithSubscriptions + { + std::weak_ptr enabled_roles; + + /// We need to keep subscriptions for all enabled roles to be able to recalculate EnabledRolesInfo when some of the roles change. + /// `cache` also keeps subscriptions but that's not enough because values can be purged from the `cache` anytime. + SubscriptionsOnRoles subscriptions_on_roles; + }; + + std::map TSA_GUARDED_BY(mutex) enabled_roles_by_params; + mutable std::mutex mutex; }; From b2b8b7f872591714e10a702c64fd58aebd0e6457 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Mar 2023 19:05:38 +0100 Subject: [PATCH 378/470] Fix RewriteArrayExistsToHasPass --- src/Analyzer/Passes/ArrayExistsToHasPass.cpp | 130 ++++++++++-------- src/Analyzer/Passes/ArrayExistsToHasPass.h | 12 +- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 3 +- .../Passes/ShardNumColumnToFunctionPass.h | 3 + 4 files changed, 87 insertions(+), 61 deletions(-) diff --git a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp index b4b8b5b4579..c0f958588f1 100644 --- a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp +++ b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -8,71 +10,85 @@ #include #include -#include "ArrayExistsToHasPass.h" - namespace DB { + namespace { - class RewriteArrayExistsToHasVisitor : public InDepthQueryTreeVisitorWithContext + +class RewriteArrayExistsToHasVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) { - public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + if (!getSettings().optimize_rewrite_array_exists_to_has) + return; - void visitImpl(QueryTreeNodePtr & node) + auto * array_exists_function_node = node->as(); + if (!array_exists_function_node || array_exists_function_node->getFunctionName() != "arrayExists") + return; + + auto & array_exists_function_arguments_nodes = array_exists_function_node->getArguments().getNodes(); + if (array_exists_function_arguments_nodes.size() != 2) + return; + + /// lambda function must be like: x -> x = elem + auto * lambda_node = array_exists_function_arguments_nodes[0]->as(); + if (!lambda_node) + return; + + auto & lambda_arguments_nodes = lambda_node->getArguments().getNodes(); + if (lambda_arguments_nodes.size() != 1) + return; + + const auto & lambda_argument_column_node = lambda_arguments_nodes[0]; + if (lambda_argument_column_node->getNodeType() != QueryTreeNodeType::COLUMN) + return; + + auto * filter_node = lambda_node->getExpression()->as(); + if (!filter_node || filter_node->getFunctionName() != "equals") + return; + + const auto & filter_arguments_nodes = filter_node->getArguments().getNodes(); + if (filter_arguments_nodes.size() != 2) + return; + + const auto & filter_lhs_argument_node = filter_arguments_nodes[0]; + auto filter_lhs_argument_node_type = filter_lhs_argument_node->getNodeType(); + + const auto & filter_rhs_argument_node = filter_arguments_nodes[1]; + auto filter_rhs_argument_node_type = filter_rhs_argument_node->getNodeType(); + + QueryTreeNodePtr has_constant_element_argument; + + if (filter_lhs_argument_node_type == QueryTreeNodeType::COLUMN && + filter_rhs_argument_node_type == QueryTreeNodeType::CONSTANT && + filter_lhs_argument_node->isEqual(*lambda_argument_column_node)) { - if (!getSettings().optimize_rewrite_array_exists_to_has) - return; - - auto * function_node = node->as(); - if (!function_node || function_node->getFunctionName() != "arrayExists") - return; - - auto & function_arguments_nodes = function_node->getArguments().getNodes(); - if (function_arguments_nodes.size() != 2) - return; - - /// lambda function must be like: x -> x = elem - auto * lambda_node = function_arguments_nodes[0]->as(); - if (!lambda_node) - return; - - auto & lambda_arguments_nodes = lambda_node->getArguments().getNodes(); - if (lambda_arguments_nodes.size() != 1) - return; - auto * column_node = lambda_arguments_nodes[0]->as(); - - auto * filter_node = lambda_node->getExpression()->as(); - if (!filter_node || filter_node->getFunctionName() != "equals") - return; - - auto filter_arguments_nodes = filter_node->getArguments().getNodes(); - if (filter_arguments_nodes.size() != 2) - return; - - ColumnNode * filter_column_node = nullptr; - if (filter_arguments_nodes[1]->as() && (filter_column_node = filter_arguments_nodes[0]->as()) - && filter_column_node->getColumnName() == column_node->getColumnName()) - { - /// Rewrite arrayExists(x -> x = elem, arr) -> has(arr, elem) - function_arguments_nodes[0] = std::move(function_arguments_nodes[1]); - function_arguments_nodes[1] = std::move(filter_arguments_nodes[1]); - function_node->resolveAsFunction( - FunctionFactory::instance().get("has", getContext())->build(function_node->getArgumentColumns())); - } - else if ( - filter_arguments_nodes[0]->as() && (filter_column_node = filter_arguments_nodes[1]->as()) - && filter_column_node->getColumnName() == column_node->getColumnName()) - { - /// Rewrite arrayExists(x -> elem = x, arr) -> has(arr, elem) - function_arguments_nodes[0] = std::move(function_arguments_nodes[1]); - function_arguments_nodes[1] = std::move(filter_arguments_nodes[0]); - function_node->resolveAsFunction( - FunctionFactory::instance().get("has", getContext())->build(function_node->getArgumentColumns())); - } + /// Rewrite arrayExists(x -> x = elem, arr) -> has(arr, elem) + has_constant_element_argument = filter_rhs_argument_node; } - }; + else if (filter_lhs_argument_node_type == QueryTreeNodeType::CONSTANT && + filter_rhs_argument_node_type == QueryTreeNodeType::COLUMN && + filter_rhs_argument_node->isEqual(*lambda_argument_column_node)) + { + /// Rewrite arrayExists(x -> elem = x, arr) -> has(arr, elem) + has_constant_element_argument = filter_lhs_argument_node; + } + else + { + return; + } + + auto has_function = FunctionFactory::instance().get("has", getContext()); + array_exists_function_arguments_nodes[0] = std::move(array_exists_function_arguments_nodes[1]); + array_exists_function_arguments_nodes[1] = std::move(has_constant_element_argument); + array_exists_function_node->resolveAsFunction(has_function->build(array_exists_function_node->getArgumentColumns())); + } +}; } diff --git a/src/Analyzer/Passes/ArrayExistsToHasPass.h b/src/Analyzer/Passes/ArrayExistsToHasPass.h index 7d9d1cf3d68..8f4623116e3 100644 --- a/src/Analyzer/Passes/ArrayExistsToHasPass.h +++ b/src/Analyzer/Passes/ArrayExistsToHasPass.h @@ -4,8 +4,15 @@ namespace DB { -/// Rewrite possible 'arrayExists(func, arr)' to 'has(arr, elem)' to improve performance -/// arrayExists(x -> x = 1, arr) -> has(arr, 1) + +/** Rewrite possible 'arrayExists(func, arr)' to 'has(arr, elem)' to improve performance. + * + * Example: SELECT arrayExists(x -> x = 1, arr); + * Result: SELECT has(arr, 1); + * + * Example: SELECT arrayExists(x -> 1 = x, arr); + * Result: SELECT has(arr, 1); + */ class RewriteArrayExistsToHasPass final : public IQueryTreePass { public: @@ -15,4 +22,5 @@ public: void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; }; + } diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index fdf818681d7..fa5fc0e75a8 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -22,8 +22,7 @@ public: void visitImpl(QueryTreeNodePtr & node) { - const auto & context = getContext(); - if (!context->getSettingsRef().final) + if (!getSettings().final) return; const auto * query_node = node->as(); diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h index 83b974954fa..71a038bcf39 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.h @@ -6,6 +6,9 @@ namespace DB { /** Rewrite _shard_num column into shardNum() function. + * + * Example: SELECT _shard_num FROM distributed_table; + * Result: SELECT shardNum() FROM distributed_table; */ class ShardNumColumnToFunctionPass final : public IQueryTreePass { From 20c0476d2fb9cc4964b8075fdc8507d6f3c467b3 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 8 Mar 2023 10:51:16 +0100 Subject: [PATCH 379/470] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: pufit --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 92a8270ac0f..9d8f3ba8355 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -989,7 +989,7 @@ Default value: `100`. ## max_io_thread_pool_free_size {#max-io-thread-pool-free-size} -If the number of **idle** threads in the IO Thread pool is greater than `max_io_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. +If the number of **idle** threads in the IO Thread pool exceeds `max_io_thread_pool_free_size`, ClickHouse will release resources occupied by idling threads and decrease the pool size. Threads can be created again if necessary. Possible values: From 880b34fded1b8a3cee8b690309b11c9903af091a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 8 Mar 2023 10:51:34 +0100 Subject: [PATCH 380/470] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: pufit --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 9d8f3ba8355..3fe815bc79a 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1020,7 +1020,7 @@ Default value: `1000`. ## max_backups_io_thread_pool_free_size {#max-backups-io-thread-pool-free-size} -If the number of **idle** threads in the Backups IO Thread pool is greater than `max_backup_io_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. +If the number of **idle** threads in the Backups IO Thread pool exceeds `max_backup_io_thread_pool_free_size`, ClickHouse will release resources occupied by idling threads and decrease the pool size. Threads can be created again if necessary. Possible values: From b6f3b81403dd8051be2fc511f4ec87023ef1aca2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 6 Mar 2023 12:50:09 +0100 Subject: [PATCH 381/470] Query plan filter push down optimization filled JOIN fix --- src/Processors/QueryPlan/JoinStep.h | 2 ++ .../QueryPlan/Optimizations/filterPushDown.cpp | 13 ++++++++----- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index a814d541574..e7185f36588 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -49,6 +49,8 @@ public: String getName() const override { return "FilledJoin"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + const JoinPtr & getJoin() const { return join; } + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index d466c52725f..37bc894339f 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -314,11 +314,14 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - if (auto * join = typeid_cast(child.get())) + auto * join = typeid_cast(child.get()); + auto * filled_join = typeid_cast(child.get()); + + if (join || filled_join) { auto join_push_down = [&](JoinKind kind) -> size_t { - const auto & table_join = join->getJoin()->getTableJoin(); + const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. @@ -326,8 +329,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 0; bool is_left = kind == JoinKind::Left; - const auto & input_header = is_left ? join->getInputStreams().front().header : join->getInputStreams().back().header; - const auto & res_header = join->getOutputStream().header; + const auto & input_header = is_left ? child->getInputStreams().front().header : child->getInputStreams().back().header; + const auto & res_header = child->getOutputStream().header; Names allowed_keys; const auto & source_columns = input_header.getNames(); for (const auto & name : source_columns) @@ -372,7 +375,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return updated_steps; /// For full sorting merge join we push down both to the left and right tables, because left and right streams are not independent. - if (join->allowPushDownToRight()) + if (join && join->allowPushDownToRight()) { if (size_t updated_steps = join_push_down(JoinKind::Right)) return updated_steps; From 3aed5a4ab455ce1c6600a5a011381432e5543ba4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 7 Mar 2023 12:35:17 +0100 Subject: [PATCH 382/470] Added tests --- ...dicate_push_down_filled_join_fix.reference | 33 +++++++++++++++++++ ...75_predicate_push_down_filled_join_fix.sql | 26 +++++++++++++++ 2 files changed, 59 insertions(+) create mode 100644 tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference create mode 100644 tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference new file mode 100644 index 00000000000..ecdb62c5cb5 --- /dev/null +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -0,0 +1,33 @@ +Expression ((Project names + (Projection + ))) +Header: t1.id UInt64 + t1.value String + t2.value String +Actions: INPUT : 0 -> t1.id_0 UInt64 : 0 + INPUT : 1 -> t1.value_1 String : 1 + INPUT : 2 -> t2.value_2 String : 2 + ALIAS t1.id_0 :: 0 -> t1.id UInt64 : 3 + ALIAS t1.value_1 :: 1 -> t1.value String : 0 + ALIAS t2.value_2 :: 2 -> t2.value String : 1 +Positions: 3 0 1 + FilledJoin (Filled JOIN) + Header: t1.id_0 UInt64 + t1.value_1 String + t2.value_2 String + Filter (( + (JOIN actions + Change column names to column identifiers))) + Header: t1.id_0 UInt64 + t1.value_1 String + Filter column: equals(t1.id_0, 0_UInt8) (removed) + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value String : 1 + COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 + ALIAS id :: 0 -> t1.id_0 UInt64 : 3 + ALIAS value :: 1 -> t1.value_1 String : 0 + FUNCTION equals(t1.id_0 : 3, 0_UInt8 :: 2) -> equals(t1.id_0, 0_UInt8) UInt8 : 1 + Positions: 1 3 0 + ReadFromMergeTree (default.test_table) + Header: id UInt64 + value String + ReadType: Default + Parts: 1 + Granules: 1 +0 Value JoinValue diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql new file mode 100644 index 00000000000..78cb423216b --- /dev/null +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql @@ -0,0 +1,26 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (0, 'Value'); + +DROP TABLE IF EXISTS test_table_join; +CREATE TABLE test_table_join +( + id UInt64, + value String +) ENGINE = Join(All, inner, id); + +INSERT INTO test_table_join VALUES (0, 'JoinValue'); + +EXPLAIN header = 1, actions = 1 SELECT t1.id, t1.value, t2.value FROM test_table AS t1 INNER JOIN test_table_join AS t2 ON t1.id = t2.id WHERE t1.id = 0; + +SELECT t1.id, t1.value, t2.value FROM test_table AS t1 INNER JOIN test_table_join AS t2 ON t1.id = t2.id WHERE t1.id = 0; + +DROP TABLE test_table_join; +DROP TABLE test_table; From ff6c9916e6e511c333b33d0ece1607897a2aa75e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 8 Mar 2023 12:13:28 +0100 Subject: [PATCH 383/470] Improvements based on the PR --- src/Client/ClientBase.cpp | 16 ++++++++-------- src/Client/TestHint.cpp | 8 ++++---- src/Client/TestHint.h | 23 +++++++++++++---------- 3 files changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b5c662b4a80..4476678f0f4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1834,7 +1834,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) { /// disable logs if expects errors TestHint test_hint(all_queries_text); - if (!test_hint.clientErrors().empty() || !test_hint.serverErrors().empty()) + if (test_hint.hasClientErrors() || test_hint.hasServerErrors()) processTextAsSingleQuery("SET send_logs_level = 'fatal'"); } @@ -1876,7 +1876,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) // the query ends because we failed to parse it, so we consume // the entire line. TestHint hint(String(this_query_begin, this_query_end - this_query_begin)); - if (!hint.serverErrors().empty()) + if (hint.hasServerErrors()) { // Syntax errors are considered as client errors current_exception->addMessage("\nExpected server error: {}.", hint.serverErrors()); @@ -1886,7 +1886,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) if (std::find(hint.clientErrors().begin(), hint.clientErrors().end(), current_exception->code()) == hint.clientErrors().end()) { - if (!hint.clientErrors().empty()) + if (hint.hasClientErrors()) current_exception->addMessage("\nExpected client error: {}.", hint.clientErrors()); current_exception->rethrow(); @@ -1936,7 +1936,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) bool error_matches_hint = true; if (have_error) { - if (!test_hint.serverErrors().empty()) + if (test_hint.hasServerErrors()) { if (!server_exception) { @@ -1953,7 +1953,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) test_hint.serverErrors(), server_exception->code(), full_query); } } - if (!test_hint.clientErrors().empty()) + if (test_hint.hasClientErrors()) { if (!client_exception) { @@ -1970,7 +1970,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) test_hint.clientErrors(), client_exception->code(), full_query); } } - if (test_hint.clientErrors().empty() && test_hint.serverErrors().empty()) + if (!test_hint.hasClientErrors() && !test_hint.hasServerErrors()) { // No error was expected but it still occurred. This is the // default case without test hint, doesn't need additional @@ -1980,14 +1980,14 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) } else { - if (!test_hint.clientErrors().empty()) + if (test_hint.hasClientErrors()) { error_matches_hint = false; fmt::print(stderr, "The query succeeded but the client error '{}' was expected (query: {}).\n", test_hint.clientErrors(), full_query); } - if (!test_hint.serverErrors().empty()) + if (test_hint.hasServerErrors()) { error_matches_hint = false; fmt::print(stderr, diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index 1500a8baf26..8cc91250721 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -9,7 +9,7 @@ namespace DB::ErrorCodes { -extern const int CANNOT_PARSE_TEXT; + extern const int CANNOT_PARSE_TEXT; } namespace DB @@ -54,9 +54,9 @@ TestHint::TestHint(const String & query_) void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) { - std::unordered_set commands{"echo", "echoOn", "echoOff"}; + std::unordered_set commands{"echo", "echoOn", "echoOff"}; - std::unordered_set command_errors{ + std::unordered_set command_errors{ "serverError", "clientError", }; @@ -76,7 +76,7 @@ void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) else if (!is_leading_hint && token.type == TokenType::BareWord && command_errors.contains(item)) { /// Everything after this must be a list of errors separated by comma - error_vector error_codes; + ErrorVector error_codes; while (!token.isEnd()) { token = comment_lexer.nextToken(); diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index d9650b9d4d7..fb1dbfb72dc 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -18,7 +18,7 @@ class Lexer; /// The following comment hints are supported: /// /// - "-- { serverError 60 }" -- in case of you are expecting server error. -/// - "-- { serverError 16, 36 }" -- in case of you are expecting one of the 2 errors +/// - "-- { serverError 16, 36 }" -- in case of you are expecting one of the 2 errors. /// /// - "-- { clientError 20 }" -- in case of you are expecting client error. /// - "-- { clientError 20, 60, 92 }" -- It's expected that the client will return one of the 3 errors. @@ -52,17 +52,20 @@ class Lexer; class TestHint { public: - using error_vector = std::vector; + using ErrorVector = std::vector; TestHint(const String & query_); const auto & serverErrors() const { return server_errors; } const auto & clientErrors() const { return client_errors; } std::optional echoQueries() const { return echo; } + bool hasClientErrors() { return !client_errors.empty(); } + bool hasServerErrors() { return !server_errors.empty(); } + private: const String & query; - error_vector server_errors{}; - error_vector client_errors{}; + ErrorVector server_errors{}; + ErrorVector client_errors{}; std::optional echo; void parse(Lexer & comment_lexer, bool is_leading_hint); @@ -91,7 +94,7 @@ private: } template <> -struct fmt::formatter +struct fmt::formatter { static constexpr auto parse(format_parse_context & ctx) { @@ -106,13 +109,13 @@ struct fmt::formatter } template - auto format(const DB::TestHint::error_vector & error_vector, FormatContext & ctx) + auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) { - if (error_vector.empty()) + if (ErrorVector.empty()) return format_to(ctx.out(), "{}", 0); - else if (error_vector.size() == 1) - return format_to(ctx.out(), "{}", error_vector[0]); + else if (ErrorVector.size() == 1) + return format_to(ctx.out(), "{}", ErrorVector[0]); else - return format_to(ctx.out(), "One of [{}]", fmt::join(error_vector, ", ")); + return format_to(ctx.out(), "[{}]", fmt::join(ErrorVector, ", ")); } }; From 969bb0e1afc8f60699b46fcbef29e69f29a211f5 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 8 Mar 2023 12:31:55 +0000 Subject: [PATCH 384/470] Remove test 02153_native_bounds_check.sh --- .../0_stateless/02153_native_bounds_check.reference | 1 - .../queries/0_stateless/02153_native_bounds_check.sh | 11 ----------- 2 files changed, 12 deletions(-) delete mode 100644 tests/queries/0_stateless/02153_native_bounds_check.reference delete mode 100755 tests/queries/0_stateless/02153_native_bounds_check.sh diff --git a/tests/queries/0_stateless/02153_native_bounds_check.reference b/tests/queries/0_stateless/02153_native_bounds_check.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02153_native_bounds_check.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02153_native_bounds_check.sh b/tests/queries/0_stateless/02153_native_bounds_check.sh deleted file mode 100755 index a3475ddacae..00000000000 --- a/tests/queries/0_stateless/02153_native_bounds_check.sh +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Should correctly handle error. - -${CLICKHOUSE_LOCAL} --query "SELECT toString(number) AS a, toString(number) AS a FROM numbers(10)" --output-format Native | - ${CLICKHOUSE_LOCAL} --query "SELECT * FROM table" --input-format Native --structure 'a LowCardinality(String)' 2>&1 | - grep -c -F Exception From 79ef95bc0c2c2e983cdbb90afc75177aa8597430 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 8 Mar 2023 12:54:22 +0000 Subject: [PATCH 385/470] Fix tests and builds --- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../01308_orc_output_format_arrays.reference | Bin 567 -> 36 bytes .../01308_orc_output_format_arrays.sh | 2 +- .../queries/0_stateless/02426_orc_bug.reference | Bin 312 -> 36 bytes tests/queries/0_stateless/02426_orc_bug.sh | 9 +++++++++ tests/queries/0_stateless/02426_orc_bug.sql | 3 --- 6 files changed, 11 insertions(+), 5 deletions(-) create mode 100755 tests/queries/0_stateless/02426_orc_bug.sh delete mode 100644 tests/queries/0_stateless/02426_orc_bug.sql diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index cedd8a9c54c..759f773a574 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes namespace { -static parquet::ParquetVersion::type getParquetVersion(const FormatSettings & settings) +parquet::ParquetVersion::type getParquetVersion(const FormatSettings & settings) { switch (settings.parquet.output_version) { diff --git a/tests/queries/0_stateless/01308_orc_output_format_arrays.reference b/tests/queries/0_stateless/01308_orc_output_format_arrays.reference index 1f9646ac112132378f512bb4e3a610f6019698e1..7feea7cec35600166ea51a6950459c05eb6c7d88 100644 GIT binary patch literal 36 rcmXpqH%qZFO-ZpdOEI=INij9EFfubvv`jTIH%c@%Pfk%#(B%REtfL6? literal 567 zcmaKou}Z{15Qb-Fv+J_CUfj1(^yJ73Txpxm?(hk$eF1A9K)8pp z5ER#OZ*wFHHuLYd|I9G_i{;#AZq?1nOH zi@h-PB)k2_Z@x78PeSsIycPNP4-!J}r2;h0Sd#;1L6P@^5+4Osk&~#JS^h!#DJTY| z99adXz?c*3z$*GG`bp%3P4IQ>*K8yV+53Mj)#Y{L&fPY*Z5^i6IX{^Uqp&J}r7{({ zSCX4ST<`_rgrW2m5pNorkz6v0~6|4#a#fA`ch4R^#kxNzslh121M bv&Wk&w9hE%uGh(gAgwNH^YEtBVmaq;55+tO diff --git a/tests/queries/0_stateless/01308_orc_output_format_arrays.sh b/tests/queries/0_stateless/01308_orc_output_format_arrays.sh index 1d9aea353b6..498854874cf 100755 --- a/tests/queries/0_stateless/01308_orc_output_format_arrays.sh +++ b/tests/queries/0_stateless/01308_orc_output_format_arrays.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE orc (array1 Array(Int32), array2 Array( $CLICKHOUSE_CLIENT --query="INSERT INTO orc VALUES ([1,2,3,4,5], [[1,2], [3,4], [5]]), ([42], [[42, 42], [42]])"; -$CLICKHOUSE_CLIENT --query="SELECT * FROM orc FORMAT ORC"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM orc FORMAT ORC SETTINGS output_format_orc_compression_method='none'" | md5sum; $CLICKHOUSE_CLIENT --query="DROP TABLE orc"; diff --git a/tests/queries/0_stateless/02426_orc_bug.reference b/tests/queries/0_stateless/02426_orc_bug.reference index e5ad2b492892318e8fa83954efdecbb2bf287a91..baa88da215881130acc288c7ce2a15bd34f45e90 100644 GIT binary patch literal 36 rcmcCawlFudut-WXGfYc0H%zs(G)*xwOi4*gGB-9%F*jFG(B%REwulK^ literal 312 zcmeYda^~XZVrF1q5MtqA3Si*ke3ufJp+V>i>cN2mUcI{%`ot z$i>FNCcq%U3#J(*_`x)jggB690E#n#X+|*3B)}-a4CVt30n&^h!+_QT)ghBo% Date: Wed, 8 Mar 2023 13:06:10 +0000 Subject: [PATCH 386/470] Add logs --- src/Coordination/Changelog.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ddedae4fa0f..13da4dffac8 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -211,13 +211,20 @@ public: void flush() { auto * file_buffer = tryGetFileBuffer(); + LOG_TRACE(log, "Trying to flush"); if (file_buffer) { /// Fsync file system if needed if (log_file_settings.force_sync) + { file_buffer->sync(); + } else + { + LOG_TRACE(log, "Flushing with next, in buffer {}", file_buffer->offset()); file_buffer->next(); + LOG_TRACE(log, "Flushed in total {}", file_buffer->count()); + } } } From 07a39887499c7eadb4198a724d024c98607065d3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 8 Mar 2023 08:26:07 -0500 Subject: [PATCH 387/470] geteuid is enough --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 1374c269dc1..d41b9b1ebe1 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -170,7 +170,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n bool isSudo() { - return geteuid() == 0 && getenv("SUDO_USER") && getenv("SUDO_UID") && getenv("SUDO_GID"); // NOLINT(concurrency-mt-unsafe) + return geteuid() == 0; } /// Read data about files and decomrpess them. From df698355dff9d3b364cbbff50c6e84b5192236cc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 8 Mar 2023 14:25:24 +0000 Subject: [PATCH 388/470] comment --- src/Analyzer/QueryTreeBuilder.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index ed956071d5a..34a75dd0c33 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -356,6 +356,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by, current_context); /// Combine limit expression with limit and offset settings into final limit expression + /// The sequence of application is next - offset expression, limit expression, offset setting, limit setting. + /// Since offset setting is applied after limit expression, but we want to transfer settings into expression + /// we must decrease limit expression by offset setting and then add offset setting to offset expression. /// select_limit - limit expression /// limit - limit setting /// offset - offset setting From 0fb537e8805aa54f1d951bd1b5a58d28b92c69c5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Mar 2023 15:27:43 +0100 Subject: [PATCH 389/470] suppress expected errors from test 01111 in Upgrade check --- docker/test/upgrade/run.sh | 4 +++- src/Interpreters/DDLWorker.cpp | 12 ++++++------ .../01111_create_drop_replicated_db_stress.sh | 2 +- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index ce8a56c777e..93f07bbb551 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -161,7 +161,9 @@ rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ -e "Authentication failed" \ -e "Cannot flush" \ -e "Container already exists" \ - /var/log/clickhouse-server/clickhouse-server.upgrade.log | zgrep -Fa "" > /test_output/upgrade_error_messages.txt \ + clickhouse-server.upgrade.log \ + | grep -av -e "_repl_01111_.*Mapping for table with UUID" \ + | zgrep -Fa "" > /test_output/upgrade_error_messages.txt \ && echo -e "Error message in clickhouse-server.log (see upgrade_error_messages.txt)$FAIL$(head_escaped /test_output/upgrade_error_messages.txt)" \ >> /test_output/test_results.tsv \ || echo -e "No Error messages after server upgrade$OK" >> /test_output/test_results.tsv diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 7179ce94e0b..537380862fa 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -169,11 +169,11 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r return {}; } - auto write_error_status = [&](const String & host_id, const String & error_message, const String & reason) + auto write_error_status = [&](const String & host_id, const ExecutionStatus & status, const String & reason) { - LOG_ERROR(log, "Cannot parse DDL task {}: {}. Will try to send error status: {}", entry_name, reason, error_message); + LOG_ERROR(log, "Cannot parse DDL task {}: {}. Will try to send error status: {}", entry_name, reason, status.message); createStatusDirs(entry_path, zookeeper); - zookeeper->tryCreate(fs::path(entry_path) / "finished" / host_id, error_message, zkutil::CreateMode::Persistent); + zookeeper->tryCreate(fs::path(entry_path) / "finished" / host_id, status.serializeText(), zkutil::CreateMode::Persistent); }; try @@ -187,7 +187,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r /// We can try to create fail node using FQDN if it equal to host name in cluster config attempt will be successful. /// Otherwise, that node will be ignored by DDLQueryStatusSource. out_reason = "Incorrect task format"; - write_error_status(host_fqdn_id, ExecutionStatus::fromCurrentException().serializeText(), out_reason); + write_error_status(host_fqdn_id, ExecutionStatus::fromCurrentException(), out_reason); return {}; } @@ -212,7 +212,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r catch (...) { out_reason = "Cannot parse query or obtain cluster info"; - write_error_status(task->host_id_str, ExecutionStatus::fromCurrentException().serializeText(), out_reason); + write_error_status(task->host_id_str, ExecutionStatus::fromCurrentException(), out_reason); return {}; } @@ -650,7 +650,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) bool status_written_by_table_or_db = task.ops.empty(); if (status_written_by_table_or_db) { - throw Exception(ErrorCodes::UNFINISHED, "Unexpected error: {}", task.execution_status.serializeText()); + throw Exception(ErrorCodes::UNFINISHED, "Unexpected error: {}", task.execution_status.message); } else { diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 8c9efb75e96..770a0780ca2 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -15,7 +15,7 @@ function create_db() # Multiple database replicas on one server are actually not supported (until we have namespaces). # So CREATE TABLE queries will fail on all replicas except one. But it's still makes sense for a stress test. $CLICKHOUSE_CLIENT --allow_experimental_database_replicated=1 --query \ - "create database if not exists ${CLICKHOUSE_DATABASE}_repl_$SUFFIX engine=Replicated('/test/01111/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '$SHARD', '$REPLICA')" \ + "create database if not exists ${CLICKHOUSE_DATABASE}_repl_01111_$SUFFIX engine=Replicated('/test/01111/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '$SHARD', '$REPLICA')" \ 2>&1| grep -Fa "Exception: " | grep -Fv "REPLICA_ALREADY_EXISTS" | grep -Fiv "Will not try to start it up" | \ grep -Fv "Coordination::Exception" | grep -Fv "already contains some data and it does not look like Replicated database path" sleep 0.$RANDOM From debd69f03af5939b266c713c9ff9deb1884d3691 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 7 Mar 2023 20:17:09 +0100 Subject: [PATCH 390/470] fix an assertion with implicit transactions in interserver mode --- src/Interpreters/executeQuery.cpp | 60 ++++++++----------- .../02345_implicit_transaction.reference | 3 + .../02345_implicit_transaction.sql | 12 +++- 3 files changed, 38 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e2aa2c02fc8..fb90ba2da7b 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -451,10 +451,24 @@ static std::tuple executeQueryImpl( /// Avoid early destruction of process_list_entry if it was not saved to `res` yet (in case of exception) ProcessList::EntryPtr process_list_entry; BlockIO res; - std::shared_ptr implicit_txn_control{}; + auto implicit_txn_control = std::make_shared(false); String query_database; String query_table; + auto execute_implicit_tcl_query = [implicit_txn_control](const ContextMutablePtr & query_context, ASTTransactionControl::QueryType tcl_type) + { + /// Unset the flag on COMMIT and ROLLBACK + SCOPE_EXIT({ if (tcl_type != ASTTransactionControl::BEGIN) *implicit_txn_control = false; }); + + ASTPtr tcl_ast = std::make_shared(tcl_type); + InterpreterTransactionControlQuery tc(tcl_ast, query_context); + tc.execute(); + + /// Set the flag after successful BIGIN + if (tcl_type == ASTTransactionControl::BEGIN) + *implicit_txn_control = true; + }; + try { if (auto txn = context->getCurrentTransaction()) @@ -674,14 +688,7 @@ static std::tuple executeQueryImpl( if (context->isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions"); - /// If there is no session (which is the default for the HTTP Handler), set up one just for this as it is necessary - /// to control the transaction lifetime - if (!context->hasSessionContext()) - context->makeSessionContext(); - - auto tc = std::make_shared(ast, context); - tc->executeBegin(context->getSessionContext()); - implicit_txn_control = std::move(tc); + execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN); } catch (Exception & e) { @@ -949,6 +956,7 @@ static std::tuple executeQueryImpl( log_processors_profiles = settings.log_processors_profiles, status_info_to_query_log, implicit_txn_control, + execute_implicit_tcl_query, pulling_pipeline = pipeline.pulling(), query_span](QueryPipeline & query_pipeline) mutable { @@ -1062,21 +1070,8 @@ static std::tuple executeQueryImpl( } } - if (implicit_txn_control) - { - try - { - implicit_txn_control->executeCommit(context->getSessionContext()); - implicit_txn_control.reset(); - } - catch (const Exception &) - { - /// An exception might happen when trying to commit the transaction. For example we might get an immediate exception - /// because ZK is down and wait_changes_become_visible_after_commit_mode == WAIT_UNKNOWN - implicit_txn_control.reset(); - throw; - } - } + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::COMMIT); } if (query_span) @@ -1104,13 +1099,11 @@ static std::tuple executeQueryImpl( quota(quota), status_info_to_query_log, implicit_txn_control, + execute_implicit_tcl_query, query_span](bool log_error) mutable { - if (implicit_txn_control) - { - implicit_txn_control->executeRollback(context->getSessionContext()); - implicit_txn_control.reset(); - } + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::ROLLBACK); else if (auto txn = context->getCurrentTransaction()) txn->onException(); @@ -1179,15 +1172,10 @@ static std::tuple executeQueryImpl( } catch (...) { - if (implicit_txn_control) - { - implicit_txn_control->executeRollback(context->getSessionContext()); - implicit_txn_control.reset(); - } + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::ROLLBACK); else if (auto txn = context->getCurrentTransaction()) - { txn->onException(); - } if (!internal) onExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds()); diff --git a/tests/queries/0_stateless/02345_implicit_transaction.reference b/tests/queries/0_stateless/02345_implicit_transaction.reference index e4dd35600f7..fb4254ec5a7 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.reference +++ b/tests/queries/0_stateless/02345_implicit_transaction.reference @@ -12,3 +12,6 @@ in_transaction 10000 out_transaction 0 {"'implicit_True'":"implicit_True","all":"2","is_empty":0} {"'implicit_False'":"implicit_False","all":"2","is_empty":1} +0 +0 +0 diff --git a/tests/queries/0_stateless/02345_implicit_transaction.sql b/tests/queries/0_stateless/02345_implicit_transaction.sql index e3f9cca37d1..b0cb4ab6305 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.sql +++ b/tests/queries/0_stateless/02345_implicit_transaction.sql @@ -1,4 +1,4 @@ --- Tags: no-ordinary-database +-- Tags: no-ordinary-database, no-fasttest CREATE TABLE landing (n Int64) engine=MergeTree order by n; CREATE TABLE target (n Int64) engine=MergeTree order by n; @@ -92,3 +92,13 @@ WHERE query LIKE '-- Verify that the transaction_id column is NOT populated without transaction%' GROUP BY transaction_id FORMAT JSONEachRow; + +SET implicit_transaction=1; +SET throw_on_unsupported_query_inside_transaction=1; +SELECT * FROM system.one; +SELECT * FROM cluster('test_cluster_interserver_secret', system, one); -- { serverError NOT_IMPLEMENTED } +SELECT * FROM cluster('test_cluster_two_shards', system, one); -- { serverError NOT_IMPLEMENTED } +SET throw_on_unsupported_query_inside_transaction=0; +-- there's not session in the interserver mode +SELECT * FROM cluster('test_cluster_interserver_secret', system, one) FORMAT Null; -- { serverError INVALID_TRANSACTION } +SELECT * FROM cluster('test_cluster_two_shards', system, one); From 95beedc8a4bb3fc8bd33318b825d3718bdff945c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 8 Mar 2023 15:56:13 +0000 Subject: [PATCH 391/470] Fix 02343_group_by_use_nulls test --- tests/queries/0_stateless/02343_group_by_use_nulls.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02343_group_by_use_nulls.sql b/tests/queries/0_stateless/02343_group_by_use_nulls.sql index a979a78be0d..e1d4021a943 100644 --- a/tests/queries/0_stateless/02343_group_by_use_nulls.sql +++ b/tests/queries/0_stateless/02343_group_by_use_nulls.sql @@ -1,3 +1,4 @@ +set optimize_group_by_function_keys=0; -- { echoOn } SELECT number, number % 2, sum(number) AS val FROM numbers(10) From 52767ea186da08f8a48f5ee2941c1a86d20b0521 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Mar 2023 19:25:08 +0300 Subject: [PATCH 392/470] Update 02368_cancel_write_into_hdfs.sh --- tests/queries/0_stateless/02368_cancel_write_into_hdfs.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02368_cancel_write_into_hdfs.sh b/tests/queries/0_stateless/02368_cancel_write_into_hdfs.sh index 8262cd7eab5..65d0b3f434f 100755 --- a/tests/queries/0_stateless/02368_cancel_write_into_hdfs.sh +++ b/tests/queries/0_stateless/02368_cancel_write_into_hdfs.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-stress +# Tags: no-fasttest, no-asan, no-tsan, no-msan, no-ubsan, no-debug +# FIXME https://github.com/ClickHouse/ClickHouse/issues/47207 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c3b9af96c460077d364c7d8e95e73340456d6281 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 8 Mar 2023 17:19:04 +0000 Subject: [PATCH 393/470] fix ALTER CLEAR COLUMN with sparse columns --- src/Storages/MergeTree/MergeTask.cpp | 40 ++++++++++++++++++- src/Storages/MergeTree/MutateTask.cpp | 6 +++ ...2675_sparse_columns_clear_column.reference | 6 +++ .../02675_sparse_columns_clear_column.sql | 34 ++++++++++++++++ 4 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02675_sparse_columns_clear_column.reference create mode 100644 tests/queries/0_stateless/02675_sparse_columns_clear_column.sql diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index b961b70428e..9d9d8420e2c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -96,6 +96,32 @@ static void extractMergingAndGatheringColumns( } } +static void addMissedColumnsToSerializationInfos( + size_t num_rows_in_parts, + const Names & part_columns, + const ColumnsDescription & storage_columns, + const SerializationInfo::Settings & info_settings, + SerializationInfoByName & new_infos) +{ + NameSet part_columns_set(part_columns.begin(), part_columns.end()); + + for (const auto & column : storage_columns) + { + if (part_columns_set.contains(column.name)) + continue; + + if (column.default_desc.kind != ColumnDefaultKind::Default) + continue; + + if (column.default_desc.expression) + continue; + + auto new_info = column.type->createSerializationInfo(info_settings); + new_info->addDefaults(num_rows_in_parts); + new_infos.emplace(column.name, std::move(new_info)); + } +} + bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { @@ -205,7 +231,19 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->force_ttl = true; } - infos.add(part->getSerializationInfos()); + if (!info_settings.isAlwaysDefault()) + { + auto part_infos = part->getSerializationInfos(); + + addMissedColumnsToSerializationInfos( + part->rows_count, + part->getColumns().getNames(), + global_ctx->metadata_snapshot->getColumns(), + info_settings, + part_infos); + + infos.add(part_infos); + } } global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index bcb1d5d2c28..526f869a3ac 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -626,6 +626,12 @@ static NameToNameVector collectFilesForRenames( } } + if (!source_part->getSerializationInfos().empty() + && new_part->getSerializationInfos().empty()) + { + rename_vector.emplace_back(IMergeTreeDataPart::SERIALIZATION_FILE_NAME, ""); + } + return rename_vector; } diff --git a/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference b/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference new file mode 100644 index 00000000000..56fa4a9ebea --- /dev/null +++ b/tests/queries/0_stateless/02675_sparse_columns_clear_column.reference @@ -0,0 +1,6 @@ +arr Default +v Sparse +arr Default +arr Default +v Sparse +0 [] diff --git a/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql b/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql new file mode 100644 index 00000000000..781030ef7b4 --- /dev/null +++ b/tests/queries/0_stateless/02675_sparse_columns_clear_column.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS t_sparse_columns_clear; + +CREATE TABLE t_sparse_columns_clear (arr Array(UInt64), v UInt64) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS + ratio_of_defaults_for_sparse_serialization = 0.9, + min_bytes_for_wide_part=0; + +INSERT INTO t_sparse_columns_clear SELECT [number], 0 FROM numbers(1000); + +SELECT column, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_columns_clear' AND active +ORDER BY column; + +SET mutations_sync = 2; +SET alter_sync = 2; + +ALTER TABLE t_sparse_columns_clear CLEAR COLUMN v; + +SELECT column, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_columns_clear' AND active +ORDER BY column; + +OPTIMIZE TABLE t_sparse_columns_clear FINAL; + +SELECT column, serialization_kind FROM system.parts_columns +WHERE database = currentDatabase() AND table = 't_sparse_columns_clear' AND active +ORDER BY column; + +DROP TABLE t_sparse_columns_clear SYNC; + +SYSTEM FLUSH LOGS; + +SELECT count(), groupArray(message) FROM system.text_log WHERE logger_name LIKE '%' || currentDatabase() || '.t_sparse_columns_clear' || '%' AND level = 'Error'; From 86afb8a72d7143c6662cdcdf2beed79c2c2315f6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 9 Mar 2023 02:24:37 +0000 Subject: [PATCH 394/470] shortcut for trivial case, fix test, fix comment --- src/Analyzer/QueryTreeBuilder.cpp | 52 +++++++++++-------- .../02565_analyzer_limit_settings.reference | 1 - 2 files changed, 30 insertions(+), 23 deletions(-) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 34a75dd0c33..7dd988619ac 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -356,7 +356,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by, current_context); /// Combine limit expression with limit and offset settings into final limit expression - /// The sequence of application is next - offset expression, limit expression, offset setting, limit setting. + /// The sequence of application is the following - offset expression, limit expression, offset setting, limit setting. /// Since offset setting is applied after limit expression, but we want to transfer settings into expression /// we must decrease limit expression by offset setting and then add offset setting to offset expression. /// select_limit - limit expression @@ -377,32 +377,40 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q auto select_limit = select_query_typed.limitLength(); if (select_limit) { - /// expr 3 - auto expr_3 = std::make_shared("minus"); - expr_3->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); - expr_3->getArguments().getNodes().push_back(std::make_shared(offset)); + /// Shortcut + if (offset == 0 && limit == 0) + { + current_query_tree->getLimit() = buildExpression(select_limit, current_context); + } + else + { + /// expr 3 + auto expr_3 = std::make_shared("minus"); + expr_3->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); + expr_3->getArguments().getNodes().push_back(std::make_shared(offset)); - /// expr 2 - auto expr_2 = std::make_shared("least"); - expr_2->getArguments().getNodes().push_back(expr_3->clone()); - expr_2->getArguments().getNodes().push_back(std::make_shared(limit)); + /// expr 2 + auto expr_2 = std::make_shared("least"); + expr_2->getArguments().getNodes().push_back(expr_3->clone()); + expr_2->getArguments().getNodes().push_back(std::make_shared(limit)); - /// expr 0 - auto expr_0 = std::make_shared("greaterOrEquals"); - expr_0->getArguments().getNodes().push_back(std::make_shared(offset)); - expr_0->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); + /// expr 0 + auto expr_0 = std::make_shared("greaterOrEquals"); + expr_0->getArguments().getNodes().push_back(std::make_shared(offset)); + expr_0->getArguments().getNodes().push_back(buildExpression(select_limit, current_context)); - /// expr 1 - auto expr_1 = std::make_shared(limit > 0); + /// expr 1 + auto expr_1 = std::make_shared(limit > 0); - auto function_node = std::make_shared("multiIf"); - function_node->getArguments().getNodes().push_back(expr_0); - function_node->getArguments().getNodes().push_back(std::make_shared(0)); - function_node->getArguments().getNodes().push_back(expr_1); - function_node->getArguments().getNodes().push_back(expr_2); - function_node->getArguments().getNodes().push_back(expr_3); + auto function_node = std::make_shared("multiIf"); + function_node->getArguments().getNodes().push_back(expr_0); + function_node->getArguments().getNodes().push_back(std::make_shared(0)); + function_node->getArguments().getNodes().push_back(expr_1); + function_node->getArguments().getNodes().push_back(expr_2); + function_node->getArguments().getNodes().push_back(expr_3); - current_query_tree->getLimit() = std::move(function_node); + current_query_tree->getLimit() = std::move(function_node); + } } else if (limit > 0) current_query_tree->getLimit() = std::make_shared(limit); diff --git a/tests/queries/0_stateless/02565_analyzer_limit_settings.reference b/tests/queries/0_stateless/02565_analyzer_limit_settings.reference index 6f23097612e..87e9f407cc8 100644 --- a/tests/queries/0_stateless/02565_analyzer_limit_settings.reference +++ b/tests/queries/0_stateless/02565_analyzer_limit_settings.reference @@ -62,7 +62,6 @@ SELECT * FROM numbers(10); SELECT * FROM numbers(10) LIMIT 3 OFFSET 2; 3 4 -5 SELECT * FROM numbers(10) LIMIT 5 OFFSET 2; 3 4 From 1728d6ed2c15f363fccf2091d819a042e1ded820 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Thu, 9 Mar 2023 11:44:27 +0800 Subject: [PATCH 395/470] Update settings.md - Change Prometheus port to 9363 as listed in https://clickhouse.com/docs/en/guides/sre/network-ports/ --- .../server-configuration-parameters/settings.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 3fe815bc79a..99daddeeb99 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1318,12 +1318,12 @@ Settings: ``` xml - /metrics - 8001 - true - true - true - + /metrics + 9363 + true + true + true + ``` ## query_log {#server_configuration_parameters-query-log} From 0ad436aa2c25d534b7b05d48b14a5537ad4e1f88 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 9 Mar 2023 08:07:59 +0000 Subject: [PATCH 396/470] Revert debug changes --- src/Coordination/Changelog.cpp | 7 ------- tests/config/config.d/keeper_port.xml | 1 - 2 files changed, 8 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 13da4dffac8..ddedae4fa0f 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -211,20 +211,13 @@ public: void flush() { auto * file_buffer = tryGetFileBuffer(); - LOG_TRACE(log, "Trying to flush"); if (file_buffer) { /// Fsync file system if needed if (log_file_settings.force_sync) - { file_buffer->sync(); - } else - { - LOG_TRACE(log, "Flushing with next, in buffer {}", file_buffer->offset()); file_buffer->next(); - LOG_TRACE(log, "Flushed in total {}", file_buffer->count()); - } } } diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 6c60c8936a9..cffd325e968 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -14,7 +14,6 @@ 1000000000000000 100000 - trace 0 From 7161bbeb8850e2ec15010c8dda376af7e4b4a34d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Mar 2023 11:16:29 +0100 Subject: [PATCH 397/470] More improvements based on PR review --- src/Client/ClientBase.cpp | 11 +++-------- src/Client/TestHint.cpp | 10 ++++++++++ src/Client/TestHint.h | 3 +++ 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 4476678f0f4..876b1d1906d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1883,8 +1883,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) current_exception->rethrow(); } - if (std::find(hint.clientErrors().begin(), hint.clientErrors().end(), current_exception->code()) - == hint.clientErrors().end()) + if (!hint.hasExpectedClientError(current_exception->code())) { if (hint.hasClientErrors()) current_exception->addMessage("\nExpected client error: {}.", hint.clientErrors()); @@ -1944,9 +1943,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) fmt::print(stderr, "Expected server error code '{}' but got no server error (query: {}).\n", test_hint.serverErrors(), full_query); } - else if ( - std::find(test_hint.serverErrors().begin(), test_hint.serverErrors().end(), server_exception->code()) - == test_hint.serverErrors().end()) + else if (!test_hint.hasExpectedServerError(server_exception->code())) { error_matches_hint = false; fmt::print(stderr, "Expected server error code: {} but got: {} (query: {}).\n", @@ -1961,9 +1958,7 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text) fmt::print(stderr, "Expected client error code '{}' but got no client error (query: {}).\n", test_hint.clientErrors(), full_query); } - else if ( - std::find(test_hint.clientErrors().begin(), test_hint.clientErrors().end(), client_exception->code()) - == test_hint.clientErrors().end()) + else if (!test_hint.hasExpectedClientError(client_exception->code())) { error_matches_hint = false; fmt::print(stderr, "Expected client error code '{}' but got '{}' (query: {}).\n", diff --git a/src/Client/TestHint.cpp b/src/Client/TestHint.cpp index 8cc91250721..b64882577ee 100644 --- a/src/Client/TestHint.cpp +++ b/src/Client/TestHint.cpp @@ -52,6 +52,16 @@ TestHint::TestHint(const String & query_) } } +bool TestHint::hasExpectedClientError(int error) +{ + return std::find(client_errors.begin(), client_errors.end(), error) != client_errors.end(); +} + +bool TestHint::hasExpectedServerError(int error) +{ + return std::find(server_errors.begin(), server_errors.end(), error) != server_errors.end(); +} + void TestHint::parse(Lexer & comment_lexer, bool is_leading_hint) { std::unordered_set commands{"echo", "echoOn", "echoOff"}; diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index fb1dbfb72dc..63f16b1dd97 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -62,6 +62,9 @@ public: bool hasClientErrors() { return !client_errors.empty(); } bool hasServerErrors() { return !server_errors.empty(); } + bool hasExpectedClientError(int error); + bool hasExpectedServerError(int error); + private: const String & query; ErrorVector server_errors{}; From 7bf1b8341440c90912db0d1c1f0f2e596bcf7de3 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 9 Mar 2023 12:37:57 +0100 Subject: [PATCH 398/470] add checks --- src/Functions/array/arrayReverse.cpp | 2 ++ src/Interpreters/OptimizeIfChains.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Functions/array/arrayReverse.cpp b/src/Functions/array/arrayReverse.cpp index 912adbadc7c..654a6c4cebf 100644 --- a/src/Functions/array/arrayReverse.cpp +++ b/src/Functions/array/arrayReverse.cpp @@ -91,6 +91,8 @@ ColumnPtr FunctionArrayReverse::executeImpl(const ColumnsWithTypeAndName & argum || executeFixedString(*src_inner_col, offsets, *res_inner_col) || executeGeneric(*src_inner_col, offsets, *res_inner_col); + chassert(bool(src_nullable_col) == bool(res_nullable_col)); + if (src_nullable_col) if (!executeNumber(src_nullable_col->getNullMapColumn(), offsets, res_nullable_col->getNullMapColumn())) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of null map of the first argument of function {}", diff --git a/src/Interpreters/OptimizeIfChains.cpp b/src/Interpreters/OptimizeIfChains.cpp index ba4c7bcd95f..9a5f9bcb2e1 100644 --- a/src/Interpreters/OptimizeIfChains.cpp +++ b/src/Interpreters/OptimizeIfChains.cpp @@ -64,6 +64,7 @@ ASTs OptimizeIfChainsVisitor::ifChain(const ASTPtr & child) throw Exception(ErrorCodes::UNEXPECTED_AST_STRUCTURE, "Unexpected AST for function 'if'"); const auto * function_args = function_node->arguments->as(); + chassert(function_args); if (!function_args || function_args->children.size() != 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, From 433c226b1fc6770729f3c3d76bf33f03140c5a27 Mon Sep 17 00:00:00 2001 From: Pradeep Chhetri <30620077+chhetripradeep@users.noreply.github.com> Date: Thu, 9 Mar 2023 20:51:45 +0800 Subject: [PATCH 399/470] Revert "Revert "Add join_algorithm='grace_hash' to stress tests"" (#47372) --- tests/ci/stress.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 4f723dba101..12c40ea1f66 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -30,13 +30,15 @@ def get_options(i, upgrade_check): if i % 2 == 1: join_alg_num = i // 2 - if join_alg_num % 4 == 0: + if join_alg_num % 5 == 0: client_options.append("join_algorithm='parallel_hash'") - if join_alg_num % 4 == 1: + if join_alg_num % 5 == 1: client_options.append("join_algorithm='partial_merge'") - if join_alg_num % 4 == 2: + if join_alg_num % 5 == 2: client_options.append("join_algorithm='full_sorting_merge'") - if join_alg_num % 4 == 3: + if join_alg_num % 5 == 3: + client_options.append("join_algorithm='grace_hash'") + if join_alg_num % 5 == 4: client_options.append("join_algorithm='auto'") client_options.append("max_rows_in_join=1000") From 89de49ef3c3842ca47fd61cf93656ac99bd4b24c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 9 Mar 2023 08:31:28 -0500 Subject: [PATCH 400/470] add tabs for GCS and S3 --- .../example-datasets/nyc-taxi.md | 82 +++++++++++++++---- 1 file changed, 64 insertions(+), 18 deletions(-) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index 69098f63037..36bcb6f232e 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -5,17 +5,19 @@ sidebar_position: 2 description: Data for billions of taxi and for-hire vehicle (Uber, Lyft, etc.) trips originating in New York City since 2009 --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + # New York Taxi Data The New York taxi data consists of 3+ billion taxi and for-hire vehicle (Uber, Lyft, etc.) trips originating in New York City since 2009. The dataset can be obtained in a couple of ways: -- insert the data directly into ClickHouse Cloud from S3 +- insert the data directly into ClickHouse Cloud from S3 or GCS - download prepared partitions -## Retrieve the Data from S3 +## Create the table trips -Let's grab a small subset of the data for getting familiar with it. The data is in TSV files in AWS S3, which is easily streamed into -ClickHouse Cloud using the `s3` table function. Start by creating a table for the taxi rides: +Start by creating a table for the taxi rides: ```sql CREATE TABLE trips ( @@ -38,9 +40,50 @@ CREATE TABLE trips ( dropoff_ntaname LowCardinality(String) ) ENGINE = MergeTree -PRIMARY KEY (pickup_datetime, dropoff_datetime) +PRIMARY KEY (pickup_datetime, dropoff_datetime); ``` +## Retrieve the Data from Object Storage + +Let's grab a small subset of the data for getting familiar with it. The data is in TSV files in object storage, which is easily streamed into +ClickHouse Cloud using the `s3` table function. + +The same data is stored in both S3 and GCS; choose either tab. + + + + +The following command streams three files from a GCS bucket into the `trips` table (the `{0..2}` syntax is a wildcard for the values 0, 1, and 2): + +```sql +INSERT INTO trips +SELECT + trip_id, + pickup_datetime, + dropoff_datetime, + pickup_longitude, + pickup_latitude, + dropoff_longitude, + dropoff_latitude, + passenger_count, + trip_distance, + fare_amount, + extra, + tip_amount, + tolls_amount, + total_amount, + payment_type, + pickup_ntaname, + dropoff_ntaname +FROM s3( + 'https://storage.googleapis.com/clickhouse-public-datasets/nyc-taxi/trips_{0..2}.gz', + 'TabSeparatedWithNames' +); +``` + + + + The following command streams three files from an S3 bucket into the `trips` table (the `{0..2}` syntax is a wildcard for the values 0, 1, and 2): ```sql @@ -66,14 +109,17 @@ SELECT FROM s3( 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/nyc-taxi/trips_{0..2}.gz', 'TabSeparatedWithNames' -) +); ``` + + + Let's see how many rows were inserted: ```sql SELECT count() -FROM trips +FROM trips; ``` Each TSV file has about 1M rows, and the three files have 3,000,317 rows. Let's look at a few rows: @@ -81,7 +127,7 @@ Each TSV file has about 1M rows, and the three files have 3,000,317 rows. Let's ```sql SELECT * FROM trips -LIMIT 10 +LIMIT 10; ``` Notice there are columns for the pickup and dropoff dates, geo coordinates, fare details, New York neighborhoods, and more: @@ -110,7 +156,7 @@ SELECT FROM trips GROUP BY pickup_ntaname ORDER BY count DESC -LIMIT 10 +LIMIT 10; ``` The result is: @@ -137,7 +183,7 @@ SELECT passenger_count, avg(total_amount) FROM trips -GROUP BY passenger_count +GROUP BY passenger_count; ``` ```response @@ -165,7 +211,7 @@ SELECT count(*) FROM trips GROUP BY passenger_count, year, distance -ORDER BY year, count(*) DESC +ORDER BY year, count(*) DESC; ``` The first part of the result is: @@ -211,7 +257,7 @@ If you will run the queries described below, you have to use the full table name Q1: ``` sql -SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type +SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type; ``` 0.490 seconds. @@ -219,7 +265,7 @@ SELECT cab_type, count(*) FROM trips_mergetree GROUP BY cab_type Q2: ``` sql -SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count +SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenger_count; ``` 1.224 seconds. @@ -227,7 +273,7 @@ SELECT passenger_count, avg(total_amount) FROM trips_mergetree GROUP BY passenge Q3: ``` sql -SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year +SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetree GROUP BY passenger_count, year; ``` 2.104 seconds. @@ -238,7 +284,7 @@ Q4: SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) FROM trips_mergetree GROUP BY passenger_count, year, distance -ORDER BY year, count(*) DESC +ORDER BY year, count(*) DESC; ``` 3.593 seconds. @@ -254,19 +300,19 @@ Creating a table on three servers: On each server: ``` sql -CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192) +CREATE TABLE default.trips_mergetree_third ( trip_id UInt32, vendor_id Enum8('1' = 1, '2' = 2, 'CMT' = 3, 'VTS' = 4, 'DDS' = 5, 'B02512' = 10, 'B02598' = 11, 'B02617' = 12, 'B02682' = 13, 'B02764' = 14), pickup_date Date, pickup_datetime DateTime, dropoff_date Date, dropoff_datetime DateTime, store_and_fwd_flag UInt8, rate_code_id UInt8, pickup_longitude Float64, pickup_latitude Float64, dropoff_longitude Float64, dropoff_latitude Float64, passenger_count UInt8, trip_distance Float64, fare_amount Float32, extra Float32, mta_tax Float32, tip_amount Float32, tolls_amount Float32, ehail_fee Float32, improvement_surcharge Float32, total_amount Float32, payment_type_ Enum8('UNK' = 0, 'CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), trip_type UInt8, pickup FixedString(25), dropoff FixedString(25), cab_type Enum8('yellow' = 1, 'green' = 2, 'uber' = 3), pickup_nyct2010_gid UInt8, pickup_ctlabel Float32, pickup_borocode UInt8, pickup_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), pickup_ct2010 FixedString(6), pickup_boroct2010 FixedString(7), pickup_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), pickup_ntacode FixedString(4), pickup_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), pickup_puma UInt16, dropoff_nyct2010_gid UInt8, dropoff_ctlabel Float32, dropoff_borocode UInt8, dropoff_boroname Enum8('' = 0, 'Manhattan' = 1, 'Bronx' = 2, 'Brooklyn' = 3, 'Queens' = 4, 'Staten Island' = 5), dropoff_ct2010 FixedString(6), dropoff_boroct2010 FixedString(7), dropoff_cdeligibil Enum8(' ' = 0, 'E' = 1, 'I' = 2), dropoff_ntacode FixedString(4), dropoff_ntaname Enum16('' = 0, 'Airport' = 1, 'Allerton-Pelham Gardens' = 2, 'Annadale-Huguenot-Prince\'s Bay-Eltingville' = 3, 'Arden Heights' = 4, 'Astoria' = 5, 'Auburndale' = 6, 'Baisley Park' = 7, 'Bath Beach' = 8, 'Battery Park City-Lower Manhattan' = 9, 'Bay Ridge' = 10, 'Bayside-Bayside Hills' = 11, 'Bedford' = 12, 'Bedford Park-Fordham North' = 13, 'Bellerose' = 14, 'Belmont' = 15, 'Bensonhurst East' = 16, 'Bensonhurst West' = 17, 'Borough Park' = 18, 'Breezy Point-Belle Harbor-Rockaway Park-Broad Channel' = 19, 'Briarwood-Jamaica Hills' = 20, 'Brighton Beach' = 21, 'Bronxdale' = 22, 'Brooklyn Heights-Cobble Hill' = 23, 'Brownsville' = 24, 'Bushwick North' = 25, 'Bushwick South' = 26, 'Cambria Heights' = 27, 'Canarsie' = 28, 'Carroll Gardens-Columbia Street-Red Hook' = 29, 'Central Harlem North-Polo Grounds' = 30, 'Central Harlem South' = 31, 'Charleston-Richmond Valley-Tottenville' = 32, 'Chinatown' = 33, 'Claremont-Bathgate' = 34, 'Clinton' = 35, 'Clinton Hill' = 36, 'Co-op City' = 37, 'College Point' = 38, 'Corona' = 39, 'Crotona Park East' = 40, 'Crown Heights North' = 41, 'Crown Heights South' = 42, 'Cypress Hills-City Line' = 43, 'DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill' = 44, 'Douglas Manor-Douglaston-Little Neck' = 45, 'Dyker Heights' = 46, 'East Concourse-Concourse Village' = 47, 'East Elmhurst' = 48, 'East Flatbush-Farragut' = 49, 'East Flushing' = 50, 'East Harlem North' = 51, 'East Harlem South' = 52, 'East New York' = 53, 'East New York (Pennsylvania Ave)' = 54, 'East Tremont' = 55, 'East Village' = 56, 'East Williamsburg' = 57, 'Eastchester-Edenwald-Baychester' = 58, 'Elmhurst' = 59, 'Elmhurst-Maspeth' = 60, 'Erasmus' = 61, 'Far Rockaway-Bayswater' = 62, 'Flatbush' = 63, 'Flatlands' = 64, 'Flushing' = 65, 'Fordham South' = 66, 'Forest Hills' = 67, 'Fort Greene' = 68, 'Fresh Meadows-Utopia' = 69, 'Ft. Totten-Bay Terrace-Clearview' = 70, 'Georgetown-Marine Park-Bergen Beach-Mill Basin' = 71, 'Glen Oaks-Floral Park-New Hyde Park' = 72, 'Glendale' = 73, 'Gramercy' = 74, 'Grasmere-Arrochar-Ft. Wadsworth' = 75, 'Gravesend' = 76, 'Great Kills' = 77, 'Greenpoint' = 78, 'Grymes Hill-Clifton-Fox Hills' = 79, 'Hamilton Heights' = 80, 'Hammels-Arverne-Edgemere' = 81, 'Highbridge' = 82, 'Hollis' = 83, 'Homecrest' = 84, 'Hudson Yards-Chelsea-Flatiron-Union Square' = 85, 'Hunters Point-Sunnyside-West Maspeth' = 86, 'Hunts Point' = 87, 'Jackson Heights' = 88, 'Jamaica' = 89, 'Jamaica Estates-Holliswood' = 90, 'Kensington-Ocean Parkway' = 91, 'Kew Gardens' = 92, 'Kew Gardens Hills' = 93, 'Kingsbridge Heights' = 94, 'Laurelton' = 95, 'Lenox Hill-Roosevelt Island' = 96, 'Lincoln Square' = 97, 'Lindenwood-Howard Beach' = 98, 'Longwood' = 99, 'Lower East Side' = 100, 'Madison' = 101, 'Manhattanville' = 102, 'Marble Hill-Inwood' = 103, 'Mariner\'s Harbor-Arlington-Port Ivory-Graniteville' = 104, 'Maspeth' = 105, 'Melrose South-Mott Haven North' = 106, 'Middle Village' = 107, 'Midtown-Midtown South' = 108, 'Midwood' = 109, 'Morningside Heights' = 110, 'Morrisania-Melrose' = 111, 'Mott Haven-Port Morris' = 112, 'Mount Hope' = 113, 'Murray Hill' = 114, 'Murray Hill-Kips Bay' = 115, 'New Brighton-Silver Lake' = 116, 'New Dorp-Midland Beach' = 117, 'New Springville-Bloomfield-Travis' = 118, 'North Corona' = 119, 'North Riverdale-Fieldston-Riverdale' = 120, 'North Side-South Side' = 121, 'Norwood' = 122, 'Oakland Gardens' = 123, 'Oakwood-Oakwood Beach' = 124, 'Ocean Hill' = 125, 'Ocean Parkway South' = 126, 'Old Astoria' = 127, 'Old Town-Dongan Hills-South Beach' = 128, 'Ozone Park' = 129, 'Park Slope-Gowanus' = 130, 'Parkchester' = 131, 'Pelham Bay-Country Club-City Island' = 132, 'Pelham Parkway' = 133, 'Pomonok-Flushing Heights-Hillcrest' = 134, 'Port Richmond' = 135, 'Prospect Heights' = 136, 'Prospect Lefferts Gardens-Wingate' = 137, 'Queens Village' = 138, 'Queensboro Hill' = 139, 'Queensbridge-Ravenswood-Long Island City' = 140, 'Rego Park' = 141, 'Richmond Hill' = 142, 'Ridgewood' = 143, 'Rikers Island' = 144, 'Rosedale' = 145, 'Rossville-Woodrow' = 146, 'Rugby-Remsen Village' = 147, 'Schuylerville-Throgs Neck-Edgewater Park' = 148, 'Seagate-Coney Island' = 149, 'Sheepshead Bay-Gerritsen Beach-Manhattan Beach' = 150, 'SoHo-TriBeCa-Civic Center-Little Italy' = 151, 'Soundview-Bruckner' = 152, 'Soundview-Castle Hill-Clason Point-Harding Park' = 153, 'South Jamaica' = 154, 'South Ozone Park' = 155, 'Springfield Gardens North' = 156, 'Springfield Gardens South-Brookville' = 157, 'Spuyten Duyvil-Kingsbridge' = 158, 'St. Albans' = 159, 'Stapleton-Rosebank' = 160, 'Starrett City' = 161, 'Steinway' = 162, 'Stuyvesant Heights' = 163, 'Stuyvesant Town-Cooper Village' = 164, 'Sunset Park East' = 165, 'Sunset Park West' = 166, 'Todt Hill-Emerson Hill-Heartland Village-Lighthouse Hill' = 167, 'Turtle Bay-East Midtown' = 168, 'University Heights-Morris Heights' = 169, 'Upper East Side-Carnegie Hill' = 170, 'Upper West Side' = 171, 'Van Cortlandt Village' = 172, 'Van Nest-Morris Park-Westchester Square' = 173, 'Washington Heights North' = 174, 'Washington Heights South' = 175, 'West Brighton' = 176, 'West Concourse' = 177, 'West Farms-Bronx River' = 178, 'West New Brighton-New Brighton-St. George' = 179, 'West Village' = 180, 'Westchester-Unionport' = 181, 'Westerleigh' = 182, 'Whitestone' = 183, 'Williamsbridge-Olinville' = 184, 'Williamsburg' = 185, 'Windsor Terrace' = 186, 'Woodhaven' = 187, 'Woodlawn-Wakefield' = 188, 'Woodside' = 189, 'Yorkville' = 190, 'park-cemetery-etc-Bronx' = 191, 'park-cemetery-etc-Brooklyn' = 192, 'park-cemetery-etc-Manhattan' = 193, 'park-cemetery-etc-Queens' = 194, 'park-cemetery-etc-Staten Island' = 195), dropoff_puma UInt16) ENGINE = MergeTree(pickup_date, pickup_datetime, 8192); ``` On the source server: ``` sql -CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()) +CREATE TABLE trips_mergetree_x3 AS trips_mergetree_third ENGINE = Distributed(perftest, default, trips_mergetree_third, rand()); ``` The following query redistributes data: ``` sql -INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree +INSERT INTO trips_mergetree_x3 SELECT * FROM trips_mergetree; ``` This takes 2454 seconds. From 504486e7b9f4769f41bfde17d6e4f06da07ad1c8 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 9 Mar 2023 08:34:05 -0500 Subject: [PATCH 401/470] correct heading --- docs/en/getting-started/example-datasets/nyc-taxi.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index 36bcb6f232e..aee2da5a545 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -43,7 +43,7 @@ ENGINE = MergeTree PRIMARY KEY (pickup_datetime, dropoff_datetime); ``` -## Retrieve the Data from Object Storage +## Load the Data directly from Object Storage Let's grab a small subset of the data for getting familiar with it. The data is in TSV files in object storage, which is easily streamed into ClickHouse Cloud using the `s3` table function. From c69d2c45b7d3e6a4661b7be6f05ce1aa42e56206 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 9 Mar 2023 08:35:29 -0500 Subject: [PATCH 402/470] correct heading --- docs/en/getting-started/example-datasets/nyc-taxi.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index aee2da5a545..0dc317eeeae 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -115,6 +115,8 @@ FROM s3( +## Sample Queries + Let's see how many rows were inserted: ```sql From 687fe3ea5198a5fa29ca6338ea9b4065acfa8296 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 9 Mar 2023 08:39:44 -0500 Subject: [PATCH 403/470] add note about prepared partitions --- docs/en/getting-started/example-datasets/nyc-taxi.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index 0dc317eeeae..9730faa873c 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -237,6 +237,10 @@ The first part of the result is: ## Download of Prepared Partitions {#download-of-prepared-partitions} +:::note +The following steps provide information about the original dataset, and a method for loading prepared partitions into a self-managed ClickHouse server environment. +::: + See https://github.com/toddwschneider/nyc-taxi-data and http://tech.marksblogg.com/billion-nyc-taxi-rides-redshift.html for the description of a dataset and instructions for downloading. Downloading will result in about 227 GB of uncompressed data in CSV files. The download takes about an hour over a 1 Gbit connection (parallel downloading from s3.amazonaws.com recovers at least half of a 1 Gbit channel). From 3aa0c32101fadf683a644f47972c117173a99e8a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 9 Mar 2023 14:50:13 +0100 Subject: [PATCH 404/470] Update docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md --- .../aggregate-functions/reference/exponentialmovingaverage.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index 296aae41daa..5546ade1758 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -181,7 +181,7 @@ ORDER BY time ASC; └───────┴─────────────────────┴─────────────┴──────────┘ --- Calculate timeunit timeunit using toRelativeHourNum +-- Calculate timeunit using toRelativeHourNum SELECT value, time, From 1d78c1b6d9232703d4e77cc98772a5d97a04cce1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 9 Mar 2023 14:45:26 +0000 Subject: [PATCH 405/470] Use force_sync=true in upgrade check --- docker/test/upgrade/run.sh | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index ce8a56c777e..de1f92823b2 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -60,6 +60,13 @@ install_packages previous_release_package_folder export USE_S3_STORAGE_FOR_MERGE_TREE=1 # Previous version may not be ready for fault injections export ZOOKEEPER_FAULT_INJECTION=0 + +# force_sync=false doesn't work correctly on some older versions +sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ + | sed "s|false|true|" \ + > /etc/clickhouse-server/config.d/keeper_port.xml.tmp +sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml + configure # But we still need default disk because some tables loaded only into it From b0931c89675c08ec41bfc51710a7cb6bd39ecc45 Mon Sep 17 00:00:00 2001 From: Clayton McClure <103603310+cmcclure-twilio@users.noreply.github.com> Date: Thu, 9 Mar 2023 07:49:00 -0700 Subject: [PATCH 406/470] Update copier to use group by to find partitions (#47386) --- programs/copier/ClusterCopier.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index bc882719a08..d3696f2cf12 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1867,8 +1867,8 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti String query; { WriteBufferFromOwnString wb; - wb << "SELECT DISTINCT " << partition_name << " AS partition FROM" - << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; + wb << "SELECT " << partition_name << " AS partition FROM " + << getQuotedTable(task_shard.table_read_shard) << " GROUP BY partition ORDER BY partition DESC"; query = wb.str(); } From 53277831cc07642963e6a64425700be8046022a1 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 9 Mar 2023 16:02:06 +0100 Subject: [PATCH 407/470] Remove dated warning scary language is inappropriately scary. --- docs/en/operations/utilities/clickhouse-local.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index a4fa5579638..79b8bc90634 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -14,10 +14,6 @@ Accepts data that represent tables and queries them using [ClickHouse SQL dialec By default `clickhouse-local` does not have access to data on the same host, but it supports loading server configuration using `--config-file` argument. -:::warning -It is not recommended to load production server configuration into `clickhouse-local` because data can be damaged in case of human error. -::: - For temporary data, a unique temporary data directory is created by default. ## Usage {#usage} From d81065ea646e482bd8f180d094fa20aed3be2904 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 9 Mar 2023 16:14:54 +0100 Subject: [PATCH 408/470] Fix headers in schema inference docs --- docs/en/interfaces/schema-inference.md | 80 +++++++++++++------------- 1 file changed, 40 insertions(+), 40 deletions(-) diff --git a/docs/en/interfaces/schema-inference.md b/docs/en/interfaces/schema-inference.md index 25bdb0c36a3..e028b4a6d96 100644 --- a/docs/en/interfaces/schema-inference.md +++ b/docs/en/interfaces/schema-inference.md @@ -117,7 +117,7 @@ clickhouse-local --file='hobbies.jsonl' --table='hobbies' --query='SELECT * FROM 4 47 Brayan ['movies','skydiving'] ``` -# Using structure from insertion table {#using-structure-from-insertion-table} +## Using structure from insertion table {#using-structure-from-insertion-table} When table functions `file/s3/url/hdfs` are used to insert data into a table, there is an option to use the structure from the insertion table instead of extracting it from the data. @@ -222,7 +222,7 @@ INSERT INTO hobbies4 SELECT id, empty(hobbies) ? NULL : hobbies[1] FROM file(hob In this case, there are some operations performed on the column `hobbies` in the `SELECT` query to insert it into the table, so ClickHouse cannot use the structure from the insertion table, and schema inference will be used. -# Schema inference cache {#schema-inference-cache} +## Schema inference cache {#schema-inference-cache} For most input formats schema inference reads some data to determine its structure and this process can take some time. To prevent inferring the same schema every time ClickHouse read the data from the same file, the inferred schema is cached and when accessing the same file again, ClickHouse will use the schema from the cache. @@ -326,14 +326,14 @@ SELECT count() FROM system.schema_inference_cache WHERE storage='S3' └─────────┘ ``` -# Text formats {#text-formats} +## Text formats {#text-formats} For text formats, ClickHouse reads the data row by row, extracts column values according to the format, and then uses some recursive parsers and heuristics to determine the type for each value. The maximum number of rows read from the data in schema inference is controlled by the setting `input_format_max_rows_to_read_for_schema_inference` with default value 25000. By default, all inferred types are [Nullable](../sql-reference/data-types/nullable.md), but you can change this by setting `schema_inference_make_columns_nullable` (see examples in the [settings](#settings-for-text-formats) section). -## JSON formats {#json-formats} +### JSON formats {#json-formats} In JSON formats ClickHouse parses values according to the JSON specification and then tries to find the most appropriate data type for them. @@ -464,9 +464,9 @@ most likely this column contains only Nulls or empty Arrays/Maps. ... ``` -### JSON settings {#json-settings} +#### JSON settings {#json-settings} -#### input_format_json_read_objects_as_strings +##### input_format_json_read_objects_as_strings Enabling this setting allows reading nested JSON objects as strings. This setting can be used to read nested JSON objects without using JSON object type. @@ -486,7 +486,7 @@ DESC format(JSONEachRow, $$ └──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -#### input_format_json_try_infer_numbers_from_strings +##### input_format_json_try_infer_numbers_from_strings Enabling this setting allows inferring numbers from string values. @@ -507,7 +507,7 @@ DESC format(JSONEachRow, $$ └───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -#### input_format_json_read_numbers_as_strings +##### input_format_json_read_numbers_as_strings Enabling this setting allows reading numeric values as strings. @@ -528,7 +528,7 @@ DESC format(JSONEachRow, $$ └───────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -#### input_format_json_read_bools_as_numbers +##### input_format_json_read_bools_as_numbers Enabling this setting allows reading Bool values as numbers. @@ -549,7 +549,7 @@ DESC format(JSONEachRow, $$ └───────┴─────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## CSV {#csv} +### CSV {#csv} In CSV format ClickHouse extracts column values from the row according to delimiters. ClickHouse expects all types except numbers and strings to be enclosed in double quotes. If the value is in double quotes, ClickHouse tries to parse the data inside quotes using the recursive parser and then tries to find the most appropriate data type for it. If the value is not in double quotes, ClickHouse tries to parse it as a number, @@ -726,7 +726,7 @@ $$) └──────────────┴───────────────┘ ``` -## TSV/TSKV {#tsv-tskv} +### TSV/TSKV {#tsv-tskv} In TSV/TSKV formats ClickHouse extracts column value from the row according to tabular delimiters and then parses extracted value using the recursive parser to determine the most appropriate type. If the type cannot be determined, ClickHouse treats this value as String. @@ -1019,7 +1019,7 @@ DESC format(TSV, '[1,2,3] 42.42 Hello World!') └──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## CustomSeparated {#custom-separated} +### CustomSeparated {#custom-separated} In CustomSeparated format ClickHouse first extracts all column values from the row according to specified delimiters and then tries to infer the data type for each value according to escaping rule. @@ -1080,7 +1080,7 @@ $$) └────────┴───────────────┴────────────┘ ``` -## Template {#template} +### Template {#template} In Template format ClickHouse first extracts all column values from the row according to the specified template and then tries to infer the data type for each value according to its escaping rule. @@ -1120,7 +1120,7 @@ $$) └──────────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## Regexp {#regexp} +### Regexp {#regexp} Similar to Template, in Regexp format ClickHouse first extracts all column values from the row according to specified regular expression and then tries to infer data type for each value according to the specified escaping rule. @@ -1142,9 +1142,9 @@ Line: value_1=2, value_2="Some string 2", value_3="[4, 5, NULL]"$$) └──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## Settings for text formats {settings-for-text-formats} +### Settings for text formats {#settings-for-text-formats} -### input_format_max_rows_to_read_for_schema_inference +#### input_format_max_rows_to_read_for_schema_inference This setting controls the maximum number of rows to be read while schema inference. The more rows are read, the more time is spent on schema inference, but the greater the chance to @@ -1152,7 +1152,7 @@ correctly determine the types (especially when the data contains a lot of nulls) Default value: `25000`. -### column_names_for_schema_inference +#### column_names_for_schema_inference The list of column names to use in schema inference for formats without explicit column names. Specified names will be used instead of default `c1,c2,c3,...`. The format: `column1,column2,column3,...`. @@ -1169,7 +1169,7 @@ DESC format(TSV, 'Hello, World! 42 [1, 2, 3]') settings column_names_for_schema_ └──────┴────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -### schema_inference_hints +#### schema_inference_hints The list of column names and types to use in schema inference instead of automatically determined types. The format: 'column_name1 column_type1, column_name2 column_type2, ...'. This setting can be used to specify the types of columns that could not be determined automatically or for optimizing the schema. @@ -1189,7 +1189,7 @@ DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : nul └─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -### 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, all inferred type will be `Nullable`, if disabled, the inferred type will be `Nullable` only if the column contains `NULL` in a sample that is parsed during schema inference. @@ -1232,7 +1232,7 @@ DESC format(JSONEachRow, $$ └─────────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -### input_format_try_infer_integers +#### input_format_try_infer_integers If enabled, ClickHouse will try to infer integers instead of floats in schema inference for text formats. If all numbers in the column from sample data are integers, the result type will be `Int64`, if at least one number is float, the result type will be `Float64`. @@ -1289,7 +1289,7 @@ DESC format(JSONEachRow, $$ └────────┴───────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -### input_format_try_infer_datetimes +#### 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)`, @@ -1337,7 +1337,7 @@ DESC format(JSONEachRow, $$ Note: Parsing datetimes during schema inference respect setting [date_time_input_format](/docs/en/operations/settings/settings-formats.md#date_time_input_format) -### input_format_try_infer_dates +#### input_format_try_infer_dates If enabled, ClickHouse will try to infer type `Date` from string fields in schema inference for text formats. If all fields from a column in sample data were successfully parsed as dates, the result type will be `Date`, @@ -1383,14 +1383,14 @@ DESC format(JSONEachRow, $$ └──────┴──────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -# Self describing formats {#self-describing-formats} +## Self describing formats {#self-describing-formats} Self-describing formats contain information about the structure of the data in the data itself, it can be some header with a description, a binary type tree, or some kind of table. To automatically infer a schema from files in such formats, ClickHouse reads a part of the data containing information about the types and converts it into a schema of the ClickHouse table. -## Formats with -WithNamesAndTypes suffix {#formats-with-names-and-types} +### Formats with -WithNamesAndTypes suffix {#formats-with-names-and-types} ClickHouse supports some text formats with the suffix -WithNamesAndTypes. This suffix means that the data contains two additional rows with column names and types before the actual data. While schema inference for such formats, ClickHouse reads the first two rows and extracts column names and types. @@ -1412,7 +1412,7 @@ $$) └──────┴──────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## JSON formats with metadata {#json-with-metadata} +### JSON formats with metadata {#json-with-metadata} Some JSON input formats ([JSON](formats.md#json), [JSONCompact](formats.md#json-compact), [JSONColumnsWithMetadata](formats.md#jsoncolumnswithmetadata)) contain metadata with column names and types. In schema inference for such formats, ClickHouse reads this metadata. @@ -1465,7 +1465,7 @@ $$) └──────┴──────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## Avro {#avro} +### Avro {#avro} In Avro format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches: @@ -1485,7 +1485,7 @@ In Avro format ClickHouse reads its schema from the data and converts it to Clic Other Avro types are not supported. -## Parquet {#parquet} +### Parquet {#parquet} In Parquet format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches: @@ -1513,7 +1513,7 @@ In Parquet format ClickHouse reads its schema from the data and converts it to C Other Parquet types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`. -## Arrow {#arrow} +### Arrow {#arrow} In Arrow format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches: @@ -1541,7 +1541,7 @@ In Arrow format ClickHouse reads its schema from the data and converts it to Cli Other Arrow types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`. -## ORC {#orc} +### ORC {#orc} In ORC format ClickHouse reads its schema from the data and converts it to ClickHouse schema using the following type matches: @@ -1564,17 +1564,17 @@ In ORC format ClickHouse reads its schema from the data and converts it to Click Other ORC types are not supported. By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`. -## Native {#native} +### Native {#native} Native format is used inside ClickHouse and contains the schema in the data. In schema inference, ClickHouse reads the schema from the data without any transformations. -# Formats with external schema {#formats-with-external-schema} +## Formats with external schema {#formats-with-external-schema} Such formats require a schema describing the data in a separate file in a specific schema language. To automatically infer a schema from files in such formats, ClickHouse reads external schema from a separate file and transforms it to a ClickHouse table schema. -# Protobuf {#protobuf} +### Protobuf {#protobuf} In schema inference for Protobuf format ClickHouse uses the following type matches: @@ -1592,7 +1592,7 @@ In schema inference for Protobuf format ClickHouse uses the following type match | `repeated T` | [Array(T)](../sql-reference/data-types/array.md) | | `message`, `group` | [Tuple](../sql-reference/data-types/tuple.md) | -# CapnProto {#capnproto} +### CapnProto {#capnproto} In schema inference for CapnProto format ClickHouse uses the following type matches: @@ -1615,13 +1615,13 @@ In schema inference for CapnProto format ClickHouse uses the following type matc | `struct` | [Tuple](../sql-reference/data-types/tuple.md) | | `union(T, Void)`, `union(Void, T)` | [Nullable(T)](../sql-reference/data-types/nullable.md) | -# Strong-typed binary formats {#strong-typed-binary-formats} +## Strong-typed binary formats {#strong-typed-binary-formats} In such formats, each serialized value contains information about its type (and possibly about its name), but there is no information about the whole table. In schema inference for such formats, ClickHouse reads data row by row (up to `input_format_max_rows_to_read_for_schema_inference` rows) and extracts the type (and possibly name) for each value from the data and then converts these types to ClickHouse types. -## MsgPack {msgpack} +### MsgPack {#msgpack} In MsgPack format there is no delimiter between rows, to use schema inference for this format you should specify the number of columns in the table using the setting `input_format_msgpack_number_of_columns`. ClickHouse uses the following type matches: @@ -1641,7 +1641,7 @@ using the setting `input_format_msgpack_number_of_columns`. ClickHouse uses the By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`. -## BSONEachRow {#bsoneachrow} +### BSONEachRow {#bsoneachrow} In BSONEachRow each row of data is presented as a BSON document. In schema inference ClickHouse reads BSON documents one by one and extracts values, names, and types from the data and then transforms these types to ClickHouse types using the following type matches: @@ -1661,11 +1661,11 @@ values, names, and types from the data and then transforms these types to ClickH By default, all inferred types are inside `Nullable`, but it can be changed using the setting `schema_inference_make_columns_nullable`. -# Formats with constant schema {#formats-with-constant-schema} +## Formats with constant schema {#formats-with-constant-schema} Data in such formats always have the same schema. -## LineAsString {#line-as-string} +### LineAsString {#line-as-string} In this format, ClickHouse reads the whole line from the data into a single column with `String` data type. The inferred type for this format is always `String` and the column name is `line`. @@ -1680,7 +1680,7 @@ DESC format(LineAsString, 'Hello\nworld!') └──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## JSONAsString {#json-as-string} +### JSONAsString {#json-as-string} In this format, ClickHouse reads the whole JSON object from the data into a single column with `String` data type. The inferred type for this format is always `String` and the column name is `json`. @@ -1695,7 +1695,7 @@ DESC format(JSONAsString, '{"x" : 42, "y" : "Hello, World!"}') └──────┴────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -## JSONAsObject {#json-as-object} +### JSONAsObject {#json-as-object} In this format, ClickHouse reads the whole JSON object from the data into a single column with `Object('json')` data type. Inferred type for this format is always `String` and the column name is `json`. From 95351bc2d365843387a9709cd8b936572ab3f929 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 9 Mar 2023 12:05:26 -0500 Subject: [PATCH 409/470] standardize admonitions --- .../database-engines/materialized-mysql.md | 4 +-- .../materialized-postgresql.md | 4 +-- docs/en/engines/database-engines/mysql.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +-- .../table-engines/integrations/kafka.md | 2 +- .../integrations/materialized-postgresql.md | 2 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 4 +-- .../mergetree-family/aggregatingmergetree.md | 2 +- .../mergetree-family/collapsingmergetree.md | 2 +- .../custom-partitioning-key.md | 4 +-- .../mergetree-family/graphitemergetree.md | 6 ++-- .../mergetree-family/invertedindexes.md | 2 +- .../mergetree-family/mergetree.md | 4 +-- .../mergetree-family/replacingmergetree.md | 4 +-- .../mergetree-family/replication.md | 2 +- .../mergetree-family/summingmergetree.md | 2 +- .../versionedcollapsingmergetree.md | 2 +- docs/en/interfaces/formats.md | 28 +++++++++---------- docs/en/interfaces/http.md | 4 +-- docs/en/interfaces/postgresql.md | 2 +- .../third-party/client-libraries.md | 2 +- .../en/interfaces/third-party/integrations.md | 2 +- docs/en/operations/access-rights.md | 4 +-- .../external-authenticators/kerberos.md | 6 ++-- docs/en/operations/opentelemetry.md | 2 +- docs/en/operations/query-cache.md | 2 +- .../settings.md | 4 +-- .../settings/merge-tree-settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings-users.md | 4 +-- docs/en/operations/settings/settings.md | 18 ++++++------ docs/en/operations/storing-data.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/tips.md | 2 +- .../operations/utilities/clickhouse-copier.md | 2 +- .../parametric-functions.md | 4 +-- docs/en/sql-reference/data-types/float.md | 2 +- docs/en/sql-reference/data-types/json.md | 4 +-- .../data-types/special-data-types/interval.md | 2 +- .../external-dicts-dict-layout.md | 4 +-- .../external-dicts-dict-structure.md | 4 +-- docs/en/sql-reference/distributed-ddl.md | 4 +-- .../sql-reference/functions/hash-functions.md | 4 +-- .../sql-reference/functions/introspection.md | 2 +- .../sql-reference/functions/nlp-functions.md | 2 +- .../functions/other-functions.md | 8 +++--- docs/en/sql-reference/operators/exists.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../sql-reference/statements/alter/column.md | 2 +- .../statements/alter/constraint.md | 2 +- .../statements/create/row-policy.md | 2 +- .../sql-reference/statements/create/table.md | 8 +++--- .../sql-reference/statements/create/user.md | 2 +- docs/en/sql-reference/statements/delete.md | 2 +- docs/en/sql-reference/statements/optimize.md | 2 +- docs/en/sql-reference/statements/system.md | 2 +- docs/en/sql-reference/statements/watch.md | 4 +-- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- .../table-functions/hdfsCluster.md | 2 +- .../en/sql-reference/table-functions/index.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- .../table-functions/s3Cluster.md | 2 +- 64 files changed, 113 insertions(+), 113 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 899c8d024f1..9b2d1d7e1de 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -6,7 +6,7 @@ sidebar_position: 70 # [experimental] MaterializedMySQL -:::warning +:::important This is an experimental feature that should not be used in production. ::: @@ -245,7 +245,7 @@ extra care needs to be taken. You may specify overrides for tables that do not exist yet. -:::warning +:::important It is easy to break replication with table overrides if not used with care. For example: * If an ALIAS column is added with a table override, and a column with the same name is later added to the source diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index b43f71a7576..7be520ee25e 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -54,7 +54,7 @@ After `MaterializedPostgreSQL` database is created, it does not automatically de ATTACH TABLE postgres_database.new_table; ``` -:::warning +:::important Before version 22.1, adding a table to replication left an unremoved temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. ::: @@ -145,7 +145,7 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` -:::warning +:::important Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/database-engines/mysql.md b/docs/en/engines/database-engines/mysql.md index e4ff734d55f..e2c4f134a90 100644 --- a/docs/en/engines/database-engines/mysql.md +++ b/docs/en/engines/database-engines/mysql.md @@ -60,7 +60,7 @@ These variables are supported: - `version` - `max_allowed_packet` -:::warning +:::important By now these variables are stubs and don't correspond to anything. ::: diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 7c04a6594a6..c14d05aa03a 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -64,7 +64,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not recommended. - :::warning Zero-copy replication is not ready for production + :::important Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -110,7 +110,7 @@ Table consists of all the files in both directories (all files should satisfy fo CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV') ``` -:::warning +:::important If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 255ba06f056..86647fb6978 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -102,7 +102,7 @@ Examples: Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index 11e7928c3ed..c9b3f351568 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -52,6 +52,6 @@ PRIMARY KEY key; SELECT key, value, _version FROM postgresql_db.postgresql_replica; ``` -:::warning +:::important Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index b73d28c8508..d338a2a58bd 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -74,7 +74,7 @@ All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` samp PostgreSQL `Array` types are converted into ClickHouse arrays. -:::warning +:::important Be careful - in PostgreSQL an array data, created like a `type_name[]`, may contain multi-dimensional arrays of different dimensions in different table rows in same column. But in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. ::: diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 723425429a5..8e1a4d91cac 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -63,7 +63,7 @@ For more information about virtual columns see [here](../../../engines/table-eng - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported. - :::warning Zero-copy replication is not ready for production + :::important Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -78,7 +78,7 @@ For more information about virtual columns see [here](../../../engines/table-eng Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. -:::warning +:::important If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 9677f75a358..9b7f0cd2486 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -43,7 +43,7 @@ When creating an `AggregatingMergeTree` table the same [clauses](../../../engine Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md index 0bd665116f0..9acff099557 100644 --- a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -45,7 +45,7 @@ When creating a `CollapsingMergeTree` table, the same [query clauses](../../../e Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index b1e79c4c3fd..f01cd90ccf0 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -6,7 +6,7 @@ sidebar_label: Custom Partitioning Key # Custom Partitioning Key -:::warning +:::important In most cases you do not need a partition key, and in most other cases you do not need a partition key more granular than by months. You should never use too granular of partitioning. Don't partition your data by client identifiers or names. Instead, make a client identifier or name the first column in the ORDER BY expression. @@ -159,7 +159,7 @@ FROM session_log GROUP BY UserID; ``` -:::warning +:::important Performance of such a query heavily depends on the table layout. Because of that the optimisation is not enabled by default. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md index 104ec049ec4..d71004135f1 100644 --- a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md @@ -55,7 +55,7 @@ When creating a `GraphiteMergeTree` table, the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: @@ -129,7 +129,7 @@ default ... ``` -:::warning +:::important Patterns must be strictly ordered: 1. Patterns without `function` or `retention`. @@ -263,6 +263,6 @@ Valid values: ``` -:::warning +:::important Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer). ::: diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index aa11258dc4a..91ee0313a69 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -15,7 +15,7 @@ tokenized cells of the string column. For example, the string cell "I will be a " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more useful the resulting inverted index will be. -:::warning +:::important Inverted indexes are experimental and should not be used in production environments yet. They may change in the future in backward-incompatible ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index fc8060077b0..5dbe7602856 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -192,7 +192,7 @@ The `index_granularity` setting can be omitted because 8192 is the default value Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects. If possible, switch old projects to the method described above. ::: @@ -1087,7 +1087,7 @@ Other parameters: Examples of working configurations can be found in integration tests directory (see e.g. [test_merge_tree_azure_blob_storage](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml) or [test_azure_blob_storage_zero_copy_replication](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml)). - :::warning Zero-copy replication is not ready for production + :::important Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index f5d81182898..0b76500f2a6 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -30,7 +30,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md). -:::warning +:::important Uniqueness of rows is determined by the `ORDER BY` table section, not `PRIMARY KEY`. ::: @@ -96,7 +96,7 @@ When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 37ab8ac9fd3..f9adad0fa6d 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -43,7 +43,7 @@ ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/clickhouse-keeper To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. -:::warning +:::important Don’t neglect the security setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index b2b6272c58e..bea37706939 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -44,7 +44,7 @@ When creating a `SummingMergeTree` table the same [clauses](../../../engines/tab Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index 2891907f79a..1a44569711a 100644 --- a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -58,7 +58,7 @@ When creating a `VersionedCollapsingMergeTree` table, the same [clauses](../../. Deprecated Method for Creating a Table -:::warning +:::important Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index db2e773a685..fa15de16c71 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -205,7 +205,7 @@ Differs from the `TabSeparated` format in that the column names are written in t During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness. -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns of the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -217,7 +217,7 @@ This format is also available under the name `TSVWithNames`. Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns in the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -470,7 +470,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -480,7 +480,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -500,7 +500,7 @@ There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [Templat Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -510,7 +510,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -969,7 +969,7 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -979,7 +979,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -991,7 +991,7 @@ the types from input data will be compared with the types of the corresponding c Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1001,7 +1001,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1120,7 +1120,7 @@ CREATE TABLE IF NOT EXISTS example_table - If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type). - If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`. -:::warning +:::important When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHouse consumes more computational resources, compared to insertion with `input_format_defaults_for_omitted_fields = 0`. ::: @@ -1450,7 +1450,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N) - N `String`s specifying column names -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1464,7 +1464,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - N `String`s specifying column names - N `String`s specifying column types -:::warning +:::important If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1912,7 +1912,7 @@ SET format_avro_schema_registry_url = 'http://schema-registry'; SELECT * FROM topic1_stream; ``` -:::warning +:::important Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` to maintain it’s value after a restart. Also you can use the `format_avro_schema_registry_url` setting of the `Kafka` table engine. ::: diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 9af6df0c87d..efa51bb34ea 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -445,7 +445,7 @@ Next are the configuration methods for different `type`. The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_final_threads` settings, then queries the system table to check whether these settings were set successfully. -:::warning +:::important To keep the default `handlers` such as` query`, `play`,` ping`, add the `` rule. ::: @@ -476,7 +476,7 @@ $ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost: max_final_threads 2 ``` -:::warning +:::important In one `predefined_query_handler` only supports one `query` of an insert type. ::: diff --git a/docs/en/interfaces/postgresql.md b/docs/en/interfaces/postgresql.md index 9ff83559787..0cc70cce571 100644 --- a/docs/en/interfaces/postgresql.md +++ b/docs/en/interfaces/postgresql.md @@ -54,7 +54,7 @@ default=> And that's it! You now have a PostgreSQL client connected to ClickHouse, and all commands and queries are executed on ClickHouse. -:::caution +:::important The PostgreSQL protocol currently only supports plain-text passwords. ::: diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 0e065cb7179..4ce63ba647b 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -6,7 +6,7 @@ sidebar_label: Client Libraries # Client Libraries from Third-party Developers -:::warning +:::important ClickHouse Inc does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality. ::: diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 90a4f088be7..8197a1d1f58 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -6,7 +6,7 @@ sidebar_label: Integrations # Integration Libraries from Third-party Developers -:::warning Disclaimer +:::important Disclaimer ClickHouse, Inc. does **not** maintain the tools and libraries listed below and haven’t done extensive testing to ensure their quality. ::: diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md index 4c4a06dbe1e..38d32e3f1f7 100644 --- a/docs/en/operations/access-rights.md +++ b/docs/en/operations/access-rights.md @@ -24,7 +24,7 @@ You can configure access entities using: We recommend using SQL-driven workflow. Both of the configuration methods work simultaneously, so if you use the server configuration files for managing accounts and access rights, you can smoothly switch to SQL-driven workflow. -:::warning +:::important You can’t manage the same access entity by both configuration methods simultaneously. ::: @@ -102,7 +102,7 @@ Privileges can be granted to a role by the [GRANT](../sql-reference/statements/g Row policy is a filter that defines which of the rows are available to a user or a role. Row policy contains filters for one particular table, as well as a list of roles and/or users which should use this row policy. -:::warning +:::important 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. ::: diff --git a/docs/en/operations/external-authenticators/kerberos.md b/docs/en/operations/external-authenticators/kerberos.md index 95944e96194..b49291f8e2e 100644 --- a/docs/en/operations/external-authenticators/kerberos.md +++ b/docs/en/operations/external-authenticators/kerberos.md @@ -59,11 +59,11 @@ With filtering by realm: ``` -:::warning +:::important You can define only one `kerberos` section. The presence of multiple `kerberos` sections will force ClickHouse to disable Kerberos authentication. ::: -:::warning +:::important `principal` and `realm` sections cannot be specified at the same time. The presence of both `principal` and `realm` sections will force ClickHouse to disable Kerberos authentication. ::: @@ -103,7 +103,7 @@ Example (goes into `users.xml`): ``` -:::warning +:::important Note that Kerberos authentication cannot be used alongside with any other authentication mechanism. The presence of any other sections like `password` alongside `kerberos` will force ClickHouse to shutdown. ::: diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 1de5a09db0c..7faa4b1ee7c 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -7,7 +7,7 @@ title: "[experimental] Tracing ClickHouse with OpenTelemetry" [OpenTelemetry](https://opentelemetry.io/) is an open standard for collecting traces and metrics from the distributed application. ClickHouse has some support for OpenTelemetry. -:::warning +:::important This is an experimental feature that will change in backwards-incompatible ways in future releases. ::: diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 1a486de7904..a7ac011c796 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -29,7 +29,7 @@ Transactionally inconsistent caching is traditionally provided by client tools o the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side. This reduces maintenance effort and avoids redundancy. -:::warning +:::important The query cache is an experimental feature that should not be used in production. There are known cases (e.g. in distributed query processing) where wrong results are returned. ::: diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 3fe815bc79a..4494ad39a0e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -24,7 +24,7 @@ Default value: 3600. Data compression settings for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-engine tables. -:::warning +:::tip Don’t use it if you have just started using ClickHouse. ::: @@ -1367,7 +1367,7 @@ The following settings are available: Changed settings take effect immediately. -:::warning +:::important Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query cache altogether. ::: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 5bc174727ad..6290a23378c 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -289,7 +289,7 @@ Default value: 0 (seconds) When this setting has a value greater than than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled. -:::warning Zero-copy replication is not ready for production +:::important Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 919ebaf562f..b03f922d0d4 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -142,7 +142,7 @@ y Nullable(String) z IPv4 ``` -:::warning +:::important If the `schema_inference_hints` is not formated properly, or if there is a typo or a wrong datatype, etc... the whole schema_inference_hints will be ignored. ::: diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index b55d64fc4f7..ac0025a1137 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -118,8 +118,8 @@ To open access for user from any network, specify: ::/0 ``` -:::warning -It’s insecure to open access from any network unless you have a firewall properly configured or the server is not directly connected to Internet. +:::important +It is insecure to open access from any network unless you have a firewall properly configured or the server is not directly connected to Internet. ::: To open access only from localhost, specify: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 94dcf159ca9..809c7c3e837 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -460,7 +460,7 @@ Possible values: Changes the behaviour of join operations with `ANY` strictness. -:::warning +:::important This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables. ::: @@ -550,7 +550,7 @@ Default value: 64. Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations. -:::warning +:::important Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour. ::: @@ -942,7 +942,7 @@ Higher values will lead to higher memory usage. The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. -:::warning +:::important This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -960,7 +960,7 @@ We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won’t be decompressed. -:::warning +:::important This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -1247,7 +1247,7 @@ Possible values: Default value: 1. -:::warning +:::important Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) without [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key). If [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. @@ -1277,7 +1277,7 @@ Default value: `1`. This options will produce different results depending on the settings used. -:::warning +:::important This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: @@ -2186,7 +2186,7 @@ Default value: 0. This setting also affects broken batches (that may appears because of abnormal server (machine) termination and no `fsync_after_insert`/`fsync_directories` for [Distributed](../../engines/table-engines/special/distributed.md) table engine). ::: -:::warning +:::important You should not rely on automatic batch splitting, since this may hurt performance. ::: @@ -2194,7 +2194,7 @@ You should not rely on automatic batch splitting, since this may hurt performanc Sets the priority ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) for threads that execute queries. The OS scheduler considers this priority when choosing the next thread to run on each available CPU core. -:::warning +:::important To use this setting, you need to set the `CAP_SYS_NICE` capability. The `clickhouse-server` package sets it up during installation. Some virtual environments do not allow you to set the `CAP_SYS_NICE` capability. In this case, `clickhouse-server` shows a message about it at the start. ::: @@ -2858,7 +2858,7 @@ Possible values: Default value: `0`. -:::warning +:::important Nullable primary key usually indicates bad design. It is forbidden in almost all main stream DBMS. The feature is mainly for [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) and is not heavily tested. Use with care. ::: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 3f9a0f67187..21c3bf84250 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -471,6 +471,6 @@ Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#htt Zero-copy replication is possible, but not recommended, with `S3` and `HDFS` disks. Zero-copy replication means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. -:::warning Zero-copy replication is not ready for production +:::important Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 106d3c59dea..c9477214fd6 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -99,7 +99,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -:::warning +:::important The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index da34a6b7e9c..00e7155edc1 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -36,7 +36,7 @@ $ echo 0 | sudo tee /proc/sys/vm/overcommit_memory Use `perf top` to watch the time spent in the kernel for memory management. Permanent huge pages also do not need to be allocated. -:::warning +:::important If your system has less than 16 GB of RAM, you may experience various memory exceptions because default settings do not match this amount of memory. The recommended amount of RAM is 32 GB or more. You can use ClickHouse in a system with a small amount of RAM, even with 2 GB of RAM, but it requires additional tuning and can ingest at a low rate. ::: diff --git a/docs/en/operations/utilities/clickhouse-copier.md b/docs/en/operations/utilities/clickhouse-copier.md index 87280bc3ba8..1b75530dda7 100644 --- a/docs/en/operations/utilities/clickhouse-copier.md +++ b/docs/en/operations/utilities/clickhouse-copier.md @@ -8,7 +8,7 @@ sidebar_label: clickhouse-copier Copies data from the tables in one cluster to tables in another (or the same) cluster. -:::warning +:::important To get a consistent copy, the data in the source tables and partitions should not change during the entire process. ::: diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 40184c0aa02..0bd43034c1b 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -90,7 +90,7 @@ Checks whether the sequence contains an event chain that matches the pattern. sequenceMatch(pattern)(timestamp, cond1, cond2, ...) ``` -:::warning +:::important Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: @@ -176,7 +176,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. -:::warning +:::important Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 38c414fa8cd..7a5fa088f6e 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -6,7 +6,7 @@ sidebar_label: Float32, Float64 # Float32, Float64 -:::warning +:::important If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: ``` diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index d9099ba5ad3..29d6553b888 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -6,7 +6,7 @@ sidebar_label: JSON # JSON -:::warning +:::important This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/guides/developer/working-with-json/json-load-data.md) instead. ::: @@ -14,7 +14,7 @@ Stores JavaScript Object Notation (JSON) documents in a single column. `JSON` is an alias for `Object('json')`. -:::warning +:::important The JSON data type is an experimental feature. To use it, set `allow_experimental_object_type = 1`. ::: 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 5169bc646c9..68494e52360 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 @@ -8,7 +8,7 @@ sidebar_label: Interval The family of data types representing time and date intervals. The resulting types of the [INTERVAL](../../../sql-reference/operators/index.md#operator-interval) operator. -:::warning +:::important `Interval` data type values can’t be stored in tables. ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 4dc6fd33849..7af5923e052 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -299,7 +299,7 @@ Example: The table contains discounts for each advertiser in the format: To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). These elements must contain elements `name` and `type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others). -:::warning +:::important Values of `range_min` and `range_max` should fit in `Int64` type. ::: @@ -588,7 +588,7 @@ Set a large enough cache size. You need to experiment to select the number of ce 3. Assess memory consumption using the `system.dictionaries` table. 4. Increase or decrease the number of cells until the required memory consumption is reached. -:::warning +:::important Do not use ClickHouse as a source, because it is slow to process queries with random reads. ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 8271a342941..06d1c817a13 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -60,7 +60,7 @@ ClickHouse supports the following types of keys: An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. -:::warning +:::important You must not describe key as an attribute. ::: @@ -178,4 +178,4 @@ Configuration fields: ## Related Content -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) \ No newline at end of file +- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) diff --git a/docs/en/sql-reference/distributed-ddl.md b/docs/en/sql-reference/distributed-ddl.md index ff5155391be..d170f3765c2 100644 --- a/docs/en/sql-reference/distributed-ddl.md +++ b/docs/en/sql-reference/distributed-ddl.md @@ -18,6 +18,6 @@ In order to run these queries correctly, each host must have the same cluster de The local version of the query will eventually be executed on each host in the cluster, even if some hosts are currently not available. -:::warning +:::important The order for executing queries within a single host is guaranteed. -::: \ No newline at end of file +::: diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 69dc73e2fb0..833e0a40b2b 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -125,7 +125,7 @@ SELECT sipHash64Keyed((506097522914230528, 1084818905618843912), array('e','x',' Like [sipHash64](#hash_functions-siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. -:::warning +:::important This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128Reference](#hash_functions-siphash128reference). @@ -165,7 +165,7 @@ Result: Same as [sipHash128](#hash_functions-siphash128) but additionally takes an explicit key argument instead of using a fixed key. -:::warning +:::important This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128ReferenceKeyed](#hash_functions-siphash128referencekeyed). diff --git a/docs/en/sql-reference/functions/introspection.md b/docs/en/sql-reference/functions/introspection.md index 9357f75b8e6..78f763318df 100644 --- a/docs/en/sql-reference/functions/introspection.md +++ b/docs/en/sql-reference/functions/introspection.md @@ -8,7 +8,7 @@ sidebar_label: Introspection You can use functions described in this chapter to introspect [ELF](https://en.wikipedia.org/wiki/Executable_and_Linkable_Format) and [DWARF](https://en.wikipedia.org/wiki/DWARF) for query profiling. -:::warning +:::important These functions are slow and may impose security considerations. ::: diff --git a/docs/en/sql-reference/functions/nlp-functions.md b/docs/en/sql-reference/functions/nlp-functions.md index f68448af2be..62116fe50a3 100644 --- a/docs/en/sql-reference/functions/nlp-functions.md +++ b/docs/en/sql-reference/functions/nlp-functions.md @@ -5,7 +5,7 @@ sidebar_label: NLP title: "[experimental] Natural Language Processing functions" --- -:::warning +:::important This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in future releases. Set `allow_experimental_nlp_functions = 1` to enable it. ::: diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 7146484361e..0f95f16ffed 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -792,7 +792,7 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::warning +:::important It can reach the neighbor rows only inside the currently processed data block. ::: @@ -902,7 +902,7 @@ Result: Calculates the difference between successive row values ​​in the data block. Returns 0 for the first row and the difference from the previous row for each subsequent row. -:::warning +:::important It can reach the previous row only inside the currently processed data block. ::: @@ -986,7 +986,7 @@ Each event has a start time and an end time. The start time is included in the e The function calculates the total number of active (concurrent) events for each event start time. -:::warning +:::important Events must be ordered by the start time in ascending order. If this requirement is violated the function raises an exception. Every data block is processed separately. If events from different data blocks overlap then they can not be processed correctly. ::: @@ -1674,7 +1674,7 @@ Result: Accumulates states of an aggregate function for each row of a data block. -:::warning +:::important The state is reset for each new data block. ::: diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 4bc29389c9c..6819048e28f 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -7,7 +7,7 @@ The `EXISTS` operator checks how many records are in the result of a subquery. I `EXISTS` can be used in a [WHERE](../../sql-reference/statements/select/where.md) clause. -:::warning +:::important References to main query tables and columns are not supported in a subquery. ::: diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 0fe7ebbf4b6..98dee830cd4 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -229,7 +229,7 @@ Types of intervals: You can also use a string literal when setting the `INTERVAL` value. For example, `INTERVAL 1 HOUR` is identical to the `INTERVAL '1 hour'` or `INTERVAL '1' hour`. -:::warning +:::important Intervals with different types can’t be combined. You can’t use expressions like `INTERVAL 4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, `INTERVAL 25 HOUR`. You can use consecutive operations, like in the example below. ::: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index d580efa4992..99958c31b92 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -75,7 +75,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -:::warning +:::important You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: diff --git a/docs/en/sql-reference/statements/alter/constraint.md b/docs/en/sql-reference/statements/alter/constraint.md index 844b24d7374..0b19966798b 100644 --- a/docs/en/sql-reference/statements/alter/constraint.md +++ b/docs/en/sql-reference/statements/alter/constraint.md @@ -17,7 +17,7 @@ See more on [constraints](../../../sql-reference/statements/create/table.md#cons Queries will add or remove metadata about constraints from table so they are processed immediately. -:::warning +:::important Constraint check **will not be executed** on existing data if it was added. ::: diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 31ce9221eea..93bc5729c8e 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -7,7 +7,7 @@ title: "CREATE ROW POLICY" Creates a [row policy](../../../operations/access-rights.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. -:::warning +:::important 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. ::: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 50e74920e4b..7da1c9669ff 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -286,7 +286,7 @@ ENGINE = engine PRIMARY KEY(expr1[, expr2,...]); ``` -:::warning +:::important You can't combine both ways in one query. ::: @@ -342,7 +342,7 @@ ALTER TABLE codec_example MODIFY COLUMN float_value CODEC(Default); Codecs can be combined in a pipeline, for example, `CODEC(Delta, Default)`. -:::warning +:::important You can’t decompress ClickHouse database files with external utilities like `lz4`. Instead, use the special [clickhouse-compressor](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utility. ::: @@ -437,11 +437,11 @@ Encryption codecs: These codecs use a fixed nonce and encryption is therefore deterministic. This makes it compatible with deduplicating engines such as [ReplicatedMergeTree](../../../engines/table-engines/mergetree-family/replication.md) but has a weakness: when the same data block is encrypted twice, the resulting ciphertext will be exactly the same so an adversary who can read the disk can see this equivalence (although only the equivalence, without getting its content). -:::warning +:::important Most engines including the "\*MergeTree" family create index files on disk without applying codecs. This means plaintext will appear on disk if an encrypted column is indexed. ::: -:::warning +:::important If you perform a SELECT query mentioning a specific value in an encrypted column (such as in its WHERE clause), the value may appear in [system.query_log](../../../operations/system-tables/query_log.md). You may want to disable the logging. ::: diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index a756b3d4a0d..e2cf195ebd6 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -55,7 +55,7 @@ Another way of specifying host is to use `@` syntax following the username. Exam - `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. - `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. -:::warning +:::important ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we do not recommend to do so. ::: diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index e1987e50af4..e5a0a8f5b2b 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -32,7 +32,7 @@ SET allow_experimental_lightweight_delete = true; An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. -:::warning +:::important Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on an OLTP system. Lightweight deletes are currently efficient for wide parts, but for compact parts, they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. ::: diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 78615a2f9ad..ee5ad512dd5 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -7,7 +7,7 @@ title: "OPTIMIZE Statement" This query tries to initialize an unscheduled merge of data parts for tables. -:::warning +:::important `OPTIMIZE` can’t fix the `Too many parts` error. ::: diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index f9f55acfcec..a7fd0944593 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -312,7 +312,7 @@ One may execute query after: Replica attaches locally found parts and sends info about them to Zookeeper. Parts present on a replica before metadata loss are not re-fetched from other ones if not being outdated (so replica restoration does not mean re-downloading all data over the network). -:::warning +:::important Parts in all states are moved to `detached/` folder. Parts active before data loss (committed) are attached. ::: diff --git a/docs/en/sql-reference/statements/watch.md b/docs/en/sql-reference/statements/watch.md index 90d19e6be0e..e4220be60e9 100644 --- a/docs/en/sql-reference/statements/watch.md +++ b/docs/en/sql-reference/statements/watch.md @@ -6,7 +6,7 @@ sidebar_label: WATCH # WATCH Statement (Experimental) -:::warning +:::important This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable live views and `WATCH` query using `set allow_experimental_live_view = 1`. ::: @@ -107,4 +107,4 @@ The `FORMAT` clause works the same way as for the [SELECT](../../sql-reference/s :::note The [JSONEachRowWithProgress](../../interfaces/formats.md#jsoneachrowwithprogress) format should be used when watching [LIVE VIEW](./create/view.md#live-view) tables over the HTTP interface. The progress messages will be added to the output to keep the long-lived HTTP connection alive until the query result changes. The interval between progress messages is controlled using the [live_view_heartbeat_interval](./create/view.md#live-view-settings) setting. -::: \ No newline at end of file +::: diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 594c328c3ff..bf0fca648a3 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -109,7 +109,7 @@ Query the number of rows in all files of these two directories: SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 97a253a5356..784e45d0f81 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -79,7 +79,7 @@ SELECT count(*) FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfsCluster.md b/docs/en/sql-reference/table-functions/hdfsCluster.md index 231c552610f..78f56372310 100644 --- a/docs/en/sql-reference/table-functions/hdfsCluster.md +++ b/docs/en/sql-reference/table-functions/hdfsCluster.md @@ -50,7 +50,7 @@ SELECT count(*) FROM hdfsCluster('cluster_simple', 'hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index b49c2f8da20..940fae73b66 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -20,6 +20,6 @@ You can use table functions in: - [INSERT INTO TABLE FUNCTION](../../sql-reference/statements/insert-into.md#inserting-into-table-function) query. -:::warning +:::tip You can’t use table functions if the [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. ::: diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index d7199717798..62b8150de69 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -112,7 +112,7 @@ FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/ └─────────┘ ``` -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index f420a69596c..504f92b4dc0 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -42,7 +42,7 @@ SELECT * FROM s3Cluster( Count the total amount of rows in all files in the cluster `cluster_simple`: -:::warning +:::tip If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: From 46979e383f2f893c18ec2a1ef021c973cf6f0d06 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 9 Mar 2023 18:21:47 +0000 Subject: [PATCH 410/470] Fix big numbers inference in CSV --- src/Formats/SchemaInferenceUtils.cpp | 3 +++ .../0_stateless/02587_csv_big_numbers_inference.reference | 4 ++++ .../queries/0_stateless/02587_csv_big_numbers_inference.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02587_csv_big_numbers_inference.reference create mode 100644 tests/queries/0_stateless/02587_csv_big_numbers_inference.sql diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 00eb686385d..7bd220e529b 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -984,6 +984,9 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting if (tryReadIntText(tmp_int, buf) && buf.eof()) return std::make_shared(); + /// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof. + buf.position() = buf.buffer().begin(); + /// In case of Int64 overflow, try to infer UInt64 UInt64 tmp_uint; if (tryReadIntText(tmp_uint, buf) && buf.eof()) diff --git a/tests/queries/0_stateless/02587_csv_big_numbers_inference.reference b/tests/queries/0_stateless/02587_csv_big_numbers_inference.reference new file mode 100644 index 00000000000..5b38606d1fd --- /dev/null +++ b/tests/queries/0_stateless/02587_csv_big_numbers_inference.reference @@ -0,0 +1,4 @@ +c1 Nullable(Float64) +100000000000000000000 +c1 Nullable(Float64) +-100000000000000000000 diff --git a/tests/queries/0_stateless/02587_csv_big_numbers_inference.sql b/tests/queries/0_stateless/02587_csv_big_numbers_inference.sql new file mode 100644 index 00000000000..45a93034524 --- /dev/null +++ b/tests/queries/0_stateless/02587_csv_big_numbers_inference.sql @@ -0,0 +1,5 @@ +desc format('CSV', '100000000000000000000'); +select * from format('CSV', '100000000000000000000'); +desc format('CSV', '-100000000000000000000'); +select * from format('CSV', '-100000000000000000000'); + From 7cbf77a178d1815e606262a9927fc62178ce782f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 9 Mar 2023 19:55:58 +0100 Subject: [PATCH 411/470] Update LRUFileCachePriority.cpp --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 8010b9c682b..c20379e1fc1 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -34,7 +34,7 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements); - LOG_TRACE(log, "Added entry into LRU queue, key: {}, offset: {}", key.toString(), offset); + LOG_TEST(log, "Added entry into LRU queue, key: {}, offset: {}", key.toString(), offset); return std::make_shared(this, iter); } @@ -54,7 +54,7 @@ void LRUFileCachePriority::removeAll(std::lock_guard &) CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, cache_size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements, queue.size()); - LOG_TRACE(log, "Removed all entries from LRU queue"); + LOG_TEST(log, "Removed all entries from LRU queue"); queue.clear(); cache_size = 0; @@ -88,7 +88,7 @@ void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guar CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, queue_iter->size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements); - LOG_TRACE(cache_priority->log, "Removed entry from LRU queue, key: {}, offset: {}", queue_iter->key.toString(), queue_iter->offset); + LOG_TEST(cache_priority->log, "Removed entry from LRU queue, key: {}, offset: {}", queue_iter->key.toString(), queue_iter->offset); queue_iter = cache_priority->queue.erase(queue_iter); } From de68dade71add8c4351d35ae1f8b35ec34b3fcd1 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Thu, 9 Mar 2023 14:30:40 -0700 Subject: [PATCH 412/470] Revert "standardize admonitions" --- .../database-engines/materialized-mysql.md | 4 +-- .../materialized-postgresql.md | 4 +-- docs/en/engines/database-engines/mysql.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +-- .../table-engines/integrations/kafka.md | 2 +- .../integrations/materialized-postgresql.md | 2 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 4 +-- .../mergetree-family/aggregatingmergetree.md | 2 +- .../mergetree-family/collapsingmergetree.md | 2 +- .../custom-partitioning-key.md | 4 +-- .../mergetree-family/graphitemergetree.md | 6 ++-- .../mergetree-family/invertedindexes.md | 2 +- .../mergetree-family/mergetree.md | 4 +-- .../mergetree-family/replacingmergetree.md | 4 +-- .../mergetree-family/replication.md | 2 +- .../mergetree-family/summingmergetree.md | 2 +- .../versionedcollapsingmergetree.md | 2 +- docs/en/interfaces/formats.md | 28 +++++++++---------- docs/en/interfaces/http.md | 4 +-- docs/en/interfaces/postgresql.md | 2 +- .../third-party/client-libraries.md | 2 +- .../en/interfaces/third-party/integrations.md | 2 +- docs/en/operations/access-rights.md | 4 +-- .../external-authenticators/kerberos.md | 6 ++-- docs/en/operations/opentelemetry.md | 2 +- docs/en/operations/query-cache.md | 2 +- .../settings.md | 4 +-- .../settings/merge-tree-settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings-users.md | 4 +-- docs/en/operations/settings/settings.md | 18 ++++++------ docs/en/operations/storing-data.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/tips.md | 2 +- .../operations/utilities/clickhouse-copier.md | 2 +- .../parametric-functions.md | 4 +-- docs/en/sql-reference/data-types/float.md | 2 +- docs/en/sql-reference/data-types/json.md | 4 +-- .../data-types/special-data-types/interval.md | 2 +- .../external-dicts-dict-layout.md | 4 +-- .../external-dicts-dict-structure.md | 4 +-- docs/en/sql-reference/distributed-ddl.md | 4 +-- .../sql-reference/functions/hash-functions.md | 4 +-- .../sql-reference/functions/introspection.md | 2 +- .../sql-reference/functions/nlp-functions.md | 2 +- .../functions/other-functions.md | 8 +++--- docs/en/sql-reference/operators/exists.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../sql-reference/statements/alter/column.md | 2 +- .../statements/alter/constraint.md | 2 +- .../statements/create/row-policy.md | 2 +- .../sql-reference/statements/create/table.md | 8 +++--- .../sql-reference/statements/create/user.md | 2 +- docs/en/sql-reference/statements/delete.md | 2 +- docs/en/sql-reference/statements/optimize.md | 2 +- docs/en/sql-reference/statements/system.md | 2 +- docs/en/sql-reference/statements/watch.md | 4 +-- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- .../table-functions/hdfsCluster.md | 2 +- .../en/sql-reference/table-functions/index.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- .../table-functions/s3Cluster.md | 2 +- 64 files changed, 113 insertions(+), 113 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 9b2d1d7e1de..899c8d024f1 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -6,7 +6,7 @@ sidebar_position: 70 # [experimental] MaterializedMySQL -:::important +:::warning This is an experimental feature that should not be used in production. ::: @@ -245,7 +245,7 @@ extra care needs to be taken. You may specify overrides for tables that do not exist yet. -:::important +:::warning It is easy to break replication with table overrides if not used with care. For example: * If an ALIAS column is added with a table override, and a column with the same name is later added to the source diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 7be520ee25e..b43f71a7576 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -54,7 +54,7 @@ After `MaterializedPostgreSQL` database is created, it does not automatically de ATTACH TABLE postgres_database.new_table; ``` -:::important +:::warning Before version 22.1, adding a table to replication left an unremoved temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. ::: @@ -145,7 +145,7 @@ FROM pg_class WHERE oid = 'postgres_table'::regclass; ``` -:::important +:::warning Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/database-engines/mysql.md b/docs/en/engines/database-engines/mysql.md index e2c4f134a90..e4ff734d55f 100644 --- a/docs/en/engines/database-engines/mysql.md +++ b/docs/en/engines/database-engines/mysql.md @@ -60,7 +60,7 @@ These variables are supported: - `version` - `max_allowed_packet` -:::important +:::warning By now these variables are stubs and don't correspond to anything. ::: diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index c14d05aa03a..7c04a6594a6 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -64,7 +64,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not recommended. - :::important Zero-copy replication is not ready for production + :::warning Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -110,7 +110,7 @@ Table consists of all the files in both directories (all files should satisfy fo CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV') ``` -:::important +:::warning If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 86647fb6978..255ba06f056 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -102,7 +102,7 @@ Examples: Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/integrations/materialized-postgresql.md b/docs/en/engines/table-engines/integrations/materialized-postgresql.md index c9b3f351568..11e7928c3ed 100644 --- a/docs/en/engines/table-engines/integrations/materialized-postgresql.md +++ b/docs/en/engines/table-engines/integrations/materialized-postgresql.md @@ -52,6 +52,6 @@ PRIMARY KEY key; SELECT key, value, _version FROM postgresql_db.postgresql_replica; ``` -:::important +:::warning Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.html) values is not supported. The default value for the data type will be used. ::: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index d338a2a58bd..b73d28c8508 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -74,7 +74,7 @@ All joins, aggregations, sorting, `IN [ array ]` conditions and the `LIMIT` samp PostgreSQL `Array` types are converted into ClickHouse arrays. -:::important +:::warning Be careful - in PostgreSQL an array data, created like a `type_name[]`, may contain multi-dimensional arrays of different dimensions in different table rows in same column. But in ClickHouse it is only allowed to have multidimensional arrays of the same count of dimensions in all table rows in same column. ::: diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 8e1a4d91cac..723425429a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -63,7 +63,7 @@ For more information about virtual columns see [here](../../../engines/table-eng - Indexes. - [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported. - :::important Zero-copy replication is not ready for production + :::warning Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: @@ -78,7 +78,7 @@ For more information about virtual columns see [here](../../../engines/table-eng Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. -:::important +:::warning If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 9b7f0cd2486..9677f75a358 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -43,7 +43,7 @@ When creating an `AggregatingMergeTree` table the same [clauses](../../../engine Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md index 9acff099557..0bd665116f0 100644 --- a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -45,7 +45,7 @@ When creating a `CollapsingMergeTree` table, the same [query clauses](../../../e Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index f01cd90ccf0..b1e79c4c3fd 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -6,7 +6,7 @@ sidebar_label: Custom Partitioning Key # Custom Partitioning Key -:::important +:::warning In most cases you do not need a partition key, and in most other cases you do not need a partition key more granular than by months. You should never use too granular of partitioning. Don't partition your data by client identifiers or names. Instead, make a client identifier or name the first column in the ORDER BY expression. @@ -159,7 +159,7 @@ FROM session_log GROUP BY UserID; ``` -:::important +:::warning Performance of such a query heavily depends on the table layout. Because of that the optimisation is not enabled by default. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md index d71004135f1..104ec049ec4 100644 --- a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md @@ -55,7 +55,7 @@ When creating a `GraphiteMergeTree` table, the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: @@ -129,7 +129,7 @@ default ... ``` -:::important +:::warning Patterns must be strictly ordered: 1. Patterns without `function` or `retention`. @@ -263,6 +263,6 @@ Valid values: ``` -:::important +:::warning Data rollup is performed during merges. Usually, for old partitions, merges are not started, so for rollup it is necessary to trigger an unscheduled merge using [optimize](../../../sql-reference/statements/optimize.md). Or use additional tools, for example [graphite-ch-optimizer](https://github.com/innogames/graphite-ch-optimizer). ::: diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index 91ee0313a69..aa11258dc4a 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -15,7 +15,7 @@ tokenized cells of the string column. For example, the string cell "I will be a " wi", "wil", "ill", "ll ", "l b", " be" etc. The more fine-granular the input strings are tokenized, the bigger but also the more useful the resulting inverted index will be. -:::important +:::warning Inverted indexes are experimental and should not be used in production environments yet. They may change in the future in backward-incompatible ways, for example with respect to their DDL/DQL syntax or performance/compression characteristics. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 5dbe7602856..fc8060077b0 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -192,7 +192,7 @@ The `index_granularity` setting can be omitted because 8192 is the default value Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects. If possible, switch old projects to the method described above. ::: @@ -1087,7 +1087,7 @@ Other parameters: Examples of working configurations can be found in integration tests directory (see e.g. [test_merge_tree_azure_blob_storage](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml) or [test_azure_blob_storage_zero_copy_replication](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml)). - :::important Zero-copy replication is not ready for production + :::warning Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index 0b76500f2a6..f5d81182898 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -30,7 +30,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md). -:::important +:::warning Uniqueness of rows is determined by the `ORDER BY` table section, not `PRIMARY KEY`. ::: @@ -96,7 +96,7 @@ When creating a `ReplacingMergeTree` table the same [clauses](../../../engines/t Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects and, if possible, switch old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index f9adad0fa6d..37ab8ac9fd3 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -43,7 +43,7 @@ ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/clickhouse-keeper To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. -:::important +:::warning Don’t neglect the security setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index bea37706939..b2b6272c58e 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -44,7 +44,7 @@ When creating a `SummingMergeTree` table the same [clauses](../../../engines/tab Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects and, if possible, switch the old projects to the method described above. ::: diff --git a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index 1a44569711a..2891907f79a 100644 --- a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -58,7 +58,7 @@ When creating a `VersionedCollapsingMergeTree` table, the same [clauses](../../. Deprecated Method for Creating a Table -:::important +:::warning Do not use this method in new projects. If possible, switch old projects to the method described above. ::: diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index fa15de16c71..db2e773a685 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -205,7 +205,7 @@ Differs from the `TabSeparated` format in that the column names are written in t During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness. -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns of the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -217,7 +217,7 @@ This format is also available under the name `TSVWithNames`. Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from the input data will be mapped to the columns in the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -470,7 +470,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -480,7 +480,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -500,7 +500,7 @@ There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [Templat Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -510,7 +510,7 @@ Otherwise, the first row will be skipped. Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -969,7 +969,7 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -979,7 +979,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -991,7 +991,7 @@ the types from input data will be compared with the types of the corresponding c Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1001,7 +1001,7 @@ Otherwise, the first row will be skipped. Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1120,7 +1120,7 @@ CREATE TABLE IF NOT EXISTS example_table - If `input_format_defaults_for_omitted_fields = 0`, then the default value for `x` and `a` equals `0` (as the default value for the `UInt32` data type). - If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`. -:::important +:::warning When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHouse consumes more computational resources, compared to insertion with `input_format_defaults_for_omitted_fields = 0`. ::: @@ -1450,7 +1450,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N) - N `String`s specifying column names -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1464,7 +1464,7 @@ Similar to [RowBinary](#rowbinary), but with added header: - N `String`s specifying column names - N `String`s specifying column types -:::important +:::warning If setting [input_format_with_names_use_header](/docs/en/operations/settings/settings-formats.md/#input_format_with_names_use_header) is set to 1, the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](/docs/en/operations/settings/settings-formats.md/#input_format_skip_unknown_fields) is set to 1. Otherwise, the first row will be skipped. @@ -1912,7 +1912,7 @@ SET format_avro_schema_registry_url = 'http://schema-registry'; SELECT * FROM topic1_stream; ``` -:::important +:::warning Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` to maintain it’s value after a restart. Also you can use the `format_avro_schema_registry_url` setting of the `Kafka` table engine. ::: diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index efa51bb34ea..9af6df0c87d 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -445,7 +445,7 @@ Next are the configuration methods for different `type`. The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_final_threads` settings, then queries the system table to check whether these settings were set successfully. -:::important +:::warning To keep the default `handlers` such as` query`, `play`,` ping`, add the `` rule. ::: @@ -476,7 +476,7 @@ $ curl -H 'XXX:TEST_HEADER_VALUE' -H 'PARAMS_XXX:max_threads' 'http://localhost: max_final_threads 2 ``` -:::important +:::warning In one `predefined_query_handler` only supports one `query` of an insert type. ::: diff --git a/docs/en/interfaces/postgresql.md b/docs/en/interfaces/postgresql.md index 0cc70cce571..9ff83559787 100644 --- a/docs/en/interfaces/postgresql.md +++ b/docs/en/interfaces/postgresql.md @@ -54,7 +54,7 @@ default=> And that's it! You now have a PostgreSQL client connected to ClickHouse, and all commands and queries are executed on ClickHouse. -:::important +:::caution The PostgreSQL protocol currently only supports plain-text passwords. ::: diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 4ce63ba647b..0e065cb7179 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -6,7 +6,7 @@ sidebar_label: Client Libraries # Client Libraries from Third-party Developers -:::important +:::warning ClickHouse Inc does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality. ::: diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 8197a1d1f58..90a4f088be7 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -6,7 +6,7 @@ sidebar_label: Integrations # Integration Libraries from Third-party Developers -:::important Disclaimer +:::warning Disclaimer ClickHouse, Inc. does **not** maintain the tools and libraries listed below and haven’t done extensive testing to ensure their quality. ::: diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md index 38d32e3f1f7..4c4a06dbe1e 100644 --- a/docs/en/operations/access-rights.md +++ b/docs/en/operations/access-rights.md @@ -24,7 +24,7 @@ You can configure access entities using: We recommend using SQL-driven workflow. Both of the configuration methods work simultaneously, so if you use the server configuration files for managing accounts and access rights, you can smoothly switch to SQL-driven workflow. -:::important +:::warning You can’t manage the same access entity by both configuration methods simultaneously. ::: @@ -102,7 +102,7 @@ Privileges can be granted to a role by the [GRANT](../sql-reference/statements/g Row policy is a filter that defines which of the rows are available to a user or a role. Row policy contains filters for one particular table, as well as a list of roles and/or users which should use this row policy. -:::important +:::warning 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. ::: diff --git a/docs/en/operations/external-authenticators/kerberos.md b/docs/en/operations/external-authenticators/kerberos.md index b49291f8e2e..95944e96194 100644 --- a/docs/en/operations/external-authenticators/kerberos.md +++ b/docs/en/operations/external-authenticators/kerberos.md @@ -59,11 +59,11 @@ With filtering by realm: ``` -:::important +:::warning You can define only one `kerberos` section. The presence of multiple `kerberos` sections will force ClickHouse to disable Kerberos authentication. ::: -:::important +:::warning `principal` and `realm` sections cannot be specified at the same time. The presence of both `principal` and `realm` sections will force ClickHouse to disable Kerberos authentication. ::: @@ -103,7 +103,7 @@ Example (goes into `users.xml`): ``` -:::important +:::warning Note that Kerberos authentication cannot be used alongside with any other authentication mechanism. The presence of any other sections like `password` alongside `kerberos` will force ClickHouse to shutdown. ::: diff --git a/docs/en/operations/opentelemetry.md b/docs/en/operations/opentelemetry.md index 7faa4b1ee7c..1de5a09db0c 100644 --- a/docs/en/operations/opentelemetry.md +++ b/docs/en/operations/opentelemetry.md @@ -7,7 +7,7 @@ title: "[experimental] Tracing ClickHouse with OpenTelemetry" [OpenTelemetry](https://opentelemetry.io/) is an open standard for collecting traces and metrics from the distributed application. ClickHouse has some support for OpenTelemetry. -:::important +:::warning This is an experimental feature that will change in backwards-incompatible ways in future releases. ::: diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index a7ac011c796..1a486de7904 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -29,7 +29,7 @@ Transactionally inconsistent caching is traditionally provided by client tools o the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side. This reduces maintenance effort and avoids redundancy. -:::important +:::warning The query cache is an experimental feature that should not be used in production. There are known cases (e.g. in distributed query processing) where wrong results are returned. ::: diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 4494ad39a0e..3fe815bc79a 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -24,7 +24,7 @@ Default value: 3600. Data compression settings for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)-engine tables. -:::tip +:::warning Don’t use it if you have just started using ClickHouse. ::: @@ -1367,7 +1367,7 @@ The following settings are available: Changed settings take effect immediately. -:::important +:::warning Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query cache altogether. ::: diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 6290a23378c..5bc174727ad 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -289,7 +289,7 @@ Default value: 0 (seconds) When this setting has a value greater than than zero only a single replica starts the merge immediately if merged part on shared storage and `allow_remote_fs_zero_copy_replication` is enabled. -:::important Zero-copy replication is not ready for production +:::warning Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index b03f922d0d4..919ebaf562f 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -142,7 +142,7 @@ y Nullable(String) z IPv4 ``` -:::important +:::warning If the `schema_inference_hints` is not formated properly, or if there is a typo or a wrong datatype, etc... the whole schema_inference_hints will be ignored. ::: diff --git a/docs/en/operations/settings/settings-users.md b/docs/en/operations/settings/settings-users.md index ac0025a1137..b55d64fc4f7 100644 --- a/docs/en/operations/settings/settings-users.md +++ b/docs/en/operations/settings/settings-users.md @@ -118,8 +118,8 @@ To open access for user from any network, specify: ::/0 ``` -:::important -It is insecure to open access from any network unless you have a firewall properly configured or the server is not directly connected to Internet. +:::warning +It’s insecure to open access from any network unless you have a firewall properly configured or the server is not directly connected to Internet. ::: To open access only from localhost, specify: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 809c7c3e837..94dcf159ca9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -460,7 +460,7 @@ Possible values: Changes the behaviour of join operations with `ANY` strictness. -:::important +:::warning This setting applies only for `JOIN` operations with [Join](../../engines/table-engines/special/join.md) engine tables. ::: @@ -550,7 +550,7 @@ Default value: 64. Enables legacy ClickHouse server behaviour in `ANY INNER|LEFT JOIN` operations. -:::important +:::warning Use this setting only for backward compatibility if your use cases depend on legacy `JOIN` behaviour. ::: @@ -942,7 +942,7 @@ Higher values will lead to higher memory usage. The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). Specifying a smaller block size generally leads to slightly reduced compression ratio, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. -:::important +:::warning This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -960,7 +960,7 @@ We are writing a UInt32-type column (4 bytes per value). When writing 8192 rows, We are writing a URL column with the String type (average size of 60 bytes per value). When writing 8192 rows, the average will be slightly less than 500 KB of data. Since this is more than 65,536, a compressed block will be formed for each mark. In this case, when reading data from the disk in the range of a single mark, extra data won’t be decompressed. -:::important +:::warning This is an expert-level setting, and you shouldn't change it if you're just getting started with ClickHouse. ::: @@ -1247,7 +1247,7 @@ Possible values: Default value: 1. -:::important +:::warning Disable this setting if you use [max_parallel_replicas](#settings-max_parallel_replicas) without [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key). If [parallel_replicas_custom_key](#settings-parallel_replicas_custom_key) is set, disable this setting only if it's used on a cluster with multiple shards containing multiple replicas. If it's used on a cluster with a single shard and multiple replicas, disabling this setting will have negative effects. @@ -1277,7 +1277,7 @@ Default value: `1`. This options will produce different results depending on the settings used. -:::important +:::warning This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: @@ -2186,7 +2186,7 @@ Default value: 0. This setting also affects broken batches (that may appears because of abnormal server (machine) termination and no `fsync_after_insert`/`fsync_directories` for [Distributed](../../engines/table-engines/special/distributed.md) table engine). ::: -:::important +:::warning You should not rely on automatic batch splitting, since this may hurt performance. ::: @@ -2194,7 +2194,7 @@ You should not rely on automatic batch splitting, since this may hurt performanc Sets the priority ([nice](https://en.wikipedia.org/wiki/Nice_(Unix))) for threads that execute queries. The OS scheduler considers this priority when choosing the next thread to run on each available CPU core. -:::important +:::warning To use this setting, you need to set the `CAP_SYS_NICE` capability. The `clickhouse-server` package sets it up during installation. Some virtual environments do not allow you to set the `CAP_SYS_NICE` capability. In this case, `clickhouse-server` shows a message about it at the start. ::: @@ -2858,7 +2858,7 @@ Possible values: Default value: `0`. -:::important +:::warning Nullable primary key usually indicates bad design. It is forbidden in almost all main stream DBMS. The feature is mainly for [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) and is not heavily tested. Use with care. ::: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 21c3bf84250..3f9a0f67187 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -471,6 +471,6 @@ Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#htt Zero-copy replication is possible, but not recommended, with `S3` and `HDFS` disks. Zero-copy replication means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. -:::important Zero-copy replication is not ready for production +:::warning Zero-copy replication is not ready for production Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use. ::: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index c9477214fd6..106d3c59dea 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -99,7 +99,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -:::important +:::warning The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 00e7155edc1..da34a6b7e9c 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -36,7 +36,7 @@ $ echo 0 | sudo tee /proc/sys/vm/overcommit_memory Use `perf top` to watch the time spent in the kernel for memory management. Permanent huge pages also do not need to be allocated. -:::important +:::warning If your system has less than 16 GB of RAM, you may experience various memory exceptions because default settings do not match this amount of memory. The recommended amount of RAM is 32 GB or more. You can use ClickHouse in a system with a small amount of RAM, even with 2 GB of RAM, but it requires additional tuning and can ingest at a low rate. ::: diff --git a/docs/en/operations/utilities/clickhouse-copier.md b/docs/en/operations/utilities/clickhouse-copier.md index 1b75530dda7..87280bc3ba8 100644 --- a/docs/en/operations/utilities/clickhouse-copier.md +++ b/docs/en/operations/utilities/clickhouse-copier.md @@ -8,7 +8,7 @@ sidebar_label: clickhouse-copier Copies data from the tables in one cluster to tables in another (or the same) cluster. -:::important +:::warning To get a consistent copy, the data in the source tables and partitions should not change during the entire process. ::: diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 0bd43034c1b..40184c0aa02 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -90,7 +90,7 @@ Checks whether the sequence contains an event chain that matches the pattern. sequenceMatch(pattern)(timestamp, cond1, cond2, ...) ``` -:::important +:::warning Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: @@ -176,7 +176,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. -:::important +:::warning Events that occur at the same second may lay in the sequence in an undefined order affecting the result. ::: diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 7a5fa088f6e..38c414fa8cd 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -6,7 +6,7 @@ sidebar_label: Float32, Float64 # Float32, Float64 -:::important +:::warning If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: ``` diff --git a/docs/en/sql-reference/data-types/json.md b/docs/en/sql-reference/data-types/json.md index 29d6553b888..d9099ba5ad3 100644 --- a/docs/en/sql-reference/data-types/json.md +++ b/docs/en/sql-reference/data-types/json.md @@ -6,7 +6,7 @@ sidebar_label: JSON # JSON -:::important +:::warning This feature is experimental and is not production ready. If you need to work with JSON documents, consider using [this guide](/docs/en/guides/developer/working-with-json/json-load-data.md) instead. ::: @@ -14,7 +14,7 @@ Stores JavaScript Object Notation (JSON) documents in a single column. `JSON` is an alias for `Object('json')`. -:::important +:::warning The JSON data type is an experimental feature. To use it, set `allow_experimental_object_type = 1`. ::: 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 68494e52360..5169bc646c9 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 @@ -8,7 +8,7 @@ sidebar_label: Interval The family of data types representing time and date intervals. The resulting types of the [INTERVAL](../../../sql-reference/operators/index.md#operator-interval) operator. -:::important +:::warning `Interval` data type values can’t be stored in tables. ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 7af5923e052..4dc6fd33849 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -299,7 +299,7 @@ Example: The table contains discounts for each advertiser in the format: To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). These elements must contain elements `name` and `type` (if `type` is not specified, the default type will be used - Date). `type` can be any numeric type (Date / DateTime / UInt64 / Int32 / others). -:::important +:::warning Values of `range_min` and `range_max` should fit in `Int64` type. ::: @@ -588,7 +588,7 @@ Set a large enough cache size. You need to experiment to select the number of ce 3. Assess memory consumption using the `system.dictionaries` table. 4. Increase or decrease the number of cells until the required memory consumption is reached. -:::important +:::warning Do not use ClickHouse as a source, because it is slow to process queries with random reads. ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 06d1c817a13..8271a342941 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -60,7 +60,7 @@ ClickHouse supports the following types of keys: An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. -:::important +:::warning You must not describe key as an attribute. ::: @@ -178,4 +178,4 @@ Configuration fields: ## Related Content -- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) +- [Using dictionaries to accelerate queries](https://clickhouse.com/blog/faster-queries-dictionaries-clickhouse) \ No newline at end of file diff --git a/docs/en/sql-reference/distributed-ddl.md b/docs/en/sql-reference/distributed-ddl.md index d170f3765c2..ff5155391be 100644 --- a/docs/en/sql-reference/distributed-ddl.md +++ b/docs/en/sql-reference/distributed-ddl.md @@ -18,6 +18,6 @@ In order to run these queries correctly, each host must have the same cluster de The local version of the query will eventually be executed on each host in the cluster, even if some hosts are currently not available. -:::important +:::warning The order for executing queries within a single host is guaranteed. -::: +::: \ No newline at end of file diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 833e0a40b2b..69dc73e2fb0 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -125,7 +125,7 @@ SELECT sipHash64Keyed((506097522914230528, 1084818905618843912), array('e','x',' Like [sipHash64](#hash_functions-siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. -:::important +:::warning This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128Reference](#hash_functions-siphash128reference). @@ -165,7 +165,7 @@ Result: Same as [sipHash128](#hash_functions-siphash128) but additionally takes an explicit key argument instead of using a fixed key. -:::important +:::warning This 128-bit variant differs from the reference implementation and it's weaker. This version exists because, when it was written, there was no official 128-bit extension for SipHash. New projects should probably use [sipHash128ReferenceKeyed](#hash_functions-siphash128referencekeyed). diff --git a/docs/en/sql-reference/functions/introspection.md b/docs/en/sql-reference/functions/introspection.md index 78f763318df..9357f75b8e6 100644 --- a/docs/en/sql-reference/functions/introspection.md +++ b/docs/en/sql-reference/functions/introspection.md @@ -8,7 +8,7 @@ sidebar_label: Introspection You can use functions described in this chapter to introspect [ELF](https://en.wikipedia.org/wiki/Executable_and_Linkable_Format) and [DWARF](https://en.wikipedia.org/wiki/DWARF) for query profiling. -:::important +:::warning These functions are slow and may impose security considerations. ::: diff --git a/docs/en/sql-reference/functions/nlp-functions.md b/docs/en/sql-reference/functions/nlp-functions.md index 62116fe50a3..f68448af2be 100644 --- a/docs/en/sql-reference/functions/nlp-functions.md +++ b/docs/en/sql-reference/functions/nlp-functions.md @@ -5,7 +5,7 @@ sidebar_label: NLP title: "[experimental] Natural Language Processing functions" --- -:::important +:::warning This is an experimental feature that is currently in development and is not ready for general use. It will change in unpredictable backwards-incompatible ways in future releases. Set `allow_experimental_nlp_functions = 1` to enable it. ::: diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 0f95f16ffed..7146484361e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -792,7 +792,7 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::important +:::warning It can reach the neighbor rows only inside the currently processed data block. ::: @@ -902,7 +902,7 @@ Result: Calculates the difference between successive row values ​​in the data block. Returns 0 for the first row and the difference from the previous row for each subsequent row. -:::important +:::warning It can reach the previous row only inside the currently processed data block. ::: @@ -986,7 +986,7 @@ Each event has a start time and an end time. The start time is included in the e The function calculates the total number of active (concurrent) events for each event start time. -:::important +:::warning Events must be ordered by the start time in ascending order. If this requirement is violated the function raises an exception. Every data block is processed separately. If events from different data blocks overlap then they can not be processed correctly. ::: @@ -1674,7 +1674,7 @@ Result: Accumulates states of an aggregate function for each row of a data block. -:::important +:::warning The state is reset for each new data block. ::: diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 6819048e28f..4bc29389c9c 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -7,7 +7,7 @@ The `EXISTS` operator checks how many records are in the result of a subquery. I `EXISTS` can be used in a [WHERE](../../sql-reference/statements/select/where.md) clause. -:::important +:::warning References to main query tables and columns are not supported in a subquery. ::: diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 98dee830cd4..0fe7ebbf4b6 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -229,7 +229,7 @@ Types of intervals: You can also use a string literal when setting the `INTERVAL` value. For example, `INTERVAL 1 HOUR` is identical to the `INTERVAL '1 hour'` or `INTERVAL '1' hour`. -:::important +:::warning Intervals with different types can’t be combined. You can’t use expressions like `INTERVAL 4 DAY 1 HOUR`. Specify intervals in units that are smaller or equal to the smallest unit of the interval, for example, `INTERVAL 25 HOUR`. You can use consecutive operations, like in the example below. ::: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 99958c31b92..d580efa4992 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -75,7 +75,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -:::important +:::warning You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: diff --git a/docs/en/sql-reference/statements/alter/constraint.md b/docs/en/sql-reference/statements/alter/constraint.md index 0b19966798b..844b24d7374 100644 --- a/docs/en/sql-reference/statements/alter/constraint.md +++ b/docs/en/sql-reference/statements/alter/constraint.md @@ -17,7 +17,7 @@ See more on [constraints](../../../sql-reference/statements/create/table.md#cons Queries will add or remove metadata about constraints from table so they are processed immediately. -:::important +:::warning Constraint check **will not be executed** on existing data if it was added. ::: diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 93bc5729c8e..31ce9221eea 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -7,7 +7,7 @@ title: "CREATE ROW POLICY" Creates a [row policy](../../../operations/access-rights.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table. -:::important +:::warning 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. ::: diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 7da1c9669ff..50e74920e4b 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -286,7 +286,7 @@ ENGINE = engine PRIMARY KEY(expr1[, expr2,...]); ``` -:::important +:::warning You can't combine both ways in one query. ::: @@ -342,7 +342,7 @@ ALTER TABLE codec_example MODIFY COLUMN float_value CODEC(Default); Codecs can be combined in a pipeline, for example, `CODEC(Delta, Default)`. -:::important +:::warning You can’t decompress ClickHouse database files with external utilities like `lz4`. Instead, use the special [clickhouse-compressor](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utility. ::: @@ -437,11 +437,11 @@ Encryption codecs: These codecs use a fixed nonce and encryption is therefore deterministic. This makes it compatible with deduplicating engines such as [ReplicatedMergeTree](../../../engines/table-engines/mergetree-family/replication.md) but has a weakness: when the same data block is encrypted twice, the resulting ciphertext will be exactly the same so an adversary who can read the disk can see this equivalence (although only the equivalence, without getting its content). -:::important +:::warning Most engines including the "\*MergeTree" family create index files on disk without applying codecs. This means plaintext will appear on disk if an encrypted column is indexed. ::: -:::important +:::warning If you perform a SELECT query mentioning a specific value in an encrypted column (such as in its WHERE clause), the value may appear in [system.query_log](../../../operations/system-tables/query_log.md). You may want to disable the logging. ::: diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index e2cf195ebd6..a756b3d4a0d 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -55,7 +55,7 @@ Another way of specifying host is to use `@` syntax following the username. Exam - `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. - `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. -:::important +:::warning ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we do not recommend to do so. ::: diff --git a/docs/en/sql-reference/statements/delete.md b/docs/en/sql-reference/statements/delete.md index e5a0a8f5b2b..e1987e50af4 100644 --- a/docs/en/sql-reference/statements/delete.md +++ b/docs/en/sql-reference/statements/delete.md @@ -32,7 +32,7 @@ SET allow_experimental_lightweight_delete = true; An [alternative way to delete rows](./alter/delete.md) in ClickHouse is `ALTER TABLE ... DELETE`, which might be more efficient if you do bulk deletes only occasionally and don't need the operation to be applied instantly. In most use cases the new lightweight `DELETE FROM` behavior will be considerably faster. -:::important +:::warning Even though deletes are becoming more lightweight in ClickHouse, they should still not be used as aggressively as on an OLTP system. Lightweight deletes are currently efficient for wide parts, but for compact parts, they can be a heavyweight operation, and it may be better to use `ALTER TABLE` for some scenarios. ::: diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index ee5ad512dd5..78615a2f9ad 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -7,7 +7,7 @@ title: "OPTIMIZE Statement" This query tries to initialize an unscheduled merge of data parts for tables. -:::important +:::warning `OPTIMIZE` can’t fix the `Too many parts` error. ::: diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index a7fd0944593..f9f55acfcec 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -312,7 +312,7 @@ One may execute query after: Replica attaches locally found parts and sends info about them to Zookeeper. Parts present on a replica before metadata loss are not re-fetched from other ones if not being outdated (so replica restoration does not mean re-downloading all data over the network). -:::important +:::warning Parts in all states are moved to `detached/` folder. Parts active before data loss (committed) are attached. ::: diff --git a/docs/en/sql-reference/statements/watch.md b/docs/en/sql-reference/statements/watch.md index e4220be60e9..90d19e6be0e 100644 --- a/docs/en/sql-reference/statements/watch.md +++ b/docs/en/sql-reference/statements/watch.md @@ -6,7 +6,7 @@ sidebar_label: WATCH # WATCH Statement (Experimental) -:::important +:::warning This is an experimental feature that may change in backwards-incompatible ways in the future releases. Enable live views and `WATCH` query using `set allow_experimental_live_view = 1`. ::: @@ -107,4 +107,4 @@ The `FORMAT` clause works the same way as for the [SELECT](../../sql-reference/s :::note The [JSONEachRowWithProgress](../../interfaces/formats.md#jsoneachrowwithprogress) format should be used when watching [LIVE VIEW](./create/view.md#live-view) tables over the HTTP interface. The progress messages will be added to the output to keep the long-lived HTTP connection alive until the query result changes. The interval between progress messages is controlled using the [live_view_heartbeat_interval](./create/view.md#live-view-settings) setting. -::: +::: \ No newline at end of file diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index bf0fca648a3..594c328c3ff 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -109,7 +109,7 @@ Query the number of rows in all files of these two directories: SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` -:::tip +:::warning If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 784e45d0f81..97a253a5356 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -79,7 +79,7 @@ SELECT count(*) FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::tip +:::warning If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/hdfsCluster.md b/docs/en/sql-reference/table-functions/hdfsCluster.md index 78f56372310..231c552610f 100644 --- a/docs/en/sql-reference/table-functions/hdfsCluster.md +++ b/docs/en/sql-reference/table-functions/hdfsCluster.md @@ -50,7 +50,7 @@ SELECT count(*) FROM hdfsCluster('cluster_simple', 'hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value UInt32') ``` -:::tip +:::warning If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index 940fae73b66..b49c2f8da20 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -20,6 +20,6 @@ You can use table functions in: - [INSERT INTO TABLE FUNCTION](../../sql-reference/statements/insert-into.md#inserting-into-table-function) query. -:::tip +:::warning You can’t use table functions if the [allow_ddl](../../operations/settings/permissions-for-queries.md#settings_allow_ddl) setting is disabled. ::: diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 62b8150de69..d7199717798 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -112,7 +112,7 @@ FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/ └─────────┘ ``` -:::tip +:::warning If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: diff --git a/docs/en/sql-reference/table-functions/s3Cluster.md b/docs/en/sql-reference/table-functions/s3Cluster.md index 504f92b4dc0..f420a69596c 100644 --- a/docs/en/sql-reference/table-functions/s3Cluster.md +++ b/docs/en/sql-reference/table-functions/s3Cluster.md @@ -42,7 +42,7 @@ SELECT * FROM s3Cluster( Count the total amount of rows in all files in the cluster `cluster_simple`: -:::tip +:::warning If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: From 6f35d46ac8d9ee3484357015033e35c3912fff89 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 10 Mar 2023 05:01:06 +0300 Subject: [PATCH 413/470] Update SchemaInferenceUtils.cpp --- src/Formats/SchemaInferenceUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 7bd220e529b..011860948c3 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -984,7 +984,7 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting if (tryReadIntText(tmp_int, buf) && buf.eof()) return std::make_shared(); - /// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof. + /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. buf.position() = buf.buffer().begin(); /// In case of Int64 overflow, try to infer UInt64 @@ -993,7 +993,7 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting return std::make_shared(); } - /// We cam safely get back to the start of buffer, because we read from a string and we didn't reach eof. + /// We can safely get back to the start of buffer, because we read from a string and we didn't reach eof. buf.position() = buf.buffer().begin(); Float64 tmp; From 40eed939cca48687ce2fa13b721a822b908b0c0c Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 10 Mar 2023 10:07:53 +0100 Subject: [PATCH 414/470] Update README.md Slacks invite links are unpleasant. We now redirect so we don't have to constantly change. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index fcbe65e8223..17b4df154a9 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,7 @@ curl https://clickhouse.com/ | sh * [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query a small ClickHouse cluster. * [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information. * [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format. -* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-1gh9ds7f4-PgDhJAaF8ad5RbWBAAjzFg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. +* [Slack](https://clickhouse.com/slack) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events. * [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation. * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev. From b17ba66353b1325781f5811c2d956c8370163b78 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Fri, 10 Mar 2023 14:25:54 +0300 Subject: [PATCH 415/470] Be more explicit about functional tests output location --- docs/en/development/tests.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 9ae49e8f707..8402206c794 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -31,6 +31,9 @@ folder and run the following command: PATH=$PATH: tests/clickhouse-test 01428_hash_set_nan_key ``` +Test results (`stderr` and `stdout`) are written to files `01428_hash_set_nan_key.[stderr|stdout]` which +are located near the test file itself (so for `queries/0_stateless/foo.sql` output will be in `queries/0_stateless/foo.stdout`). + For more options, see `tests/clickhouse-test --help`. You can simply run all tests or run subset of tests filtered by substring in test name: `./clickhouse-test substring`. There are also options to run tests in parallel or in randomized order. ### Adding a New Test From e3d71f4f1be8a7e66d7212b5809afe74fe73697d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Mar 2023 14:36:05 +0100 Subject: [PATCH 416/470] suppress hung check with ubsan --- tests/ci/stress.py | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 12c40ea1f66..d571240a5a2 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -223,6 +223,19 @@ def prepare_for_hung_check(drop_databases): pass return True +def is_ubsan_build(): + try: + query = """clickhouse client -q "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'" """ + output = ( + check_output(query, shell=True, stderr=STDOUT, timeout=30) + .decode("utf-8") + .strip() + ) + return b"-fsanitize=undefined" in output + except Exception as e: + logging.info("Failed to get build flags: ", str(e)) + return False + if __name__ == "__main__": logging.basicConfig(level=logging.INFO, format="%(asctime)s %(message)s") @@ -243,6 +256,10 @@ if __name__ == "__main__": args = parser.parse_args() if args.drop_databases and not args.hung_check: raise Exception("--drop-databases only used in hung check (--hung-check)") + + # FIXME Hung check with ubsan is temporarily disabled due to https://github.com/ClickHouse/ClickHouse/issues/45372 + suppress_hung_check = is_ubsan_build() + func_pipes = [] func_pipes = run_func_test( args.test_cmd, @@ -307,7 +324,7 @@ if __name__ == "__main__": res = call(cmd, shell=True, stdout=tee.stdin, stderr=STDOUT) if tee.stdin is not None: tee.stdin.close() - if res != 0 and have_long_running_queries: + if res != 0 and have_long_running_queries and not suppress_hung_check: logging.info("Hung check failed with exit code %d", res) else: hung_check_status = "No queries hung\tOK\t\\N\t\n" From ce3ab32d897ed605d106286ded9cd528f4bd191b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 10 Mar 2023 13:53:47 +0000 Subject: [PATCH 417/470] Automatic style fix --- tests/ci/stress.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index d571240a5a2..6c7118a9335 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -223,6 +223,7 @@ def prepare_for_hung_check(drop_databases): pass return True + def is_ubsan_build(): try: query = """clickhouse client -q "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'" """ From 3c5cb817d7def8834840acae78120539c53c36e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Mar 2023 17:04:38 +0300 Subject: [PATCH 418/470] Update stress.py --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 6c7118a9335..881b45e4a78 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -234,7 +234,7 @@ def is_ubsan_build(): ) return b"-fsanitize=undefined" in output except Exception as e: - logging.info("Failed to get build flags: ", str(e)) + logging.info("Failed to get build flags: " + str(e)) return False From d95b69ca768316a895be7feaecd7fccbfce0124f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Mar 2023 17:22:09 +0300 Subject: [PATCH 419/470] Update stress.py --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 881b45e4a78..8ced7f2e3bc 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -234,7 +234,7 @@ def is_ubsan_build(): ) return b"-fsanitize=undefined" in output except Exception as e: - logging.info("Failed to get build flags: " + str(e)) + logging.info("Failed to get build flags: %s", str(e)) return False From bb8f0cb3ab248fe242797bbbae49d7c434c76436 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Mar 2023 18:09:32 +0300 Subject: [PATCH 420/470] Update stress.py --- tests/ci/stress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 8ced7f2e3bc..37b3b1c181e 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -232,7 +232,7 @@ def is_ubsan_build(): .decode("utf-8") .strip() ) - return b"-fsanitize=undefined" in output + return "-fsanitize=undefined" in output except Exception as e: logging.info("Failed to get build flags: %s", str(e)) return False From d20cbfe05192cfbef9adb332943cc2dcdf98ff37 Mon Sep 17 00:00:00 2001 From: kolechenkov <60389882+kolechenkov@users.noreply.github.com> Date: Fri, 10 Mar 2023 19:00:02 +0300 Subject: [PATCH 421/470] Update gui.md Update information about Yandex DataLens --- docs/ru/interfaces/third-party/gui.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index c34b799347b..e7190362dc4 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -177,19 +177,20 @@ sidebar_label: "Визуальные интерфейсы от сторонни ### Yandex DataLens {#yandex-datalens} -[Yandex DataLens](https://cloud.yandex.ru/services/datalens) — cервис визуализации и анализа данных. +[Yandex DataLens](https://datalens.yandex.ru) — cервис визуализации и анализа данных. Основные возможности: - Широкий выбор инструментов визуализации, от простых столбчатых диаграмм до сложных дашбордов. - Возможность опубликовать дашборды на широкую аудиторию. - Поддержка множества источников данных, включая ClickHouse. -- Хранение материализованных данных в кластере ClickHouse DataLens. -Для небольших проектов DataLens [доступен бесплатно](https://cloud.yandex.ru/docs/datalens/pricing), в том числе и для коммерческого использования. +DataLens [доступен бесплатно](https://cloud.yandex.ru/docs/datalens/pricing), в том числе и для коммерческого использования. +- [Знакомство с DataLens]((https://youtu.be/57ngi_6BINE). +- [Чат сообщества DataLens](https://t.me/YandexDataLens) - [Документация DataLens](https://cloud.yandex.ru/docs/datalens/). -- [Пособие по визуализации данных из ClickHouse](https://cloud.yandex.ru/docs/solutions/datalens/data-from-ch-visualization). +- [Сценарий по визуализации данных из ClickHouse](https://cloud.yandex.ru/docs/solutions/datalens/data-from-ch-visualization). ### Holistics Software {#holistics-software} From 9bcd93d959cfe0a64517a82fe0a9530ccc224b1b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 10 Mar 2023 17:47:42 +0100 Subject: [PATCH 422/470] Update InterpreterInsertQuery.cpp --- src/Interpreters/InterpreterInsertQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 3cae219fa60..b4a19ea7403 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -161,7 +161,7 @@ Block InterpreterInsertQuery::getSampleBlock( if (table_sample_physical.has(current_name)) { if (!allow_materialized) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is a MATERIALIZED column", + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", current_name); res.insert(ColumnWithTypeAndName(table_sample_physical.getByName(current_name).type, current_name)); } @@ -527,7 +527,7 @@ BlockIO InterpreterInsertQuery::execute() { for (const auto & column : metadata_snapshot->getColumns()) if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is a MATERIALIZED column", column.name); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); } res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); From 363989f3f86ac96662624cf2272d5875b721233b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 10 Mar 2023 17:56:29 +0000 Subject: [PATCH 423/470] Update version_date.tsv and changelogs after v23.2.4.12-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.2.4.12-stable.md | 20 ++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 24 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.2.4.12-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 500249b5bd6..34c1406b687 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -29,7 +29,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.2.3.17" +ARG VERSION="23.2.4.12" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 5f613eea4d0..f4ca498a7e7 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.2.3.17" +ARG VERSION="23.2.4.12" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 3e99bca8bc2..13b3ebdb01c 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.2.3.17" +ARG VERSION="23.2.4.12" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.2.4.12-stable.md b/docs/changelogs/v23.2.4.12-stable.md new file mode 100644 index 00000000000..2b6a689aee5 --- /dev/null +++ b/docs/changelogs/v23.2.4.12-stable.md @@ -0,0 +1,20 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.2.4.12-stable (8fe866cb035) FIXME as compared to v23.2.3.17-stable (dec18bf7281) + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#47277](https://github.com/ClickHouse/ClickHouse/issues/47277): Fix IPv4/IPv6 serialization/deserialization in binary formats that was broken in https://github.com/ClickHouse/ClickHouse/pull/43221. Closes [#46522](https://github.com/ClickHouse/ClickHouse/issues/46522). [#46616](https://github.com/ClickHouse/ClickHouse/pull/46616) ([Kruglov Pavel](https://github.com/Avogar)). +* Backported in [#47212](https://github.com/ClickHouse/ClickHouse/issues/47212): `INSERT` queries through native TCP protocol and HTTP protocol were not canceled correctly in some cases. It could lead to a partially applied query if a client canceled the query, or if a client died or, in rare cases, on network errors. As a result, it could lead to not working deduplication. Fixes [#27667](https://github.com/ClickHouse/ClickHouse/issues/27667) and [#45377](https://github.com/ClickHouse/ClickHouse/issues/45377). [#46681](https://github.com/ClickHouse/ClickHouse/pull/46681) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#47363](https://github.com/ClickHouse/ClickHouse/issues/47363): Fix possible deadlock on distributed query cancellation. [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 34bc3f646fc..50a9314bc00 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.2.4.12-stable 2023-03-10 v23.2.3.17-stable 2023-03-06 v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 From dc0858475d105d8b6a3561d972a60ef5637dabd8 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 10 Mar 2023 10:57:41 -0500 Subject: [PATCH 424/470] Add s390x cross build instructions --- docs/en/development/build-cross-s390x.md | 123 +++++++++++++++++++++++ docs/en/development/contrib.md | 2 +- docs/en/development/style.md | 2 +- docs/en/development/tests.md | 2 +- 4 files changed, 126 insertions(+), 3 deletions(-) create mode 100644 docs/en/development/build-cross-s390x.md diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md new file mode 100644 index 00000000000..21cc563ff06 --- /dev/null +++ b/docs/en/development/build-cross-s390x.md @@ -0,0 +1,123 @@ +--- +slug: /en/development/build-cross-s390x +sidebar_position: 69 +title: How to Build, Run and Debug ClickHouse on Linux for s390x (zLinux) +sidebar_label: Build on Linux for s390x (zLinux) +--- + +As of writing (2023/3/10) building for s390x considered to be experimental. Not all features can be enabled, has broken features and is currently under active development. + + +## Building + +As s390x does not support boringSSL, it uses openSSL and has two related build options. +- By default, the s390x build will create dynamiclly link to openSSL libraries. It will build openSSL, so its not neccessary to install openSSL beforehand. This is recommended in all cases. +- Another option is to build openSSL in-tree. In this case two build options need to be supplied to cmake +```bash +-DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1 +``` + +These instructions assume that the host machine is x86_64 and has all the tooling required to build natively base based on the [Build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but should work on Ubuntu 20.04. + +In addition to installing the tooling used to build natively the following additional pacakges need to be installed: + +```bash +apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-gnu binfmt-support qemu-user-static +``` + +If you wish to cross compile rust code.. +```bash +rustup target add s390x-unknown-linux-gnu +``` + +To build for s390x, +```bash +cmake -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-s390x.cmake .. +ninja +``` + +## Running + +Once built, the binary can be run with: + +```bash +qemu-s390x-static -L /usr/s390x-linux-gnu ./clickhouse +``` + +## Debugging + +Install lldb: + +```bash +apt-get install lldb-15 +``` + +To Debug a s390x executable run clickhouse using qemu debug mode: + +```bash +qemu-s390x-static -g 31338 -L /usr/s390x-linux-gnu ./clickhouse +``` + +In antother shell run lldb and attach, replace `` and `` with the values corresponding to your environment. +```bash +lldb-15 +(lldb) target create ./clickhouse +Current executable set to '//ClickHouse//programs/clickhouse' (s390x). +(lldb) settings set target.source-map //ClickHouse +(lldb) gdb-remote 31338 +Process 1 stopped +* thread #1, stop reason = signal SIGTRAP + frame #0: 0x0000004020e74cd0 +-> 0x4020e74cd0: lgr %r2, %r15 + 0x4020e74cd4: aghi %r15, -160 + 0x4020e74cd8: xc 0(8,%r15), 0(%r15) + 0x4020e74cde: brasl %r14, 275429939040 +(lldb) b main +Breakpoint 1: 9 locations. +(lldb) c +Process 1 resuming +Process 1 stopped +* thread #1, stop reason = breakpoint 1.1 + frame #0: 0x0000004005cd9fc0 clickhouse`main(argc_=1, argv_=0x0000004020e594a8) at main.cpp:450:17 + 447 #if !defined(FUZZING_MODE) + 448 int main(int argc_, char ** argv_) + 449 { +-> 450 inside_main = true; + 451 SCOPE_EXIT({ inside_main = false; }); + 452 + 453 /// PHDR cache is required for query profiler to work reliably +``` + +## Visual Studio Code integration + +- (CodeLLDB extension)[https://github.com/vadimcn/vscode-lldb] is required for visual debugging, (Command Variable)[https://github.com/rioj7/command-variable] extension can help dynamic lauches if using cmake variants. +- Make sure to set the backend to your llvm installation ex. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"` +- Launcher: +```json +{ + "version": "0.2.0", + "configurations": [ + { + "name": "Debug", + "type": "lldb", + "request": "custom", + "targetCreateCommands": ["target create ${command:cmake.launchTargetDirectory}/clickhouse"], + "processCreateCommands": ["settings set target.source-map ${input:targetdir} ${workspaceFolder}", "gdb-remote 31338"], + "sourceMap": { "${input:targetdir}": "${workspaceFolder}" }, + } + ], + "inputs": [ + { + "id": "targetdir", + "type": "command", + "command": "extension.commandvariable.transform", + "args": { + "text": "${command:cmake.launchTargetDirectory}", + "find": ".*/([^/]+)/[^/]+$", + "replace": "$1" + } + } + ] +} +``` +- Make sure to run the clickhouse executable in debug mode prior to launch. (It is possible to create a preLaunchTask that automates this) \ No newline at end of file diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 04158a0c3f7..4b296c43db4 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -1,6 +1,6 @@ --- slug: /en/development/contrib -sidebar_position: 71 +sidebar_position: 72 sidebar_label: Third-Party Libraries description: A list of third-party libraries used --- diff --git a/docs/en/development/style.md b/docs/en/development/style.md index 35683aa7822..78b1d1e9ebb 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -1,6 +1,6 @@ --- slug: /en/development/style -sidebar_position: 69 +sidebar_position: 70 sidebar_label: C++ Guide description: A list of recommendations regarding coding style, naming convention, formatting and more --- diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 9ae49e8f707..a4f23545176 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -1,6 +1,6 @@ --- slug: /en/development/tests -sidebar_position: 70 +sidebar_position: 71 sidebar_label: Testing title: ClickHouse Testing description: Most of ClickHouse features can be tested with functional tests and they are mandatory to use for every change in ClickHouse code that can be tested that way. From 54dad1f72ae603f5365501ab7b48e72766c5291e Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 10 Mar 2023 12:28:57 -0500 Subject: [PATCH 425/470] Fix typos --- docs/en/development/build-cross-s390x.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md index 21cc563ff06..2e4acc68f50 100644 --- a/docs/en/development/build-cross-s390x.md +++ b/docs/en/development/build-cross-s390x.md @@ -11,13 +11,13 @@ As of writing (2023/3/10) building for s390x considered to be experimental. Not ## Building As s390x does not support boringSSL, it uses openSSL and has two related build options. -- By default, the s390x build will create dynamiclly link to openSSL libraries. It will build openSSL, so its not neccessary to install openSSL beforehand. This is recommended in all cases. +- By default, the s390x build will dyamically link to openSSL libraries. It will build openSSL shared objects, so its not neccessary to install openSSL beforehand. (This options is recommended in all cases.) - Another option is to build openSSL in-tree. In this case two build options need to be supplied to cmake ```bash -DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1 ``` -These instructions assume that the host machine is x86_64 and has all the tooling required to build natively base based on the [Build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but should work on Ubuntu 20.04. +These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [Build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but should work on Ubuntu 20.04. In addition to installing the tooling used to build natively the following additional pacakges need to be installed: @@ -90,7 +90,7 @@ Process 1 stopped ## Visual Studio Code integration -- (CodeLLDB extension)[https://github.com/vadimcn/vscode-lldb] is required for visual debugging, (Command Variable)[https://github.com/rioj7/command-variable] extension can help dynamic lauches if using cmake variants. +- (CodeLLDB extension)[https://github.com/vadimcn/vscode-lldb] is required for visual debugging, (Command Variable)[https://github.com/rioj7/command-variable] extension can help dynamic launches if using cmake variants. - Make sure to set the backend to your llvm installation ex. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"` - Launcher: ```json From d26466bab903a6dbef1aa36e99a63fb7c88e4b22 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 10 Mar 2023 13:00:10 -0500 Subject: [PATCH 426/470] aspell corrections --- docs/en/development/build-cross-s390x.md | 14 +++++----- .../aspell-ignore/en/aspell-dict.txt | 27 ++++++++++++------- 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md index 2e4acc68f50..e02b22e6a42 100644 --- a/docs/en/development/build-cross-s390x.md +++ b/docs/en/development/build-cross-s390x.md @@ -10,16 +10,16 @@ As of writing (2023/3/10) building for s390x considered to be experimental. Not ## Building -As s390x does not support boringSSL, it uses openSSL and has two related build options. -- By default, the s390x build will dyamically link to openSSL libraries. It will build openSSL shared objects, so its not neccessary to install openSSL beforehand. (This options is recommended in all cases.) -- Another option is to build openSSL in-tree. In this case two build options need to be supplied to cmake +As s390x does not support boringssl, it uses OpenSSL and has two related build options. +- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so its not necessary to install OpenSSL beforehand. (This options is recommended in all cases.) +- Another option is to build OpenSSL in-tree. In this case two build options need to be supplied to cmake ```bash -DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1 ``` These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [Build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but should work on Ubuntu 20.04. -In addition to installing the tooling used to build natively the following additional pacakges need to be installed: +In addition to installing the tooling used to build natively the following additional packages need to be installed: ```bash apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-gnu binfmt-support qemu-user-static @@ -46,19 +46,19 @@ qemu-s390x-static -L /usr/s390x-linux-gnu ./clickhouse ## Debugging -Install lldb: +Install LLDB: ```bash apt-get install lldb-15 ``` -To Debug a s390x executable run clickhouse using qemu debug mode: +To Debug a s390x executable run clickhouse using QEMU debug mode: ```bash qemu-s390x-static -g 31338 -L /usr/s390x-linux-gnu ./clickhouse ``` -In antother shell run lldb and attach, replace `` and `` with the values corresponding to your environment. +In another shell run LLDB and attach, replace `` and `` with the values corresponding to your environment. ```bash lldb-15 (lldb) target create ./clickhouse diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 1ad7432a5bf..e2750622469 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 484 +personal_ws-1.1 en 544 AArch ACLs AMQP @@ -25,6 +25,7 @@ CentOS ClickHouse ClickHouse's CodeBlock +CodeLLDB Config ConnectionDetails Contrib @@ -56,9 +57,8 @@ Hostname IPv IntN Integrations -invariants -JSONAsString JSONAsObject +JSONAsString JSONColumns JSONColumnsWithMetadata JSONCompact @@ -82,6 +82,7 @@ Jemalloc Jepsen KDevelop LGPL +LLDB LOCALTIME LOCALTIMESTAMP LibFuzzer @@ -104,10 +105,11 @@ NULLIF NVME NYPD NuRaft -ObjectId -Ok OLAP OLTP +ObjectId +Ok +OpenSSL OpenSUSE OpenStack OpenTelemetry @@ -128,10 +130,10 @@ PrettySpaceNoEscapes PrettySpaceNoEscapesMonoBlock Protobuf ProtobufSingle +QEMU QTCreator QueryCacheHits QueryCacheMisses -QEMU RBAC RawBLOB RedHat @@ -178,14 +180,14 @@ Valgrind Vectorized VirtualBox Werror +WithNamesAndTypes Woboq WriteBuffer WriteBuffers -WithNamesAndTypes XCode YAML -Yasm YYYY +Yasm Zipkin ZooKeeper ZooKeeper's @@ -200,6 +202,7 @@ autostart avro avx aws +backend backoff backticks benchmarking @@ -309,6 +312,7 @@ instantiation integrational integrations interserver +invariants jdbc jemalloc json @@ -333,8 +337,8 @@ jsonstringseachrowwithprogress kafka kafkacat konsole -latencies laion +latencies lexicographically libFuzzer libc @@ -373,9 +377,10 @@ mutex mysql mysqldump mysqljs +natively noop -nullable nullability +nullable num obfuscator odbc @@ -395,6 +400,7 @@ postfix postfixes postgresql pre +preLaunchTask prebuild prebuilt preemptable @@ -532,6 +538,7 @@ xcode xml xz zLib +zLinux zkcopy zlib znodes From 1df592b06b3733f242b6a55eae8a293b8438eb50 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 10 Mar 2023 13:04:32 -0500 Subject: [PATCH 427/470] Remove exclusion --- docs/en/development/build-cross-s390x.md | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md index e02b22e6a42..1616fde3611 100644 --- a/docs/en/development/build-cross-s390x.md +++ b/docs/en/development/build-cross-s390x.md @@ -120,4 +120,4 @@ Process 1 stopped ] } ``` -- Make sure to run the clickhouse executable in debug mode prior to launch. (It is possible to create a preLaunchTask that automates this) \ No newline at end of file +- Make sure to run the clickhouse executable in debug mode prior to launch. (It is possible to create a `preLaunchTask` that automates this) \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index e2750622469..013e9b32fd4 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -400,7 +400,6 @@ postfix postfixes postgresql pre -preLaunchTask prebuild prebuilt preemptable From 2fc5864a11e808aa71e2bd2400ed01812471b7ec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Mar 2023 19:37:42 +0000 Subject: [PATCH 428/470] Fix aggregation-in-order with aliases. --- src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 0874a3771ae..9407504579b 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -519,8 +519,9 @@ AggregationInputOrder buildInputOrderInfo( enreachFixedColumns(sorting_key_dag, fixed_key_columns); - for (auto it = matches.cbegin(); it != matches.cend(); ++it) + for (const auto * output : dag->getOutputs()) { + auto it = matches.find(output); const MatchedTrees::Match * match = &it->second; if (match->node) { From 3e7510a584fdd5d5e46e4020ecb1856cfb3710fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 10 Mar 2023 19:56:53 +0000 Subject: [PATCH 429/470] Fix 01429_join_on_error_messages --- tests/queries/0_stateless/01429_join_on_error_messages.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 9b8688c8415..23aed83a66f 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -15,7 +15,7 @@ SET join_algorithm = 'partial_merge'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } -- works for a = b OR a = b because of equivalent disjunct optimization -SET join_algorithm = 'auto'; +SET join_algorithm = 'grace_hash'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } -- works for a = b OR a = b because of equivalent disjunct optimization From 663b4fb4a82cdd8a3bc58b84ee7146414f3236f1 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 10 Mar 2023 15:20:57 -0500 Subject: [PATCH 430/470] Small tweaks --- docs/en/development/build-cross-s390x.md | 22 +++++++++---------- .../aspell-ignore/en/aspell-dict.txt | 2 +- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/docs/en/development/build-cross-s390x.md b/docs/en/development/build-cross-s390x.md index 1616fde3611..bfb94ef7ed0 100644 --- a/docs/en/development/build-cross-s390x.md +++ b/docs/en/development/build-cross-s390x.md @@ -11,26 +11,26 @@ As of writing (2023/3/10) building for s390x considered to be experimental. Not ## Building As s390x does not support boringssl, it uses OpenSSL and has two related build options. -- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so its not necessary to install OpenSSL beforehand. (This options is recommended in all cases.) -- Another option is to build OpenSSL in-tree. In this case two build options need to be supplied to cmake +- By default, the s390x build will dynamically link to OpenSSL libraries. It will build OpenSSL shared objects, so it's not necessary to install OpenSSL beforehand. (This option is recommended in all cases.) +- Another option is to build OpenSSL in-tree. In this case two build flags need to be supplied to cmake ```bash -DENABLE_OPENSSL_DYNAMIC=0 -DENABLE_OPENSSL=1 ``` -These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [Build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but should work on Ubuntu 20.04. +These instructions assume that the host machine is x86_64 and has all the tooling required to build natively based on the [build instructions](../development/build.md). It also assumes that the host is Ubuntu 22.04 but the following instructions should also work on Ubuntu 20.04. -In addition to installing the tooling used to build natively the following additional packages need to be installed: +In addition to installing the tooling used to build natively, the following additional packages need to be installed: ```bash apt-get install binutils-s390x-linux-gnu libc6-dev-s390x-cross gcc-s390x-linux-gnu binfmt-support qemu-user-static ``` -If you wish to cross compile rust code.. +If you wish to cross compile rust code install the rust cross compile target for s390x: ```bash rustup target add s390x-unknown-linux-gnu ``` -To build for s390x, +To build for s390x: ```bash cmake -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-s390x.cmake .. ninja @@ -38,7 +38,7 @@ ninja ## Running -Once built, the binary can be run with: +Once built, the binary can be run with, eg.: ```bash qemu-s390x-static -L /usr/s390x-linux-gnu ./clickhouse @@ -52,7 +52,7 @@ Install LLDB: apt-get install lldb-15 ``` -To Debug a s390x executable run clickhouse using QEMU debug mode: +To Debug a s390x executable, run clickhouse using QEMU in debug mode: ```bash qemu-s390x-static -g 31338 -L /usr/s390x-linux-gnu ./clickhouse @@ -90,8 +90,8 @@ Process 1 stopped ## Visual Studio Code integration -- (CodeLLDB extension)[https://github.com/vadimcn/vscode-lldb] is required for visual debugging, (Command Variable)[https://github.com/rioj7/command-variable] extension can help dynamic launches if using cmake variants. -- Make sure to set the backend to your llvm installation ex. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"` +- (CodeLLDB extension)[https://github.com/vadimcn/vscode-lldb] is required for visual debugging, the (Command Variable)[https://github.com/rioj7/command-variable] extension can help dynamic launches if using (cmake variants)[https://github.com/microsoft/vscode-cmake-tools/blob/main/docs/variants.md]. +- Make sure to set the backend to your llvm installation eg. `"lldb.library": "/usr/lib/x86_64-linux-gnu/liblldb-15.so"` - Launcher: ```json { @@ -120,4 +120,4 @@ Process 1 stopped ] } ``` -- Make sure to run the clickhouse executable in debug mode prior to launch. (It is possible to create a `preLaunchTask` that automates this) \ No newline at end of file +- Make sure to run the clickhouse executable in debug mode prior to launch. (It is also possible to create a `preLaunchTask` that automates this) \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 013e9b32fd4..f1bba4dc2fc 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 544 +personal_ws-1.1 en 543 AArch ACLs AMQP From 86abe160fabb9ca09a1d7ae1d994aff38c277f04 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 10 Mar 2023 20:23:57 +0000 Subject: [PATCH 431/470] Update version_date.tsv and changelogs after v23.1.5.24-stable --- docs/changelogs/v23.1.5.24-stable.md | 28 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 29 insertions(+) create mode 100644 docs/changelogs/v23.1.5.24-stable.md diff --git a/docs/changelogs/v23.1.5.24-stable.md b/docs/changelogs/v23.1.5.24-stable.md new file mode 100644 index 00000000000..1c2c127a8c3 --- /dev/null +++ b/docs/changelogs/v23.1.5.24-stable.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.1.5.24-stable (0e51b53ba99) FIXME as compared to v23.1.4.58-stable (9ed562163a5) + +#### Build/Testing/Packaging Improvement +* Backported in [#47060](https://github.com/ClickHouse/ClickHouse/issues/47060): Fix error during server startup on old distros (e.g. Amazon Linux 2) and on ARM that glibc 2.28 symbols are not found. [#47008](https://github.com/ClickHouse/ClickHouse/pull/47008) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#46401](https://github.com/ClickHouse/ClickHouse/issues/46401): Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#46905](https://github.com/ClickHouse/ClickHouse/issues/46905): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#47211](https://github.com/ClickHouse/ClickHouse/issues/47211): `INSERT` queries through native TCP protocol and HTTP protocol were not canceled correctly in some cases. It could lead to a partially applied query if a client canceled the query, or if a client died or, in rare cases, on network errors. As a result, it could lead to not working deduplication. Fixes [#27667](https://github.com/ClickHouse/ClickHouse/issues/27667) and [#45377](https://github.com/ClickHouse/ClickHouse/issues/45377). [#46681](https://github.com/ClickHouse/ClickHouse/pull/46681) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#47118](https://github.com/ClickHouse/ClickHouse/issues/47118): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#46883](https://github.com/ClickHouse/ClickHouse/issues/46883): Fix MSan report in the `maxIntersections` function. This closes [#43126](https://github.com/ClickHouse/ClickHouse/issues/43126). [#46847](https://github.com/ClickHouse/ClickHouse/pull/46847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#47361](https://github.com/ClickHouse/ClickHouse/issues/47361): Fix possible deadlock on distributed query cancellation. [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 50a9314bc00..515c4addf83 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v23.2.4.12-stable 2023-03-10 v23.2.3.17-stable 2023-03-06 v23.2.2.20-stable 2023-03-01 v23.2.1.2537-stable 2023-02-23 +v23.1.5.24-stable 2023-03-10 v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 From eab72044682457d7897438809277de7202c4a6fe Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 10 Mar 2023 21:42:42 +0000 Subject: [PATCH 432/470] Update version_date.tsv and changelogs after v22.12.5.34-stable --- docs/changelogs/v22.12.5.34-stable.md | 29 +++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v22.12.5.34-stable.md diff --git a/docs/changelogs/v22.12.5.34-stable.md b/docs/changelogs/v22.12.5.34-stable.md new file mode 100644 index 00000000000..95befaa88ff --- /dev/null +++ b/docs/changelogs/v22.12.5.34-stable.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.12.5.34-stable (b82d6401ca1) FIXME as compared to v22.12.4.76-stable (cb5772db805) + +#### Improvement +* Backported in [#46983](https://github.com/ClickHouse/ClickHouse/issues/46983): - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#45729](https://github.com/ClickHouse/ClickHouse/issues/45729): Fix key description when encountering duplicate primary keys. This can happen in projections. See [#45590](https://github.com/ClickHouse/ClickHouse/issues/45590) for details. [#45686](https://github.com/ClickHouse/ClickHouse/pull/45686) ([Amos Bird](https://github.com/amosbird)). +* Backported in [#46398](https://github.com/ClickHouse/ClickHouse/issues/46398): Fix `SYSTEM UNFREEZE` queries failing with the exception `CANNOT_PARSE_INPUT_ASSERTION_FAILED`. [#46325](https://github.com/ClickHouse/ClickHouse/pull/46325) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Backported in [#46903](https://github.com/ClickHouse/ClickHouse/issues/46903): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#47210](https://github.com/ClickHouse/ClickHouse/issues/47210): `INSERT` queries through native TCP protocol and HTTP protocol were not canceled correctly in some cases. It could lead to a partially applied query if a client canceled the query, or if a client died or, in rare cases, on network errors. As a result, it could lead to not working deduplication. Fixes [#27667](https://github.com/ClickHouse/ClickHouse/issues/27667) and [#45377](https://github.com/ClickHouse/ClickHouse/issues/45377). [#46681](https://github.com/ClickHouse/ClickHouse/pull/46681) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Backported in [#47157](https://github.com/ClickHouse/ClickHouse/issues/47157): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#46881](https://github.com/ClickHouse/ClickHouse/issues/46881): Fix MSan report in the `maxIntersections` function. This closes [#43126](https://github.com/ClickHouse/ClickHouse/issues/43126). [#46847](https://github.com/ClickHouse/ClickHouse/pull/46847) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#47359](https://github.com/ClickHouse/ClickHouse/issues/47359): Fix possible deadlock on distributed query cancellation. [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Use /etc/default/clickhouse in systemd too [#47003](https://github.com/ClickHouse/ClickHouse/pull/47003) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-up to [#46681](https://github.com/ClickHouse/ClickHouse/issues/46681) [#47284](https://github.com/ClickHouse/ClickHouse/pull/47284) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 515c4addf83..9e99475523a 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -7,6 +7,7 @@ v23.1.4.58-stable 2023-03-01 v23.1.3.5-stable 2023-02-03 v23.1.2.9-stable 2023-01-29 v23.1.1.3077-stable 2023-01-25 +v22.12.5.34-stable 2023-03-10 v22.12.4.76-stable 2023-03-01 v22.12.3.5-stable 2023-01-10 v22.12.2.25-stable 2023-01-06 From 6e7ffd9a7baf5f588c9267ee9fbc12d4b8a983d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 11 Mar 2023 00:45:09 +0300 Subject: [PATCH 433/470] Update ReplicatedMergeTreeAttachThread.cpp --- src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index 557123ddae2..c859c994818 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -60,11 +60,11 @@ void ReplicatedMergeTreeAttachThread::run() if (needs_retry) { - LOG_ERROR(log, "Initialization failed. Error: {}", e.message()); + LOG_ERROR(log, "Initialization failed. Error: {}", getCurrentExceptionMessage(/* with_stacktrace */ true)); } else { - LOG_ERROR(log, "Initialization failed, table will remain readonly. Error: {}", e.message()); + LOG_ERROR(log, "Initialization failed, table will remain readonly. Error: {}", getCurrentExceptionMessage(/* with_stacktrace */ true)); storage.initialization_done = true; } } From 96477e1ea51218a472458aae7bd4020a14334c5f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 10 Mar 2023 22:01:27 +0000 Subject: [PATCH 434/470] Update version_date.tsv and changelogs after v22.8.15.23-lts --- docs/changelogs/v22.8.15.23-lts.md | 28 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 29 insertions(+) create mode 100644 docs/changelogs/v22.8.15.23-lts.md diff --git a/docs/changelogs/v22.8.15.23-lts.md b/docs/changelogs/v22.8.15.23-lts.md new file mode 100644 index 00000000000..096a504c9c2 --- /dev/null +++ b/docs/changelogs/v22.8.15.23-lts.md @@ -0,0 +1,28 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v22.8.15.23-lts (d36fa168bbf) FIXME as compared to v22.8.14.53-lts (4ea67c40077) + +#### Improvement +* Backported in [#46981](https://github.com/ClickHouse/ClickHouse/issues/46981): - Apply `ALTER TABLE table_name ON CLUSTER cluster MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name'` to all replicas. Because `ALTER TABLE t MOVE` is not replicated. [#46402](https://github.com/ClickHouse/ClickHouse/pull/46402) ([lizhuoyu5](https://github.com/lzydmxy)). + +#### Bug Fix +* Backported in [#47336](https://github.com/ClickHouse/ClickHouse/issues/47336): Sometimes after changing a role that could be not reflected on the access rights of a user who uses that role. This PR fixes that. [#46772](https://github.com/ClickHouse/ClickHouse/pull/46772) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#46901](https://github.com/ClickHouse/ClickHouse/issues/46901): - Fix incorrect alias recursion in QueryNormalizer. [#46609](https://github.com/ClickHouse/ClickHouse/pull/46609) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#47156](https://github.com/ClickHouse/ClickHouse/issues/47156): - Fix arithmetic operations in aggregate optimization with `min` and `max`. [#46705](https://github.com/ClickHouse/ClickHouse/pull/46705) ([Duc Canh Le](https://github.com/canhld94)). +* Backported in [#46987](https://github.com/ClickHouse/ClickHouse/issues/46987): Fix result of LIKE predicates which translate to substring searches and contain quoted non-LIKE metacharacters. [#46875](https://github.com/ClickHouse/ClickHouse/pull/46875) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#47357](https://github.com/ClickHouse/ClickHouse/issues/47357): Fix possible deadlock on distributed query cancellation. [#47161](https://github.com/ClickHouse/ClickHouse/pull/47161) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Reduce updates of Mergeable Check [#46781](https://github.com/ClickHouse/ClickHouse/pull/46781) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Update typing for a new PyGithub version [#47123](https://github.com/ClickHouse/ClickHouse/pull/47123) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Add a manual trigger for release workflow [#47302](https://github.com/ClickHouse/ClickHouse/pull/47302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 9e99475523a..c2d9781177d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -32,6 +32,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.15.23-lts 2023-03-10 v22.8.14.53-lts 2023-02-27 v22.8.13.20-lts 2023-01-29 v22.8.12.45-lts 2023-01-10 From a5c8f23f69adbad74b0eca35777f775daac24737 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 10 Mar 2023 23:33:56 +0100 Subject: [PATCH 435/470] Remove a feature --- src/Interpreters/InterpreterExplainQuery.cpp | 5 +- src/QueryPipeline/printPipeline.cpp | 177 ------------------- src/QueryPipeline/printPipeline.h | 7 - 3 files changed, 1 insertion(+), 188 deletions(-) delete mode 100644 src/QueryPipeline/printPipeline.cpp diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 3c225522cc4..b2172a07e91 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -504,10 +504,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline), resources); const auto & processors = pipe.getProcessors(); - if (settings.compact) - printPipelineCompact(processors, buf, settings.query_pipeline_options.header); - else - printPipeline(processors, buf); + printPipeline(processors, buf); } else { diff --git a/src/QueryPipeline/printPipeline.cpp b/src/QueryPipeline/printPipeline.cpp deleted file mode 100644 index 40c88502ed0..00000000000 --- a/src/QueryPipeline/printPipeline.cpp +++ /dev/null @@ -1,177 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ - -void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header) -{ - struct Node; - - /// Group by processors name, QueryPlanStep and group in this step. - struct Key - { - size_t group; - IQueryPlanStep * step; - std::string name; - - auto getTuple() const { return std::forward_as_tuple(group, step, name); } - - bool operator<(const Key & other) const - { - return getTuple() < other.getTuple(); - } - }; - - /// Group ports by header. - struct EdgeData - { - Block header; - size_t count; - }; - - using Edge = std::vector; - - struct Node - { - size_t id = 0; - std::map edges = {}; - std::vector agents = {}; - }; - - std::map graph; - - auto get_key = [](const IProcessor & processor) - { - return Key{processor.getQueryPlanStepGroup(), processor.getQueryPlanStep(), processor.getName()}; - }; - - /// Fill nodes. - for (const auto & processor : processors) - { - auto res = graph.emplace(get_key(*processor), Node()); - auto & node = res.first->second; - node.agents.emplace_back(processor.get()); - - if (res.second) - node.id = graph.size(); - } - - Block empty_header; - - /// Fill edges. - for (const auto & processor : processors) - { - auto & from = graph[get_key(*processor)]; - - for (auto & port : processor->getOutputs()) - { - if (!port.isConnected()) - continue; - - auto & to = graph[get_key(port.getInputPort().getProcessor())]; - auto & edge = from.edges[&to]; - - /// Use empty header for each edge if with_header is false. - const auto & header = with_header ? port.getHeader() - : empty_header; - - /// Group by header. - bool found = false; - for (auto & item : edge) - { - if (blocksHaveEqualStructure(header, item.header)) - { - found = true; - ++item.count; - break; - } - } - - if (!found) - edge.emplace_back(EdgeData{header, 1}); - } - } - - /// Group processors by it's QueryPlanStep. - std::map> steps_map; - - for (const auto & item : graph) - steps_map[item.first.step].emplace_back(&item.second); - - out << "digraph\n{\n"; - out << " rankdir=\"LR\";\n"; - out << " { node [shape = rect]\n"; - - /// Nodes // TODO quoting and escaping - size_t next_step = 0; - for (const auto & item : steps_map) - { - /// Use separate clusters for each step. - if (item.first != nullptr) - { - out << " subgraph cluster_" << next_step << " {\n"; - out << " label =\"" << item.first->getName() << "\";\n"; - out << " style=filled;\n"; - out << " color=lightgrey;\n"; - out << " node [style=filled,color=white];\n"; - out << " { rank = same;\n"; - - ++next_step; - } - - for (const auto & node : item.second) - { - const auto & processor = node->agents.front(); - out << " n" << node->id << " [label=\"" << processor->getName(); - - if (node->agents.size() > 1) - out << " × " << node->agents.size(); - - const auto & description = processor->getDescription(); - if (!description.empty()) - out << ' ' << description; - - out << "\"];\n"; - } - - if (item.first != nullptr) - { - out << " }\n"; - out << " }\n"; - } - } - - out << " }\n"; - - /// Edges - for (const auto & item : graph) - { - for (const auto & edge : item.second.edges) - { - for (const auto & data : edge.second) - { - out << " n" << item.second.id << " -> " << "n" << edge.first->id << " [label=\""; - - if (data.count > 1) - out << "× " << data.count; - - if (with_header) - { - for (const auto & elem : data.header) - { - out << "\n"; - elem.dumpStructure(out); - } - } - - out << "\"];\n"; - } - } - } - out << "}\n"; -} - -} diff --git a/src/QueryPipeline/printPipeline.h b/src/QueryPipeline/printPipeline.h index 76143211875..ff3b53300ce 100644 --- a/src/QueryPipeline/printPipeline.h +++ b/src/QueryPipeline/printPipeline.h @@ -10,7 +10,6 @@ namespace DB * You can render it with: * dot -T png < pipeline.dot > pipeline.png */ - template void printPipeline(const Processors & processors, const Statuses & statuses, WriteBuffer & out) { @@ -65,10 +64,4 @@ void printPipeline(const Processors & processors, WriteBuffer & out) printPipeline(processors, std::vector(), out); } -/// Prints pipeline in compact representation. -/// Group processors by it's name, QueryPlanStep and QueryPlanStepGroup. -/// If QueryPlanStep wasn't set for processor, representation may be not correct. -/// If with_header is set, prints block header for each edge. -void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header); - } From eead7fcbac86d6d7736aaac7214f68a404ec62f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 02:24:07 +0100 Subject: [PATCH 436/470] Fix error in `decodeURLComponent` --- src/Functions/URL/decodeURLComponent.cpp | 25 +++++++++++++------ .../02677_decode_url_component.reference | 2 ++ .../02677_decode_url_component.sql | 5 ++++ 3 files changed, 24 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02677_decode_url_component.reference create mode 100644 tests/queries/0_stateless/02677_decode_url_component.sql diff --git a/src/Functions/URL/decodeURLComponent.cpp b/src/Functions/URL/decodeURLComponent.cpp index 7d98ccd63a0..05e3fbea3fd 100644 --- a/src/Functions/URL/decodeURLComponent.cpp +++ b/src/Functions/URL/decodeURLComponent.cpp @@ -14,28 +14,33 @@ namespace ErrorCodes static size_t encodeURL(const char * __restrict src, size_t src_size, char * __restrict dst, bool space_as_plus) { char * dst_pos = dst; - for (size_t i = 0; i < src_size; i++) + for (size_t i = 0; i < src_size; ++i) { if ((src[i] >= '0' && src[i] <= '9') || (src[i] >= 'a' && src[i] <= 'z') || (src[i] >= 'A' && src[i] <= 'Z') || src[i] == '-' || src[i] == '_' || src[i] == '.' || src[i] == '~') { - *dst_pos++ = src[i]; + *dst_pos = src[i]; + ++dst_pos; } else if (src[i] == ' ' && space_as_plus) { - *dst_pos++ = '+'; + *dst_pos = '+'; + ++dst_pos; } else { - *dst_pos++ = '%'; - *dst_pos++ = hexDigitUppercase(src[i] >> 4); - *dst_pos++ = hexDigitUppercase(src[i] & 0xf); + dst_pos[0] = '%'; + ++dst_pos; + writeHexByteUppercase(src[i], dst_pos); + dst_pos += 2; } } - *dst_pos++ = src[src_size]; + *dst_pos = 0; + ++dst_pos; return dst_pos - dst; } + /// We assume that size of the dst buf isn't less than src_size. static size_t decodeURL(const char * __restrict src, size_t src_size, char * __restrict dst, bool plus_as_space) { @@ -120,10 +125,14 @@ struct CodeURLComponentImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { if (code_strategy == encode) - //the destination(res_data) string is at most three times the length of the source string + { + /// the destination(res_data) string is at most three times the length of the source string res_data.resize(data.size() * 3); + } else + { res_data.resize(data.size()); + } size_t size = offsets.size(); res_offsets.resize(size); diff --git a/tests/queries/0_stateless/02677_decode_url_component.reference b/tests/queries/0_stateless/02677_decode_url_component.reference new file mode 100644 index 00000000000..5f88856dc1c --- /dev/null +++ b/tests/queries/0_stateless/02677_decode_url_component.reference @@ -0,0 +1,2 @@ +%D0%BA%D0%BB%D0%B8%D0%BA%D1%85%D0%B0%D1%83%D1%81 1 +1 diff --git a/tests/queries/0_stateless/02677_decode_url_component.sql b/tests/queries/0_stateless/02677_decode_url_component.sql new file mode 100644 index 00000000000..68345b5de16 --- /dev/null +++ b/tests/queries/0_stateless/02677_decode_url_component.sql @@ -0,0 +1,5 @@ +SELECT + encodeURLComponent('кликхаус') AS encoded, + decodeURLComponent(encoded) = 'кликхаус' AS expected_EQ; + +SELECT DISTINCT decodeURLComponent(encodeURLComponent(randomString(100) AS x)) = x FROM numbers(100000); From 2fbb3660de4682e29c4501a25bb250f463c5f04c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 03:32:05 +0100 Subject: [PATCH 437/470] Add a test --- .../0_stateless/02678_explain_pipeline_graph.reference | 0 tests/queries/0_stateless/02678_explain_pipeline_graph.sql | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02678_explain_pipeline_graph.reference create mode 100644 tests/queries/0_stateless/02678_explain_pipeline_graph.sql diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph.reference b/tests/queries/0_stateless/02678_explain_pipeline_graph.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph.sql b/tests/queries/0_stateless/02678_explain_pipeline_graph.sql new file mode 100644 index 00000000000..48cfbf2b349 --- /dev/null +++ b/tests/queries/0_stateless/02678_explain_pipeline_graph.sql @@ -0,0 +1,7 @@ +-- The server does not crash after these queries: + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1(ID UInt64, name String) engine=MergeTree order by ID; +insert into t1(ID, name) values (1, 'abc'), (2, 'bbb'); +explain pipeline graph=1 select count(ID) from t1 FORMAT Null; +DROP TABLE t1; From a762112e15a7d1bd3c3c6202e55c4bc37bd44b79 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Mar 2023 17:01:50 +0100 Subject: [PATCH 438/470] Analyzer support distributed JOINS and subqueries in IN functions --- src/Analyzer/FunctionNode.cpp | 30 +- src/Analyzer/InDepthQueryTreeVisitor.h | 17 +- src/Analyzer/JoinNode.h | 6 + src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 2 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 +- src/Analyzer/Utils.cpp | 50 +++ src/Analyzer/Utils.h | 9 + src/Planner/PlannerContext.cpp | 2 +- src/Planner/PlannerContext.h | 24 +- src/Planner/PlannerJoinTree.cpp | 170 ++++++--- src/Planner/TableExpressionData.h | 11 + src/Planner/Utils.cpp | 34 ++ src/Planner/Utils.h | 5 + src/Storages/StorageDistributed.cpp | 355 +++++++++++++++++- src/Storages/StorageMergeTree.cpp | 9 +- src/Storages/StorageView.cpp | 2 +- .../0_stateless/00858_issue_4756.reference | 2 + .../queries/0_stateless/00858_issue_4756.sql | 20 +- .../02420_final_setting_analyzer.reference | 3 - .../02420_final_setting_analyzer.sql | 3 - 20 files changed, 657 insertions(+), 102 deletions(-) diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 718dcf4bb58..68bdb0f9956 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -2,18 +2,21 @@ #include #include -#include -#include #include #include +#include +#include + #include #include #include +#include +#include #include namespace DB @@ -44,17 +47,28 @@ const DataTypes & FunctionNode::getArgumentTypes() const ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const { const auto & arguments = getArguments().getNodes(); + size_t arguments_size = arguments.size(); + ColumnsWithTypeAndName argument_columns; argument_columns.reserve(arguments.size()); - for (const auto & arg : arguments) + for (size_t i = 0; i < arguments_size; ++i) { - ColumnWithTypeAndName argument; - argument.type = arg->getResultType(); - if (auto * constant = arg->as()) - argument.column = argument.type->createColumnConst(1, constant->getValue()); - argument_columns.push_back(std::move(argument)); + const auto & argument = arguments[i]; + + ColumnWithTypeAndName argument_column; + + if (isNameOfInFunction(function_name) && i == 1) + argument_column.type = std::make_shared(); + else + argument_column.type = argument->getResultType(); + + if (auto * constant = argument->as()) + argument_column.column = argument_column.type->createColumnConst(1, constant->getValue()); + + argument_columns.push_back(std::move(argument_column)); } + return argument_columns; } diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index af69fc55589..1cc48fb1e53 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -99,8 +99,9 @@ class InDepthQueryTreeVisitorWithContext public: using VisitQueryTreeNodeType = std::conditional_t; - explicit InDepthQueryTreeVisitorWithContext(ContextPtr context) + explicit InDepthQueryTreeVisitorWithContext(ContextPtr context, size_t initial_subquery_depth = 0) : current_context(std::move(context)) + , subquery_depth(initial_subquery_depth) {} /// Return true if visitor should traverse tree top to bottom, false otherwise @@ -125,11 +126,17 @@ public: return current_context->getSettingsRef(); } + size_t getSubqueryDepth() const + { + return subquery_depth; + } + void visit(VisitQueryTreeNodeType & query_tree_node) { auto current_scope_context_ptr = current_context; SCOPE_EXIT( current_context = std::move(current_scope_context_ptr); + --subquery_depth; ); if (auto * query_node = query_tree_node->template as()) @@ -137,6 +144,8 @@ public: else if (auto * union_node = query_tree_node->template as()) current_context = union_node->getContext(); + ++subquery_depth; + bool traverse_top_to_bottom = getDerived().shouldTraverseTopToBottom(); if (!traverse_top_to_bottom) visitChildren(query_tree_node); @@ -145,7 +154,12 @@ public: if (traverse_top_to_bottom) visitChildren(query_tree_node); + + getDerived().leaveImpl(query_tree_node); } + + void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) + {} private: Derived & getDerived() { @@ -172,6 +186,7 @@ private: } ContextPtr current_context; + size_t subquery_depth = 0; }; template diff --git a/src/Analyzer/JoinNode.h b/src/Analyzer/JoinNode.h index 0d856985794..f58fe3f1af5 100644 --- a/src/Analyzer/JoinNode.h +++ b/src/Analyzer/JoinNode.h @@ -106,6 +106,12 @@ public: return locality; } + /// Set join locality + void setLocality(JoinLocality locality_value) + { + locality = locality_value; + } + /// Get join strictness JoinStrictness getStrictness() const { diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index fa5fc0e75a8..15326ca1dc8 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -42,7 +42,7 @@ private: return; const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); - bool is_final_supported = storage && storage->supportsFinal() && !storage->isRemote(); + bool is_final_supported = storage && storage->supportsFinal(); if (!is_final_supported) return; diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 34c03a9ffb6..5a726ab4bb5 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2914,7 +2914,10 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id break; IdentifierLookup column_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; - if (tryBindIdentifierToAliases(column_identifier_lookup, scope) || + if (tryBindIdentifierToAliases(column_identifier_lookup, scope)) + break; + + if (table_expression_data.should_qualify_columns && tryBindIdentifierToTableExpressions(column_identifier_lookup, table_expression_node, scope)) break; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index c5a5c042cbc..5c1873bd6e4 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -79,6 +79,56 @@ bool isNameOfInFunction(const std::string & function_name) return is_special_function_in; } +bool isNameOfLocalInFunction(const std::string & function_name) +{ + bool is_special_function_in = function_name == "in" || + function_name == "notIn" || + function_name == "nullIn" || + function_name == "notNullIn" || + function_name == "inIgnoreSet" || + function_name == "notInIgnoreSet" || + function_name == "nullInIgnoreSet" || + function_name == "notNullInIgnoreSet"; + + return is_special_function_in; +} + +bool isNameOfGlobalInFunction(const std::string & function_name) +{ + bool is_special_function_in = function_name == "globalIn" || + function_name == "globalNotIn" || + function_name == "globalNullIn" || + function_name == "globalNotNullIn" || + function_name == "globalInIgnoreSet" || + function_name == "globalNotInIgnoreSet" || + function_name == "globalNullInIgnoreSet" || + function_name == "globalNotNullInIgnoreSet"; + + return is_special_function_in; +} + +std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & function_name) +{ + if (function_name == "in") + return "globalIn"; + else if (function_name == "notIn") + return "globalNotIn"; + else if (function_name == "nullIn") + return "globalNullIn"; + else if (function_name == "notNullIn") + return "globalNotNullIn"; + else if (function_name == "inIgnoreSet") + return "globalInIgnoreSet"; + else if (function_name == "notInIgnoreSet") + return "globalNotInIgnoreSet"; + else if (function_name == "nullInIgnoreSet") + return "globalNullInIgnoreSet"; + else if (function_name == "notNullInIgnoreSet") + return "globalNotNullInIgnoreSet"; + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid local IN function name {}", function_name); +} + QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, const DataTypePtr & type, const ContextPtr & context, diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 3e2d95c6012..9132a1d72bc 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -13,6 +13,15 @@ bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root); /// Returns true if function name is name of IN function or its variations, false otherwise bool isNameOfInFunction(const std::string & function_name); +/// Returns true if function name is name of local IN function or its variations, false otherwise +bool isNameOfLocalInFunction(const std::string & function_name); + +/// Returns true if function name is name of global IN function or its variations, false otherwise +bool isNameOfGlobalInFunction(const std::string & function_name); + +/// Returns global in function name for local in function name +std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & function_name); + /** Build cast function that cast expression into type. * If resolve = true, then result cast function is resolved during build, otherwise * result cast function is not resolved during build. diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 9f4a489bf5f..59ae0f20fac 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -45,7 +45,7 @@ bool GlobalPlannerContext::hasColumnIdentifier(const ColumnIdentifier & column_i return column_identifiers.contains(column_identifier); } -PlannerContext::PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_) +PlannerContext::PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_) : query_context(std::move(query_context_)) , global_planner_context(std::move(global_planner_context_)) {} diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 63874bf7ab9..e47198bfe5f 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -88,16 +88,22 @@ class PlannerContext { public: /// Create planner context with query context and global planner context - PlannerContext(ContextPtr query_context_, GlobalPlannerContextPtr global_planner_context_); + PlannerContext(ContextMutablePtr query_context_, GlobalPlannerContextPtr global_planner_context_); /// Get planner context query context - const ContextPtr & getQueryContext() const + ContextPtr getQueryContext() const { return query_context; } - /// Get planner context query context - ContextPtr & getQueryContext() + /// Get planner context mutable query context + const ContextMutablePtr & getMutableQueryContext() const + { + return query_context; + } + + /// Get planner context mutable query context + ContextMutablePtr & getMutableQueryContext() { return query_context; } @@ -137,12 +143,18 @@ public: */ TableExpressionData * getTableExpressionDataOrNull(const QueryTreeNodePtr & table_expression_node); - /// Get table expression node to data read only map + /// Get table expression node to data map const std::unordered_map & getTableExpressionNodeToData() const { return table_expression_node_to_data; } + /// Get table expression node to data map + std::unordered_map & getTableExpressionNodeToData() + { + return table_expression_node_to_data; + } + /** Get column node identifier. * For column node source check if table expression data is registered. * If table expression data is not registered exception is thrown. @@ -184,7 +196,7 @@ public: private: /// Query context - ContextPtr query_context; + ContextMutablePtr query_context; /// Global planner context GlobalPlannerContextPtr global_planner_context; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 6f818e2c8f7..edea51aba45 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -246,17 +246,87 @@ bool applyTrivialCountIfPossible( return true; } -JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expression, - const SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, - PlannerContextPtr & planner_context, - bool is_single_table_expression) +void prepareBuildQueryPlanForTableExpression(const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + auto columns_names = table_expression_data.getColumnNames(); + + auto * table_node = table_expression->as(); + auto * table_function_node = table_expression->as(); + auto * query_node = table_expression->as(); + auto * union_node = table_expression->as(); + + /** The current user must have the SELECT privilege. + * We do not check access rights for table functions because they have been already checked in ITableFunction::execute(). + */ + if (table_node) + { + auto column_names_with_aliases = columns_names; + const auto & alias_columns_names = table_expression_data.getAliasColumnsNames(); + column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end()); + checkAccessRights(*table_node, column_names_with_aliases, query_context); + } + + if (columns_names.empty()) + { + NameAndTypePair additional_column_to_read; + + if (table_node || table_function_node) + { + const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); + const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + additional_column_to_read = chooseSmallestColumnToReadFromStorage(storage, storage_snapshot); + + } + else if (query_node || union_node) + { + const auto & projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); + NamesAndTypesList projection_columns_list(projection_columns.begin(), projection_columns.end()); + additional_column_to_read = ExpressionActions::getSmallestColumn(projection_columns_list); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected table, table function, query or union. Actual {}", + table_expression->formatASTForErrorMessage()); + } + + auto & global_planner_context = planner_context->getGlobalPlannerContext(); + const auto & column_identifier = global_planner_context->createColumnIdentifier(additional_column_to_read, table_expression); + columns_names.push_back(additional_column_to_read.name); + table_expression_data.addColumn(additional_column_to_read, column_identifier); + } + + /// Limitation on the number of columns to read + if (settings.max_columns_to_read && columns_names.size() > settings.max_columns_to_read) + throw Exception(ErrorCodes::TOO_MANY_COLUMNS, + "Limit for number of columns to read exceeded. Requested: {}, maximum: {}", + columns_names.size(), + settings.max_columns_to_read); +} + +JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, + const SelectQueryInfo & select_query_info, + const SelectQueryOptions & select_query_options, + PlannerContextPtr & planner_context, + bool is_single_table_expression, + bool wrap_read_columns_in_subquery) +{ + const auto & query_context = planner_context->getQueryContext(); + const auto & settings = query_context->getSettingsRef(); + + auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); + QueryProcessingStage::Enum from_stage = QueryProcessingStage::Enum::FetchColumns; + if (wrap_read_columns_in_subquery) + { + auto columns = table_expression_data.getColumns(); + table_expression = buildSubqueryToReadColumnsFromTableExpression(columns, table_expression, query_context); + } + auto * table_node = table_expression->as(); auto * table_function_node = table_expression->as(); auto * query_node = table_expression->as(); @@ -264,8 +334,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl QueryPlan query_plan; - auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); - if (table_node || table_function_node) { const auto & storage = table_node ? table_node->getStorage() : table_function_node->getStorage(); @@ -362,32 +430,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl auto columns_names = table_expression_data.getColumnNames(); - /** The current user must have the SELECT privilege. - * We do not check access rights for table functions because they have been already checked in ITableFunction::execute(). - */ - if (table_node) - { - auto column_names_with_aliases = columns_names; - const auto & alias_columns_names = table_expression_data.getAliasColumnsNames(); - column_names_with_aliases.insert(column_names_with_aliases.end(), alias_columns_names.begin(), alias_columns_names.end()); - checkAccessRights(*table_node, column_names_with_aliases, planner_context->getQueryContext()); - } - - /// Limitation on the number of columns to read - if (settings.max_columns_to_read && columns_names.size() > settings.max_columns_to_read) - throw Exception(ErrorCodes::TOO_MANY_COLUMNS, - "Limit for number of columns to read exceeded. Requested: {}, maximum: {}", - columns_names.size(), - settings.max_columns_to_read); - - if (columns_names.empty()) - { - auto additional_column_to_read = chooseSmallestColumnToReadFromStorage(storage, storage_snapshot); - const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); - columns_names.push_back(additional_column_to_read.name); - table_expression_data.addColumn(additional_column_to_read, column_identifier); - } - bool need_rewrite_query_with_final = storage->needRewriteQueryWithFinal(columns_names); if (need_rewrite_query_with_final) { @@ -464,16 +506,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl } else { - if (table_expression_data.getColumnNames().empty()) - { - const auto & projection_columns = query_node ? query_node->getProjectionColumns() : union_node->computeProjectionColumns(); - NamesAndTypesList projection_columns_list(projection_columns.begin(), projection_columns.end()); - auto additional_column_to_read = ExpressionActions::getSmallestColumn(projection_columns_list); - - const auto & column_identifier = planner_context->getGlobalPlannerContext()->createColumnIdentifier(additional_column_to_read, table_expression); - table_expression_data.addColumn(additional_column_to_read, column_identifier); - } - auto subquery_options = select_query_options.subquery(); Planner subquery_planner(table_expression, subquery_options, planner_context->getGlobalPlannerContext()); /// Propagate storage limits to subquery @@ -516,10 +548,11 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(const QueryTreeNodePtr & tabl planner.buildQueryPlanIfNeeded(); auto expected_header = planner.getQueryPlan().getCurrentDataStream().header; - materializeBlockInplace(expected_header); if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, expected_header)) { + materializeBlockInplace(expected_header); + auto rename_actions_dag = ActionsDAG::makeConvertingActions( query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), expected_header.getColumnsWithTypeAndName(), @@ -1059,14 +1092,40 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { - const auto & query_node_typed = query_node->as(); - auto table_expressions_stack = buildTableExpressionsStack(query_node_typed.getJoinTree()); + auto table_expressions_stack = buildTableExpressionsStack(query_node->as().getJoinTree()); size_t table_expressions_stack_size = table_expressions_stack.size(); bool is_single_table_expression = table_expressions_stack_size == 1; std::vector table_expressions_outer_scope_columns(table_expressions_stack_size); ColumnIdentifierSet current_outer_scope_columns = outer_scope_columns; + /// For each table, table function, query, union table expressions prepare before query plan build + for (size_t i = 0; i < table_expressions_stack_size; ++i) + { + const auto & table_expression = table_expressions_stack[i]; + auto table_expression_type = table_expression->getNodeType(); + if (table_expression_type == QueryTreeNodeType::JOIN || + table_expression_type == QueryTreeNodeType::ARRAY_JOIN) + continue; + + prepareBuildQueryPlanForTableExpression(table_expression, planner_context); + } + + /** If left most table expression query plan is planned to stage that is not equal to fetch columns, + * then left most table expression is responsible for providing valid JOIN TREE part of final query plan. + * + * Examples: Distributed, LiveView, Merge storages. + */ + auto left_table_expression = table_expressions_stack.front(); + auto left_table_expression_query_plan = buildQueryPlanForTableExpression(left_table_expression, + select_query_info, + select_query_options, + planner_context, + is_single_table_expression, + false /*wrap_read_columns_in_subquery*/); + if (left_table_expression_query_plan.from_stage != QueryProcessingStage::FetchColumns) + return left_table_expression_query_plan; + for (Int64 i = static_cast(table_expressions_stack_size) - 1; i >= 0; --i) { table_expressions_outer_scope_columns[i] = current_outer_scope_columns; @@ -1120,19 +1179,22 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, } else { - const auto & table_expression_data = planner_context->getTableExpressionDataOrThrow(table_expression); - if (table_expression_data.isRemote() && i != 0) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "JOIN with multiple remote storages is unsupported"); + if (table_expression == left_table_expression) + { + query_plans_stack.push_back(std::move(left_table_expression_query_plan)); + continue; + } + /** If table expression is remote and it is not left most table expression, we wrap read columns from such + * table expression in subquery. + */ + bool is_remote = planner_context->getTableExpressionDataOrThrow(table_expression).isRemote(); query_plans_stack.push_back(buildQueryPlanForTableExpression(table_expression, select_query_info, select_query_options, planner_context, - is_single_table_expression)); - - if (query_plans_stack.back().from_stage != QueryProcessingStage::FetchColumns) - break; + is_single_table_expression, + is_remote /*wrap_read_columns_in_subquery*/)); } } diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index e828f128e38..0f74e671ac7 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -101,6 +101,17 @@ public: return column_names; } + NamesAndTypes getColumns() const + { + NamesAndTypes result; + result.reserve(column_names.size()); + + for (const auto & column_name : column_names) + result.push_back(column_name_to_column.at(column_name)); + + return result; + } + ColumnIdentifiers getColumnIdentifiers() const { ColumnIdentifiers result; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c5eadac55d..4f320f144df 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -19,6 +20,7 @@ #include #include +#include #include #include #include @@ -391,4 +393,36 @@ QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNo return query_node->cloneAndReplace(replacement_map); } +QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns, + const QueryTreeNodePtr & table_expression, + const ContextPtr & context) +{ + auto projection_columns = columns; + + QueryTreeNodes subquery_projection_nodes; + subquery_projection_nodes.reserve(projection_columns.size()); + + for (const auto & column : projection_columns) + subquery_projection_nodes.push_back(std::make_shared(column, table_expression)); + + if (subquery_projection_nodes.empty()) + { + auto constant_data_type = std::make_shared(); + subquery_projection_nodes.push_back(std::make_shared(1UL, constant_data_type)); + projection_columns.push_back({"1", std::move(constant_data_type)}); + } + + auto context_copy = Context::createCopy(context); + updateContextForSubqueryExecution(context_copy); + + auto query_node = std::make_shared(std::move(context_copy)); + + query_node->resolveProjectionColumns(projection_columns); + query_node->getProjection().getNodes() = std::move(subquery_projection_nodes); + query_node->getJoinTree() = table_expression; + query_node->setIsSubquery(true); + + return query_node; +} + } diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 0520bd67d26..1e7a98b7256 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -72,4 +72,9 @@ QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNo const ContextPtr & context, ResultReplacementMap * result_replacement_map = nullptr); +/// Build subquery to read specified columns from table expression +QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(const NamesAndTypes & columns, + const QueryTreeNodePtr & table_expression, + const ContextPtr & context); + } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4eb454e5156..9f84e7d210a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -39,11 +39,16 @@ #include #include +#include +#include #include #include #include +#include +#include #include #include +#include #include #include @@ -55,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -69,12 +75,14 @@ #include #include +#include #include #include #include #include +#include #include #include #include @@ -138,6 +146,7 @@ namespace ErrorCodes extern const int DISTRIBUTED_TOO_MANY_PENDING_BYTES; extern const int ARGUMENT_OUT_OF_BOUND; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED; } namespace ActionLocks @@ -634,12 +643,265 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( namespace { -QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const SelectQueryInfo & query_info, +/// Visitor that collect column source to columns mapping from query and all subqueries +class CollectColumnSourceToColumnsVisitor : public InDepthQueryTreeVisitor +{ +public: + struct Columns + { + NameSet column_names; + NamesAndTypes columns; + + void addColumn(NameAndTypePair column) + { + if (column_names.contains(column.name)) + return; + + column_names.insert(column.name); + columns.push_back(std::move(column)); + } + }; + + const std::unordered_map & getColumnSourceToColumns() const + { + return column_source_to_columns; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * column_node = node->as(); + if (!column_node) + return; + + auto column_source = column_node->getColumnSourceOrNull(); + if (!column_source) + return; + + auto it = column_source_to_columns.find(column_source); + if (it == column_source_to_columns.end()) + { + auto [insert_it, _] = column_source_to_columns.emplace(column_source, Columns()); + it = insert_it; + } + + it->second.addColumn(column_node->getColumn()); + } + +private: + std::unordered_map column_source_to_columns; +}; + +/** Visitor that rewrites IN and JOINs in query and all subqueries according to distributed_product_mode and + * prefer_global_in_and_join settings. + * + * Additionally collects GLOBAL JOIN and GLOBAL IN query nodes. + * + * If distributed_product_mode = deny, then visitor throws exception if there are multiple distributed tables. + * If distributed_product_mode = local, then visitor collects replacement map for tables that must be replaced + * with local tables. + * If distributed_product_mode = global or prefer_global_in_and_join setting is true, then visitor rewrites JOINs and IN functions that + * contain distributed tables to GLOBAL JOINs and GLOBAL IN functions. + * If distributed_product_mode = allow, then visitor does not rewrite query if there are multiple distributed tables. + */ +class DistributedProductModeRewriteInJoinVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + explicit DistributedProductModeRewriteInJoinVisitor(const ContextPtr & context_) + : Base(context_) + {} + + struct InFunctionOrJoin + { + QueryTreeNodePtr query_node; + size_t subquery_depth = 0; + }; + + const std::unordered_map & getReplacementMap() const + { + return replacement_map; + } + + const std::vector & getGlobalInOrJoinNodes() const + { + return global_in_or_join_nodes; + } + + bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) + { + auto * function_node = parent->as(); + if (function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) + return false; + + auto * join_node = parent->as(); + if (join_node && join_node->getLocality() == JoinLocality::Global && join_node->getRightTableExpression() == child) + return false; + + return true; + } + + void visitImpl(QueryTreeNodePtr & node) + { + auto * function_node = node->as(); + auto * join_node = node->as(); + + if ((function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) || + (join_node && join_node->getLocality() == JoinLocality::Global)) + { + InFunctionOrJoin in_function_or_join_entry; + in_function_or_join_entry.query_node = node; + in_function_or_join_entry.subquery_depth = getSubqueryDepth(); + global_in_or_join_nodes.push_back(std::move(in_function_or_join_entry)); + return; + } + + if ((function_node && isNameOfLocalInFunction(function_node->getFunctionName())) || + (join_node && join_node->getLocality() != JoinLocality::Global)) + { + InFunctionOrJoin in_function_or_join_entry; + in_function_or_join_entry.query_node = node; + in_function_or_join_entry.subquery_depth = getSubqueryDepth(); + in_function_or_join_stack.push_back(in_function_or_join_entry); + return; + } + + if (node->getNodeType() == QueryTreeNodeType::TABLE) + tryRewriteTableNodeIfNeeded(node); + } + + void leaveImpl(QueryTreeNodePtr & node) + { + if (!in_function_or_join_stack.empty() && node.get() == in_function_or_join_stack.back().query_node.get()) + in_function_or_join_stack.pop_back(); + } + +private: + void tryRewriteTableNodeIfNeeded(const QueryTreeNodePtr & table_node) + { + const auto & table_node_typed = table_node->as(); + const auto * distributed_storage = typeid_cast(table_node_typed.getStorage().get()); + if (!distributed_storage) + return; + + bool distributed_valid_for_rewrite = distributed_storage->getShardCount() >= 2; + if (!distributed_valid_for_rewrite) + return; + + auto distributed_product_mode = getSettings().distributed_product_mode; + + if (distributed_product_mode == DistributedProductMode::LOCAL) + { + StorageID remote_storage_id = StorageID{distributed_storage->getRemoteDatabaseName(), + distributed_storage->getRemoteTableName()}; + auto resolved_remote_storage_id = getContext()->resolveStorageID(remote_storage_id); + const auto & distributed_storage_columns = table_node_typed.getStorageSnapshot()->metadata->getColumns(); + auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_columns); + auto replacement_table_expression = std::make_shared(std::move(storage), getContext()); + replacement_map.emplace(table_node.get(), std::move(replacement_table_expression)); + } + else if ((distributed_product_mode == DistributedProductMode::GLOBAL || getSettings().prefer_global_in_and_join) && + !in_function_or_join_stack.empty()) + { + auto * in_or_join_node_to_modify = in_function_or_join_stack.back().query_node.get(); + + if (auto * in_function_to_modify = in_or_join_node_to_modify->as()) + { + auto global_in_function_name = getGlobalInFunctionNameForLocalInFunctionName(in_function_to_modify->getFunctionName()); + auto global_in_function_resolver = FunctionFactory::instance().get(global_in_function_name, getContext()); + in_function_to_modify->resolveAsFunction(global_in_function_resolver->build(in_function_to_modify->getArgumentColumns())); + } + else if (auto * join_node_to_modify = in_or_join_node_to_modify->as()) + { + join_node_to_modify->setLocality(JoinLocality::Global); + } + + global_in_or_join_nodes.push_back(in_function_or_join_stack.back()); + } + else if (distributed_product_mode == DistributedProductMode::ALLOW) + { + return; + } + else if (distributed_product_mode == DistributedProductMode::DENY) + { + throw Exception(ErrorCodes::DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED, + "Double-distributed IN/JOIN subqueries is denied (distributed_product_mode = 'deny'). " + "You may rewrite query to use local tables " + "in subqueries, or use GLOBAL keyword, or set distributed_product_mode to suitable value."); + } + } + + std::vector in_function_or_join_stack; + std::unordered_map replacement_map; + std::vector global_in_or_join_nodes; +}; + +/** Execute subquery node and put result in mutable context temporary table. + * Returns table node that is initialized with temporary table storage. + */ +QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, + ContextMutablePtr & mutable_context, + size_t subquery_depth) +{ + auto subquery_hash = subquery_node->getTreeHash(); + String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); + + const auto & external_tables = mutable_context->getExternalTables(); + auto external_table_it = external_tables.find(temporary_table_name); + if (external_table_it != external_tables.end()) + { + auto temporary_table_expression_node = std::make_shared(external_table_it->second, mutable_context); + temporary_table_expression_node->setTemporaryTableName(temporary_table_name); + return temporary_table_expression_node; + } + + auto subquery_ast = subquery_node->toAST(); + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); + auto context_copy = Context::createCopy(mutable_context); + updateContextForSubqueryExecution(context_copy); + + InterpreterSelectQueryAnalyzer interpreter(subquery_ast, context_copy, subquery_options); + + Block sample = interpreter.getSampleBlock(); + NamesAndTypesList columns = sample.getNamesAndTypesList(); + + auto external_storage_holder = TemporaryTableHolder( + mutable_context, + ColumnsDescription{columns}, + ConstraintsDescription{}, + nullptr /*query*/, + true /*create_for_global_subquery*/); + + StoragePtr external_storage = external_storage_holder.getTable(); + auto temporary_table_expression_node = std::make_shared(external_storage, mutable_context); + temporary_table_expression_node->setTemporaryTableName(temporary_table_name); + + auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); + auto io = interpreter.execute(); + io.pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); + + mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); + + return temporary_table_expression_node; +} + +QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, const StorageSnapshotPtr & distributed_storage_snapshot, const StorageID & remote_storage_id, const ASTPtr & remote_table_function) { - const auto & query_context = query_info.planner_context->getQueryContext(); + auto & planner_context = query_info.planner_context; + const auto & query_context = planner_context->getQueryContext(); + + std::optional table_expression_modifiers; + + if (auto * query_info_table_node = query_info.table_expression->as()) + table_expression_modifiers = query_info_table_node->getTableExpressionModifiers(); + else if (auto * query_info_table_function_node = query_info.table_expression->as()) + table_expression_modifiers = query_info_table_function_node->getTableExpressionModifiers(); QueryTreeNodePtr replacement_table_expression; @@ -651,6 +913,9 @@ QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const Sele auto table_function_node = std::make_shared(remote_table_function_node.getFunctionName()); table_function_node->getArgumentsNode() = remote_table_function_node.getArgumentsNode(); + if (table_expression_modifiers) + table_function_node->setTableExpressionModifiers(*table_expression_modifiers); + QueryAnalysisPass query_analysis_pass; query_analysis_pass.run(table_function_node, query_context); @@ -660,13 +925,85 @@ QueryTreeNodePtr buildQueryTreeDistributedTableReplacedWithLocalTable(const Sele { auto resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); + auto table_node = std::make_shared(std::move(storage), query_context); - replacement_table_expression = std::make_shared(std::move(storage), query_context); + if (table_expression_modifiers) + table_node->setTableExpressionModifiers(*table_expression_modifiers); + + replacement_table_expression = std::move(table_node); } replacement_table_expression->setAlias(query_info.table_expression->getAlias()); - return query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); + auto query_tree_to_modify = query_info.query_tree->cloneAndReplace(query_info.table_expression, std::move(replacement_table_expression)); + + CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor; + collect_column_source_to_columns_visitor.visit(query_tree_to_modify); + + const auto & column_source_to_columns = collect_column_source_to_columns_visitor.getColumnSourceToColumns(); + + DistributedProductModeRewriteInJoinVisitor visitor(query_info.planner_context->getQueryContext()); + visitor.visit(query_tree_to_modify); + + auto replacement_map = visitor.getReplacementMap(); + const auto & global_in_or_join_nodes = visitor.getGlobalInOrJoinNodes(); + + for (const auto & global_in_or_join_node : global_in_or_join_nodes) + { + if (auto * join_node = global_in_or_join_node.query_node->as()) + { + auto join_right_table_expression = join_node->getRightTableExpression(); + auto join_right_table_expression_node_type = join_right_table_expression->getNodeType(); + + QueryTreeNodePtr subquery_node; + + if (join_right_table_expression_node_type == QueryTreeNodeType::QUERY || + join_right_table_expression_node_type == QueryTreeNodeType::UNION) + { + subquery_node = join_right_table_expression; + } + else if (join_right_table_expression_node_type == QueryTreeNodeType::TABLE || + join_right_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION) + { + const auto & columns = column_source_to_columns.at(join_right_table_expression).columns; + subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns, + join_right_table_expression, + planner_context->getQueryContext()); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected JOIN righ table expression to be table, table function, query or union node. Actual {}", + join_right_table_expression->formatASTForErrorMessage()); + } + + auto temporary_table_expression_node = executeSubqueryNode(subquery_node, + planner_context->getMutableQueryContext(), + global_in_or_join_node.subquery_depth); + temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); + replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); + continue; + } + else if (auto * in_function_node = global_in_or_join_node.query_node->as()) + { + auto & in_function_subquery_node = in_function_node->getArguments().getNodes().at(1); + auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, + planner_context->getMutableQueryContext(), + global_in_or_join_node.subquery_depth); + in_function_subquery_node = std::move(temporary_table_expression_node); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected global IN or JOIN query node. Actual {}", + global_in_or_join_node.query_node->formatASTForErrorMessage()); + } + } + + if (!replacement_map.empty()) + query_tree_to_modify = query_tree_to_modify->cloneAndReplace(replacement_map); + + return query_tree_to_modify; } } @@ -694,17 +1031,13 @@ void StorageDistributed::read( if (!remote_table_function_ptr) remote_storage_id = StorageID{remote_database, remote_table}; - auto query_tree_with_replaced_distributed_table = buildQueryTreeDistributedTableReplacedWithLocalTable(query_info, + auto query_tree_distributed = buildQueryTreeDistributed(query_info, storage_snapshot, remote_storage_id, remote_table_function_ptr); - query_ast = queryNodeToSelectQuery(query_tree_with_replaced_distributed_table); - - Planner planner(query_tree_with_replaced_distributed_table, SelectQueryOptions(processed_stage).analyze()); - planner.buildQueryPlanIfNeeded(); - - header = planner.getQueryPlan().getCurrentDataStream().header; + query_ast = queryNodeToSelectQuery(query_tree_distributed); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); } else { diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index aadd7b8c20a..d9bb189524c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -223,8 +224,12 @@ void StorageMergeTree::read( auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); - Block header = - InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + Block header; + + if (local_context->getSettingsRef().allow_experimental_analyzer) + header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + else + header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 1a7050b4dff..a5c908352ee 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -167,7 +167,7 @@ void StorageView::read( query_plan.addStep(std::move(materializing)); /// And also convert to expected structure. - const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names,parameter_values); + const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names, parameter_values); const auto & header = query_plan.getCurrentDataStream().header; const auto * select_with_union = current_inner_query->as(); diff --git a/tests/queries/0_stateless/00858_issue_4756.reference b/tests/queries/0_stateless/00858_issue_4756.reference index d00491fd7e5..e8183f05f5d 100644 --- a/tests/queries/0_stateless/00858_issue_4756.reference +++ b/tests/queries/0_stateless/00858_issue_4756.reference @@ -1 +1,3 @@ 1 +1 +1 diff --git a/tests/queries/0_stateless/00858_issue_4756.sql b/tests/queries/0_stateless/00858_issue_4756.sql index 3da0766c4e9..9eacd5ef364 100644 --- a/tests/queries/0_stateless/00858_issue_4756.sql +++ b/tests/queries/0_stateless/00858_issue_4756.sql @@ -1,3 +1,4 @@ +set allow_experimental_analyzer = 1; set distributed_product_mode = 'local'; drop table if exists shard1; @@ -21,7 +22,7 @@ where distr1.id in from distr1 join distr2 on distr1.id = distr2.id where distr1.id > 0 -); -- { serverError 288 } +); select distinct(d0.id) from distr1 d0 where d0.id in @@ -32,15 +33,14 @@ where d0.id in where d1.id > 0 ); --- TODO ---select distinct(distr1.id) from distr1 ---where distr1.id in ---( --- select distr1.id --- from distr1 as d1 --- join distr2 as d2 on distr1.id = distr2.id --- where distr1.id > 0 ---); +select distinct(distr1.id) from distr1 +where distr1.id in +( + select distr1.id + from distr1 as d1 + join distr2 as d2 on distr1.id = distr2.id + where distr1.id > 0 +); drop table shard1; drop table shard2; diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.reference b/tests/queries/0_stateless/02420_final_setting_analyzer.reference index ee7c2541bcf..9a03c484765 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.reference +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.reference @@ -108,9 +108,6 @@ select left_table.id,val_left, val_middle, val_right from left_table ORDER BY left_table.id, val_left, val_middle, val_right; 1 c a c 1 c b c --- no distributed tests because it is not currently supported: --- JOIN with remote storages is unsupported. - -- Quite exotic with Merge engine DROP TABLE IF EXISTS table_to_merge_a; DROP TABLE IF EXISTS table_to_merge_b; diff --git a/tests/queries/0_stateless/02420_final_setting_analyzer.sql b/tests/queries/0_stateless/02420_final_setting_analyzer.sql index 5937e536239..14c832cfaf5 100644 --- a/tests/queries/0_stateless/02420_final_setting_analyzer.sql +++ b/tests/queries/0_stateless/02420_final_setting_analyzer.sql @@ -79,9 +79,6 @@ select left_table.id,val_left, val_middle, val_right from left_table inner join (SELECT * FROM right_table WHERE id = 1) r on middle_table.id = r.id ORDER BY left_table.id, val_left, val_middle, val_right; --- no distributed tests because it is not currently supported: --- JOIN with remote storages is unsupported. - -- Quite exotic with Merge engine DROP TABLE IF EXISTS table_to_merge_a; DROP TABLE IF EXISTS table_to_merge_b; From 677408e02ece0be3d39c3a77672a054c0a529de6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Mar 2023 19:03:40 +0100 Subject: [PATCH 439/470] Fixed style check --- src/Analyzer/Utils.cpp | 1 + src/Storages/StorageDistributed.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 5c1873bd6e4..aec4031d136 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -32,6 +32,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } bool isNodePartOfTree(const IQueryTreeNode * node, const IQueryTreeNode * root) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 9f84e7d210a..a16a98edc0a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -973,7 +973,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, else { throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected JOIN righ table expression to be table, table function, query or union node. Actual {}", + "Expected JOIN right table expression to be table, table function, query or union node. Actual {}", join_right_table_expression->formatASTForErrorMessage()); } From 0358cb36d878ca7744ec86f56e4d2c6b1c1c2f23 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 4 Mar 2023 18:46:40 +0100 Subject: [PATCH 440/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 18 +-- src/Analyzer/TableNode.cpp | 17 ++- src/Analyzer/Utils.cpp | 19 +++ src/Analyzer/Utils.h | 3 + src/Analyzer/WindowNode.cpp | 14 +- src/Interpreters/FullSortingMergeJoin.h | 4 + .../InterpreterSelectQueryAnalyzer.cpp | 6 + .../InterpreterSelectQueryAnalyzer.h | 2 + src/Planner/Planner.cpp | 9 +- src/Planner/PlannerJoinTree.cpp | 11 ++ src/Storages/HDFS/StorageHDFSCluster.cpp | 9 +- src/Storages/MergeTree/RPNBuilder.cpp | 2 +- src/Storages/StorageDistributed.cpp | 23 ++- src/Storages/StorageS3Cluster.cpp | 18 ++- src/Storages/StorageView.cpp | 5 + src/TableFunctions/TableFunctionMerge.cpp | 2 +- src/TableFunctions/TableFunctionURL.cpp | 20 +++ src/TableFunctions/TableFunctionURL.h | 7 +- .../00378_json_quote_64bit_integers.reference | 20 +-- .../00445_join_nullable_keys.reference | 10 +- .../0_stateless/00445_join_nullable_keys.sql | 1 + .../0_stateless/00722_inner_join.reference | 30 ++-- .../00853_join_with_nulls_crash.reference | 2 +- .../00853_join_with_nulls_crash.sql | 6 +- .../01010_pmj_right_table_memory_limits.sql | 30 +++- .../01056_predicate_optimizer_bugs.reference | 6 +- .../01056_predicate_optimizer_bugs.sql | 4 +- .../01291_aggregation_in_order.reference | 4 +- .../01291_aggregation_in_order.sql | 4 +- .../01455_opentelemetry_distributed.reference | 8 +- .../01455_opentelemetry_distributed.sh | 7 +- .../0_stateless/01457_int256_hashing.sql | 14 +- ..._explain_select_with_union_query.reference | 144 +++++++++--------- ...monotonous_functions_in_order_by.reference | 12 +- .../0_stateless/01596_full_join_chertus.sql | 4 +- .../0_stateless/01911_logical_error_minus.sql | 4 +- .../01913_names_of_tuple_literal.sql | 2 + .../02048_clickhouse_local_stage.reference | 6 +- ...2184_hash_functions_and_ip_types.reference | 100 ++++++------ .../02184_hash_functions_and_ip_types.sql | 2 + .../02227_union_match_by_name.reference | 72 +++++---- ...4_nothing_arguments_in_functions_errors.sh | 4 +- .../0_stateless/02303_query_kind.reference | 32 ++-- .../02381_join_dup_columns_in_plan.reference | 44 +++--- .../02402_external_disk_mertrics.sql | 3 +- .../02451_order_by_monotonic.reference | 32 ++-- ...cannot-find-column-in-projection.reference | 0 ...02521_cannot-find-column-in-projection.sql | 3 - ...cannot_find_column_in_projection.reference | 1 + ...02521_cannot_find_column_in_projection.sql | 7 + .../0_stateless/02579_fill_empty_chunk.sql | 2 + 51 files changed, 481 insertions(+), 328 deletions(-) delete mode 100644 tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference delete mode 100644 tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql create mode 100644 tests/queries/0_stateless/02521_cannot_find_column_in_projection.reference create mode 100644 tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5a726ab4bb5..7d60b843694 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1954,21 +1954,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden * * Example: SELECT (SELECT 2 AS x, x) */ - NameSet block_column_names; - size_t unique_column_name_counter = 1; - - for (auto & column_with_type : block) - { - if (!block_column_names.contains(column_with_type.name)) - { - block_column_names.insert(column_with_type.name); - continue; - } - - column_with_type.name += '_'; - column_with_type.name += std::to_string(unique_column_name_counter); - ++unique_column_name_counter; - } + makeUniqueColumnNamesInBlock(block); scalar_block.insert({ ColumnTuple::create(block.getColumns()), @@ -5062,7 +5048,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi /// Do not constant fold get scalar functions bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || - function_name == "shardCount"; + function_name == "shardCount" || function_name == "hostName"; /** If function is suitable for constant folding try to convert it to constant. * Example: SELECT plus(1, 1); diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index a746986be04..f315d372bc9 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -61,12 +61,17 @@ bool TableNode::isEqualImpl(const IQueryTreeNode & rhs) const void TableNode::updateTreeHashImpl(HashState & state) const { - auto full_name = storage_id.getFullNameNotQuoted(); - state.update(full_name.size()); - state.update(full_name); - - state.update(temporary_table_name.size()); - state.update(temporary_table_name); + if (!temporary_table_name.empty()) + { + state.update(temporary_table_name.size()); + state.update(temporary_table_name); + } + else + { + auto full_name = storage_id.getFullNameNotQuoted(); + state.update(full_name.size()); + state.update(full_name); + } if (table_expression_modifiers) table_expression_modifiers->updateTreeHash(state); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index aec4031d136..16f2e5b75e6 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -130,6 +130,25 @@ std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & fu throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid local IN function name {}", function_name); } +void makeUniqueColumnNamesInBlock(Block & block) +{ + NameSet block_column_names; + size_t unique_column_name_counter = 1; + + for (auto & column_with_type : block) + { + if (!block_column_names.contains(column_with_type.name)) + { + block_column_names.insert(column_with_type.name); + continue; + } + + column_with_type.name += '_'; + column_with_type.name += std::to_string(unique_column_name_counter); + ++unique_column_name_counter; + } +} + QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, const DataTypePtr & type, const ContextPtr & context, diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 9132a1d72bc..e3357c21bf2 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -22,6 +22,9 @@ bool isNameOfGlobalInFunction(const std::string & function_name); /// Returns global in function name for local in function name std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & function_name); +/// Add unique suffix to names of duplicate columns in block +void makeUniqueColumnNamesInBlock(Block & block); + /** Build cast function that cast expression into type. * If resolve = true, then result cast function is resolved during build, otherwise * result cast function is not resolved during build. diff --git a/src/Analyzer/WindowNode.cpp b/src/Analyzer/WindowNode.cpp index 3e8537302e5..d516f7a58b8 100644 --- a/src/Analyzer/WindowNode.cpp +++ b/src/Analyzer/WindowNode.cpp @@ -113,11 +113,17 @@ ASTPtr WindowNode::toASTImpl() const window_definition->parent_window_name = parent_window_name; - window_definition->children.push_back(getPartitionByNode()->toAST()); - window_definition->partition_by = window_definition->children.back(); + if (hasPartitionBy()) + { + window_definition->children.push_back(getPartitionByNode()->toAST()); + window_definition->partition_by = window_definition->children.back(); + } - window_definition->children.push_back(getOrderByNode()->toAST()); - window_definition->order_by = window_definition->children.back(); + if (hasOrderBy()) + { + window_definition->children.push_back(getOrderByNode()->toAST()); + window_definition->order_by = window_definition->children.back(); + } window_definition->frame_is_default = window_frame.is_default; window_definition->frame_type = window_frame.type; diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index fa7d0478535..a94d7a7dfc6 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -44,6 +44,10 @@ public: const auto & on_expr = table_join->getOnlyClause(); bool support_conditions = !on_expr.on_filter_condition_left && !on_expr.on_filter_condition_right; + if (!on_expr.analyzer_left_filter_condition_column_name.empty() || + !on_expr.analyzer_right_filter_condition_column_name.empty()) + support_conditions = false; + /// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it bool support_using_and_nulls = !table_join->hasUsing() || !table_join->joinUseNulls(); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 0536ee10f7c..98f70c25dcd 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -226,6 +226,12 @@ BlockIO InterpreterSelectQueryAnalyzer::execute() return result; } +QueryPlan & InterpreterSelectQueryAnalyzer::getQueryPlan() +{ + planner.buildQueryPlanIfNeeded(); + return planner.getQueryPlan(); +} + QueryPlan && InterpreterSelectQueryAnalyzer::extractQueryPlan() && { planner.buildQueryPlanIfNeeded(); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 681a9cfe5a3..2c8af49cf0e 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -51,6 +51,8 @@ public: BlockIO execute() override; + QueryPlan & getQueryPlan(); + QueryPlan && extractQueryPlan() &&; QueryPipelineBuilder buildQueryPipeline(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2ce470d9ecf..37a4614bad3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -214,9 +214,14 @@ public: { /// Constness of limit is validated during query analysis stage limit_length = query_node.getLimit()->as().getValue().safeGet(); - } - if (query_node.hasOffset()) + if (query_node.hasOffset() && limit_length) + { + /// Constness of offset is validated during query analysis stage + limit_offset = query_node.getOffset()->as().getValue().safeGet(); + } + } + else if (query_node.hasOffset()) { /// Constness of offset is validated during query analysis stage limit_offset = query_node.getOffset()->as().getValue().safeGet(); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index edea51aba45..52a3c926f65 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -465,6 +465,17 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); storage->read(query_plan, columns_names, storage_snapshot, table_expression_query_info, query_context, from_stage, max_block_size, max_streams); + + if (query_context->hasQueryContext() && !select_query_options.is_internal) + { + auto local_storage_id = storage->getStorageID(); + query_context->getQueryContext()->addQueryAccessInfo( + backQuoteIfNeed(local_storage_id.getDatabaseName()), + local_storage_id.getFullTableName(), + columns_names, + {}, + {}); + } } if (query_plan.isInitialized()) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 8dbaa0796e9..a88470d01c7 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -83,8 +84,12 @@ Pipe StorageHDFSCluster::read( auto extension = getTaskIteratorExtension(query_info.query, context); /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - Block header = - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + Block header; + + if (context->getSettingsRef().allow_experimental_analyzer) + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); + else + header = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index cee5038ed21..fb3592a1541 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -59,7 +59,7 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o { auto name = node.function_base->getName(); if (legacy && name == "modulo") - writeCString("moduleLegacy", out); + writeCString("moduloLegacy", out); else writeString(name, out); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a16a98edc0a..77972b67644 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -729,7 +729,7 @@ public: return global_in_or_join_nodes; } - bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) { auto * function_node = parent->as(); if (function_node && isNameOfGlobalInFunction(function_node->getFunctionName())) @@ -856,12 +856,25 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, return temporary_table_expression_node; } - auto subquery_ast = subquery_node->toAST(); auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth, true /*is_subquery*/); auto context_copy = Context::createCopy(mutable_context); updateContextForSubqueryExecution(context_copy); - InterpreterSelectQueryAnalyzer interpreter(subquery_ast, context_copy, subquery_options); + InterpreterSelectQueryAnalyzer interpreter(subquery_node, context_copy, subquery_options); + auto & query_plan = interpreter.getQueryPlan(); + + auto sample_block_with_unique_names = query_plan.getCurrentDataStream().header; + makeUniqueColumnNamesInBlock(sample_block_with_unique_names); + + if (!blocksHaveEqualStructure(sample_block_with_unique_names, query_plan.getCurrentDataStream().header)) + { + auto actions_dag = ActionsDAG::makeConvertingActions( + query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), + sample_block_with_unique_names.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(actions_dag)); + query_plan.addStep(std::move(converting_step)); + } Block sample = interpreter.getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); @@ -987,6 +1000,10 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, else if (auto * in_function_node = global_in_or_join_node.query_node->as()) { auto & in_function_subquery_node = in_function_node->getArguments().getNodes().at(1); + auto in_function_node_type = in_function_subquery_node->getNodeType(); + if (in_function_node_type != QueryTreeNodeType::QUERY && in_function_node_type != QueryTreeNodeType::UNION) + continue; + auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index c7535bb4550..92be3f50150 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -102,7 +103,19 @@ Pipe StorageS3Cluster::read( auto extension = getTaskIteratorExtension(query_info.query, context); /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) - auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); + + Block sample_block; + ASTPtr query_to_send = query_info.query; + + if (context->getSettingsRef().allow_experimental_analyzer) + { + sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query_info.query, context, SelectQueryOptions(processed_stage)); + } + else + { + auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); + query_to_send = interpreter.getQueryInfo().query->clone(); + } const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; @@ -110,7 +123,6 @@ Pipe StorageS3Cluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - ASTPtr query_to_send = interpreter.getQueryInfo().query->clone(); if (!structure_argument_was_provided) addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); @@ -136,7 +148,7 @@ Pipe StorageS3Cluster::read( shard_info.pool, std::vector{try_result}, queryToString(query_to_send), - interpreter.getSampleBlock(), + sample_block, context, /*throttler=*/nullptr, scalars, diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index a5c908352ee..7e12a972768 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -117,6 +118,10 @@ StorageView::StorageView( SelectQueryDescription description; description.inner_query = query.select->ptr(); + + NormalizeSelectWithUnionQueryVisitor::Data data{SetOperationMode::Unspecified}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(description.inner_query); + is_parameterized_view = query.isParameterizedView(); parameter_types = analyzeReceiveQueryParamsWithType(description.inner_query); storage_metadata.setSelectQuery(description); diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 066caa8170d..586cee54085 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -53,7 +53,7 @@ std::vector TableFunctionMerge::skipAnalysisForArguments(const QueryTree result.push_back(i); } - return {0}; + return result; } void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr context) diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 5de6c6b4ccc..cc3a858e4dc 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include #include #include @@ -26,6 +28,24 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +std::vector TableFunctionURL::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +{ + auto & table_function_node = query_node_table_function->as(); + auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); + size_t table_function_arguments_size = table_function_arguments_nodes.size(); + + std::vector result; + + for (size_t i = 0; i < table_function_arguments_size; ++i) + { + auto * function_node = table_function_arguments_nodes[i]->as(); + if (function_node && function_node->getFunctionName() == "headers") + result.push_back(i); + } + + return result; +} + void TableFunctionURL::parseArguments(const ASTPtr & ast, ContextPtr context) { const auto & ast_function = assert_cast(ast.get()); diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index a670bdc0682..dca5123fb69 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -12,7 +12,7 @@ class Context; /* url(source, format[, structure, compression]) - creates a temporary storage from url. */ -class TableFunctionURL : public ITableFunctionFileLike +class TableFunctionURL final: public ITableFunctionFileLike { public: static constexpr auto name = "url"; @@ -23,10 +23,11 @@ public: ColumnsDescription getActualTableStructure(ContextPtr context) const override; -protected: +private: + std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; + void parseArguments(const ASTPtr & ast, ContextPtr context) override; -private: StoragePtr getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const override; diff --git a/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference b/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference index 5174c13a9e0..b8d51e5d078 100644 --- a/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference +++ b/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference @@ -48,10 +48,10 @@ { "i0": "0", "u0": "0", - "ip": "9223372036854775807", - "in": "-9223372036854775808", - "up": "18446744073709551615", - "arr": ["0"], + "ip": "0", + "in": "0", + "up": "0", + "arr": [], "tuple": ["0","0"] }, @@ -119,7 +119,7 @@ ["0", "0", "9223372036854775807", "-9223372036854775808", "18446744073709551615", ["0"], ["0","0"]] ], - "totals": ["0", "0", "9223372036854775807", "-9223372036854775808", "18446744073709551615", ["0"], ["0","0"]], + "totals": ["0", "0", "0", "0", "0", [], ["0","0"]], "extremes": { @@ -180,10 +180,10 @@ { "i0": 0, "u0": 0, - "ip": 9223372036854775807, - "in": -9223372036854775808, - "up": 18446744073709551615, - "arr": [0], + "ip": 0, + "in": 0, + "up": 0, + "arr": [], "tuple": [0,0] }, @@ -251,7 +251,7 @@ [0, 0, 9223372036854775807, -9223372036854775808, 18446744073709551615, [0], [0,0]] ], - "totals": [0, 0, 9223372036854775807, -9223372036854775808, 18446744073709551615, [0], [0,0]], + "totals": [0, 0, 0, 0, 0, [], [0,0]], "extremes": { diff --git a/tests/queries/0_stateless/00445_join_nullable_keys.reference b/tests/queries/0_stateless/00445_join_nullable_keys.reference index afc8003910c..cc1c06d593b 100644 --- a/tests/queries/0_stateless/00445_join_nullable_keys.reference +++ b/tests/queries/0_stateless/00445_join_nullable_keys.reference @@ -22,13 +22,13 @@ 13 13 14 14 \N 8 -0 0 -0 2 -0 4 -0 6 -0 8 1 1 3 3 5 5 7 7 9 9 +\N 0 +\N 2 +\N 4 +\N 6 +\N 8 diff --git a/tests/queries/0_stateless/00445_join_nullable_keys.sql b/tests/queries/0_stateless/00445_join_nullable_keys.sql index a0453356e98..774594f90f3 100644 --- a/tests/queries/0_stateless/00445_join_nullable_keys.sql +++ b/tests/queries/0_stateless/00445_join_nullable_keys.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET join_use_nulls = 0; SET any_join_distinct_right_table_keys = 1; diff --git a/tests/queries/0_stateless/00722_inner_join.reference b/tests/queries/0_stateless/00722_inner_join.reference index 86c07e6e84e..b5e8a77a20d 100644 --- a/tests/queries/0_stateless/00722_inner_join.reference +++ b/tests/queries/0_stateless/00722_inner_join.reference @@ -16,24 +16,24 @@ ┌─x──────┬─name─┐ │ system │ one │ └────────┴──────┘ -┌─database─┬─t.name─┐ -│ system │ one │ -└──────────┴────────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ ┌─db.x───┬─name─┐ │ system │ one │ └────────┴──────┘ -┌─db.name─┬─t.name─┐ -│ system │ one │ -└─────────┴────────┘ -┌─db.name─┬─t.name─┐ -│ system │ one │ -└─────────┴────────┘ -┌─t.database─┬─t.name─┐ -│ system │ one │ -└────────────┴────────┘ -┌─database─┬─t.name─┐ -│ system │ one │ -└──────────┴────────┘ +┌─db.name─┬─name─┐ +│ system │ one │ +└─────────┴──────┘ +┌─db.name─┬─name─┐ +│ system │ one │ +└─────────┴──────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ +┌─database─┬─name─┐ +│ system │ one │ +└──────────┴──────┘ 2 2 2 diff --git a/tests/queries/0_stateless/00853_join_with_nulls_crash.reference b/tests/queries/0_stateless/00853_join_with_nulls_crash.reference index 459b73acdbf..5df14d02d5e 100644 --- a/tests/queries/0_stateless/00853_join_with_nulls_crash.reference +++ b/tests/queries/0_stateless/00853_join_with_nulls_crash.reference @@ -15,5 +15,5 @@ bar bar 1 2 String Nullable(String) \N 0 1 String Nullable(String) foo 2 0 String bar 1 2 String -test 0 1 String + 0 1 String 0 1 String diff --git a/tests/queries/0_stateless/00853_join_with_nulls_crash.sql b/tests/queries/0_stateless/00853_join_with_nulls_crash.sql index c63c2d99cba..b620b8a7902 100644 --- a/tests/queries/0_stateless/00853_join_with_nulls_crash.sql +++ b/tests/queries/0_stateless/00853_join_with_nulls_crash.sql @@ -27,7 +27,7 @@ SELECT s1.other, s2.other, count_a, count_b, toTypeName(s1.other), toTypeName(s2 ( SELECT other, count() AS count_a FROM table_a GROUP BY other ) s1 ALL FULL JOIN ( SELECT other, count() AS count_b FROM table_b GROUP BY other ) s2 -USING other +ON s1.other = s2.other ORDER BY s2.other DESC, count_a, s1.other; SELECT s1.something, s2.something, count_a, count_b, toTypeName(s1.something), toTypeName(s2.something) FROM @@ -41,7 +41,7 @@ SELECT s1.something, s2.something, count_a, count_b, toTypeName(s1.something), t ( SELECT something, count() AS count_a FROM table_a GROUP BY something ) s1 ALL RIGHT JOIN ( SELECT something, count() AS count_b FROM table_b GROUP BY something ) s2 -USING (something) +ON s1.something = s2.something ORDER BY count_a DESC, s1.something, s2.something; SET joined_subquery_requires_alias = 0; @@ -50,7 +50,7 @@ SELECT something, count_a, count_b, toTypeName(something) FROM ( SELECT something, count() AS count_a FROM table_a GROUP BY something ) as s1 ALL FULL JOIN ( SELECT something, count() AS count_b FROM table_b GROUP BY something ) as s2 -USING (something) +ON s1.something = s2.something ORDER BY count_a DESC, something DESC; DROP TABLE table_a; 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 7804ce32a5a..f9f30b44700 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 @@ -3,7 +3,10 @@ SET max_memory_usage = 32000000; SET join_on_disk_max_files_to_merge = 4; -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) @@ -13,14 +16,20 @@ USING n; -- { serverError 241 } SET join_algorithm = 'partial_merge'; SET default_max_bytes_in_join = 0; -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) ) js2 USING n; -- { serverError 12 } -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) @@ -28,7 +37,10 @@ ANY LEFT JOIN ( USING n SETTINGS max_bytes_in_join = 30000000; -- { serverError 241 } -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) @@ -39,7 +51,10 @@ SETTINGS max_bytes_in_join = 10000000; SET partial_merge_join_optimizations = 1; -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) @@ -50,7 +65,10 @@ SETTINGS max_rows_in_join = 100000; SET default_max_bytes_in_join = 10000000; -SELECT number * 200000 as n, j FROM numbers(5) nums +SELECT n, j FROM +( + SELECT number * 200000 as n FROM numbers(5) +) nums JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 7df4dc7ead5..dd52fd1c1b2 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -50,12 +50,12 @@ FROM WHERE (co != 0) AND (co2 != 2) 1 0 3 1 1 0 0 1 -SELECT alias AS name +SELECT name FROM ( - SELECT name AS alias + SELECT name FROM system.settings - WHERE alias = \'enable_optimize_predicate_expression\' + WHERE name = \'enable_optimize_predicate_expression\' ) ANY INNER JOIN ( diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 6d2bb2964d6..f237768bc4c 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -12,8 +12,8 @@ EXPLAIN SYNTAX SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FRO SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; -- https://github.com/ClickHouse/ClickHouse/issues/6734 -EXPLAIN SYNTAX SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; -SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; +EXPLAIN SYNTAX SELECT name FROM ( SELECT name FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; +SELECT name FROM ( SELECT name FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; -- https://github.com/ClickHouse/ClickHouse/issues/6767 DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/01291_aggregation_in_order.reference b/tests/queries/0_stateless/01291_aggregation_in_order.reference index c072a8aed3e..cf058b9f2f5 100644 --- a/tests/queries/0_stateless/01291_aggregation_in_order.reference +++ b/tests/queries/0_stateless/01291_aggregation_in_order.reference @@ -22,8 +22,8 @@ 2 4 109 2 1 619 1 2 537 2 -1 619 1 -2 537 2 +-2 537 2 +-1 619 1 2019-05-05 00:00:00 -45363190 2019-05-05 00:00:00 -1249512288 2019-05-05 00:00:00 345522721 diff --git a/tests/queries/0_stateless/01291_aggregation_in_order.sql b/tests/queries/0_stateless/01291_aggregation_in_order.sql index c4357811520..e93eadc3329 100644 --- a/tests/queries/0_stateless/01291_aggregation_in_order.sql +++ b/tests/queries/0_stateless/01291_aggregation_in_order.sql @@ -14,7 +14,7 @@ SELECT a FROM pk_order GROUP BY a ORDER BY a; SELECT a, b, sum(c), avg(d) FROM pk_order GROUP BY a, b ORDER BY a, b; SELECT a, sum(c), avg(d) FROM pk_order GROUP BY a ORDER BY a; -SELECT a, sum(c), avg(d) FROM pk_order GROUP BY -a ORDER BY a; +SELECT -a, sum(c), avg(d) FROM pk_order GROUP BY -a ORDER BY -a; DROP TABLE IF EXISTS pk_order; @@ -27,7 +27,7 @@ INSERT INTO pk_order set max_block_size = 1; SELECT d, max(b) FROM pk_order GROUP BY d, a ORDER BY d, a LIMIT 5; -SELECT d, avg(a) FROM pk_order GROUP BY toString(d) ORDER BY toString(d) LIMIT 5; +SELECT toString(d), avg(a) FROM pk_order GROUP BY toString(d) ORDER BY toString(d) LIMIT 5; SELECT toStartOfHour(d) as d1, min(a), max(b) FROM pk_order GROUP BY d1 ORDER BY d1 LIMIT 5; DROP TABLE pk_order; diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference index d48b3738bc2..0729dd7b881 100644 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.reference +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.reference @@ -1,10 +1,10 @@ ===http=== -{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} +{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} -{"query":"SELECT 1 FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one`","status":"QueryFinish","tracestate":"some custom state","sorted_by_start_time":1} {"query":"DESC TABLE system.one","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} -{"query":"SELECT 1 FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} -{"query":"select 1 from remote('127.0.0.2', system, one) format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} +{"query":"SELECT 1 AS `1` FROM `system`.`one`","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} +{"query":"select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null\n","query_status":"QueryFinish","tracestate":"some custom state","sorted_by_finish_time":1} {"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"} {"initial query spans with proper parent":"1"} {"unique non-empty tracestate values":"1"} diff --git a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh index b2b5ae89105..0dfec6097db 100755 --- a/tests/queries/0_stateless/01455_opentelemetry_distributed.sh +++ b/tests/queries/0_stateless/01455_opentelemetry_distributed.sh @@ -12,6 +12,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function check_log { ${CLICKHOUSE_CLIENT} --format=JSONEachRow -nq " +set allow_experimental_analyzer = 1; system flush logs; -- Show queries sorted by start time. @@ -55,7 +56,7 @@ select count(*) "'"'"initial query spans with proper parent"'"'" where trace_id = UUIDNumToString(toFixedString(unhex('$trace_id'), 16)) and operation_name = 'query' - and parent_span_id in ( + and parent_span_id in ( select span_id from system.opentelemetry_span_log where trace_id = UUIDNumToString(toFixedString(unhex('$trace_id'), 16)) and parent_span_id = reinterpretAsUInt64(unhex('73')) ) ; @@ -76,7 +77,7 @@ select uniqExact(value) "'"'"unique non-empty tracestate values"'"'" # Generate some random trace id so that the prevous runs of the test do not interfere. echo "===http===" -trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") +trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4())))) settings allow_experimental_analyzer = 1") # Check that the HTTP traceparent is read, and then passed through `remote` # table function. We expect 4 queries -- one initial, one SELECT and two @@ -86,7 +87,7 @@ ${CLICKHOUSE_CURL} \ --header "traceparent: 00-$trace_id-0000000000000073-01" \ --header "tracestate: some custom state" "$CLICKHOUSE_URL" \ --get \ - --data-urlencode "query=select 1 from remote('127.0.0.2', system, one) format Null" + --data-urlencode "query=select 1 from remote('127.0.0.2', system, one) settings allow_experimental_analyzer = 1 format Null" check_log diff --git a/tests/queries/0_stateless/01457_int256_hashing.sql b/tests/queries/0_stateless/01457_int256_hashing.sql index 861e567950a..510d25f6b8c 100644 --- a/tests/queries/0_stateless/01457_int256_hashing.sql +++ b/tests/queries/0_stateless/01457_int256_hashing.sql @@ -1,13 +1,15 @@ -- Tags: no-fasttest +SET joined_subquery_requires_alias = 0; + SELECT toUInt256(123) IN (NULL); SELECT toUInt256(123) AS k GROUP BY k; -SELECT toUInt256(123) AS k FROM system.one INNER JOIN (SELECT toUInt256(123) AS k) t USING k; +SELECT k FROM (SELECT toUInt256(123) AS k FROM system.one) INNER JOIN (SELECT toUInt256(123) AS k) t USING k; SELECT arrayEnumerateUniq([toUInt256(123), toUInt256(456), toUInt256(123)]); SELECT toInt256(123) IN (NULL); SELECT toInt256(123) AS k GROUP BY k; -SELECT toInt256(123) AS k FROM system.one INNER JOIN (SELECT toInt256(123) AS k) t USING k; +SELECT k FROM (SELECT toInt256(123) AS k FROM system.one) INNER JOIN (SELECT toInt256(123) AS k) t USING k; SELECT arrayEnumerateUniq([toInt256(123), toInt256(456), toInt256(123)]); -- SELECT toUInt128(123) IN (NULL); @@ -17,17 +19,17 @@ SELECT arrayEnumerateUniq([toInt256(123), toInt256(456), toInt256(123)]); SELECT toInt128(123) IN (NULL); SELECT toInt128(123) AS k GROUP BY k; -SELECT toInt128(123) AS k FROM system.one INNER JOIN (SELECT toInt128(123) AS k) t USING k; +SELECT k FROM (SELECT toInt128(123) AS k FROM system.one) INNER JOIN (SELECT toInt128(123) AS k) t USING k; SELECT arrayEnumerateUniq([toInt128(123), toInt128(456), toInt128(123)]); SELECT toNullable(toUInt256(321)) IN (NULL); SELECT toNullable(toUInt256(321)) AS k GROUP BY k; -SELECT toNullable(toUInt256(321)) AS k FROM system.one INNER JOIN (SELECT toUInt256(321) AS k) t USING k; +SELECT k FROM (SELECT toNullable(toUInt256(321)) AS k FROM system.one) INNER JOIN (SELECT toUInt256(321) AS k) t USING k; SELECT arrayEnumerateUniq([toNullable(toUInt256(321)), toNullable(toUInt256(456)), toNullable(toUInt256(321))]); SELECT toNullable(toInt256(321)) IN (NULL); SELECT toNullable(toInt256(321)) AS k GROUP BY k; -SELECT toNullable(toInt256(321)) AS k FROM system.one INNER JOIN (SELECT toInt256(321) AS k) t USING k; +SELECT k FROM (SELECT toNullable(toInt256(321)) AS k FROM system.one) INNER JOIN (SELECT toInt256(321) AS k) t USING k; SELECT arrayEnumerateUniq([toNullable(toInt256(321)), toNullable(toInt256(456)), toNullable(toInt256(321))]); -- SELECT toNullable(toUInt128(321)) IN (NULL); @@ -37,5 +39,5 @@ SELECT arrayEnumerateUniq([toNullable(toInt256(321)), toNullable(toInt256(456)), SELECT toNullable(toInt128(321)) IN (NULL); SELECT toNullable(toInt128(321)) AS k GROUP BY k; -SELECT toNullable(toInt128(321)) AS k FROM system.one INNER JOIN (SELECT toInt128(321) AS k) t USING k; +SELECT k FROM (SELECT toNullable(toInt128(321)) AS k FROM system.one) INNER JOIN (SELECT toInt128(321) AS k) t USING k; SELECT arrayEnumerateUniq([toNullable(toInt128(321)), toNullable(toInt128(456)), toNullable(toInt128(321))]); diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference index c18e6b70b0d..27cf4c129b1 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.reference +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.reference @@ -1,180 +1,180 @@ Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Distinct Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) Union - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) - Expression ((Projection + Before ORDER BY)) + Expression ((Project names + (Projection + Change column names to column identifiers))) ReadFromStorage (SystemOne) diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index 8c8bb73b801..49b4d22ea71 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -4,10 +4,10 @@ SELECT FROM test_order_by ORDER BY timestamp ASC LIMIT 10 -Expression (Projection) +Expression (Project names) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) + Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) ReadFromMergeTree (default.test_order_by) SELECT timestamp, @@ -15,10 +15,10 @@ SELECT FROM test_order_by ORDER BY toDate(timestamp) ASC LIMIT 10 -Expression (Projection) +Expression (Project names) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) + Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) ReadFromMergeTree (default.test_order_by) SELECT timestamp, @@ -28,10 +28,10 @@ ORDER BY toDate(timestamp) ASC, timestamp ASC LIMIT 10 -Expression (Projection) +Expression (Project names) Limit (preliminary LIMIT (without OFFSET)) Sorting (Sorting for ORDER BY) - Expression (Before ORDER BY) + Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) ReadFromMergeTree (default.test_order_by) SELECT timestamp, diff --git a/tests/queries/0_stateless/01596_full_join_chertus.sql b/tests/queries/0_stateless/01596_full_join_chertus.sql index 162b9f7be8f..32911abb792 100644 --- a/tests/queries/0_stateless/01596_full_join_chertus.sql +++ b/tests/queries/0_stateless/01596_full_join_chertus.sql @@ -1,9 +1,9 @@ select toTypeName(materialize(js1.k)), toTypeName(materialize(js2.k)), toTypeName(materialize(js1.s)), toTypeName(materialize(js2.s)) from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 full join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 -using k order by js1.k, js2.k; +ON js1.k = js2.k order by js1.k, js2.k; select toTypeName(js1.k), toTypeName(js2.k), toTypeName(js1.s), toTypeName(js2.s) from (select number k, toLowCardinality(toString(number)) s from numbers(2)) as js1 full join (select toLowCardinality(number+1) k, toString(number+1) s from numbers(2)) as js2 -using k order by js1.k, js2.k; +ON js1.k = js2.k order by js1.k, js2.k; diff --git a/tests/queries/0_stateless/01911_logical_error_minus.sql b/tests/queries/0_stateless/01911_logical_error_minus.sql index 3dcdedd38f5..7f371a463f8 100644 --- a/tests/queries/0_stateless/01911_logical_error_minus.sql +++ b/tests/queries/0_stateless/01911_logical_error_minus.sql @@ -26,7 +26,7 @@ INSERT INTO codecTest (key, name, ref_valueF64, valueF64, ref_valueF32, valueF32 INSERT INTO codecTest (key, name, ref_valueF64, valueF64, ref_valueF32, valueF32) SELECT number AS n, 'sin(n*n*n)*n', sin(n * n * n * n* n) AS v, v, v, v FROM system.numbers LIMIT 301, 100; -SELECT IF(-2, NULL, 0.00009999999747378752), IF(1048577, 1048576, NULL), c1.key, IF(1, NULL, NULL), c2.key FROM codecTest AS c1 , codecTest AS c2 WHERE ignore(IF(257, -2, NULL), arrayJoin([65537]), IF(3, 1024, 9223372036854775807)) AND IF(NULL, 256, NULL) AND (IF(NULL, '1048576', NULL) = (c1.key - NULL)) LIMIT 65535; +SELECT IF(2, NULL, 0.00009999999747378752), IF(104, 1048576, NULL), c1.key, IF(1, NULL, NULL), c2.key FROM codecTest AS c1 , codecTest AS c2 WHERE ignore(IF(255, -2, NULL), arrayJoin([65537]), IF(3, 1024, 9223372036854775807)) AND IF(NULL, 256, NULL) AND (IF(NULL, '1048576', NULL) = (c1.key - NULL)) LIMIT 65535; SELECT c1.key, c1.name, c1.ref_valueF64, c1.valueF64, c1.ref_valueF64 - c1.valueF64 AS dF64, '', c2.key, c2.ref_valueF64 FROM codecTest AS c1 , codecTest AS c2 WHERE (dF64 != 3) AND c1.valueF64 != 0 AND (c2.key = (c1.key - 1048576)) LIMIT 0; @@ -72,7 +72,7 @@ INSERT INTO codecTest (key, ref_valueU64, valueU64, ref_valueU32, valueU32, ref_ SELECT number as n, n + (rand64() - 9223372036854775807)/1000 as v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, v, toDateTime(v), toDateTime(v), toDate(v), toDate(v) FROM system.numbers LIMIT 3001, 1000; -SELECT IF(-2, NULL, 0.00009999999747378752), IF(1048577, 1048576, NULL), c1.key, IF(1, NULL, NULL), c2.key FROM codecTest AS c1 , codecTest AS c2 WHERE ignore(IF(257, -2, NULL), arrayJoin([65537]), IF(3, 1024, 9223372036854775807)) AND IF(NULL, 256, NULL) AND (IF(NULL, '1048576', NULL) = (c1.key - NULL)) LIMIT 65535; +SELECT IF(2, NULL, 0.00009999999747378752), IF(104, 1048576, NULL), c1.key, IF(1, NULL, NULL), c2.key FROM codecTest AS c1 , codecTest AS c2 WHERE ignore(IF(255, -2, NULL), arrayJoin([65537]), IF(3, 1024, 9223372036854775807)) AND IF(NULL, 256, NULL) AND (IF(NULL, '1048576', NULL) = (c1.key - NULL)) LIMIT 65535; DROP TABLE codecTest; diff --git a/tests/queries/0_stateless/01913_names_of_tuple_literal.sql b/tests/queries/0_stateless/01913_names_of_tuple_literal.sql index 09de9e8cf37..879f4c91587 100644 --- a/tests/queries/0_stateless/01913_names_of_tuple_literal.sql +++ b/tests/queries/0_stateless/01913_names_of_tuple_literal.sql @@ -1,2 +1,4 @@ +SET allow_experimental_analyzer = 0; + SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames; SELECT ((1, 2), (2, 3), (3, 4)) FORMAT TSVWithNames SETTINGS legacy_column_name_of_tuple_literal = 1; diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 44c39f2a444..676cca1e866 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -2,13 +2,13 @@ execute: default "foo" 1 execute: --stage fetch_columns -"dummy" +"system.one.dummy_0" 0 execute: --stage with_mergeable_state -"1" +"1_UInt8" 1 execute: --stage with_mergeable_state_after_aggregation -"1" +"1_UInt8" 1 execute: --stage complete "foo" diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference index 07705827428..b305806cd08 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.reference @@ -1,54 +1,54 @@ Row 1: ────── -ipv4: 1.2.3.4 -halfMD5(toIPv4('1.2.3.4')): 14356538739656272800 -farmFingerprint64(toIPv4('1.2.3.4')): 5715546585361069049 -xxh3(toIPv4('1.2.3.4')): 14355428563589734825 -wyHash64(toIPv4('1.2.3.4')): 13096729196120951355 -xxHash32(toIPv4('1.2.3.4')): 2430391091 -gccMurmurHash(toIPv4('1.2.3.4')): 5478801830569062645 -murmurHash2_32(toIPv4('1.2.3.4')): 1658978282 -javaHashUTF16LE(toIPv4('1.2.3.4')): 24190 -intHash64(toIPv4('1.2.3.4')): 5715546585361069049 -intHash32(toIPv4('1.2.3.4')): 3152671896 -metroHash64(toIPv4('1.2.3.4')): 5715546585361069049 -hex(murmurHash3_128(toIPv4('1.2.3.4'))): 549E9EF692591F6BB55874EF9A0DE88E -jumpConsistentHash(toIPv4('1.2.3.4'), 42): 37 -sipHash64(toIPv4('1.2.3.4')): 10711397536826262068 -hex(sipHash128(toIPv4('1.2.3.4'))): DBB6A76B92B59789EFB42185DC32311D -kostikConsistentHash(toIPv4('1.2.3.4'), 42): 0 -xxHash64(toIPv4('1.2.3.4')): 14496144933713060978 -murmurHash2_64(toIPv4('1.2.3.4')): 10829690723193326442 -cityHash64(toIPv4('1.2.3.4')): 5715546585361069049 -hiveHash(toIPv4('1.2.3.4')): 122110 -murmurHash3_64(toIPv4('1.2.3.4')): 16570805747704317665 -murmurHash3_32(toIPv4('1.2.3.4')): 1165084099 -yandexConsistentHash(toIPv4('1.2.3.4'), 42): 0 +ipv4: 1.2.3.4 +halfMD5(ipv4): 14356538739656272800 +farmFingerprint64(ipv4): 5715546585361069049 +xxh3(ipv4): 14355428563589734825 +wyHash64(ipv4): 13096729196120951355 +xxHash32(ipv4): 2430391091 +gccMurmurHash(ipv4): 5478801830569062645 +murmurHash2_32(ipv4): 1658978282 +javaHashUTF16LE(ipv4): 24190 +intHash64(ipv4): 5715546585361069049 +intHash32(ipv4): 3152671896 +metroHash64(ipv4): 5715546585361069049 +hex(murmurHash3_128(ipv4)): 549E9EF692591F6BB55874EF9A0DE88E +jumpConsistentHash(ipv4, 42): 37 +sipHash64(ipv4): 10711397536826262068 +hex(sipHash128(ipv4)): DBB6A76B92B59789EFB42185DC32311D +kostikConsistentHash(ipv4, 42): 0 +xxHash64(ipv4): 14496144933713060978 +murmurHash2_64(ipv4): 10829690723193326442 +cityHash64(ipv4): 5715546585361069049 +hiveHash(ipv4): 122110 +murmurHash3_64(ipv4): 16570805747704317665 +murmurHash3_32(ipv4): 1165084099 +yandexConsistentHash(ipv4, 42): 0 Row 1: ────── -ipv6: fe80::62:5aff:fed1:daf0 -halfMD5(toIPv6('fe80::62:5aff:fed1:daf0')): 9503062220758009199 -hex(MD4(toIPv6('fe80::62:5aff:fed1:daf0'))): E35A1A4FB3A3953421AB348B2E1A4A1A -hex(MD5(toIPv6('fe80::62:5aff:fed1:daf0'))): 83E1A8BD8AB7456FC229208409F79798 -hex(SHA1(toIPv6('fe80::62:5aff:fed1:daf0'))): A6D5DCE882AC44804382DE4639E6001612E1C8B5 -hex(SHA224(toIPv6('fe80::62:5aff:fed1:daf0'))): F6995FD7BED2BCA21F68DAC6BBABE742DC1BA177BA8594CEF1715C52 -hex(SHA256(toIPv6('fe80::62:5aff:fed1:daf0'))): F75497BAD6F7747BD6B150B6F69BA2DEE354F1C2A34B7BEA6183973B78640250 -hex(SHA512(toIPv6('fe80::62:5aff:fed1:daf0'))): 0C2893CCBF44BC19CCF339AEED5B68CBFD5A2EF38263A48FE21C3379BA4438E7FF7A02F59D7542442C6E6ED538E6D13D65D3573DADB381651D3D8A5DEA232EAC -farmFingerprint64(toIPv6('fe80::62:5aff:fed1:daf0')): 6643158734288374888 -javaHash(toIPv6('fe80::62:5aff:fed1:daf0')): 684606770 -xxh3(toIPv6('fe80::62:5aff:fed1:daf0')): 4051340969481364358 -wyHash64(toIPv6('fe80::62:5aff:fed1:daf0')): 18071806066582739916 -xxHash32(toIPv6('fe80::62:5aff:fed1:daf0')): 3353862080 -gccMurmurHash(toIPv6('fe80::62:5aff:fed1:daf0')): 11049311547848936878 -murmurHash2_32(toIPv6('fe80::62:5aff:fed1:daf0')): 1039121047 -javaHashUTF16LE(toIPv6('fe80::62:5aff:fed1:daf0')): -666938696 -metroHash64(toIPv6('fe80::62:5aff:fed1:daf0')): 15333045864940909774 -hex(sipHash128(toIPv6('fe80::62:5aff:fed1:daf0'))): 31D50562F877B1F92A99B05B646568B7 -hex(murmurHash3_128(toIPv6('fe80::62:5aff:fed1:daf0'))): 6FFEF0C1DF8B5B472FE2EDF0C76C12B9 -sipHash64(toIPv6('fe80::62:5aff:fed1:daf0')): 5681592867096972315 -xxHash64(toIPv6('fe80::62:5aff:fed1:daf0')): 4533874364641685764 -murmurHash2_64(toIPv6('fe80::62:5aff:fed1:daf0')): 11839090601505681839 -cityHash64(toIPv6('fe80::62:5aff:fed1:daf0')): 1599722731594796935 -hiveHash(toIPv6('fe80::62:5aff:fed1:daf0')): 684606770 -murmurHash3_64(toIPv6('fe80::62:5aff:fed1:daf0')): 18323430650022796352 -murmurHash3_32(toIPv6('fe80::62:5aff:fed1:daf0')): 3971193740 +ipv6: fe80::62:5aff:fed1:daf0 +halfMD5(ipv6): 9503062220758009199 +hex(MD4(ipv6)): E35A1A4FB3A3953421AB348B2E1A4A1A +hex(MD5(ipv6)): 83E1A8BD8AB7456FC229208409F79798 +hex(SHA1(ipv6)): A6D5DCE882AC44804382DE4639E6001612E1C8B5 +hex(SHA224(ipv6)): F6995FD7BED2BCA21F68DAC6BBABE742DC1BA177BA8594CEF1715C52 +hex(SHA256(ipv6)): F75497BAD6F7747BD6B150B6F69BA2DEE354F1C2A34B7BEA6183973B78640250 +hex(SHA512(ipv6)): 0C2893CCBF44BC19CCF339AEED5B68CBFD5A2EF38263A48FE21C3379BA4438E7FF7A02F59D7542442C6E6ED538E6D13D65D3573DADB381651D3D8A5DEA232EAC +farmFingerprint64(ipv6): 6643158734288374888 +javaHash(ipv6): 684606770 +xxh3(ipv6): 4051340969481364358 +wyHash64(ipv6): 18071806066582739916 +xxHash32(ipv6): 3353862080 +gccMurmurHash(ipv6): 11049311547848936878 +murmurHash2_32(ipv6): 1039121047 +javaHashUTF16LE(ipv6): -666938696 +metroHash64(ipv6): 15333045864940909774 +hex(sipHash128(ipv6)): 31D50562F877B1F92A99B05B646568B7 +hex(murmurHash3_128(ipv6)): 6FFEF0C1DF8B5B472FE2EDF0C76C12B9 +sipHash64(ipv6): 5681592867096972315 +xxHash64(ipv6): 4533874364641685764 +murmurHash2_64(ipv6): 11839090601505681839 +cityHash64(ipv6): 1599722731594796935 +hiveHash(ipv6): 684606770 +murmurHash3_64(ipv6): 18323430650022796352 +murmurHash3_32(ipv6): 3971193740 diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql index 67aae812144..d96574ef4fe 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest +SET allow_experimental_analyzer = 1; + SELECT toIPv4('1.2.3.4') AS ipv4, halfMD5(ipv4), diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index cebcc42dcba..b2c17142aa6 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -1,40 +1,52 @@ -- { echo } EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); -Expression (Projection) +Expression (Project names) Header: avgWeighted(x, y) Nullable(Float64) - Expression (Before ORDER BY) - Header: avgWeighted(x, y) Nullable(Float64) + Expression (Projection) + Header: avgWeighted(x_0, y_1) Nullable(Float64) Aggregating - Header: avgWeighted(x, y) Nullable(Float64) + Header: avgWeighted(x_0, y_1) Nullable(Float64) Expression (Before GROUP BY) - Header: x Nullable(UInt8) - y UInt8 - Union - Header: x Nullable(UInt8) - y UInt8 - Expression (Conversion before UNION) - Header: x Nullable(UInt8) + Header: x_0 Nullable(UInt8) + y_1 UInt8 + Expression (Change column names to column identifiers) + Header: x_0 Nullable(UInt8) + y_1 UInt8 + Union + Header: NULL Nullable(UInt8) + x Nullable(UInt8) y UInt8 - Expression (Projection) - Header: x UInt8 + Expression (Conversion before UNION) + Header: NULL Nullable(UInt8) + x Nullable(UInt8) y UInt8 - Expression (Before ORDER BY) - Header: 255 UInt8 - 1 UInt8 - dummy UInt8 - ReadFromStorage (SystemOne) - Header: dummy UInt8 - Expression (Conversion before UNION) - Header: x Nullable(UInt8) - y UInt8 - Expression (Projection) - Header: x Nullable(Nothing) + Expression (Project names) + Header: NULL Nullable(Nothing) + x UInt8 + y UInt8 + Expression (Projection) + Header: NULL_Nullable(Nothing) Nullable(Nothing) + 255_UInt8 UInt8 + 1_UInt8 UInt8 + Expression (Change column names to column identifiers) + Header: system.one.dummy_0 UInt8 + ReadFromStorage (SystemOne) + Header: dummy UInt8 + Expression (Conversion before UNION) + Header: NULL Nullable(UInt8) + x Nullable(UInt8) y UInt8 - Expression (Before ORDER BY) - Header: 1 UInt8 - NULL Nullable(Nothing) - dummy UInt8 - ReadFromStorage (SystemOne) - Header: dummy UInt8 + Expression (Project names) + Header: y UInt8 + x Nullable(Nothing) + y UInt8 + Expression (Projection) + Header: 1_UInt8 UInt8 + NULL_Nullable(Nothing) Nullable(Nothing) + 1_UInt8 UInt8 + Expression (Change column names to column identifiers) + Header: system.one.dummy_0 UInt8 + ReadFromStorage (SystemOne) + Header: dummy UInt8 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); 255 diff --git a/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh b/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh index 931985340c2..8c526cd8da8 100755 --- a/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh +++ b/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh @@ -5,6 +5,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(NULL)" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" -$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL))" 2>&1 | grep -q "ILLEGAL_TYPE_OF_ARGUMENT" && echo "OK" || echo "FAIL" -$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) from numbers(10)" 2>&1 | grep -q "ILLEGAL_TYPE_OF_ARGUMENT" && echo "OK" || echo "FAIL" +$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" +$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) from numbers(10) SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 163f8b0ed5e..b6acf3d2ee1 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -1,36 +1,36 @@ clickhouse-client --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy) String - Expression (Before GROUP BY) - Header: toString(dummy) String + Header: toString(system.one.dummy_0) String + Expression ((Before GROUP BY + Change column names to column identifiers)) + Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy) String - Expression (Before GROUP BY) - Header: toString(dummy) String + Header: toString(system.one.dummy_0) String + Expression ((Before GROUP BY + Change column names to column identifiers)) + Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-client --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy) String - Expression (Before GROUP BY) - Header: toString(dummy) String + Header: toString(system.one.dummy_0) String + Expression ((Before GROUP BY + Change column names to column identifiers)) + Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(dummy) String - Expression (Before GROUP BY) - Header: toString(dummy) String + Header: toString(system.one.dummy_0) String + Expression ((Before GROUP BY + Change column names to column identifiers)) + Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index bbf288c45d7..31a37862663 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -2,51 +2,51 @@ Expression Header: key String value String Join - Header: key String - value String + Header: s1.key_0 String + s2.value_1 String Expression - Header: key String + Header: s1.key_0 String ReadFromStorage Header: dummy UInt8 Union - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String Expression - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String ReadFromStorage Header: dummy UInt8 Expression Header: key String value String Join - Header: key String - s2.key String - value String + Header: s1.key_0 String + s2.key_2 String + s2.value_1 String Sorting - Header: key String + Header: s1.key_0 String Expression - Header: key String + Header: s1.key_0 String ReadFromStorage Header: dummy UInt8 Sorting - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String Union - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String Expression - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key String - value String + Header: s2.key_2 String + s2.value_1 String ReadFromStorage Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02402_external_disk_mertrics.sql b/tests/queries/0_stateless/02402_external_disk_mertrics.sql index b675c05f45c..e9696eb7122 100644 --- a/tests/queries/0_stateless/02402_external_disk_mertrics.sql +++ b/tests/queries/0_stateless/02402_external_disk_mertrics.sql @@ -20,7 +20,8 @@ SET join_algorithm = 'partial_merge'; SET default_max_bytes_in_join = 0; SET max_bytes_in_join = 10000000; -SELECT number * 200000 as n, j * 2097152 FROM numbers(5) nums +SELECT n, j * 2097152 FROM +(SELECT number * 200000 as n FROM numbers(5)) nums ANY LEFT JOIN ( SELECT number * 2 AS n, number AS j FROM numbers(1000000) ) js2 USING n ORDER BY n diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index d3de324a7e1..37256be556e 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(t) ASC - Result sort description: toStartOfMinute(t) ASC, c1 ASC - Prefix sort description: toStartOfMinute(t) ASC - Result sort description: toStartOfMinute(t) ASC - Prefix sort description: negate(a) ASC - Result sort description: negate(a) ASC - Prefix sort description: negate(a) ASC, negate(b) ASC - Result sort description: negate(a) ASC, negate(b) ASC - Prefix sort description: a DESC, negate(b) ASC - Result sort description: a DESC, negate(b) ASC - Prefix sort description: negate(a) ASC, b DESC - Result sort description: negate(a) ASC, b DESC - Prefix sort description: negate(a) ASC - Result sort description: negate(a) ASC, b ASC - Prefix sort description: a ASC - Result sort description: a ASC, negate(b) ASC + Prefix sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC + Result sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC, test.t_order_by_monotonic.c1_1 ASC + Prefix sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC + Result sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC + Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC + Result sort description: negate(test.t_order_by_monotonic.a_0) ASC + Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC, negate(test.t_order_by_monotonic.b_1) ASC + Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, negate(test.t_order_by_monotonic.b_1) ASC + Prefix sort description: test.t_order_by_monotonic.a_0 DESC, negate(test.t_order_by_monotonic.b_1) ASC + Result sort description: test.t_order_by_monotonic.a_0 DESC, negate(test.t_order_by_monotonic.b_1) ASC + Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 DESC + Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 DESC + Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC + Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 ASC + Prefix sort description: test.t_order_by_monotonic.a_0 ASC + Result sort description: test.t_order_by_monotonic.a_0 ASC, negate(test.t_order_by_monotonic.b_1) ASC diff --git a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql b/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql deleted file mode 100644 index 31602c5bae2..00000000000 --- a/tests/queries/0_stateless/02521_cannot-find-column-in-projection.sql +++ /dev/null @@ -1,3 +0,0 @@ -create table test(day Date, id UInt32) engine=MergeTree partition by day order by tuple(); -insert into test select toDate('2023-01-05') AS day, number from numbers(10); -with toUInt64(id) as id_with select day, count(id_with) from test where day >= '2023-01-01' group by day limit 1000; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } diff --git a/tests/queries/0_stateless/02521_cannot_find_column_in_projection.reference b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.reference new file mode 100644 index 00000000000..2cd767c8054 --- /dev/null +++ b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.reference @@ -0,0 +1 @@ +2023-01-05 10 diff --git a/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql new file mode 100644 index 00000000000..255c6f56ab3 --- /dev/null +++ b/tests/queries/0_stateless/02521_cannot_find_column_in_projection.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer = 1; + +drop table if exists test; +create table test(day Date, id UInt32) engine=MergeTree partition by day order by tuple(); +insert into test select toDate('2023-01-05') AS day, number from numbers(10); +with toUInt64(id) as id_with select day, count(id_with) from test where day >= '2023-01-01' group by day limit 1000; +drop table test; diff --git a/tests/queries/0_stateless/02579_fill_empty_chunk.sql b/tests/queries/0_stateless/02579_fill_empty_chunk.sql index 14ae322d8c9..cbdbd7a9f84 100644 --- a/tests/queries/0_stateless/02579_fill_empty_chunk.sql +++ b/tests/queries/0_stateless/02579_fill_empty_chunk.sql @@ -1,5 +1,7 @@ -- this SELECT produces empty chunk in FillingTransform +SET enable_positional_arguments = 0; + SELECT 2 AS x, arrayJoin([NULL, NULL, NULL]) From 3d342479989c764e26d377cf55d34d76c83b2728 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 5 Mar 2023 21:45:53 +0100 Subject: [PATCH 441/470] Fixed tests --- .../OptimizeGroupByFunctionKeysPass.cpp | 4 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 101 ++++++++++++++++-- src/Analyzer/Utils.cpp | 22 ++++ src/Analyzer/Utils.h | 5 +- src/Analyzer/ValidationUtils.cpp | 49 +++++++++ src/Analyzer/ValidationUtils.h | 7 ++ src/Functions/FunctionFile.cpp | 2 + src/Planner/PlannerJoins.cpp | 3 + src/Planner/Utils.cpp | 21 ---- src/Planner/Utils.h | 3 - .../MergeTree/MergeTreeIndexInverted.cpp | 4 +- .../00848_join_use_nulls_segfault.reference | 24 ++--- .../00848_join_use_nulls_segfault.sql | 17 +-- .../00988_expansion_aliases_limit.sql | 4 +- .../01018_ambiguous_column.reference | 9 +- .../0_stateless/01018_ambiguous_column.sql | 12 ++- ...invalidate_query_switchover_long.reference | 2 +- ...ionary_invalidate_query_switchover_long.sh | 2 +- .../01561_clickhouse_client_stage.reference | 6 +- .../01591_window_functions.reference | 24 ++--- .../0_stateless/01823_explain_json.reference | 30 +++--- .../queries/0_stateless/01823_explain_json.sh | 17 +-- ...t_if_condition_and_not_existing_column.sql | 4 +- .../02160_untuple_exponential_growth.sh | 4 +- .../02174_cte_scalar_cache.reference | 6 +- .../0_stateless/02174_cte_scalar_cache.sql | 2 + ...2483_cuturlparameter_with_arrays.reference | 8 +- .../02483_cuturlparameter_with_arrays.sql | 3 + .../02494_query_cache_explain.reference | 4 +- .../0_stateless/02494_query_cache_explain.sql | 1 + .../02567_and_consistency.reference | 2 - .../0_stateless/02567_and_consistency.sql | 21 ---- 32 files changed, 277 insertions(+), 146 deletions(-) diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index f6c4d2bc15d..c97645219da 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -69,8 +69,7 @@ private: for (auto it = function_arguments.rbegin(); it != function_arguments.rend(); ++it) candidates.push_back({ *it, is_deterministic }); - // Using DFS we traverse function tree and try to find if it uses other keys as function arguments. - // TODO: Also process CONSTANT here. We can simplify GROUP BY x, x + 1 to GROUP BY x. + /// Using DFS we traverse function tree and try to find if it uses other keys as function arguments. while (!candidates.empty()) { auto [candidate, parents_are_only_deterministic] = candidates.back(); @@ -108,6 +107,7 @@ private: return false; } } + return true; } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7d60b843694..8dc0b771cf2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -193,13 +193,9 @@ namespace ErrorCodes * lookup should not be continued, and exception must be thrown because if lookup continues identifier can be resolved from parent scope. * * TODO: Update exception messages - * TODO: JOIN TREE subquery constant columns * TODO: Table identifiers with optional UUID. * TODO: Lookup functions arrayReduce(sum, [1, 2, 3]); - * TODO: SELECT (compound_expression).*, (compound_expression).COLUMNS are not supported on parser level. - * TODO: SELECT a.b.c.*, a.b.c.COLUMNS. Qualified matcher where identifier size is greater than 2 are not supported on parser level. * TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns. - * TODO: Scalar subqueries cache. */ namespace @@ -1336,6 +1332,9 @@ private: /// Global resolve expression node to projection names map std::unordered_map resolved_expressions; + /// Global resolve expression node to tree size + std::unordered_map node_to_tree_size; + /// Global scalar subquery to scalar value map std::unordered_map scalar_subquery_to_scalar_value; @@ -1864,7 +1863,10 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden Block scalar_block; - QueryTreeNodePtrWithHash node_with_hash(node); + auto node_without_alias = node->clone(); + node_without_alias->removeAlias(); + + QueryTreeNodePtrWithHash node_with_hash(node_without_alias); auto scalar_value_it = scalar_subquery_to_scalar_value.find(node_with_hash); if (scalar_value_it != scalar_subquery_to_scalar_value.end()) @@ -2334,7 +2336,13 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con storage_id = context->resolveStorageID(storage_id); bool is_temporary_table = storage_id.getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE; - auto storage = DatabaseCatalog::instance().tryGetTable(storage_id, context); + StoragePtr storage; + + if (is_temporary_table) + storage = DatabaseCatalog::instance().getTable(storage_id, context); + else + storage = DatabaseCatalog::instance().tryGetTable(storage_id, context); + if (!storage) return {}; @@ -3007,11 +3015,39 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromJoin(const IdentifierLoo resolved_identifier = std::move(result_column_node); } - else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table) + else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false})) { + const auto & identifier_path_part = identifier_lookup.identifier.front(); + auto * left_resolved_identifier_column = left_resolved_identifier->as(); + auto * right_resolved_identifier_column = right_resolved_identifier->as(); + + if (left_resolved_identifier_column && right_resolved_identifier_column) + { + const auto & left_column_source_alias = left_resolved_identifier_column->getColumnSource()->getAlias(); + const auto & right_column_source_alias = right_resolved_identifier_column->getColumnSource()->getAlias(); + + /** If column from right table was resolved using alias, we prefer column from right table. + * + * Example: SELECT dummy FROM system.one JOIN system.one AS A ON A.dummy = system.one.dummy; + * + * If alias is specified for left table, and alias is not specified for right table and identifier was resolved + * without using left table alias, we prefer column from right table. + * + * Example: SELECT dummy FROM system.one AS A JOIN system.one ON A.dummy = system.one.dummy; + * + * Otherwise we prefer column from left table. + */ + if (identifier_path_part == right_column_source_alias) + return right_resolved_identifier; + else if (!left_column_source_alias.empty() && + right_column_source_alias.empty() && + identifier_path_part != left_column_source_alias) + return right_resolved_identifier; + } + return left_resolved_identifier; } - else if (left_resolved_identifier->isEqual(*right_resolved_identifier, IQueryTreeNode::CompareOptions{.compare_aliases = false})) + else if (scope.joins_count == 1 && scope.context->getSettingsRef().single_join_prefer_left_table) { return left_resolved_identifier; } @@ -4455,6 +4491,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi bool is_special_function_dict_get = false; bool is_special_function_join_get = false; bool is_special_function_exists = false; + bool is_special_function_if = false; if (!lambda_expression_untyped) { @@ -4462,6 +4499,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi is_special_function_dict_get = functionIsDictGet(function_name); is_special_function_join_get = functionIsJoinGet(function_name); is_special_function_exists = function_name == "exists"; + is_special_function_if = function_name == "if"; auto function_name_lowercase = Poco::toLower(function_name); @@ -4560,6 +4598,38 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi is_special_function_in = true; } + if (is_special_function_if && !function_node_ptr->getArguments().getNodes().empty()) + { + /** Handle special case with constant If function, even if some of the arguments are invalid. + * + * SELECT if(hasColumnInTable('system', 'numbers', 'not_existing_column'), not_existing_column, 5) FROM system.numbers; + */ + auto & if_function_arguments = function_node_ptr->getArguments().getNodes(); + auto if_function_condition = if_function_arguments[0]; + resolveExpressionNode(if_function_condition, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); + + QueryTreeNodePtr constant_if_result_node; + auto constant_condition = tryExtractConstantFromConditionNode(if_function_condition); + + if (constant_condition.has_value() && if_function_arguments.size() == 3) + { + if (*constant_condition) + constant_if_result_node = if_function_arguments[1]; + else + constant_if_result_node = if_function_arguments[2]; + } + + if (constant_if_result_node) + { + auto result_projection_names = resolveExpressionNode(constant_if_result_node, + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + node = std::move(constant_if_result_node); + return result_projection_names; + } + } + /// Resolve function arguments bool allow_table_expressions = is_special_function_in; @@ -5422,9 +5492,9 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id } } - if (node - && scope.nullable_group_by_keys.contains(node) - && !scope.expressions_in_resolve_process_stack.hasAggregateFunction()) + validateTreeSize(node, scope.context->getSettingsRef().max_expanded_ast_elements, node_to_tree_size); + + if (scope.nullable_group_by_keys.contains(node) && !scope.expressions_in_resolve_process_stack.hasAggregateFunction()) { node = node->clone(); node->convertToNullable(); @@ -6746,6 +6816,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls }); + for (const auto & column : projection_columns) + { + if (isNotCreatable(column.type->getTypeId())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Invalid projection column with type {}. In scope {}", + column.type->getName(), + scope.scope_node->formatASTForErrorMessage()); + } + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions * and CTE for other sections to use. * diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 16f2e5b75e6..eb7aceef1e8 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -172,6 +173,27 @@ QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, return cast_function_node; } +std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node) +{ + const auto * constant_node = condition_node->as(); + if (!constant_node) + return {}; + + const auto & value = constant_node->getValue(); + auto constant_type = constant_node->getResultType(); + constant_type = removeNullable(removeLowCardinality(constant_type)); + + auto which_constant_type = WhichDataType(constant_type); + if (!which_constant_type.isUInt8() && !which_constant_type.isNothing()) + return {}; + + if (value.isNull()) + return false; + + UInt8 predicate_value = value.safeGet(); + return predicate_value > 0; +} + static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expression_node) { ASTPtr table_expression_node_ast; diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index e3357c21bf2..5802c86c462 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -19,7 +19,7 @@ bool isNameOfLocalInFunction(const std::string & function_name); /// Returns true if function name is name of global IN function or its variations, false otherwise bool isNameOfGlobalInFunction(const std::string & function_name); -/// Returns global in function name for local in function name +/// Returns global IN function name for local IN function name std::string getGlobalInFunctionNameForLocalInFunctionName(const std::string & function_name); /// Add unique suffix to names of duplicate columns in block @@ -34,6 +34,9 @@ QueryTreeNodePtr buildCastFunction(const QueryTreeNodePtr & expression, const ContextPtr & context, bool resolve = true); +/// Try extract boolean constant from condition node +std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node); + /** Add table expression in tables in select query children. * If table expression node is not of identifier node, table node, query node, table function node, join node or array join node type throws logical error exception. */ diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index 8ccecc9769c..d70ed1170fc 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes { extern const int NOT_AN_AGGREGATE; extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } class ValidateGroupByColumnsVisitor : public ConstInDepthQueryTreeVisitor @@ -283,4 +284,52 @@ void assertNoFunctionNodes(const QueryTreeNodePtr & node, visitor.visit(node); } +void validateTreeSize(const QueryTreeNodePtr & node, + size_t max_size, + std::unordered_map & node_to_tree_size) +{ + size_t tree_size = 0; + std::vector> nodes_to_process; + nodes_to_process.emplace_back(node, false); + + while (!nodes_to_process.empty()) + { + const auto [node_to_process, processed_children] = nodes_to_process.back(); + nodes_to_process.pop_back(); + + if (processed_children) + { + ++tree_size; + node_to_tree_size.emplace(node_to_process, tree_size); + continue; + } + + auto node_to_size_it = node_to_tree_size.find(node_to_process); + if (node_to_size_it != node_to_tree_size.end()) + { + tree_size += node_to_size_it->second; + continue; + } + + nodes_to_process.emplace_back(node_to_process, true); + + for (const auto & node_to_process_child : node_to_process->getChildren()) + { + if (!node_to_process_child) + continue; + + nodes_to_process.emplace_back(node_to_process_child, false); + } + + auto * constant_node = node_to_process->as(); + if (constant_node && constant_node->hasSourceExpression()) + nodes_to_process.emplace_back(constant_node->getSourceExpression(), false); + } + + if (tree_size > max_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Query tree is too big. Maximum: {}", + max_size); +} + } diff --git a/src/Analyzer/ValidationUtils.h b/src/Analyzer/ValidationUtils.h index b8ba6b8cc10..8abd458e77c 100644 --- a/src/Analyzer/ValidationUtils.h +++ b/src/Analyzer/ValidationUtils.h @@ -31,4 +31,11 @@ void assertNoFunctionNodes(const QueryTreeNodePtr & node, std::string_view exception_function_name, std::string_view exception_place_message); +/** Validate tree size. If size of tree is greater than max size throws exception. + * Additionally for each node in tree, update node to tree size map. + */ +void validateTreeSize(const QueryTreeNodePtr & node, + size_t max_size, + std::unordered_map & node_to_tree_size); + } diff --git a/src/Functions/FunctionFile.cpp b/src/Functions/FunctionFile.cpp index 240732965f4..fa7dda82e1c 100644 --- a/src/Functions/FunctionFile.cpp +++ b/src/Functions/FunctionFile.cpp @@ -38,6 +38,8 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 2a7bd49d6a3..63fe3cc7b55 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -61,6 +62,8 @@ void JoinClause::dump(WriteBuffer & buffer) const for (const auto & dag_node : dag_nodes) { dag_nodes_dump += dag_node->result_name; + dag_nodes_dump += " "; + dag_nodes_dump += dag_node->result_type->getName(); dag_nodes_dump += ", "; } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 4f320f144df..2018ddafcdd 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -343,27 +343,6 @@ QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, con return function_node; } -std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node) -{ - const auto * constant_node = condition_node->as(); - if (!constant_node) - return {}; - - const auto & value = constant_node->getValue(); - auto constant_type = constant_node->getResultType(); - constant_type = removeNullable(removeLowCardinality(constant_type)); - - auto which_constant_type = WhichDataType(constant_type); - if (!which_constant_type.isUInt8() && !which_constant_type.isNothing()) - return {}; - - if (value.isNull()) - return false; - - UInt8 predicate_value = value.safeGet(); - return predicate_value > 0; -} - QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, const ContextPtr & context, ResultReplacementMap * result_replacement_map) diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 1e7a98b7256..0effb1d08ae 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -63,9 +63,6 @@ bool queryHasWithTotalsInAnySubqueryInJoinTree(const QueryTreeNodePtr & query_no /// Returns `and` function node that has condition nodes as its arguments QueryTreeNodePtr mergeConditionNodes(const QueryTreeNodes & condition_nodes, const ContextPtr & context); -/// Try extract boolean constant from condition node -std::optional tryExtractConstantFromConditionNode(const QueryTreeNodePtr & condition_node); - /// Replace tables nodes and table function nodes with dummy table nodes using ResultReplacementMap = std::unordered_map; QueryTreeNodePtr replaceTablesAndTableFunctionsWithDummyTables(const QueryTreeNodePtr & query_node, diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index e7d86f2a635..8e8409f3868 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -201,6 +201,7 @@ MergeTreeConditionInverted::MergeTreeConditionInverted( rpn.push_back(RPNElement::FUNCTION_UNKNOWN); return; } + rpn = std::move( RPNBuilder( query_info.filter_actions_dag->getOutputs().at(0), context_, @@ -208,10 +209,10 @@ MergeTreeConditionInverted::MergeTreeConditionInverted( { return this->traverseAtomAST(node, out); }).extractRPN()); + return; } ASTPtr filter_node = buildFilterNode(query_info.query); - if (!filter_node) { rpn.push_back(RPNElement::FUNCTION_UNKNOWN); @@ -226,7 +227,6 @@ MergeTreeConditionInverted::MergeTreeConditionInverted( query_info.prepared_sets, [&](const RPNBuilderTreeNode & node, RPNElement & out) { return traverseAtomAST(node, out); }); rpn = std::move(builder).extractRPN(); - } /// Keep in-sync with MergeTreeConditionFullText::alwaysUnknownOrTrue diff --git a/tests/queries/0_stateless/00848_join_use_nulls_segfault.reference b/tests/queries/0_stateless/00848_join_use_nulls_segfault.reference index 6bfe0db1448..43f48089b06 100644 --- a/tests/queries/0_stateless/00848_join_use_nulls_segfault.reference +++ b/tests/queries/0_stateless/00848_join_use_nulls_segfault.reference @@ -10,13 +10,13 @@ l \N \N String Nullable(String) \N \N \N \N using -l \N String Nullable(String) - \N String Nullable(String) -l \N String Nullable(String) +l \N Nullable(String) Nullable(String) +l \N Nullable(String) Nullable(String) +\N \N Nullable(String) Nullable(String) +\N \N Nullable(String) Nullable(String) +l \N Nullable(String) Nullable(String) +l \N Nullable(String) Nullable(String) \N \N Nullable(String) Nullable(String) -l \N String Nullable(String) - \N String Nullable(String) -l \N String Nullable(String) \N \N Nullable(String) Nullable(String) \N \N \N \N @@ -32,13 +32,13 @@ l \N \N Nullable(String) Nullable(String) \N \N \N \N using + join_use_nulls -l \N String Nullable(String) l \N Nullable(String) Nullable(String) -\N \N Nullable(String) Nullable(String) -\N \N Nullable(String) Nullable(String) -l \N String Nullable(String) l \N Nullable(String) Nullable(String) -\N \N Nullable(String) Nullable(String) -\N \N Nullable(String) Nullable(String) +r \N Nullable(String) Nullable(String) +r \N Nullable(String) Nullable(String) +l \N Nullable(String) Nullable(String) +l \N Nullable(String) Nullable(String) +r \N Nullable(String) Nullable(String) +r \N Nullable(String) Nullable(String) \N \N \N \N diff --git a/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql b/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql index 57eca0eb9e0..2f6cca0284c 100644 --- a/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql +++ b/tests/queries/0_stateless/00848_join_use_nulls_segfault.sql @@ -1,4 +1,5 @@ SET any_join_distinct_right_table_keys = 1; +SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS t1_00848; DROP TABLE IF EXISTS t2_00848; @@ -53,16 +54,16 @@ SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 ON t1 SELECT 'using + join_use_nulls'; -SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; -SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; -SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY t2.id, t3.id; +SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY id; +SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY id; +SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 ANY FULL JOIN t3_00848 t3 USING(id) ORDER BY id; -SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; -SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 FULL JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; -SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 FULL JOIN t3_00848 t3 USING(id) ORDER BY t2.id, t3.id; +SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY id; +SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1_00848 t1 FULL JOIN t3_00848 t3 USING(id) ORDER BY id; +SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2_00848 t2 FULL JOIN t3_00848 t3 USING(id) ORDER BY id; -SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; -SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY t1.id, t3.id; +SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 ANY LEFT JOIN t3_00848 t3 USING(id) ORDER BY id; +SELECT t3.id = 'l', t3.not_id = 'l' FROM t1_00848 t1 LEFT JOIN t3_00848 t3 USING(id) ORDER BY id; DROP TABLE t1_00848; DROP TABLE t2_00848; diff --git a/tests/queries/0_stateless/00988_expansion_aliases_limit.sql b/tests/queries/0_stateless/00988_expansion_aliases_limit.sql index 15c9f82da6f..e78ccf56093 100644 --- a/tests/queries/0_stateless/00988_expansion_aliases_limit.sql +++ b/tests/queries/0_stateless/00988_expansion_aliases_limit.sql @@ -1 +1,3 @@ -SELECT 1 AS a, a + a AS b, b + b AS c, c + c AS d, d + d AS e, e + e AS f, f + f AS g, g + g AS h, h + h AS i, i + i AS j, j + j AS k, k + k AS l, l + l AS m, m + m AS n, n + n AS o, o + o AS p, p + p AS q, q + q AS r, r + r AS s, s + s AS t, t + t AS u, u + u AS v, v + v AS w, w + w AS x, x + x AS y, y + y AS z; -- { serverError 168 } +SET allow_experimental_analyzer = 1; + +SELECT 1 AS a, a + a AS b, b + b AS c, c + c AS d, d + d AS e, e + e AS f, f + f AS g, g + g AS h, h + h AS i, i + i AS j, j + j AS k, k + k AS l, l + l AS m, m + m AS n, n + n AS o, o + o AS p, p + p AS q, q + q AS r, r + r AS s, s + s AS t, t + t AS u, u + u AS v, v + v AS w, w + w AS x, x + x AS y, y + y AS z; -- { serverError 36 } diff --git a/tests/queries/0_stateless/01018_ambiguous_column.reference b/tests/queries/0_stateless/01018_ambiguous_column.reference index a2a1d6ea4f6..308726fa184 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.reference +++ b/tests/queries/0_stateless/01018_ambiguous_column.reference @@ -1,12 +1,15 @@ 0 0 0 0 +0 0 0 0 0 0 -┌─one.dummy─┬─A.dummy─┬─B.dummy─┐ -│ 0 │ 0 │ 0 │ -└───────────┴─────────┴─────────┘ +0 +0 +┌─system.one.dummy─┬─A.dummy─┬─B.dummy─┐ +│ 0 │ 0 │ 0 │ +└──────────────────┴─────────┴─────────┘ ┌─A.dummy─┬─one.dummy─┬─two.dummy─┐ │ 0 │ 0 │ 0 │ └─────────┴───────────┴───────────┘ diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index 54603aab810..620bdb6ba3f 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -1,4 +1,6 @@ -select * from system.one cross join system.one; -- { serverError 352 } +SET allow_experimental_analyzer = 1; + +select * from system.one cross join system.one; select * from system.one cross join system.one r; select * from system.one l cross join system.one; select * from system.one left join system.one using dummy; @@ -8,10 +10,10 @@ USE system; SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy; SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy; -SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -- { serverError 352 } -SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 352 } -SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -- { serverError 352 } -SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 352 } +SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; +SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 403 } +SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; +SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 403 } SELECT * from one JOIN one A ON one.dummy = A.dummy diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.reference b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.reference index c89fe48d9f9..8d40aebacf2 100644 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.reference +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.reference @@ -1,5 +1,5 @@ 122 -Table dictdb_01041_01040.dict_invalidate doesn\'t exist +1 133 diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 7249d5e1a82..6856f952a47 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -53,7 +53,7 @@ function check_exception_detected() export -f check_exception_detected; timeout 30 bash -c check_exception_detected 2> /dev/null -$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 | grep -Eo "Table dictdb_01041_01040.dict_invalidate .* exist" +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb_01041_01040' AND name = 'invalidate'" 2>&1 | grep -Eo "dictdb_01041_01040.dict_invalidate.*UNKNOWN_TABLE" | wc -l $CLICKHOUSE_CLIENT --query " CREATE TABLE dictdb_01041_01040.dict_invalidate diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 44c39f2a444..676cca1e866 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -2,13 +2,13 @@ execute: default "foo" 1 execute: --stage fetch_columns -"dummy" +"system.one.dummy_0" 0 execute: --stage with_mergeable_state -"1" +"1_UInt8" 1 execute: --stage with_mergeable_state_after_aggregation -"1" +"1_UInt8" 1 execute: --stage complete "foo" diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index aaa88d66ca0..b981a46b4fd 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -915,12 +915,12 @@ from (select number, intDiv(number, 3) p, mod(number, 5) o from numbers(16)) t ; -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + Projection)) Window (Window step for window \'\') - Window (Window step for window \'PARTITION BY p\') - Window (Window step for window \'PARTITION BY p ORDER BY o ASC\') - Sorting (Sorting for window \'PARTITION BY p ORDER BY o ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) + Window (Window step for window \'PARTITION BY t.p_0\') + Window (Window step for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') + Sorting (Sorting for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') + Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) explain select count(*) over (order by o, number), @@ -929,13 +929,13 @@ from (select number, intDiv(number, 3) p, mod(number, 5) o from numbers(16)) t ; -Expression ((Projection + Before ORDER BY)) - Window (Window step for window \'ORDER BY o ASC, number ASC\') - Sorting (Sorting for window \'ORDER BY o ASC, number ASC\') - Window (Window step for window \'ORDER BY number ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY)) [lifted up part]) - Sorting (Sorting for window \'ORDER BY number ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) +Expression ((Project names + Projection)) + Window (Window step for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') + Sorting (Sorting for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') + Window (Window step for window \'ORDER BY t.number_1 ASC\') + Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) + Sorting (Sorting for window \'ORDER BY t.number_1 ASC\') + Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) -- A test case for the sort comparator found by fuzzer. SELECT diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index 9df7c16e4f4..befbf82f4fb 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -37,63 +37,59 @@ "Node Type": "Aggregating", "Header": [ { - "Name": "number", + "Name": "number_0", "Type": "UInt64" }, { - "Name": "plus(number, 1)", - "Type": "UInt64" - }, - { - "Name": "quantile(0.2)(number)", + "Name": "quantile(0.2_Float64)(number_0)", "Type": "Float64" }, { - "Name": "sumIf(number, greater(number, 0))", + "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", "Type": "UInt64" } ], - "Keys": ["number", "plus(number, 1)"], + "Keys": ["number_0"], "Aggregates": [ { - "Name": "quantile(0.2)(number)", + "Name": "quantile(0.2_Float64)(number_0)", "Function": { "Name": "quantile", "Parameters": ["0.2"], "Argument Types": ["UInt64"], "Result Type": "Float64" }, - "Arguments": ["number"] + "Arguments": ["number_0"] }, { - "Name": "sumIf(number, greater(number, 0))", + "Name": "sumIf(number_0, greater(number_0, 0_UInt8))", "Function": { "Name": "sumIf", "Argument Types": ["UInt64", "UInt8"], "Result Type": "UInt64" }, - "Arguments": ["number", "greater(number, 0)"] + "Arguments": ["number_0", "greater(number_0, 0_UInt8)"] } ], -------- "Node Type": "ArrayJoin", "Left": false, - "Columns": ["x", "y"], + "Columns": ["x_0", "y_1"], -------- "Node Type": "Distinct", - "Columns": ["intDiv(number, 3)", "intDiv(number, 2)"], + "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], -- "Node Type": "Distinct", - "Columns": ["intDiv(number, 3)", "intDiv(number, 2)"], + "Columns": ["intDiv(number_0, 2_UInt8)", "intDiv(number_0, 3_UInt8)"], -------- "Sort Description": [ { - "Column": "number", + "Column": "number_0", "Ascending": false, "With Fill": false }, { - "Column": "plus(number, 1)", + "Column": "plus(number_0, 1_UInt8)", "Ascending": true, "With Fill": false } diff --git a/tests/queries/0_stateless/01823_explain_json.sh b/tests/queries/0_stateless/01823_explain_json.sh index 7868bc0cc78..39128773069 100755 --- a/tests/queries/0_stateless/01823_explain_json.sh +++ b/tests/queries/0_stateless/01823_explain_json.sh @@ -5,26 +5,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw" +opts=( + "--allow_experimental_analyzer=1" +) +$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw" echo "--------" -$CLICKHOUSE_CLIENT -q "explain json = 1, description = 0, header = 1 select 1, 2 + dummy FORMAT TSVRaw" 2> /dev/null | grep Header -m 1 -A 8 +$CLICKHOUSE_CLIENT "${opts[@]}" -q "explain json = 1, description = 0, header = 1 select 1, 2 + dummy FORMAT TSVRaw" 2> /dev/null | grep Header -m 1 -A 8 echo "--------" -$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, header = 1, description = 0 +$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, header = 1, description = 0 SELECT quantile(0.2)(number), sumIf(number, number > 0) from numbers(2) group by number, number + 1 FORMAT TSVRaw - " | grep Aggregating -A 40 + " | grep Aggregating -A 36 echo "--------" -$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0 +$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0 SELECT x, y from numbers(2) array join [number, 1] as x, [number + 1] as y FORMAT TSVRaw " | grep ArrayJoin -A 2 echo "--------" -$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0 +$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0 SELECT distinct intDiv(number, 2), intDiv(number, 3) from numbers(10) FORMAT TSVRaw " | grep Distinct -A 1 echo "--------" -$CLICKHOUSE_CLIENT -q "EXPLAIN json = 1, actions = 1, description = 0 +$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json = 1, actions = 1, description = 0 SELECT number + 1 from numbers(10) order by number desc, number + 1 limit 3 FORMAT TSVRaw " | grep "Sort Description" -A 12 diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql index 4aad7ae3694..822ffb19764 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql @@ -6,9 +6,9 @@ insert into test values (0); select if(0, y, 42) from test; select if(1, 42, y) from test; select if(toUInt8(0), y, 42) from test; -select if(toInt8(0), y, 42) from test; +select if(toUInt8(0), y, 42) from test; +select if(toUInt8(1), 42, y) from test; select if(toUInt8(1), 42, y) from test; -select if(toInt8(1), 42, y) from test; select if(toUInt8(toUInt8(0)), y, 42) from test; select if(cast(cast(0, 'UInt8'), 'UInt8'), y, 42) from test; explain syntax select x, if((select hasColumnInTable(currentDatabase(), 'test', 'y')), y, x || '_') from test; diff --git a/tests/queries/0_stateless/02160_untuple_exponential_growth.sh b/tests/queries/0_stateless/02160_untuple_exponential_growth.sh index 9ec6594af69..2bc8f74a524 100755 --- a/tests/queries/0_stateless/02160_untuple_exponential_growth.sh +++ b/tests/queries/0_stateless/02160_untuple_exponential_growth.sh @@ -7,5 +7,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Should finish in reasonable time (milliseconds). # In previous versions this query led to exponential complexity of query analysis. -${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple((1, untuple((untuple(tuple(untuple(tuple(untuple((untuple((1, 1, 1, 1)), 1, 1, 1)))))), 1, 1))))))" 2>&1 | grep -cF 'TOO_BIG_AST' -${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple((1, 1, 1, 1, 1))))))))))))))))))))))))))" 2>&1 | grep -cF 'TOO_BIG_AST' +${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple((1, untuple((untuple(tuple(untuple(tuple(untuple((untuple((1, 1, 1, 1)), 1, 1, 1)))))), 1, 1))))))" 2>&1 | grep -cF 'too big' +${CLICKHOUSE_LOCAL} --query "SELECT untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple(tuple(untuple((1, 1, 1, 1, 1))))))))))))))))))))))))))" 2>&1 | grep -cF 'too big' diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache.reference b/tests/queries/0_stateless/02174_cte_scalar_cache.reference index 817116eda88..1acbef35325 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache.reference +++ b/tests/queries/0_stateless/02174_cte_scalar_cache.reference @@ -1,3 +1,3 @@ -02177_CTE_GLOBAL_ON 5 500 11 0 5 -02177_CTE_GLOBAL_OFF 1 100 5 0 1 -02177_CTE_NEW_ANALYZER 2 200 3 0 2 +02177_CTE_GLOBAL_ON 1 100 4 0 1 +02177_CTE_GLOBAL_OFF 1 100 4 0 1 +02177_CTE_NEW_ANALYZER 1 100 4 0 1 diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache.sql b/tests/queries/0_stateless/02174_cte_scalar_cache.sql index 9ed80d08cff..50a10834e64 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + WITH ( SELECT sleep(0.0001) FROM system.one ) as a1, ( SELECT sleep(0.0001) FROM system.one ) as a2, diff --git a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference index dd677873c7c..fa7f9ee7a2c 100644 --- a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference +++ b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference @@ -30,7 +30,7 @@ SELECT FORMAT Vertical; Row 1: ────── -cutURLParameter('http://bigmir.net/?a=b&c=d', []): http://bigmir.net/?a=b&c=d +cutURLParameter('http://bigmir.net/?a=b&c=d', array()): http://bigmir.net/?a=b&c=d cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']): http://bigmir.net/?c=d cutURLParameter('http://bigmir.net/?a=b&c=d', ['a', 'c']): http://bigmir.net/? cutURLParameter('http://bigmir.net/?a=b&c=d', ['c']): http://bigmir.net/?a=b @@ -43,7 +43,7 @@ cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['c', 'g']): http: cutURLParameter('http://bigmir.net/?a=b&c=d#e&g=h', ['e', 'g']): http://bigmir.net/?a=b&c=d#e cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h cutURLParameter('http://bigmir.net/?a=b&c=d#test?e=f&g=h', ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f -cutURLParameter('//bigmir.net/?a=b&c=d', []): //bigmir.net/?a=b&c=d +cutURLParameter('//bigmir.net/?a=b&c=d', array()): //bigmir.net/?a=b&c=d cutURLParameter('//bigmir.net/?a=b&c=d', ['a']): //bigmir.net/?c=d cutURLParameter('//bigmir.net/?a=b&c=d', ['a', 'c']): //bigmir.net/? cutURLParameter('//bigmir.net/?a=b&c=d#e=f', ['a', 'e']): //bigmir.net/?c=d# @@ -88,7 +88,7 @@ SELECT FORMAT Vertical; Row 1: ────── -cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), []): http://bigmir.net/?a=b&c=d +cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), array()): http://bigmir.net/?a=b&c=d cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a']): http://bigmir.net/?c=d cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['a', 'c']): http://bigmir.net/? cutURLParameter(materialize('http://bigmir.net/?a=b&c=d'), ['c']): http://bigmir.net/?a=b @@ -101,7 +101,7 @@ cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['c', 'g']): cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#e&g=h'), ['e', 'g']): http://bigmir.net/?a=b&c=d#e cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'e']): http://bigmir.net/?a=b&c=d#test?g=h cutURLParameter(materialize('http://bigmir.net/?a=b&c=d#test?e=f&g=h'), ['test', 'g']): http://bigmir.net/?a=b&c=d#test?e=f -cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), []): //bigmir.net/?a=b&c=d +cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), array()): //bigmir.net/?a=b&c=d cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a']): //bigmir.net/?c=d cutURLParameter(materialize('//bigmir.net/?a=b&c=d'), ['a', 'c']): //bigmir.net/? cutURLParameter(materialize('//bigmir.net/?a=b&c=d#e=f'), ['a', 'e']): //bigmir.net/?c=d# diff --git a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql index ea2d6ae104f..6d64d2685b7 100644 --- a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql +++ b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.sql @@ -1,4 +1,7 @@ +SET allow_experimental_analyzer = 1; + -- { echoOn } + SELECT cutURLParameter('http://bigmir.net/?a=b&c=d', []), cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']), diff --git a/tests/queries/0_stateless/02494_query_cache_explain.reference b/tests/queries/0_stateless/02494_query_cache_explain.reference index ecc965ac391..690e75bca7c 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.reference +++ b/tests/queries/0_stateless/02494_query_cache_explain.reference @@ -1,9 +1,9 @@ 1 1 -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + (Projection + Change column names to column identifiers))) Limit (preliminary LIMIT (without OFFSET)) ReadFromStorage (SystemNumbers) -Expression ((Projection + Before ORDER BY)) +Expression ((Project names + (Projection + Change column names to column identifiers))) Limit (preliminary LIMIT (without OFFSET)) ReadFromStorage (SystemNumbers) (Expression) diff --git a/tests/queries/0_stateless/02494_query_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql index 67717efde13..68b7e0005f8 100644 --- a/tests/queries/0_stateless/02494_query_cache_explain.sql +++ b/tests/queries/0_stateless/02494_query_cache_explain.sql @@ -1,6 +1,7 @@ -- Tags: no-parallel -- Tag no-parallel: Messes with internal cache +SET allow_experimental_analyzer = 1; SET allow_experimental_query_cache = true; SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02567_and_consistency.reference b/tests/queries/0_stateless/02567_and_consistency.reference index bcb2b5aecfb..e0014f187a8 100644 --- a/tests/queries/0_stateless/02567_and_consistency.reference +++ b/tests/queries/0_stateless/02567_and_consistency.reference @@ -6,10 +6,8 @@ true ===== true ===== -===== 1 ===== -===== allow_experimental_analyzer true #45440 diff --git a/tests/queries/0_stateless/02567_and_consistency.sql b/tests/queries/0_stateless/02567_and_consistency.sql index f02185a1a52..8ad06bd68cb 100644 --- a/tests/queries/0_stateless/02567_and_consistency.sql +++ b/tests/queries/0_stateless/02567_and_consistency.sql @@ -42,31 +42,10 @@ SETTINGS enable_optimize_predicate_expression = 0; SELECT '====='; -SELECT toBool(sin(SUM(number))) AS x -FROM -( - SELECT 1 AS number -) -GROUP BY number -HAVING 1 AND sin(sum(number)) -SETTINGS enable_optimize_predicate_expression = 1; -- { serverError 59 } - -SELECT '====='; - SELECT 1 and sin(1); SELECT '====='; -SELECT toBool(sin(SUM(number))) AS x -FROM -( - SELECT 1 AS number -) -GROUP BY number -HAVING x AND sin(1) -SETTINGS enable_optimize_predicate_expression = 0; -- { serverError 59 } - -SELECT '====='; SELECT 'allow_experimental_analyzer'; SET allow_experimental_analyzer = 1; From 25088187d409db5e883b66b63b42839d5e7a3c1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 6 Mar 2023 10:53:55 +0100 Subject: [PATCH 442/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 50 +++++++++++++------ src/Planner/PlannerJoinTree.cpp | 1 + ...4_nothing_arguments_in_functions_errors.sh | 4 +- .../02451_order_by_monotonic.reference | 32 ++++++------ .../0_stateless/02451_order_by_monotonic.sh | 20 ++++---- ...2483_cuturlparameter_with_arrays.reference | 1 + 6 files changed, 66 insertions(+), 42 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8dc0b771cf2..c483a9983b9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4608,25 +4608,47 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto if_function_condition = if_function_arguments[0]; resolveExpressionNode(if_function_condition, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - QueryTreeNodePtr constant_if_result_node; auto constant_condition = tryExtractConstantFromConditionNode(if_function_condition); if (constant_condition.has_value() && if_function_arguments.size() == 3) { - if (*constant_condition) - constant_if_result_node = if_function_arguments[1]; - else - constant_if_result_node = if_function_arguments[2]; - } + QueryTreeNodePtr constant_if_result_node; + QueryTreeNodePtr possibly_invalid_argument_node; - if (constant_if_result_node) - { - auto result_projection_names = resolveExpressionNode(constant_if_result_node, - scope, - false /*allow_lambda_expression*/, - false /*allow_table_expression*/); - node = std::move(constant_if_result_node); - return result_projection_names; + if (*constant_condition) + { + possibly_invalid_argument_node = if_function_arguments[2]; + constant_if_result_node = if_function_arguments[1]; + } + else + { + possibly_invalid_argument_node = if_function_arguments[1]; + constant_if_result_node = if_function_arguments[2]; + } + + bool apply_constant_if_optimization = false; + + try + { + resolveExpressionNode(possibly_invalid_argument_node, + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + } + catch (...) + { + apply_constant_if_optimization = true; + } + + if (apply_constant_if_optimization) + { + auto result_projection_names = resolveExpressionNode(constant_if_result_node, + scope, + false /*allow_lambda_expression*/, + false /*allow_table_expression*/); + node = std::move(constant_if_result_node); + return result_projection_names; + } } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 52a3c926f65..b29c4baf1b0 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1193,6 +1193,7 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, if (table_expression == left_table_expression) { query_plans_stack.push_back(std::move(left_table_expression_query_plan)); + left_table_expression = {}; continue; } diff --git a/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh b/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh index 8c526cd8da8..931985340c2 100755 --- a/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh +++ b/tests/queries/0_stateless/02294_nothing_arguments_in_functions_errors.sh @@ -5,6 +5,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh $CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(NULL)" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" -$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" -$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) from numbers(10) SETTINGS allow_experimental_analyzer = 1" 2>&1 | grep -q "ILLEGAL_COLUMN" && echo "OK" || echo "FAIL" +$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL))" 2>&1 | grep -q "ILLEGAL_TYPE_OF_ARGUMENT" && echo "OK" || echo "FAIL" +$CLICKHOUSE_LOCAL -q "SELECT assumeNotNull(materialize(NULL)) from numbers(10)" 2>&1 | grep -q "ILLEGAL_TYPE_OF_ARGUMENT" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index 37256be556e..f9f0ef38be1 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC - Result sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC, test.t_order_by_monotonic.c1_1 ASC - Prefix sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC - Result sort description: toStartOfMinute(test.t_order_by_monotonic.t_0) ASC - Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC - Result sort description: negate(test.t_order_by_monotonic.a_0) ASC - Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC, negate(test.t_order_by_monotonic.b_1) ASC - Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, negate(test.t_order_by_monotonic.b_1) ASC - Prefix sort description: test.t_order_by_monotonic.a_0 DESC, negate(test.t_order_by_monotonic.b_1) ASC - Result sort description: test.t_order_by_monotonic.a_0 DESC, negate(test.t_order_by_monotonic.b_1) ASC - Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 DESC - Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 DESC - Prefix sort description: negate(test.t_order_by_monotonic.a_0) ASC - Result sort description: negate(test.t_order_by_monotonic.a_0) ASC, test.t_order_by_monotonic.b_1 ASC - Prefix sort description: test.t_order_by_monotonic.a_0 ASC - Result sort description: test.t_order_by_monotonic.a_0 ASC, negate(test.t_order_by_monotonic.b_1) ASC + Prefix sort description: toStartOfMinute(test.t_0) ASC + Result sort description: toStartOfMinute(test.t_0) ASC, test.c1_1 ASC + Prefix sort description: toStartOfMinute(test.t_0) ASC + Result sort description: toStartOfMinute(test.t_0) ASC + Prefix sort description: negate(test.a_0) ASC + Result sort description: negate(test.a_0) ASC + Prefix sort description: negate(test.a_0) ASC, negate(test.b_1) ASC + Result sort description: negate(test.a_0) ASC, negate(test.b_1) ASC + Prefix sort description: test.a_0 DESC, negate(test.b_1) ASC + Result sort description: test.a_0 DESC, negate(test.b_1) ASC + Prefix sort description: negate(test.a_0) ASC, test.b_1 DESC + Result sort description: negate(test.a_0) ASC, test.b_1 DESC + Prefix sort description: negate(test.a_0) ASC + Result sort description: negate(test.a_0) ASC, test.b_1 ASC + Prefix sort description: test.a_0 ASC + Result sort description: test.a_0 ASC, negate(test.b_1) ASC diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.sh b/tests/queries/0_stateless/02451_order_by_monotonic.sh index cc26ba91e1c..4fdc5a9a002 100755 --- a/tests/queries/0_stateless/02451_order_by_monotonic.sh +++ b/tests/queries/0_stateless/02451_order_by_monotonic.sh @@ -17,12 +17,12 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_order_by_monotonic (t DateTime, c1 String) SELECT '2022-09-09 12:00:30', toString(number % 2)|| 'x' FROM numbers(3)" $CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -q "SELECT count() FROM - (SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic GROUP BY s, c1)" + (SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test GROUP BY s, c1)" -$CLICKHOUSE_CLIENT --optimize_read_in_order=1 -q "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic ORDER BY s, c1" +$CLICKHOUSE_CLIENT --optimize_read_in_order=1 -q "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s, c1" -explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic ORDER BY s, c1" -explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic ORDER BY s" +explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s, c1" +explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_order_by_monotonic" @@ -30,11 +30,11 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_order_by_monotonic (a Int64, b Int64) ENGI $CLICKHOUSE_CLIENT -q "INSERT INTO t_order_by_monotonic VALUES (1, 1) (1, 2), (2, 1) (2, 2)" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY -a" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY -a, -b" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY a DESC, -b" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY -a, b DESC" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY -a, b" -explain_sort_description "SELECT * FROM t_order_by_monotonic ORDER BY a, -b" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a, -b" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY a DESC, -b" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a, b DESC" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a, b" +explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY a, -b" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_order_by_monotonic" diff --git a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference index fa7f9ee7a2c..348408a15cc 100644 --- a/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference +++ b/tests/queries/0_stateless/02483_cuturlparameter_with_arrays.reference @@ -1,4 +1,5 @@ -- { echoOn } + SELECT cutURLParameter('http://bigmir.net/?a=b&c=d', []), cutURLParameter('http://bigmir.net/?a=b&c=d', ['a']), From 5d86df963e2197c33de1e6b8541de4916c39cea2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 7 Mar 2023 12:30:54 +0100 Subject: [PATCH 443/470] Analyzer disable StorageWindowView support --- src/Storages/WindowView/StorageWindowView.cpp | 4 +++ .../01047_window_view_parser_inner_table.sql | 1 + .../0_stateless/01048_window_view_parser.sql | 1 + .../01050_window_view_parser_tumble.sql | 1 + .../01051_window_view_parser_hop.sql | 1 + .../01052_window_view_proc_tumble_to_now.sh | 16 +++++++----- .../01053_window_view_proc_hop_to_now.sh | 16 +++++++----- .../01054_window_view_proc_tumble_to.sh | 16 +++++++----- .../01055_window_view_proc_hop_to.sh | 16 +++++++----- .../01056_window_view_proc_hop_watch.py | 2 ++ ..._window_view_event_tumble_to_strict_asc.sh | 16 +++++++----- ...058_window_view_event_hop_to_strict_asc.sh | 16 +++++++----- ..._window_view_event_hop_watch_strict_asc.py | 2 ++ .../01060_window_view_event_tumble_to_asc.sh | 16 +++++++----- .../01061_window_view_event_hop_to_asc.sh | 16 +++++++----- .../01062_window_view_event_hop_watch_asc.py | 2 ++ ...063_window_view_event_tumble_to_bounded.sh | 16 +++++++----- .../01064_window_view_event_hop_to_bounded.sh | 16 +++++++----- ...065_window_view_event_hop_watch_bounded.py | 2 ++ ...iew_event_tumble_to_strict_asc_lateness.sh | 16 +++++++----- ...indow_view_event_tumble_to_asc_lateness.sh | 16 +++++++----- ...w_view_event_tumble_to_bounded_lateness.sh | 16 +++++++----- .../01069_window_view_proc_tumble_watch.py | 2 ++ .../01070_window_view_watch_events.py | 2 ++ ...01071_window_view_event_tumble_asc_join.sh | 18 ++++++++----- ...72_window_view_multiple_columns_groupby.sh | 16 +++++++----- ...indow_view_event_tumble_to_asc_populate.sh | 20 ++++++++------ ...dow_view_event_tumble_asc_join_populate.sh | 18 ++++++++----- ...window_view_proc_tumble_to_now_populate.sh | 16 +++++++----- .../01076_window_view_alter_query_to.sh | 24 ++++++++++------- ...indow_view_alter_query_to_modify_source.sh | 26 +++++++++++-------- .../01078_window_view_alter_query_watch.py | 2 ++ ...9_window_view_inner_table_memory_tumble.sh | 16 +++++++----- ...1080_window_view_inner_table_memory_hop.sh | 16 +++++++----- .../01081_window_view_target_table_engine.sh | 14 ++++++---- .../01082_window_view_watch_limit.py | 2 ++ .../0_stateless/01083_window_view_select.sh | 18 ++++++++----- ...01084_window_view_with_table_identifier.sh | 14 ++++++---- .../0_stateless/01085_window_view_attach.sql | 1 + .../0_stateless/01086_window_view_cleanup.sh | 24 ++++++++++------- .../01087_window_view_alter_query.sh | 22 +++++++++------- .../01088_window_view_default_column.sh | 16 +++++++----- .../02125_query_views_log_window_function.sql | 2 ++ .../02342_window_view_different_struct.sql | 1 + .../0_stateless/02364_window_view_segfault.sh | 6 ++++- 45 files changed, 332 insertions(+), 188 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3a74fd5fc75..86fc0157bcd 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1158,6 +1158,10 @@ StorageWindowView::StorageWindowView( , fire_signal_timeout_s(context_->getSettingsRef().wait_for_window_view_fire_signal_timeout.totalSeconds()) , clean_interval_usec(context_->getSettingsRef().window_view_clean_interval.totalMicroseconds()) { + if (context_->getSettingsRef().allow_experimental_analyzer) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Experimental WINDOW VIEW feature is not supported with new infrastructure for query analysis (the setting 'allow_experimental_analyzer')"); + if (!query.select) throw Exception(ErrorCodes::INCORRECT_QUERY, "SELECT query is not specified for {}", getName()); diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index 2d9911287a3..bf1ac254783 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01047; set allow_deprecated_database_ordinary=1; diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index 4c329f99f6e..f87d9aa023e 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -1,5 +1,6 @@ -- Tags: no-parallel +SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; DROP DATABASE IF EXISTS test_01048; set allow_deprecated_database_ordinary=1; diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index d9604bb2b52..f49fbc251fd 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index 472dc66f1a2..45877cf0647 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; 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 9fdc66191d7..e75b7d9570b 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 @@ -4,7 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --multiquery < Date: Tue, 7 Mar 2023 12:31:32 +0100 Subject: [PATCH 444/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c483a9983b9..ae42e5dbd06 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5166,7 +5166,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi /** Do not perform constant folding if there are aggregate or arrayJoin functions inside function. * Example: SELECT toTypeName(sum(number)) FROM numbers(10); */ - if (column && isColumnConst(*column) && (!hasAggregateFunctionNodes(node) && !hasFunctionNode(node, "arrayJoin"))) + if (column && isColumnConst(*column) && !typeid_cast(column.get())->getDataColumn().isDummy() && + (!hasAggregateFunctionNodes(node) && !hasFunctionNode(node, "arrayJoin"))) { /// Replace function node with result constant node Field column_constant_value; From 8bc92c197f9ffbc2d7a4e7ff1321d6150653f7b8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 8 Mar 2023 10:43:57 +0100 Subject: [PATCH 445/470] Fixed build --- src/Storages/WindowView/StorageWindowView.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 86fc0157bcd..3471e4ea6bf 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -78,6 +78,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int TABLE_WAS_NOT_DROPPED; extern const int NOT_IMPLEMENTED; + extern const int UNSUPPORTED_METHOD; } namespace From faaa1d8570dd425e3f05023ab324260789917793 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 8 Mar 2023 11:54:49 +0100 Subject: [PATCH 446/470] Fixed tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 + src/Analyzer/ValidationUtils.h | 2 +- src/Planner/PlannerExpressionAnalysis.cpp | 47 +++++++++-------------- 3 files changed, 21 insertions(+), 30 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index ae42e5dbd06..b85999523c3 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -697,7 +697,9 @@ struct IdentifierResolveScope } if (auto * union_node = scope_node->as()) + { context = union_node->getContext(); + } else if (auto * query_node = scope_node->as()) { context = query_node->getContext(); diff --git a/src/Analyzer/ValidationUtils.h b/src/Analyzer/ValidationUtils.h index 8abd458e77c..c15a3531c8d 100644 --- a/src/Analyzer/ValidationUtils.h +++ b/src/Analyzer/ValidationUtils.h @@ -7,7 +7,7 @@ namespace DB struct ValidationParams { - bool group_by_use_nulls; + bool group_by_use_nulls = false; }; /** Validate aggregates in query node. diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 9a7340f936c..11444503c5f 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -34,15 +34,13 @@ namespace * It is client responsibility to update filter analysis result if filter column must be removed after chain is finalized. */ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_node, - const ColumnsWithTypeAndName & current_output_columns, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto & filter_input = current_output_columns; - FilterAnalysisResult result; - result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, filter_input, planner_context); + result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; actions_chain.addStep(std::make_unique(result.filter_actions)); @@ -52,8 +50,8 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ -std::pair, std::optional> analyzeAggregation(const QueryTreeNodePtr & query_tree, - const ColumnsWithTypeAndName & current_output_columns, +std::optional analyzeAggregation(const QueryTreeNodePtr & query_tree, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { @@ -69,9 +67,7 @@ std::pair, std::optional(group_by_input); + ActionsDAGPtr before_aggregation_actions = std::make_shared(input_columns); before_aggregation_actions->getOutputs().clear(); std::unordered_set before_aggregation_actions_output_node_names; @@ -203,14 +199,14 @@ std::pair, std::optional analyzeWindow(const QueryTreeNodePtr & query_tree, - const ColumnsWithTypeAndName & current_output_columns, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { @@ -220,11 +216,9 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query auto window_descriptions = extractWindowDescriptions(window_function_nodes, *planner_context); - const auto & window_input = current_output_columns; - PlannerActionsVisitor actions_visitor(planner_context); - ActionsDAGPtr before_window_actions = std::make_shared(window_input); + ActionsDAGPtr before_window_actions = std::make_shared(input_columns); before_window_actions->getOutputs().clear(); std::unordered_set before_window_actions_output_node_names; @@ -299,12 +293,11 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query * It is client responsibility to update projection analysis result with project names actions after chain is finalized. */ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, - const ColumnsWithTypeAndName & current_output_columns, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto & projection_input = current_output_columns; - auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), projection_input, planner_context); + auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context); auto projection_columns = query_node.getProjectionColumns(); size_t projection_columns_size = projection_columns.size(); @@ -347,13 +340,11 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, * Actions before sort are added into actions chain. */ SortAnalysisResult analyzeSort(const QueryNode & query_node, - const ColumnsWithTypeAndName & current_output_columns, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - const auto & order_by_input = current_output_columns; - - ActionsDAGPtr before_sort_actions = std::make_shared(order_by_input); + ActionsDAGPtr before_sort_actions = std::make_shared(input_columns); auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); before_sort_actions_outputs.clear(); @@ -436,13 +427,12 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, * Actions before limit by are added into actions chain. */ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, - const ColumnsWithTypeAndName & current_output_columns, + const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, const NameSet & required_output_nodes_names, ActionsChain & actions_chain) { - const auto & limit_by_input = current_output_columns; - auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), limit_by_input, planner_context); + auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context); NameSet limit_by_column_names_set; Names limit_by_column_names; @@ -480,8 +470,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo std::optional where_analysis_result_optional; std::optional where_action_step_index_optional; - const auto * input_columns = actions_chain.getLastStepAvailableOutputColumnsOrNull(); - ColumnsWithTypeAndName current_output_columns = input_columns ? *input_columns : join_tree_input_columns; + ColumnsWithTypeAndName current_output_columns = join_tree_input_columns; if (query_node.hasWhere()) { @@ -490,9 +479,9 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } - auto [aggregation_analysis_result_optional, aggregated_columns_optional] = analyzeAggregation(query_tree, current_output_columns, planner_context, actions_chain); - if (aggregated_columns_optional) - current_output_columns = std::move(*aggregated_columns_optional); + auto aggregation_analysis_result_optional = analyzeAggregation(query_tree, current_output_columns, planner_context, actions_chain); + if (aggregation_analysis_result_optional) + current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); std::optional having_analysis_result_optional; std::optional having_action_step_index_optional; From d6c7c2193ccdb10b4f3faf34fda19bf650168c6c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 8 Mar 2023 14:39:48 +0100 Subject: [PATCH 447/470] Fixed tests --- src/Analyzer/FunctionNode.cpp | 3 ++- ...cpp => LogicalExpressionOptimizerPass.cpp} | 8 ++++-- src/Analyzer/Passes/QueryAnalysisPass.cpp | 26 ++++++++++++++----- 3 files changed, 28 insertions(+), 9 deletions(-) rename src/Analyzer/Passes/{LogicalExpressionOptimizer.cpp => LogicalExpressionOptimizerPass.cpp} (97%) diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 68bdb0f9956..fe170c8482e 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -63,7 +63,8 @@ ColumnsWithTypeAndName FunctionNode::getArgumentColumns() const else argument_column.type = argument->getResultType(); - if (auto * constant = argument->as()) + auto * constant = argument->as(); + if (constant && !isNotCreatable(argument_column.type)) argument_column.column = argument_column.type->createColumnConst(1, constant->getValue()); argument_columns.push_back(std::move(argument_column)); diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp similarity index 97% rename from src/Analyzer/Passes/LogicalExpressionOptimizer.cpp rename to src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 73585a4cd23..3d65035f9fd 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizer.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -7,8 +7,6 @@ #include #include -#include - namespace DB { @@ -100,6 +98,9 @@ private: } } + if (and_operands.size() == function_node.getArguments().getNodes().size()) + return; + if (and_operands.size() == 1) { /// AND operator can have UInt8 or bool as its type. @@ -207,6 +208,9 @@ private: or_operands.push_back(std::move(in_function)); } + if (or_operands.size() == function_node.getArguments().getNodes().size()) + return; + if (or_operands.size() == 1) { /// if the result type of operand is the same as the result type of OR diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b85999523c3..38575965973 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6676,6 +6676,17 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier /// Resolve query node sections. + NamesAndTypes projection_columns; + + if (!scope.group_by_use_nulls) + { + projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); + if (query_node_typed.getProjection().getNodes().empty()) + throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, + "Empty list of columns in projection. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + } + if (query_node_typed.hasWith()) resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -6770,11 +6781,14 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier convertLimitOffsetExpression(query_node_typed.getOffset(), "OFFSET", scope); } - auto projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); - if (query_node_typed.getProjection().getNodes().empty()) - throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, - "Empty list of columns in projection. In scope {}", - scope.scope_node->formatASTForErrorMessage()); + if (scope.group_by_use_nulls) + { + projection_columns = resolveProjectionExpressionNodeList(query_node_typed.getProjectionNode(), scope); + if (query_node_typed.getProjection().getNodes().empty()) + throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_QUERIED, + "Empty list of columns in projection. In scope {}", + scope.scope_node->formatASTForErrorMessage()); + } /** Resolve nodes with duplicate aliases. * Table expressions cannot have duplicate aliases. @@ -6843,7 +6857,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier for (const auto & column : projection_columns) { - if (isNotCreatable(column.type->getTypeId())) + if (isNotCreatable(column.type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Invalid projection column with type {}. In scope {}", column.type->getName(), From 56adb475253ce371a6f54808c84d018528eefcd4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 10 Mar 2023 10:45:49 +0100 Subject: [PATCH 448/470] Fixed tests --- .../01556_explain_select_with_union_query.sql | 1 + ...imize_monotonous_functions_in_order_by.sql | 1 + .../0_stateless/02227_union_match_by_name.sql | 7 +++++- .../0_stateless/02303_query_kind.reference | 8 +++---- tests/queries/0_stateless/02303_query_kind.sh | 8 +++++-- .../02381_join_dup_columns_in_plan.sql | 1 + .../0_stateless/02451_order_by_monotonic.sh | 22 +++++++++++-------- ...75_predicate_push_down_filled_join_fix.sql | 1 + 8 files changed, 33 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql index dcd9bbe7347..bbd96ef5c69 100644 --- a/tests/queries/0_stateless/01556_explain_select_with_union_query.sql +++ b/tests/queries/0_stateless/01556_explain_select_with_union_query.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET union_default_mode = 'DISTINCT'; set enable_global_with_statement = 1; diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql index 15ddb5a848f..2fe24d1662d 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET optimize_monotonous_functions_in_order_by = 1; SET optimize_read_in_order = 1; diff --git a/tests/queries/0_stateless/02227_union_match_by_name.sql b/tests/queries/0_stateless/02227_union_match_by_name.sql index cc0ab8ba5aa..6a19add1d37 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.sql +++ b/tests/queries/0_stateless/02227_union_match_by_name.sql @@ -1,3 +1,8 @@ --- { echo } +SET allow_experimental_analyzer = 1; + +-- { echoOn } + EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); + +-- { echoOff } diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index b6acf3d2ee1..5af8c2b743f 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -1,4 +1,4 @@ -clickhouse-client --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -7,7 +7,7 @@ Header: dummy String Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-local --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -16,7 +16,7 @@ Header: dummy String Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-client --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating @@ -25,7 +25,7 @@ Header: dummy String Header: toString(system.one.dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 -clickhouse-local --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy +clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating diff --git a/tests/queries/0_stateless/02303_query_kind.sh b/tests/queries/0_stateless/02303_query_kind.sh index 5ad5f9ec6f4..1d883a2dcc7 100755 --- a/tests/queries/0_stateless/02303_query_kind.sh +++ b/tests/queries/0_stateless/02303_query_kind.sh @@ -4,6 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +opts=( + "--allow_experimental_analyzer=1" +) + function run_query() { echo "clickhouse-client $*" @@ -12,5 +16,5 @@ function run_query() echo "clickhouse-local $*" $CLICKHOUSE_LOCAL "$@" } -run_query --query_kind secondary_query -q "explain plan header=1 select toString(dummy) as dummy from system.one group by dummy" -run_query --query_kind initial_query -q "explain plan header=1 select toString(dummy) as dummy from system.one group by dummy" +run_query "${opts[@]}" --query_kind secondary_query -q "explain plan header=1 select toString(dummy) as dummy from system.one group by dummy" +run_query "${opts[@]}" --query_kind initial_query -q "explain plan header=1 select toString(dummy) as dummy from system.one group by dummy" diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql index 4ed6d965292..dfcd8c12e11 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SET join_algorithm = 'hash'; EXPLAIN actions=0, description=0, header=1 diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.sh b/tests/queries/0_stateless/02451_order_by_monotonic.sh index 4fdc5a9a002..7d1356b4445 100755 --- a/tests/queries/0_stateless/02451_order_by_monotonic.sh +++ b/tests/queries/0_stateless/02451_order_by_monotonic.sh @@ -4,31 +4,35 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +opts=( + "--allow_experimental_analyzer=1" +) + function explain_sort_description() { - out=$($CLICKHOUSE_CLIENT --optimize_read_in_order=1 -q "EXPLAIN PLAN actions = 1 $1") + out=$($CLICKHOUSE_CLIENT "${opts[@]}" --optimize_read_in_order=1 -q "EXPLAIN PLAN actions = 1 $1") echo "$out" | grep "Prefix sort description:" echo "$out" | grep "Result sort description:" } -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_order_by_monotonic" -$CLICKHOUSE_CLIENT -q "CREATE TABLE t_order_by_monotonic (t DateTime, c1 String) ENGINE = MergeTree ORDER BY (t, c1) +$CLICKHOUSE_CLIENT "${opts[@]}" -q "DROP TABLE IF EXISTS t_order_by_monotonic" +$CLICKHOUSE_CLIENT "${opts[@]}" -q "CREATE TABLE t_order_by_monotonic (t DateTime, c1 String) ENGINE = MergeTree ORDER BY (t, c1) AS SELECT '2022-09-09 12:00:00', toString(number % 2) FROM numbers(2) UNION ALL SELECT '2022-09-09 12:00:30', toString(number % 2)|| 'x' FROM numbers(3)" -$CLICKHOUSE_CLIENT --optimize_aggregation_in_order=1 -q "SELECT count() FROM +$CLICKHOUSE_CLIENT "${opts[@]}" --optimize_aggregation_in_order=1 -q "SELECT count() FROM (SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test GROUP BY s, c1)" -$CLICKHOUSE_CLIENT --optimize_read_in_order=1 -q "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s, c1" +$CLICKHOUSE_CLIENT "${opts[@]}" --optimize_read_in_order=1 -q "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s, c1" explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s, c1" explain_sort_description "SELECT toStartOfMinute(t) AS s, c1 FROM t_order_by_monotonic AS test ORDER BY s" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_order_by_monotonic" +$CLICKHOUSE_CLIENT "${opts[@]}" -q "DROP TABLE IF EXISTS t_order_by_monotonic" -$CLICKHOUSE_CLIENT -q "CREATE TABLE t_order_by_monotonic (a Int64, b Int64) ENGINE = MergeTree ORDER BY (a, b)" +$CLICKHOUSE_CLIENT "${opts[@]}" -q "CREATE TABLE t_order_by_monotonic (a Int64, b Int64) ENGINE = MergeTree ORDER BY (a, b)" -$CLICKHOUSE_CLIENT -q "INSERT INTO t_order_by_monotonic VALUES (1, 1) (1, 2), (2, 1) (2, 2)" +$CLICKHOUSE_CLIENT "${opts[@]}" -q "INSERT INTO t_order_by_monotonic VALUES (1, 1) (1, 2), (2, 1) (2, 2)" explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a" explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a, -b" @@ -37,4 +41,4 @@ explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY -a, b" explain_sort_description "SELECT * FROM t_order_by_monotonic AS test ORDER BY a, -b" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS t_order_by_monotonic" +$CLICKHOUSE_CLIENT "${opts[@]}" -q "DROP TABLE IF EXISTS t_order_by_monotonic" diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql index 78cb423216b..73baad11634 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.sql @@ -1,4 +1,5 @@ SET allow_experimental_analyzer = 1; +SET single_join_prefer_left_table = 0; DROP TABLE IF EXISTS test_table; CREATE TABLE test_table From fcb9c07949e0a8d4db75d3ad5e50ecb2c2730a89 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 10 Mar 2023 11:36:54 +0100 Subject: [PATCH 449/470] Fixed tests --- tests/queries/0_stateless/02227_union_match_by_name.reference | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index b2c17142aa6..e51ea983f7f 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -1,4 +1,5 @@ --- { echo } +-- { echoOn } + EXPLAIN header = 1, optimize = 0 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); Expression (Project names) Header: avgWeighted(x, y) Nullable(Float64) From c835fa3958fc30d238569f46ec45cbc2b2b88f80 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 10 Mar 2023 17:34:40 +0100 Subject: [PATCH 450/470] Fixed tests --- src/Storages/StorageS3Cluster.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 92be3f50150..a3fd24b4f98 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -114,6 +114,7 @@ Pipe StorageS3Cluster::read( else { auto interpreter = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()); + sample_block = interpreter.getSampleBlock(); query_to_send = interpreter.getQueryInfo().query->clone(); } From c331bc48a398e9689b5df0f207e75d038b248e18 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 11 Mar 2023 14:25:30 +0100 Subject: [PATCH 451/470] Fixed tests --- .../00378_json_quote_64bit_integers.sql | 1 + .../01561_clickhouse_client_stage.reference | 10 +++++----- .../0_stateless/01561_clickhouse_client_stage.sh | 14 +++++++++----- .../queries/0_stateless/01591_window_functions.sql | 2 ++ .../02048_clickhouse_local_stage.reference | 10 +++++----- .../0_stateless/02048_clickhouse_local_stage.sh | 14 +++++++++----- 6 files changed, 31 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql b/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql index 3a70b64bc86..e7b59bc3f7f 100644 --- a/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql +++ b/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql @@ -2,6 +2,7 @@ SET output_format_write_statistics = 0; SET extremes = 1; +SET allow_experimental_analyzer = 1; SET output_format_json_quote_64bit_integers = 1; SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSON; diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 676cca1e866..00e0f4ddb2e 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -1,15 +1,15 @@ -execute: default +execute: --allow_experimental_analyzer=1 "foo" 1 -execute: --stage fetch_columns +execute: --allow_experimental_analyzer=1 --stage fetch_columns "system.one.dummy_0" 0 -execute: --stage with_mergeable_state +execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" 1 -execute: --stage with_mergeable_state_after_aggregation +execute: --allow_experimental_analyzer=1 --stage with_mergeable_state_after_aggregation "1_UInt8" 1 -execute: --stage complete +execute: --allow_experimental_analyzer=1 --stage complete "foo" 1 diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.sh b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh index 72161333812..99267458421 100755 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.sh +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh @@ -5,6 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +opts=( + "--allow_experimental_analyzer=1" +) + function execute_query() { if [ $# -eq 0 ]; then @@ -15,8 +19,8 @@ function execute_query() ${CLICKHOUSE_CLIENT} "$@" --format CSVWithNames -q "SELECT 1 AS foo" } -execute_query # default -- complete -execute_query --stage fetch_columns -execute_query --stage with_mergeable_state -execute_query --stage with_mergeable_state_after_aggregation -execute_query --stage complete +execute_query "${opts[@]}" # default -- complete +execute_query "${opts[@]}" --stage fetch_columns +execute_query "${opts[@]}" --stage with_mergeable_state +execute_query "${opts[@]}" --stage with_mergeable_state_after_aggregation +execute_query "${opts[@]}" --stage complete diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 3f4a028eac2..3c9c1f9cea7 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -1,5 +1,7 @@ -- Tags: long +SET allow_experimental_analyzer = 1; + -- { echo } -- just something basic diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 676cca1e866..00e0f4ddb2e 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -1,15 +1,15 @@ -execute: default +execute: --allow_experimental_analyzer=1 "foo" 1 -execute: --stage fetch_columns +execute: --allow_experimental_analyzer=1 --stage fetch_columns "system.one.dummy_0" 0 -execute: --stage with_mergeable_state +execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" 1 -execute: --stage with_mergeable_state_after_aggregation +execute: --allow_experimental_analyzer=1 --stage with_mergeable_state_after_aggregation "1_UInt8" 1 -execute: --stage complete +execute: --allow_experimental_analyzer=1 --stage complete "foo" 1 diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh index 5c1303b5160..182acc23a13 100755 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh @@ -5,6 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +opts=( + "--allow_experimental_analyzer=1" +) + function execute_query() { if [ $# -eq 0 ]; then @@ -15,8 +19,8 @@ function execute_query() ${CLICKHOUSE_LOCAL} "$@" --format CSVWithNames -q "SELECT 1 AS foo" } -execute_query # default -- complete -execute_query --stage fetch_columns -execute_query --stage with_mergeable_state -execute_query --stage with_mergeable_state_after_aggregation -execute_query --stage complete +execute_query "${opts[@]}" # default -- complete +execute_query "${opts[@]}" --stage fetch_columns +execute_query "${opts[@]}" --stage with_mergeable_state +execute_query "${opts[@]}" --stage with_mergeable_state_after_aggregation +execute_query "${opts[@]}" --stage complete From 3687698b155f5dca42a4a1266b8027128e519dae Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 15:03:10 +0000 Subject: [PATCH 452/470] Fix explain graph with projection --- src/Processors/QueryPlan/ISourceStep.cpp | 7 +- src/Processors/QueryPlan/ISourceStep.h | 2 + src/QueryPipeline/Pipe.cpp | 10 ++ src/QueryPipeline/Pipe.h | 2 + src/QueryPipeline/QueryPipelineBuilder.h | 2 + ... => 02677_grace_hash_limit_race.reference} | 0 ...ce.sql => 02677_grace_hash_limit_race.sql} | 0 ...78_explain_graph_with_projection.reference | 116 ++++++++++++++++++ .../02678_explain_graph_with_projection.sql | 11 ++ 9 files changed, 149 insertions(+), 1 deletion(-) rename tests/queries/0_stateless/{25340_grace_hash_limit_race.reference => 02677_grace_hash_limit_race.reference} (100%) rename tests/queries/0_stateless/{25340_grace_hash_limit_race.sql => 02677_grace_hash_limit_race.sql} (100%) create mode 100644 tests/queries/0_stateless/02678_explain_graph_with_projection.reference create mode 100644 tests/queries/0_stateless/02678_explain_graph_with_projection.sql diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 0644d9b44eb..562f92ff63b 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -12,8 +12,13 @@ ISourceStep::ISourceStep(DataStream output_stream_) QueryPipelineBuilderPtr ISourceStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings & settings) { auto pipeline = std::make_unique(); - QueryPipelineProcessorsCollector collector(*pipeline, this); initializePipeline(*pipeline, settings); + QueryPipelineProcessorsCollector collector(*pipeline, this); + + /// Properly collecting processors from Pipe. + /// At the creation time of a Pipe, since `collected_processors` is nullptr, + /// the processors can not be collected. + pipeline->collectProcessors(); auto added_processors = collector.detachProcessors(); processors.insert(processors.end(), added_processors.begin(), added_processors.end()); return pipeline; diff --git a/src/Processors/QueryPlan/ISourceStep.h b/src/Processors/QueryPlan/ISourceStep.h index 744b6f9b5c4..1327444697e 100644 --- a/src/Processors/QueryPlan/ISourceStep.h +++ b/src/Processors/QueryPlan/ISourceStep.h @@ -10,6 +10,8 @@ class ISourceStep : public IQueryPlanStep public: explicit ISourceStep(DataStream output_stream_); + String getName() const override { return "ISourceStep"; } + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; virtual void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) = 0; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 91ba01c479f..a20b154054a 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -867,4 +867,14 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } +void Pipe::collectProcessors() +{ + if (collected_processors) + { + for (const auto & processor : *processors) + { + collected_processors->emplace_back(processor); + } + } +} } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 09931e38578..a19364f7907 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -104,6 +104,8 @@ public: std::shared_ptr getProcessorsPtr() { return processors; } + void collectProcessors(); + private: /// Header is common for all output below. Block header; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 4f984680c75..51a1aa69563 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -185,6 +185,8 @@ public: static Pipe getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources); static QueryPipeline getPipeline(QueryPipelineBuilder builder); + void collectProcessors() { pipe.collectProcessors(); } + private: /// Destruction order: processors, header, locks, temporary storages, local contexts diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.reference b/tests/queries/0_stateless/02677_grace_hash_limit_race.reference similarity index 100% rename from tests/queries/0_stateless/25340_grace_hash_limit_race.reference rename to tests/queries/0_stateless/02677_grace_hash_limit_race.reference diff --git a/tests/queries/0_stateless/25340_grace_hash_limit_race.sql b/tests/queries/0_stateless/02677_grace_hash_limit_race.sql similarity index 100% rename from tests/queries/0_stateless/25340_grace_hash_limit_race.sql rename to tests/queries/0_stateless/02677_grace_hash_limit_race.sql diff --git a/tests/queries/0_stateless/02678_explain_graph_with_projection.reference b/tests/queries/0_stateless/02678_explain_graph_with_projection.reference new file mode 100644 index 00000000000..0c513f735cc --- /dev/null +++ b/tests/queries/0_stateless/02678_explain_graph_with_projection.reference @@ -0,0 +1,116 @@ +digraph +{ + rankdir="LR"; + { node [shape = rect] + subgraph cluster_0 { + label ="Expression"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n4 [label="ExpressionTransform"]; + } + } + subgraph cluster_1 { + label ="ReadFromStorage"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n3 [label="AggregatingTransform"]; + n2 [label="ExpressionTransform"]; + n1 [label="SourceFromSingleChunk"]; + } + } + } + n3 -> n4 [label=""]; + n2 -> n3 [label=""]; + n1 -> n2 [label=""]; +} +digraph +{ + rankdir="LR"; + { node [shape = rect] + subgraph cluster_0 { + label ="Expression"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n4 [label="ExpressionTransform"]; + } + } + subgraph cluster_1 { + label ="ReadFromStorage"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n3 [label="AggregatingTransform"]; + n2 [label="ExpressionTransform"]; + n1 [label="SourceFromSingleChunk"]; + } + } + } + n3 -> n4 [label=""]; + n2 -> n3 [label=""]; + n1 -> n2 [label=""]; +} +digraph +{ + rankdir="LR"; + { node [shape = rect] + subgraph cluster_0 { + label ="Expression"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n4 [label="ExpressionTransform"]; + } + } + subgraph cluster_1 { + label ="ReadFromStorage"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n3 [label="AggregatingTransform"]; + n2 [label="ExpressionTransform"]; + n1 [label="SourceFromSingleChunk"]; + } + } + } + n3 -> n4 [label=""]; + n2 -> n3 [label=""]; + n1 -> n2 [label=""]; +} +digraph +{ + rankdir="LR"; + { node [shape = rect] + subgraph cluster_0 { + label ="Expression"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n4 [label="ExpressionTransform"]; + } + } + subgraph cluster_1 { + label ="ReadFromStorage"; + style=filled; + color=lightgrey; + node [style=filled,color=white]; + { rank = same; + n3 [label="AggregatingTransform"]; + n2 [label="ExpressionTransform"]; + n1 [label="SourceFromSingleChunk"]; + } + } + } + n3 -> n4 [label=""]; + n2 -> n3 [label=""]; + n1 -> n2 [label=""]; +} diff --git a/tests/queries/0_stateless/02678_explain_graph_with_projection.sql b/tests/queries/0_stateless/02678_explain_graph_with_projection.sql new file mode 100644 index 00000000000..06ee1078e74 --- /dev/null +++ b/tests/queries/0_stateless/02678_explain_graph_with_projection.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS t1; +CREATE TABLE t1(ID UInt64, name String) engine=MergeTree order by ID; + +insert into t1(ID, name) values (1, 'abc'), (2, 'bbb'); + +explain pipeline graph=1 select count(ID) from t1; +explain pipeline graph=1 select sum(1) from t1; +explain pipeline graph=1 select min(ID) from t1; +explain pipeline graph=1 select max(ID) from t1; + +DROP TABLE t1; From 3e554c2925c0a8604d53bcb665ce8f1f5d2ebc9c Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 15:24:19 +0000 Subject: [PATCH 453/470] fix --- src/Processors/QueryPlan/ISourceStep.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/QueryPlan/ISourceStep.h b/src/Processors/QueryPlan/ISourceStep.h index 1327444697e..744b6f9b5c4 100644 --- a/src/Processors/QueryPlan/ISourceStep.h +++ b/src/Processors/QueryPlan/ISourceStep.h @@ -10,8 +10,6 @@ class ISourceStep : public IQueryPlanStep public: explicit ISourceStep(DataStream output_stream_); - String getName() const override { return "ISourceStep"; } - QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; virtual void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) = 0; From 8585fae467476d5f7830fcfc7e8773bc292e5176 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 11 Mar 2023 16:31:31 +0100 Subject: [PATCH 454/470] Fixed tests --- tests/queries/0_stateless/00722_inner_join.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/00722_inner_join.sql b/tests/queries/0_stateless/00722_inner_join.sql index 75ef40ff2b7..0d5a543b99d 100644 --- a/tests/queries/0_stateless/00722_inner_join.sql +++ b/tests/queries/0_stateless/00722_inner_join.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + SET allow_experimental_analyzer = 1; DROP TABLE IF EXISTS one; From 530c9ae4905fb5f52b7ac460cf09fe0568cd72ea Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 15:43:19 +0000 Subject: [PATCH 455/470] revert --- src/Interpreters/InterpreterExplainQuery.cpp | 5 +- src/QueryPipeline/printPipeline.cpp | 173 ++++++++++++++++++ src/QueryPipeline/printPipeline.h | 5 + .../02678_explain_pipeline_graph.reference | 0 .../02678_explain_pipeline_graph.sql | 7 - ..._pipeline_graph_with_projection.reference} | 0 ...xplain_pipeline_graph_with_projection.sql} | 0 7 files changed, 182 insertions(+), 8 deletions(-) create mode 100644 src/QueryPipeline/printPipeline.cpp delete mode 100644 tests/queries/0_stateless/02678_explain_pipeline_graph.reference delete mode 100644 tests/queries/0_stateless/02678_explain_pipeline_graph.sql rename tests/queries/0_stateless/{02678_explain_graph_with_projection.reference => 02678_explain_pipeline_graph_with_projection.reference} (100%) rename tests/queries/0_stateless/{02678_explain_graph_with_projection.sql => 02678_explain_pipeline_graph_with_projection.sql} (100%) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index b2172a07e91..3c225522cc4 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -504,7 +504,10 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline), resources); const auto & processors = pipe.getProcessors(); - printPipeline(processors, buf); + if (settings.compact) + printPipelineCompact(processors, buf, settings.query_pipeline_options.header); + else + printPipeline(processors, buf); } else { diff --git a/src/QueryPipeline/printPipeline.cpp b/src/QueryPipeline/printPipeline.cpp new file mode 100644 index 00000000000..de978578618 --- /dev/null +++ b/src/QueryPipeline/printPipeline.cpp @@ -0,0 +1,173 @@ +#include +#include +#include +#include + +namespace DB +{ + +void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header) +{ + struct Node; + + /// Group by processors name, QueryPlanStep and group in this step. + struct Key + { + size_t group; + IQueryPlanStep * step; + std::string name; + + auto getTuple() const { return std::forward_as_tuple(group, step, name); } + + bool operator<(const Key & other) const { return getTuple() < other.getTuple(); } + }; + + /// Group ports by header. + struct EdgeData + { + Block header; + size_t count; + }; + + using Edge = std::vector; + + struct Node + { + size_t id = 0; + std::map edges = {}; + std::vector agents = {}; + }; + + std::map graph; + + auto get_key = [](const IProcessor & processor) { + return Key{processor.getQueryPlanStepGroup(), processor.getQueryPlanStep(), processor.getName()}; + }; + + /// Fill nodes. + for (const auto & processor : processors) + { + auto res = graph.emplace(get_key(*processor), Node()); + auto & node = res.first->second; + node.agents.emplace_back(processor.get()); + + if (res.second) + node.id = graph.size(); + } + + Block empty_header; + + /// Fill edges. + for (const auto & processor : processors) + { + auto & from = graph[get_key(*processor)]; + + for (auto & port : processor->getOutputs()) + { + if (!port.isConnected()) + continue; + + auto & to = graph[get_key(port.getInputPort().getProcessor())]; + auto & edge = from.edges[&to]; + + /// Use empty header for each edge if with_header is false. + const auto & header = with_header ? port.getHeader() : empty_header; + + /// Group by header. + bool found = false; + for (auto & item : edge) + { + if (blocksHaveEqualStructure(header, item.header)) + { + found = true; + ++item.count; + break; + } + } + + if (!found) + edge.emplace_back(EdgeData{header, 1}); + } + } + + /// Group processors by it's QueryPlanStep. + std::map> steps_map; + + for (const auto & item : graph) + steps_map[item.first.step].emplace_back(&item.second); + + out << "digraph\n{\n"; + out << " rankdir=\"LR\";\n"; + out << " { node [shape = rect]\n"; + + /// Nodes // TODO quoting and escaping + size_t next_step = 0; + for (const auto & item : steps_map) + { + /// Use separate clusters for each step. + if (item.first != nullptr) + { + out << " subgraph cluster_" << next_step << " {\n"; + out << " label =\"" << item.first->getName() << "\";\n"; + out << " style=filled;\n"; + out << " color=lightgrey;\n"; + out << " node [style=filled,color=white];\n"; + out << " { rank = same;\n"; + + ++next_step; + } + + for (const auto & node : item.second) + { + const auto & processor = node->agents.front(); + out << " n" << node->id << " [label=\"" << processor->getName(); + + if (node->agents.size() > 1) + out << " × " << node->agents.size(); + + const auto & description = processor->getDescription(); + if (!description.empty()) + out << ' ' << description; + + out << "\"];\n"; + } + + if (item.first != nullptr) + { + out << " }\n"; + out << " }\n"; + } + } + + out << " }\n"; + + /// Edges + for (const auto & item : graph) + { + for (const auto & edge : item.second.edges) + { + for (const auto & data : edge.second) + { + out << " n" << item.second.id << " -> " + << "n" << edge.first->id << " [label=\""; + + if (data.count > 1) + out << "× " << data.count; + + if (with_header) + { + for (const auto & elem : data.header) + { + out << "\n"; + elem.dumpStructure(out); + } + } + + out << "\"];\n"; + } + } + } + out << "}\n"; +} + +} diff --git a/src/QueryPipeline/printPipeline.h b/src/QueryPipeline/printPipeline.h index ff3b53300ce..e91909cb50b 100644 --- a/src/QueryPipeline/printPipeline.h +++ b/src/QueryPipeline/printPipeline.h @@ -64,4 +64,9 @@ void printPipeline(const Processors & processors, WriteBuffer & out) printPipeline(processors, std::vector(), out); } +/// Prints pipeline in compact representation. +/// Group processors by it's name, QueryPlanStep and QueryPlanStepGroup. +/// If QueryPlanStep wasn't set for processor, representation may be not correct. +/// If with_header is set, prints block header for each edge. +void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header); } diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph.reference b/tests/queries/0_stateless/02678_explain_pipeline_graph.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph.sql b/tests/queries/0_stateless/02678_explain_pipeline_graph.sql deleted file mode 100644 index 48cfbf2b349..00000000000 --- a/tests/queries/0_stateless/02678_explain_pipeline_graph.sql +++ /dev/null @@ -1,7 +0,0 @@ --- The server does not crash after these queries: - -DROP TABLE IF EXISTS t1; -CREATE TABLE t1(ID UInt64, name String) engine=MergeTree order by ID; -insert into t1(ID, name) values (1, 'abc'), (2, 'bbb'); -explain pipeline graph=1 select count(ID) from t1 FORMAT Null; -DROP TABLE t1; diff --git a/tests/queries/0_stateless/02678_explain_graph_with_projection.reference b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference similarity index 100% rename from tests/queries/0_stateless/02678_explain_graph_with_projection.reference rename to tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference diff --git a/tests/queries/0_stateless/02678_explain_graph_with_projection.sql b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql similarity index 100% rename from tests/queries/0_stateless/02678_explain_graph_with_projection.sql rename to tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql From 191bc9717e5f6dd7e6e3065e1588d98a3e0fa110 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 15:58:16 +0000 Subject: [PATCH 456/470] fix style --- src/QueryPipeline/printPipeline.cpp | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/src/QueryPipeline/printPipeline.cpp b/src/QueryPipeline/printPipeline.cpp index de978578618..40c88502ed0 100644 --- a/src/QueryPipeline/printPipeline.cpp +++ b/src/QueryPipeline/printPipeline.cpp @@ -1,7 +1,7 @@ -#include -#include -#include #include +#include +#include +#include namespace DB { @@ -19,7 +19,10 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool auto getTuple() const { return std::forward_as_tuple(group, step, name); } - bool operator<(const Key & other) const { return getTuple() < other.getTuple(); } + bool operator<(const Key & other) const + { + return getTuple() < other.getTuple(); + } }; /// Group ports by header. @@ -40,7 +43,8 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool std::map graph; - auto get_key = [](const IProcessor & processor) { + auto get_key = [](const IProcessor & processor) + { return Key{processor.getQueryPlanStepGroup(), processor.getQueryPlanStep(), processor.getName()}; }; @@ -60,7 +64,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool /// Fill edges. for (const auto & processor : processors) { - auto & from = graph[get_key(*processor)]; + auto & from = graph[get_key(*processor)]; for (auto & port : processor->getOutputs()) { @@ -71,7 +75,8 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool auto & edge = from.edges[&to]; /// Use empty header for each edge if with_header is false. - const auto & header = with_header ? port.getHeader() : empty_header; + const auto & header = with_header ? port.getHeader() + : empty_header; /// Group by header. bool found = false; @@ -148,8 +153,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool { for (const auto & data : edge.second) { - out << " n" << item.second.id << " -> " - << "n" << edge.first->id << " [label=\""; + out << " n" << item.second.id << " -> " << "n" << edge.first->id << " [label=\""; if (data.count > 1) out << "× " << data.count; From 0359b3e336f3e207daca42c73286ca7b5603cfb9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 11 Mar 2023 17:01:10 +0100 Subject: [PATCH 457/470] Revert "Merge pull request #46622 from ClickHouse/async-insert-memory-fix" This reverts commit 67469ad46b017360389ab80459682f6bffa26ed7, reversing changes made to 678e4250cd40c8849a0c01e1de70545455cf0c06. --- src/Common/CurrentThread.cpp | 19 --------- src/Common/CurrentThread.h | 6 --- src/Interpreters/AsynchronousInsertQueue.cpp | 6 +-- src/Interpreters/AsynchronousInsertQueue.h | 42 +------------------ .../test_async_insert_memory/__init__.py | 0 .../test_async_insert_memory/test.py | 40 ------------------ 6 files changed, 3 insertions(+), 110 deletions(-) delete mode 100644 tests/integration/test_async_insert_memory/__init__.py delete mode 100644 tests/integration/test_async_insert_memory/test.py diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index b54cf3b9371..188e78fe69b 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -110,23 +110,4 @@ ThreadGroupStatusPtr CurrentThread::getGroup() return current_thread->getThreadGroup(); } -MemoryTracker * CurrentThread::getUserMemoryTracker() -{ - if (unlikely(!current_thread)) - return nullptr; - - auto * tracker = current_thread->memory_tracker.getParent(); - while (tracker && tracker->level != VariableContext::User) - tracker = tracker->getParent(); - - return tracker; -} - -void CurrentThread::flushUntrackedMemory() -{ - if (unlikely(!current_thread)) - return; - current_thread->flushUntrackedMemory(); -} - } diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index ffc00c77504..f4975e800ca 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -40,12 +40,6 @@ public: /// Group to which belongs current thread static ThreadGroupStatusPtr getGroup(); - /// MemoryTracker for user that owns current thread if any - static MemoryTracker * getUserMemoryTracker(); - - /// Adjust counters in MemoryTracker hierarchy if untracked_memory is not 0. - static void flushUntrackedMemory(); - /// A logs queue used by TCPHandler to pass logs to a client static void attachInternalTextLogsQueue(const std::shared_ptr & logs_queue, LogsLevel client_logs_level); diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 78b173de6dc..590cbc9ba83 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -104,10 +103,9 @@ bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) return query_str == other.query_str && settings == other.settings; } -AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_) +AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_) : bytes(std::move(bytes_)) , query_id(std::move(query_id_)) - , user_memory_tracker(user_memory_tracker_) , create_time(std::chrono::system_clock::now()) { } @@ -236,7 +234,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) if (auto quota = query_context->getQuota()) quota->used(QuotaType::WRITTEN_BYTES, bytes.size()); - auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), CurrentThread::getUserMemoryTracker()); + auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId()); InsertQuery key{query, settings}; InsertDataPtr data_to_process; diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index e6b7bff8d26..23a2860364d 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -60,31 +59,6 @@ private: UInt128 calculateHash() const; }; - struct UserMemoryTrackerSwitcher - { - explicit UserMemoryTrackerSwitcher(MemoryTracker * new_tracker) - { - auto * thread_tracker = CurrentThread::getMemoryTracker(); - prev_untracked_memory = current_thread->untracked_memory; - prev_memory_tracker_parent = thread_tracker->getParent(); - - current_thread->untracked_memory = 0; - thread_tracker->setParent(new_tracker); - } - - ~UserMemoryTrackerSwitcher() - { - CurrentThread::flushUntrackedMemory(); - auto * thread_tracker = CurrentThread::getMemoryTracker(); - - current_thread->untracked_memory = prev_untracked_memory; - thread_tracker->setParent(prev_memory_tracker_parent); - } - - MemoryTracker * prev_memory_tracker_parent; - Int64 prev_untracked_memory; - }; - struct InsertData { struct Entry @@ -92,10 +66,9 @@ private: public: const String bytes; const String query_id; - MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; - Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_); + Entry(String && bytes_, String && query_id_); void finish(std::exception_ptr exception_ = nullptr); std::future getFuture() { return promise.get_future(); } @@ -106,19 +79,6 @@ private: std::atomic_bool finished = false; }; - ~InsertData() - { - auto it = entries.begin(); - // Entries must be destroyed in context of user who runs async insert. - // Each entry in the list may correspond to a different user, - // so we need to switch current thread's MemoryTracker parent on each iteration. - while (it != entries.end()) - { - UserMemoryTrackerSwitcher switcher((*it)->user_memory_tracker); - it = entries.erase(it); - } - } - using EntryPtr = std::shared_ptr; std::list entries; diff --git a/tests/integration/test_async_insert_memory/__init__.py b/tests/integration/test_async_insert_memory/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py deleted file mode 100644 index 279542f087c..00000000000 --- a/tests/integration/test_async_insert_memory/test.py +++ /dev/null @@ -1,40 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance("node") - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_memory_usage(): - node.query( - "CREATE TABLE async_table(data Array(UInt64)) ENGINE=MergeTree() ORDER BY data" - ) - - node.get_query_request("SELECT count() FROM system.numbers") - - INSERT_QUERY = "INSERT INTO async_table SETTINGS async_insert=1, wait_for_async_insert=1 VALUES ({})" - for iter in range(10): - values = list(range(iter * 5000000, (iter + 1) * 5000000)) - node.query(INSERT_QUERY.format(values)) - - response = node.get_query_request( - "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format( - 30 * (2**23) - ) - ) - - _, err = response.get_answer_and_error() - assert err == "", "Query failed with error {}".format(err) - - node.query("DROP TABLE async_table") From 6724bfdf389faedc95bc79a8e457883b3a1b7650 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 11 Mar 2023 19:06:07 +0300 Subject: [PATCH 458/470] Disable grace hash join in upgrade check (#47474) * Update stress.py * Update stress.py * Update stress.py --- tests/ci/stress.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 37b3b1c181e..5e151e6c098 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -36,7 +36,8 @@ def get_options(i, upgrade_check): client_options.append("join_algorithm='partial_merge'") if join_alg_num % 5 == 2: client_options.append("join_algorithm='full_sorting_merge'") - if join_alg_num % 5 == 3: + if join_alg_num % 5 == 3 and not upgrade_check: + # Some crashes are not fixed in 23.2 yet, so ignore the setting in Upgrade check client_options.append("join_algorithm='grace_hash'") if join_alg_num % 5 == 4: client_options.append("join_algorithm='auto'") From 3190b89f9aff8fc906e36634ef002bfc125feaa6 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 18:04:19 +0000 Subject: [PATCH 459/470] Fix --- src/Processors/QueryPlan/ISourceStep.cpp | 8 +- ...n_pipeline_graph_with_projection.reference | 116 ------------------ ...explain_pipeline_graph_with_projection.sql | 9 +- 3 files changed, 11 insertions(+), 122 deletions(-) diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 562f92ff63b..062f6d85bac 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -12,15 +12,19 @@ ISourceStep::ISourceStep(DataStream output_stream_) QueryPipelineBuilderPtr ISourceStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings & settings) { auto pipeline = std::make_unique(); + + /// Why we need first initializePipeline first: since it's not + /// add new Processors to `pipeline->pipe`, but make an assign + /// with new created Pipe. And Processors for the Step is added here. initializePipeline(*pipeline, settings); + QueryPipelineProcessorsCollector collector(*pipeline, this); /// Properly collecting processors from Pipe. /// At the creation time of a Pipe, since `collected_processors` is nullptr, /// the processors can not be collected. pipeline->collectProcessors(); - auto added_processors = collector.detachProcessors(); - processors.insert(processors.end(), added_processors.begin(), added_processors.end()); + collector.detachProcessors(); return pipeline; } diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference index 0c513f735cc..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference +++ b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.reference @@ -1,116 +0,0 @@ -digraph -{ - rankdir="LR"; - { node [shape = rect] - subgraph cluster_0 { - label ="Expression"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n4 [label="ExpressionTransform"]; - } - } - subgraph cluster_1 { - label ="ReadFromStorage"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n3 [label="AggregatingTransform"]; - n2 [label="ExpressionTransform"]; - n1 [label="SourceFromSingleChunk"]; - } - } - } - n3 -> n4 [label=""]; - n2 -> n3 [label=""]; - n1 -> n2 [label=""]; -} -digraph -{ - rankdir="LR"; - { node [shape = rect] - subgraph cluster_0 { - label ="Expression"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n4 [label="ExpressionTransform"]; - } - } - subgraph cluster_1 { - label ="ReadFromStorage"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n3 [label="AggregatingTransform"]; - n2 [label="ExpressionTransform"]; - n1 [label="SourceFromSingleChunk"]; - } - } - } - n3 -> n4 [label=""]; - n2 -> n3 [label=""]; - n1 -> n2 [label=""]; -} -digraph -{ - rankdir="LR"; - { node [shape = rect] - subgraph cluster_0 { - label ="Expression"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n4 [label="ExpressionTransform"]; - } - } - subgraph cluster_1 { - label ="ReadFromStorage"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n3 [label="AggregatingTransform"]; - n2 [label="ExpressionTransform"]; - n1 [label="SourceFromSingleChunk"]; - } - } - } - n3 -> n4 [label=""]; - n2 -> n3 [label=""]; - n1 -> n2 [label=""]; -} -digraph -{ - rankdir="LR"; - { node [shape = rect] - subgraph cluster_0 { - label ="Expression"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n4 [label="ExpressionTransform"]; - } - } - subgraph cluster_1 { - label ="ReadFromStorage"; - style=filled; - color=lightgrey; - node [style=filled,color=white]; - { rank = same; - n3 [label="AggregatingTransform"]; - n2 [label="ExpressionTransform"]; - n1 [label="SourceFromSingleChunk"]; - } - } - } - n3 -> n4 [label=""]; - n2 -> n3 [label=""]; - n1 -> n2 [label=""]; -} diff --git a/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql index 06ee1078e74..e8b7405d602 100644 --- a/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql +++ b/tests/queries/0_stateless/02678_explain_pipeline_graph_with_projection.sql @@ -3,9 +3,10 @@ CREATE TABLE t1(ID UInt64, name String) engine=MergeTree order by ID; insert into t1(ID, name) values (1, 'abc'), (2, 'bbb'); -explain pipeline graph=1 select count(ID) from t1; -explain pipeline graph=1 select sum(1) from t1; -explain pipeline graph=1 select min(ID) from t1; -explain pipeline graph=1 select max(ID) from t1; +-- The returned node order is uncertain +explain pipeline graph=1 select count(ID) from t1 FORMAT Null; +explain pipeline graph=1 select sum(1) from t1 FORMAT Null; +explain pipeline graph=1 select min(ID) from t1 FORMAT Null; +explain pipeline graph=1 select max(ID) from t1 FORMAT Null; DROP TABLE t1; From 26828449518ace4433b001243052af249ea1c1fc Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 18:06:50 +0000 Subject: [PATCH 460/470] fix --- src/Processors/QueryPlan/ISourceStep.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 062f6d85bac..0ea386518b7 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -13,8 +13,8 @@ QueryPipelineBuilderPtr ISourceStep::updatePipeline(QueryPipelineBuilders, const { auto pipeline = std::make_unique(); - /// Why we need first initializePipeline first: since it's not - /// add new Processors to `pipeline->pipe`, but make an assign + /// Why we need initializePipeline first: since it's not add + /// new Processors to `pipeline->pipe`, but make an assign /// with new created Pipe. And Processors for the Step is added here. initializePipeline(*pipeline, settings); From e56eeac05fe4afcf3c2f73c1e28d8a5f39d09ab3 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 11 Mar 2023 18:48:36 +0000 Subject: [PATCH 461/470] fix --- src/Processors/QueryPlan/ISourceStep.cpp | 20 ++++++++++---------- src/QueryPipeline/Pipe.cpp | 10 ---------- src/QueryPipeline/Pipe.h | 2 -- src/QueryPipeline/QueryPipelineBuilder.h | 2 -- 4 files changed, 10 insertions(+), 24 deletions(-) diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 0ea386518b7..37f56bc7a43 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -13,18 +13,18 @@ QueryPipelineBuilderPtr ISourceStep::updatePipeline(QueryPipelineBuilders, const { auto pipeline = std::make_unique(); - /// Why we need initializePipeline first: since it's not add - /// new Processors to `pipeline->pipe`, but make an assign - /// with new created Pipe. And Processors for the Step is added here. + /// For `Source` step, since it's not add new Processors to `pipeline->pipe` + /// in `initializePipeline`, but make an assign with new created Pipe. + /// And Processors for the Step is added here. So we do not need to use + /// `QueryPipelineProcessorsCollector` to collect Processors. initializePipeline(*pipeline, settings); - QueryPipelineProcessorsCollector collector(*pipeline, this); - - /// Properly collecting processors from Pipe. - /// At the creation time of a Pipe, since `collected_processors` is nullptr, - /// the processors can not be collected. - pipeline->collectProcessors(); - collector.detachProcessors(); + /// But we need to set QueryPlanStep manually for the Processors, which + /// will be used in `EXPLAIN PIPELINE` + for (auto & processor : processors) + { + processor->setQueryPlanStep(this); + } return pipeline; } diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index a20b154054a..91ba01c479f 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -867,14 +867,4 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } -void Pipe::collectProcessors() -{ - if (collected_processors) - { - for (const auto & processor : *processors) - { - collected_processors->emplace_back(processor); - } - } -} } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index a19364f7907..09931e38578 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -104,8 +104,6 @@ public: std::shared_ptr getProcessorsPtr() { return processors; } - void collectProcessors(); - private: /// Header is common for all output below. Block header; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 51a1aa69563..4f984680c75 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -185,8 +185,6 @@ public: static Pipe getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources); static QueryPipeline getPipeline(QueryPipelineBuilder builder); - void collectProcessors() { pipe.collectProcessors(); } - private: /// Destruction order: processors, header, locks, temporary storages, local contexts From 4cfc040829ea169f6d25a29ef9c35431559bcd9f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 20:16:06 +0100 Subject: [PATCH 462/470] Fix parameterized queries --- .../ReplaceQueryParameterVisitor.cpp | 10 ++++++ src/Parsers/ASTAlterQuery.h | 5 +++ src/Parsers/ASTBackupQuery.h | 7 ++++ src/Parsers/ASTConstraintDeclaration.h | 6 ++++ src/Parsers/ASTCreateQuery.cpp | 5 +++ src/Parsers/ASTCreateQuery.h | 34 +++++++++++++++++++ src/Parsers/ASTDictionary.h | 5 +++ src/Parsers/ASTExternalDDLQuery.h | 5 +++ .../ASTFunctionWithKeyValueArguments.h | 5 +++ src/Parsers/ASTIndexDeclaration.h | 6 ++++ src/Parsers/ASTProjectionDeclaration.h | 5 +++ src/Parsers/ASTTableOverrides.h | 6 ++++ src/Parsers/IAST.h | 14 ++++++++ src/Parsers/MySQL/ASTAlterCommand.h | 9 +++++ src/Parsers/MySQL/ASTCreateDefines.h | 8 ++++- 15 files changed, 129 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index f271de26ca4..893c93f0950 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -50,7 +50,16 @@ void ReplaceQueryParameterVisitor::visit(ASTPtr & ast) void ReplaceQueryParameterVisitor::visitChildren(ASTPtr & ast) { for (auto & child : ast->children) + { + void * old_ptr = child.get(); visit(child); + void * new_ptr = child.get(); + + /// Some AST classes have naked pointers to children elements as members. + /// We have to replace them if the child was replaced. + if (new_ptr != old_ptr) + ast->updatePointerToChild(old_ptr, new_ptr); + } } const String & ReplaceQueryParameterVisitor::getParamValue(const String & name) @@ -89,6 +98,7 @@ void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) literal = value; else literal = temp_column[0]; + ast = addTypeConversionToAST(std::make_shared(literal), type_name); /// Keep the original alias. diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 2a48f5bbd9e..1400113fa9c 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -256,6 +256,11 @@ protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; bool isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&command_list)); + } }; } diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index a3e3a144c72..0201c2b14f9 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -94,5 +94,12 @@ public: void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override; QueryKind getQueryKind() const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&backup_name)); + f(reinterpret_cast(&base_backup_name)); + } }; + } diff --git a/src/Parsers/ASTConstraintDeclaration.h b/src/Parsers/ASTConstraintDeclaration.h index 437aab1a82d..f48d7ef77fe 100644 --- a/src/Parsers/ASTConstraintDeclaration.h +++ b/src/Parsers/ASTConstraintDeclaration.h @@ -25,5 +25,11 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&expr)); + } }; + } diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 955ce62b0f7..e28e863c21f 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -91,6 +91,11 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&elem)); + } }; ASTPtr ASTColumnsElement::clone() const diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 90a15e09369..230996f610e 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -32,6 +32,17 @@ public: void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; bool isExtendedStorageDefinition() const; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&engine)); + f(reinterpret_cast(&partition_by)); + f(reinterpret_cast(&primary_key)); + f(reinterpret_cast(&order_by)); + f(reinterpret_cast(&sample_by)); + f(reinterpret_cast(&ttl_table)); + f(reinterpret_cast(&settings)); + } }; @@ -57,6 +68,16 @@ public: return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty()) && (!projections || projections->children.empty()); } + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&columns)); + f(reinterpret_cast(&indices)); + f(reinterpret_cast(&primary_key)); + f(reinterpret_cast(&constraints)); + f(reinterpret_cast(&projections)); + f(reinterpret_cast(&primary_key)); + } }; @@ -126,6 +147,19 @@ public: protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&columns_list)); + f(reinterpret_cast(&inner_storage)); + f(reinterpret_cast(&storage)); + f(reinterpret_cast(&as_table_function)); + f(reinterpret_cast(&select)); + f(reinterpret_cast(&comment)); + f(reinterpret_cast(&table_overrides)); + f(reinterpret_cast(&dictionary_attributes_list)); + f(reinterpret_cast(&dictionary)); + } }; } diff --git a/src/Parsers/ASTDictionary.h b/src/Parsers/ASTDictionary.h index 3611621b8ad..8c332247d52 100644 --- a/src/Parsers/ASTDictionary.h +++ b/src/Parsers/ASTDictionary.h @@ -47,6 +47,11 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(¶meters)); + } }; diff --git a/src/Parsers/ASTExternalDDLQuery.h b/src/Parsers/ASTExternalDDLQuery.h index 7913d44b970..96600b07f29 100644 --- a/src/Parsers/ASTExternalDDLQuery.h +++ b/src/Parsers/ASTExternalDDLQuery.h @@ -41,6 +41,11 @@ public: } QueryKind getQueryKind() const override { return QueryKind::ExternalDDL; } + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&from)); + } }; } diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.h b/src/Parsers/ASTFunctionWithKeyValueArguments.h index 67d591dfcdc..75a8ae0415e 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.h +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.h @@ -33,6 +33,11 @@ public: bool hasSecretParts() const override; void updateTreeHashImpl(SipHash & hash_state) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&second)); + } }; diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index e22c1da4489..bd52a611f3f 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -23,6 +23,12 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&expr)); + f(reinterpret_cast(&type)); + } }; } diff --git a/src/Parsers/ASTProjectionDeclaration.h b/src/Parsers/ASTProjectionDeclaration.h index 53c681c3ec1..df7a7c832a6 100644 --- a/src/Parsers/ASTProjectionDeclaration.h +++ b/src/Parsers/ASTProjectionDeclaration.h @@ -18,6 +18,11 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&query)); + } }; } diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index c47260789d8..1df267acaa9 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -27,6 +27,12 @@ public: String getID(char) const override { return "TableOverride " + table_name; } ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&columns)); + f(reinterpret_cast(&storage)); + } }; /// List of table overrides, for example: diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 627b1174b33..5928506aa5b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -175,6 +175,16 @@ public: field = nullptr; } + /// After changing one of `children` elements, update the corresponding member pointer if needed. + void updatePointerToChild(void * old_ptr, void * new_ptr) + { + forEachPointerToChild([old_ptr, new_ptr](void ** ptr) mutable + { + if (*ptr == old_ptr) + *ptr = new_ptr; + }); + } + /// Convert to a string. /// Format settings. @@ -295,6 +305,10 @@ public: protected: bool childrenHaveSecretParts() const; + /// Some AST classes have naked pointers to children elements as members. + /// This method allows to iterate over them. + virtual void forEachPointerToChild(std::function) {} + private: size_t checkDepthImpl(size_t max_depth) const; diff --git a/src/Parsers/MySQL/ASTAlterCommand.h b/src/Parsers/MySQL/ASTAlterCommand.h index f097ed71219..87b665ec6a5 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.h +++ b/src/Parsers/MySQL/ASTAlterCommand.h @@ -80,6 +80,15 @@ protected: { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method formatImpl is not supported by MySQLParser::ASTAlterCommand."); } + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&index_decl)); + f(reinterpret_cast(&default_expression)); + f(reinterpret_cast(&additional_columns)); + f(reinterpret_cast(&order_by_columns)); + f(reinterpret_cast(&properties)); + } }; class ParserAlterCommand : public IParserBase diff --git a/src/Parsers/MySQL/ASTCreateDefines.h b/src/Parsers/MySQL/ASTCreateDefines.h index 3d2a79568ab..7c23d1cb87f 100644 --- a/src/Parsers/MySQL/ASTCreateDefines.h +++ b/src/Parsers/MySQL/ASTCreateDefines.h @@ -31,6 +31,13 @@ protected: { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method formatImpl is not supported by MySQLParser::ASTCreateDefines."); } + + void forEachPointerToChild(std::function f) override + { + f(reinterpret_cast(&columns)); + f(reinterpret_cast(&indices)); + f(reinterpret_cast(&constraints)); + } }; class ParserCreateDefines : public IParserBase @@ -44,4 +51,3 @@ protected: } } - From dccd919a41cfb2e0897aef5bd9b4013eb348520c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 20:18:01 +0100 Subject: [PATCH 463/470] Add a test --- .../02679_query_parameters_dangling_pointer.reference | 0 .../0_stateless/02679_query_parameters_dangling_pointer.sql | 4 ++++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02679_query_parameters_dangling_pointer.reference create mode 100644 tests/queries/0_stateless/02679_query_parameters_dangling_pointer.sql diff --git a/tests/queries/0_stateless/02679_query_parameters_dangling_pointer.reference b/tests/queries/0_stateless/02679_query_parameters_dangling_pointer.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02679_query_parameters_dangling_pointer.sql b/tests/queries/0_stateless/02679_query_parameters_dangling_pointer.sql new file mode 100644 index 00000000000..7705b860e8e --- /dev/null +++ b/tests/queries/0_stateless/02679_query_parameters_dangling_pointer.sql @@ -0,0 +1,4 @@ +-- There is no use-after-free in the following query: + +SET param_o = 'a'; +CREATE TABLE test.xxx (a Int64) ENGINE=MergeTree ORDER BY ({o:String}); -- { serverError 44 } From ff47aa1eb25c22574e7cbd06a5d1dd53c3c25905 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 21:30:41 +0100 Subject: [PATCH 464/470] Small refinement in test --- ...x_extra_seek_on_compressed_cache.reference | 2 ++ ...test_fix_extra_seek_on_compressed_cache.sh | 21 +++++++++++++------ 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.reference b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.reference index f1839bae259..e142c6c79fe 100644 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.reference +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.reference @@ -1 +1,3 @@ +99999 +99999 0 0 13 diff --git a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh index 390d6a70ef1..7bf4a88e972 100755 --- a/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh +++ b/tests/queries/0_stateless/00933_test_fix_extra_seek_on_compressed_cache.sh @@ -13,15 +13,24 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE small_table (a UInt64 default 0, n UInt $CLICKHOUSE_CLIENT --query="INSERT INTO small_table (n) SELECT * from system.numbers limit 100000;" $CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE small_table FINAL;" -cached_query="SELECT count() FROM small_table where n > 0;" +cached_query="SELECT count() FROM small_table WHERE n > 0;" -$CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query="$cached_query" &> /dev/null - -$CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --allow_prefetched_read_pool_for_remote_filesystem=0 --allow_prefetched_read_pool_for_local_filesystem=0 --query_id="test-query-uncompressed-cache" --query="$cached_query" &> /dev/null +$CLICKHOUSE_CLIENT --log_queries 1 --use_uncompressed_cache 1 --query="$cached_query" +$CLICKHOUSE_CLIENT --log_queries 1 --use_uncompressed_cache 1 --allow_prefetched_read_pool_for_remote_filesystem 0 --allow_prefetched_read_pool_for_local_filesystem 0 --query_id="test-query-uncompressed-cache" --query="$cached_query" $CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS" - -$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents['Seek'], ProfileEvents['ReadCompressedBytes'], ProfileEvents['UncompressedCacheHits'] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') and current_database = currentDatabase() AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1" +$CLICKHOUSE_CLIENT --query=" + SELECT + ProfileEvents['Seek'], + ProfileEvents['ReadCompressedBytes'], + ProfileEvents['UncompressedCacheHits'] AS hit + FROM system.query_log + WHERE query_id = 'test-query-uncompressed-cache' + AND current_database = currentDatabase() + AND type = 2 + AND event_date >= yesterday() + ORDER BY event_time DESC + LIMIT 1" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table" From 218ac10425d6b06f1eb181bf1b660d8a3af8ecc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 21:44:38 +0100 Subject: [PATCH 465/470] More detailed error messages on too many parts --- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 73007e3f178..d7cea944689 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4121,9 +4121,9 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( ErrorCodes::TOO_MANY_PARTS, - "Too many parts ({}) in all partitions in total. This indicates wrong choice of partition key. The threshold can be modified " + "Too many parts ({}) in all partitions in total in table '{}'. This indicates wrong choice of partition key. The threshold can be modified " "with 'max_parts_in_total' setting in element in config.xml or with per-table setting.", - parts_count_in_total); + parts_count_in_total, getLogName()); } size_t outdated_parts_over_threshold = 0; @@ -4137,8 +4137,8 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( ErrorCodes::TOO_MANY_PARTS, - "Too many inactive parts ({}). Parts cleaning are processing significantly slower than inserts", - outdated_parts_count_in_partition); + "Too many inactive parts ({}) in table '{}'. Parts cleaning are processing significantly slower than inserts", + outdated_parts_count_in_partition, getLogName()); } if (settings->inactive_parts_to_delay_insert > 0 && outdated_parts_count_in_partition >= settings->inactive_parts_to_delay_insert) outdated_parts_over_threshold = outdated_parts_count_in_partition - settings->inactive_parts_to_delay_insert + 1; @@ -4151,6 +4151,7 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex const auto active_parts_to_throw_insert = query_settings.parts_to_throw_insert ? query_settings.parts_to_throw_insert : settings->parts_to_throw_insert; size_t active_parts_over_threshold = 0; + { bool parts_are_large_enough_in_average = settings->max_avg_part_size_for_too_many_parts && average_part_size > settings->max_avg_part_size_for_too_many_parts; @@ -4160,9 +4161,10 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, const Contex ProfileEvents::increment(ProfileEvents::RejectedInserts); throw Exception( ErrorCodes::TOO_MANY_PARTS, - "Too many parts ({} with average size of {}). Merges are processing significantly slower than inserts", + "Too many parts ({} with average size of {}) in table '{}'. Merges are processing significantly slower than inserts", parts_count_in_partition, - ReadableSize(average_part_size)); + ReadableSize(average_part_size), + getLogName()); } if (active_parts_to_delay_insert > 0 && parts_count_in_partition >= active_parts_to_delay_insert && !parts_are_large_enough_in_average) From bd8277c001786d491212c35a6925f56c26a73936 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 21:44:56 +0100 Subject: [PATCH 466/470] Fix "00933_test_fix_extra_seek_on_compressed_cache" in releases --- tests/queries/0_stateless/02286_drop_filesystem_cache.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh index b563c487646..991622446b8 100755 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh @@ -16,7 +16,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do ORDER BY key SETTINGS storage_policy='$STORAGE_POLICY', min_bytes_for_wide_part = 10485760" - $CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES" + $CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES test_02286" $CLICKHOUSE_CLIENT --query "SYSTEM DROP FILESYSTEM CACHE" $CLICKHOUSE_CLIENT --query "SELECT count() FROM system.filesystem_cache" From 898f980b2dcef7a77195fc59acc5c338f35b66eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 22:00:54 +0100 Subject: [PATCH 467/470] Fix long test 02371_select_projection_normal_agg.sql --- .../0_stateless/02371_select_projection_normal_agg.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02371_select_projection_normal_agg.sql b/tests/queries/0_stateless/02371_select_projection_normal_agg.sql index 283aec0b122..8650fb6b843 100644 --- a/tests/queries/0_stateless/02371_select_projection_normal_agg.sql +++ b/tests/queries/0_stateless/02371_select_projection_normal_agg.sql @@ -11,7 +11,8 @@ CREATE TABLE video_log ) ENGINE = MergeTree PARTITION BY toDate(datetime) -ORDER BY (user_id, device_id); +ORDER BY (user_id, device_id) +SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; DROP TABLE IF EXISTS rng; @@ -57,7 +58,8 @@ CREATE TABLE video_log_result ) ENGINE = MergeTree PARTITION BY toDate(hour) -ORDER BY sum_bytes; +ORDER BY sum_bytes +SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; INSERT INTO video_log_result SELECT toStartOfHour(datetime) AS hour, From 5a18acde90d2ef04b4126ed6d52493c47cf94ebd Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 11 Mar 2023 21:15:14 +0000 Subject: [PATCH 468/470] Revert #45878 and add a test --- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - .../Formats/Impl/ParquetBlockInputFormat.cpp | 58 +++++++++---------- 4 files changed, 26 insertions(+), 35 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f67ce6be9ed..881cbe42a02 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -803,7 +803,6 @@ class IColumn; M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ - M(UInt64, input_format_parquet_max_block_size, 8192, "Max block size for parquet reader.", 0) \ M(Bool, input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format Protobuf", 0) \ M(Bool, input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format CapnProto", 0) \ M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a951a7fdd92..a6db9d5ba0d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -117,7 +117,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; - format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 7be7b5b98aa..ef6be805bea 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -211,7 +211,6 @@ struct FormatSettings std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; - UInt64 max_block_size = 8192; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; } parquet; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index fca097d8ea7..87a0fbf77a8 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -45,44 +45,38 @@ Chunk ParquetBlockInputFormat::generate() block_missing_values.clear(); if (!file_reader) - { prepareReader(); - file_reader->set_batch_size(format_settings.parquet.max_block_size); - std::vector row_group_indices; - for (int i = 0; i < row_group_total; ++i) - { - if (!skip_row_groups.contains(i)) - row_group_indices.emplace_back(i); - } - auto read_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, ¤t_record_batch_reader); - if (!read_status.ok()) - throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); - } if (is_stopped) return {}; - auto batch = current_record_batch_reader->Next(); - if (!batch.ok()) - { - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", - batch.status().ToString()); - } - if (*batch) - { - auto tmp_table = arrow::Table::FromRecordBatches({*batch}); - /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. - /// Otherwise fill the missing columns with zero values of its type. - BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); - } - else - { - current_record_batch_reader.reset(); - file_reader.reset(); - return {}; - } + while (row_group_current < row_group_total && skip_row_groups.contains(row_group_current)) + ++row_group_current; + if (row_group_current >= row_group_total) + return res; + + std::shared_ptr table; + + std::unique_ptr<::arrow::RecordBatchReader> rbr; + std::vector row_group_indices { row_group_current }; + arrow::Status get_batch_reader_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, &rbr); + + if (!get_batch_reader_status.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", + get_batch_reader_status.ToString()); + + arrow::Status read_status = rbr->ReadAll(&table); + + if (!read_status.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString()); + + ++row_group_current; + + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, table->num_rows(), block_missing_values_ptr); return res; } From c915ec8a48305156a516a5ea59a21482ae0c063c Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 11 Mar 2023 21:16:05 +0000 Subject: [PATCH 469/470] Add test --- .../0_stateless/02588_parquet_bug.reference | 3 +++ .../queries/0_stateless/02588_parquet_bug.sh | 11 +++++++++++ .../data_parquet/02588_data.parquet | Bin 0 -> 2699811 bytes 3 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02588_parquet_bug.reference create mode 100755 tests/queries/0_stateless/02588_parquet_bug.sh create mode 100644 tests/queries/0_stateless/data_parquet/02588_data.parquet diff --git a/tests/queries/0_stateless/02588_parquet_bug.reference b/tests/queries/0_stateless/02588_parquet_bug.reference new file mode 100644 index 00000000000..44de58ae5c3 --- /dev/null +++ b/tests/queries/0_stateless/02588_parquet_bug.reference @@ -0,0 +1,3 @@ +cta 224.0.90.10 1670964058771367936 64066044 NYSE cqs_pillar quote \N \N \N 82.92 1 R 82.97 2 R +1670964058771367936 +1670946478544048640 ARCA cqs_pillar diff --git a/tests/queries/0_stateless/02588_parquet_bug.sh b/tests/queries/0_stateless/02588_parquet_bug.sh new file mode 100755 index 00000000000..f7e4ecf5e4c --- /dev/null +++ b/tests/queries/0_stateless/02588_parquet_bug.sh @@ -0,0 +1,11 @@ +#!/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 * from file('$CURDIR/data_parquet/02588_data.parquet') where exchange_ts = 1670964058771367936" +$CLICKHOUSE_LOCAL -q "select exchange_ts from file('$CURDIR/data_parquet/02588_data.parquet') where exchange_ts = 1670964058771367936" +$CLICKHOUSE_LOCAL -q "select exchange_ts, market, product from file('$CURDIR/data_parquet/02588_data.parquet') where exchange_ts = 1670946478544048640" + diff --git a/tests/queries/0_stateless/data_parquet/02588_data.parquet b/tests/queries/0_stateless/data_parquet/02588_data.parquet new file mode 100644 index 0000000000000000000000000000000000000000..e00b869233d7c0a3a4484bf968f39123c9836913 GIT binary patch literal 2699811 zcmcG$2Ut|e*6&>nyI}(j-Ox0mqU4N9l3;8yA~~x_j)F)QF(Hx_1e9b%MZ`FWidiw| zgb3=WW6oK}9KL_`%sDgXeLd%%`#ksa@wa+c)v8>zD(t3x9sDgsj7USI<%tiXLWq*J zG$AA=D~f((XQhdVNKMqwOjIf@5{P6(G8#6-NE*TFR6s?+h$@y!b2z2a#9%o`L&WPl zR6u0ZcXUR~J1Ic#86+ zB@La=Gb7;}PPKTO_aXkry16V*y#3HNj(~Mod6>(R)ru^+smL;BtK`&qgH|2*%rRn2 zSIe<_<%wRVK8Q0YLyl!nADqt>c#brxIO7kmn&6n^P<+013oPNV>;j`8F3V&Lmt&c4 zGa%Sy5GyxAygUvu@&ppFofyouV+NV4RM!a!D`d{am8?EXH?41C0tKd{5IsLBbHI9HRP-I!X zxR+c3Zzd!6D@WZ^93$Nw1}&f7Elv~h*;piW;jp}XgP1tv@iZERJsErz`hw735;*GY zT-QBl&3Z+aCz~gg!cpgm;xtn^EE$0^^evYYkU-?nF?SkQi+6tSyEKjwyH)ykBg4|j72GVKESICQ5Kajxn zSD9HHb4K|!D%=O1ITl19HQzUKS|k>Zd0;tc2ggtdeW@&NDAR8gT1kC0BAG#XN-9dY z4-Y;$D8s;Vc#ng6JbY^A$*GeTIBvng&X-eXmRBKa|3-*gC@k~+62#_iMA-|CftQhz z9v<9#k<$1p@J>bKQ{V!8bFay1G0T60FN(wdkj>`4Vzn+EL_H1!JjvA#<+Ue#nud0s zqgyf^tyhjHYxR)gZp1$aeuu~145Tq26S=CcL^s?DxzBYc4-p_U4w|SA5rY7b0MAov z<&2mctKU9SHvRkyWW429VKJt|lsgOH$f`eDf)S=trdf(oEQxU| zLkAq6a~wfK&g_~Gz3XF-VYFsWL=_gJ3NNJ|aMgLzFP>JSOOw|<#~`S|AW;1Fqza=V zXYMn2EQ5#C+K_5^WK120o^u^G0SR7s{<(&uPCg=3b6xjBq`hkFWvHK|>N%Q=pq|{M z1@-TGI#546b6PD&of(JOMJ6NX#Z_my>dbERJ3EMT4kPjG)&ngZbKcD1U29Qxvs>Tx zW29dG0rjuSEPG){ClY*bVu1v|ow7uN0Y1MUL4vuya;zN_E`OE|^XJ#z&`%h=RnnsM z(tk5~>g+!P@${!bQ6ok_@r41r$a!_)PZP*m5ss8J!VE3ib{NevWieuq}E5h zZ@>N;0|shpY3u0f=^Gdt8Jn1z4H`Ve+``hzdZ>-9o&B)kBSwxIEp~8pa&~cbb06a| z*3-+|$Jfu#KOitDI3y%=-1rG$6T>4SqoQMC z&>x9Xf)wEOnnU(~k|yrb4AEd1VhQ3v4Ojtofjz*2%MeH41R7)+@&FK)AyOa}7|Su_ zBiJI(ke6UMk0A?zgMc9iLm4umks)DVA_xZ&AQGg3-_yCoF@sBfWTKE+TvCRD_kdMD@R#!$PH!llqCu%924jvgFcKS>iTBmJFQ@-AY-K2$H}gu-uCf zqefX`4~Bt9EogJ_9h_PxOU{EH@NFyn!0;XD8{h^?cgm6quo&##FH81-y&$ktmP`cU zz=37SFn{Ecz!GC%0#2o1)d1H((F~RpgScQqRyDD7EB8N56t+qX4N_<5WWm4xpUu!8 zR_K2kp}*On|EDIX^WdSwv{D8QgET}^f0`kg%((Q_?8$#@8#fBn_6rPJj18`7&~4-2 z+v2BIrjjEn;CPBuBu2)~4)dP4IJ<>X!|gael*8va_&Ygp`Ruq*DUKRf#6zG1XF#ZD zV1NteGjSc`?e8Fi2XBV}X9quY$Tg1WCg*Q!1_~kbN1!@G(}M2gSN?x442+dPifBpc zSin~YNIB~0#6>#m;Ge&yO6jmR36>ZE#|U zLCav;SpQnipEj4BXv9Ab3>qo&C{#>(W<*+2N=j6Q#Fh0A*Uyd%@-hDmR+s-PB2xJ; z4hvtc|Eh#+TE>*v?3h1m@rtANkz;V(%AnAS*5cpg^3(e0iH!bH9~EcXq3PLEvf}7N zRz_58+|O#km4v8sC7S4`Tr~6@(8)0NVQ+;D5D+4eHHMZFPp~F$FfOD)a?Ur%7yo6mVgt)h(7%tCTXhGs@u2NR9uQ&rhqwim*A7l2nP| z(XybpqePB>>pGeYq5VNLbN^Gf;52Tq)A6qn%KxKmF;h}wld_Vgr2b(m8aQfA9D_~= zg9Zml0cpxVEm)_ts7K0EWG5-wEi{xRhcMn^F5fwSthje|+DLiZKyeryqP##kbu;28 zE_&*>bKqrPu@#5ItoFkhTRW;0_eO}YJy84%ibcr*az99a5NY0!{D+;s(x=T+4o&CHSAd*G8AN8$T} zr!Rjq;7EVh;c&lmxkCs;j%Ya-l6-+V%AP4BQM>BVFM92PRiPSM-sJc2`ed)i*R}ufIC8fDyGR9xZU0q7H zTHsPZ+^;n&su0Z@)ubh>+`F2Sg!x7_&0}=zN%tbxF2;SK9?57)@*o_X0;FVER*NI0 z$1zyy{nu<{X~X*&mhsAD3l1MjU^2 z4)%6f>;6;SI1i+VLAUQ;vr&clgUvp)fPbX?)1qOasQhc8AViU) zC7DxI(r_{v*|Yohbj?7Y&g2YTAEl0cHph=t${G6KsOafv&Wb6=qIAe0)4MS~#r42j!Or z2D#|d4HK8>2Q?D8prBgfTRJXCu8LGO!n-P{v9WPHfm4p;6V{h-eYw75J;tBG{NTS$ zh^WluzfXuGj9M#WP!nKq&kqyge;1z4{r~Ozb2F$V((@Ftk_jQ8tBPbI(7{2M9E^{} z98uHXn~)AP8uZUe;PuM$KQ)t-P;6q-w7zw{$$G8G@%@NYb9swEsz1?c`fR>lDRjf( zPCnWA;}>(DG&4ubh&3Sw`Q!gK;%UwQ8u81e)LNws8bS@Ogh)pG-wN{6VfFuf#P1CI z+lZ%S`{{_^JHeo4!vA5!{}=b`SB(DC{rVLn4(EyyF}NB2SCRgCzkbE&&-?YlaDx*Q z>2B?>3;Lh7)c@TLdz-XqCoY`+Uci5U@lD7T&Z}P9RkX;`QiE}^9uCHX?_7vy43U<; z#Uau{EPM_EBCTyiWDWE3Rfu$jD#BHXEXQ1INu;^fL|VBY;c~SGlKw|0rC43vRmg6i zYRZvUAqoc_Rs~9F5UzB7BI1M*A;;WI-q8_er=3H%%35adjYs+#L@r^Wo0JxjmUATr z9NB_>vcnM6A%$FcVsE0Tti{>?@}??hWPHc0=r!@Y_2uTC;$yCnC(JaIS@!}lch46s zp3E(EL~6o9cQf{{_W#$n1L%ZjjM_)W;83K&g9zFQ|Gtra+9_@AQljQVu!}7Qk_p<- z$nhl=`((b*>%TkgawHs|yVpt$!R6t&PFx<&!Cm8dzBKL{TLid3Z8U&iCoWdoEO5bF zVhOKvDe$@tujEDWa;$}y+X@Jcz3_hv|BJZUW*?g|+%%?h5CWFlLOivDsPut|!wn=8 zG9KbyI>cVQAYdBln*>2KE;pB0ak-&9rwTL)cdi&{t`_!`z6>~UZp2>kydapb5)%`{Ia2a9LitF7ilLzBA z^yQw|b-4MpJJ_%R8Tgz*>}p(UGfB8GW?XS&&5U{j@%25#gAXXz5l$ml%mk>zZQ2(? zLl+`Q527>~V(VOpWrYyaDLqoZFOF;)qta@(8$Z*A8}Yc?ezw# z5$Hu1yo2F&JNkGTS)aJR{vNVEcDKNa2US)(-& z&EQ$(V2WnQZ*YTOSgPqD_?=Qqw?huE`XH^_+7S78l||N~93dl6Tz6M^*@Z*sMnJU0 zAn+^#NfH8MXaR=NUUs1`O+w+XUJ3udeZ{ejkQqNvjyTFT7ubg8shC)h`}f> z(?~~H#7hVpv@9p%y( zb4(*fIg|Dg4H_+#%57l880C^zLOvRZF^mo$#t`E4Xox^D0;eN`?MQO12Qnx?Lov-L zkYr+NRN&o=6G|FYo8+pk&P;7W=+@m3%lAMmI12IZ1_XN(B8Zkmd<$X##tAFWOu`lh z1Bj#mYH2Ph;T@2+F%)-kk+)D=Tp548!*_WosaY481a~a z9sZbgX9M>};6pq`HyR(D{L_i}7-pBBj)A6pZWRVvPui^Mc)cU!u7soCO5>Rrb@Hx7 z`OrJ>yrKwtdY9=tu;XXc@b-yaaBU4HsSg^0otkihD_|~9LZ;rDI%Rmrbga!8?}#wJ zcp;m+D=_3|QZtge3nH41Cey1Bt7s?Y(M}BchE#Z>%Q!ZCK<(`9QDNO-2zcQH;q3x3 zY9fRv96}}$LUT4+;yrY`%b{72oWmcu(^%7edFdK5j=LX1pgWpiSaSa*uc)ffSO~~4q)N%zR9R0QtS#_AB z-v%9of9HW6TqD-C=nPH4=Ne+Qq?5%QUp8R16}(2LpM_W2ev>75b2b;%ZVYaNiY<<3FIO}hF=i?Y6) zuW0QE1~pn&J#ZP~q-M^6MrQ!k`?lu+^8PVl$#RYs%b)QSks46`E|h=DOdP=J`_s^T zST3(SoE029X2Jv1N4B6plDt-g$4AG!@%SDVfDMjI#N&u0Jmzm)g2#ARPWFh$c08(# zMLX}O!TJe`U8Ji zhIWc*8qkc+p4^PEMF=}Na}u@?OJSSHH#m<@NQLu31gdZ`5_w#&TMc(LwFR?ai`dV+ zHn?{Q;O?&s_bjQ4aOcV(I$mP)Jf93$!>M9wUm$$hy^wx4qB#j~T$$xq z^vE+*TOZYCZ3fw(fybF6v%GadXd5p~2wW&)@^*wG@4J|Stg7D>gy;kzWFkV;-KTTa zi77@2EDG6%q_RA5GeaXysA%JwGlQ$4xf;&AU8nFK*=L8&cFd_IJ5kCov<3Y_0dsFJ zayM?6!o^L&Quq}?y~eH#S*{oXCvA>4j&27hl)2AE5dxhfP=s9=JIqaAh`ZzAxr7tR z)nXTQMPbm={>y)ea?&^I?BUsI7%cvV=?Hm=+zv)zU=&6p&i(E=sIX-Z(nz?5j9wx1 z%>#sH(>1}zWHr}Z@`65Y7yQ;7hTl!=HpKWm1~FD6fA-DV?I@)@I)(L%If_#K0_)2b zU&q^O_R^xe$ni7MRIEk(ng)bq?Y@urdZ;+7xa3qgnemS0bfIlUy+XDP*gJvLC-F~LuL=?f?6H?>M*07FHYvRHA0?vyUe}%Idla%i5c^RROdtd_=h9Q*@ ztR!sLts;bL-hn%tF5X#n#c*F;iAQc(3(7Ljcr8j%Wd(&*))41MLaZGHF`ZgHdKzHs zXRYJfv1KY7Q0la?M`2Giei-fLG>{OK;C3^r%V?~D-rZZ-w6)LIp|;ouu@mj<5u}Z2 z&*~bkL-SD2t;Y_*vm1t&uJF7gj~(~H4o-Ov_k9G{)`X#etLor<4d;Uy>)_m#kD{=! zR;;M}B)VYzHLOUyVOMSTb3(~?s0|U>B6*xS*lh>JD8r%Sn@}q~y4@9{+H|(?oMzuV zh(%7kQjeg~#3yWvhEYTK561nHw0}P&vPjJok`o$^W?2TM zUhou(%Hy$Z3Zq?>Mn~o2mYHZ}v)XbfyN^OEvk89qW)ZL9xbhKB6+2NMUp!)73_`>k z{&d}qrLzc{(`+I ztEh4hb^cu&kGXEI5O-&vw@@(a3x%u!@I7yYA<4^_$sFbEWLjw-w9r2Ink4?(xINQkh4G86)W@W{o_6qqaHvpEZXe^mi?LWNFFnTwUEz7^6^S=R>1kG5-q|w)j+-9LSz{| z_%_^WmI&T}v2v`(@FXXaDW;Ydqm8`}qZkl`Iw zczYT-AK=6N3GQ(>r=P)6S$PNU6L2rg>^Y0C4oxub#n9L=zy3USdZHt#x1mrYUn9Z_q(7qJDptK144ddiEM2Kq=yb`~oTSGIFJV4>>+pcO?G6<5n)E7E zP2PG7Jo4<|@!`rT^q$OhB;|m-Z!eGSM#qjueI-Tjm3oEoaaB(JGERR>s6jlk*#Z&P zV;V^&^UT52D4#!xkg)922yvQ*5IRkV^|_A__x^C!A!Wz47U=2V(8|7q!x41C!|#;U5vDv3h0dd zs$PxZ4qL;kkJ5*~y#c(Gj35FBLkvVK(!DQp&;fxR?nv}> z`9f|k6W@r?`%Q3r4aewUGyB(~z~wN*tj8}{zE}&><;X7-QTq%*RAcOP=sAPcywJuq zBCYVZxQOjv3LY4I%a3)y7toB@3nO|^R@K9F51kF|#Wh&CnIbyEONKl|viZs{p-`{z z4mJN0od2tKJ~L|qigF3(T%Lp3p6eKEJr;N#a(By346PSdgKl9=X)b(>2qC#Px6xGD zvyhxpWGIq*c;q#Tef2C-_;C)g5_LGg+gdz3Y(=Dmk?H3AFo{ zLvh_fn|mB{_WGBlNO#r1oe!|j>yE@Oo>$oS@k3<%W1tYXCrlh&q`A)rBCm_SLlL{y z4{6H#!*^qNBs!`B6P#IrZGhyKqdXgJRh9!+8ydzo7Y#ErbvAB>M$ATx^+DJ(@J0qG zJm!=#%LAeH16nQ<5Qh_m@HHhiaz<=u*Aon;mPnLxi@oMk6l^c&8*<(wg$7L&zQF9$ zc}7i6O2#knKAVYjuv;bXkXC6Pj!w?`C{hwEC8^KY~|47sCbfm%o}WcXCj%CebKda|7S!G_uuGPGou5G}-5BOqSGi$~|dxj?zFQb7NK>$P-s>uup{X;|YwbB0La6B3Ub(1it8`7g@31$ZK zgPve|QXtl*L!5*~V5D{-qA_%<4nsHg7`$3(XAiyru?W+Pl@i73n@d!LYEYSv)r4$C z1SuW3)Ua!k%#Flha1C@uqzdfDncQHAWSZ4**cRsQZHRTxAI)M09ozxEC5RV(13?FF!mWC*5d(7uYC#VZ%$4sLg}1$yU^D#*{lMs_ zK_^6G1tx@>dlT}8g=1>RpvTYSq1h7;I5aU&eerdePeH0(SCZs1sS(sgbm(LiX`0L0u3yfW2k6p2lS(<8le=%dXKf zgug&njKauftIz$8Aln`UtwL9^o6ca}<^6g_^X-Mip+O^VvK_ zx}`Gqlx(S_(hz9BXBGl?Vr#>6q15CCO2R|Mn5W0#m3juknC4#h0OH9@h)1s=1g{|y zK0?@kL%K=KY;GEB%PB&N=M76CH;$m&a6>m}yz_>T^?|5Pg0894RT%cZ!|p>%@dN#q zdgUC>>Fi#}0*VkX1xz*h_t&8Mx8~by4g_%ss!sovh=Ed<74qqiC}tzasph3XP-EHsb3V2W}>@Hk^mKY*?u?? z4=6lSOz-AxCLx>_2=;e7nRpD>W8XCiSr zJWaDuvTb-!!$yX^`tU1mrFtJhx#uy&rY8`6-Xhv?sc#tNQN?hVlwhM0K1c53$~I<9 z0X($kLR7(5vW##`kzycHWatGleFq_Vqos^D08Bn@wW>uB{&f)9EfCALLU?pQY~Kf6 z9GsafTK@=Y@(!R1%&Sunai>v=UsV;joy=9tEk;!h%159)o0{cCbU~ke2P$q)1ax(N z3ip6#aGzJg0gySzhkK0xVho*yP1JO6d(;mb+ZRaU*kjavv7-hQi<}Tr;h=*yi^H^J z*ACFdsr@#Tb`FHn^V4r$18G;=RyYuh}t~q-?d?bwHl2|{;=X@+5ObqHQ*|>Gn_eVO^ zG$k28U)yXd605{+g7I~LaHm##<&kL^HnV?)dmD@%bNDMf`I(jQjGc~7x3)6<6otP* zvHqia^kSE>8j3N#8}*>`SHlJltg>Vc9RBujRN!GG_F|INKNb$oa1_~K1X6H-9v%9n zpb9Jncq2sz5qSb$L)&1aoUg?6Q2T^gN_QX(GYS(&7Y=H2`~fvNbPb;dsOjm3QrKuY zcS+AYfWumhq6|j}HWrdb--k~O9?6tdaDb7*p-p0v=+r%o81jzDLm7^RfZDrr@SNlX z&ogkmg`=M{(tLx5lP+jkbyu{P8XA!vf$6Orqw0=uA3O-L(_$uI0!l0)x|^}^K^rvt z!H33Azz&OP3r74#x?3AaH%0U+joCE`VlcK6%=#>d$~;8ajtH?c5Fxw*UQufxO!h$- z??>QNWZe!0yMwq8+M;L{_s9EMq6GaX#?V@hMV`GYXO6*^Z~*tB)C5Q@>Znc>HV%&Y zh~AFq%u_o3vM~K5&7q4|_{__*h%L+a=0>vq&Ge7C^FP`U|O z{R-%nu;l%4bRinC z4Q)%U)vzM8?Pu6Z=6pHCs09$ison0Org2X_#6w#D^R((-=8N&2iFzxz^{MTyZfu76 za9@S=u&A=LC#{9^Vh)0NQ{m?dznTl{FO( ziV#(e`zUr!p!8R5Ev5lsObbnHHHyOi2+%_;uRyr9=&;PD({ZLFI@l+A61u@`>ysyYV$etytU~JJ#VFFmLcVh`G~@$4L@fYO6Is2B}>PoWS<{tMtSC* zM2Ph%xO)z6!!~{JdfV@uR`$1=QHWdI2!G*0djC+AIu9QGGvTpT3V#*L_A|q~O{VW4 zcs%L%1u6OK!BP&@Q~iM}xo~r&x^51_zKzsJb>=)WLV>(T;RPw9Iu>DU07M%$e9Zm{ z5dEp`?-vCj9gQC3q6hl|Q(%TdtH#3qlG0&+p(WX`xg%5q8#2b1PW~u5`7c*M6jwq_uZ0** zt;q9shz&a+JYkk3+g#c{)T}EXMpF84(EV>FJiZ*kdaXk@S1#C6Nmi-J2*RdKQnBQd zsMriRsG-)qfPy{5LkPZRB-yjVSMsi0wil^X;BtV;fivdW(gtrl2Knwc4qlfh%##Y zmn=t&Lx|zmidtR6gWg6Id@WrL61PIUItLMd9^&97h`qfKx354b-GOjHyRs1;LEKj6 z0i9nJ{t(Zo5i1UXaGDA+3R9h3F(U+J(ZlGW(}XSFjCZ)K^4N4(GxP>~VQ>a&^J5p( zdv`-*?tvJ)7vk`7h;C{v*8EDVosx<5Ve!&z%Av5 z-KJAMhTfwICn8{G90KGQUgjF<{ZPT*k>d+bzX;*~1zTj+C|86^y^+;N*`6ocLAF1e z19{h^9+@S(LdC1C0XA$}<7s%`nt``9?0_|R!@z4w_Tg|6Sx!wTytEs`q5mr_yk#pQ zWo$}Ay_oHId@?&5^_pIaAVn<0tnbDg1f4@ivD{5F5Y(^@j~6x;AeSBU%aP2)MbN#z z1iqs;RzY_^)eW9s3*8%ZE+1S7_mLEOp%p}j*teJ!2-=>4$6{0ZwjjP`8_M+S6#N}F z`~J)gB$?ffc)MtCY(8@rwmWd0qLj9PRq!8xz`HbN_Y?Y#?#zQA#Mx_0mxglVWeEH< z2!Uz_*O0?NG&?iD1MXhakH~zm6&~$%z9VR-F)Vsme@2><%f#ry`<i)&Tk(?a(1(ZMDnqie z(_=Uzz_y6)bs0C>y=TKwp3F?7vf|_%+~+3JO6t&oJ_~yl`d8+XP4>+jFdNi3UAu)y z9`~T1GP{!N%c|zmReF-#7RVc4aizkuTinRS1xnFAv>$Iom8J}C*6G$ma&zrfgUm~i zjtBh`rz&P4qf9%zW(~xaqe#|gxE9Vb+ke61%^6mx*NOAAA%)uxLeBI&1Q|UKw?&dy zeLV3rMOjW2dv!8ehn=u18S|W@ zrwfSWnnB)8E|e(2K_l3JF@P=rDF90owVRUD2VJ#k#4dn?sAO6E9iMdJ)^L|AC0cM; z1G<2RfEs}Pq2yGC)mifOI2sX)y5yaK>lobW1i`Te(BfftF1fRyHxp<|+G3JrEz|?2 zuS7T&0GcX29nmAhYh-OY7RRR1aA2vIYzX4?^`95hL~lkSD5iVr+7Z~l30g+mv; zwqhcDaR(g(2aWy+e1#|RgEk)y13+7wwsaKO01kjlP*U{Anb94NQm_wPgr_FG>$ZZ! z7K{gVfF4I_hh6~PfVLVsmKhg_PECb_c3vT<2Q&-1?~w~aiz&h}0GI>XX(50%?JjT@ z(4ANRV6^*SI2=JRpv`(3U>GoiLXb)*9EpIAmFeIH0?a~DZ<;La0NPR~0PX+l;4KuS z>2)EJWqikD@|DB)85~O|V9?UOQGwYZYVaKiXx}UW?EuU1Kd&91Kv8)jiZ>RXIp7gI zCWd2IMH`Xs*nZHYBH$4Q$9gz;k;s6y2{i!gal^;FO2FJyNy6Ohf%i9fr=`F<5nDyc zAmHOQ8vSw|(}kTPGX&caCWLN8?w3J?!m=_s*f2;oyQ<5ex)iC;X->fn#56q~9&b-i+YUJK={eV2oh< zm>+Z_p|TeumTm!rJrL*V(zP8w>EO#;h{faQ68c(*hecI#U8+g%(XKp$;^G$w{gRN6 z^YGMJczidc6pA-35VU>tpSh@up{<0qok7D8(iMDGoV(AN-o?+`Cs_6@$iA9S9tpcE>{jJ^r~`M<$`q`(Oc^hk&wAMsFi zQlJ|b1wF|p{y=EsNKwbgzx^!;{<7-NKs^2V#|mo1=qJATi;$jy?}GS${%HtrHvSk- z%A7F1L?(@}RYbmx@GcOgS41G-sF-t9eK~4r9Q7(rzc!A>16V$0ju`4`yAS8SC) zu0~64p;)dqQm#H3*cr!+fR3O-fHYm7d>HdP(@NwfJwv{NE$_ zAM*L1n)qKj`QL8we-HsjTYx{o;YJB$=LqCh3KR|scozjquLS~SWfco$RX=65bY=Bw z<$ha~H7+U-e66gdtfFI~qUWb#kgj4>tzxo8#q6TW;MXeV%0f#Ep|zjTCS7P(EgZH* zIO3vk)N7$wS=G@()!9$gHC@%cTGeBVs^>*j@7Jon$|C&iSfHOMI9(K4EgHW?6n0S* z{#q2NtQKvd7VD=LpRSfztu|?kTFOPW)Yoch%6&2{`ega_$w}{%TiqvbOP?7R`^!miCNx-^E+{F1^@y`Rl$- z%KcVZ^lSF(*OK0^wYuN>E&VoL?6>)Kzc%Im?H2vF`StHe@4s_S|6MEl?>X3i-|hYf zI2s4FH4Zyy9F5XAK1bu^N{!P8HO}7FIL{eyQG388hXI$P2K3GuaJ72CwJig#UmS4r z^?+N-1MgT2yyrLYLHfW))dQbw8Tjntz!$FvzEakFW1%_wouB6Wbj^>|nxD65e!Zyq z{k0}h(UP*%V*IsaGPKwlE%|mWUYC~A8!dr~wu+^;s=u~chPHZ*cE5IQjV|qhZ?v^k zbaX6r^!#-UGIWe;bWGZH%(`?2ztJ&Q(Y3VHwf5Jw$)Eix~ik_pT zp0mH6YlfbCjh;ulo@bYy_ZvN56@7n8{Xl>H;0*oH8vXI@`e9xA;cxUKRScpn4PyNb z;xh~qYYZl}8>Dm@q`omoQ!&i2G|cih%*im!tuf4NH=NOBIO~mJzKT(SrBR{3QE`S* zX^l~NyV3kEqsli%RVv0cmd1Wy1U(UZRB`W```zj`*7$%P>1pV|J?D z>`a&0xi@APR0eff4(j$F)RQsjO3k2a?Srm&4Z8Ve&@Gk0cPt0r^B?>mWALMz!B5%; zKkFL&;?3Y!Dns5_4teK4^#~!x! zx@+wtJ=9NkXn@nupqQZ{bBB&wJ#@n1p%d>8jgYp9(zS_kvWbhaNtkPswAv>5u+8MV zHmPrIrbyeS>)K{I*=EPs=49IDF0{?tYCGeS?X0)9`9iw_E4#t~yW&i{(uH>ATkYmw zva5V+S0%Ksv9ezjU|$D6cIbb^VZc2HO~z4M&r#ReQ9stvu+Y)C z+0pcfRVPbpC+k2bn=B{0MNY%EIgRLc8uiXetm^D&?TmjevCeLV&SRRL z#~yL^y65b}xcKS01US0{#kzzPx{PafnQ+8q;ysrL#x+XMHOARBF4i@n&^4*qHTj6^ z{z#{g>KWD-KHOLn|aS|Hsd}=&wZ}5dr_==NuhgLv-`Xw?iKgk z7cgV0^~Nl89#b1TroM1YWAm6LN5(9>H)aLnu~N@twX?^XSdX=Z9_yMtHXQNTbkE}# zX6zQdv0I(TZjT+iqj2o5=CON@jNR8gcK^GvovNOPtUZqedLGO2Jh900)HcsE-Ja*( zd0tTU>azCg4)p5D^18Cf>)JN2>)l>A?|J?9&g+({_Z>a&JJ#NJoxSe`dOyhWezeH@ zakKZ+ZQjqiylS`mWP@rpepnGkg$M!(a%Yoj%2l|SF{D%ew1_cFY2Zh!Kjo%&=b~$L` z{h$bLaFl*6c<`j6k4_>bl%a>iu<7pxZ|qz$1QXjR~t93zGz(InsG~xj$3wr z+zRgaCeip+L&rA^w- <{hIL`j*j1SfBY}p30w3hY;~D1dwbl39Yqs%t(mar z=!AXuCmi609n=py>=Je~F6?+w*vU0vr;mo6y&ra-JMn^OV%N}#-9Zz3vL{}top^Q4 z#B19pUcWr?=I;}4iNf#bhu?JxzaJO=uqgcTn((Jb!=D`ue||swB{$-=e#G0M5$}Q` z-e*UAtd01*J>u)-i0{8g5Vc4tn@A=&k{cf>TO29Z5~*-3Qn4pe>3yU?ElR~EN;No2 zEhkF7E~;Ngltxd~!1qyFYSB72(R#tr20772bmw8As}|>P6Big97n~Cp zS{FCIBQC5bF8qC5q*{EmO?+%{e0)xPVqN^Cj`)HVIk52{}0lxpfJ7 z9SJjf5@x+m$X81&ut_WoPAtwzEUil{??{~AlUVsau}Uqe#wKY|a8g}PQbS$R;*O-H zJxR;oCpD=}T4gh-Ie1b_&ZO45N$Wc%ZS0w}`Te9ewd8i2ymeOB=7A> z-v2(iQ!V9?P0Eqrlw&z5C+bp8b)=l>NjdjE<$~JeE}O~S!IOJ(CSR$We63^h^`6N$ z-%q}!mU_n~^}#i@^5QlE6BKI=(+@gAGRDX$Htymg)Od;F9S#Zx}DO!;zb z%D0{=-``ImebP9FX;QXnOh}r{)HJp}O@2oj?|7Qx!!)IbX?)pqWy5r#Te>JAy-!Jc z-?i!ekEaiKn64?Cp>3F<>z1LPkYQMoVZ1iO^mxXghZ%!EWDJqbH1CsXVVG%Yn`z~i zIW!^Dwj|SjZRYUfnIj)&j{1-(?vv$co8{z|<&u!)R+2SlZPwW1SzZsbd}Ooz46_5= zvV#(`LrSv8t<9crJbU89>x16|yoP?5`q_sK8$8#n>%$XuPHQjJ(rrXr) zgsD?YrcPTsb^7tCGapW!Et@;XFn6w7Zc##RNl9+m+T3}^b1NR^E|8rzyV`KtLbqwP z3DfFJrZujew&eJa=Griq*`nHhi z9aE?8te?Jn$Mn6u)AxUv-q~lyA=?>8LS`JBI^#tBj8i*ioavo$?!$}=eP(vq&g>4E z*)w(KmHL_2cFerqJM-p;nYa4Px??-*UdXHmQ)fM@pY>$NtY^KmUVNDKs?Y2?rojd%*+>wvwj%EuTj0&CH3tbWm-AW6`v=)v%QRwxk(1$JZGb##jFA7R5 z3Mnlb*IG2;MA5`YMGbvdvPrOhZe`@7AKY#C$$zQpD0ebQk?n`r;L&`^^y#u zk_@|&O!tzk(30%XlAOensih^k4JCOyOQxSFnQ^6L=A)9?A4~GpOAG8u=em~`x|bFu zmX?&3mNk@??<}2nqO{^s=>oQ_+Nf-yds%H_SzT^feQ8-^YuS>WWlOJ=E&o{7q+Y(t zuDm(4yd}51wV{0d&hm{{$~S*3Z&RPwZZxmmZr--gc^$d)kl3zi#FI2RyFUl&jG;uF)M+qn}h` zSXN`auEz9a&7j9ML*y1(7%#LMvv6qALff*1_Ujf7Ke=$^)rF%zEfn`%RhdEBBt zuG1E|H!kwnwaD}8BJWR&eEZh=+t&t;sSQf14JoS~x2|@=$=ZpJYa`_9qKxZe#?-|n z)g?@;OKhy0w5u-VYF+B5y0pIa8TR#AW9qY$>T{;mPc5sTwyu8q$@-aB>t}tc&+pq% zVBb(UuAz8ZLuq3}`L2ffR~srnH7t;8tTv`UEP{UqRS%{9?QdY>rwIyPOR440kutc3 zU(~3Ir@vkG-yYk4`dbCJR&hj6R^hh_@S_m)hcMgx-u+^vKS+*fPQ@5&RGehS1FJRhJyb{Z&AA32pk4Sz)^4v90w=BNpK3B24}!oa1NXY7r;f(1ulVZa2fP~UT_6m1=j#GQh_9; zAZ`vJJQ0s5f*WQ$QXkAC4WJP$21~$Funa5*D?k%i308sCpc$+IEnqEZ1?#|iumNlY zo4{u93upsdKs(q9wt?-S1MC1h!7i{He2?Ri0S9T!9;K2V;N- z7z;ds7w`r?z!&%de-HoyK@bQAAs`ft1LMI25C$fKa1a3^K@>PJR*7_igWwQ2432=K z;21a#PJol(6gUmefV1EnI1es>i=Yc!0^Q&;=mEXp3b+ccfnULOa0A=~zkyreHh2`G zL>_}D;3;?po`VX|03Tw}Mk#2ExhO-4649$sA`6<7NS~GH%MOHr=3Pj)6X^rp zQ%WQU%sY$zJ%|1Ub3g&O*@dxj31b6Hy`e;M!8A|=iop`F6f6VF!3xj>R)SSvHE0HF zKnqw4TERN79&7*`!6xtv(0Zvvw1McY64?trgYtJuWFJ`gyAr8*uSBZANAL-J24BH< z;PC-v0@Wbun-aj)1e^9I$5jWGL_je&CrLpFCCI6KzF4UTyG6A|K%b zJ_!R8!8vdq6e;t`Z52L=6Y|MkpsCI$pZf91XYd7l1>b;Fe?DOc@QEBa29ATWfqYU9 z`fKuu2Iv93^jV8f27rM;6L@R$i4X7t{@@C@3a)`)L7EPqq=PQN)#VeBKA)(8RY$Bmol}#0O?zEbs(gU?2i>;PT?s5kHf z+k@c`)`juOdQcY6Crw}_SOr#tX3ztAL2Dv(!Fq5M90Tvc2k;RrPU4d#U^!?4t3fj` zO5+n_U;@@<@X6*JKKTW-fi0jN)Xd-3rNvA0VxLWb_qzq0ep?_jDSRdC@}ewfTV&cAmlOrz}Q5Y zsDr-1&`OyY0~0VHRGAC}n&5h-GPwb6g7tHh$p-KXI0McBg;HfQ3XBF~-~b$f6L1Eb zzypw1sZ72%E0Z60}|9lr1D9ON7LJzK{eo3dwjdZHbWFTq-07mI+BG2wN#6!K;KM zvt3AvKq)BMCM0^hg+w11f|&h6k_h+*g(UuvkR*X*&R45h39>*o7?q|< z{DEPKDlrEYwW_26G=icwRZ};-~s0RhH`--zYbg~}+S+QVYwJQ4tzEA7 z_a4-Hz4!L}{omjJ|NNil_uSt%@S4w=Gc#uB)A$ZsWI-lm!-z?8oWoI^!%=J# z`t^Ws;5IykXYf~GbaG-2-Y^En!X%gh!BgZU1j3*deuLkkda9iKkt8R7fxiMtmJ_|Qoz<~@q(o$kaz5wM8I}&@+ zj!YS;Aa`OFy9hP_G=2V19n0c?1J4;+^HbaVM<~Sw?dR8HCIVy!u|(J@>F0?e2whMVDJNf&_D=; zLKw6|H+(0wCpRF%%$`{L+mpg@d-B=W_T&<*uBT)be2#PrtVQ|@#A|RBzJnWZ6TXMr za0l+fJ-83TZ|sQ(Bwzz}U<*pH2M6d2ZqN^gz%bCj2Uy<-#9)You`nLiBYzrV7Gy&X z@9jxir#;Dl zOz^*LPlm!Ucn3tsDal7%0ln|olLVLm6JZj(3zNYG^CcqoMRWxn(o8U+A|EoRm`VHIu2ev`%K)T?nu>(moaUdlyL*hVEA?hT> zWm66$8gVw_9GDABVHxB@0W60VPza%B4x}4aT04*;D25WK0LIaQ40UoKKRP>*vu+L~ zOYK0i!DhGvk%K)rK(Db5#2mt69Pmrc4y4b=loTNr!xk8X@xpNqWbtHd0~|_389E1& zl;l8K;bF1^@yK-`?=Em4e(N1b$+_N{a)zcf*0) zhachGO$Txw8sP#+A3KmM&6LOx-M|&x!2H|+0nAP3;QWpC057oZ6~foAv=F2ah7 zlzfI*h$yXmxyZ-*TH3^+n^l`829EbNSDG_NPi6*k^TnpI&4C^48BErGh#VZz;_tG0b7yY23z2Jj8`JwMC^p^NdJI% z8*afJ*nzyei1*+=?8Nwwh(EzYcmP$%dxThx_!w~);%?|ddJp1W*a!RJXXG6~JP3#2 z3C2G}tbwOUe}veLco^{+;&V8H^e>1n;8*w!euqEcFYpp-G0$HS>kx@mZ$g0wd|*HT zMj!+ch`|_4z!Z*RISHZ}^a3fE!*S#vgL<&QxFuMD3~b;8^5uwja1v<+qCGgkDUA0< zJdMaAs^ARLj))&4Iw3Y7IwSUhvq-xj_61jPgMQE-+`$6|z(7!gCwRd-5C9qogm{<= zE8#rKya@5K-XsAg!UUKE@4{rb1-Ic2+=YAaBYcJBx)9&9=}i{F`>+_6fF71Y9xQ`= zD1ha#0t#UjtcEqP4n6<_6v2Af0L8Eo%D`EU{Qxe|7u?|;2!PAb2JH}L*PD!jaEJsg zyoA33Dp^QxU_qe7F$Z_>fB`V#GfK?uaq$mWfefrc4mMy5cAx+y*h6oSIUpaDfF8Pt z1N4ST;M==583cad4?|%X99QA-0Vm-UoQ5+{;)H3=*cNxJ(--RoTd)HKIKU0K3Ex8} z*bKrvUyN0%PQv zKseH-h!Kd9Fd8HnAA@LyI2JJqq9F#xK`-RTA!;EO#)A}j@rVgvj`Re?i7*M?1qU@BN)JP9!wronWW0W##LAflfXG81tY*dRR{aSqIdd5{L_kO>)(1=)}T z^C1@&z(RN*7Q+(I!&1nDWsnaAupCyvDp(C`zyRx^2sS_&Y=&~EfGw~UPQht71LvU` zMoq-?cru>D;0fN~1HRw~{xA#zU^oOrDBPNY*SqB2q#iE7XP}slGQc7Q`w^@_1~woE zTkwTJP??F>b=VF&p$hiEURW_7_Y3lK@!Gfm#~jR{7g&HLoCDE9Y{Pqajabr~e7gm& z$5q%b?ReeagR)>Rtj73$!~<{;TEOf8$^t8xUDKOHVif}i0DJcVv}2G8LacmaQa-#6H25D2pCczl2jJiSKA9>l${4?c$$_yStt5q7QgmGPXy?Tb@3+tf>Vkt|s5C_wtM8uN*N|t;NM|!iQ7FM~jWF3^lC(sIC z!|&cK`2*^GSaJde4`tCQ9ZTlGT*v}xAWJOaKqN~JLLJmY2b_$?<>ZhX&yrs!vBW%$ zCHvD^V!wzb4&VeXaNq|@Z1Y&syMQGusK96yw&M=YKWbRg29J-jWWxoP6vH;Cy3Uen zNcjQV1G^ruWH%g!r;l0kpbHHIf5Lkc^o1o)S)vD5q}?D7>3)d)VHwi-;Er?w;&Sjn zdLRq{HLO5hA*=*XjITmm4PHpEL0k*#-~%wgdMJVoPz)td3LBveHo;~nhYHvNTVWei z!glaR**g$-f)CPFh`V4n?18r;;17N<6o$Yscn1PtI0S+Qf?xy$!$=5$ zP-sVaVTe}{ufjFxfUjT_^26ae(h-P}FdD|dcgP!y7zNP~12>R&6TXK|_yKOgZMXw> z;U3(FAK@o>01x32Jccg#83uQ=#1DqSJ1`sqAqYl56pVwJFbn3uIw<1O*p^F(qn@!O z93nyW9FIrn2WmJBwQw9xz)3g*A43D2gA}47sW21fz+A|HOvr+4$b|*45Z;4DunJbg zTKE7AumOr;AMA$%a1ahbEr_U!O5+DcWLoO_Y1@Im$f;?CQ7A7k4 z%v42wffq1Osv>Fd9Xy1e!O~nstYD$Fio9>9B1<6;mVvcOMQp$o?)6cT`|u_`?9@Ax?x1#q=?l4&O|5BE9E25zRs;qFC%i{LFbItiXwk zf^hK1_-Mp2FczX98r}s16vK229^sWbkwkcm^qOiX5_;5$cr-YXPd{@a%f56XcfNBX z_h8^%C$h(yNA|&9*bfIljr@a%hu}k~fseow`9}~BLoIk=ybkdw9D{oBM&3!p6L1_( z!zuU}&Oig4gR{^G=fMZdTtI9BU!(`YCrCGgAJT*2BGR9NKhmEeUVjrtQHyv7F%EG&#KT>T zPe4q7i7*N7A@4rCi}YmBK_dKwydPl-(o^99(n*NPFb$@|3`l`gc!=pU5of_8q#wg< zr02pM_!;9}Fc0ach)*C5>2%0|Zj5IlWheRRwAx~)j%=625~K{gAc#} z>w$-9MTi@q7)qcNHbNQjF?}=QCSZ`RKr9CV(pwR?K_zU59bkm~orqOX4SQfW>;fUC z?L`zJ?n4wK?ngWT2jLKW2qwrkh8mxsUw@64?F_#|3sm6;#76*bRGNFYJTokMS6T&MvGQZozH11LS8shTwCMKf&|i z1s>l}1=X+zj>9>S89S3FX3pd(bVH^&`oDwk?48LC7^!k5ArJ~-5C;PsoykD>i>EVr z3Dv&NWEa#z9UO&YP!GqU8Lq-L_ztEGb0(=U6V8P=lk?CB7oZ9Lq&XvcZDP1d_?h%~ z33vOyr^%vPTpT4XO%s#dUSx+FKHW}RxYTTy0r(#-A|%zmsfd(dk3sLQNN+UtqBS9ese z=V`rO7<&Cy)9a7cUN5_P5pyZe6aB?Vjnbu}^-|-H{-?d7qJF*N#7Bx#Un_yk(T23ltY%8LW-Y6F$+ zfezh)tW}VsPmptLkV{sOYjIG&+93D#paI=M{}1gFV}nDog2Re~!)t>h+k?k+2S-_r zjPV&68#^*CYh--!$O*M0C$*29+&xlf6*AQ)Bsn%@dR9nEamdWtklF1abGt**tU@z< zLbGB+bFxBni$fRIhAwIkUECe2w+hSi3CoWSTb>nGSRA&hHf&9M*t+g8gVm@apHaoJ zqe`lbTB=U96jQ>B^**XOsS^jp08uF!Y^#OFQ{a_T(k^N+$nUH~GPp$&a2* z?vf=w`ET168s+Fjr8?uII@A9z4GZIv2IVCAl_m{2nl$WcQo!>hjdgO+pyc4Vz0$Xu6k5-FGH70R+`Q*G^InwB`|aqwKi;$_{O87m z*V4>>Nt4>7TMSOO8lP@GKV7~t-R@Yr@>;sXFX^mJhU4H2=kXaX^D|sGX7oFj;eL&4 zMu;|qXh7&QK6B9gOuvnpLylz*yY~0igTdpoLgr_MZOjTkmKAv|Ys@cMQ8w8zgR^7D zXUEOYj^CI);aK*hYuS^3$=2EAOdXt)JU(ao{G61HIWv#t%)XX0_m`YBoB0`o=Vy(d zpEG}c?#B5GkIi3nZT{k4=Id>8^9JYUkI!8`KeupW?y6(CYp&(4`z6<4v!H13g5vQD zO6M;q+qhu!u>}>^7Hs`xL8Z;Y9fKEE{r}c-aLK>hbnw`cKdvo#`O6X_*Yo`J=ugfl zS1&5l8`tYiJM?BR^wNKAH`uY%;l)x`p6BS7=NzBslAGsRme;R7&%GmWz>7S!e3_Tu zGN1TmgL0Sol`R`mzie2?vVa%MH1hl)zx?3%{E*!Iu(JH{`uxa_{4p=`qvQoKeg(1d z1#!6r@nr=Q>I)`y6ij|mpp!43>i4F_VA=AS^>3OBqOG7G*Ge!a_jUU~G!9(+re$D$ zeBtuk!or?5fonPn*S#n-$XBB8vf_BIL10&zje3%*6^?XO{+h7Pm{m+ z51!|K@S^O4-+J2ny?ntn_T%-m^n1IRpV?b&{LsKpzG=PP@%75D);s*Vp0zD<^e=Kw zC~{d)$q zDw%M+WYSk9lYcGI*_KZAFHKG;o&Kk7`VzPXeYu-9E%C1|e>}eR!B<-!{kpZwcH0yGZQTjm zo-f$;V$-(Yj&J+pt8FiT-A3#xc|$6h36(|*D@B_tjZajXeqCwyTcy-)yTy?0Rui^c zFWfHQyxs1^cIDUG9e&%++U;;0vcq}84wr>HTsQCNcVb7+Cu`#J{8Q%deSaHePD`Jg zJx9i~Mr+02Zc^)Y-&214cCp09f7Xf$>>pm@rh9*17jk&HAcpc8;N>_ z6SlIg*vhxN%;rapiR34;4uTP!V zn^}ZaEn~tvE1k_SaEh_GSD0@$eTU7bc(5 zvmM+@H2LPsMNVrbRMk`)n`8Gi2HkgcGK|`<)QVL3BF93Ow{irLJIi7kHJ!yV2EQsc zJuI-(5M*pIV{bKAKu__@8ddpH(xAbeGx*&T;m+krHCKZ=*LXF$b&9gM`KDS~eB%0g zW^N}m24dNDF=5JvhA^TR81vWA$_AY?rR~w93YSF&h04-Ry+%ofR~z`YY2n8$f>Qdp59XPunCkHKep`sNzu5uL zaJLHwW0druq0?^J>@}WOheZ0sDbxGOn_}I}`fRV@Us~&FWKwB3XO^^vHp!DN$(!OQ z&Xo-AP&XX+l~1hnYq}nvbX;JR^r>gl)v1&;#ZNs@7uun7*EgVN`eNS@QfcMYK|Z1} zS9MDTH?7mxk@4ZGgMKl_#KLSzgXHe=5p&jjiGnLKms^e~i*1%CnR>R!mlZ}E^)Hab z#k(JKgt;EauBJ^pSa&siRzaPW>&4?iMnkJf zbyU_E{~#O$)9oU!QA+J~-Ly4&pg1OcNL$6oz_6oY)=Iz2k;{@jRE$?YH?7G0ZedXP5L4UY zHGZLPAn^WBgWB*y6K^MKInHpX2OETwK^Q)v+N!A*PmoaIOT?ZjEQ0Jju$m ze}QLH2S%DUD-K#+E}F6mqf-v3r?x~8=8oQJT8(+0YXgprYJbWK_leO;b}^3yE%w#GBWcDZSgD7Df5of^6u8fO_=4eDKmCfvfs_C{eJKp9OBl;Hr6}ZtGChyrS(gN%bw2}mvx_^69_y3|NcJor|+YFvSE=jrsrK$Ih%Wf zOG?j7OiP_TdrC&nSk{{{yv^0{W%c}je48^)`r8b0e0?@E!s{|DrPs>CHp>mI&!0EB57^OCcZULS#5jCB{|^86m-vh`li5%{lm(Vrd; zXPN7p!(kE3J)qOG=VeXf5?L8jlBfOoK@jvzlM8z0{ZHTZ`A^^4^`>*CWoAy9F)cA` zLE5xe&wx{mbu;6lf6Ta?depQ3|8ej9>0YlimaP&-%Bs-v&4LgEb~mN2H?^8cADnyU zYO2-HUTV^+)jW4a##*h$n^OG|t)}3o=z51MT8;6wooVk})M{et9SVN>Qmc8->QLzS z=Oj|sPqiBL$-cczFb}m!URbNuRHeORd<~0+X8hXgKH~QaG4EN4+VreeGo9kSQwwQI#GuE^ZH3Ae8>-zUzKZfG?R zuuTQuNc@!7wVE9xk3Rj2xf;KDs_Ah{Qik6x?qb`tn)A z_8S?RaZsySZ0|QCxK1KBJ)_n1R=JTglAGisNvpI*t2q=(f0N)6HHl7i7^{5Dq#g7K z!*9!`Kqaa0FrYuz4s^VtMsK%K=nzYX@jd7<>IHtnHVVDBq>vqwc>FA`A6e|t%9dy~ zrvw&O!xv(^wxXO(SZ=e22EQU3Mv6Re<(;Gp`8UZ$hqKC=b*ju z#x|D(nP;tx%ObW%HCP!B5cqEl`aZ^0r?&lkc^jHK5Tg0vSN$+mBwcn^7wlxQGPvLR zFy}ULNzB1`kJ6uR&pfMB$D3ST_38B5%@0G)e>)o?=g7CS2TvTcyX|4l$#3T#`{CK5 z)7yG=m>91(_x;1?mS*#V?bG9YT4o1*KR5T}{ppS6+-!MgZz9;wJNu1CuBpSkV~Qs^ zCk6A)R>dqh=~i;==94B?xM^YC-pi%^cgAH;H-jI|`S%|# zfBLG@!N@w9NLlHtSCs?2?*5c}L$m@VaJ!NFcskA%O*9c5K!%qdcT8ZB?gRKpi0l3c z34j5>eW;Z}H|B{#3`c?7+jI@aH(*&Y(k3tvromDy(2OX;bT33(7z9Hg2YFb83#Z?>666fs6f#0`k7@4KfvMJNz@p4i0 zy)JUS-sJqukwY94_iI;m4>>h?xv{=6EXQYe@sr|xu|}GhMl~KaE)RT-pl+%71>0{OMCuBr}ba@p|6v8SXL6J=p%`>=q9;RMOza zoyxW%{-?bY(61TdQdNe1`{uJmuUGpvY2MuA)U4Ebb6=fOtz)b=GcJdV1ump?nqF+rIa6J%hDc~UcYYKtdi@@a*mLTF-{NOebmLx~MH#<7 zdqV6vkukwZ#y~Z93f6PZgZK`0GnL?ENq~#P#Tuq77Trm>Xf=XR_r5qC;LfKI>uE0f zO76k9X~k*iyYrR)k~BscEAAQS(UVh;*&f`94Hg}M^BwM_4v{rDo3W!r z$SY$qxI~!50ELV&8Y43$_lT8e6p3{+5I)Wp85GOo-Y9TZN3q09Zi$y3FUT^FG!lVr z969X)5sU(=Q;Qxg*mS1 zKf!=5It!V5v8gx7{W)di4$ejIN@~z6LM37cdUG(4+a9@O=jc6THWoO7eK09nGfj@p z5DLjsj}-ErM)<9mLaW}HVo&R)@hmPaf4Hk6Vhqw8@78&9TqsF~BTqtI{z zcgWyTMi!0^W8!1Q2Ard6a2~9|$#NP~qjigCrfA)|(5Gh&vtOIqN{-^#ZN+IU*G)yT z#~xBQdJl6Dn53Qub z&`Mg4j!f#glXCLqXs%b*O7y>>CgbO*nRxA9f?h)8=+bo$XvlUPnIcIu7HB|+N*eU$ zAU8LVQV&=3TUJAoJoLDeh1jk%oVM4HMVMQUqL*Ud9zv0aJn(}R*o0*m&q0YRJXnmU zpjXx?^cd2G?vWZu9(HK~9w-~}_*{?OoQSp0NB%@?{(R)8pxj9Z_n=eA7wC$jl1#%6 zPQmoK*qLK7Z#?c+0@5=iWyH}Vig{oAX*~K}(V+8|4s?b1f@ER^t0ZBrsjoj7>3UA@ zxAv9o>H80-_tUxCRqphjd)3BGZ~ZmnD*D%KO!~sIyG7jdqW+fwl>c%vOn{dH3?49) z|Hn;A`47~Te@06A-?>s@N5K*c%FnREbSo-+`NhXxl>eI#IvT3u+Q@=H2~&QBuJlOH#1RBx?Eci?(N{Pk(#ufQgPYr z`oFN0|2%}c5aIhlFNR>!Q6ARv^y4v9^619PK&i@#yt8T4(k&A94)zSg4Xx(Gu>{E8$Vf+4EB;->e`nVGti4lb&u~=+u zOsTPniK)Z{H#4(dy`;UQ=2A+{EzBvku(YtWvZT~XW<{xtQfrBojog}AQ<=3b#%yiv zDCPbY7$}q+drA@Q9jKi9liT#}&EnQu<>||)*Z%Hq?kwwpw8wz{EIVKz#s;cA)GX^cz>_P?YoM1m#=LzzeOT7l%Xbjc zg9fv#&tN}{1BR@el@IbCB?8|y3a_xF|HueU-mXz-xHg9rQh`3)H|bO<|i*wA6` z40|Wwod9{G_j|sxI1Y`PGI+h(B%wu@SjEahk9^0o+RAeNLrjgN+(J?VGk)e@& z6^MNyW=TvahT~#mBO^n#+Q>+wIEV4$$46@^9T=YwpAel89izoBXvb<}kgY{>Li_}7 z2JUYtZqDP!4;_j*kvDYWq=^XcPJS1|fkR^l#tzjcCXP?kMUS7No0>F!eA4)#m}x4e zPEMNqYMAJjG&E^wGB=ur53i8w`vu4jyze=f7=FFW#>7YT#N9u3Gd2{Bad8K>J$jHb*9>y@t z%*?_tJ3D9okc_^RBAqcaBX_~V@#AyedkS zd<+Yg7tra5D`HoyC|pTb%v`l{)vA>%SFc>TX58A9<5x}}KYi`^btqx=>eY1hy45rB zM^~=;VD$$EgJJzTT4cyrZ^+14pJB+@u+s3s%9Se(#VbonN{S5`B_#$*GcpV%#WaH& z3>gMPDJB>UB^f1a))?rzb*oorl+cYE%SuYtm2Aw|w5fFS<`To^lJXLYB}*#Sm2BCB z1y*B4TeogSDx*AO>(=stTT3urUScR;osm(#x}s#`rlO*4m4?caZ6$`HqRPrmMXOhD zTy3b_v=PfxZYrtVws5;)+rn+zlCka`J9f|=s}1Xt4coRY+)`9(FcfXuwsZCBbw%3^ zJ2A1uu(7hd(!j0Mu#p>Jt1*)ynR44+Qo0e#bA@cI+_A$@vYBow*+Hu|8w_i4$Ev8I zstWT~WA@~-DvbQ?Mv;Y0_+a$##t)z)3e!Ro4Z)f5$#(Oq>FM^R?k?%H)#yDO@W_1tzHt=hc{ zccZ+%riLD^p*2To4Ywh5_wnP$j@R!#R$pFUk1al4TU&GNM0t5_?XlhEwYzI^+qJ7= zckPK20Tq~`=dXTuKtMow^~qDGPaZq-alo>{k#VlHv|>ehc|$cfvZDGtJ%6&1o@*?>(3p6xu`!^rv9Y@G-08-~lNF8C z$Y?wl&=e5x$$8|SZ)$E%Y;0;wY-;@UVq@tiO&4)PXlkNmrKQ!MeRiq(v(mE9TT0h_ z@p((*xeP9}e%bsbrOg?ot$0qBCT5fdWR!lIn0PwivrDHlPL`H7UP^4da4GS^<>r9F zr6(_+EWLCo;N;26mjW&~HwRRlYc9QXGLZ%(VoE?jV#TF^faa6UmlNCCF123?IN4s< zUUBv0wQC)poNWK9v^n7F)qv8{6Q2Z>oos8n+J3UVz2fWkug_uYKlvsg;QDpi9B@6L z`IGjF>)fs5+wTH8Xh%A3-`u#CSa^*V-oWi9H~603Nax~>jp;)$7}EK}t#*1VJ^j{= z^z@4K+nt@G)9>|+-$_runts1iTh{sGPwC}9mEXU5C;is#t3RQr`?u0>-_ibb z|JLpDtL3-3xo_Xb${uuL)?1yo%R9?&?=COz?CiXM@6J!1_d3fv?|0t*>E2I2-Np@b zRaaA5U0q!c_p5Iswp2g-4B4I4cQIQ1@KN=n+jpz0?>@X+{rJ)2yLa!l(7P>NKeZtB z^Fs_CK5V)B$WG4@YnK$oBHGql20DBJ!yN| z_N3zHwmY}mZr}OoUQ1ic(+kZPaO-StNpDWS-PV?#-gY-V{ZZ$SEkCwk;fGJ|cH(X| zH>bC>G(UQj-m|>5AWW?%6{tZzT5q*rMo-*;lu9q^z!=~*LHW`>+IZqrz|}k z8R^)Gch+Jr-tR7NE^lt`?!HrAjydk#+rUjEE~nbws^1e&O-+8gc5g?hwSI_8(SCu; zsZ9dZ?)~d&;eU0u_@|$A4OC|M+XX0LsE-oUuHihKS(hievPZ+LnTH}y_yt} zJ2@4DDd5|Np87OnL=!+Y#BVmTLpE=n)(*lcEW^zw=PKs7+h0t&&++~fPhABPl8;M0 zFzC5)JF}<8^Hu#NA=X5u!qq!mHU{~xQap6=huz9DxTuC@-b`Ti+>+SU(sex4+i;7x zlEfrjI3n*E;p-`vM@0Dj0|Y&>Fm@#j8%gvcOEeXgqm5{uT&f{PA`9Y&mFoR){Pb$H zOgEOIl5`D9o}zI>y(*7ruql)!J>^Z&U=*eBii26*d4O>RBG;X+RL_`@2?y3cB zE6q38VB0l>a+i~kDY%k>(0ptYcU6vv>eWFawg`DFio#BjYMfPiX;7gvmKCe?YE93z z4~0^Ogc0s42oa+^dp0&?%qn;J)(5 z1ifpJC<-%MIWtm~2;07gsHDi#uOfk>^+YpLM59o6Y`zo|7)_s^#fT+CgnFI@_BpGU z@2mXIi!%=Cwk6 zrjW(YxGue*mkLPDRSVpos}>RZI)>~Ztg%#&@6K^&ScXLEQHUvy0)88m^c-uwCeR9{ zXx!K1h`M8)Ix*$e8^zU_+qzMr(Ku2}qXcCT7CW4hz(m5(6_uDR3}ZyoAU6;97WNsD zKa`Z^pW`>6ydWtNqP^#LxFiZo3{o?wzh8p&qGX~EtPupeQydf#VQ?oYK#R1JiRN?=dZ$2svz^gLN*wChvbPgjJbtol*pNTe5-=-STkUSXkZ>j zJn|VF6-)@!lVBnW+;3$jdlrY@L|qN!?i7OA*oKOO3E6!+2CBC4vdcApCd+cMvIlX_QFW{5tM# zu(1<}8@k^M6SIscuu7oenN$fb^LS#S51ZSczCOOu$1ih!_~v zp)O4fqtQF(V{aIW!bagM5a}qUvkO3z63s+W`+SgZk06KpfSkHIy0`NW#nmo?^|+c@+Z&y`k7r9rAtt4eG^ zSP=Cll~jP8iERv0eTvunU_wSl3#h8iG@TSuDiPfbX8SC5XL?@!Q8sQa^^e}FrOP`ui0e|iuM=CXR@NLkUV?;4s#Y|q64#&PKW zl@e!t!mEnr8(OhkgX;}?D#%~Ny{gClMWPnrq65Kj=eu#Qs=wVQh*{ISJy!~ZU5(Sc z`FgVFY5^=wg{EoWBG@oi-$`_?U9T(JJGd#%t_R{?Rc=2*RptW;%f5JBTg`Qt+lH!K zQJlEO^usu=svG0+B2AozhKg`~gV-Q?j?IvoKI7`Xzhu6|g$K-cTJauqVVs8R3h>J+ z5*;ItLrtGnd{2by4*K}vl7ceko>u&VT=S4>uD!Y5;PoXGwkxRTAeWCq@ztqaH6VV}E?}{Q@?y$)kpGh;wUT>@h&yaVzs!>z0eIc=wPW zu`odeSh;%D-a514$*Z_2@!=ndsd(ji8?W z_4St(zoW|e7Z1Yx%+>Y(hLI1YwDLq+xHSjQ)$b4eriLI(=DfO#%Rx|mB`WpRzf75zmwV*#$} zR^3zcT#b79&qg*n+d-}=_IlXo2DHrv8>*7A)pKDbAH5)G$?UvnwS6{{chrMdNu2Cg;V)WqkoR#8@ z3Ga0H=OPa&uF93qCk)=}1o(PP>|)AI5eUrDsWGJ?VT?dvB&0$r5DHQMD->Wz`KCev zrD6+eBj7Il=0zEaP;)4zVq>9DfSN?ih3Z7H05!J4kX|TMfZtpYhy*HIfxt?HNfIZ4 zt=LL7PHZf2%0l(7kg}wYa>UnwZTBhFm`fkW+k)~VnpnoW+|1~VUw(6 zGHWYq8CHZb1ZyjyTxd6A$*l!C>J@9tFnqq4u}8s- zLe58!NiFTE9cyK6ZI2@D9mKOGLa}66Z(A|LsuT)^Om3x;F)G0z>MOV+cE-JuDi~@d z5U|$Pj<#ZJYbQr)aZiTs?M2QaYTL)9Z(loG7duxsrXOF~r&k|kpZNn?UtT>1!w3ZWgvR1bHL zJ}&MOYucL;_NAi)0*Q~n#@wHZ1+%&)E5r(qz6znjU91ob6$-ln9ztPXMW2CoYAVMA zL}ZI0b@%kLMXO#1523djUq1+io(>M~?$|mXtixU5>+LE&;aIAo{?jpkj#9L8{a!G)kpXA$14}QU;Dt z3=%1Vl_RNAiCm468L7}P8l?h%JvVnH%0#wOimX5^Aq|lA33)Y55fa1%DWu%YBYT!m zgmTOEEU8e2^~{Bb$jBh2LSV&W8O)^!Pzv59A=IZCTd7bCWst21QH;b)BZIIJAt9(} zM-d~bVk8rUX^IdfRboYopr8;Mst-|`Dgs808imL1D49|g5X6ic6(E&iBNbGk0kNsG zB7_MK2~jZN0jlt>$>9tWh`kujD3D^92-64}60V@80g8}7ML@X5M2Ulj z@VrM+xP^+G6@8G5R44)!5rLy|3q|$a7#a{THXvXWSCxm`aDfpM;7o=3z^G_>KtOZ^ zrpCmKkq3$}UtmB)#Ax}raj^lh+6XGL8mo=LB9v*42vCoqapMCdaa1y#4v(h^@$o+K zaS3DNv~lW$1dK@}~B*G5Rj$|aK`COMCmh(?Pr9Q|%!AeNU%Cd*`rT%07K zy}C4$CP^gr_Ap5jCy5v~NtPI=lSEUUWWc);G^>r%>2!%Y9W!joFiB$c0NqrPXCSh4 zQ;esm8QoB3iVm+FI*|yo2#lG;0Rkajza~!6O&P#2I@y!~1DF9iT?A!B!y=}L`wOVg zfTW0|q$HGpVbU;aqQi;YG+okkoo+fZW(=44wE@SBmEnyO0`CFy4RB}L4d6)2i5N*p#!mzp{(Ri{gxGcA>wV?0ak#`clnKNT1H#R4A&J6sgq(medV_-aNW=c|OYT}F;DKioS@r9J}oRm3p zX3d#3*IYPPI1jU@nwzK2nqwZ3mY$K88i9>Sox%OZy$Y{~nOq*-L3_Dlj)=_6)Cgn0 z8S@0@jBo~3uvx;PLhICV=BX0er7;%@vr}`-X-Qd4tgEY3<@n!VI6)lO??mo;PY47zx6P7Yn1IzzuKC&x~x)!HqxDJaM;C`e7s zPAw>yq1D=Gv8S^AGBX4H`egeJjflvMz-{^RMT=HMM5Je@X6vcmCTCIV;%vQ5Vd3)S zvld0*O`W*hsWSTeMT6ognX}%mRJ`tQy=tJ#QN5AN^=DkA;=i65^;9bUYxbXCK9eS! z8;RE(u3pfO;i?o*aVHjB?IAu_;)CX^+Qm0Di#Q?ZscAGp6a!b&h);G^RlXx24bQh^m5d zy$koZoVYG$U0huBMtMeTN}fwfT3-y8a|s&@ue|BV=LsK%cm)MxJDhkvcogs`hH8O#6wyXS33x;h zg{3?R+NPjPu@#SJ;i!`05n}Gn6=p%kxf7`)8-yK=&2nTNd3*tn<&!8@g?k>Q_TojU zakp_Jlx0;G+y@21?HPh42r3&0whDU>v+!9bEa!*^seo1Sv3eAMC%GB{_mJg9d0`za z(rPTg?OLwtj9b<*if|L~@61vxfr7c2kZ|-Kts!JIHd7{Hxb2HUCf~{FGCxMmSCc;8 z7;xK-0rnr3K%qp%Elzk|-?FVJoyYAxEaWKRJ0jPShxKuruEy^N_S_NdNi4vtJI!Nb z1h_{8YwM|Fmy<+p0+! z4cu=s>pW|h#@$9PCw3XVs)2j!HT6#`fBvTi?q9A>`gdyJn$O`VytyJ(!X1S_t$`~( zW8`x6^s7Vh-_*b-L+ijv<1+B5T6u zhY%BUUVR~TM7kBPYc!Y$#8dE+Nb!0`C1@ZX$kU;X-C(>ca_#g4mZnjJuX4ahF=jtI zl?dGgX1w{5)<{fWBrTpj=dr`n8?J}KOc3xzBJbN zD+?;@SEs-?INS63Xs}QU=EXV$lpREbNo)fy?bF9&6t7sqRCm1{>1AH%A~Gu6!EMWI zu!9WXwns;V9#R8d(D-=2=tVWNX>QGmJZ^Ll#P?+5jtkdeJE-0o15p4sa6pYsz)b7~ zjc7W~=%z&WLjP-giWyWQu07#Y;C374D=@OdUX!riSVK?7D4K;t5Q>p-iii(PbtzcreVuJ}{cPn;_jde-c-L2T&-S>CS^}p`V^WuJ9?65mK zJ6p3ev++9|%$E0Lw&GbwZHkXSY7zMVe}VjeE`a3OCZ{cs(ajs=)! zLQ)%&)pHtlqjmbg1xJ|M2JLJs!`P1ZV4%|}6Y<#Ed_ANrWMIU+s%jcPE(wrP?wa&@T zK^4kNiP6OXDei!N*J#i<4qfL%Q0TUuvnnXu-jrEjYN?#dLUF(kiU;JbtwNJ02QD*t z>cB-S;k+%2))_Kr%-zg`CI{_T`YP@8WmI@^tMz7=J0@p+G0%OF1B<}vYf-=#sk&(i zJ8d2i?U2Jw^X^%z!oOjyD->oVQh`q80rY&yK;59Tj;x~oOAfa;GIUC$bco8}fk_fv9ZwWsk$eYK` z-%x`@J;D~k4v^KI>^ylTtvRRHoY`yc3bf%b8CCPz`52-|#1obf7-hpXGCcB;5lJGA zgu62`GCw~p{6VTh0ooHy)=0u35Ltz}iQOry!HLeo^Hh-{ z(Z4(Z=RE|13;s(g1a`y$CsSM~=6)?3Lz^(~k&j@%A?tap=KzZ~a?cNixB?jOe zOU|Zo=a`wjB2T-VA0U^(v&Kd z0NVoqF9TsV#P*>nRv-TM4%ry@9fZT(56DKnmvE8U8}pX2y-%e5H|iP>g<%uYYCGez z$zNqHp}_8#uSnPGW8aa;gX;|TaiRKp4_nwp3?R2qeLHfeUC0aK%|#IFbFll-x{?<% z+r{h+x`h+yQQ3W|)*(01XXctv#0LgLQPdCtY!q?;n4|Y~s51xC^n0VAo1AL6oYdep~Y`X5iK(CJd@a)x(g42?Of(q>9K$^;{cZ@ z1HMfGq-+E@(_%d)>Vhd;&mWNm<2OIw!?>X=*%TA=WzM<8#?h^%)(vxlf7g;S2ciH& zw?Pp-#}Qq=uz!g3%eyHA`D=jQ+%tj^kDzzDQiZmRc+$=oCZAEFq8})wd^%BFrMJCPf8*UI@gKW4CWJeOogqI)+tY@&A6Y<|b7f@KDDY=i;s?tVs{x@PO=O$Dq zCXX*+G~csp!*?hSZid-;5)TPqLBwvM$N>@$NEF^mkzf)%31bLnK(5(F5g7yqK-nfk zQGi4MiEi6TNT-gIsJVm0Eh0yEGT<)|Syfk-Ojris14}_QJ;l&ZgTm`H?UsRQLYPVr zg$@r(apPG#Kh@wIq*0JgI?vL{v>rc!4^0chBM#Y8-TeX0M9FwJ0+b2|T!;XS5joF= zRH}j4e0Ae3;M*raFosG^`vD01i$p%%G{F?BJ|>f{xf4)9bnweeXgU&V-BGt7xy&q1bR0$%Dp%LNPNsvi;L&i`~jL|D1*Ge*!S6R@8=cGD#LH~?M zXlWuM^CNx9cdAHRQmX)bs{>}%gdBNQduox{REFw6UBJz@fIvJCl{N`*X(iwvyavN# zT3tJ6kB5)Wzi%I{_NEm`tZsBG4RwnbXLGObV1x59HMsDjk)eesw$5_cNk*}+Fs_Y_YA=2 znSjxhRq?D~z4nYY`6wGCK_BR{h(Wy;e~B1sW-8YA0Z}>4Sm3)z?$QG{#Jq zzjp`n;`&x05kcm#Ii~N;RfgOGJAP?g()2#0*TNP%E53wYoc~MtuNt2LgNt0ZI-B zT*rJXk5SOd{Cm%Wc9~4z#xH=mBHZP^(P%rHmZlXdF*|7!UZC2Q4-j1x;3pRPbQwSs zu_)JM@e~xXSi7Ba{d>1IwbXO+wx@ynWWfD02hw|^!ypjWg+Sa?8?ghe-n->>^HbNQ zt`>%(kT^t2r=G|nIX`F%BrdUT`HViQXB=(aha`z4o*Ik%J7L;XhKODZw7>oxTpkl& zrw(LeA*mgTNQ)X0Rfv94h3lf1%8_VE=tbC1I7f!A$Kr?}F_R!FsX%=c|H?2JFK-Ai zHiD%m_=bEYss=LL7N5?c2^A~^owF&CO_arwsJXoNQ6Z=f!)H?aT2R8&iqv%SQRQky zYw~Nhf$hNdfC(MQy9~ZkB$RG4i}__mJ?RZ_W8Uudk+R{=4|o6#fqW2MCq%%Yv59fL?S^9m2oXo_FqS;uA@jiIoHSuThB1g*v5F2 z5?9LN_qrV+<}8ae^A@1NZ9u*!fV@wsSyZWO(aIryDx?^=*t3AOBANDUs-SPT%&N|k zo3jWfPL*2^y+{e25GhfgHXsm5I5q0y1xh?fXojSvB$D1Jp7<%z7|j;~?kpmESkG=K z`+WoZ!SLg=VK6-U3iP@0!x8T~YsFEky#eC-serm^fa6!l-%xwH$x|d;eW}B6H47zU z38JUkBXMefb3h?6Nh@0d+C&0gMFXnz06dEU#KY9G1=&WkufnIz7KX16k(~ROh>BqA z650~R5tvkU<{+YfpU685s8X0JagEeqY^3TV<+6<`W2t^|(jz@4TU%2$x9 zrMbD9gWTDF7aB+WiJKG~eV9p;smz~-WvcjPHv*r(so45vG_v%%3>ul4LfL+pD}A~r zCSBDR1EfZfPB?RvR^xoC?3bwiFXE-|bUcZ1A1A^b&dBH)ZB9|9s)%_zESkDajvkwI z7NViDcs}E5>3u!3AX?s+^gCf}+WH*E-Dtv5ryn$-5hlx=W1GP_XM8rju4heFUe8|8 z^(pDkX674o+xD|4mHPpZPt{1)XTdx3pz^%W$o(bX{c8%|$eQZ1lY1{?dYdtow}}iL zi&1ZBc%PG9t z8m@P2;p$Vq1pDsq($p%_%FUuOHnjlEYYBLci=eOk@HF}9vWAk>JlzWHR4G0qCm_`c z;9VFn1)0}dT|(uo4awJKo{ReGnPUfq{Di8C8`y^h_b>yqWRc{CO8I&M)|LcBlp=dm z!d@gp?L3Am{Q37e(5CiB8Oo+mjBN|}hygSo@^&idma3~R^AqN>5^!L|$m&@R9T}vw zyZy;{Dq3`j)nuBn;Gvui{e#b^^1TGRYWhN&VO~IbD8+TatHq>KKN6kyGKm5?$FHGs z+gCd&wh*-PV#Kxo!+nsQ6`j-IGwINV2S^`d+;sbJ=1=`;Lv&yLV?^uIWIezB8KS(& z!`BSEMD(si6T@y0EzAPaLG|wuO>aWfuEArXj>2q0*h`}Mgy3lX_cUkRji1Ox)%end z+@e*71n`wMmx1K*ewvb9ej$LCzaj*#u1>-8c-Y9@q0@$x1mh6OO4gkXjh+T zEA@x+{=Yqn@CvU>bun27MS@1+OsRVGKvhJHI2!?)i4TpibR+v|Z@|qV zOr>vIdskz$8q*H8F-3f+etAlLDt^9r0kG#Rra_&Y1NFsafcMf>bqL{88_ z**(iXv<^W`V)iR6Fc%WOgb+e+0;ScgtC+;KMD|{%HCI~?0**c-Cr9jK)Zqwpf;zJikVWHklQ~Zr`hEgkww?HnpO+!s^Sg#@&x*?H7I$15Go2A)-;<#C$%H8B z^?Qz&DHTD6Z#|NpyEP^#F zKj}tQNWU*~jOrVU4*4v4`FtQCRP6e(QDm1|c#1voD3+i$ok3w)ChhH6hT!@OXx@G% zN8@D;dky*HagZlZ0eEev%$pDSJ&c}eov6Acaiu8pd~5@F znA7yRC^)AmWv)K;S@t z6BDJ!@)K<_>Wh)+#fjpf4Vy}IPufnZ?dOJHr0Pf|%}M~2n8?>M6_@Cbs)YN9-8hV$ zjb_}`4-~TM-4LXq{t;xNkns`4M_PgPDM-JnpUP9*EmZA83n-f_G%)~~y?KpQro)gv z70Ke#5e>@I2wUOOe&1-KhsCfwV3yU1R6s%|IjcvXWE`M-8xlYXYjuTo!9f{s(vVE9&?Kq9;tdsiV3r(SSCb6RT<6p;{#(pS6Me z9~H2P=&-#{$T}hVy$SWJnP&l=UIWxOI@Z3tv4mTwuJ*PV6?$+;Bzp|~^lAgdV!QQI zkhv9PFKn$njefekSr*wqCrZ0I12j{ky%#h`_NOc)T~BmL&gCGdmg>eZTGvm2z9TD7 zJw>kcb%!6cD@ckya|Oi6gv@6y%$2?vN*ZVA8SVus(98qco8JPNM}rPRWYRAqMCQEh z@|EIDqFQ@$z3NS1*~tDou|9&5N2*9}K!#!WdXuL)bbSRc?uT=C3ZV3$ew%^8m$)8K zRd3S8B@|n9i^Me*9Q+SITJioAicvz$nK4UD=fSE)$^>EEKO^Je!}8Tx zp)wx#PE%!j$=aPUlPde3BD&__9I_V6f>|gFrZh;EaR*TF5!qhpi%{KX1eqem!=&o{ zX)BO++l+tchm90kNGsF=VfW`$CL-o=hV*Jk)ikOz|B5k`apT{g#Py>4N5f8GT5d47 z#pG~{vmo`N9izM{67uYbpk>cjBbq$R40I>2sz^*$U*XpuFiZx$j-SPlpcxS#i6yqhftm%PFZWh27+vB*;R_eVev$Ijk0TLIfRj;grLzl9WBU_D};zK zB-jwE{IN6!6=*qNTM0tNQNe8@hOy)%kh6k>aUAqcB&ri86P6L=WJx;PaXcQRP;Mc- z1W{!IJR%+0hQLCcsbZ7p>AZM4^m#HsPoY|H%jEo%a3*BEoSZA)M0mEH2a`sqFZCgv-0z0- z=;B|=7ns#-yTNwf3;$E-N5}D(nbdmHgtpRzb+Rw6S^5UZml!bh+Xt|zJjlaEpMxAk zz6#q8@ZJYdVi$_00_LRwtifx3@uL+_|)z(MLOcK97VmrRJ!9246n8Y2qfk6PscjRO_i7 z#{>I@tN<>3COSI(Im0ZP)$s+BGVT@iTjf27*|V7QYo$nc+jXAxs+bhLwebrY{9(cu z>ia$2Sl$?~<}QZ8c^DYC(S3XjeeaDm-p{=PTzu*pbEV0#{lwxOO@tC@)YerhbBK|T zGEP+~gTSg^HLwtOcGRQdJ_i_xJBO)l($#@d@jaru>=&UZOb&1;#RxYhbSB{N^UH6R z6B~=JcXH%Bm!mezBIv~5p#)hBOR1xJE*8OHBGq!!)%{`^>xrSkm(>*ES` zaxuVQcR()u2Y$ehhZ=Hdcu5vfd09lKy#fBX8FgKa5D!PU1M>%p)pZx|hOUde(n*xQ87l%sfw=e0tRRCqL0)C0f zX!e-w@N;=F8HKNrj*tZ&e;05@jEQS(KBOy;sJLF!Nqbh!k965FW2y{Ji3+O=nAHeS z`z4^nN1FbJe$D#9R32Gf&=jiHY$oI07J%a}iJ@9_NJJsnHfT?JHI9qEZtnuyHTg*A z&kv|y1mIm9V0{Ge?KGg1%yr#SuGn8fffT=iFdW_MN9(-VJseUX99hAniD&=s3bUT@%>E{7%45ltIpcIs!hLNt1KdlRO z^@4IzZ_?%alXh|NCf#0SpqI4CCwpnqCvuQ}o(m8pQ(0N2a!(4`JqMSiiycOh-aHy` zcRZjP6QoNQ^`WAj9H*UQ)SRzL;~*oAp4D7 z70UE-BzoDcDr)G6sP(n5Q+~{*{(ZPQ#qS&=-9!eh?KwbML|Py2RfEQ5A4$67IMQ27 z)uM4Dd`bVU2>4FlR4ry#6}Sz!{1BQ;8UAdPtZ`@Lep`*bi1e-P5)@I|Bx21P)gR*$rAL%<@D*=g?puC!2<{0q&M6)l+OX41^4 z&7_@cx1>xCA@3|58u=UKQ`L%QuhXOl&~w!UlcP3lCuczG){OX_DAE}$Z}m=i7SU}W zpKMP$O6DeuxluKblCv+eE#zCGNe}Nw`gp;1)YiHf=~Kl4<5B^gj+0}w9*gyU+FLf0 z?M8rTGBE|)GCk@ih`UnNv2M6R;kQ+#oGWNyRDSHmQ3rMy>WD0nR(IW4B3+O+mHj0` z%(h0qZ(5Xg6+xu<4PY$~UH?0IQ!dTzJT0e_tw6>SW;3lp5g^40J|%m%CavK0vMpe> z=&WY6;s4&UPV0)-yBo^-J)xY?i~N6NnFI^~gv|mxm02mc7EpQ}1)D*;hr=K3SFe}{ z@8e1CO{pyAK&}-W9WK*su^j`A-dU!{)S_VB< zD+>UQ7X(ZxL=CcPi|nF?xqrr+{Pml;6BwY>gvr1g#S&px!X0)SS1yCec9~DR9e~h- zAdStJQ~zkxkFMNtJNd7BlC?l4EYWTU7WnCD@fUKOr5abpN)9)<Ig&W`dk+rd+@sFN3gK%54D47Q=F3#zmE-??ftR#N)>DI9l$Mk2Aq3ebxZb98 z2@-w;*&<}S1Hzt2i@Qw3(FYXzE8>&;5hWW!luTZ($N%HL4TQ{^`x8pFCvxyPrCeT6 zpxJA{aaomvS(TQP2geVzuz`?&BvPN*n`Js25~`9!&pXV73Qt*8ngxs4xeW89vze^q z3m3?p!zk%V4)Q+msZ8^OJb;!?fB`P##3r3&MV(kfw8c(NgmmvL*}M4DXk9JlB9qr( z80_h>47a-lco3?r z3X?91=vbb4kG%DaO|WC!OQb69$!Fa0!cz>Dlb%^TSL+${^(K8?UX0%dP_0W*a(8uM z7X-K?b}q3cAgVPWt^*(lJ;Lv0GtBIK_2u(s_9;OXbF-0`8mqx1I>gAM{+gN!CD=tLz~7 z5g9h+(e!@EuvuJ1g(Xn6<<-;X$&7uP0l75yjnq|*E7Un=PiIu92m8yl;L#3`LRP%m zC)0H1LlXw{Gsz&g=_en*1yy zdVg1I+J8N78(Qw?42Mej0c&NUZ1n()$Fj*0kKUOt3Q1jC8k~DYSeSElVhnoaCVgFs z6k?_<$243J=`=4Mx2S35p&RTi2VQ3+I>Jj1yq@?H?LK@k)lMJ9`MPR1f#Pq)pK2-U z!)EFjL};_P3}u_h+2CtgY}-ZIlxq*T)`gtiGsaH*HW!l>j&V0Gq`hnNSx{v>v?J*r|nbO8X=X z)Pe4RZT(@|G9i-l`zMI1szCFUaS0%H6*(8f`k=^4ig6C7A;-q`ry*bP;#3sWdZBp` z>HcEvW{Dc~5;a(#QPY3D22(IXMkk;52%0=kqUF5eh%S+V`s_A|XkDfeQS0EbXB{vD_LVNtr3Zl#Z61~}RCF84mWFVGp16j`IsqdU>xXsXIjCqkrw!V6I zKUquB;>nGf$Ln>=!?x383~lMhx=1}@Ta$0MFhru>K48V#BK4M2 zK5=0RoWdp-1W|^761BV&n!#WCVVT&y5P6;9@>Yy)6M`6C>$5DNyd>nbqXt1vJh~8K z2s;V82uC3_p0f+L9P#DmMU%y>RwGBOCacC`eG>S)gpf?QMtDIf#|s=gE+!#bCY9Sz zWq*nJuSBXZcdnN4sLjo%rT+(^qrWPB1iI<&nHhmr1c)Vn51j=c z28Y1nR85Ap<+0Ok&O(&aJ2>B)Z^zdeCl^4b7DBcgK5t>1ZxM>~c-QP4DfV;B!8*Gm zp0JCouHdPT$a4giNCiv>cw@Enl1w>Nm~{;nYhm#T7G=eq5tG>b2DO=_r}98d=UnT= zCTRC&($-s{XJS-xG9)z)fO0+t$*qSNMm^8IV$ zO4GVfRgBOj7p`-}qOui0`cr+cns+pPzV5?^$}XtKwnOm$d#O~@-_74-!x zv1{cNQ8(*gqU9yp$Vx0ilr_;G#hVe$S%R|r#2YBT59F;Oo9Xn9p`<&-P~Udy%Q1?c z1jAkx5?vjEu9jz2%AccVRcjojdU@{Qu(Wqea#wc%*hB)7J45#eItQ+7{(V{5jRH1O z;OEBud?j%tjD@TQHUxnaNn`dJDYx4Sz~`LWkcVLx4-siqS8S1jHKK9uu$9pX*^YTXH^f6zb*YIL<@WSYh3JB81Bu&WF%#Kqgpc!5$NAfzHRk&MNSh;wR&J z1sGeZI7Ci2tj~iXl_5j%ycNJ6D=yzusjLokzAOWyk2i`(6v}^`No9@d--c~f^s>1| ze;sIy*8LjFlT5E!eZhC04f*t4T%hQBw`B7*pLL}x{Xs*D7q3hAlW*zDd=@jk4ePC* z(P@3E+&P+7XfhwunNw1ugN!jc_sVr77SAppY;@3HJ`JLV&0a$(WhbktE%r}dx~e)j zbt)koRQ)D^4zykE$Bom?E5w*W^;chMg9{2i>yyoS zskU$Qo@P7^<;dC`kXS5)!|MC3s-mpL?XcrPjiE4+7Hz0+ZcVQG@B`vKp&J-0N`^dpg7F5z3)VfQD15A z1sJui0Hc=D*ZW*@NLTMwhV)V`Nay%Xp@YxobLy^drSo1&S8Q>Jwr-@{hL-g6!@k3; ztatk<@Rmib52T4_`6=)Zz57-rTWvn;Nqtn2hB&{Xg5&hxHD@JCY;lJ8^WlCp^v!)a zYHFb0>#p$a`72UVVn_@Mp zGOp$((KeyRQ1x~o=%Wmpu5u+C#E;+DOC9gw(2_Qk@OGezsL~MCF9SH@2j#iEHF35} zw!%d+hpz8m7onSTeFRy7fgso!S`7Fz{)&| zF`7l}j$*SUF)YJ!P6nBaGNRhknj&h^cNi=RiQ5u&Q6~%w}Li1TYLs4{A)x+E(bq?!L^vtaX zc^eoVRLnzayZ#7p?J?k|Orq@%c=w6ShhLVhhTIHh?(>BAQKUcxrU6!nls7=(tFiw{ zxwxLbr&%hvow{kRk~k9|Phwlt@Y9f0eL+JIWo?_Y9u$I1dkU>As0I43b zptHc2+zvi-2c7xteU%TT(i)U+=oX{TZjyGDflB4y3b-Z|(cTeMd18AXnl$$0Vk&0s za+mxMW9=IW)ey0RdnZHsdYjy8XC@QP$jwtC^L-!xMr_5~6mCM{sP4ZQ-8R35hI6Z1 zEg#!Y9o>C6(B|!Es%?xR(AhV|GH)FMmVurfybJn}Cp#J&{ngs1RPv6&QWM_-QdmtE z)fDiBo*ZXcSzZ);`^#{=GVH|P;&(hbF%bW(Fj6HJMn%{|yS^={Bc%4XBcBo@*jE(!&~OlSEN5ovw>HH*QF{0`wp^o}04=spv=vBD>Y#kv|? zcd`~sFS2doma6WFjcX=TnordBP?^_fB5-w20NVVe_#1wt)L*&epx^GVil}I z_CqwDPIL_*{alnusypfPWgAo5>pBcSn-N2a<>A4thruf6yJH5KVQc z3+Tu-;=IhBFb2UHv&E*FF(@pJjt9w)P|Kve||?4Im*i-kpFQ zu)OaoxdVgZv6}|{f-E$J=Fh>JSuB+8K~fE6)i2Bdl?uV(Oi<-zNIo40Y(E3YeHP$w z9&lI&r@APy^P)$l+_Uv?R*6}ZJxX=2nX2eDW){%!osjI@1^x}VTW94^-xCIU%bozb zSANkn>Gg3r%PV*p()?Ed$=E8({!#X5SO3u2TSOdL5SD#-XKG*ubRNSBo*yqI1)b%0 z3^pl@M39Nn(s3CX)`5z%!6wfkA@&$4QnT6A7*>)HLyu_}*;gg)Fw*|nsNO3(poSSR z8M#w#tU0wY7ob-jz{-4pTDX_WsRY2MEVTv0;}bl(`m$1{lm|UPT-=LI0dt$tTXizY z!&9$sFAvi9%T}$2Xq%$l0W24M@2Y2(mEDYYIhc9*0*X`xbgu!J z#8_*G`u&hc*C|sWdb|K^g@s`Ie!s;Ma?!VuyF46`7~@qPPei|*>!23wVs=~~4Ef@x zP?mlv@5+L|5bYH#XR?LPPomZb4d|s#b>9H=i^m;MFIWl|O|*d@|4@5b>-tvKAsmO( zFUhb7?JMSXI-@9>=q}y!f6Wxhv}6~hI0JQJc@5U&CrWYdxx=Pty)z>n{e6=`8Eo~1 zjoBZFBjsx>2L&;nsOoD&YPk$$PFFf`J-B#FHy`CvA4C-y{zXv`%3A2(5yEQ^LD|wE zo%A-4%c6FVf@%7>S72AZgYY(lHQ&?BT|{Nkp-`KwBkKHZ6xA-b#h~d= zFUJ9`)44h0r`4JN2Q>RRk!(+yki20tu%c_9*SO}TUgQPK4{!x>kU z+6otAq#Nz^>xy%;Jq5%z?25AA&seZCRJHMql~+1tD*ff~Umr4bqzc=4bEKkQi(t4} zmQTu1dS@!4xhN)t;Oi@FOpv zHB+dLFkhDUaa)ob(c)aW8q?NJ<$A#|6HTgPy!tX4lV6Mkv5_^V;=X}BF=j5>8AYMn z$~RIvT#J{udAm~&W3u)gQXd}44&fAWr%JhVHNWv+(v$B7tit}SPK}VOxvo=4iFp!D zGie4UrN9CPIQi+)o*X??!d=u8dU9p0JB(Tn%NctZ4c+a=4q)3I2dUBSO`TpwKk<$& z$~EvUEUJB?l^&E;3sKneO{+e70M(_f+`G}D$sTasoEQyP{{b-|rVa$*S4vLR`eDZO z=~9!)`&e@UwX~XpjHrE#zxhX$eIMfrHMSFs+hfVhEv&2cH9D)iC}H&uRj9}PZpf6W ze!XefumwK-=+0|#M03EU%9&6~cq#^UK(9m<(nTGrxQ@HcUL)$C}>L#ga| zOROutufWEl!SS^j z*dlYtKF0(tO-COozf6(?q27FiR@?82XRx-P`^zx-I6Dt;d#6U_he3}4@?&Ekm^Gya z1EOL8V@5M0R~uOk#Sl@Gv)h7(9d@i4!||^AY`i^_c9POM0N{i!UR3HS95FT()#qi#hc$>95B~S1Rl^Qjh=vdsM=a;o$4cA>}!#|;4^$+U>dUI-Hrbt#hKhdtSkssmmSfAS`8{6$V7RYA15etsJ*ZGQZ!zzEp2dLtBX z0n24cN1}biiP4*y$_u^<%}5?%46`>!;y5n0T%+~o?|%;PLo%~Putz8i(uGitz$&mD zs{L;Ph%@`|HFXQFwAPT=NBB>llxd0VkA#hPE9U8#_zp+gQ0+}nhPI%#p@h|Urq1#M zWI3W}Ngc|w*OsR7FSP$XEA#EhzO*)pz64CFG*N|gf;l@}%a3NrioV_*pMHLCIA&uK zEbBFh4^GspA)po`<(BXBZO4Ezj0e?WBB1>w@C)Ix84fw)S-=%ZNIe@tm=jpu<%#s- z{@(J^Np=Ni5>XNI*j`P;*s8esfcm1TBGv*fBm+L~1bjUO_%4$86-iVNK2p^e%C|s^ zsKC#p{F(upjIzHKBhod zDaYeN%OjcrzB1rcs|^{)nauBUQ0M4YoeMbL z6sf}TfSMiWPAvtt~8ywQGl)@{@jEOOmgDwydH+zcom~0s z9L=&7sa$|6wm>vFEf(Y{|2T?1j1WKNLn{hco@!PSq3}Q`RIzx->oF|t2|sAr+Zo8$ zO6hisakF@9lm9UuiJXjR{{)Ub^^tOCVR-QQI~cZn35SSRfbVq7Tr;8E$fS3cudzDI zOORz&V(R`LZv$yJjxa3Go_2B@t$vEk{Bs%*dIr3lqYXI6h>W2T6G)ik-Zzn1QA?PL ze}B@C4S~hQ-Y7pnIQkA~Ircscb-}v`v~2N5lTcX{bwItS98Ur1&mg=)9_!pveDibq zcaSz)PI^1Nh3hJc)nY57n*IQdNz(x~Z9V+dhLSL7&$`lQ|B5I4D-|b9{h`^q_41#i4>Ca@OTlYoeYb`uAY`{+zs*u%e8RoC(vR}gFZ7jcRO_A0;yqV=+n9a zE_MT)?EyHv7%*}xU?^2t?mo+1ru=g3jQ(cRnfq~jG6#484vJa*fR0sT>H&hWN~*Xh zfhg+PHI`DdrU=DUz(Uy#1+ffNcbw0EpY)b>=)wc8GLEG0GE&MGvHn!KtN$YH8)sXqYduGXv6zbpz?Wx z;?zUE=J&2tb{46@k(6CU#z+nf4ijW__Ectqny`{AFZNim%DyJ^+n;Q-mt4+UIe&7b$>+- z96YIj*YFjW^&|IBMvQB9c$3)QL%?8p<^7!yD8y8?H3@TP)eCMzs0^6sB+@(`fNK5=|P@ z;-gIU$(A-9;AOgW-TG zBLS8-|2ASdO%tH)3bQ1_ZOSZ~jATa4AnVRd+Bjt8N)LWn!EHb}H{bM_wiVer)nH-;Z1|Ix13q z?oGwnqYvORO>Sct!)8R@_^HY=97$*%9rVVC3mtW}6aH$4T(x649|&h5>yelEnLPCl zEvz5K{v{ebmLg@x0pb(>ZHu~so>y(BfT}Gr`*Q&x+Y&&~QVhc3ErvL*{nyx9(X-z_ zxMR{IrWT@q(`jiEdjM69fwTx`u{orsq4{*Vv`QQktPvHpTEuAiQ9vg;uQHAUKA#0N zyFfA4mX0ap$A$U#%Ip(#iLYWM%k=F|1p!pBV*o7EkUSlJc0FFTLButyVH!P2X$uo? z8G)MORws&E9n}%gPbA`iNW`ih04v5$y%D+fchBT9EIW+f*NHd zZ);0iL%0Xcr%jLKXK=g`nOZE(+4BR7Eb2iY(1j_dt|FwW+Zez_S&iZZ zq|Zcz15Cwmh<4kMvC9^aR|7Vi0rPVJPUZqM%MDmi0OrQh6`6^1o8%nddmDUMH7cI# zW97FC8u=heT}_41{zYT<2;WbME$xiC)E&;=V@gh4fuPbI(r2c;W^jsSOr7Lr&=IF+ zBbK=ewu|>rpRpHFXCgXCtb=74=BAB3m!DlFvkzSc4O56zB}@a0Q+PTP)&J zSYN0}a~@S9bAnjJE_I;@#Fg|@>uICzRJ0wP98!#|fg)Q$G7<&-XkZx%m#9Gls}kXv zE>9qXD6q0Kc*AHQ>H=6G4Ue$Z`4>gkE)B2pF~q&I$W=&=>K)EsW}4q)K_^r~hfV52Jd48Fg78 z>vS4L^!`IPkZY^u_h4806NgyZcji-=9zxHE{nI;VzhcQ9Wx0h7t1`uy^6x9x$zAJH zZ0UK_f$~PKRKB|~+kANdw5gQJ>L_PYzTU^kPh@49>lPS1jlKHeYS2d4fj3UC@(yLb zPJQA9j~F3se9YEUJ%0m<&nHNFKvGcLxk=YAL9&fDxIYe=&c1Xcc39u2*_=8O+KXd8 zQIw2JXRw36{YAh_W*BF1sFNuQK7*l9-$lI@J_SLOXn9rV(rp%f&2w}U)9PJoAZk_c@Hux0B=+ zKUI$5ynhC2`DIY~DOxpL-h*sM8q~#WfFW4`%R`StkBQ}Z4aoT(+KbS-en4tNSrY7{ z_3tH>@MRq^JeF&**W_C*!yjlzzP!SbC7WY8>z9qUGIQK*@oK)vFV5Jlq>yYSR;>mt zKA_@i=KucM?{N^N)>3jOY{Z4-k&UzEB5?+Y)gU^qC-*NAzm2fFwi$MRNLASamq=)E zHZ7lq_-@68Xttd`EZzfHkOHBs-Jb_2)=w;53mJ^1()~kNUNr}`DDAX7>E)9-XUoeU z(X9>du-q%$CmU}{U#VKzBUL>^c6E^xT%fALjIa942gZPp3mVbP%0|@Esc^u_#(?8Z0Lz;K2DAX!w}h}?x29NzvC*VsyOUlN z+nhzU6=wbA(ZU(Q=p~ zI-&fDMzo=K+2*vNM>(?V|C2x)42Zgl649%6auo#MiljkcD;8E<^*NQO^#Bombe&fe zji75Abe?0;am>Jc*k<;>jd(xA?x=GE0sTh-HjM@pi3Ma%0z}25Oxo}Dna2;R?6_9} z{+C6DzdQnzdkV{pP+G4_4JR66R~-d$w4*%{7_JYz35*!ZUR&2{ z@EK0K=&5RmV3;SAx1Mi|*G*#WK-;Z`*NfJ*=%H5|D2B__+?1&)n|_qKCQ7$cSYc-0 zjy4N;WBNOefzeCG4g_8<$|aOqhheZB8=s;3_C5$2j({FobUUAPFC(_6=w7(*hDu+k zb_g$4eE2@qnL1ltejzVGK3n<0=0YF@Ne_75tDjiG*SvIz^J31p>#wH;IX95GHGzMc zpj$<_V_IVwR;;3)AKw9u*fw_25{_Cc#6h}#EZ|52^p%InPcr9{e`26!<_0ln!4F!J zV==wiIkCY~+$&RBf0%i}tP7zVK@NIW5X5!gMYuy_tx+EbrFcOUbf%+706A8sA~-aLP9P4-a1gYUS2Yuim%W@OgB4lEqYXR07sl+$*iIPKe_-cQ{|BelX_bR zn)>P50qmJ|-hD%vutMqoYsAP5*8HWE;Y{WCeIv<HsYo)q8QAG*H$V-6pv&TkYiA=}dZ zbScL-d<9+9@i@(Q%LP$QVlD13<7W)SXQLuSIuSzF`qbifm3b zJ!OsF8vsZb{n2_FZIEV`<2FQ3F8#_Bqiw&vN2IZbdP8q6>$H>M2etmTg1oK*gqi@> zW`Ji7s{iTui??ncCe&J%#P!&6Fpb`}XOWJCYp!>U0)lF>NRIE3I)qgf6VoXN0CKQ$ z7=+tLOjth>n@H^Z3EBVFhBEvjkx9;Va-x2d5Zh+Y`x6~(-ND4KaMKRU;fzJ%b3(w; zvRdXtSIhP$YbIsa5cYvB(1&W86WKb5?1ocm>c>I7`2fgvFwrF^I8D$8$1Y-^XaD5k zgp^bFe$iGr18|lz<0cGBx|KW*sccP=xJ`!{`2#7agGI-@mb)dJua+sWf@|=0RlMO+ z>sb@vBoqFr8el_XQYpS^5^wP>zqBQ8g!xcFd=hHJ#dF~7#-ofdw86Qz?1xzJYqDxOC^Q2Er+D$PBG3p2Q0Ztr(!TmHo7}Rt6;lZg# zBfu6J39m2kdM=M}V??rxC(#qxU+9*jKGOFQMe(;(2T!Oy0s#|(0BwWmU41LVD&u+mv{R6BVWB6ZiI+lNrlr5HfRqbl@j5Vbi z3dCB+7#v)!LSyaxjiq8uDTaIo>(2ZWc%z@c?K6W*mUV0_*|iOJp2p%?YDH|G6;=_3 z?9s+DS*FxjXIJa4h8*jSm9lKIVhgRX9%8V6W~`ZIYlw5jei~v(&*!0lbH(3E~(}v#tJiRE~gfV zwC zP8Tt{k5@B&a)k!g^|32q^jNLtCg*AySihfLS!3}`wa_PbbYO#IyUNBAkJQrS+%bWT z-r3c{=9`jy^27%=_Q_VqSkm3JCOOYMtESP}8s0Bi-L&4pX?1$D71^4nm1=9+Ji}>M zIz!Yh$!m~lTb9#5RxJ(LqXN9AOxW(9_qbIn&+J|AdoMBVnvwT%dh3?id&ZScp0Fn? z?_I05vDy0vlsP?tk$f?>T}t+$#=MrfNz-1$?*K%8OX}QfbzX;&YsU-J)@f;M^ZqADK;0w z3xtj9?w4b|TgCMgucQ`eJ+5a=j%2q=$0srs(LueFbL=%%eq?hexnN9C-*-6<->+<# z#Jo(j?(doN=zTxeNsm?(nrA&QI_H?P_pA6%dgfYqt@YsbIj^QwZEO2#M&VrX1J$C^|kznp@TJlfF5)77dP%cMK?6Z+wEEqi4~O_3=(=MO%+gjB)fSTKjl>{(xdV z#!pRg^ebBDS$yHNV#CKz|K{jlG*l)76MWo?A5{xIb9a3Zwj!ZIQt_*5adhrJ4;rK< z__>vMqL!}DJ>)^dtc2=GCB9@X%gQ|}z0of_e-_?^70!9aj}6Z=Ik2&3wh2jodE$!_ z3EA%H9I~okp6OOig0s~vSJE>C*_v<s8d5H-xA+|YtU)OzfANo$D_txl5}prIj>}&%DE0b!OWA~%zRJn@;g+d4yI83t zG-oMi<}dY4tyynnoY2~!ymnXqvX!ZKyI7A;XzM>DKQ zPdO}OUgz+nQfd9jaMjexEmq?vzTGsWbmU6aw6-my%O?KXH0GKU8=Nf)ehKiJ1py6=RCbiXT919`|Y=6{oN-msM5QpR%*YaEhC_8QuJnn zpK4cCvwCa|8t;<0*~m)0QZ-w1Ye<<(&SsM+^?Ry0j$6aVPd>QWtW^Dx)Xv!h7M`43 z;$-nj{U51aaRb7Qr&Kyw8EL$h$}JcWd2&jFlTEnB2dUjR21FT8ZF3rZLgTyCp5_6o zPfq>eH1?Cme^Ps;2i8Q;rIOWUEPc=)!0JkRaDJi@f6v>X_>tysmYu$R{X zZSR(YGq)Yi^E%th=Z$t?%b|7Kj;nZA^zzfv32iyNciZVa?}yd?K{^Y|k6e^5k@ab= zp1w~fvi#^1`HLAoU#kOO=&UY3_DBAztglqf%n`b4HZ~nbl7+czp6EMI+V18(CR&n(oPoenz^5 z^%3BgMMa*3m=>w?DJ*b%*lthn4OA$cz^J;W%HLzK3Z#bA^y?V!85+h zUpM)g+5-g zbJ44Xm;03Tw0If%Bz@=7S&Q!UIqzigHuUN7oy%V>dUl%*{gcD9TT0O$i`#Esx@Pgy z;h%P;HT8@C+@^Q$GV!^z@&=D3(s!=eTK1gyLRUGyeu;AG^>E96-Cx@8+B7m;y>Iye zONG}jXYNYQ4>#|7^Nr=u*RR&?N*}q@p>LIzmHO-Ey}L5=mwNZD3A56F-EvhqYrwME zcW&*oGP~LOM0wZNWf6Doys#R5^YtI)y#pfR?%W$;J>h2Cpjcd9s=|%5W<{pz=UY6RRXfwO0JuCNA?DD$QCO@0C3yR*C zy~*cG2spEcUV6n*%Sdqr}kMA~!3QE^2d2kpKdyHYmorQfLJ zqED8)Z%Rg~q%~)c$}IZqySpYf$~djHan!D&FDrN7m5d&r)}}bRpy+GX?)unh_q4Zu zqYoE#mh65cxhgoVJ$v-&qHhhmAIGj*n%2=c`eM=dAG`mNjM9%RR3pSLdaD%^q{V=$G%Fcd@HarhRK1^Q7qa$~_+>*OaCGP#oJ_^e1c2 zm)JFRX}|o&wio?dvgfy6X9HH}j)mW0#n4v5>}p5DvfR=2oo>E3}-vBBw_0H(#=8uu#2#V$?n+hjYcxckq&!=&Oir1u{@ z-oChp;=Yk_aT)2d{^R-AgRJ&xNNvnZm&+ONS=`HSpHAGylj#agW*Qw4he6Y(lBy!1pSF z_mg634I~_{IS%SGGT1XYxz=dBLzSbV-pJ7V$-8PzHaOHe4xXIPXY9c(X6GFmG?l{g z7v4xYzs2I0!*fmLjU&TNQ!Cx9j3>VCF=YS9$kV9}ZZ=CNcC-z-mcLRptyO09!HJ!f zL!aiaic9;vWlY_~Uvr0j%3sr*_E%6;W7mqW~WQn*;+gH&>Eptz~}Pd47+H@ z{#q(d1)J@+XlG0~?>I#4d?b&F|+qq~!_lSWUq{i@qeXZX5L zn&+hdM(tWbmg?3}*~tf;%-^WLDA-l7b-C=+w@zc;Xmr`XS2Z(U)=k^l;f>~q{RIV? z+hslGIZt_`WxKyvbz7mV*FopC-fy&n_8%_Tc2?Hst#jZTo%Q>Vt8TB5_0ygd`bKx( z{?i59AIeUfH)+uuy=(hRRClz>1{|EU{EhyL{TB;%e3PB=cG8+R23-pI%{nP5mu@H0h6f0yI-@%dCZla zoWI$0OJUvYoGAn5IZr+qW_F_RVRKI4fCY(@Pdb_3R&Vm$xuAYw+2j%}ix%~#r+2RO zSp0VK@S~Ze}9vTn_9JfuBP0!JkR@y!ESoh6E0~ElFy%gKRM6M ztlHtJrt-G@h|JU`w^1B2#ax`MkxQ3ww{>>xd0>QD!6wfw+U^c-9Y-8c-CvM-Kf~YM z<)D+<0WMp1%Vj3IyE!{M9ndn{Uo5xnpu1PKv)=)ojr&h#Zf|m*o;WE|%b;`rWx1?g z9>LBoo3)J03M=HYRXjqfU2?Vl_o3vOX6%A?GfcX^}W`Z@}ggvdwzMW?d954+t#eOhu1!1&-k~lLk`+)ESBAt z=kJ+((9P=L#I6TawiV=fW*&6+J2+|M0po3jO`g1ar1sR#1LM4kWxetfJ-2By16Qwu z?p}uyz0PR+bRL}PbvVQ8RHF9-?P-yRmUtbl^SY4e)2Th9^Uyl4yczPZ)Yv)cLMeN}3g^o7Q<^oOh+H-^ird z+jO@^o^@qx0lU?^_vuCP|?WbazIcTH<}D&d(-kPN(jk&Qt5W@5%bx zCC!!B%a1&r?p^2ZKPf59Os}Z(^j`1#8U7wg^Sty9MV>kC{jl2Kf71N0BS+4hIs5S8 zOaFkH`K$Gg%b&gRut8;7NX>#=z0(`d)@^U{pSIx8!qOw>x}JNv{YlQWh(n7Sk6hSz z?(_C%P19lyE&h4rO4kyJ9nS|(k2|zP@#yu9C9*qS`cF?j6mE6&X4g`c9bEfuOlKhUKVr+#x{&iz2EKI;I5=K}|}CkC9H zzr6NnQ^FBeT3o(yANCaM+w6c%EORtN`J3d(lR_a7)9cwvrar?k8 z^8#!Uw;GdgS5JT>6i(B~b0n`Znuw6@dW zzs@VdmkUnGmT4e5>hk8@q>J*fBmApQ1@*NzWTzaBcM&!Hgm z_V^daxkz0;sNcJwG3^OmP7GJS5kE*qC)lBV(})wQg*Ubj8ZbY2O8aKp6PoJf1r2if z!Cto#1B`UHm!ED>coiIQI4SCcLAQ#stU*e%W*<(@K4B7HQI|D%+N=eKQ%X-*bi4U7 zOL^z4h{LIkCv4(xe$E>Dcvj5ew4W!&cB_=g9dx7jhco<6PKvKG&Q^asd)MKuQ75N%s~(@NsT5LhICGn^$MR}dKW&eYL&@9D z82fyy4ph`l3OTu8d+ka8(3(Yx`ZXcvU3R=T4t!V>(`Xi`{yU;3^ zY(10Ehqrb$@(7~VCCBL$m&pgS%`SvC=PtEx*y{E-wb>UsEeqPP__tu_&PjNXdkuy$d zuGEpeil<}a@5$zjpEj2#8fK;m-|h|fx8FK+(8Re`k*{os|7&Om}>%o9XYeHOwTX@PX;h<@HPar__hpq!e|U?)g@~ z&fj&wJiC-)d9(cG_tX8|J?2eHIbdd1)Ovq+6W8NzlMgzbIpFZ%V2;;?dA@pw!p*mu{#?By2hN{a!KZc0HT7i_$Je8^d@;bnIQPg{_D7Po81~!cFx|p zAT#Bpmw8QSqrOsT&4QeTr^3(PX>A;%G`IJ{yoIN?oUIFOnxZss(!zra&zv~>u(c^r zX+iA56O+&0HgEENykO_T3kys1&b>MNwBm8}&c(kMUeznYY(Gi`(=r z9XQun@vLFms%48i-e0WC7lqUQ?AvGbSo?YL%J(nI5gy7I* zG{-A%(OK%yQ8}ViwWxW!^49rFr*u@=mTGFW6e@2!wA8z!I;d2)sO7Bkj(1A~J8IUK z8fvswC}-;|3+=eIuhg`t_2HA8S<4pQxqZpXa^UOcC%f{OMc%pd)N0g@*I%FPezk1% zoxA^9jT_h|^>pvZh>dsd4Y9W0(I)paZ&pOgo!ZgXjsxGQJ}tz;dXTIAuQ4@%Gbbbl9|dw6H$>7x%D&reHu zw>0?+~Z2iZyaq*@?Q-mm)up2Doq4WAv`!3%v`sMl0+gYoR-0dvA{HXiy;2G^( zSD#Az_TX~UoZpLvb=0grKlS^Y%TLe!UOD5#m(^GL{pdROg~gvuGd`KDx!Lch+SrzJ zf3jwLS+?eGzh4u^zOneXct+=`HIMrJ4jKFY+`lEmzQ0@Z?C$TVD<8W5E!*=$X>Ied zKiOBlB>b)0^K07L_GAB+Uiserzn6RdY+d^$?eBxJzgPVCamL^JwZGH;>m2*vcOFm* z>2(rIgzQy`xuV~2372)fmUJ<`+O16t8!i>MPR3PY%D7&>UG$!Jy|GTwRdUg|z71VQ zKj&s}gsW8QI9Xqb$gU`h+Pe1WYpdeRU!&pKr;}?>eOvVgsoduSVm7!M_Hw?a<=FMmUipg~d<}cgyQWv! z^}^nPe>TiCluo>6WZA81rlM(V=tt?iYo^8B9u23rT6C|^Wn0TZ-CLe37sN*1>-+4Q z&8F^O_71%tyDGikH`}p4x_1d0E*BS@-d~|?{M;V>MyR;NB`%RME^}z>F>Hita$Lp| zS@$yMxjhX=sNagqSu$Yh_$duNZT4yWiQ8)^mssZJ*lW^0tx+4-78}YREc2@DHEo|x z%*Nw}3e{zPj=ksa)4RB_#Bku-vVh9otM(cE*?84(P_OGjj?zi{j7G&*8Y*gE52=*i zxz8jf{+{7r=j&mPeGUYfosNIhUn$gXVM(9TAd9c@|MXXmw+pxITNPwwn((^+kV3o2 zlD>^WHVYCuJ`O3nzEZQ_>k*?5Cw%)j^yT%K;(p&pjBQW&`*E1Wjdhy+yQ|phZtC`F zxXO*V;{F3w>=tb5`)S1Z8=Ev`G+x>lY?7~43A9hnk}=D3c(O_P#K`CyX+31^@*I^n ztDaEJxv_PVtVf>n^v${_)Xv-Q_#qnKx{#cXYzp28E|Dr^O^KG19Lt=f?S6Uckkql|}|{%P&<9 z{GB)BPtrOg!(J8KMfc7R8kM}s$Vj`QymF9I{;ZhfbR%Qu3T~-&@}%bzSHZa&vkP8i8TR(!G?Ib%eY}?YgrklMjVb}n zNpV|@+#T*X4)zVjzb$qu5M0~%CL2J?CYfBr<&5d#XiHy-AUg`y-_u{ znP>8D?#@%3wEfhMG;h0+;paYSk&}L-+NI_lQd@)ET~eKl{nT$aXWMTLbDwNxd?l@WdX}oXFy|FdQc&pgIQv=YgoPYAz3rT=Ppfl4sZLbLv9H^%eD;y3nW?3lpmsucd&Os8 zm73WHba%9zoW1?lNk8*RbAIaX3pt&+y*|U=d(!+Uy%Qm4?{9ycF>U#z#XtEfp+tJe z3-{@%la~7F-)LcKmdo(J^UFqrtx| zSERDKc+BkUvcb==cj2{BSv^dG)Lk}~8Y+ffkI3pbAlTv7CM_ck_43163SWYQZzUfv zvQWR-o;CE#tPQu)wTvgKSLtS}f0>gTVSd|c7IytP4fXGr~m-IGsen?Go6n!R&j{ld$WOL|(oDtxkQ zXT+&RO}8&>w)j%`?8(lUQ;UDyzVg$e%be$OyJEL4QJ8W)%2H;|OV3?NTf@z#-27=d zVor1Bu8d_%-S1R~Ss7@w-rAM3Y}wK~cN(om7rp+oYp+Q}{FGWP>xmj~baM;qBX&=@ zpKa~+`t9`G!>1z8PI>gxI%rONX6~u2%OBop3bR?P(QzZU)MG`*9Uhacn)Bgn?v(*6 zyG?yzHEQ#mPo}#ozN}Qa+k9Zu4vjAhcHj9DHU4g!*697OzZUF%P#+yQmF;ro^|$8T zPwQ8$nfg(2O!?~{s(V`MV|Gvdnmy)0(XZ)y+E1-MfA@ROv9F5$?Ar77)S9Ndzc-Kl zQuO!9op)9VJ`lS~{ppjaYcuk`8l*7v&ivgO}*-L`i?<_2%q z-qE&3bGtp*J2)bCk!#;3+tF=&?=!+PF4a}mfBfXOo)h+Inr=LJkH<07ihHl#XJ{H< zeNWNZZhmo}Gy5#$5?Wn{oVSab+plxqSh-Ez+(ua2Z=Oq+J5g@4x*Hv9_S^xHc~j*Q z9o)36Cmfh7e370s5R7fxT&eAmoF3gH94==^q|8dO{InT zi@v6m)mjdo*wUuFKR@z*>NB@dITJgJhd#((m6`Ud{QvReK#IEZQv67fgy4Vu2_a~2 zdL|*C`Aot^zRX8n$O;j6tjpI}pPLS952nS3=DD63LqCQEU!C9Qc$9z@c%kbF!sv-_b2bte7=7&|-b6i!l<2^d? z0Ut5*ys8MscI;J^5c_c9vZ}a*a%?YC73<1X#d^dd5o;?{#X79V22`OM!>Uxpa6GC} z6%A;;&GJ816%I&v&3d)7J-;(9_We{9dDxEvNc~b3FQwFkTn{zTrH`6$l~EIJaEH0P zny|pfp=#o`mYTSOuG$j9!ca|E!V2AJk0Wg{9(L%?Fne+W9N>tFaDp=?!3Dh-uP1t7 zGVR{v6mlwDAx+zj^ng41(C!OQYA^V}8@}+vbWDRk0uYE9n28_+V-{v31fiINxd_8N z%tt@w(H{$_7h(|>V+q2s6w45SNJL>JqOl4wSdBG^$7UoU1!*{Lq$W<_Bu?Qp&f*Nt zp#-Hkj|;enOSp_{D8qHs;5Mciv+fAM49rBN3Cn~5X6#!G#2_eQFqEK-AsC8b7y%WG zgeuhFgRK^7A`{zi5tnfdS5b!RxPfw1z}QkvnBf${t<;2qjhYxWN==MK2L_F0Iq(7F z#&axSD(2d;z2Q5NZRV&ZjybcPW~m98NY(?xFm5@=8pgu~0a%E?>o{&Vv79OYJGLZ5 zMXs8-iAq$V8k&35MEYK~Ss}+V?xGft&;XZ192ZB}&sYWZ<2bD*5^(h_>x?qoKm|sY zu$`a=8;pZ3Cc+Uen2czQKhJq|QB8cpXH2`!e2wWhD%8YAG*G`KQ|r~l*N1B28&;V} zh`2{;LZeAdJbcM`&1yo+R6=MYo;m@Wuo;O+LNZd23LVBvBh#@38Q6+UY{xe2Ko+vG z6T7e*d!WnodeDaf3}FOQ7{dg68E+r*U`E><7RaYvfc+>!F$!VHYX`_fIEcfrV%QP# zIF8~NPT(Z0dF>Q=8fS18=U~I|QnCc2sLzuZa0wT26_;@Z*HDJ*xPj4(KL+L06}X8i zRH7O+xP{xei#xc7TGU}I(~g5J>S>QB?Qoy^0qm(Cl8?}U3A7u@COpOyJjFBogXegG zmw1I{wBj||@CI-34()i45BP{r_>3?3icUB%zwhKP{Kg;ri@*2}qJ_R55|D%xx}qDp zqX&9I8vW28GLXdp$YCG`VF+}g2Ynd85Js@XD2&D!jDsygFdJQ3Sq40V;T!fHF5@bi z(a^3Y8j#2tg=P7@mwd)N>Jrd6PU8rCr zCYwu$XTRt_gt~A-BC;9wx{JC9mrxf=5setE#s)}AstYL;si})5Lv`^OPtc4OwBtQW zEhR(;`2im>&{$o#jZzox_&Z)*1UadTjZ@UcArIaoNL@UISgtNoqSVEzb-ad_6m?;l zt1gBXs*5kD)rH)7bs>*|P{1GzgCXoInWk1LMB2G5I~)wZDc)LNXpIYKVb@HN*jZ z4KdA4Lrg~i0x<(K5rkmOLaQ_Vhlhq3i*XnaTeLC!4JJ@Kz!^@M2uDnU3ns%AZg7VO zJmCdz_`nx_@W*rnVg_a+2*H?z*$Bm4gkc`$V*wUo5f)0eo!waf09eca4f|#L?9B&u@X`E#dyE*2hp@wVKri~25Yen z>#+fGh{Z<4V-pgw8OcaND$@fijaKuD7!wHk%g2|YIsc?lG+~EOF zc)=S!n8^HnNk`HNe$>w7G}0ddn2t#dn}W%3fh(rM9d7V|C%oVdU--ZeflNOG{?s$c zAWWkUCTC$fb-@3|!))5Y|l1Xh)EdSpL5@R{jseqsS;Snv5o6um)?f4(kz%jY!0CoWPK| z8p04WFmEBtgaufLMOcg_2**+^Lj)qR3NcuXHCT)FIJa0sTtgYk@tS37!yCNATLdj( z+aUx+*dETYi%AjeFSx=3;aG+UTtgYI;|3N*(w86#@kqcXY=*;f`d2t&SG0!6#crHi zr6EetzD7g5hub>N+f>dM^h(nZy^*np{sM)_+|Rj>d?cOK5Yjg_L?85pJQUzkr6F$J z)esG@IB#)~Kf~rfSW~F_axJW@DNJF83HqAik-4TwVt6uA&`3L#OhY=hAOl;GiEY@9 z=ZyCP%?Ps86j4Y)D$F47dMmuQOh*nn7kU%@hLVi}NzbR=xo6#3bjq5#diHATi=O|cbO zaLeQGKEeaK`Iki`e?Y!`XRQAZGbM7nnD6HRcw!H=HI~laTe!Lf>QkcqA9w5)fCb=(?v_{ zchM4~=1Pceep;LhTEb|bmbg%$B__fsTco08skR7+))tZ5w1q~LgwRCQ zPHmw@>OdR1&_gxDjmR4E4(eb`yPh;5O>v*vjC@Ew!UHs*5#|hUA|K-kENDL^|G{%u z(tb(4z$-MP1+93EHoV0fyhjH<;UhldJN}@{E^Q$JDRf0QbVm>LL@)G(3}i6?li-5M zn1UHtiH#VYt1TpVYm1$E+F}=Su@4C>??Lh?j^Q{?;tbB>IW{rQ=Krl1bt3tSY(@)O zk;Jez@(teN9g-Q=PQFJ6KHwuh;WNJAE56}7e&8p5;Wz%^FaE`U5c#Y>dY~tILlJ|a zj3F2ab!b8x%>^tUUc;wYTlir*<|E;dHs=rPgCrzF`mnZ;g922r5R0%FkywG1h{CG> zwNKDS6#ksy*f`7a083cI2BWaxJoCgNEJiX?;D3RAe2x7ADRjfZN^LQ%QCkFJ4wk?+ zMnXi8k$C)$;|^^f+1_}Icj&-p4F1Hi`kV3a4Ca5dg(K1+{<5xE+D%6+Lj) zt|Mkh>xdv6!%P_+5scXgg}joEP{2S8f+D`C>4;8z#~&D4F^#j180DrT#=;g&K04wb z)UMGHfje|W(FGmxv`I(2Kns2#qFF~o!uYk0$nDe-OaJJI3<+H^OltaLxY(ohC>}1&_rOEt_WMEE9PSXO4sU& zo|kn+FX&#^70>SIihuANFA(sUZGf2wg3mKu@vu!-JVFC{$mj_<49?-Yl^l;>1N6ku zfqG&X;uyY2Sx>kO)f184dSW?l&C?UNaR+yC4@V>Pg!4W@^_9YjQ{mA~1fh^=84+YG*uP1so>51OxgTCkw8OUM)xlq-d#@*cU}J}#h)0)?dO`w=KI@6a zX#b)o-s9g+Jt6gt^}%Ia!WCRY8Ls07%JC47pzwp~elp!JJ<%PV_=cUo^+YbVchwgT zz4iI+etq$@zrJ{e=h!2wFZM!qfWDA}ww%7uL!*Me&>g5RUJlh4&A2#BUtGeo;rikq z#E#Gx8}S|;@EWNvWK{Kq23Dx+iw#K9&=)C4#TINuCcHHDg*SZQ3k4m0zKhW3d9gm< zH|X;ngT9z3;b)EAHN4A1cb*$i(cThNNv$YIz{>_RSf z;~m4>v4{FSxtH9BJmjN;VFly|@)JH{KkY(%rv8dA_>Ld=iC_2^e<7muMHfgw65Y@p zJJPt680yz+aFc^wZ!VnC_aEyQoRG|iSXhIiy z(1#HWU^R$o}+*+z~>4BO1QV+2$%5~@&x zIy9gOEokH9cDBb!?%nq4i@ACFqU&My6T0L1b@s;%eeteBU)-x^Tiw+cp-=QhNsGQv z{>Xf#3`CZ_ffzH@K#YZskAYC1V;~01GZ5C12BLYrfvDPOAgWP=Teyvvmn1|TS&#e3 zO)(I8Se0QQ?rk>^CRZdx?;HamjXvm$cKqCJARg|g=5_xf12GNP4;hFXC`Scq5PZx) z%t8oUuNnw97*!dFIc0oLK$^fDW-!HEhFg%9u!1#gFbbnF24gV}wipjPgfZO&(g71O zkM<7CvNq zT$d1!p#9uH=)w%<$iPvcQX{7o-7;wfG<^rg14a<2qlccSd2#;;*o&O$izV$!YRo4Ff9gR5ENkm zLm0sr?(l#oyx@ymNG=B8cbEqm3TxP46h^}vewdC2c!&nHpcQRs4>lBiXBmqA zkU!&Y=XQC__1%LJWm7Ccy=3 zuommE8wDuFrcgt%7ng8l4#zIaQGseS;t8I@XRe``hUo}EAZB0O!E>~r6|d2O5BLb#`D_~~zyTBC2p8jgEGEDm9`HmkW??oWu^cP#79a2%T^Dk0!5Aj6gcZhM1)>m* zRft7AHX#>#uotIr2Bmm}COk$f-s3xZEix3nAq_bwVB>Pm&lT)5^h{vg@9=Cthjm9% zuA#WSk74`Sw)k1d@jDs!4!w&YC00&Hj6DGqIp74SX zeBlRwOv7{pAP_S!6S3HeZK#JrG5ZL^p@K(ffXo5b`*G%9#{BW(I{O)~(2N$e z;x#_uWx1h{ub|()$+E!>@^{$(7={thhAuSgSYJ3H4o4nxEa4ckYbC^SGKV}tp2R7f z#u=Q&PF_1l?jlRbQk+LF?cKOQeGz-9_uvxsKJqeo1y^wmWyoXrb@B%CsVm6+WC3nc z7m}4^6^f{<$zt*VZc!g158^Ox(>_Wb!7&`i3EW}$N%AiF01xp94QNCY9^(m~;VJ&X zbG*Pyyh1Zt(2Cb+!yCNCJA8r6BepHZVjS#op@HpzOSq0Q+(0=hPz94F`eICZ!gj@O zT*M_@#uZ#e8Lr_5%29!vs6-X2QG+_%$JT%N%)l?!e+T*gIp^dHj=dK4Wh;FtN>GY9 zulX#(JcMHzA`pq?Sb>#@!os(FuDzrG!AYFP8JxvAd_zVZKLbGiz)y5}Zzv=n{l!r9 zL0|NTaVP!aH}*RUQH;YFRxcrjL;feri=|kG2yEzXB!2cX62I^pZ#0aAyrz-3r(-1Q z#u1|cRZMBBobyDiFhO<1ykl4iK$q& z&`5lUHWC@@jKo&_#ecXE&wJt$E+c=3k(k}Y_a%8oVkM#wja7(2*Fqz4yVywF!CjP| zV%e^dxB%m4+=DeSFMPx&e1_W-mJwdq&}JlJaiZNwjQqrMcN&RDsO)0QKgVLsziDhN z20oV%gRqENkz7m;CYO-mSc(WNLnM}C1y&*os}PMCtif8W!+LDMM#Lilo3I%daS4}k z1y^wmWk~L4EK*=Sz*xA!9Ukz6H+6u*>8XT!klXFL=WT?}Lqnm}M-wpeOpkB*a*l!VF1BMhY(C5-#HkuHq3I za6i;oJV3%+#)m~1+XO!IjDSC4~^YJj+Sk$dC7B62*h;hlr!WQqT z|08Fn7>ghThLwyS|)OX0+SWUeKcd6Hs>#!bih{Z;1Ks*w#37b*N z_=#j4nMBr;$>am_K2oS3k*Q<@nNFsmk$MZc6&Yxvok?!PW9lb(if8x-&+!s3@Cwan z!FI-fO}61J-e3pA-jP{kJDE+sCp+)~pYRzu4Bv@P>RseF@&|IM|C0YfTs0OFkc1Sv zp)0zh2YR9}`a=c|m||IpRmL})mgh*{1i!u^Ycc(L9@)Ww>JdfZ3PbQAG=C)40R#O*TSyCoB0 zdd)+PoE2FjuI-c&5W<3FY%tcRqSk9k;xg;;~TwCKNCjhb_!RyS@d;n> z72nW_@A!dV@MrozcO{6TQ$I(&&S}=!gE0fh-0< z4)RcdA_hYVLogJ>F$^Q1f{{?g3-gjQH|4@g=PUx`5iwH z^4VO>K`7?J_^Y`vfk!V3akj68uCt~hK1;+X(2A_T8JxngfdqP(Lc~a$Uz>1pbib# z2V02aTNr1nh48*eLTEQD>7g}8)O#*!lC73+XBB+>SL&3xgHY0!VmHu+^CVv&KZ*p408i9N`}KIEei z*Cj2*E*(o@YhWqH!w!yEJIYdc+FFVd7fVqJcTY><0dGvlw;7h=JBouXMd&I^(SL)b z=ySnR3~Azh9`k+}`NUEjeQGI=q1Q7@k=br3WIr>Wot5Z|e&`Pw%(j;lvSbJu3OVXI zq&%sBK~RJeMq?hYjUmTk9BeTjcCg2MUY|fZU;*_+(h*K@#w1LJ3#MWUT;T?Hc)|l- z@P;2YBN00$T8S)VL&C{QNJ7=!N~l8{I#_~QA1hIZdW;U>z2N~LOv7QEgmfU=0g6z< zjP+Jx7D90-k@0W@@>{G#&vT5A{*Zwz20#vjpah>vD={6*upDcVjRPn{IZiiQ@jD?_ z{N9HZzxQFq?{8S~dk!|1d9=lrC0_(#`VWEOh*L6BC#ARuo8g`4?q<4400xd z5R7PstwIb|V-~~KkZZ9H>#+fG*ob%}U=ubY1*tfLFZha1NO-ef&=b922ospX0Upy> ze|W(g&I_!?Buv5iwbr6C$y%ICu@>J_t;NMOYw;n&T71MOe8v~_-pc+%>@JoM8!_sX zwU~gSOH2nVB^z;1*+w*?&oCR&7mCosZ7mz&plc(Z8rXEvFLJU@84c1~E)?*XKI@^eG7!NzxV*;X)gr`0> z;y@t#*4IV^Vg_a+2*H?z`svIQ5AYC=(11oXAwR%I+}vR!{*SdgfotO4|AqexizZPt zAXr4L2@puazFAfcfj|;O7B|FNacd!HtF=}ZtpPzpSOg3^$PP&m1r=pgc0$w^tx}iM zQ$4LMU~TnuW6{(5@tptv{-6JS{`YzA^W5v}_0IPHU~ z3pYTc!t`oP59JqpNN=AH>4Sb4fDzDw^{5YFgAD}z=0k!Z(b|`!K|0*D_a*mSeaUT4 zU$W54mzctaa9^?ojz##AR9LaWmlVe7lVzKHi3Kc&6=1lzl}kwIaK zFL_?>Oa5%|CEwlgB~KytCto}-Fx|W!M+ox+u z@R7Ad1gFldB@Zgr62ZB(g!0u|q7MeJ;_+J2^9t>NGQub2Pyto2K%Y;N4EQ7&y65xB zw?#P4Sn$blcxxq}n8Px#faS0PR)QCkPoluhmQPkg4(u$}Cv$du@(Q$Wd@>3v-r*DF zRz9iR#U~$r!Y6_we6*L3_VP(wr9LSf=aV8RhYC0gRZt1lPy@9f#kBVkUqZt}K9T(e zZ3OSH`Q+MP`Q$LwpL_~OAOR8~368=sC|KlAzFzE4z5#!RKiOpNPc}maC}6`1f3lS2 zPvoopNfvxl?oS@Ww_r*U5HnZ`8|Dd!o{4~bcv+u(7a<@|;d^)mW?L`~T!P`fFbG)U>?i|D$sxqM!*9*um=Zl1vhX94_FQFfENVAxiSH1 zhV$^GT0p*q@8BuiZWNHv2?2?HDIjrR>l{EhU!gEltURDaR?*{ zkO)c8u{Mx&gET&n#NWZ+MG!xM!|*BWM_mhIE09kD2?hER0?FS#3nZJ$14&CmAlZH) zkh}*wVK?l7y>J`G;XdSF3M9jAfn)>>?gkP=m<9kqsAE7Z;NX~)nJ|Quv5Ryf(0=$6_!Z*X`gk&YW z4WIo@pD+c zL_j2LfQ_&THp3Q(f~}B+c|{|>3)^5jWTWmq#2pX=J7E{(pnf;v9@q=-!v_!t2jLiG zKsMySX}AQJp$)FU8O-AI&fVJEghD25U!gKxWp z$l2rpj!UVY%pCKq9 zRVO4_F#bwNOkgR5KGi3y5LsXeR$vY8umh6s3rYC{A*p~Lo}kUp z{R6)DpYc6G4xEMqhC$>Yd<<wvI#bW@1#EQSQSK`Lp;j+K%W&vDxnbNVrWKr2x?HCL3{;g ztb@o^xB>UzDePdQ%`ndt)VC;`=R1`$a;4N4l5=0!q9kxla{U5;g4+-%11bml2;~F?Sh(tj&Y=v#` zE^LSQUL&XQ+fKsD>J- zg*s@2fTAF>@CE(`gctOj`(QtO2p>T*eEWM4`3_26W0?zr$zS0c_zu=A3??0x!9@H) zFpQ2IKTxS*nvGbfFt~)JecHE1(Pr9gGrt`nB>D5P{NDuVDtwQO#bK%CiDA(2@Mv& zLRbV0Fa{G?0&l?uD1RJ`{w9LSJv{^Rm?|RQz!UfzG#ZJJt5}4Lq9WpGA|kn7B9aeE z_ya7XMP#vDM3#UFn1UHB1#?&dD`6Edfd!VZ6`~;f45wQYmU<2)*i15A<5j(I4 z2XF)@a0VA}h4cv#j`boO>qR6B@*p3IpcqQwz|SJ`&I=J)4W8fy-rxgkU@a`2#{6Gn z{_qt`Ozy)2D6$fhr=eo9G+a!M9}|%YwMQ!^d*J{af=|GF6m5m8&2Yli3O)+VM$6tua zx9}Z&4?n;JOu`gAgCAkXZ80Ga#AMxXViF3GumLu~IwBzl7D-45TS68@NXU&{60+J7 zxs&%u$g;f>!i0+$_CF{g0$2}WaQ~2mJb-^-T;^d3F-ew?Utsr938}F%ATN(e$lsyo zGYPo{f96UE$&-+*atVpFH6Zef5|RPmUzU&`;Ji*k?Cwg46RZjdA;pdcgo(%k%QwTp zA^4ki2q}d%P6p)ih7i&QSKunRZwVnDuo@0H8xZ?XLrBlj5VAKdgv7(gAl8JC=%x^| z4c>+A@E&Zr973Kp_|NM|1Vq9H*a(|oGi-q<*b3WV2gJZm*af@cF+71|nAdUm z3{pS_-=h9+@Etsb@8JiSfJt};Kf-hP2~zQ%pAlccOZYqd0>8prm5St@t54?|Z6k;^EC4`cxs`s1L6yQV1Qvmmk(UJ9Jm)* zM{~mn^K2NQ>@gsEpbu;e8z35jEz0u{xrp--d5Cu40FK}UG}Kdp4$c^M0atK?1*mgJ zT!^?D@g492BMdJBZh}&T& z?12y9eUQS(a0otyL|BgLSHMb8U^o-9pa7KcHtNd|E1?=1K?NFEg>fB-ozM-};5zg| zKWO0w+=2VBpc3m2K@bL8;Rs|x7F2@M-&nJcpm)lP|)^i?J~BGrWWgcf&}|co@lrJU9a-PzrM(dVzI^ z_h&GDtO5A|F&5%L3h}TH_QQwp5gdSn@G%^MA}EGZD2HmOg(hf*D{vFOgGrcz=kOCO zp2g<__TT`H-~`U#3hr?Kbr^X7ELu3JWrUN>#^GcOM8Q^whHdaJY=`&YmPI%j1Cd`i zxmFcUX4K*2gO9P_T?r>H;0oCt;iLeHpb!=?BFLj<5#%v^3tt?;zGxjm-T_b8Z5KiI zz%B@QCxZNSeFXU-HiAsT6g-0;!5}VzlzwJFEKf%eD`0~QctNK!f^sPuuP>DuwNpHq z^Cx60d-aoKi>E1kCY74P;u|Pr#-4g2CXJKA?lmZtElJP|XVOI}T)v@3X4bA3#as}Z z;?Qf@DSK;LFNV2Lp5nrvr;{!7)Q@EvsZu<8=RK4yPtZTWT%=9$;?JLwt!&p%U@o3W z@#~#GD_b?KFJm&Oss4N_J%#0IAZHqLQiZ)#ixjH_gIuPGC{@g-@lx3B2F1)Jv8kcG zH18DKX@e@JsXR4;PnV={Jq=Y%Gga!QUV3zjU4mf?bE!6UD}RAB#i8A>o%z;8>h|6R zNhwa#hP_O4>WQ8Fg^Cmx&v{zrGR}#;y$efI+!E%EF)c(V;`m0I6p!|K_nFIMPwekC z`u9BE@)HO7i*zYop7SS|D^({B_bz&v;*&7{1@mp~iA4V5sT9BV`P0l*6DN-KE}l(U zJ3XJUnAEfsK7*d>?@6VySe&%nSd$MdN(kgnFB&CK;)A%e7 zHNBc|sz{CSq>ETwPI`T>X=&<)1bR4&CrVfI%`~Z-+UZd&yV&&RUbD{BEz|TEmc2Z^ zmA_P%y47<*EXzTa-qySHVd}Po1qWD;+Vl?oTT`jq+ZQCToF>wHdf%E&-7&pD#&V{f z?Bko$PwezuC}+8FP7d{&Tb$UPurQb9Dmsab=)4nq+ZPtI++t7O>Rsl2;)CghRV;V; zNu)=YoQU%@Qn5T#C;!I;-O>~L+l|^;?@XNhx_5cfiI1j@dRd;-Q&0FS6ekXPF4D5R zIH#WWt|&cmC}GhU%Ug76lD|@O;&A(-`z)W>Q_p)>cAhvgz34H^SAOax|83ogM9;+& zEI-w$-+JGEcp^Dr@e9@(?Wr05s;Lvl+7}<6X04q#^}2V}?1|&ki-{$lDyR4}>1ipR z463C+M{d~1v`9OVz+hMkL~@!xizO_&=1BJ1x)CL}V%EWU%|JOViFIn1owO zL>XLvwkEBh-6YB~BsK%Rc6B&MewzFxK(@SKQ;Z%jc|9_<6RtWnz7U{JKrn#1pB8AwW%S*3s zH!Zf@5UU97<9eqzPMcO)Zj>t`{CSdewWpcNa+6B2sgD<(-jrb0V!2tX*y?X5O>b^D zYq#7oq1fJMmy~{C+N{?yiki98-(Hd4>bX>Fxs{W-x6i&b{ZhixG0SLCW}Lr+CcUkF z>3z#>v6=h(96HmlPA`3I`K~}g~tKG3#*?lhFCr75unO1w`S$Y2dO190h+N;Vc=yQ!ec_U$&v(@|BtP+1W>B(E| z%RH?Q=b4xn;#q=`1RV+2T+TUGq68*f3tl~J?^?mN8C%;Uv2)B}oveo_`nv?h1 zEuyUAW3!w4JUUN)HEj`NwNIYi>c3id5?L)`t@f+3+xk{NJozYL`2njBwb>p1?@XQi zx_x)Yp8RHdxyeFD_g8S)#g0# z_mQ4@-oCQk>c~XS*L^-or+%JZ*=v-1vYf3? zYIBzf0;F=I4wk3&sfpZ`{Q*hx#WO6vwVayA5(Fyb#$J{p>kLjFyFaj0z9i8y+*%>Z z;|hctxmky0lyzoooUx-C~`y-|Ib;p?^~C}p4s0Y z+L`grjLl=~a`~Bqg7vx#FE85(>k8GG!~N?YX80u9zOX*4J(DO1o67L(u${K9oH%o= zKWsK*?Tjs9S5cKIf^fRR--|^iwJPY@|lh`qQRcD+hj&>tDC2ub8Rvm3O^62S(kBD90+$!?rbR`hR3QiRQL z`D_)npjxm|p@{I}iP&mRL4E(mQpJWuUN~DLDo_hHX%w3}cv0-8*n;N%O`VD@GrSn~ zIe9^=V6#rK)ypoH-K;8T>)-rPu`SW=0Qm_7!d61Hm?F=Kcu{Q8U!N#?-M?ct^Y{$< z3+bg6Qvzb>St(x5RGU6dvEe|BMb?Q#XNFC`sF)V8lb4m=;cRX*5L;|Cu+uy1)QmIJ zW>8*i9I#80mEq;Wu^Cbon+@!W&dN-5akd%O7B36fEzQdAaPhPmnJ8X4usbR1^o$GN zMoTSW1?*8|<$1Y^Y(_aH?14R{S!WVm!)m{peOcEILyZHZUF2UA%U9c~FWU;Ha^c2?z#o6Kg6 zTIwGVOV6(Ma+llO=9CHtVlA?36Ww!d?ubgo0dc(S`VRMEn=fNaLkHr#vm0mJt8DJd zOCtiLl5Dk?hsx%js&vzUG&;K}(WAv?TwA&|AYPi?+~Lt~^VLM@_JR1M>k@aLV=$>?_T_l(>4#=Ao!8E?~bVyRBpOeVa$IW%~#AcV=ImS^e1N zFY>a30UzqJJG|bRu=!e5c6i{!huK|;@4T@2tF|mL;G?PRo{o2>ZN8Z(J2vpqZ1(jT z+@JlJTAmVcfS%Lm=D!YNN5IAD=8km$*<{hO#fBj6w}XQ;!|-1ghp^6Y_w-Z>*P zo=n^CtpT6vavpg3#M(YrRkRI!`Y`8FqR#=_pR^Sn0Y|2CzV7fzu>E9s~{JHx4bI&ie~^e>5P#%yOrm6HL-G^c;-SaaWY zF1GUdz_HHLf6S~o{@C`Fyz*thXS&lfUTY_8U#lvA8~E(u>AA$UFKqvzt(*xsK6Uza z$J%MzKPM_*4;-I8{pZYD!XdOON}!CMOY!DYITUV{;h@YSS3il*;OL2~Xn`rbT*FSj zIY&RP%4jgfJ9qvppUE-Es4@;rmE_XA{W%;%b(Ps*YIN>`B!6elywR#rtDnia%w7G_kt1fJC8F7pnER5JT;rUYBm7b#Ar4CfpU5t%R3MW#JDxWLAgbqRZ?Is$3$Es4$R=?u{#5cIZNVd zLI*Rv^K54Wt2m|^H4%XdNgmf*sN$HZYc>rkqVw#Mge{z0kHY?_HIkfufK&2u-!doKZa=G>OgUZtU4M~!4E>B#q4lK~*Z|amp zaqZ&jn+FRz^S8`OVz~Ah^{s)0y8Nx)A+cNsb$#1l;luoGNg)Tgj-&M*fkjjK+dD%N zxK5MxJ%dHF`8#GqWL#%jLtkJq{mf49bx1tLZ5SFXwm7ppX&rJ+i5o@(OL%AYcCJHu zsknw)gC*W)KA2rs#dXhUxE)w3ITPm{s^WU68}1F3MxTjK3T@%89&LCKSSCHQzcaL* z`_5#;*MntAXFi$@?d5vX8lMD~E6yDBUa#ePaT}iwmY1G6l(c?~>n(1a46M+cIo!Ga z_iq11fX4gOF`ern~3_Pnlljt3W98T)S-v-YOh_JX@+v~eb|a_Y>n z&ai3j+R4V(gO#&qj?ab>9-pS72&?GI6z^~<&!4L@9ICQVo=6I3@C0HNO<2uSrgw&$ z^8(^jMnl!!%2Tu9OkQAy%2-$Bu2gv@DKeZV5v#evdX2K6Gct-75~p?; zs_#@5%|^!X)@7(&gbg}niT8$BUZ`5_G1Tx-S(dcn0B`;1n}pU=%8Jen3B0gLwck+V ztg>=;gNzqW)A$Qj^nz;djdETDS0fx!SrpVJZOr9GiZx=PnpaTYxv`kHAx;xIr1mao zoZVQ(+nAw=5Nad^YVS=d-X^tX(~u^*pebon3vctNW~;DCTF~6NshziFQnP)iDXHMX z?518`6s>8e@SLKc)qAs+x0Tzpcj#Pc!KI|lW4vf_Q=G6_Q_$AA`95!3T+{xc=FWnv zvzs6D-pyz_C_Jw#=i-s_BxvWt&vZXRmu zEc|LVI>v5aMsut1imvd1_qJHO{p#knp(_syA0=%&VE5r@bBFNiRN>d1+Y;k}?d^6) zCeMF8)Rk29^X&Fsy9C;WC&F$;(M#|5w04Qy3r~l-ON)L8(G6kgL6&3Nyausf!{@Y~R}hedNqJ6_m*HhN)3czvqq zb?1(0yW^7=UJqTLE&6kI2eFsYS|~xi^kRxn4Ank`+hREUZ~3V0Q^hT`pgvx)VONZ~ z{fW31qv1a9;`wtiO#8Hq7UQ6PNiof5C&xZr-C{P}A6>j4d8f1e$)I7% zpBdNcFg(;*{MOvA82hY@R+pe*UGXxX-LdxB>Q;~8;fKY`lXoAm&lzp?3L2RzUfH!f z!T$7QtKaa*Z1JkO-7@=J+C~2$Exm;0vqx^9$Gs>V)>@QUCGW|#&lg`52aWPd*j;;y z?a#zr3>_ZzF0q~4Q)RErxEK+nlaz3M_Nwd))E74m>!M5SlJ~aQ7mi-s8gxTi;?T9X z-M(n@;`ZSiNhMBmdwcDRX_s~e-Bgsg_`I*RFX3L=JAAXW#4Y*#G5b>SrMRG5|Enpe zOMH?)cwv8b^ipEb7gHsET^~%_S596!HvGkG$=bOOh(i_aa!Sw`z0}_)mg-Q=y_`Ng zW>FfD9LsR15ns*-y3H#UcEy@I)W%)T9=`2e8ax-vbg0X?oELOQQY!X|<2cl-FBc5o zi7pLEj&pWs7`gtZ#qgI&rD1b%dZvmWDfM#BJ)Jd;gnBr-#3qE!{B}FLP+2UFi$DPcPf)vrq2O%Dpl)eBYvMck;em zhl}DXqd^aNWqZ5!6+2vtyK-yzfp^&lbNi|sE@xc19rRFA7U#2H<@?B~9ndFbd9d+UkZtzo0c|q65QI0p_Ivhrxc9s{-eH`O>Go!;L_iO-=} z$6M+SkCE>mmX{?TI^g*EXopwu4^!n8U564JznJXs8~I_jymAh`S&q><{evgy71chU z$Q^HUJB1??78SM0pX55;5qFA%CwUe1U7r*?ei_#pIx^{9(Kz=>mE+xv&WPYCNrl?y zu*&hCx^vUWRCGmC^5GW8@zKt$!Ox@>&0UAv9lx6F+&=Owsp7)i;a}e;!^UbV~!8SU2(zBH5F}LpWb(T6xX$XtOIt zx{3~;BNL8atGf=5{PeJ*EBVL^$G?ttB?kXIRngORWZLnY$*yA~KhIWNpZky0(r5d8 z5~xm3xZUX^FD%XuBquPO{wD6u2!6>sJJgk6?(}V3clO9j@3SLw2~4N&GP?7E|1LQ@ z>XXQEdaCX&82Nkj*&E4;&Q9Ntc9#VIB0YPnE78;Ghso}WkzbO|eleHGcbcH}R0sd6 zID6YCN#r!i?WrI6we;+l$w}c(Q{o`n&dG@Qhq!_0kGkRKs zf7hLT;FBEd^jzK3HuC$!vyYOK4>^F1CGN%`` zYkk4f^vWkbN99g0xz~n9rY$PJO+K3I^mp;K(cl?g<g4*^vC42uSe#RDu13k zcD&bVns)t3@GC{-OP|lQPBYx=Pe)#rR{oOw*_hL;`1)kTRo9a5Tl)d7iF!nMgmr%BVZV(|P_>?@FzH za@FEjDST%ty^kd_$gDE})#s<3KUcNt)d`t1gWm5iqAsXr`KHO8jd}e-Ep>Ud z)zP$EXA?=kSVXg{W_PC*J1>#;hiYj))wZwxbDrq`V`-wBsy(_--gjOu9oVlm>Z*R{ z)yc=sD-;64!d-jn}oF{(!p%4u}@9# zD>>7}S}~X>G6|^>`(|)l*qXrtt;x2UkfRySE;hQs648?Qn$YeHPZ!&%!3yn?xLhQmdDrb@6JBn?()vm@YmYq$+aK7%Inri zt;noB=$oT;@#2j<)vhS3J#;i@%*9(WGAUZwRC~BP=e~=NbmX~qWmoNyS2>Sed=(=v zMQ`7zP4qoI;o_$m`Az%wquS)7r(d|N(T&WARz0gd)_r=~W$o0+YwfDJ+T*WIBEN+hFPF}ta*pgTXxHAFhLRwe zQCH%7Ce}4nGwLyF^Qf-u=$Qkq>vf}EV%ukR72RhNT*IbD|9duQ*KoSdU(8uhU+t@u zyGHPI!corh`r4z)T-Qj6PAuly)z^0`i(NNJb)lnNpZdmE$|~253SESl7gDeGEl{~` z(&#pg^0w7C9W7{a-K^7X72CzvH+L7byKb4%Z6CEuuD|fApw~5ueq*QDKC`~nw@~Z4 zm3L$BsC`-erK5#ouF;Ykabkz2`nK-E`>xxhH};P@bk$#dRruKTUB!)qV#gcx9lk{q zuG=*?4v#uMs_!~l^uqN$-Hk-C)3f@X?xJbe9aA@sjXKTMUw>6Z++yfAQ^d{-8v1;T zsct)YH`7O*mp2R?EoQjwlHAM?yVx}hbr+kv?Uvrm9(D0)7B)Tiqp|ZXZnDtQd7mZusI=3EwT2eydvSp4o8Q zw^Za7$GcTO>R#6H<u>7TI|u(aId>G$}L`ct9jI;tKq9xr7>>%6t`N%t8X+s z@GXmV+poFRHoE#z!=s~R2i!i?-Rcm(^Q__P?y>~8kEU+*jJ`A1@Xf0-ncD&S=Y3+& z1&vR9%jIqdd7lrBdMOOxU_K9zN-d)k|c4X@FH))<5e|}Zb>y|+O z;)&Qdv+g;2; zqlzzHir3s|obj!ka66{?;g2nm5k9O{&p(Eo+@+q6VfpwoxhK2{_85H zdzxa*SRx2f(ft0WnVoy8J>5@DjjhxLB&!y`uI9VT>9<*uz)Y2~UyaB;gLj*)3oKJD zIaU+yu8`d3N`y@+v!0qL_e|++2c57>_15c}822p2Z5K(<4b?Kg+F18&&20}|&?D9I zW3>m|b9A@8B*D*AD|>1a+)q#4_R|H=saCzNmAU8A@AykZ3)C#XI=Oou?~YI>TCTP_ zR+sCZFS#R@i0#zup1NZ9GtxVuI zjJcOezKoNEHmN;&8t%K7Nx$5$3++jZyW4f?8_1f2s#G{ISH$@V@K;!SHqIy*G?xyR)mumu! zsTdwLlDip_2s@3iM`i9&E4`boi}29|zg96l>J)eLB#|K+v7egbQLnjMpo`q52|1>A z_Gr-EEs<=9*M#<{Jv|zy?pEkFBx}N6tN9)(`n_t&#!O9wpGM@N=H08;Z7kDlIHn2r z&`9p7C7YTwn|d@+9!=7F&ALrpnk}z2F&^g>_gW>JZ)mpqHN|=~Ywoq_Hb2sAJJxi- zRK&-_zXgadm3^Yu&cwrjK4X_jSHm2 z=qTKaN&G{FQ55RJ^;EuF?Fbk8N3Nro1%_=5c(cg$ z8~(Y-bsIMbapO|%YP0gy_HS~EGtB0h&3m&V`r#X*%njZ&2~*3L|9VdE&D;97Pz=qO zggK89sNA*4#Ja%LfYh5Z1G&PCI=8!1%N$g#*Y(|S`%6Ni777t7L<@bpsx%dPlcr>HEBaicOaR6TAQ zP1^Oe#>jn8H}#W zT5Z9trMTpqNb3v*o?^Mugb390WRZe8WtwD)Qy(qZyMml#5LwL~y;$X)`zDs$`n%@( z>@?FTQN0U8Pyh0G^z!C58`D?~g%Y@2?_AwUh9IqlkeGVC&6Z+zfvLsrh3?AeSXG|E zR*_lNJXR({KxOS`=jj0BwW&FoU_cIdMGu@j#hP7+Dsv7 zjDGX9+!Va)3omNhu=#FTYs!7eK5qR+ivS~~ESkf0H`mYd7><^#a~Qub&E(Fzzf<8d zUc76a%dqM0ESI6B+Z@y`r4|%(>m0^z1LMt-5cT*{u?9XfBCJN0;EL`_>m(1xaL zVOqFD7KP7nXuB(lZ}zws9l|sED5h>6Uw^kl|2U6J2{&cbxD-Y$;rmrvgvY1%-goUq zCb{L|Ty`2(!CDzfuc9m!EjafvIr!n_c@|MkEA&V#A-VkJ0xBU?BX{P;cmcIuDG5Et zP+HJaSQ{h7?zf}kL#XxPODx& z^h?B47J2i@<0fM&+W;kY+M7W_(qzhfj=(f8-$mTUmPI@4g(l(pB##h{ z2g@|JrbHR2CsrAJ?ndPI2*aPs3XJ4ZH}GBbWRV6OnQ72C&+wd~LXSqAWpPy$R&@Nn zmSKKmZCfE-q-neRkY#SPpdMRGd~;g^pV!=W_f2U$I>Z4l@oyU)ztA=w8Df-mHxe~? zv1wbajFq7YOVCQ}@)R=9-#}KauzXDT*bx{=$X3UUq*H`OcXhGNs`2OzMlwCZH871N zCR_^JAUwvtbRnwS*bh>HFO@BU>aX-4oW3Q5CC)*_9N*$^<>!mkW zZRVdyKYWuePjBmY2~aU*;epvM)JU@mX8 zYQut7o4^zCG*();zI=RRe5if6Xan7YIghGmU!c^JW--$ur?Pg2Q@Q3p%##YJ)mq~k zkHhOcsU_7GWItzYn8#-rYb}@tjGO+Fls9e8Q1(jH!h6FE0k=THCCc4;3`x4nOh;g z14a$fMU&jfNG@A&SHz_qjCAE1nMI~D-p-4Z?5fv~cnba%uy{xEtc)&o7AFNRI zJ&SGRvQinT@xZ4RQilgL=+3N2%VuR{{!TlZ{~d3^3ev03Oe-J}A)IP0&Ca>nVk?Uw zkgzn$H@j7HTYh+bTJN1`wn0IDtYyaKNQpy=fr*IAZhdGjEx<{FES;eznrvYR7$I`y zL)Q0g%5mxURDU_CVN{WDL&jkx5!hce;AE_5$q(PdKo3pn zO<}YRTJV{JgBF(TyE*3ch1L9c(K*Iu>bJ!wH^z@pS%Z3XTC0ITAHT8;fxRBX#IiXp zTq0I0OCx#Qs?sQb14=KQ!l>O$-IpFCX9Uz4C?lJVnUQ)DYy%8|xz+Fnsd-G6UYzwk zB}1KYSwO0kh6D#Z!sil3O@5@PdR!UJqEcj$6Is>kBvdkHN*rZ$Xx8tG2a|g^ER8< zVb3ANwBXHG%Kqm8PbQ(^NXxgRp&0?yyT?4|(e6cW<5r&xj}MhMoMkSc?4IX2&x5_0 zVOf_J9Uq&{%rWM&+zl9FYeLuGnS$8~RA#9r$0aGsQXKknO>o>cq_Z?CoUrlhFW>b1 zp2TEv<0;__$-Lqs!oZf#vt6@o^eOQY`fB{xBs$I!tGz7hxMn_DW%Qo-69@Qk7v>Qa4u=Z zuQMp6^maEWu~Nj;`D8P`bb6fuU$S2{ETHbpa#2c+NcBU4=?E4SV?gGSUkDMiF9n83 z>+A9@^t0=71p-_CY`rOX$L>y`X?ZiaQK;Q3{jh5X>^ z+l%6w>#YML?Wzy1+s5_CkJMwT(O<{z=6eq*^`<6z3`24kI~t|_E}>r@4~k%&GnDDY zQ*!T`i`jWHy0d3dq%^cX=*=&}-N;>HZeF;AOF3{eysqAQ+$7$pooFF+FCFKNA=M{M#degmX#L|F55mBi``$$s z_--~zR2k#ZKJ&82qnF?}d~^I_!Qn;Wl9aoVIO1Gc%CgQ7n&3Bo{2|?;MHwB>sUMG& z?oZE+j91qy3!`MJ!aI5*4_2?q{<u$9O?rlEb5+<{<|6c!pP>l;h-qW zImSGqC+pixH6m983~u(kG;@dSAcF+jWoztb)H4?1Ivk@TdFOH?C3b17NbLE}o2gc8 z=fHTw?zT6(e`^{e1gFTBQpTGW7z>mZ{y44&MaG@0k?2XAFQPve3Y)*nq50zGZJ{zo z^oC|N%QQ~F$qzE=S22k5Le9NXk(filvDv(Sj8S8HMc}+UT_)oXm{-Xo={T?0ZFMfD zpB|8*9Cj8kECX4oM2{L?)l4>%sy1vV3V8uN*4iLf#*h)$Qu6@z(*8)hG^K3U&UD67 z5|lE&bSumJlE7I{u3SihxGbffP42nHE6f)D`xWW_ogc^V{p4R)q=jBfv%GB1|F*~` z?UzOW!)4rmc{%E7=DW_!;GY+xEB3v)SloT^(2fIf`}Xbp_@6_c{QD5D5@j?q1?@jy zCDM2Q_X*7D|M6DLr`O;GI$HVX6(c2*7}`>o5`Fz3vcOocT~4t?d5l8zDd_5mwv&MZ zgP2f^JQ;fadNCHn@Ev31x1qd;{!JE!qvMgcF`&fceY?nfuHgan)gD3FNam;Ly;w~# zyrXYI7C#b29DaUH553Q=+J`6qU(dI}Taw6pO0OP+ESRe?G^U6!v{X>@W~m3@G7A3I zME}BL3Zb2NF*#&O4E6b+V$7SRamWG+1=k3sMBf%mYWjE5zvw7NrAlNjBsDLOM!53{b5rMRb20?saBEF*0WM(p%B$JRy zqRvbvW9F5d%$&r`GxPW*o|$AOCns_4w;D{Gnf&hO-t+t2-yipr4YjIj)v8siR;{YN zyY||9Q(z)u@zV9s0xxYNGN!pqtwWa?5X49gsO3eVY6IN^Yy`BX;?mJV08z|rK_VY) zG<39Hqpe*65Ymce5Le1D0w|2O2&F~`qk+8LH_b#S!P`DDx{^K85in-5X-^D9_09X2 zJ#l6LZAwH$I>O;n>V&@ZJ9Imugmq{ktcp~S)KUE zWi>VR|Cud*solqin(x|uZ~;C0Z`!(X%Nm;4vbni+&6iudI8bH~2WY>t@vm(Be~XP@ zdg&e;S9Y%H>1kfOX7!f7?lrS(`mkJoOrH3&Z>bYUzZEd_e>B4{HSepzq34xlp=Skw zm3>@x`)c$9f2?}?q?>=d@ZIe?#DA)K+VsGroBzS9IF7sd9Xga8bMq*DyzsP}FQSQa zZvKr&yO+ojk3Ff}bO~|ts)OJ8=)9Yk(}Ig`{@4VL3vNE09$$0wNmB=|zKi^(C7`(K z<}pnOL3{>m-gwN-|J4iz({7%MV?rFL_o$nH=_3?8;^wO%)*sRS8%!^E#m!SL-IrD8 zMVH*N5#>=g-^MUxOjRd6?B5@UWYzm+388;tr^R#Bw0PueonI2Geiw?VG1LR?q zfqU4^k9P&{of%7&a)WOE7fxZXVnoHqkGpwv2JO0&4fyuYfM`Fh@gTVGb@K`BNPUdY z1mU&UDg)_C^^x7FKIM>`U&X8>!zzyWRoS8uck(zHP9@5RAUgA1awxTqjHv3!DO5eC zx=kih2^q(mW2S)Hjd&O0ogm-p=5=;0x5dqu*$&t}n^ZTXJ!o?TpEBwdZE_bI@Li+Z z_>R#GzFjnf8Uy&Yk%4Sby+^uKQ)E3_S+6SM``mow$)@D*?{xF!UQJz2tEx_Gk!}a0 ztN0GmO??087QTN}gl`{>bIoo(^~n<-ob#p{IG>x(c;xH0k4MqQ3Gkg#jg!Nw^JG$0 z#9>T5_f?qm{#0oJ?BbK%>z~^KWlu#c`1klRck(U%tef9=@&^}xc*xCnuGsbXQ^(MV zUnM1vr;d;-VDv<)2QNe$Am4Bmk*8H7WF}RL4<}`l7gDd{BT65WS5nWDS5;>6x{4Fs zbQgQbYpC?R%7?g!|Dl_&%lZ0^`<{36TMrvVE1p-qNxq(nH-x5Ek{hWS^8M5)^1aj> zU~pCX12^x$1xBiwd=GtmS*0P*KsV1J{Vc*Ws&?`%RVDJ>o+if*_?@Bhv;4D`X0pBp63Xh%Pu-M~_gVrnIbHuMaaA zXjw1`Cb(2(!XqM$QKsnVXrn1Q%4jqJnvBs=(Wa;f9MOUQsHo^bE-gaHMZ`o##>T}* zBqSt8L?oHb$;tEPrKBXM%uks=e|~D(0vwk43D$+_8JU@x8RpE)ESn7}dSvHhXJ#f4 zvWP6sU6Q+W=~A*ZE+;3)XfhfXE;Q1%aTpimr7g(gt#*4}UjBmoWd#mrVNr2Osh#I} zJ8v&6&R^y%EGxuOL;*dDW>fk?kU9hZ6z8H;Mf51QR8&+JR#uc-s;X<27rQH$TUJ?%1+_ z;J*9rd*HqUgF^=n92iER$3q7W3=IwYhlYm^9vB)KIW#&t8aR6rlKA1TaE&0J`^BdS zT!yah-n_15?h@gV1pS%l#AC0k6K}q(HvYd^$G-gPfen3zEem~>nZ0@tet49TLK1p) zjQ1^(R)k7~a0L1z5kGwigFs(!a2p7`@1Hz^phJj4pg#jm2-s31^nRrQAp;>70UKW= z41q(yyAp{J;WEMq7?vW&x2cF4;RM2b1Qmi3L5K1p#A66I5F`kz5rPruPH#0r2tpOY zE(pII@nVE!2xbHYLJ&e6!U2RB1PcPT4agRR0)*!fMiAzsEqtJg;0G0nMLrJkL3HXJ zq#s0j0pcjc^iF9L0;UFufzTfTSBvyC!Ulv6glrlEZ$(^z&3Iv?SexV8dt>&K7SQbuMBHF$|gofzxxb(vq;a8`suYQOB zcYlXVFx5HEpp@?8rz{quW`6;Hi0Aamm#1Fu&Lt}T(LKHbF8DxetSmnGK!Ko{%P@xy zz1aOCzwnq!GWvU^{A9(8_(Jb|Qi`cKdmzD)%)P#Ln)PYtu(*9nQ=A2FPK=lF)dD5U z)NmT9NU(>$m0?k`EanU0DwFrEe!Tar;=CH%GXXe%0 z7K2xFjM>90S=m0uYG_^m@Wi$IUi!_d%ReI?IyZ(qj|e$UoX)G0Kg1Gd@b*@zXh1}+ z{^|ijzE5s08;*ZI`etje^8W9{JTZ`-&yBqFtAS5*vB=nu7zGx=aBp1MX3dB>;lVvj zRVk+t&$JwvvPyoiM?|vE#|uUI`=r^zJ-$>;?@~^4L!2p%U%r38YLPHLcE3?z<&|j% z&O7Esl`(|be*2*#L2S&{BZl~k=l9z0zZxN^?O-)0}f`RYHb+2g@Fw(KH zhjRsc_TT3*D4!CF!ZcE)V2_>aHW=7F#QT6ezHra`Ykb)vjZlWAwpBrUY3O5uahJD< z(~y}krko=O>58os?Ct)hHT!^Y?|-bjef7|W-@NGI!r`-F?c!f*B8hK)hBMAxuNV33 z8K-C6_PG#PMiPHHp!s5>Ts>>VEgQR9*KJw1v1@jB``3!c%3eheFaZ`O#X6i)~CQo82$MCz9Ffjt)F1A8O*6&~1Q*)g}tP~nyt zD(mMqLhxHZuyL}EZj>+!McA|`)R>2{U6RHSX(h51@$zkRyCE_7@!Y5CEo%Q_mnN_~ zvzBgwh^pFEJ}Gu3;2>7GN5#+IG`Gj{0DkLYucQS(f4lKBxL4)DXN}L3y{akfX_yhZ zMc6!f6Pp`S>{fK+w|_f1t$Lcr9!p>^CU$ObXc(18RJWzrV~DYdOw%BLKxL6`#wJ6U z^4rV#$79;bNmZR_k2@KO)OIqcx`EA#8SKjNqEWXjn>1k$q#1qis|-s2-5s29atS}! z&tjk8Vf^%8f=!4A@gw~(ez@;eapWPCH6iT6#>X~pf%8NYiWFOgxeTVf3dsXeCt?uMr(f!Md+sVpQo-~6~2U_B> zJFx=&xeqOOfMX}P@6=X8mPRPU0se%NOMKrK{ zosX2I-X!JFLm8x4fRVDJza`)u4+--TN?jhZ6mpaUpNGNHfonRX-IsbBBYd9JroKl; zQnSg!*jt%$nTa*9-DIGaw5ARcav0k8$Wcgs9=k9nR8M1b%10hWkItpy6Xj^(2xK0A zNJ=Ka{c&t;)hs{oCV3151F27AsPHaf4Cz%6A40{;fjy(EX!xPWjml{aJaknSCtv8jZm|J4Xoqd^jB%I6W}pH~*$*gr7PY#rvNXZo3nU^i75l-YHZies zI{urF4+h&m%VmVOIvB}P!N-ZHvYnCK$@@&hC`65)ynm+%x!-^L5!z}u!q<^_KAsU) z-eMTpPPKkfEF=E(h5JxKTQf6?J9qB9@PUpIzP}K{1~cM2r*4P<&+TIv^(R};Zd`X@ zY9}b?A)o`FmZ7X4{rbr$!-#wL7cilJja&GXvvA#G-%AW8lAo6RTEj@1cD==ez8$n^ zLVV|)EJkwtojh#b^w%N#Su7(N`*D8+qkezezy0k$w>45xIuIfdHX~4RJdET=A8dS@ z_9KH)-+AHio&6QP_tR_^LR|ojet+zT9C!Nl0DoIbfr$}+^5S|Z;f24oF|s=+zjj-P z9^V1M+kZ2lmKT9)4RqW8GXhHc^E{kM%^eCImtiGfiS?Ru`fVH?{1cy~U=fwl`` zlnC#5Xe%%l7>TqWn=jaX5eUUXmX%1PGHkxch1h-x3JOxG)qoncTpO$i2@xq2x=@8) zCk$hPf(+q7T18MqXrxwaR4BAK=3pQd6{HPQgarj@P1+!Bw3Z9iMk}9hz^1* zVPT<3h#(YdL~COeu@SMc;bF1i;jyu?p%fq+9UB{=3j)MpHk$!Kgk)HlIaC)wP>>v< z*UeMJYGY&d^XA3IYJ-CG+St(8d9h%xkI)BcBjy2#dRl!@EQ*36B4{c|7om^UYW1-U z69Fwm#A=yXC^=%Do&f_RD4|zvEJGQ`#)6)L(0M78uQ4SkC?y4DK&9wto1rMyp&TGI z1*Ne;%zOwFn-Yfpp_!ntu-Met2)!b8UYcH?rbkW434+kr9cI+m2bmd0%S4ARP%JQ7 ztXAv7XrslPo^Hmr*TRJh5)%^_$QLY-%gyGD%&Y~WL8(DmK?~%eC{4~8;@f4u>NpB_*OjQJ~1r56_QKq=XklD8eK1Qyh+Y3Y0H%IKxx8a7Q=}r_-Sb z&(C)h=FdYhS5#c=a4;%{O;NCgB`J>53{{Dv%$1VjU=WJz;BB{4=+J;1{~DJdBgI~x zf{$$BPcvVo(r^l{T&1eu94d}W!Jo@fk)m=Tqi}GJ3XXF)6n0Jp7A^-@;c!)m^}q>S zXhgt)Lg5fo{z%bacC3|Fa*nDh2j?oU5LZ+<6bg=GtE;(cP*zu0&#$g_R4;J_Gc~~q zhiiE?=Wu1r8dt)RAEQD2c+UQtm&u??%hP0+9^rM#k^<^zBxGVAN>>*@d$D;(t& zY^I`Kqfyk?Q*?cOZ9OO|5Y1)-xcYk6TwGu8peaYaLoX5h zUS2(NdG)Z=yx=@xAV$)d*FT@4P@I?N3cvsja)tG-yvEt0M#N2Z_PokGHm}a4$;+$L zO!zmM%qA10HaY4XCVM0Dd5w*xJX7UrQ-(=X zm|=IhShh9;XoQSfx}A+E&%~NC3N?0`tIfzTIUIR~d4;to&!9Guw>rZVS*0+sRr<)P zT8Bel<#ALoQhikwBa!GMBO@#(OI1%LO%mVZXf=DnZOe`&2bUXE7${R@NH8uzquZ+o9pqvGdn{DVxEdsxo0x)=7N zJ709>-jaLMmb)4Km(nWg9Bq29*Wy(`agNr=NNHqSOKTmp2lB{Bg_kO<)j@HskzR+_ zV)07XFjSG0MA5od2)H)T29?IEh*UIsTO;dQ8(SOOEKFS+>yC_cxPu*bsl(B(Z+AF? zp|N(myRFW#VujS<7U)@_fL*sjv0??=rq>Gv?d#XC-{1uiqiFBw>`-)dcDAo8FW*?# zT~X1wvAeUQv~yGG=8c^!i+c@h=jL^7Y(-s#yRM6MyIFT>>873@mKJtaw70WWimJ}4 zs?N<@w%%7nzeF(+9cMnBxuc^1ct z^=*pv>pP2}{`Kp32H@vFEG{Ywpm&xl%FD|;5z05>f9Fn`D&M`kyc0Ec@7}$>yd3T9 z*}b#8LQ$~`-RNX1#CxzH@2c2?vy1rt?b~tpr=w$c#r9o$_f)i1>@C_;w61g4o{o-P zAm6nY^c5MMo%{Ax?Ag=V(UGah#99v9*t25Aip~}Huh@=1dhF>e>Uv-UyJGwH?f2im zZ(m18yP{njh&Qww6oyXmE^%M4qIbjYUa{D)%fRk}F|_ZxPkf)GT_W+)e}z|!SmGsK zFDqd;NxBrQ*Sl6CUhD1M)Y$7?+bB^`!rtCZpz7`2v8i{50_1Dgc6DJa8XK#zh*wu5 z!dbFfEb%rHyStfij7^> z;UEb|E4Uj|-OfYY)aHFwNCGj>rs8UImJ?O&So3%Ll5v)F=%e8i`(TYf6 zR5Udy6up@;mC>l`ZMEXYj>u@Vc61nh;kbW-MEHIYK1hJ(RGF&Y-u=dTig`w%P?*^w z8IUYn79PH=z7Hp%aASCQX4ZlF(yFS`Wu;cNI!hw4W@YW)pM{zdiP5-!f0ePS%2nku zA_CiG%V@$iNLoz~8jbkJJxv6&*($8EqW6@lS8X;Lp%0@`U4?{oKrmn&uv$T5-Hq5f zFo64bL&J!H9vB*^7npm2gSP91LP@V_-n{xSp`bpjsvi9TYhk_cp@+f*p@OjburMp+ zstT*B>g|2d@ApHvurMk_*gR_(3aJJl2o(nb7=IX`)o-QgdCUA^eam)Rt>D>e9Q6Cc zBCP|Fkpt>Te;9b7u6)T7xmtek;DFqis7_2oQ;^U&U^Ld(CnnZ_K$obiLDJuZ%K+PI zwM{15y}LjXAsHE|k<|F*%ma!tx|xYMV%ej6ctoXR9fQ;PSXUNutdb5it@GVT%~)kFbFYXH(lE`oP>)V6&;! zHg$oGaxCET@{o{FY+fMW*l4o>)6tPfOB)+II%;b2)FTD(F-L)I1R;;3ZF4k>dbCy_ zNYU=0Lns82;|lWgxsDDF#rd{^eA=Aa@2AB6Jij_Xaj3vsV_UY&=C>{5IMnmjEc5%h zDpc|o6y(_sK_F058T$NHRgj1tRcOWrK{`6BP+m}A^XFCh^Yc48@}u&js;c~cplr7M z00enJHV6^uSYH7;57`iGwxv))D(6>Mr-ny+9H$>x(NZfwp!emp)s_H=waO4CoLS0SPTRK>@uR5@pI;!ms6PsdN6 zNuNG`{P;8oPa`)SkE(uuy1(x@1e`oOeR_I&Iz1j!)}P+o+Ba5{isEWki~&&XZO!xPlKR+EFP77%X{^{Se3OO3)j&sxJ0;%-7 zi0P9jKTp!c9PMm*&h**ooO9=3|`!AfhptzvAAW>Z$oxZ40 zT)g<$MTP(3?BNe2nl4?ubSZ%FBfjK6dD4IBqThe{@}j z8M@+EjQKIn{+uC*aOKpME8~}QvQYs*aT(Dm{^Ute4)OezD~c=A{B+WEQc`TJ|M+qL z<2loQKP&_Qy~Zgnns z`geQ?m51;@1O`J%NlC}6VNw1hfAz_ehZPvK$?D1Khx|F`&OVv*wXZe9%ugzECXauu z`D-x9!;`7UC;k2$l%%F&yi;=~rxnwwlar}eujWi&oq(sppWeq!n7h6!GVi+9dp%dK zcIp4znt0@>Ch?O48hEb!Ki>rY(pxmWq4$-AngVVsxf%4{C;erO8=>$~kuk@qlL)d$y;uuoW$$oz* z3l5B?HH28T8V0+n#G`@BfS%LyAz{3Rbi?5Byarpq6CP$O=n27dHAK1z69aE^;H`x9 zu-dgm(=4zACJLh!qJB5#K~xCQd9+M3(Rd^tq6r~F*3+i(Fl;*bg)kaCbZB8b#6p{A zc*3%6ZA63TY6yqtOQEVD`ok31e|fnd6A8hD{y1+`X0H5pI``UsaSx`Ko; zA(n*;V~-^9j7&qc8jXZ>W6u)Whg#85^jWhFqXC`cp?$JOq6sJ*c`XQthwo;(B{o8O z2wbwX1jPY05I*3y1~x?5%y7YBA=kCqsHis9qirGDX842|_3%nF@(`SZtVTf(9h7q-PFbcQZ?YYB@N69`T%e0s3xY}p+aj&p&*P_)G$`sC7RN*Ob8sj_$f-21nD8Hu<{(za*zq1kl@`6 zun)Z)b>I|jEGtzOE#e-i6Lwl6pjOI>9_<&5YonzZOCgR z)*cjzNFh22Mz||NV=Os3ZDD~NW()+Q(J`g)<74Bpn zID(>O4cXX2F9ZmAifFCrLOMg5ojhG%Jw*Jp0bQ~Bq$ETeNoCUV=!$}9(uwG52=&Wk z2nx}WEY^cNRk%CDuwu>hs);)zO zp|S4W15rXS!vb<+0n#!X^zB5WU|SR|EL6j`VP5kZk5$N!D&J#xxDNF#O!HX{gKE+} zTIv@|^%_tzMA7O$QxKq*Fx+d6I6y2?a168!tjjEUB?&aq zUcHLEN2Bl{vIHulL5r0|TYADYw*=&%<{`|ofD!EmuSZwG2z{|wc@3;fL(m-UqekBf zJ0f^11J4UZJWf~5Xdg#l>43&pSm=^WsKXmh9pJ-O!ZPSot6;Pu9)m&9FKWJR&9GvN z1)X3#Ah2LctX2JFnGvFGW}DFum;~)4)_sfzJrzJ?A+5m&ghK&=6dGa=oP&@f ziLXs*zi?fY(jOdOI4(YD(fcilKl%?%;>>N0@t?2izw|eSF4S=MQBiq39hGVdIQ&TX zi9Gae`9J+gV5$uahw_W!4=>C*Fn;Bs@|B0mf5StCnUWA8^FCHu=C3e$<{4bxGI*DW zka>`%&d32;5zYz$zC$TyDE=*){VmPX=f)qQiHj8MM)-tsJWq4)Q}74MY!$-CLX!CO zWlWm8?-WqcN%K#RlHXD2Zz>XxKcY>1@3b~Bb7oEbOXtkZ(9qZ5D*4u|tK@s~Hv`Xc zetXXA@muiNqrQ`0pL1}0an399Yje+Vz6n3bz_CImmCnGmG0(i@d!Ij&DhoJ>;iYlk zRB4oWvUC%(jz(~Hod09@tPkdE@RVdW;jzGW@?&_P&8V`?H|N|Me*|``iwe;s9xPoV z|J|H-X7mbKoybxvGjrdW^XL2+K5sWw-P}uaew;rClNaWED}M?mN0r@p0FgN;GQEN( z{s>pU*Hu%}Kfp^eOr(1a{&%-jQwWvw-pe124Vnh|j{ia2$Ss zXW;KQj`VbDFS(4Co`A35c}Q~JH3HAQDfkK2iGI7HcnS`JKKjIK;3>=@6r4((!gHe^ zlSkkN=}+~LPgPgRIkY?oXTpcT{)QU3jhe{=@H5yx9$aR}MZNCgH_1+ndg6^#jVJ>|n|UwK7- z;rApi!10q0I94W-%jm|bLMB%-GKC5kmr{j9B^1deQXwOkNn~=lOfH*)fs{moeHCyJ zOJowbTuOv;p;%is0FynlWJM5M9T`qVv$xQX0;+MtCEYwT2`Z$YSh8% z5Vb%6C%zzoKp&i#3^|qSW!y z1TLCmHHk?{=A?K$Co4~mj+!T*XO4?Y;kfzcl;n8;^SJ0ZE@fVnIBH&8O3J)>VwQ~) zt2y=jd90X?i;iQ((GpI~ilR7lh|`Kicv9Dt0I|5}R8g8TRVi5ztyC(b<;n#X$P+C| zHJDPBnt7>8rCcquNUSo8#cCDJQ>(3ND9K8Y!Zvv7_EsjMQcr_xEO6_Xr>z7%*sr#Wpk$fxEyg-W>!#W zOpYNe68J@iO#F<<$cWVCsuSv+$EZ%;1hH z+-e*Jj)DRPz#(@!oze6uVh0j9_`5OB7Xpof*~pn!h@>+zBg2_j5Wp5j88R}W38HI&mv_^DEVTrt?q_ni8#93NaA}@_DbB2}5<*xEl`d1O+3UlEoD+eekak-)l z&ami$kb;7!N<%>bu(ZHc?xX;BexnPV7J)K4%oP?*fh$blbcNwxSy>gt`!y5(~Biqg`$(z>z+r*e5+U0rE*cDB9N zUMs7ruC=dPwW?-S>8jes#!`gF+NS!N)s3E7ewD1YR*pxwn*yN$u$FJCZI(BKq}I8* zrna`R)*fQF)AHJe=H|vGSCiA;RMS+uY85ag7?b6zT(rY4X9``Q2u!WB$U23 zKuQy}_LdfLWD9NEN&n^Y(h_-7Lv4%8)g*7N50KPAo`#whyQ`^5?y|c|pcGdDw1VL% zciGpJH(?0cj>*@qTi;qAS=d^?-nl-qwWPH*vc&Fm+9U0D zXQZ<*(&>z}JL?OZob^pjPNoS5)0CU*%*{n_azT+>xK>`DyJ|y2?kampLqkJ-NK<)J z33}|t)NRUbXd;TH277(I-My;A*^%4PVd|)_?@$tZLsL_I;i?XIZf6JDb2^1~dxOwk z-{H*d6xusY1)ZQp+?i{KuT(*Uu(P3~AySB$W5W5Pz*GPcogMa$+y-YugFx6IXmCbC zpn_aMLxH`cpu^qK(c$czWg--IbaXC+=Zn0fL0;Iw|!Lnyp6lKKul%6iHM5w!(E)>i;mw#0n>oNcVeJ&F|6o9RfDO)+0hZ%0G&59xLoosR~HgZPK--~ ztIMU>=;|`Tt5M$MT7zMd7j$%Dwsg3f0ObH4RB?Ik9&Yz+s>$PVc}&1}2lAUX1)|L! zj|su)Sw*v(J$TQ8qNfw7&6_)VdOA#TU9^p3<$}o|RtLYS&h|r_{T3HPTyGN3LGEa>a_Rt5^4K-MY13cFLW7PS48OBXaM`mHjJ4+qcW*ppo}SW}+OiSKeRW zFW=dV7#^YCemrB)zcWxGzYi@@lDWTn?@n)dd2hK$)Z4qFw>K#XZg8I7-jzK)=ql2? z`}%qyP2b9W@_nKmJ687X*oPxC37v4t>pe6rUs?6Q0}VTN?C|Z_;dQTkU`HmX`}Xy@ zJrDLJ30FSo7S>=z=mY&e0JnFaXQk8Y?nBii%4?;3pFC;b4*3oQG_r4H($1ZIE1hyM z+_A%5R_5Lzmshz`{45B%Z1exDPJBaT@E7AV95bDr>h>S z87dJ~4GubostV-QgVluvgG7{8K!8i@a^p=?N@Zd7U>X@5EUc!w8xoPUf`Xxfv^1p% z6U^!CcOt73rKJ_fAeS?h2jLY4ysz z+xHW)e^9sG>-FL_i46^iaSSTGR-dH7s@%R&DcSC|!bcP?q+RaH?M`Rc_U$_KXgiYI z@iLm#Udie-N!sAxz(7}}+v{AveqbQ25PpwJGLW`@{rbVx!NbFa$~5chp~2zd0Td1l z=mv&23~r#g;o-qFB_d^7+Ij@tKw4TF*MWCheiY!F;>D*ck(&jkb-&gT=*2Ih})kpHnwf9NcBk4@UW*v&-#tnl^%9aIkpD zj<-8}Cel%!RO?J;hKtzxUhBmUw1p(;01 z#aC4oj5HV*L> z?)(vV{>BmG#?a7>8;3S-bm!;0;TScv(Z6vpbfnNf2!T3{o#-lL3l8QBg99C-oj3Xm z2S*AAjsB5BqtQP!BpEVF{1T_ce{je+gdd$yE#N`%&`8&i#68qCGLkyTvLoWbkwXv^ z-mkz99l~K65{rR9G-zYl!I8nPuC9?Tc7(~N_q+~?yYTeSp+Rwe*GPIssym}Ay^Dw+ z5_SzP%gPc8ySfeu51~R%SC}}+9>7(&nBHjYUp^P4hiK9t?3v>%c@laf3Rvi2d)6?S? z#4T8mj>q0c-A6~qvuN5Jn>FrUAdVE9vks3&MU9S5n8(CpVsD1_+(%{^q5=J} zbb0#ZM7lZsuy_oOkB%NcdL(voG+mq?pFTdGzVwJVJrU2~EgfAdHXqH3G;!*!m|LEx0 zC|oU5Qi_t09-Z)yBAWJ(`i{B%DaVU+OAi&vM?+@&a%MV2J~159U7g7{Au9mBh?mBwm!Z^D`J)Sgr{`_IJ`tXIr5cI;~sKfp< zNuWM*=E!7%JR#+9iu^*t;WLLXBmkF?aCrJ~LPCg~4LLk^VOowXG6@&h5cWbsLc-`M z!f}wG`~uC%&m<%qf#j$%nQ|CeOEOt`LgtyNqNPWR5;8A> zk35QvS~{MvfSVi%G?5@ZpI{3aTWkxFN~L4x7uY68ijJlnMFz)tY2?w!qeqS$Ns&s6 zq>+&<8#Ng{Djf$n5;-azee8%dM4DiW0%=HybTTTMiyohpsxQeejYU6p=~Bq0ODr5Y zkI+B$(MwU2m!hJPzZ6A*8ZM%N;^;}5rAG)#qodWACZnU&XoI4_GkS9T(xuUd&C-x@ z+WP2t^kl~QqvzpPdh%q(!w;W7|M0`Oi}}{!;CtLAn|n6g>o$3Rmws?f;>@$!#G8+6 z;WqiN+rfcn=l>P`hsk1Q`PM z0qH$vI84$z`?GKIqPKBX0bJabpt)ZpN0&<`Ax}h6w8)|%nMXwkz9walq${@l)D3EmD&xnaf08+?+ z(jzM&vQmSTF9+hG_r6N>z=vM3I^tt!8|Vf-Afq(F&;qxqeMHIIp%Dn?gA>;Xfnd-u zJ`~nN_3t6sQ-r*lgMQK@q|7pdK|te`0x2VfN?Ah)h7J78ShnPrDV@j1h^RINqy}Fi z8p*#Q+e2D-=`H#iz!{99=9U2(hAaeG&#;Wj&+y=FnE}879k2#)nFCx9aCE%r8w=5C zLT=GmPZHRDUSbiG9;zCkU!)@kd-7SKUQ~w$Aee`?#ZxzQPdOBfR<9W&W3xX&Un2&EPue=-hH0^(Z$VbDf0DFt=!>`Ew3c#7C)cWCKlKt!Q(4c%Bq)k+6= zM2In+XvEX$Xei)eS=CD zgql1F7+?A=pT15SgQdYb{dKk^{jveG>smMwC^+)QAa33T3tG;??b8widU6Jn4erkP zNEheRD52qGpz~3Hj=;W@n7TsB7v_2kF%Fs~UfLyoCWjgWJlKG{?rWx<3zQO{>=jaj z>1=Ev-0YHp#gSosfl?3Ex&n&CI2af@Nd?rP?6mhdBFF*(RwU+D)bHqbp4gvq*T=&g z!zB+{o%7C90vt-Sn6xMt(vDkRq+IVdq9ngFc>Gq@*AgVSuIV8GOJ4Ll1M%nl*5{pJ zB*yL;>wefFtZ!E5!!pduVMaq1qY@pNz#2n>fX1l-2UYE?#VZ9f1h1-_nSJvoQgmg; z(LKXp)`NK<7nuJsFgHRK{5yhq`8&FRU!%I2vlsg6P$O-BTmZFDGo#ajS~|}lA?M+k z;}aMI4uOFNi6wDdARyNRLW7ZD26lsOLd-&Hkyaw2s~zKG%^a+Si4l3qED1+655j0P zy!6^W%?y($YAFK)5GMv05D|=w^YK=WbSp^=tPiI(%8?2y!D$k2-t$n+`sk|ELh&9t zH?d;Q7(jRpcp@J65GX`gf;5c#MX;y+!Fx&#!V>A!J`XQtC>O>c6rh^|YUV<`i4EPe zUL%k|3|3gygmhs?U5u!bNKivM<0Bf&OH`l)a3(V%4H#fFEF9KFCV{4JjN|k`7$UbM ziZH2WfU*Q$LQ6O#`&qiYqB&@Xc!0pir0Dko<(M?cQDDZ1(N$ceq2~pj+~72%lEWZW z;ia*GMGj|Mf(Zv5V{{;!g~|w4Fo?6XH)nl}FL_Ew1TY*QRTg0_WI_Nr(8C;@&g5hU zuJ~r6tK?BpI4Hxw1qM2x)IgU|fR2zU@WX30X#?Y%X&YE9A><%bM3p*@k68suX8;2i z0v1@0kC{0F?SM12EvgQ9#nOD>TmhU9<2^-J#TnqFFu#DJ1Iofl7X5%Y9u(nqYk{*b z&cm&QWX2@4l;HzYboDiO_=SNcreIk`ARP|>F7f0G=z@z<;jrwxfmPRsUdQ0XOgN?n zRAeL=O7PGWR%{y40ivygSzsEV0IPFAsnmCvo(*S9X$=UJfJx04;mk2we2rcbuty;L zwS5BEhKLjiKta$eTueec;6u8pgn;3#Kc2*4RKtjEn@8pM0kfU zUCRRf{4K7Xa@cXu)3eL$84_zlV4SG}JTt|Vj>Ab86{s0u9SXbz7>gF=CL@)@F}`@r z7?Drpv(Aw2GVWb^EedNcvx;$DforjrLF+Br`)kyEr>M2Cq#L@V0-dEmY{cwK-Y24` z%$3)O2TGYe?I1NnMuQ2r2UIBLaRmn1C5E0?p$tzDT}@ziJ`~+T2@cTVptu%_Ll$z* z;9(hRsDU8>D2BDdr{3VXA97+uFjdMdiFDv+0(5hY)8*A8psS{zxj_p!gN6sOkf)6u3rw2NeS7Jo z`BLBBpS&KNIQ9DM%=y3Z?KOsScm3gmGU#&AvZa|uTeoz>Cte>KF+cRp!h1d>jbeik z03-@CC494xVcskxi6`H=M=Wvc+!t@&EY$lM{TP$@^4r0Q18+kde2n^Y9$z|(b3^&N z5}CFr(89un8EKZZ97~$j@4GV8q-<28| z?Mt`-4>wSD-Slf(_%_)d5q0*)$J7`h8nBn(sY_pHQbYgVt5cuJ;(R)yzAzlG{o1dE z@gUO^;SiM?b+urOUk zC|rtDzxZSLc+sz*DOQ?2EJbVaAvKxlZ3yakN=U^0%nbppx(HO`0wR$~RrnbTod}s+ zaI;*4I{(HzCnUiPu54&0WFcmW_7kdQeK!#A(=Ou&jL?qjSSE4cy?cyCzOH+9b8G8n zd@~7e^M&R7M56zNByr%~;KWO}0+#bXAK5Rpq?t3K(A&50efAp#frrSaMg?Xvz4wDh zn=N4^ZJ$gk3V%Q*rJuVIr93Fph;|+VYU~(*|6Q3>nf+$)8HN}A_TS*Q{`HH+VvY2( zYv;sLQSl8RPIbKn#A@}=fjFxJBKYuoVvR_11Qmy?lct$U(UUbmKe1>CbUU8>1L*W$ z2O{#47;7U8d{&irH{$9P<9H_SDeN?;l0h;)r_>ptWxG1y~ zEQY}%sZ2&)qow4I3U9po)yrVz{Wa?D+;BszQCjYtKhG42M*a>==b~Rg*t3g~bY1*2 zWSd^`A!NJniI4Ch`^M*=yTD|No}EGI{GN~C!+#pgr?#M3!B#X?sgf$sd=hpMG=@(h z#+vzx$6)xqXmr7M^1o- zCy=kFTAPN#q>n!IBsko+?s996)Xiz;6=( zUQYyAF%RHO3Rnr_-WFFX)wc^C$AcS3{{!d?e+Jl(ev5wm83@{EFfLLqu@Rh&%YbNp zX&<`t*vlaJ?1um|H=(lsf&zB6T0hQ9~! z5N%@UO=$0N#lEZ1-gz|;*HKaWf#4cgJ=xRt1jC8W?f}MIvlb%1UWeqaH}@m?Hk}$_ z-R~o*--NbKp{)~}A^yr<`4e@+ph(Nv2ntbH5Ubp8B&L@Wa7o zz>AF?=!tJ5$ZNX+q|;#d7cg9R7S{S_B>r>}oXZ~r`VHhRUq;UIBp8>)^@=@sd9ze$ z3jWpCm=V#pO{o7I22I-k-9KT(m!YS4azTnONJ&NWUXvL}D-y{+1cL+6y6EChQ2w7l zDSwKl_8{?#-=XZ^{tC6!*6Y5(42Ztf0=iwTU^$R9A4YTAv;fi$QJXkHZ6Yxj;NPnO z(&<$E9=a~I#W}>;pe#i1o;&ykW=#IG&ZqDkd((G7^4pgH4qpS^t?!qKIce#KuYu!F z?f^SZr=H9DLx^s*L9{wZ^mYy;VwRxvrKJF!)Px4#ct@PAJo^pHHyNL3*Hb`SC}-Et zP?mD@&lv6NST#hiP+JoJ0Z6y-FKFq#?oS}gFaI9$G*c;Ez~~^}hJDw8dO0WVTZ}->jL#65JR8-ooUkZwE{MGXeV-fA87Ojk2gE_qN_w87wehCT7{tO9UfQ^c7 zegyFI{{qc3&%A@hMf5!KN2#TM`xPM1efu|H7^ET9hqX?$1CooD()F^K>SaO7AHdRw{^KDQsYrvhOk}J=!5d|_#b(j6#XwmmwIuw<- zk(?=wKr)PCx0Z59en@HOmnI^4lqP>rIv>e0O7QElg=p?CfAc`83vO-)EWfiK@EP_f z;AzD>fUmpXg({y_Zha1S60Uq>2T*S)_W^RwV#5|b>%m~=k+Zw0I_;EjVn!nB5kp9`D2)1|~d1DN7?{Z8TE(ND&ydY%~<)s})TzYO40s7#be zR}P;b5o_Y_4J*+6AH#utfiAgYkpLm_kmgNH**~HiKTg6^V(%EdVqUs;-(e76cmM;r z_}vvRFay#>{1`CVcJOm7yNza_ei37>Y+1BuhA~S&TDuda`8YF*e!UssxgOAEwI0CS z=<7s&s0&~bU35Hj(TT?@CQ97~uodf;wDI=CXtU&hyn?}z6ul2a4}nPG2rh)>2wM=2 zBRq<51>q@#ml1x3fU~PeTP$l25UvHH2Vpw`PLp^_Ns7n6W(j|R`1c4Opn|wW<{|Tt zsYbx5lAf^fbd(g2c}dC7K_DoV@gy0UA%v4a;B3y}+k#R&;1#%D2(tIOTAlin2#hnTw zdSWGBp5v+q2{o~@29Us5@WdF*Vinfb2aw2Y1hx^_y~s2)0ecgPBdcMK&v?+nie}nO z3kaZass~Z&eW=ldMB_Fj#*wh?MFOk3DB*2X`_(T|Ly84}yn|XF!8+5(2afkdSH!@7 zTL#dt0Ju$!XI>OQ=`IZ5ON}3jvqisOk9_t9fIm_j_#ayVentoRL7b+fZU6BRmgy(9 z!)nOi(OkzKoGjB8{2eF1w4aa{n?II@_J_a#wrZmRxlfY;pGZrCS=DEt`ki_nlVq6} z$=Z5cB#M6C2+l8|m(rV=9xNw6qNaKFk)1Hjx1QdIYK`9pLYaIT7fkhNZ%r=x`92a$ zmY|;_bhiEOA}C_sxd_>h+(N$NUD*4}(W39*q+S*SL}M()CICET2KcuWfaX+GdS_8M zDn%9|c_?Bfn*7GI>(S)t=O}~3yq6e{XjvN4;gW4H;}=b_8iZ@dUx3)UN$_a-2EZTa zndGP61bFEs6wlMlV8M-~eXaTMm!Qo2FEH3}E9@08)4m7vL_#TAdVk$+w6q9nmOi=t zI1mRaP62VU0b}+Gy60U5z4}9*5OdV{rLG@-7rcIo^RM!$cjK-x-O@X;Z(-urG-X|9 zG}6b+r5F!-65x`dmWA`c)^~Fy=B zx_Uu$gJ~DVKL+v@-v?-a6=3^Ys3`4Ph=u1BMHz@&A72A0Phc&Rrj_^O^3xC!{#~3H zAKX>|yzYMBU%pWbQI$VH^8aG%%fqRRy1vh0<37hRWtL=~$D)wRkTD5SXhM{vNh+y? zD09Y44J1WT5+zNhib^S(XG%(gG->{R`#jJ4Uf*|J-yiqS-fOQtueJ8td*Aor+yHl; z6ttkE8XB(_QurdiD6SmSN7kQy}`~R=dgkje`+FTV5@?UYhnEik!L2R*$;ptKaldpY&!HV{LmZ}A$bf0;iIroWFaJ;@iABon;|Mf zJ*UnWOTm#r`!qw9?m5h#=1B7Hu{Ce1H8PH(`l!sS%@CHR6xxU-mPE936v(_w$nxMS z;4c-a#k3cykVGO%ZO^#d0Hy~8;zMt81QWqjM;HmMAs1Gn(&m(j=!bK10$Sl+iOF8t z#Oh{Ol2^^h#erzC#v#|c4&*9`6@y&&Z$mDvN~jNY0+_)jss>aLnSZi35DcUfDP#yJ zoCu;sbuXl(>?DGFs&NKEL?Fsc*wN<*_RxXkRu3S!o-()}d57R>%mfoyK8oO`YBVV1 zsP`@Qeco4x^d2@tJgH2V%tX8BaxOdv>R^PiIE|)s8f9W-d>aA#$~%(o(#v*XEYsXD zWb1Y=K<{j!*i23JDs)$(x(?){yvsm4-d*q;iW%Ht+cFA=uuw7%nGhN#?icleS=VcT z*9@r-Jk5!97u$(SYGmJ(NunT!5+|f4uv36xbvIE<8ahH=o&lU~LL=!;gE@yr!VMtz zTOfEB=w$hI0FK`Tcsxd7jZxh~fxM?JqW9QYjN-CrOt2yNH-&-!ND3H2C&XhEk5MiD z=>-UbWnjdDcY=6jIpVApi2E*i0ViOY{l@!PM*^vykB6SKoG?83DD9L#+|s33oDF)c@5TD<nEI{^;a1EGg_DLvpiuT!u*jL|y8^Swn zcjggWl9&BZ^)V)7J6%B@b<&fd3D4>1o%(^DF>gVF)f2WZVKUKzSk!*D8WLpM>LsMg zODLQ36uz!4Q;G31;2`_W%zDiB)@%gb^cv7Af3*29o*Lxj=)++&L&O4)D;}&J^k|7S zfaePs7z4PNVu(27!GeQ-r^K*UC!ViJP&Ve6vgRR@jpq=a&mfwZfE&U_I!!UD)c-OtL%GeLiq?q1cPAp$ z@uF2vp9ZoGR#9@-9#G@}20BnS0buY%kq;30SP8>m3H@Qs1f>`hjT%$}CzpsH1D^}1 zM-Ul04L&U@uR&BWW3K}qe*=8@j@cMqe9G}Y`2a&fWL6_Ca^o~MTfWHWgX0M1fpQ~! zfjG2PL}4 z5Z_$Y1CzT;_de>-GDp+k>N6)2pMvYUx6$C)et21ZP*B8)wfUJS$rT-><2Su zpIm;00ZttNSD=I-RsaW2F|;~b84fk6q|5QRDsY;jSm!JqxC4+gB24Y-`6f$@hw%mE ztWcVQR&5!J;1880sOq7^aum|2whDN5+%6PlKOTuk-t0%>u8HqJ5DEjKt_3Fm8o$Y`mIyZthQ0ryX1*cGwe4q!xhUd2)7xtcEiBjZ? zOqLrpeq+~yFzrk%icY0=X&;Kwp+;TREe=K`9tM^8xdU#^1EYQmH8+;1lwpa`P4!|1 z!RdxO5d38Zt_r#qFP{upJPk1D2e3r*7*Ea#*x*6`UexIdD??VohLFucsImc74uzoQ zua|+OM$J}bIKU+fPz4zx$BR*dR`MO8Kqhzo0~oM>yh62Z5PCAH1~B^+s@;fb+MWLz(^z_M5KPK(Lty%u0z+0k zF!qI7H!nLjzK@N=qW9YUY#bIxO$V1S z)J0L!P(DU3$QZ2Q(`_*xwJ+(pk0P5MXKWHcxmy|mM$~Scc?uYL26*}$xm2@?ge>t3 zCJec~2Q%l;hlDNtfMqX{`YLHZW-hiI!4*XnsQ5JIoE#`YykPB7)cmRl!RGiANaCS+ z_^A?{Yb=D-A+y#aWt_qTw8?VGXS5@Uj^Di5ILT)Q#*cz<)DUF}MFG6QOQynPFo)Sc zF!Il7lNX61&U*P57K|R#1VYOhX+^qmUe(PSP#a}T`UpBXbDuTJD%g)4t4jeg5Ig3Z ziVY_HQ5434l*NN80dXL`cK~YlkgmVl4%JLqgZS~a0JDvNkS$2rvd$h8yeS-x0x?bi zK|G4y5zL|PYkD%}f+!M)96)_Ok`3m1V}wFc2s3VpA2`ElH+R`BLa@UgT;9|R(1J!0 z*Cj|{hb5zC52zEfC^;QW7ATL|Dw>JlRUR!>3@}Ry5T=Nf7X!Iy&k@l)v?rdXM>_*f zxB&LJ0=#@M>hCe?FCJqeN%j-u`KMWoLGD|PgC$-?=grq@&G_jWhPmGYhm|?w4z(hC zy}|dPA7D4*S~?-E206Jlr%xS(Hy`+y~N5F zupWJ5X8G%37>WWA*P;`!Ay*#?w$hNEW3TrBHE1^4Nj*?6PYX=V+F#UX3scA3F`9bb z;QFbLY!8RRQML7ZWH|B(am({-AWDf35IkhC7OnqNw*kEKXfvNSCxF+W6~Q}s$q0UH zM)3Zb8U$}+U&DMHZbgp2|KKn(^S}K7UKsZix!d=k_(NFgn2f6sC$fDA1F$k)<&9c2 zGkM1p@RiTE1D^qG5gD@;#Ytp>*;TC&#r2ob4Zw{9ARO7!ji7q7{y38*GajOBgBM?gvdGFeNUeK^cz41t zC=k?=SEwXDgw1V&GV*-KMC^BxMhZQ1yX=hI{j-q8pH9_PFTj{Iz;BEw>7wh3@BE24 ze*I1tfU_$AArN?a8H1ap0)P|D0#lb@iN&OO=Tvk#wbKDO&!3*rT%l`9>LS2CWM%TE zm2p&=cB6Of`%D1W4-C1YATPSO83UGjksejE+HOL0z10;yz<=-6LFyPi6&#+dl;C}Y zf$x+?SwnIlyFL)Z1WI7-C-qcl9>TyA)*ti`CWE=f7Kcz8ihvf%08SeLyy1(J#KV9) zM*tIR0HVhLF?9g1lK@r&3c%SSs}_S(ate--*=bOE5i`d=p6OB5!Wdgp^JQ-fkfSQD zu^R1p`o<92EkA_#8K@3(aMo1J^x`%;|C8E5xnYRUS_Md41K7GADc5h#L6wTP5GS_*<*)$c-3!2t zK_olu^%pW_cC|&L1#{2>rniWmF`3eJh9$il&O1(t$YNPYfSD3#{=F4Y&)d6>fq#59 zO5x@JS~0)m)?TEadVT@bO@58wv$kH$SoCkIUtNtO;4q}>Wn_$y0yPx5mVizhK>bv} zZ#zH_Y$7i~n>}pTOzPaFVDtdseAIVKe*^L<>Uq{Sm7h?D6{Je)DAY;w~ z=2Kfag`+D}$(SV=p>dIEpbn%1rso3GivSDgbjD*9Cs~jK@&t<~Q*nO^JpR}qxMl~fihpA=u8CCl~#gihtB<53r&-I zi8~xJeiV7WOmjr34;-muQ@y4MW%q0a|8@RAQL`Ab16;ek`;dX_gF>*yq1O-m&7r9; zHl=irwh@~y@-!Bl&kJ9ozsebOHMGh@D=S;vzOlO*<7;zKL)5ik;Mc^7&ha6r0y%6jqe=m-mGTk6kHJ}EV zzqcOqh~vUiK|~u5-OW>Brw0Z>c(B6Zze_Xpu9e?-dWdiq&rLj3AvljY(RAm=vcyBj zV+EcrB-lSed$JIr2Q_n^VN8}HQig}tKMbP$b2JWR6a5z$OIn5v9$E%A`F!EPrixx2 z(DLXCj2~d+yXQop50_*NA~JZ!;?cn~1RVH5riC>Dzt@Pp*jJ>} zQCNXfdb&|3;dnKJDAJW;HWmq@{RgtQgrk~--*Dz3(;~2_Q)4#?g|5MK6a*J>+`cVV z0sA}XFWIgOsL}%nQWJd~;>$nRfT)mBoNEwWe~_gDKz2t7tsp1O$MF6_U_QNfTaCc6 z1W0m`w3<*iQP&20#3VFC>YxUUvxbW%c7hmg;_C|TR|qU~Ljc1>e$Phpt36S43ySUm z#daQQJPHLNW8h`*uO!s4%_U>6qrmOobcN!`z+VIvg!Ew1e~Kd5^?;s$mg~c@qB2h8 z>9O+*3#9Bk?19}qiYB=@*&yL`WnX$82?e9`;X~WFu7`@XxPw|f3$Wb-Nz1jcb!N`x zWh3eFV-TW*bFe1T3yQ@eXf~#rQBSW#xfW>h0>t`VkHdEHFPxX61mXq9mgsl`(iQ?_ z&>5zAs20BXOlr2Qq_8()${SCj0>v|6P;CKK)~FSQTr@*4bSzd;(&hvU<7b=5gZe4M1-E&Nqm_ zSRWK1hmGpUVMlduBh{4x0|1x$NGHMoEh_+p5rFPhfV1!-iBAT=AQN!18}Q&3p!*)c z>;YizLx3%0m5?W>v=o(IL0utF(F`l<_kA4zgu_GQI|)K>&?_G>Od3XGY+yX_U*P{^ z=%jz*h*`txoK_hAfjq;C1e4iRl#&EtWXy&yLTOgF&_(WB8%nH!PLl68kt8d88A=m& z2tELQnikG=B}#6Gi_8Qc$6H`by6`t-+QR{1-+@06%hxB7RkZ=&K({U4)KQ5(2e@|u zV!xzX;#Yhv-`QSJi;@Ju!{j+gV!wHE34)PbIPzd7y|;(p&r8Aa7^6722>4}Y2&(b$ z0?SXb64mU2bW?8+S}2Zg!fZ43-n-xr%*JBOytkyM*3#BUE}nw8l;|_)U@;HD50)>H z9oyAmH&{2i` z;R<{l#Wy@B^6NE}4NLaLjE-w@=*t4i2opOo<5wm%rF-orHTQ*{$;b;7~krp+4zs z+M31wfNI!s#`2gGDn3Q0U^d!KmocU$ClZqtCqQlACrC6|v=^M?UOmCEY{tT+tC}jz z3dYN<%ib&oi}zdPgZh&9LztxPpO9gc+U5zYK>!I$S^8vF2NAxJne$L z(}>~-z>nOy9*O(nknaSF4#o==bO9#IQxF(INmtvX@LtVzkBMlI0$pujh={xwQvb*J z%<_Sh3x%VY=}&(u9dtJoWH&Ma8Br)~1w3Z{Rp;+= ztbczBqA`4Gd7CgN%MP4H`_hYk5Zp?wHv5f;_+vx7y3PAgDlLp-N{_dIhpp6^WqAPI~6kkcPf>W*ua3 z=5u5>DqU58G|#dlC?s`$4H~p1{R(oQqT=fhC1nN+M{)AD(qkP5Z||m2xk{naxy>sE zSxOHa0&`tC3LRI@<6>j9ubk>;<}t9?dFDZOht7|p;nbE(UcgLfAh7r%0>ubSZbkqW zh#aLiZn@|!f3tI_9kGf_s4D0hAp16;ya#Zg7m)ZBWph=3U?+|{9e7#LY4T(qpu-O! zh$b?oeSfjpv+qZ{jz`LfVpnIm4+m+yd-m!)yq|Ju7HWbbku4|bd@eD@eCn|1k(GrC zawqJgLnVsM(SLl=QnZr#k=yW4tNQ{u$ChCkGeIOA&p|v`7K!iy5U(Pz401xe5285_ zDgYY}0g~y_gJvZPtpWRQaCO#Vj$2_^`Fikx{Q+*(-*MpjZ3XDD23XL}$wNRH?t#zf zQs5E9A3>Yx(I1gHg=)54Mru>iSESZvA!t1HJN6{)?Gj>YabzB~-3Plsz9We{59XG! zHG0XWZd+B(6!bEkny3dwh>Ml`L-_M(qo-E^Y>omZVIO*<}S0zbyh#76tMBop4y6Ai9LP-$z{KC_UEmJb`#SE%ZPqh_g;_ z08S1Sf0@|TRJ127-sqP&+;T6$(s9RCfiFrAC6^Tli z95S1V-tRkrL2AF%6Ov&x!pjikuB7MR^Hw2PT#WN=UiLK=aXU|9M@M}TvN4Tm)~M(V zl#CeDk-mzNCd=qh)?*UM{!`%To8b=y-_*YW+PAlxdU-sx;}F(Yr7;qCt=Inmq*&=` zwj=~M<{2_sV43IN2`13ji6_8d^DjtE-a<^NJHz@2@^5OUlSYvC8l(lEA?I~Wj?fzB z?E!i>AW5hBF`eOFI%-}}ET;ZY9v1##@qD!4sKjn?sYxPjo;1>K4iumtwiror3A0UI zE9M-l5biN`%&32*!~U-?7Wm&;`i8$$$~gkqu}wyVZZE0m{{nHLAT%Wrk!(cf6O8u@ zVU#_KgR+-_OxF(lh%|q9M+8ODp*P}4kCXt^%LDWkz!?fo>M~+4qCg(c1o@FW;MhVy z(IPZ^wP=x+DiO~FFWh*-BAzZ#O_DVOQS!97~Pbu22ynK%$0(b zC(2{M87&9UR6=ENwHP;&1+PI5QKG#ApqC8@&Y{s<8hwvq_i(bYk!ZyDGyf#8F=1|p zVnUdO()3*Qq72BjZO1`AxsW>Q4a-63zJoX481b1`L1^_wK^s?sI_m{}-+N~t>ZTU= z`5?NmV;Bj8^4(~K2{ku$#(>Eaz|stsC-gSk0LF*Ev(CT}Q=3Fa*rGBqI9v2=n%uNUWsVLY zF*87l1PEM;8i_jXcb^GhD_uy2Cj!ze0RLbgNyJou!!+c; zu7~_~ME-bJI&if;cnj+_wPB{vVZIT%dK#&M)OeZ0+oLZyFslZi!{6zG??i;(fwl#O z-1!IjOAvU0fft3LWGuhZcWNg7KrrDxy-wslMDS)jZkBnw4=`hbYRLh*Y786$S+13B$*D<;$SNIbb5k>j3Lc0v=KWfbA21?PcWBlyKX3678y19SbqB z$Ww)wB;kQPTm}xD9xRiA>)KhUjF#k`yF$22TLjv3SJa zd`EoT4Oc8I-D@H3{7u2Ubr_3SI?@H`vUx0KGXViY?=!|>TIecDSJ2_@IG@-L4J0qq z0Ni5afG(1ASd+=&dX$`o&ylhi{{$~IA~GF3>KUL)X8NMO)+yKz5DCl?|Cs6nria;S zvItrnrXGfr1`Y71YXbZYk*Q?vawM62UWcTWe?f5NL}9!WxL{Z*3UH@7^<8Q`xR(6g z2Cg=`s2yX)fvW>niCI?>j}Z~Mn}oD;&q3HAfopcA|5F+WrWj|U^aa%8v6*Pyj27x! zFSrKk2pyxZx2XA?+k?1KFJRyih)PXGsEyrN48@MGrq|R(SPsYyFfgyql_P@+oot3q zw(Ko{>mBs=DCTe)s|u^V3tdnaV|5_AP9w{WBgZi>W2+HYtN{$vgD9Vm7h{=mD{u1y?J#DxRz`J^&1^-)r2eHYTAx?4z-!T zF74rdF zOF)#Ei1$Q^sTbn<*8zLz$TT|)fhgvP?tX_voS22ll^FjCw0YCO6M6|evMS$E+A1@| z&%$((LlZ$P;{E~CSbfB;4`JE7%n_81A0@nTAo!~QW{d}y`!@!I@7`t7#tXoO_kT7zvaAW5T1yE`H?tan>>9X1EUIES35YWMy2hmmpVF7vgz05Z`r?UVnBS zEd$q&22k4|+RUM|hr#uko(eoYkNEQf>}bfXP6S0N=}U_~RY;nC1o1Q5&mbwF4MCgy zv)BbuC-_w{q(eXDB2qS{f-@L1N*r>*vqPmB(r}SZxn3-Ne@WsNl1~2t82-f^+QJge zfjaGHwc=jK#*fEmq!x)T=}e!34O71I2D*JM z;1)W>m-DOZK@DGi4{f;?4z2?$014|r+&K@*z;qa5lh4>WJwb-?R669S4zL2~tJ<@u zPFOBL+-JfY#9#U%_{shwstv{3NIanq#B>&j#wuUYuccIcnqe^*M?+lol0!y_XJOXK zLM&grv0S!Hd>movfuIQ^!F*H}0hbNc0$mIVG0_3BWKa~%YWm7UgBH;-!Tyu+wW61+ zl5_)U$dN|<(OmE^5e1k^gBYbNi~0{wLHzAy#J_%0!MvKK&?Til1JtK5Cq$m=%Gq+H ze-hMyW6ABJ2gHsTWlg+;MY`a(GRcuzs6O>7UYI5BRQ>&)0xF&%>1CrSDj7!=EB6XO z>n0%TF;Zj}nPFBgLd=-`hs`0#H?B@Xei7v7f6%u8pW6|Ydtje;A3QEwtl^<09<%|^ z=SoluP5>tVLsDqrR8%kDi329Ggk~`Q1u9F&0o{KFv1Qg~xuE-M*+@E3jJODECVWA9 z^z~K^IH8#b8 zs*wVC+=!&OtN?Vli`LbcgZP8(^D(MYVK4yvv+b1;sACFbiGdHegsE%8gqNE%u zk;}IMCSk2+#!bVni`g@W8ietFpvo-*j8F}FIfEL6)yt?UI2Q)$;pKp+wMc5XfNdBv zG@TlSW456FvgeIGcL=f=W0oeSGT?2y77pBayqdKe@kX9hKIm2~Nu zJtY;xSYm_tj;VlUkY7eZD;&T8v44~vIwO2z9~TZ$`vKp|ENal z%f{=H^Hk^Z&_l*p1~28a>>;7YM&@ANgC51An8kPRf_YM3r5ujLKfi|P*S!^%$7V_I z8#1!LBjYkD1vId=2?EG0cbWk7%M1k$+$sd_mbi`r=FhvXfD;bYbntH}{s#{%Y;v3u zTY)k3!{SLo{Xy4U-vpxr6(LC*;LZF%U=TMVBp>rdM9u>YF92RbDai7x$dK-E7MH&D z3lS`+K8Gd`thj(04{U1&n*Kz+OxxRcDBkTS+8xbf#FNk7t4T5eYC^~op$^91G)trbAb>7E(|X;JWnx_svM6TZB#-o9e3-jWsN4JL6@pjy zy}?LlUT2MChw*gN>s4@M_Ubi`o|%3xm`dX3jd6LsnmwMa$`m)u0_7qs9$7)nLU1!+ z2x>|+scq2c26W!TSPVl#JYRT1`()^H8iZwwxPL-zf?W%sVzaRpktxW+d~C&aG&Auu zZb+E0Bh+m)yp1&eLY`W_3gUb3AmS6DzI;by8Q#bxv+skn{sFR^mPJ7iwo(u5{UN}K z!+_sdorvotBwK8b!LmYkIeB!DpHOwrDFi%&d1Q9Hih~c=Gl00s5Wwgih=V%xg-$h$ zK4Y?jzQw5kedd2N*OrDN^mc=Bb^%f=!ir(kB27=B&9}`!I6bZr;w2(=4g{YuU^9TF zCfZ0MGbaE#Ax_L>2P||X5vvJv+yd_~F}ZRb;J29shV_#HBh!)O{D(Ssl{lLvqBqcn z7d)giilHH&K@t&1Lsrou7gvB;rTQ(#K)D9-9y&9&m;kZ~L&Qj29m3>K?)ropIxr8s zS0@Bj?HU=0mLH%rVW4?YrbHbY_PK$c@b8hLZnrDeU0z-G84b27@1F7=O~^xp@ew3^ zwnD*IsZvaXabldB8gTd!fSDe3K_hu3#{{(4w$xK&-afNH*XoTiHp~TETA8R5a#*_3 zZB^)4ac!J^ZqY(P|Fi)@I_TIKD3t;p13cb%V(`$NFTK2?XUEvR{^#;am*dBOUmT|n zbO%gt#fhvQHoSBfyaCUB5UlB&@WF`SnvIMxzz)u~eQbZ=0C;4AL{ zV@C==KNa9c<<=V0O2n~llRmmm+G&l&IpFQV@fhdeLJi>Fg|M*X&mtr?CE!Z~Qn4M( z_KAR?9@I$}&ahrIbuX2Yi;nT$1kXR3h@mOON<{+RATG!;N76WK02y+XJznR$8kN#y)l*D2>}M1?j+TLMb1R&#+@ANL2I47fc5k=>^~IqxjZ?lU=&IfUEGJpGw#$S(Bk+XQ{ks|W}Cj=hoA>l!GDTxtM=o3mN zLt{ZpQUO?CDjCH+FR^mT9veU{!N)QGVuY8z#|qxC45{m*0oI$qf`e$%90Qh{)Heyr0pEwB z5zL842~z3aQELxUU+qV#e;HtGIlu^_L*!_+`wjy%F+gNdBQg}CW2!Bm@p0VvR?tE^ zP+m;ZU(D%LXd!=-gRkf$=_wl#$wI-Kv%zMFYKcCCj$FzE4CMokQOTJJ$szt|3AsQk zt3qY`@m+TUSW}4p#gVG7sYwlmEOiMNb>)N#prY)R#jC-flTSegz7q_L;UtF9Jq2 z8iC`hknJ8CMI9L&Yt#8i!=RBr&`aiEfxJ*D^KPp$YMu^ZCQ-Djq8L49vkSh8q_0Ae zOjisoH9pbwyfNMj9Gp3*YaV7r(N`U_sxTKc0YAWAe}HxXpc_VxFhKy}U@#jY|G5>& z-?$2e+iuhl@?$0j>7rh@2=IhsG?`3ueD6p;SRnpj5@6+2z@ce?dD8&~_JEl_0D70q zjL|X?a-ly6!0Zq+gA(bh(VLr%>PQdP0QeXJDjfjIj$mJTz!s+bsg@J%st%}fx_~%S zB)xEShI%vzxFbo>0MwE-pdOR=Mr#J4#-v9PP(J|Tfe-V*I-dA}H3G(y%o7IqNdxR; zk#6*IA+}x;qX_PsMBjA(I~hrLED?Y8jrvmEQt+jS5=;?O@jL|0uKkG5X~>&lu=Rcd zRmO1z*eWiq1~t7O)b+1Goi!WxLiEKjqJisLB7PRMUH5PpM?T&M&!Y#3hg5GsPEOrs z(G{Nx#A+oNB)W!|jD?6$JG@vKfk*_}b`GEEC$;l3?AAZ}4kkXNbxGdEk zdNH|+YR{BXkdDKCFx5dNIO5@~-;b_6ql^c@$OxIzw==icAUKdr-!|$vME${?Flx+> zY8YYu&uyHHe}Rc&ZAY%$RKVvnz!oS1|HYkh^p!5lQ+L7fplA9UsDZgbO~|}LfL#$7 zPlJ96$1J3owrDN2Xf6i;6%~L^*cVb?30P4F-dIR%Q_@SJX6D>47-ar;bDTdI?+(^$ zwDvcZssSohw?qI{48WfQFcd?p+41lu8N+uYSWpC*&*)4rP4nO)&FKj)q6zmY>hMDy z3X!<0XFk6A3#KWr!6ZEhC>#PO_t8ImzWDPzSNBCFNw^5|%4WcwyMSN!02^L_tDs&I z&Tw}F;II0YLmlP(1g>lBe$<)Fy&PV?%CDH$xO5nLT7O<2mB2OPI)s>Uc z=HOskbe0NB8=NxI5`tv=6@Y0g0j$*k>otHKkpPz{faW%UTO6PTlR=t}qAU-8Cw%Z* zLJf>jFu-aVKr9?Ek(RZ6Bf#x87SL*L19E5H$SlgSvf2HAg3K)l}^@dvZlVrJ@@4VanuaiAW?O3i=u>dbgF zvzHbu`4fzD{%*#|6-jOZlp*Ex zUu#q~oizo;OHR)K&i_cy?9UFPh++dVRj@v9R5_?w)YJ;oy>q2Q8={(fZc{q-FVD1GpV7k+3&`akAAoMI}YzxRnYHJ=Y2FO6<`LDa!*jzz74#R?xgWiBAl%=QyAcmEZ(c|7n z&*#3Rhhx#Nkp~+Xa+to^@C?$+baCG!_XbI5<=!mrgm_QJXq+fme^GndO&byksV4Jb z{>c9GfGHP|YtLOP?6>bDbXE}RPn5A3F==P<9t{6bPx?HH54#NRM^^yR9sjMiB2Q#_? zmW!b55s(ZS$et|%o3JzGORqi`fe=tG(bpadCBXO-v6poH`QM-3dnAXLzC22j#A-y3 z5XgfGAUU{zWbFY6#5|EmI=pdIWTuP){QrVC4!jZcB6=Qcs+J4gP?BqGWSxR`@h|e} zF$VQPtVEzj^Qn&SMy!L@f0@c?E491RDyQL|>!;Cl4Eh)oP^C-(&1Qg?69L=kYGMad zMc)E_YQ*-%#P0%4 z0`tc-SVa5NetXo2eY?G1jp|T zX9~;+Gvinf%HQ7w7M9TmoaTjxc}{HcWR&L9L+h9Y$nOG=g!qJ^a;z|98IH~QA8NqG zCj)DseWz%<)u?%^+y;oF_Tml5~QMiHP z3=#0cXFBBXEu>}8>8yW@Tnpl}vFWopgY$862m?thaxZ0QRgF>_(>PkQx@W>*%W-eH+v43o>{MVb9F`enTtX@fH~_4uQ%N z!21|PM*{IxHi&DRB_pme3-QPRz}%f6maIIBzM5qqzPb?b0OHC_JcgIXi4dlfr(!#K z61$c;c7<+o`(J^>;WfZ`5J2BJX1a6SAVJ(*#OGm^V{~ptq844)a*~A6CwIPLj_6gK zA@&+%1q>at^5ssv^8$0n9NRMrVHp_^nanO^?B9rC_(bDy^}};qwb~M|T1!OJk$+=h z1z+yc5N;V54X3+ER@uR^#`*LEO?tTLF?SkBHo+28Kq^~)5TwWIH$eKv9&f`|;B8yY zor=@yL)BA}QcI4aBTy2;*s}F)@dl(kOE82bR?D`u6&YrYXOTpfl&OG=Eyt9t!xB2klC2f+vlU&#Hem_3 zvlLAQ!)(DjnI-a)HLg}L+E!wiJ)MQKS2a_TY>SM}ERH4{2h4@G(rW}}v$!+ZdZt2^ zwkTyDOEiRSR4deID?co-n8izEo0$rC*rLc~EU|-Zi(29Pwqw@_u4ak1v#m@;25nLP zMwY}&_S9ODQQPsug45Q0xpsq%@)m4lHWXL zfq;7!$Gk=RIoaDRy+y!Nk~^bCS5>H@S!R!bPZrmwML$rerdjr!fS;sjNQ+^qP(!oa zV}S)(qERiz&xOu6%l{S#l;kD0n5qh2YF1Dd49Vi6+*_*T zw|zyNSY7<2`@)Y)$3|~Q(_7;$6+~W^Di;c_TO;0WXYDC66gRG3aO1G}b320Rt&V7on|`8@f_pc>T1i;v(u_3ySv9MXq_3yF|*UYPKrIH zBU)!QbFA!i-jRLX(py?FA9i-SyreQ)nbZU?ORkHWo^BFGr8r?uG&5G21Xv=!B_UhrPe4=rXp^L7|3Px%>9>*NCoW8@DGls>%-t zEUXmW@WS|c(z#OkZvu-4MPpx>uy!=7DhLY(>GF28m?{gmXDN)?9_-0Wi#IbDzA~(! z>VRR*ji2Bnd~J=Qaa(9JugGp5&x-Qthpr$p`i@O<0_!DvRJFUTfq=SZN+4Z#=}iMK%z%5U8&kJ~oKNqn-Kc3tGB z=lFMRTbm_**iC;Y^0#~Z?>2}l+uoMP2zaUT+T(2{IresjjPNa0<@W7yl2Z2e?hMyU z&9FVGSyIv7A%YRVrDoHzPu!01lbv;CY`X9>rnUCO$%)?EKWKZE!V9c;F7l7&ge)| zL!;b%_4#3GD_e}3l1`7xJy&0loVM1^_-fJ_OZh?dg_UXRTZ|tjosE_sR$tVSw#m-q zZPNKh`BC-7gK1k^OnxR^9F=Ej1TyLIcBb4NEtdHFDM&Xxsl`-jN81_&3BlkQyc9Dt zU8hU63i5(WLUNHi<5ez-Z%Re!}&j~VYBDl1jS7bIZ z%n33)S#a4)-rf@vqn#kb(*?tbSeco5vJ+(3S#Y_gSj7qRLZ_ZJO0xySXNXmqSyVdR zsa2XMxFSTX=7dF~69js(U__$W3A0HZP7h4SE)!gNP^{s^r29^f!p5%ZU)h||s5W^( z_(|p14gIU0XPhgW{7v}T;Mmyy)vU~BHA`WU7rM$j`q!vtwwGDTiS&mlXZEkP&FoOK z(iVADshr;yxj_7Sww3vnf#=G5+tzIn@0PN*-#Mr-?qE5Tpl7YMr}JCSaYwhWug|=D z%Q|4^Q1rN3jSU^*56q@46?tDd?o|JV=b4Yorfj$Z?bvp1`^GPsPhU)lz4DQz(&7-! zljyUvN!|I$ROO1`CQXS~Cu|BuK8LB?=-*_!Yfx?KA(5|@DtG&10(ZSDn|ey*`=H9x z{>^c_hSjEB7Wtt&exQF#>8>wj)9#A=3LF2Se{1uuQMKs}*{aDK(A6#5_sqNPABgsG&h(QoW@7h<%_7FI#P5;*MU)v5^jIT*Pmw zO-bDOMRNRzL!67Gl)6J=vW%2QBy2xa!&q+-6kur#MtaXvQ zr9M9~ElSE{#Ie~$K}usuVtQ8ggkYx|jFPKHct}QV_M|?ir;KunMr254ceZt~^9M$y zQzIs1*KqcLh@^K=JvTQ;uxlA)rIWt&It&fxr$@l zsnrxxFr2ffZ)PmVl%w4mQYe`l9PFOSnc%8@HKfQicWIydK8|^c_U({j&)o3fS;sh& zI<+5$l&r~J)i>(`$C9Jd7qTZSH!|4c2FKb}=WWQ|+T0C&9#1(oDLNw|`?_;uf@go= zOzYJ78M1#kH@0u~UydzDS0J=hGA|+6Q=Dt>s>=;6GtJx4=c&S-k)k^$^nhnxYOt3v z*QrxiDYSe|US^-yG_DIrPd)TtR$fl9w+GkFRZl;(qBd`LpZ6lJdy3wK&_msM#ldq{ zaXmWqtV0hE=k4p86U+7F=-Y=@O6DI3_Q~XWyXw1zR+;7>>hsyh^-0n94n5+Te=w+8!(iw3(IriPv{-F>;wPvzQ@c*AVRI(NCN z-u{NyV6loEPe#byxb1Iq4U1-(V|}{ZZSMg0YcOBisvH}tw@C6b(@U4wVGTC z-qs)fdC;LTL(>7U|3~HUPEcn{G=waICO(w%y z7di@l*e`yYw)Kq3sMf{7fEl??a`zBLK z?N+>1>=2}%o^-}kLAx!iP^>M;I(_F4Q&sKuGf~OO=JPZNUNQX`9SU zwJ&!RDmp9)PtQ1GW~qH;uy9=4l9=>eKg?`}JBXs%#E@k7Y_kc@!kwCmS|>va-E)^u z@EGW{Ez;EpJ;ckenlN|ZYG9FJdFUx#!J7$-2Cl^^no2Kizh0;|G4%5FJ&NYJOM9;u z2Tfdgx$B&wrS!6w*Y}i7Tz~n-V?~?XWnZrEdogkA<(t11ZKcCVSE-tL(&cVtr5U+l zGF=CP%rh?EvQcsoT&|vRFxxzL$88^_nd_Dtiyi7TFLLUMQ1a+mZqs#`ZBgdbo37+N zC)_Oq#`&=Dozh~T@^GK7BS985hwjuX&HEf4IIB9vqHgKk?&5$YDjVKu2k`+;sr=*ft}StplIx-sy; zW^9=B%7Yp8MU#3DJ@hMCv3+G-#;FIB9v^xXU9wtmRcl70)a3r9kF!c5=d8Nkb;fG) zp!So>lJ#w??$0{AcJhaTC(R|%8mkAy&R0$TcKPY!lFi#!k7QhUF!}ePXJ5u{`@EXv z(JW;txbwM;^7eIWBt2TaEJb&|FjwCBd5x+^d$y(Y&ORUIv?Xf|Z(M4zRCMl-P|oaG zYjXoWlA80&bmg2mk?uD-?X2{iUsWsb?ulG*<669xsm?&ho+5{J;hAWUrOxZYJ$u{M z#bn<6VP&f`$m}h1h)T`8Wo7N6^G0`XMO##HX3r*TPo1|ed#fDQAIZFP#@bJ3C~R*{ z+xn)=dq1oLb>1cKJ>jt7YUTs0DPcPAEB7|EZFrdZXw#H6Iv+aro^jatHuK4uDchoT zJ`V0Z-?s5*=CdDD;&g_IO3TD(X7&p+o1G$`RQI))N6Sg{FSp5j{mFJ;hsGvtiC1Md zd51@+Vs0|N`KraH$oX@G%8i~)HaB1YuqoB~lDzMhLyX((H)c~0i+nw(a_3}>-|V5~ zQ;)s=dQRoO^yZM8@5-h&T>17`<#F!j$Xy?9O+CBw`xljG{hMPYhB?z(myJs9?_07Z zQ)0w*+SO%0O!s5y+9&ZPW!f$0pYG$|&e>Xh^J~kr2hP7D#((J9dhX_T_VgFdzthK$ z%!$2z6Mo_w=ReiszxBjEzWFPD`X}eV*T?^yv+do@KWCiiqrj}`9cu75wK<6pUP zO4&2wxLIs#TSk|ySBm9MDN8WMR#I2MwUjd>UO!8y$yPyEaCs@#KI<%zpSG&HB&Aep zMuJ-w$J$O;SE#B~wml&rOEkvLR9CpORB^`k@GP+=J4;=WH>Km+x5s2j{Is*xWjJMO zGZIs?;0(Cva`eiy+7pYjq+{$ob-Au(dNY!aWXU$!`{{}&tYqa#2+VSe`7CuRbR#)zeGg+wo|a?1=+&U#2dSGE8->3zNHbAiyUr;+9dX;~5wE z_v(w<(_*}^6ko*eMrZ_?rzd-vO>nwAsGwFJl9!%$YeKM7SBK(MjWC&vgSX5NINj+` zTBs3`mr;LfQlHb4j{O} zw#}&0Rjepa%*)B~4sdg?(zB{KO!5!p_!PVORO$Oy98Jux%bEMoZ9$d6ri$ZacWaKH z(#()5!=j3liMwy-1O&{Cs4_ZJaheqLkwI&_5;spp0iyXRF+@YlW;Rb(u^^s0NY>%^7XH+qX~ zdX@>yDswf@(7x?c?B287epZ#MMUD2|-r@y4D-vd%aGlhp{m`c*qGx66tTV2Y-)leZ zE!onuTEL^l)ly8S&u341&suwrPFE`zh38>rGjH z_!qdd*ifn9 zTOR4Ni|g&E58=}Ft*G-U;LZuvpVd%h>3ev@XFt~`N#D!jh>LI41mDA4-wJ)-h9iEy zMt_qRA2=37(edzw4%mHxtpqtU*{M|>~hC*2H!EUJ@zPfVD5h3l_n5ZX{( z=zB79?oDohqrq~Eno8e>y1Dnb^Fs|*HqWYR_a`)Z_EE-x7UsJ1f>ayGe z{rO9WBFbv&D^A^&dlWu@)zHe8nntbD19DG}%-=Ay>P5}DiqqfZp1qwPJG7d848EVR z{0sdBJBHS%9c!;>l#}ldUywPp*6vt`)){U2S4S4?9*PV)cD>@vB>C5G7wj8a7k><1 zp_BX@{e_3FMeQ~2&2RF#_cnN8&9(LCjql5z4Y~L3z`}-W8=e?H&OaMBN%#y&XyE9v6! zyuY^>|4Q5T)#TsTi!48ubf92*oUEzfhGt1WfjNPq>G2k(BHx--{YY-0bb5lXspy6l zQ$OLJz_IDu*PBXwYq9lXq=Pil6SHf_EN%5r;LZ%vkN7|A-Dgk~>)Y@9O9jM?kt70= zjhF$&g@B4EMpO_p!67N2$N-`QB?<^CDk_Q*b5_7aQV|7pnK3UDre#*ZbiV&nyXx7y z>UpzY>|N)@Syi9&bq&S+^h^PC;9JmXMrpYv5(IwL*(0#Xkh>i^>WLoMDPg#@HsJ7nW?;a@G@ zPDBKxe>^n!#f9Hm2F<);16H&?Z0}RvQoHj=ujGJ?UWbRhDDR|goZ^)kkU8$Kv(LpI z+NPJhvIABIALd=|U~The-dh4zWgH&wQ!!e*%Si9t0jm!kp7f$(x^}k|@52FE*A7qd zxinY1$0hH=fHfa`Prr3((cPXuyo;KzZE$3!>E)&CEOmX#_O8|L+*(meIk9XHqRc~$G3lFo_oI|KDV2%_wEyTyfWW?z-OOF%{MIT6WqV5+}*m* zoR`fvZtoL%yy}ko!1;6DH_tiOC%pfa8g1J>bH2UJd2(dYjw|1^|9Ug$_uEYkjz*ha zRnr;LRiXB7vti$*Sy$Ut+j}ZnROZ_DP3&;ZxY}{6VoCeTEz|lg&$`yPde~EiUgg%P zzAHNX?O5$(~t-rIUc=!kG+jp$riEp9#+Oa~P z+y7QiEAS(>GQYivuFJ9kM4f3$?@W0cbcxB)h8gK#UYda=Z@de^5Fe^+PgzT zj+dL>HP!Xr6Oi@p@T}vPciio-JLgS6&buQq$FG`J57YJS8o1-#(cI&_ZJ(s;?;p7T z-Lc~1cTDg3=mzcyJpS(ZgZ}rk?k%`C_i5ni$`il(KkjgU>AfJMx#ueL^#(l4y1(jP zi09nO$^zQ~H60#oySHHL+?$n!(+0fFdT{jK!l!c|RGy3)@S(%QvU^ISdCx0PZ5;3= z>*0-i5uWqjRh~XQpti%KXZIFuo%gl!%!2_xvmSl9xA^J2UzKNn4G_)8D)*y}f>f)D z^sH3YJZ^VC)-$L@Rk5v=y5VBqYh)dN)TWfR8=iBbD+8Q#ks$!aTw>8g?-p_g(GOg-Tl(nVi zi?aJ`jpln-UEXNjcg>3%_p?3c2Ub;{wzkrI`Rx9Nt@A^xsvcMmTJ!SD{hX)sqpGg_ zvbNK#QF*Z0Xu-0ot9mw$Yiim(*y6b$wd$I!jg#gp;|JTeE?8al_cWVPYhLwzu;b~1 zjaAp9Y{qK7c6^X$6uQ0Y#zq_0HLqPC?Dh=ZS9SBWjhp5huLpa#h90ZB^}uH8nl}p` z?0*`1y6W~Xo0*z#mp(XXwD4Tj9le2GYu>JUaM*KUWz}8VfeOuc+a4ULbN`}^;OpPbkD-;#~_x?#`u-(Pofe!{B@KfoHKiTJ$>xLU2c((4;%7jnjb1FQ-yB&C5aB5BC z&$DwbuMh8g;Kkij>k~eQbzoWDIHMjd!% zeVUIuTXJr!k8nNk+WqvNgs+ElZrzKRdf-jC?Sb!K&vd%eEOOSEx2d*=Cx5&A@UBCo z@0fS{Y>(gmcK_kMxX9o!|5n--PX7M(;e*o1@G;0!$XD;uo>#zUv&7z$5pJUEk zHLD+{|Eg=$j`v@3&)nEqKS}?Mf7Jf>UyIM&G5g`8|87sz@%P`JoVmaA$3p%0Z=%k; z|K8y2W3!*}`X9S4KL5Vf@a(glKeO~d`!B9~UpM4zjoGgq`d{}fzV*I-*4ei^e;wEV z{$}yR_djCJelYueUcavEk{9oP=AQks^Y<OWYTG25Oh1wZ}G^YuEBl1KY=qRbmXss-)Sqack(Zxyhgyb4%6TMeWNQdTegCD8|xK zt)fWVsgeKY7N=sYEE~Tr(kpKixw+NP7&}Y#rp5fZH)V6{L9tGjO}Z4Dl&h@Y+;&my zSj(o)#THJgdp5T}73*f%%&)k6x$4Qynm=P_S~gEE?&YL*X|wjAIE7`4-NpUN)$VQ9 zT@)8&*|MV8+NtrI&3dQel$Nbt7u%LM{;^sAXI!+UM$?iZPU_8a4F@evv~1m_WN5j% zZm!Xyr7JAkIG2oYYSJ~=)nSXBgQ_J!#HD8xZD{mH=+wPPNhKNF;vLDRArCyi!X zh73wn?`7DfjL(+3TO1Z8YV_*tT$be2y6cvqrxLY#8Tpm*L37}i;Xe}%dKo8|WvKI) zQ~oo(TGXLqdll_|8ltAH)y!MzshBl)j`xfO|KlOCx3X0`thn9C^j1ZC)1p^xI{t6` zW%j>)Xzb&hhV9?x@X%Nud&~pa@$e z@C4C7>(B}E28S0;kO`+Jh%3CVOb~C(fdanpy2mg&cXSmatnKG2S_53=%LrHbiZL#( zGS|md=ArvsSLuo0Nv_f;*;QKLVY;idU+pTI82{E)Trm;--nmMD48T8MUF9We(D9qA z*nVevt*gZ2!*5slh)?*8FZhb@sKpQb#ECzyGD>TrxL`EKU@RKyO_W3=A$PBev@n|} zEun$dXoI$BhxX8f7PO&*0X-&4f9%?)A`9#%N+{fhP84_SpE*$u;2;j6ddWn2hxho3 zU-*M1+a^j3Vxf6>qWIpODE^4VB6NE^QF=n-Gi^c}v_(6#M+fMlH`afhD39tU%JM%G z`Pt4S`Kv%hj(46UC-59EQG?fbk5BlD@A!jCvq@5gJ*QQqee@*hfcG(z&z@35lH~4KcY`MKX&f$x8GnpCmo87)ub1cqAhY>9~1ml7w8IB$2Nt$*td$ zV@dNd^a)xmqCW{KT+nOF^14L6yHNjFqvDmfjKsomg?DM(;kB389#8Q8aRli_IC)=e5A4)JK+ z!A&|LU5k0K7PZ=LQiq*7Zjy(1T{lU<20b^~2vY+$F~c-W$0=Oy<|bwRX~#e}F|czJ zmkDkXjEIH&j55qk^cT5Fzc@GPk9Ns!GQL_xUMzQ$mzY4^Kh;eJz=L`!#!$OrGTi<% z9#b(5(=iKPn1_Tk+K4S@-K6;g9@0`oTky8TP2OQ>nVSs5a5!NEoG}uk;DUg1Hwi@P zTlVE0{qaaeW`1Qmm<=V$>)2*9cNzOwMZ8HL%z*;F@WbGi?qUnQCo1x^le?5SyUW$7 z?s5%3eBEV8pu2QPcbA$y?(zz+@dk%ptBCqxcWH&zXoI#m%y?}w>ZrRc!hj-od3WAj zir=V63I3&ik5cLnWEpu5AF0oipU4YjIWD3CpBeWBU-1o>826p5Lp^Ho1HbSSmzn;X z`~$h*E>#Ryq7ijNG=LgZp@POxM-wzfGc-pFv_vatpf%c{E!v?yIzSUwSpQXMQU6U| zgEqAeIzktE=mdQj;5zddk~hfCq!El^0yCK6Cev=goZ13i(FNVm9X())p6G?%=!3rK zhyEAvJQhas?s0~|3F!!Zn_-~v}n!W2x!G)%`V%!U`d;e#ybl)H;I zhGH0AurJfe8JG$E3i`QfPl#RDW=s(HPfJ4b|U_Lt|)nnj$*TMMvnN6ZBEjd5XM)%fKn}&3=k} zM{42}$-szTQ^XlZH9X`vP9PrzD8vO^MrB72slw%+9@5IvLlkx%(qoi|^hAS69x}6m zs<@Hvn1Xq39ukCLy!G&qcSsKMkX>s%q{AK$*>uc9HX|3?@b)D0;9q>g@hcuucFRMe z?|8_tM;_Apt%tNhTeL%a?EcF1A0Dz*rph*SZah^2Y^TbQ$y3D%+Eb@WMp}u+21a)l!wlTGK=wZD0l~Xm_3_I#BI7O`1Uy z18k?sXM96FexYFaG%3N;nbSmL)--92_E6}mN(WLCTF{0LuHkQ7#|_-XE!@T(+(k7W z;?aNV+0*16<}a8gQHa4h96LNsPT>qLqI49(qRq0Dw!xkR!f&yvC!a8g~yHnF- zgz5U3r1rM#=#Ztn1Z>Oj~94_TKv;>x+uF(mq-1m@d7XL2Ji71F;>$h7V$_& z2G-)R{d6gWxvL~QTT}T$-By<=) zMV>}S>Q2zZ8HV*q1DvHcB#X$-WHDKSQk3Bw&clpp7szsSrS66v=!M?sgT5GuL9oSG zjKg?LfGZ|q5+=h9?wA5E_~0;(pbo$A8-Fk)g<}c_+@lR&mV3%q)Z+(!;un5HHR@2XGLFaRf(k4AGgM5{DD3=wB4z&PI-%C;hV7Q_f+} z7EjrY=36;l@ZLfD_Ru#6Jf%B&peHQR7uJV8MOnZxDCAf`AKs_fZv-Rttf!=*5GOI} zvZrjT^pwR_oFCU|2UKp;4>g`L0()P3%0BGJ0UX339L5nGMF9#C`N30meD;)``1ObF zsCY>Ra+-U|R&2v=tZL!KlSI7uVU`!aT=U`?;9mUr!Ao9ic*%QAYwIP`F#|I(3pqH6 zh26Yl>_9IWhw+#IS4_ktxWOG$kbz9B#^OO<(rK`l=)(m4F#uK=3`cx&^pemKEH~0i zR$>)aBMS{&ykyg8FVPw2CA!!mkj#pB~eOM**4r;wqpmZM|q3Y zL~j`ad%VS|x!!UbXK)rpD8o5a;R>$8e4e*V2=bPX^S!0J(pyF$eTlbZVkK513u9uv zWh}%RGTTQHDpS4XI>x1Y%c4wg>AcBXe(ms< z-)O(nTRK1!TF^%ME^oPr3S7cvRO0z=Z+Quw{obOBj(B*$TOMQoL2o&LgSd9cTW+B8 zxVIcG^p+#&c+y)2!WIs2#85b6Bu3%zDdxpdoPx<|Z_zm8Evl95CmN#(GBBsgTNId& zg$P3=79k3Yu>^9@i!g`R0U6?c9NBm$GttqmRMq9Zy%9|q`!z8HX+m<>^zBdRb@Rh8Z39=JD|BOaKF8O@j%Uhu&j zD6kNbScE?;J4ci8@P!{FjrS%{$0?mTavIAz(w?5Q2L>=iYs)!ue;nJGF-K;hj}O~b z%n{qgb0j}YRq89}$fPTCq(gIsXhI8XTPV2Cl7btFEBMjAf?wV%__e2k=UXXcFZSUo z{)J_>Dt87^aECbscdb$I3tj~`MN{zOafS3^+E?-$dQ*QVYq3+4_2N2Gv=ri`tq?CA zg?LyfWGd!iLvMw+^i{}cjKOu>!CgGTKX{5~7~fAJ6EGb+k%wK_jXf~1Qb=bQ!33u0 zikmp-ppZjwaZ<=QtQ@J}HCn-Iw1U@Yg$y69knqV0i9jS4Aqw+66cPlt+3cIALLMp< zGA)ocUG_>LY~d^WTKmf8HojtD z=qtZ=sYFp~YhOrzz;em&*JjY^%uQd1fm1dX_!tw|U z_Z21DMEJ_v1Ydav#UWLRJ>@G&h`H`7@kqesqpGr~fuAhK5=6sC%}>0MfTwN!#J!WB z1q96KW0IXmQ8>~Ue7(cm!&-48xe5oIw2mNHldOvx%#ZMlg z*;YRZ+~p^F`~0L6GLHDklrw%}dPY_JU-(G?0x=i!(EO#Jv_Q97KT*{CN!AaxgSA+P zY_uv;6%DMXZcVnq2I{tCJ8Y!xK(>b_av0tOE$Yo=F0`q2u!UNe?1-(@dgL}zpX8>( zVgN&Qh7pXho$02e33gEL#BSta7t9#Hhun*O*bj5YS>OP57xEB!5QlLDN70q>$H?wv zHyo$#N%lZ5SfV%jpfB>7eggfd3&@iw#3}S=+yI=WK7+HcVz`JbMhQxB4rMrx3n+&* z^Iaq_p#qmti7H&dRb0d0uwmZo4YjYG(mH;z*x4?l5B&v(1j69 zV2S~7#4tEv1e`Gvqu_$^sQcw7DXeQbcKzX4lb6WLsKRN6uaH-91GjMpcTtV|_=r#V zj4!ytGHwn1#U0ZzyD{4Oix(8|!wFq~$wv`NP=WWI{bfxTe_4xl$VNqXf6?gWFP%zN zWe{nL!O&-TL_dFV#z-9M?=MGj0{JLF5sINbz+XDT07tF-_{5|qM z9^fG!;R*i1Q@q9-yu~~Gi}$c(S*iAy255*zP=P8MLmf@f6wS~aEzlCJpn=wCi*{&_ z4$y=)^w0_VFu({nV_NJ9oPu@bYsaDBlOURZ_{tf*!F1_3gzRe&5tkJbUw6N&8tBniuriZp2G1xQ=? zm;}fiC@{8XfK0^dUICJY4amV(?7$whv|(AaL3`L?`CqITX;^{Q&P+#JXrf?TfSiPx zTY#9u0;@d(ByMwn{PVY}4BrRzCn#-w+F67`g8>wBPBpbJ{m1xib_ zLO*P6GFOg{nk$wrb7h*xT$#9Ht{7y_6=Rq~bNyUtzGJR9?qc{2ZQebX?-l0C3%o=P zUg7(mx#E0au8c$sViAYQ=jO_f$8%-lx4Dv|HBUBT6WVr}Cmr##|2+AFNnd%Ln>CN` zY3IrP+4Dr#bDrp+v+q1Hh8JY%Jjtl#|HT{U$p{QbLe4x%gzK((G7)~o^CarsJXr#* zKlAu^O+j)K#-b)hFu^T`Z{rT`q8j&L%C!6BLp;J`Ji$MBisyKb5BP|0_>Nk<=^P|) zVbvu_tkKgsNTM34$)DgLk&qy155@c-@kJo!V%d=(iAMqwk%VNpsi{c{xg4oT!wRG$ z1DRNfRalKItif8W!v<_b4mM#AbdLr}M{GnPp5rBI@CGW!gG3c-Xbg2Ufr7U9p%=9; zdY=grXNU-}m<@S9v5%pYR2r@fA6Y--OM`#W%)%M=k15k01DnU-*qb zkb2G`G=d6Lp@zm#M-wzfGc-pFv_vatpdH$x4LU#zI?#n44AB`zFop?CVFn9yL05D` zcer2-Mq@0-VFFw+5tA?(Zg9sGSpNtX8`$D6*kK49;E176NQn5t5B>5o1+DWN_0E844$~Y91nO(GHr>#m^QY+dNX9qPYCV!D7WGTwR-sppVSc?EN`WTmS+dM?>;4V5^gvfeqKqcOH36YQZgwGh&HAI$UH}+sJ zMt7sl7>n_k09Q2Z9wI7Gg&IcoV0pM;G$L>eYCS`wF`A(o_i!KGEknexSBP|hH9{~S zp$JD!?+|&1_I=pzzU()i;2GBS3z2N>!07(84`VS7od$%68M>e+rdo%{G)%_~{K9Xj z+t4q#hx>Sdu7l`9426bmh_prV+B6p^wbbJgCdmSC$>%Jy1`ED!dDEN5hBAe0z+pqjNypGWn95s zoSwz{a2CC0b8g}(p5Zwjd4|Yqgm`hxu>ePL499T}9^M=?Z1f3{zhSAvdm{8kAM`_C z^v6A>4jz%QMzb{`Vzr(=MCT3kA9Qm#r?#*iWTF3d zt|ttDHEb{lb{GqH%tPc3ju)b^1dlObC;f?07>@zF*dDAg8P|4mUf?=z;3ih>;d;Vm z zD1zq^`UT$b!FIGg8X{NmH?HA2?xEE&_USn1ygu)nZ_sXZycHsP|M8Mz^_lau`Ft7e zJYUA3tErmI^PMk2m=-!;reZ+Ue7P1qU*2cTm!rA!#c%I?@q9I3e6Z=ueCgt`KvMh` z$Z~8CSRgyF6T7e<2XGLFa2Q8Wg_{^0xIpf}VD19xF>ir13R@tJaV%niJny3>L)R^k zVHgf4jDRynVia638e=dP<1ii*;EKs`!#wOjT=oK4ie-q0?)n8X8=mljH*!#f%nb{~ zY~KQDaAtulJi9=`phP$#pmTG9SiW8$S>G1$`FDY=uU{Y=5b$$>^iv6y(u`0^&I*;B z4WY6L8+L?B{;^OgKp{@z6i(v|&Y}oEA;&|dAsXTPiBPFU9l92T%1FeZ4)r*9GE~mv z0?Kg_m8gQ*nNYDnPxOWpM!>2ll<(+6`R+ZG@6bbK3Ow)yU-1p!p0)EWlOtycQ~b(GUHx605KpSy+R$Schz^N0Zy3 z(iF|m94+95v(@w&raubhHwFu3Bc_jED6jk$N=n#5Sq_s|3nkJZOfF1N6Yk9+$5BrG zt81A2hGCyDsk05^--d?CgJEGZ!Z%DdxT(pUNR~n0m0>a%+EdizFH)D(fgSY_IKUpm zFcgj$4ktKc1V*AG%ZwsN!v$k77UMAv6X1%8n1so2gFB|c15+^#(=h`xF$=Sy$ND`< zFL=WTbD)4P{NRtd2u27NUR0T)hvhe7_*l4 zV=~+@igvkRG$I*J#p-ooay2JRwr&cOZLr!JCf2aQN9Oy08tS)bxGhZd(F68yz-Wxa zc(`LKW?&|~;Eg#5KnNlcg(M^+4e3~c4CLY;{EJ^`xjjtUpgpu;j&2x?vGBxJ4Bg58 z<1@bEJ8JO*=DTP!dg9Q5Fgc9B4~EGt+`~gW!qY=x@*FSl5;b^**LZ`s=zom>_-tw zP=<4O@}B#;eh3o{v_>1WLtC^*2ekggFxo>CI?#q5%wU00n1Rlp!^8-tFoQWP&<(?} z_X};oejLC-9KvB7!BHH;3FM;ycTtV|i2N2Nix7pySb}K8AQo|0ie-pLA`*~<1ErKLQYlxtNC_1S16Vu>def*uw#i7!D`&(xmO^hyECVCR$2q zh8}Rn-`YyKjvLsZqZD6VrTD=g!3e>8y2gE>&Z7dLSi)wqZIcz}m+Fjq3?3if{OiTGXK)!}=-t_fbljfmxUhANax#{s_Qa z?CP(SMb=7*!eT6elMUxOoZ$-1flASWHgsTzSA%FP-rz0X;a{{EOdmi4=CHs-Ov3h| zO4)&(*n|Bzhx_Oz*$t{$ylZQgJ*bw=Xi-4yuueO z7{_^rg$RQZkywN%EXE%+9M3VEK>M&4`>|mn=Mc7I2X-P46_b^63v1kzvKH%*gQHWF zatdd#%Y*Y_D*b`hXoI$xk64VHrW6;9M%;AHaiq@Rx}L>8&EZ_Zdwj&F|70N7OpsCn z7t`h?N{RlDWlD)bEGlpb<;k4WsLy79*R#I}M`{k+d90MBXzCU&XBUP`QLvhv_&Z#F zJ`a~akQd?701eRyDrkWbX!|Z)+MxsHd=Hn>KjCr?7f_B3MiFum6}W`Us6-X6;3}@+ zZ!|EA5PORValo^#5%L_Fy&_~IHsP`@%i`%wrsK>au5J8LM2PrCNHKM9vM>6fKL%hp z&ND5Ryg)7`<8YC>92Ho`a6A%_h$LKMd@`AW<+#jnDw&3KtUv}bu@bAW8d+F_wOEI2 ztj7jyL=HA#Gjg#7Td^CL@EHH#HQr#WUxaMKcI?1T7um`6m21rFgbj^HSc;W%^`N64@Q+L1y#@D~5#J-*;OYEcKT z(Sj~I zLJwPU4^Qw6vXXNNs;fB$SsVjoV?8!tBXUr-K0?ZI71wYb+!;phpc>=0M#y+ffGdJ= zYKfYJkn^zsp;(A8C=rg+%yR}2)Mv>^auK4i7)ub17{nqDOR)^`NI)Wzkc{O>K`PRa zjups2CRSn84#q`yYG6QXPM@rK@kapn=v< zI~XaAp^ibRd{=XZbz(NWpu}LtmtTyOi|~9$`#wfW=9fr$X|zasxi6AF7@V|-`$8;| zqq`PK&GAKYplFf&DqAF%J}i=oHc@g(CrZ}#jgo$mQDU|{N?f<8i84P*!V!tglToq~ z|IH&?`ir^Ber?*w=ZfqCAyiifu(_3d9=1&Bfsbt`38ZdKdB&Olu~#?VeBIl8q~ zLnnhK<}oHkD%sI(l?}%lG6DnRO4Wk~9m-?f?V9<;bkA4KH_)t!omSK=BBp1pYLS7KdfY6# z=E*U=tklX4w9VtVC1Q3=AEnw=10AP0U%M8&WBTQ*-7(Nr#03_$D2y3UtM=HSV`5ye zUCW9XYpcdJ26}mMp+zmLV+JZ4e=z7&9v5!c>UE55e&bpL{hGL_qE_`Wf7Lb?Lj(1t zF?JeFV~1F&s~Z}cFI`rop%d$%RM#-9Xrx%0TGZMmc6hD2fuV8Y z(hR#c&aoq`nwT4!+IV4#k%A-u`)EPS-P>PZA9#t+9q~} z=IYCG?b;>Bjtv~=P;mCaWe_I8TDXs6vJ&O5*PTEjky_{t(}o47f(%{LqN zO^m;0r{f&wYtfU+U6yEE$UPq7pZLZ!EkVSLY$s@=R!eFP|Efv&Ris}Z zx1_d}bhcAZY%s*2>CzZ04fW1L%oA0M4Rn^qDK#`Y+dCyT8DiLF>9Twctv9Ht`o~wG52B5|?KT>FpeU(MsFg z$UATO>f+v0;x8$+EscE2m#-Vr$1lDzU)#!PPR;U-#eE{;uheSW87b6LbBFXzj=yH5 z<7DJ(p1QrbZ+853rOsF*Kd01PL;CHGznQP&X5_C(-B;YNF#dL}&P=0##MDDW`d7r? zwbE4>1?Ht5EAC$%e^03!WHh%twP47A*YOYXb(Kc*YEn-Z52%lSRI3|p6r`S3JjAML z!V{~GiAKTZY3GWqbP}E_JFYMaaZ0;5#JWqu^Zbr$jpi%TDvPac5?qR(g3xp?PUHi~s9EDrt9z4D?HQm#>#^6jqb=pm<Vz(!>?5 z?d_ctn^+r~8!yXS(Z0lfN@BBcLrdfM@)g?l4t|L(3Jk4`6KYoIl{iEswyHC=sZY4+RpV@@mA>{ayOa7AnBFm7uUHva;!>D2 zpw9HM@rJ~e+^f7I$=cei#&~1i%FvS0)ky=x%|00Cl&=i8AM-lNw!o~`cvH>FsFE@D zNq^OuiOFX5RWbHsn`e01SLfQhCMS=x?&4&!%Y60r64&hH3E^GFn(TI3y~}>$?&OIDUEECe zC|2()nOK-SxvtAhlf8+n57|$uNOrgGsxaA?xB6Jgr0Qgk@UB57`^#4s*iU|)JguOs z(&RwR>eD5Y>yu~Hb&WPTsGe19@76SBmUXv8lSAfN=StjkQar=EtuQ(4ly%YGy-SLB zLASLgM-*9=CGIvUbLzTnHaVJ@bt1A%ufFDq{nX@?VCx>`CI#keo|jC`PMIIx}R}Ai7Mz> zYjU<`&99Of^(jm0{_j05#k5>~ouPwQ^70Jp-cF_$&DWWfdSx$P8Qy!WX@%1|3kUDr z%U2ilb~C-CSl7MOyKwoMy52KQFDI_+<=|7Xe4TY4g=uBpx_+fT)yvn1_X#qsDqm;q zFz5C1jRk#_rdMj#*_O_!U%siXPqgV(_3R-Iil(W#)_oIAubF2LEmi2GZVm6d!t`&a z>=6#WT~fCf^j&LuU6Jil>T8p_v##%E(;JD|;~f6;gcR$3d8Rk>vL}}MO-bDo-tVC4 zt@3Pl2Yfs^Pb>A0NIg*3ugLU{`ubT80m-R{toxUn{^vO;r2*NgN5cDG zHLZ49@9Pk_JM~yW|2wAl6zc;^0}E46)b)RCdOvY}u*2Mn)B@`PHKq^p)`ynPtxi1| zKH!7t!}9gv4)b28o-Pi6fPs}$gD-F^~ zD-E~OF#E@8Ly|*qm$Y*QR$69H6&q4ZgKg3-)L9vrJxkn>;Sl1ScG23}-0XSYhSjAZ zQ_?PlTU(mFDBrNoVZL8lWr4Mo*~^*@8%yU$q+R(x`m__J+54J}r%S`?(;n6R=V2%6 zImHgjrs+?t2PK+)G|xF#s?k#Re{;t3_-|Tx$&V$m(i1hb$wnb*O>YJW8 zEJ{xQXg#>xtj>JX^U_7x>7T;~Up1?D+VsjHYIpkAg28vpekeA*D~&2l|6VtE$z!vh ziJLw;EUrkev;M2b>{s5VuceEt(|?5j^}+0S`KCIDC9l(e75r6e_NQjkuhJ#;>3{0} zKmSaV++L3H6)UuD92Dj)cIEafi?3dx8{rUS-tuCuwPV8T6?%mZO7m8)a&5~J>R0I3 zJ4BmnG}$u5F|lcep^amrd25R;L(39%GK?Y|SD3dMv1No~QkM*qLdUh{ZGE@6lqK0@ znAJOOHgA`-Wt?NObB2Y@&^+_@yS7X$OP-R^HDc&N^9~ocxI3o!Wppnbns2W8YRj~; zl!%O;^+SuywVG_5<+waKqnFLFa&v8qt=?tJvorcc47+NsGh(Z+W9sgVeucyCnCtp( z4J=D7%otEV?6G;rq^-e@X%!jPHp6So^>%FyElaD;7#K19gL$WmTf-gGUuW1B4zD%W zf3-ELEd9TIa*4&ZWo0XLG94nEG%Pxg*p}p&(Isu-Lx6Y*lvVgb3%c7F|Yc-{rV^cjm-GXE%$kzT5YetuD-*T<<*7qFd7TLylP$ zneH|t6&BrhZ9i6)Rh{V(F*3-a$Hnahj%!|LPAeR#wCMS2`{}YZ^_es3M@Cy%HrY|^ zxVGuaSvI2*EqYn(I9Ik-XQgMvs1+8yN9?%hxUS1e@4``QE&BNGs4QD&vvN-TsLd9A zlXhHl%ywSsYvYn<(Qns|n`PNkR{BS{9JJ_vamQW9^?oY@3tjRp2E5wwplp4_%6auJ zMHW^~c0O_3ki0V3W^}oQwZ+coWgD_r&W{*<)xu`P&R32b|KG=xEcu_FPf|bnvBjXI zogW=@DprQujH$7(-L>;;Sx)uJ$cQl?ECyfPS?9Rv_5Z`(yN5M(WsSr8w|?zq2~+{>4VfQPvmr|s44GGhaSbDpXtTILV@Q~lck)qyh%~rJ{)hjYN!_xF)-Mu@L@-h`>`EcaKv4P&FoAYuN7W1$?{rJh= zzDs#FMOO8&YU22%-aQZU$`uQ`e(sed&H43;oEyV)(@)&*J#Z<1gJR*m;rSCM9`qi1 zkndFFa!wl3hrIg^CoSq!Smh_vCx(Lhjy5mqQRJCVn$w3Bea9~?>R04fpUj>ZPU<`H zV9}srQP)Xp`bkUQa8kjLqG0%B!Nkd;zEjNwql(2hPA*A5Rns?eso;X5@ZQPNiBrve z&ps#^SJ*hG?CGbw`_3jUzOGmzKUFz#dZ6$5=Eb)ZMdnl0=_4om&R<%5TTxtnYQ@CJ zrM?#*EdE$g(simK{mjk2i%ErFC`yMK;!_?4pU#;Kb?6)l8vpj%TISqj0Wv~wb>?6E;XO-NAmUC*X(}%(vn2w z@_VNbOg!Jb`u6$l>&J-MimRD#}z=%SVn+oEzBvtLCCyWsP}cIQ{&|-S1o~ zvMFn;M@A;jU)ufNgQ9X}UDwFj^cQaKem|+$p ze|V{QgL2iqk(VZ3e6agB4~m`2dd`_E=@-2B+)XO!R5r-ZOiWw|+Vg31Nsn^1`OJ;< zi;6v;T`K8UuBkrr`ozVgJ)b`)8C0(AI`d}wCCi?BNu@)|#^E#XOk673^ZVw~QRTWD zXWmaAtJ(9%OQjc->+hZUaAK@^&z~NYjw?5Cp1qrXxqHvoNoCiSP4Z_yo47o%=f9fE zT5l`^SlwF75g2gR+m68@rzUI{oFFd;XU6#23mbV7>gX*uMgucmt?4Jn?1|A^?~Trwe6_is|In_F?&O@6W?bv;7wVQ~Mt8~2 zMowNE==W_|nj771J}b|dIN9$vw$v8At@^BLa^g~dz{92G(c8PuYBDBo_6O>gIihzA zpVdxI-sul+SymtY_A^NF%XXj5&J?NkLaG5i@hx42vY>As*j`hcRk;b z@y5-)b9KwVh(0j-^b_cW(|X(bYOsBg5ycEI#p-oFHB6{4%)Y@rLITyocVbcJiY;`&wF7eWiN&#*2T;_|5%&o5ogsr+Vey zi$6^M=E1(!hpSM`IOhUk`q+EFOIOd1xgx*7occIue|t;4Am*z10>^Y$vA<)iUJ~bC>?pKe*58*|fqQEvL|>~ z)wY<|sxPXhKD)Gk&%@Q_F~8`#s4@NS=Kj69HIA6qhc9ZUes^d8zLquhF>lPy*E_mU2rc(`^j=IyRaR?~gUfnie{W(;vDIoYk$n9`n9@ta9oP0|%aO zS$8Yuj(M!w^v9D2&X29T9rNqzu@zH)yma8jhwDC$`Jij8!SvQ4_2yz&q%5zjS=+n}LJBYH7}my>EVL*!1nmgYS$r+hTuT{nE(Px0epS_prG<_77bz zoi+XC=E3)MEsoef4!?AM>MwT={<@{5KK9ESFI_bK_5Q&R$67YT{^{OJFHQaR!NK1= z{O@lUd3j>$yP!j#wruQ){j>Sy8>YW04t+MZu|M{|s$YJ6>TgMhK7Y7zF!r0Sm)|r! zupGLl+cXsW?eNR*Og$(%^!t`gqp^Rv@$&nohc$=(IJW6R>|gJ_{NdEY=0ksaxT$qK z_B+licTL}SANpFi`FiZ%BpOg{-$gFBKC*jSH7M4@y?-#Ev;Y0{&?e+znT7i|IiO(t>4A| z{oX4-O#S`Ap}#+DMQY?dPGlnPATiI$RuhVG=5-W2NZ;rbsA-mQP9~uoWL|bk)byHh z@;ad#^!naOsTtkl!c1D$AZMQIKW`M7x8?u!8%6&Ac%ew9DC>xMUYAT0dGcD{by4w= ztc_i2P1Mb6doz7&k7Qr&O4Q7{f9=3^-ikI$Aw%$B;&Q za`K()LB&UFH|`kK#NV8JKQp-Y=!(lb_!l$@_b1_fnJq_GeZON|ljuElH#210(S~_X zUf0Z3Onr7eWZ%&>8=t(TNwQ4c%baoQXyfH4Z)&(#C zjyBDE>I=>MlT+V5el5$7?*@kW(H^~)g|()#VP)eAKbrq-lmE-TbpN+|=~0*ON}^6) z^$+>CgV&#Z>q-yJ?>l0FUzi_)SFUstdKR3|<{0wR3|2Zo#v&Wx#YoL(ddbY`Hhg%S zo@O!-Y+lZi{B$Cey>L+~q#KtoL!-0Pvu*IVBPhe1lgFIJGi0aRh*&m|Gub>tUaEmm zQ@Qyz8fI!ENPSMe%^Tosv%&U6Zn`0F0UR9vA=>XR(b4$*F9JW79}CM*hddGnW(LGB z_)A54 zkS1ZX!<~7@*)AhW9X=*j5G0z+_3oqf=AC3)^#tk=(Iom|k+)=-tl*|By))0kMzVt5 z*^y#oMB&>t0>M(=!G4>Dj9HCg*;9p?)&)jO%wfT&|O4VY@iiamjJ_mGN?^Gwhm-skvrkN6Vk- z&(n97`}i7>^hv>y&M-^4RDJE1QN$#>N;y#y+||?`9CyA2lgh zw%uxxFWat3gS?CN;N{t5>Dx711k0k)7Rj>C#ooneyMWCxV7&qz&JjN;8QI~9+5yLw zUK6x;Za2REd0($6-Cbaj5BFY`Amn`R&eGFF zPYGS0$hzjuMXn&n>n`R1mxSI|k47ndF#IbVz`PzLN%bqREOB0($hApNhou!u8PZ*O zZZ=CokKUiRMX+5nx_HfqYgccc!d;&uf1~fM#kS+UpT4WGy9E}zlk=vYecC0sDQWAn zx+M&EuBGK530 zA@4|Ep0W0bJMWra#<1#zN7>^Rp;7FgE5uDBJnp89^1`>Xpj+7 z4XK^+y+zXAJes7#b!-t^9(nwZacI!r=)3fs|F~?KSV{|`AJcT_wR6y&+>_f0BYQ-g z>qaBINt=CeN7yy}kh4CBy}}JronQ*O1*iOu`{#N0{mNUiVt>4ZD?9CnYANbhld{^= z&%BN3#}-Mr=-gE|8tl{Pg%xbKGcVlfPh+pVu;rG6Ep=xEpB`mE&)f0tO_Yj+ZMIgK z`dXVLoN{MKWD!@Kd6vD4`^%5?3XDLU>y+)#uq0fAJJKdkKb?EMh<$kUs)EUQPC!FW zJf6Azkqs>TL(pO3MHEIc&RwOkiNz8nBn6=zbKfmuKY4Z>IR-BZjIzjcf(*rLb3aE* zSBxIsLR;25+rsX2`x%Rrj4-xMMqP8(jyQAEgyZKAFA99sr7_B`bO>^VN_YMKT36)6 zqNSD*=dE%V`P(~*s<1?5~e0v`Z@>2Izi!=>A^ET(` zIYFCps!K2~_%tNJ((p&;oBonnZe(N=Zs6j%DaFiI6WcA|^4xYK+Xa8A-EcP%wF#td z;nkuA!(v)~&7f0|Y1B-h+yxqVNeTkv^^(P!?~kkxGIDc?E6b zSG3frv2g6?<0b6lY~dS;wH?|FL8?=O1jpE;-ZJ=WC^tIp`~;Od=Z+$8)QOy{4KOUp zBK_XHNihm?2H8c{1|PL?Q@&b4L8TQ?f;B@xq-GVp#3cCeoF66oEJrDyG zmdK5XpLfGOjiC#9CBmzHp8$#23ww}bnrR;$9(^zt-L{{t3H2y)*ZNdSGg4 z0QQg#4=59TVQ%Fau@pvR7+-)B23?I-5;LLPS~QH@GIO8Vr&&CTzy!yi+LU)yK#;Z`I!rzF{}_u zyYLAp;-YA=eyI0YZ(j6i1UZsEf{BY)CF)^WfkjA5EX)uNio<#p zM|#674X4;vB>9|Tbf@wd{yD@^%-V?S01q~c{77eBw=BIktXp=x6YiN$#CN*|=@-IE zJ#h`BKOy74IO4Zm}FFPB?8a^A=ea|?{t1w(n%qHgEAxxyo| zf;3^J^zBIr+|HTAna;4q@|D{`<}+d|qyA&0&Y&M>tM^L~R1o>_)OtCMH;aXUGXu5y z^b#-?ao)X#2BG=t%gNCUxhI;=9{oVp#)dAJ-WkS#5hW~5p3YW7JM7Pgz6b+{96v$e zVVsM{8$3obq>HST+gCpcQy^;wq#E#(X7-(G#5SOMvRA{Zr}Y{$XNcZTwOQYR{jqUQ2THw~HoCMF$AIar?*JR5^7kR@&f3aKE!C7qM*hj{_ zWrLm8$?|3HD@ckubDeom)l4}|^jJbEXe05i*ln@HJPqn`?$aWC1C|JBqt=@;EVw-7 z?uEq``J-_(%TiXlRyOo57A?d~A__ z7V@#h8a5PmZSHBf+XQGk%y5pgbM>|9VokSqXRs6>_Qa0B|xRO zpVl7t&b{e;!B(qcu+n_n&K?uz;%PRDy1z9j#ELSpT@ML3DYkemy71w~W zaHkmYhqp%vpZ_gA0?7OkeH)e<@(O{mRE#)>M#g;Nk=Mu!h6oa_;zAM;Jsd|Kb0e}>^dB9CQ3F`bSahZ-pL@(3}D#7uUc?ln2E&JC4;!7;eGvFDb0EoKP^OG*) zRlj;wMdPwZedZTl{%?Clz1Sjry}4D8f(nxC4pO@Eoh5an;&N46I#k zuYkGD&vhH7Cr!fi&;sdyIt%Vz|7$wS$-O`6EW8z!Yu4EDyJDN_E2l@9V_yEJy`rw} z3X1x1r>DpKYkU4|KN%(c9t8+;6gyF%|8@5dm46*A&>yRx@BQAZ z1^Udp&Ehjt1^P240A4Q8zjqW8rV2Pz^jLv@U@u%Z3iM(^8hx!m|LV80lBcc~=n+B4 zfW;rI7eYILiZC%uXUWX8GX;9!d9;3{K>zwL;5uEvffqWGnbffYJy8DMIz&1l-orap zp#KZ;1a-VX5A42`!T;b>g9ZAUj~cq4JV(NR`H}U!!v*>VqM8TjIRwFe9tufdmmDhK zIMD&phK`b*@UQxbyuAf_Iblbgqy%jz)$r1iad>ITApGgR6I?&U>!bS%^zYAavJLeW z=<|ph=&(vc9VyU*Fc(4(kUhXjP!4jO{Q9lzrBK+H8&kApCkymGFRS+|hR9oJANduO z)Q|dA684?~z0|~*KS1UJdT4M*f&Rnu?v1yeF3>mKi_)-r3iR!<3!d@bU7&9fgE|Ku zDf1~6{DKMu?jtkN(<(OF2~zGPZ=)XaI@$&J9@QvvLt1;bfY()^pGn+lEcyRGd@$ih1Aa2$2N@3k^Jw}X zKAOCiIGuVm#r9^sdZk<+BBD>UopnTq_-uXk>)f~ya_g&ioqDQ=oi_V$uEKwkx#mVi z+&*r{(zCYc{jXO(|9)NkzSpYOTwEJ>AZO~+uQ$vKeQN5{zd7~NH#^f5eU{^S=C$|w zbAlFGj^3|}Q~Gb4xZ|)zLr`r-$vlK1Im5$dz2Ty4AjZJXintqo~WJ2N9h!zGHnuRrsv z_ctaVzBRN+aUk#1qTlSd3H=v#sQ$NZx19}S9nV;~4#=lwswJ$`od9%A@C z$5`OkJw1B&(!TK}DnN=NaCj>lZzkg%G!8h}m4K-_LU?yL92}lT`aq%?LOwV@aC&g( zz~O1a6X33c^8$w_jCj);ev*t}2N(r_iv+g_Tr}`Vhp+_PY;X#2f#4``GH^eF6N3u| zmj$jE+)Qu=a6)kT;O2qz2bTjb6r3-(qu_$T`GAWCmk2HlTnxA);CSHDz-5Em4K5el zN^o3oOTjGzmkcfn9E?mKdHa3g{gaU?9!4PsmR-UC4`0tRzd7@t(dU23#MmpvGx~^m zPWmVHh$rf1It@q=L~i|*=*kQgj=S@u0yd-VcpgzAmKe9(LJme7i3nrR&Ll#062!tiY^g*K)mH4U*NeD09p+UzUZTIHAD- zo6b4gGsD{ywlF_qL^>J@V|k;V?$f+z{jtwzmrJ~|j~ZZQzPpDo=qYxjZM)#D>tWvM z<aw!RrK~xzedoHbYF~_gm7bdZ8M5n*zGN zh^AllL2zE;HgNTVMbZAQ@>i8_!VfX8{B;fphiEpaXb{cI(B3mRfVQN;_NXtL!9*@S z^xLRgS75k#d{E^*J>0CRZhkb{{F(2+MG$rHg`lXLXJJD4Z(aUpM-;XnZ-F1pGZ|#y zTMXYju)sX>FAGdygkS?7FA%=jM*QcAjaP*K_ZO4I1tvUyZv148z@P3A%T}$ds9syW zYUT7Y;V&$Iw^!8eSA(LyygJ=0aAW>#Zy5FSdj|T$yVD)wws*g05;yc{nhsOu$Mbl+ zI524*&HUS7Rlw_9q|YP*Ghr_5Suq@ZG}(nYGm)U44}oXuzuz-&XXG#92zaMq!9Wts zM0AF9un!dInRs$e31GH8u$Lf09xX-Iz&ZhD(w^nV9+-~qggJdXDd4&ALf|5Dl9`O2 z0?sH*V;yJ*nJa6DnKGWmg8*=h8rn!oc#u6$NnZ2+3BK)(TK>VamtY7jL*S_2eZM1F@hs&dh4)h)C}C1KPQaBjnlo{hEv zaxF~tw-=agXd@uD14kE3^NXPLVueQeI30eX85|EY) zrIiA173jJG))00fg%(u{U};c}%2fhX2+IeXij6F=6sZE~Ou#=|H4e*$3#by73U*l4 zECBoxa0RMNlt&JsEP&0Vn!#ICeG6H$ze4#?*izL7SYOE4e^{ii>;ab7Q7#l+0?&6? zZGeL*35cl&vCCm$l@D#PEc-gDgfs&zkPIMdj!KUffmk`P+|fc6;HUpdqL`wy+=YSgK0AC}Ny?}Hg4Jddal$Qy(Xw?Oo z>QSGL@vM0+Wcppr{K?Eh!uE7{Sma9Jnc}&e{uh|CpFu<7{)lgaMV=9qp#Qj=AecY4 zTEKfH6rvJvVc=v0eM=5Oj}HM@OwczA2;q-^dKlgD*Y!VAgx7DMNU0#aer)}5F(l_g z-ap}d(cmyW4mZNS)^FAU5JfP*|BZm){$*j=2Y(e2{8kb8DG!DT5G{Ir9!1MB=K@_qR1dr#1vjd(*qj7k2 z7LP@GktB=7p#wu2okgbs;6wWGe0%`qLnFPtX*?bc`S|z%XFi|Hg(RL2%+~27Y)#U5 zz=y@6dwj@BBbhXBZy#VlhZ5*G2i=DScu3=N=`2#qV ze1!fY5sU8Y$MEu}`$|NCETM=c^a}`}3xfjaP&mUYm>wKLrw90w^njp1QWzxU^S$Y$ zFMY-giNDyNkVyOm0bckDp@)**T&{oUj2Sa#(kVJ!8b%NIkBA7542ukpi1MEq78Vs2 z>hJF_2@MSmgGLJVrvoVQ_YVc(AxaONC6USHl1TV)xuHU#oEaez3WWegcmgkBs4P&S zkkCVC&7>=%qgBzW7-e*5WMn9ps{l|)Mn{K+Dj`E?C|#%sjfensKu)D1TBTA(D`I3a zoWYYzsFDGwg19m=8WL5i&_GfWDhAn9z(^Pq8WT#7jfDhE29AUjkOTryoe)Y9#>Ud4 z=^zA@5X%&kfW}RTX>mv=J=w%6p{E>`Do~+R3h7W&pa(t>h}GVK7*;5xQiTFAt-#_R zAEA&g21gf*G2dx469>QpKJ-ALH)N;N)pSx3s8)CjH5wr)rfY(NG-8dI%hf388qLhv z8jXe-IWs5-5E@c3Q>avG0t06&XGR7FM$XoR&jD|PNOQ9jb1u27+O8HD_VED{Hdd$q2n3;+>b0XuEO68n6Ty9v{oLOPB;H#uZ zN0X6Stu`z!P7d8CEIK}(%T+295)&D7=}B5GBQ7CMH*bDoQe52p1bRY3!d#%$Mn*Ei z5)!l?59*g378Xv1)8peId2WiHu1!iH^}$>NJ&A5G&LfkOlIEovlhRVslMEST21JvL zNi(%ftv1EiHzkA1O_Gq3q@;{M2FYOHa0WReDJhza&QQjXF$R;#lo6OoW@ZG!hhzle zD;kz`~9kO4|EiJ>F|k^oR91vAVuRHODm+C$gE7OBk^&eYRb^!*B|*-Z7(*th3RcCa$e5Uzq^wLbDJHBnGbSuN zJSJ?x0x~-@+yt>4a4}(FnIr@;VN4(mGbAL0g@r9#XfPQvGs*Dq@P%A%PHryb(_&0! zLPB;nmzxr9Fc=N`+}sp0T%VGiX#heQDYF<12AM%7Gnv3kmvOl`Vzv6F$Ye4FsgTWA z81nM+6$(X)ViCCrh!u(gL*ODy$|B#zwxHup(&z2vl*W>hs0k+6ULu6#6*&LZuY|byvH$&V6bdago z9GM@Po(?1HLRNmHi8KQd8DO%R;VF=jlT%!5mPlBV3`udZO(K@qBsN=QLWm^5mLQ2N z&ag>}B@&4(HPt3HnQXR_(o$1GX{jW^Bq;?r0dkf~3@k}1tF(kw5(b~?OCm|lfP9kF zoK#ClNk|CXrvtgI{`+7t^P zAb>0-W?yqzS#)ek>*Cn(@)DE9H>|A0R8kTf%jKFDEdr`C;9f#na^U%vV=2#x4k636 zA!Q+?Hiu;8m~ynNG7@~?TNV-$4Mg$exIeP9 zv!Sz1!!ZEPpC1weouw=(Bt``{VOdOiS%}F5NML9w3zS$?Df7!zIjXW0xLZ`rRl|GVl4Q4@qDp(eooZaMdFa*G0JEN>3S+|S~)>U#Vb@};RZg7PzSf_;-zH4>53M;pQn{TbK zYV);~mBE!2!CGy8C4gGr%1Ynh{QSzQDqU4@Rlet|%dgZ`<>v=ia)Y@*v^-g>1$yXN zm6cU?(q38NsH$2>F086@=t;e!qRQczN6v#Fe;&Eq5nRp9uSu?S)Yc{?Ig*ywa=F=L zw!>j1tqwaLgoUIvl`c7C8$CDP)Sn6Hg^mlarG%DuGOxMwv(xj+;p{E=Nb| za1c+%;~*Ji@syrOCPLH!B0*5GJUMvbLPv5XzVa&^jzY3RtF16<(;Rilj^*TX?Q%z@ z!$Mjd4kNe)07;E$AgM7eEjSHJIx84N$~UHgxWSkJqy+>3 zAgnPB3u;VrU_l|kf-;S@wP_w%0byKTYbPD~b}WY0(V9paL7{05M;!>{&{jgaW4XiP z0IAZ_9L2@OE0!m(gltJ<5)=V?#A2qUEpRMwc(iHEc4SXWod)zPR;FR~K9U`HL?1f)+cA>r`A^DT02sqw$K!*@u>?*;F6je z?+Fq;{=dTcsi_W7S3K!RNN@n1BQ>=$A-JB5uZ)k6x0CVq_;|9CSy^AZ8u|~HTkELJ z&#$d@1UHbufT*n{gBx(i0=*|CL9cBfYaI<>{^4+}UcDNStE+1(YvH`Mva)*Z+O_%l z_0R=s>x1iqx!kI%Z+#ZrsfKuT#qlZk!)$)*x0y{+_Y))=1nUrTU*K24I4K)$qnl_taokP3vZ$!2xwx2wj?i4}Du!=+vD;0$UCnN|Dk{i|3U_mbtD>T! zs*0>~HCMsa(a}M6bX0XTcXYS_-m-<<;%eRk*Ve6D$*o(Ow|2U=uHU+~tBdS%HFtG& z)sb~woz1Rl68-|Tk!)-R$+nT(wr<hfe_8eDr^;&YR3xayGzJAA(Pd?ey z)E!60#Q_M=R?xtX?iDMup6Y(8+vVEqa&6qWBbW@{;Q~rfOVJu~4bJ3pZE7#_RH~?X zQ_+*IqN1(M2f-^k*;k3c>sq8@SvRAK=N%)5ExQG!1Y*yCzD&Cm9j9nqa73hg1I&`b6jh> z+gdhn-kb%^x4E-3tG2inVx66xEr4|uuj=XP!M&~)!p@#frpx6Cdpe7Yp>18YE?4cE zmX?+^B)OxdxTR$Wq;~eS>?n2@yGfVZpNKx=k)L(6qO6F4^Xk4shdOgyhYk%MI<#*0fnB@$`qu4!`ssCj zhc-5mO?|Fat9Eyjhr7vj>(;r}9X@>I$dPsHj;wPXJw_fo=2}gzUhP`9dexyruASu0 zollWZJ$3vU@|kwB{VCUpP2{F^Ppvxi)b1f*)YMIO!?l^*ym{5;p%Zx0(nI$24DBvE z-cr^wJlwLokxUz2wF<_wv=b+G50$MTSCpk~+?eKCNv<4fDJvU3IrL9AeB$H~Acu#_ zPCUY|I*4P#hYtM<+3}TaWLw*YwpH!zCr-3&+{kP>alGY3d;7tLRV&&~w6(RYI&q?9 z<3`vZirRfR_@5_=oTb$jKRr*}{)GSb#Zhm)7Zla~8a_|_pKm?=?0#>#U)cgbiD&m; zEym|!G{hmc<##*mCmUh$u<6CXbP}8y zU!A_+)YJd7*@btk{_k&gG2pxmnwSO$x`YQp0)YTy;3zgg=s^i^B0i)Q;geUK<*{CI=|BQs$VuTZ=7?xC%QKu!Y438=9^V8AMGa5-=UBsUN^AFiQ*!W#6{ zQ6MabP*{nZQfS9Y6cWK3X}|{w;Yd<|%>a5RB0n;qi-f{C3?hwAX$T=mxGIy-_#^ne8t6a(-GUBf63pfZ$gTt}QFH<4eW;Z` zaR^ig@^bo!-5@uih7n0kFlT_cz)%482E@lu?>RhN1Evpio&LH-sV3k0O6FE?!a@?7i zGzP(9uHjJ<#AI>+E5z1GQa2kV5QHy-n!_|ObL<-lZvsY4p>!i#$U@9I3be<9$`C@L zhIkk$ul^LZLvQ1_=zqOcux&$^c>55dwI1u?T8Q5u_i}i%p2iiDU_)>_8!ue#rBJ z_ajPq3=sG?NMQJunFS{6BIyw}$Vd1}lOPb=IY`J1mU7vYhHiKOJNYya9jaKF$b3~G zg8pkF_A)>n3efPgu)D8@h72!*KBLE~?%n}6Rr!-5%QWJ_e)*QH#U`t&9 z!ip;4E!kk@%pgS4ttEsA`V8CwbfGWkjPRx47NOISz`g+5=@GSKj@80>kP2&r8QH-*%0%_YM8VUQG}0JRkQ8&p#5Du8k8^P3-!>bDZ&@kQ$kAUO9_d10f@^I*h@obaNqh8EXuB7 zh*fWMbty^`KQsO@NjJzR$! zcz2fn{)njG9u1DVe~`@~}t2yy?D|-#1?WHg6*GBm@QN z-_pPNX4=34&Z+)$Jotb2co6>*f>;Ue3^w9_5_|ZE0mK6uc-OFZ8AnUNy^M|jk6{mf zFF+^W!uU28c%2x!kG(J1;CXUw$0RRvz{>^q4JHeP{`V&IzsJ8OA#wlvH#?5oZ2#Y} zqu#zA95r}-x)VhsWlQEnp{p2b*1hq4+Psww&nVfob`CVlVoTq}UID{zQcLdymXX zJ^{OlH{f{xeNw`H3z9x($UcD6ct9~;$EFUy0118pNB8dl-P>^N{|;Deyy>wkp}=n4 zV|wvxpub2u7*`88x6lRB!Fw57i=0JYfl4T0&!F)*MQ2aAE{Rcc{9!*?;fXy^AI1Q!GWQI!lG2FV71?_QAl?@*gTkm`hLl>Oym zz2P^E&_1v+Ij_3K=r547(eJW92CJHKs6{6jzs1UDKh-7X8dF*l&~Lnqkprd`0r>K zEcb_hvrqV+<>mj5%Wj7aYrTv84kqkjVjC3T!RF5{tzgjs(ZW93GD; z76;LS0(qQ(fFOQwKu{1b5Fj2um>(R>4+#nh3h)gAtgo0CG=nw6mlYDqqJ^^Xsd;Fq zS19BNn1(}x`TXF}02U`GG&G18%m;p$!%W)DnZ7eaeL1X|teG=s@`EApg&dF)zzd*g z6kIG;AU~Mm2YXSh0AGrQF*A8HIU<^fCz49}!8{QFz!Li&&;#HIMA_0XUa$!2#)258 z4G$NIaMo~sFnojgGhkT-X*?9p3PTal*|E=$3`Tq~tPTgQ;wfJkIDT*hKNtg1QTU6{ ztSA;QN*2Z8L`6Y>Jvzrn#t4>keB=r_JxYP1W+8Y#uMgO6E0v6328sq0@}a>dk31^M zhsF<%LNp%)C+t2xN*|@l$4Bf#gTq*SL0LYKODUcep$HCE1kdsbW(Tk%6e6*Yh|W_e zViby*Gv%?gSW2cq6eU;0$`rA3xr~a9RZx^NRtXWgimyZ}#8=bQaxb=qrl!dhwAl)p z0?L}4bPCzPxZSP5vnG@Q|z3 z@fgcfixg_U4=qY9^5LuH3L0Q&YPFY$2Szx2%-d5~LV`M;2j`}V2?=6wH0<%@04V1s z&7CWrn~*eDET5YsPL$K;s*}`eoldRh&-0lF77e-tal(A1JRv!m7q6p5@sg9}IPNArj6iRR4sg-)Y-Vn#rQ<1n@6s0rj zXvS2dB28y7DilVw+Nd+CjYgw7!I++IG=f2ZF%CkVQfZ6>G{oW(ux~UN;~)*9ad7~p zd!?t(-U`tpR+p3XS;5z+tz_<9_N z(sk3Rj}nY?aguH>@bqAmN=ScvJ%V&P<)c(j0uVe(&>JPjRxb~dp8e%XDSmibxOSs03`^m0}B(gSr=zEYvZ7=0vxB)Qf4jmN#%S+ zid4@}ktL+)p_k^w#3&PD^f57da0xWCS+4{;E2$Z>#?W$Taf%cfANq0}Xdp(RO#u@n zr9!WVz?7mfYZmfnE2T=lUIXGwH49Ue8gq(Cp;6El0!4~mnnL3%mHb@3Qp#7R=&dxX zRWnPkP+0ZaJiW$hRcNhBD^07n=Iisbtoivg?59`((&XdInv$QDpU=fIngSnA z76-^^`EW$8ROVBQ@~u{Fj8>K_GnfmqtV#plq)m}2w8_aD%9KJ&PPV3GrI=FSW~3B` zl*L(A{o<5lZG5~nUMrU`o*v`k@wi1%{!~M97Kg)8#w!c7DPDN|QbrXNL?tWWyQol0 zE7Zos6vk*Rcx+Ls`6(PVhf_%7TNIYULWPpAEG*EftkyypgRDToSAq!}zpxNEdRYwy z8c-_Lg>b1Mzh14T73v|afWuc5@)f{}#?L9t0eYIMAO*$@4hLuo_+}Mg3ON{|E~J_H z@F_3@m93DLqlcsZ989RvY74ZBwOYP8N1LKt0@R!(DTPaN_$uw<0xz}NT#!PkwKQn8 zSiRY-p!9l`Mxlp*(rYLPAXM^mR5^Trt$IDps<)m1Y$u7b{{*OOqAJrKP2|xjtpZMt&}Ux%|1Ab0LHj@XE?cjpd~@dui#?QsdlG zTB&6zt<*>>wZmtbrPP86OG_&%u&c0?LI{_CX?6vW;#f&V>9PuYvPEI6C@HBZnLF2K z?liHY6tl;$0(^iKID{a%qQa=CD6cH3EU8e~mjR~2XoL`mJd_Hc`Ui}e;M~&z5O{!c ztbl(-J&ymgI8VM(MaiS^QR$C@(&VaAT9u-z0*;KUlCf-6rB$Gns!~OH6+{45ctS|= zq*h^)e|Ug|2o|BL3TlZxeCP}~U;vLB4_po$JU9gaqAD#!aQt!LRb@>6Cn7luQj($E zD#ct+E=&osJf_FIrgJJRWfl-`It6Gv6w}^xfzv3=1Vf%8JOPei6f}{gtg6bMn`;3)<>3%=?%cVU3SW;HK+U{>Ctg0%jDuq&j{vVfD+1x6h6#Me!_LSwzQ;G_g&vjJMsvR|8#Nend zb=1^2mRpuP9JLNw1+C6fx}q-2B3jW}SLLX(udD&(uPUb171z~e71qJE(q6u_Zbi}Z z!lI(O6)Wli(@;pm2_CS%j#g8*0!QnnQ)zWzEwQ=|J`HuN?eJ$8g;lGo?S+M_F~#Z? zH7h((tBishnLp6`EP{V4F zyKe2;YN#tv;(QP;eUx6+xVEvT2I5c?s0h@srV5v|6zG;Nt%BMt#kHQU_jEarB!Co; zGH{4Vm(KC<#SlrEBdDFl%UOscG3% zv#Eu)sbv#fK$e@^ys2d^d^X`!@S0mTEnNy06Pub_HaqM<*0O0cBy7U|=1ukW7~Z_5 zzOf}M3)4e7#!xymVW&fT;LEK4?5LA0&iUFCAO zH#=#~PPa!IXS2P%)!pK10V@!&c`1gPYyv;G8H`{Wfuf})FR!CHucHbc7LAQKu(vnc zi`&~98{1t~U{urI-q8UiL2~hy_KxBX7q+x%Zvn43tE0Vr>()*j$DdrF#oskR0hDXD z7OyGBagX!(UB$)l1%n%mbrt7!Z0X#xrK1b@ZrRe=4#n8Jz_O&Xy}h%e%ihxB0e~WC zi(ePcgiUq2u?JWCR!^{XI`F{Yj-wC-XOSGD1Xi?g0t0JhaM)%6FPX7v0_?#%L;bo#f3or9+0w`9|FB(f^1o^I z`p@Rl|BlTix>g@S+~Z$rMJIMe#(NlhcMcA^{;n0d2_8jtrf^5HSTPQ@}_QFbu#DcNzwq1R6z1 zN2AyToSi|b4Dqd8Gz#5HK=iSQlT>+I90xJjubxOnlpe(yb|D9y;vy)N&T_kaf zk=+x41a9PvB6cCO2)>kg7o#qUMt?fOETBnyAWE5gBJ@4S{@5O8?vF6ZTVqed^S>Z_NmzdmR_Q+ZjUgC5L@7OI1zgs zN#8`s$pUd5<0t~&b$bJL7M+A74t};87>LTTrsAlbEXaTaP6GZkRD#OEN=Xnutt^H= z@)AW<=mqE&qC!#RRZ8Cmb_q~NN{`~85=78-5D1l;R?uw8?mm1%q2HgM?Co;4- zi5@sXN2+#=qIxJ0EE7>|Cxd*7f(>dQk*t7ew0-v|k%_lf?j48=L|HGL3`$ z3}|2YLxgZ>J^{^)pbCR}L>WNUh-L=0x4A(;z*D#X4|``FRps;j`*UE93~*Ew1Z+S+ zP*hZGQNY5+#v}~v4pdac!p83I?k;TX?#Ax!y086wzTe;d-nD-B{(0BBYu&Yc-p=gV zv#0j#*)z|ahoq3KqF9t}Z{rkhwUEZZg)B5DdTAS!#v|0U zG$x6rIaP<(U4cbn(0mSxxqdhVG}p?NL2Z>L>pYl+_Tydwl|lA~dw3 zDdbRQlOZldMALp|t&znj(%!|C1!n0=Q^@2@n$J@>c!XRuHJYvLoop&4dWVlPnJFTN zw<*O#s^O$U7+2CjZG)Oez_kB=qX^Mxr;iBxEuYq+95EA)35HY#2oXD6xdG zJ=I-pn5AYL?@)M*(W1#AhMJPm-XwHx-q&*Z0;* z-W}*+=^PLHtf-WSeIi5VA)TL+Xg}=VV?-PIS{$V~nX+Z|3Yn}#c%p|r)ussygbX60 z6vLF)hN1DGQPAx%Iw_12b*2s(2#4?`586dlnUDzNMvSdYHd_SdGCA6Kn^HaWK^1LE z@iZy3@5ZAz5dZa3UL1)3`BHv)Uf1<`eQz(%<(srT*FP`i|6>LJZ{Ldk|CjRr&;Mrk z|L#lqjpxw6|9UAe`uE?yl>c)qm+#4A|0v=A;iY`8Y##r<6Ot)NJk!>_W8}X+y}RR+ zx@OA}knLiZ|M*Zl$?moQERMz{vx{BK=8IiSzIQMDN3|?%{`EhQ7m|B~_FPKff^LjGN&JiWOTXew4LsBmE6GJ%DQ2L8`z=`uvA;X9q}qW)j& z==dKQ-2W9Z*panUz2nP>CX>|Ka-UmU^3_H@1IcF_S?+DCS1{SRrCPlp^KHbT;rt)M zGk~jC>2hHrMXpJ3hg9yv*x4o~l$E?QNjX@OUy6_79^ipdwr%Y2rb3t7$VV$(V93OW zzp|vlWXF{s^F2N^DPL-|w+pl}ujHeQS=2ZYZQRdrh255qUV6UZ-iBQeWVw!!CZBGM zD38#@VDsj8<OY!qW^uFBIFx`&!= zE@+jJGl!Qh$mPdH)roKoi%T$`Ei5%nhs+KE6hjZ#*?TAEGWq_w{U7@5(4l)&Oq(`6 z`TYzZFhQx;I=D`D@J+jv%lFkqL#hAwO84I?dFt^D+0rNT-K=lzjqTz6CAI;kNsAr_ z#|FBYQrzBKv>u=OiNC2(H`2R;*!;ZsW!{W@ZIht8JYz`4z7Fya?f6GDA3Map|Uw% znXEL#g*zKBsk~jMrWOxfY;_y5&3*I^tQ+(Z-V;}?Rm>s!_j7L~9r}W#62;n?$$yKgWV#j{Ivg(J7fO)v9Wa*M&^Y zpg(#b1?C%<3c9`iOMxTfCs$(*twZvR0l4U^sUXr!-yuv$wD(Xh!zg8s;|dG4a=t>2 ztM%K4voVZo4*leie)Qa3c%$*})w2;)>+EPCZam;N0a&*VxV95Gw2K_655`*}bb}5H zslnMnD?sPen@Ek`pP@P~%5y}`I>47|Qr;A;!5!U*Uu4Yxn~aMtm$$gjhN!dIlk3#e6UerbfEUSid0O+xCu`>OY$DOL%D%M8nMhfl7@mzhsFv!r z+CES@t)XU5iPcz(jNv3*B9d_(<>n;fNu=s}dv|qwE6sSh9eBP2SSRDugK<(Vb_3~q zfyUCG6%GN9(g2SuDAiN1s0hzQm4i+&H9}r{W*a4|H~LKXTkaO!SYh%O(_i&C`XnILsz4n1(S%!GtpGv;Xt!7 zq^!)iou2GWq54m98a-KT?RiYVRGH}q?hyB=`~aPF+L@(ZsC$ba!qQFx{={q-Q zL-2G%g6p~@!@Tm!bPB(K#Za@P7ivf^%%b)se*UZHx)G`Ni)NoEF!eWWTKbpl+08R7 z(R>O>^Ho!AI!T(pEyIE)tKJrvYGy!MaeLsHv}_CWObx?Ynl8ZMwhoS{O_279o?RDp zs`8b@rpQW!q3<6GoJ)j`)a4v!T6U^r%%sYj2WT7%WbO#Nw60;#*_J?+ndz>oCP0fM z`}jnLuziCCbkwD0By7D7jAgvFt9c>)I#m-~Gk!UJk!>2miE1@LXQ+IZ6$P`W?I@L+ zZl#{tT;TAvZe+ZdITL*TYb(sD*dFC2`drFko5fvab|miYhJ~bP9U8Mr+R-&nySCKP(H95aQI?H@jIoCtVumRF` zNCh@QswOkDBCDJ#^#Z8;2AC+-+U~yH$E5Oj!Q2J0U&s)ck20ixs(hw@mVem82fua^ zJ@pYm1L+=hMuv8w?Or-An<98u??XujeL0Adv_>_wPnYvFepXj_t%HF#qL-$Tz^GBc z5X?LG8nqgbLPNJv?mHqCCQ)vEB3p>mo=V%irB~j~1A-R-OK7TYn|i@Algp2+K%=g* zdc1-8=QwH>Gy#^C6M^hg0GH{&W?HP@54>#&QS}EA50=V27sj42a#vHUjkC#nWe%j4 zJAj!N$Pv&p-4Y>|RZSg8Y9Lwpnif4agz5^pd=i191e%W`y~HB&oM`{d5~n!6 zAVyRzzce{onoLWzIz9=QN9`~|Z!F=aw@4Tpjf8q#C_!3Mw=06hKF{hwK$g`5JxLGm zMaHRp0NoET4+L^dMouU7))KGw;PR-mGL`Cz`nSpqOc5n4=lRYOXqa`YFJc-LB;i&7 z5K#!|I}?^8vAot zBGHi0LQbs71H7^_`fZ9zju7PL%IA*0`(ff$P9eq>1L_wiB_LBC6sMg(!O0X z>EFq%HzpK9`e?7BNWYd3hK~h+6zPPeRY+gpRtna??!;eY2O2U9luv#jtT5133NQ2& zh(Dn-FhlK>hq$e9d!gsr)wU8QV-wRsjh5j!?*`n;O-h4r;S{q4yQ#S@ z!Mw1e2HN&*9fJMd!4>-+xc?0}Bok{=hPv?Yup{0kHAdd!ItecKf$EQdi&CN^a&-OU?U)>6 z(}@4Q3bYXkm!*W&pMi5<;K^$nhl;$z?x_LUfIg+&OPPC>P1>3+B)Cp+U11K~GC@_vZUpAhi8X2|_oY?Fxqt-3p zIm?qr+n5G1&rDyL?b$u>uXM>zgOGXQe&*-;NML@L-D5fv~(<9Wmu&-2BL^_5|+E*j|} zN^y2SkcTy!OXFJEV;4=i2+a9GK%9XdzYw|lH%#d=p!TLAJ~-XZ@c$+Cc5Wt(XV?h3 zFD)FNA9a8t4M=TFs*6miGS|m2Z);P9WJv38h@HO)S9V!}+oi)QHnP_fidiLUx$*@t z@C~p~#;@QnO7%IHV(G4;ZWI4`2RML&>5mSRC^k;C{ckSfa}Q5Px5^$NUh^n0@iZ{{ zJW%EWrSzth43whxES`x6-*eAg#x&s~W2&}Yf@$GpWOQ>{hNpXFIi9ZS#R#e|JAnK* zfZkH+s9P|Nn!BF$*KY;!8as*i_uarRhZZrgnQ!c6V0#=%W4s@wt!WtJuHsNVt4HVj zjrfZeR!pP1(ia#W59CT9vz>iMG_3#2K4eM$*|@Q}v>xzy8IjZ7KR4mj0$tE^$Nyx1 zmoeMlNOsa5^kfiid7jta-5S;YcvE(~iJcK=FVZ{6{Ed!y%$923=%=)z52jUB-3*jI zK}{O}a%^S}*OhBnN&6lHbBlLkS-MyYLs-K^*Rk>Sna#da$TD_5@tYl^=+{0JVeHpd z=ZvO{=6Nro`>s```*hvh>!cvDA*J(@;nIy)Wavg$$->AF5B_n1s@}$1!fE5iKfmd? z!va}_K*zf%+A^r8dG15Vb|1Fw1YFZmiP?{la)nq;QIMB1*Uc!S`2Wfox7AdRY<1`( zOSadR^QyOKgBRwQZ&hklsSpf134|6V&lNLe})0mD+5bKc`LL98g-`y zMQ*&cB1gS9r4q?lU3m?03W_Z-N~aF^;R@j}zl zf2~KZMJaM@l=4?xff|kp)#>)=uI}{-2U^d!rqkYl)~+qhE7cnML48gHnhb?+Kv-dN zXW4%WlO-FF{WLD)0mcwj+#AJs42! z<9OPVs6ss8s1ikdk?9N#x+FzT=>=q8194cySz2+b8^IhU&*2^wLe6{S^e=^+{N<3$ z;480+^nYXsvChPnhZ4&v@%Pzq1C3Dw>1z%E>$oS&vZvKD3^O2c)>xM@kMN0X)Gx^?2fW#6(6kAaOGh zan`Oiqz`=1MzL!~vuZYGG%&MjxOg-N2&`&KQ>vh>vKQocKbl`LX;rNGzoy|QBFkDJ zMvE$kk}UP5o+?Kx)`>^$=)}Z0U`Tr)K{R~4*qM2l7?o!(dFTMOYaWHpUqWZo2&tC9+D-F#MY0%Tk)@>f1zg}# z2xZjOWNGqJnyeN`Lyn+dYA%{58*UZ1p85^{LQFbv-&bOLS!-0GU&tB$?xH2$I6b)V zo`mjype?jqLP?*y5_DK`nW@-*CG2rhi<8?RW-EG)^fKZwmM#wXR+JF_jkfW*h2ih8 zl9{JAt%Z7R9dJdeet84w!wx>hb?SPE;E3JNkUV1#Nz?WMIZ}Za_ketKpt1jVdPCod zQn%tsrS4vD?LG9UL$58-YNyP^>?eU==YZ7nz%)9>^bo!wm&qst8Gvu4o2P%o|Mf}# zPNlpV7vqdV&HYIwe!Kp_0_30a6H9UH0S9L4IE%M+LlaA@2kx>=8N<$B<8(JBmdv!w zP3h{Z_(?jlfukkFv>e(pKEAA~)BSpCY!(Fme(r+66>kJ*#kwK(1M0@Ak)2g(cyt~J z-56{+jnj?x2V<7rh#eY_BshaPFqPx|Dwc z@+KUcL@_VIrO$iRKtQINh^|iEyYmb~L`nqR_O9$xOKI(M@r$t0QK|*LfjWI1xN#fE zcOUpJy6cUV)}MlZ;qaXidFLAeU1RB&C2MV>JU1%OtuUm8Xw275#6PM=4Y!2DWD@qT zbxB=f9)UYyYfIeA9=PQM@ulx78sKPi4U6pV=Mio0uJQ&_*RFKKjpv?pL)E55sGdx3 z?H*GC{fHD_w6V-kizs$-PuSa>c1wkbCnW6S7{&C6fHebwyyPp+QK7(h>{Q?kVW7)8-u z)3gw|~>R**<^ty96${i#VVg?gp z`pD~v2m&TK(pf3pcW)Tbex76Z!QBy&y8={C5s_3K$XN?$&>W~B-f{t%Ea$e6r+BfxmT=|G$f-CfVZ78) z{09F9)Y04!QZFelmlPG(3UFhd=+2G%bBM942|-7(^J0J0t#Gm|CX3$EZxBil>cN{5 zoym*0M>_A^VbscD9*GA&B(pG-FpY-bhHB~0pQk~{?>HNRaRPfKBbE4t-t^nF06CUz zvfFuZfWq4xf&XIqPJGuqkBJX{0#y14@#bP#_Mb_cKHp1c^HzL?M{K6+_@^ro#&W9< zKarCV?`LrQ?Ai2l%|dMA){us`t{csGW}(Snhdw^6EV0?oRcNZP#N@GkC` zM2~G?F6nWh)2aCTVvDHWr(hWFR)l&zc{9qsY82LAy%`T-QSOtJX3h&kiXYIR0AMNv zcrr?2DAZNDPH)V2-V$Qn z@JXbcV~XkvXPoi>)~w1&#qV?(4~{tV66|>T6^;0HNgB6fa2$Um_4=s^P>Nk3liiuw zjB%Imn_#?d+6+FOaR^sb6o1rkj5LcyxBhZei zq0YJjULHWyA9#{h&xC#cX5zbcONl3EF>Z}d$t$0?XQ7rm59ECZB)%toX--~R=!~0a zlC1rD!pDQuaHR7g_%nOL^#yC8!_*^&vZa-{`3rH&q{ldYY1DY|Dr!8K%BoV=Nw{D4 zImSDr9+huM>I{yc)LBf8?9@~d%(A?}Q7J76>>zNt4e2Gt?gg}m3Fk!xib2lF3a1uh zm6dB3=oL`pEu{H(MXTo%Naajh&a;`2bh_Hp2e`>(Rfh%vy#@mwVwV%pjDL3GWo8h{ zE&a@XQNQW_*2i6K?m_f5;#&1V+@5%1=cER*K9)y@7X$RYge&Gbuq&ENh2*JD?mz$Z}CSgz`Vin(#(!c{RyqrK9u)X9v#J zG8E0oz@0A(wU}fPXU)Dmz5R*6$jaoMD$B;1#^k6x)R~qQA4lB8WD-w8*DUEyJqKWl z^rxKc0aW2!B**0@S+=GH${-;^q#qELwWD&NZ1-e7)$EcBy;)|*`Ma?M2iU{u<^bo@ z%s`|w&?_e}O9W04&FWSVm=-|hE0=1st!qk!FUpv@_`BQCY1Zx64OYZ$#(L0yL;scbU3 zJ4EZ<54B?wH53aH94Q`G*! z@Dy)8hG99^f_TAcK>T!|wM+mPYNO7q1!_`xom4M{ec>uciW-gGZ(N+kDzp#QBo8S@OD8IWH%mwCSbXK?v!5+C-RdkiSR~him zRMI^kQ`P<0edKp}^af+S`hdCPqD1|nPHrS-m&&!SkJcZBH=0eee%C55b?YVb@_Spr zNZwa0W?#iyBT{UT2&| zv5wB!JDE2c`+zQ3xhn{~?1DG{DLVo`_D7o7PDO&ScPy zRX=S69CJoPT9yLDhy(D10_|_)u&gxB!9F5C$4#v0b^-yn&_P*tGCv@n$3wC-eS|34 zPoCfrKQ8hyl^KK(HSsCn_MG%~r1j^Kmm0~hNPqfe<8PUEJf5 zP^%;Zjc)+{cYsMEF4HeKo46K4Q{T&bU`lojy<=TPxL`LDk)B zjAsmri~Hq(dDn;fa@|HL^hYmml!Ar$C|Va44^;0DaY@O!C`rTHXAoDBlC8h`<>?&C z+dgqQiXW^;j;5UOs7-0ar;P2unkFNfE-!)Y$CD5}6^LODi3h42XF)eJ$i_jR&tiJ7 z7A7Y#sT;2Lx05pat6?rwvD3i3`|xp;rmLqi`PHol(ER9PowQ>xZCvOpp75BpOTkuw zJHQud_rRu|#FZ%8napP>GE9`qM%JmZ#OCn^*LZ>dmq^)O3}bDE;RTU3eR1(B(`Npv ztTv2&t^rUsqZ!tgmrHf=OyMB!ePjR4LX)2Z{bAM*bglfd) zZIb?eZYCAo(0nl&KIB|NhB=;)2c^k%gu5pQ&Yis-MP9o==I8iZ5IW602BCBi#~dmx z6+saPsetD?kFMsO+I?LMB;`y%l0LNO1{jwcXUqD#W!`Rhv3r|{zGTA&nf#=AQM9}` z)6F<1>)e)D#SRqFtbJ*0k#l!~m1>QU;@$AxRM+ejG|xG2cXM|=D{d&1AMIheTYdp7 zS)HFzj(S$42jwIyai(Jn?Skj#UU+u@dBqli55@E(^JG&1;vkMe{b4ixo=0KGlD=urL?o`Wz?weC903k!q76Shk zE&=nXKyq)5Xh|mq(^x&ARc8zrhXATRO*W3W7ZM4MX8iibAdFqUiixHD>2Xx5GR-nx z>~KmTTN&g0JF){O5gm!h{0{3#&N$Wu!}CElLWcSFM#og1BsgXB?65rw4jjpY*RaCi3E17r4rA=aw&a<5AD+79=$r(4pTD5^(rt@SzsrX*N z4QcLCnybpnp4~=jnjdqZUd%(nA0#|lgoM35vZr0Qg@nwgu(n%1fD$8X$ck*L2Q|7r zP*YaKhhlREM*>@9#m&r+sS>e3v(`wIac$6^PFsXZbgekt(q63yA<>r^row9iu?>MT zjetC|wiXcgZ-%Te&D#Ov7XsrJA?Y}h!eoEoE$!MmU?ivE;t`q|3H>N$6!q&`g8N;@ zdS1B{MT!B*FN<-_iU|203Y@D9EM|)98r8<){k5bTa%81~s>4^fHmSbSvQg5q7E-B7 zO<}HCA_eB|C5bm@VO9$&0MEjJhgE=`(%knnSB1*Foc(h$*UyT1PM3j^#+DP#!8J%q zbi*R6Gw#H-FR<7TunPj#lm|{r1@cSTthxBl(PZ)7JfHUmvM_paz^ttU7GU$YiO^>mS9u|ZwizPy#&|Bns-~iulZCTGBG76YFk?E&FT1QkpXPQYeq=H5VhO|n z`PQ>oq<6zbQ5}0gh-tG4A(cDAFj%~-MxsO~x&T*6R!O2nuVnqrN)dW)(;akC85zQZ zOi|TTxbF+MO;4a)FL(xe?vj)32(cL>nfvM~7F&4?Bq=WD5T$3Wa+svj%mZ0j)G8U2 zw&IJFqC5WU<-!n7`Bh;l$%hV7F;db6I$rs$1Ny87j_w9hPXetUAiZ4FQA>(iF&iPW z!`Z;nrOq!Wp?>5s`fLGXtd1a1&o6s|MesE|eagYJK?Y@Zu*j+ev={9eAR}Hus@N8` z`EsN_Pqawvi?3Apdo36W$RLi8VX5BWq$OEDd2oT2&3^&4K^@{AGOkr{GSoL8GxDnnc&nX(@bnY$E?i$y7N5Hc+8tV zVb`8~w-IJGEKWWEC(cBFcfqZp~j-U7Ej17k^1$=`udKY*6MV0`Q_fECZdN-$r6 zfizP5p%QFze@DA3@i8--JICi?Hf&AI_FGBPnRVEx}|@S*Z@48%uv3J~Byo zkHMDtIBAtKPNw44adSb=iU`O^%J%5MAUJthO$8;O#XCkmU;|q-bTc_ zuyBeyEbr=eHo?^m6+=IP`C_PAxfU><5@VfZA@D;6f5v-cCziO$gzF>*Gol>OOVqB4 zn89|;9Cf`qFjMA95t^wzQtx0p?C2qVGq)(cb6yIN#poN`6qP}YWN-n%uORe((0`Pd z8X{DCt-CLKMbC6b*Lv#(Z8>ot&K^%lTVVh4U$fcSk@cVt8Be;9@oIKBGUtS&T;5k$ zq;Y@Wk-K?@_vD`DLfYAEz%o}D{6Bx;P(7HOo4$~9v}o=-(^m%cS{8VIX9W`7Ai5s> zCJOg}DO%uxP2WW;A-i5tvJ6tc&>xUewyU0)`;y4^~0Xum%}F%KWl)*BwUvreorg;i*0m z*gFX@AtMcEV-Gu#X|YTChjJ|28oqUp zsKEgiztv|}Fq*|=baR~to;?OKJ_X`llfT>Hbu@h?_C=le4Yf_3H!BPqf|r;c=#lt)w_HJxuwi zkXOnnD%4F>p_%BI?Pj1n`l>IDxIiCd97X(w%*VgfN~av+g^ixwo-1?uVH5fN(Fl)> z13cTSATL;&<^vYz2YMF-5{d(Pg8}C%z;Fg!J1i)R%UN_*RRk=(8xB6RR;N2lWP&3E zlcry9tV4H}m3sU*1mu-+(;GCVngf~;uiYFd-2&qDCCwn7TS`1_JFsO3#PU#^9213jHo*)|Ze$gocl4ZhO__|PAyI1q?g4%qFX zry9{yMHgSkeve*@HM1T&oh88>t<$#WrU9eB6{RePKNQn&Ok>7>rtL*|o^2g7h;3%c zZP054zrY=c=J#S9niUw#56kL@bL?b&pHN^3KP>Yi#`7UbO^6h4d^NZOJ*PQ{^!Ci-Ftx42tkf_(jE3~zo zpW{$|P1f%yIdWu;WDDJKMN|?$l^V~CRyWrGN7e$y#hR7U@AtL>+oVg~=u#EG6Iink z2s{XcF)GIIVS-17+HY<5)sY2=Ye5H=M3E?%dAm z2x&woNEM{dhcfTgE9vvx!+_pnfSsa~zr-sG5Zl&r1#osF;4H)JW1D1IsQ-2rkIJ~w zna~H(tC4(3W#7y)b<3*<`y4$nbI<{_im=Jl84L9|=4>bv%{ z(Vwq7S0Xs?E||vo%MTmI6d--Gi~$=qHKZb0CUyX>4S=Oh!>bs$B9jSTuwI9|Y&L?W z&G#9dOc|aaDXSB_r`@3Dxc`FTE6Fga&o6zmwzrjh zm#n1T$yH*w*-t&LKz8$c22dApXbQ_ZtNeyA!b^s~VTDVur#C`k<0e3exIevH0!z_x zJ+R^xj&gz>sfnWSh61{^&EGc=Q5b!Zu(F`lK&3 z--J=Q9Ye5R>;qcTulZ|gU$n({uu_A+RPR@bkWuFA`bN6E&Aa~s&a53dF}9vb8lKD@ z-WQ2Rohhu%@qKC>|l3DZT!+e(PTBZdE9cd=H zhYirc9vJ5Uly@X|zI+Q=@JrFM8ayL-){@`Ln=_jdMA&}F+au%l2p1&ea;I^7#5lbr zEVoF4eC#8QKjZ$Dh@&q)K^{_?@^HXl9K6`)M_g_qop^@u$U&f-51k>0_Twf{P9hz7 zcJkbVn8BYMc0_XU6y^!#>A;h~vxH{{&q<#5Fi$8z1;in-W#eITgBrk7fu}x?oCeEE zXcIs={zMA$1oQ9)O)lsdKhr8#h{RYTec?-lku8@>kf%$14MpC0nfLBe@rH6-;*0~PZ ze8o4I*a`UF3)nOrm^*`Ens8`QTjMuG%Kr#$l=jR%j*9j;frw$m&YvV!^%OBWUFkE7 ze;4}|mPo@R^K&Axs7%H9Evc*@n=y8J+ICb-RTs;~sA_ZMb^LJ`YEupYCAbP{y*wc2 z=%MY6`H;jxj&XS7KTJf8CI7Pod=~w@n_L_>&luA5}c?Lnu(b?WlIro4svzXo> zF`m_+uk1_y3*sGq#Y*a7)wc2jyQFa9+iL-t8UhuXlTtIwK_>F00D`NVe_$dzcltxU z>h*PlH%`0v~x(C6?^f}mv_!WNR_+{?( zm2kWkgLC9OFySVU=MP+ixepm=O-U*~p5G3XFO2J<>Y&!TPuO}&&EWJ9!hkGrf5`y^ z)1%ga0?jAz+v){gG1RlCBRg_^`Ym-2OLhN9d>w@*;mjRw%Q2t(=58zCRge zc^e!$l>ux>Kuu?Gn$1Jd$wicZ_uMMB#VMs%Q?1{l)__ePtOYHF*MpO~9RpW=Jucx= zU%|oG=veKNiR+A}?Ek2?DTM!dooxJGEG9Ge+~lappBcj6F2E+Sv)>KJ)9RC z5XBf879o=tGFOWa8R-LQDh#hMVyaA8Nc*uL>I^ERii83u!T{UKu-$}hqg>o-TnAEk z6W~EJpj6Z9;a08b1V=l3w!sB_V4h zn&3;MHj#YI;KIzSfEZfh*hQp>J${aXP+lmhIy()>IRnX<8Fg|dHM~Q>bvA)F1o){X z$Afc8m(>Dur-m;;<}3memXK6rDf!pSNM(|dve^uHU@47V?4P@ElwsQIBBnn+iK@LU z-mWxKcl7*=1z0Gi<%a0RlmsTvweF_Li;`9EWe{?U$a`xMBri&G$i95Lh-Hv;Wg!)ak%=Dv;0{uD-Q(;>Ek)C;aXsCqm{+Q@Vr;YY16Bwg>zDOXz{^K_uNC`k))ujUcOwa!GZS$K>Kx91@gn+2gNghU=JfblD+*oAl+I3tE~ zCNH8giK4xD^jPkPcjzh^ySnclAVL$Tz!xOk?xzQM-z5P{K#mA zPrPb`c~je+VG-}i9kXV9QXx1SynbTeWE!?6X`>s{`nfyY!*f7B0Qm{zpq!91cp%Sb z=3I_%wk#&-<~EQ1ausGXM;N-Qr@RXZ^dcX_q5ShByfTZAdQp;8PcIJ5^ey}4bo!m9 zm~*8tV1{e<7VcxsDPg1ZZC3hLMMaUNV(41VVG1yKx@ahyK&2&kAap0e)dW=q#!8K> z4NQ~qIn@AoFRM`m-ga;~m>#i)Qjj;x~ z-5@Cq@}^s~Vcu=X8;O;F&t1pNICl>I|_9WA>pJN5A2vXkoJZ~&a! zOJ4Bw!a36S4!99WDSzuK*{c`HPsO#<6p!7qGR% zvg<;7PFlJfU%lA27n&o2RLBKROh^yf;l37OMfrBaeU(clhLRlA=3ipA{I;RhBJ6;(wzNicRv4P}@5SXQPS z@xa0XfQ^{Mxr2a_gDDI{p(3OvSI0RPiP)2!t;azXkwOn)ey8+^nQlS;xb|h zMsx=DcY*cXuy6#qo+8*cy*6WiRYv*>Bds62XEbf2yvH0;>#~KK#hv% zhRC1#n&TxE`3ADxryn@d^QKIm=9!}N_=+2OyvsMd@$ht;>A0j*MRb?1fcFpBJ7u?J z57RNnKmDZ>&8(9;_PQV0j&$P&g--u1JDIS*@D?&Hrf%Wc$eY~{XzLHOqWcUpai04! zYCdb7p%23`-+E}zqm~HkvIiMva3dw9oxDT5`LQ6G(gTsXjq0i+m=aaZDF?Hd% zTN;kB9Q99nK5ySYPYZH20|CD=|xk$47hTS-hUg092M>;J%^*Aq3 z%o7<^Z20zo+9?BKFT*fNhM{hTIEcx1#CvJtIqr1EOdY{&7z_FICzN*Tbx(K-zJamfVZ)`zRAYan=7tMLHhH19q8P!|UY)nr>%$NYAi5TaWBC&E`aRo0~=CIf*NY@1;YQ zWu=scH|17Q3ELWAi(Jv*WZQ9|tBIMRJ$~F{a`i2oYYto8^iWE&37<^K$>@--@`@kx zs{1So)>^gC%;hNgjOfx}hFxc_!F@>U)O981b0Ri*Jv|{B(r6<+VcH5))OKL)4nQt7 ztGWk)otOmuE2bn$s56wI)vKyN{USE!3|30#?_ZG@h}Wq4*LRgExzh(y8Z#Wan?n`) zZmJ`}uCd{?D(Q9v`M<~%dTBjqjM=+gy`52~(;zWo8v;`+Z`7z_g7gNPPqZqGT zpD@FFT#&u@{)@PF@%T7P9Y8zC>rZ z&$s|PQD%K6Z7P~L`WC?vJNA>Kv)2tqXJA{uh35A9LNR}`G%HT()9V~J=n=irkTlEp zG~_4w5t0Fmp@NxS$}f83&?Ae#3tDmnG^Nntx_LT zqs~a_!N{r>tPr|D)qyaU$SEhFOQoxhKDH> z5#~+H;N#wwHh$Z>Urxuq(Ph@7blJ5USNP7DV?yqwoRZ8R24nJY82^x-Wdw|~30xdS zPY)hTTEPjVu{lK^yiF6 z<-m-}W?wxMgH|Pi)RN6eeZH*{ol;sfv>P6buH8|7;p5+jcHUKpIld(Q&j9{ zSTwu+hijuKUO)I&8p1}r@Rc%O%>NuhG|(MmDcXGNoWik~o&zMLkP zeco4uWGm)key1AjVxz)p(>bHcWnXPhR3l{3m=Xd%ZoO&}N)Oo%Pg~2K$WH6zwN%vK z?dy2wR3-E-b?g*I9ldM0tY>jQ*#Lo#pPJH)wFS1(jMCX4ihstQpfbkyN}*cwHxWu?!D#E9bvTQEf>_YR%&>qSXC==$>^jXr* zmNMhli{@sqJLa=cWf4;}m8qxde1kZy|8dlL*bw3yMg#q>LM*uJ4#WYM5GXsjnD^pIBHwu3^9FED1u0Ww~^uvd^RpmD^-Pqh8y zhnM=O=@AFz&StVUw*dU^n(Hc5$T%Zt=!V?e?vSRi<{4Ml9X%1Z3o(0mq;JwGESDGJ z6lvVLUIQU!`b5@uF#)h37`HGZ5Vb(>P23X4fel_B+F zNR?*{P_#ABqz6!UK9IZs2{OMoEQD>=B4X0l!#S%_OIE;;whAb-4#>3y=&%*|EjGC8 zW5#Phz~d^MYp|3WC(-A^$o5s%s$#=|+#_JI36rJ&@tUWYsI}qGz^(o-FjVDHdi}kp zcy9M55sa+-fnYP%9piALQ3FJ6N5AFR6E9x2q06TLS?0ij%~3t)!qJjxrXDOL|EcQV zcsabb9&tB3dA)1=AJ(Z9) z+S#-|9QPXlH>5vbt^|DPRKp5Jt!9*D)4zKIvW*`;%@#K>`vD}D56C6Ani*q$3Y?eD z$T zpfE`Z!d1V3+?RRyQ(o}2^s0sB?Ll>#=c76qX)51ltB;|XMYrgDs{KEgB#jRzpmN4f zyD!1s&w{=Whj5JNH3U-zR5~LO4<0`r^j6jeaRZDO7}CYn?Dz$LZ3tqrlX(t9%%Yi7 zwTNK%jnhJUM>ZJ7E6h}Trp`~e9y?IK-5F_G<4km=oX(3CPsq&Vk@Jhi^3rfYR?>Z4 zNEgAiI3YCFsyQ%AmoLtZnLdWqQ8j3tYQpqVDKg*%@Is9jhuIi#uoHAK1GW4jVD>W=2LoFUG!3)kMA38kAD4E&(Wd%~X+m^MlFfOsA>h^o(JHEAe9q zy`$DCOQdyVs>_yxA*#4kb&pgvtTuPl@bxTk_B_z|0&wyo5OM`ra}$9V9-QUF zQ_ZE9UA|IF237xgaf5S`m9ON|qbSU7*_Ca@Q0VKgxT0uvcg(fJ{7g6dAaA=PKSJMb zO2QO;H?xh=gQm;%q;{6CB#zC1T)rx9$TeP8yh7el=7PF_9n!8jzVVa#7j`x~`3&{D zRyoL0F^Vj0WYKxs8IJ0Ia*KM%m_*y>q3tECBwcF*d}|LpxeGXA*bJ9r`YXn{PUgHA zvffT)3AYhL79_1*{fqe9Uvl5&d=_77_^xw)Fw;p+-c+|!a5Os&?4>#C^Cx(Z+XQm( zvp<#i+9$-jMi(dVlWkn*Qq9EDR*`Pcd=4lg?>}<(!!TCP7R_Ez31%%(q?BYw41%l zr^nM67M}-;X>OS}u%wD^equz_cobBbzmwyLC+|^Yuc}9!TZ`rmI&%>A#Vlj0K1@2G z^dZJ-*x9R4H;E(M;{e`*^ zt`^^sYL&P!lM1Z4O|`n+0gB!OCZViqvCO9RCyAnbEnsOw{y)@bBcA-$6^P06ZzAUnb5}8Th#}=7q8E+Hu?n?kUWhLNI8kT>D-%f#CJE zR7;=hC|3?{ufbNSRg%Mn99D z=gT5|=g0V^m%PqWU*gD>n;tV<;z=E-#7-FT*X3&n?&^A1TIJh~^sLOIk@Hb<>4vANF~tT$qz;e|_L|V<5O0aES{4^V*?OYeHc?==+4XN4W4Djp7@p zBjY{Nm}|C$iMmqI{vZlUy8)kSev#v}cDZlT?Qg`m+&}@<^|C+~%!T$yx-IF>_esx* zjnE@{{$wr$9nE$m^qH!79X{g|Ok-FMl-FIL9_CK1eF=Pg2ftq~SC*=wo#gx~i*avp z?M2SZ)B&_jWxPy8OMO8=wgXDVhbOjpD%rLG6E?ncIyF9T>vOUpniQZ{)ARS20x zTJ#dqCOl>lR&T^mf6;ueM&-n^sM%!z7qO08Taa?A{B-PBX3UwM^J5h%``qCYvm|Y6 z$T9Ou{i*&PL{`{LC3c)L9p`TD$fL`cH{)pC*kg?Q4FZv8Y34SWao1#3vC2x}(0U^a z&Spm4x-#TP8YjGa#A2R>eT+C_HBo$H`?XhMl)8`eTZGouLvptr$NK zD%I5!xGmGxA0N~B!iPf-`rtE-YP^DGETRwecWYf<-K3AKb((9(pGLu*Lmau0E99k9 zTH0---8ljE?McA)6iG*}iLLM3>)=`bG7BfCYGfaT#mkJT*N=ka%78eQ-Fb%b{L6nf zCcX53-*`NKAYU*@nyR0Pn`g z{DsU81MK5XP7reQl;)|-QCyEFE-1s`CWR#sm=7ILn#yTtsGkMnV?Buz}^8|*B z3HBkzC8%%+azd1M5S9%;(D=4kmBHAhL?Y6qf%5v4g<5`aVL0c;tu5e z1(~Xo9C1k%#`B2tVV;ed-8PqDwwVu4FyuCnT^7)o3ag>iC$>mVeTvDHdRFXE+i&Vp zariR^0UMsf!cePK_@ag}wU#=@$1OZCtF^`)N3%1JC0nJnT+~yA#B5~RMtn0jy4{Tr zMhq{6nJHb3ck1d0Q%A)xm-P1)@-{u_-}bp=v1_Pp!P4@+NBUJ(4^OVO8sCYN#$X9# z$@-^H%hIRPm-Bh9!#TGcgL{O)A>|;#Wm^yE%E~}Y7 znfD`&|0c87_~d^TD%KSGg^EA1dd3p$S52B84n&KOxDLal?8P$X&CiG8jXSj8LO4er zGcm(C^FozTEVUzpraQgXbWO)J4ky$SnbB=ndSrn0zV~@KzrR2&r^z*sVSnJR^1}lc zq+_5b+;@vgOXPEm1&5Nag;?`Sn}JcgfXianyK{sPs6I>1s~Kw0rXS86zv~wndXc}C zhFmmY5CR4kA;+yi;se&n2O9zp427-OL-~{htw8;!`!puAse5ZOdy0Q}2`$&%V%m`Qrte^=Vtarw-cU9Q~?-bYTi;VH-n!w;%B+67hDa+6wnFXCA zfq5vWI?IWyUK`Y$drgl;=H8M(LRp}I7;t_qyFn>aPZDlTZi2%ef@t>_TKSaet-P`3opFvy(O|o zvXZ1i$Sk8|lpE!f7wG<@I z;^>`gdbd5$Ej(i=$flbIkW*0&ol$dh-mAZ|CrSB#5s zRtQ{&ukiVp+5G>!L`WPyqU1N(qM-#Rqt{cc@+)9QQZ=DaTd5JzO&SF8z5^}bI6P+`EYa{G99|y`Z}x*l65>D$vTz*7fiy)C$V)! zCShn;-&dfmK&uZ~sK6(~eosJcC6ByB44In<`TLNt9u2zzYc`%H_W00E<8erX7+0Ak zdpEzvPH#8bDtQo%mlC~C5$UoogJ30ZEQdqhPbDP(U^Rp#xgV+mSsR5-S?WrLx5>Km zaMij7SLp^AVUVum#yhz0dpQI7-%t$e#npTSaPc4HHbs37lT`A{FGx7ioRcwTQM9G- zb_j)o7Cnw~kp4ODyYV6zYCd!&%L9CGFdZ2`2!{r17(s-fye`d;C&rFrm7emm1Itzv z79L@kt^3Sje$BCfc_r-xOh4KXm``j^!c=8#L)eUHMC4f^gl*=|GDQ`|eCz0zX{;fX zIU?X`;={$1E38K_#Lxw8nZeev2QifG+Da^qCadGAs@>AFa30o17Uqxz3KoLbb+d!S z7D^5w?r+=S9&}q`wTFGZ>-nM)HQZCdN0FA)bkT_Z;il-()zB;KZP|P@+Y0 z5L{nU2?8o4lt#$J^~gA}!AjY3>pF%A$3zG&;#B+k=^vA`=&1>K{vNh$DcQ&S@#+S} zc3__BEG4`Y>o?32)bP?A7KV-)MH2nRY6v8B7NixNt-7qL%9 z%plHySdb1bfpS2UnUEwj^AT9QWe^OsII@l`EC+GB3=F_QTMnX(7&}ryDd-2IU=cUn z2PjTmj$6-cQaxTgtE`Gh998RDPJrw?O0bhh;kj=#9~&KR*0(-3tN_&x%Qp(!jx z?qUhCN@SO~m?XZPyZ-YE-o#K*(*+~C8%;`2tU%w9oJ|Y_VMNTtnCyWd$h*woq;3i0 zuoVpPqcG+nb;<3eFe=7jH2sE=v0(ZgmWjdf5)TJ7r=Aj$l$69DbvJ5XyuTHei`w25`SGA(0;;K=LOdKe`_U5RCvoF?+wb zMGcWz6FPCwd;I(mMPLb{gP09cObZ-v{lyNFhum)u53fR@L{k)Iuy#XH{6!oJp`h zIw1;TcFQA&X8jHqMAL4U#|1I`XDgsx-rkJ#OcPpSIP$x`Rlidej}l!jjnH@H2~BmC zXs)b~rRzUrXooew_4UCRc>3v#TBNw*ejSDdavY&aT!p(BwyY^;r8=}L6n=5`C4o8+sgLQ3!{;?#`77i4FB%DtOtB^F37F__4HL-%h#3CTsi6^NTt@cKFkp?@=l-b<0?Ift_8{ZIk3)U*l#W7{W79LZf3(o@W?}MQJ5yTQD0=~Ufg`67J!@y;Iz}a{G+@o!9iNet2FFt z{qGY5amXdHAVAJQrSRdx61e@JteaoNx&cuUJ;VY5XBM()hFe6f7J_TI6Pb?%f^=vM zWUoRP=0p`JmcgS2JV1`FpJMNVM^WO(E`PzYkyvMHB$U+mjW9N$qpbhT=}(L_(uJ2RP^^zePFH5kpuDs>70h9*4R1{M^2zBSRMY1=lTz3d0Hc#2PBS z?jO=o`yZaOqte9ms7e;rd0j1p*`2i$1%0Il^GmYTGs2mX_-(MeOt7B2fsY9M@7Fbr zT5)aiiBQReHywuGa&b5W1|2~Vnn#H~RPB-h=MW@+eb@EGZM2{;U7Td3wCpIwo!we^ zKvn+@w0FrLjUTC2n8SGGK0&dd=EoyMM_a>~L*BFvNj*#`Vk?9q)~~Py>0L0vwV4gB z4HcCjo?>^vDuWZG?958CdzPs@*B8OiKi5ww)e}P_F_CnTN67}lvK9UF|GqGdE=4TY zkwrJ79*HHox42MIM5`f&7}YKCst;FUfV2TVAQH5J9`FO6IFk>gh1#^qjxcH1Sok3G>fn|f9!H-hlZd@F6O6p;%YV<{3c*BP*^9xP!xF9%xO(A@r@Akr z__~M&eVx!x$FIT=Yk;v4?RNbq8~oul9dBo>C;mRxb;vu23Lj0gXd<1RmT1pRx9o!XG;j0gp*zC%Ofj}cmSdk z`ie8c5W#&(+-D%paA6!+ud-JBha_8H+&klnLMLir5Wc<;+b6W{c#a<~@?bXz1I@VW z?1f)K8ui9qLW$Yxhluw2<7yUH`p0lZe;m=4LC}%wPr{whwoF1%d(UCHcM6tOSlpuF zQ=MqUYedmYpy=0&;9dzlJb?%IDp>ju=Sy5rR}=NP_J8~sL3yC)lL91fk;|yzhL%It zra^e46aUk86(YFakAE z3h~H?5IV*Bq3vk4F%-w9g;#ivbrE-Cad-1FlG(BCHJU5dNwJJD0qH}wnuFC0A8TT# z@NyCZ(@D%ID3j%%;IO~uJ4&)f;U^RUc2&C1(}om{Jqf(AhixNd4Acr7t2*Nc*2!c5rZ8@c%?W;WLlIy5FYK& z$S8reCy|>cbqKfy!yAP*fq2MM&F`Cr zg$3d>?34=y#9kM%q)w(nPoQvI%|n~qfP|8_VbZ(a{fUo1Q+91GL6AF);W5|*9=~j= z#I-yInRP#82_%~r{e@`GM`Y2@<6!BBtR{02^OiAf7}TaPO!wpIinX<;QMdhaA#abHaut{Y`wxWN-nXWT~tCK4hq5EV!%{DUQW@+f*I zx$`fsZ&N+SHE#f}wIB*8-yaie^z=`VP%oqpwMh1SivG+w2AAz{AyftpXkHXkmKRtg zVbH^_6R~$gsWU>0BOf8M9E&Wkznu6Mo|i4~_|xlfet8=P=`QYlfPW)@;-{W`9-xJi z%b(-Ut5FzzFJRcBoiOX9(S3(*^Kt8T7POvfbQ*Hz4j9+=AmMk@ ziMNe@`f?(;jm2=Fj^Rgo_Qk@3SsaX1v_MLmjR=Y(o#@}EAnz!${u0P;66EyyN2$oE zmtd>Ufsv4lcnPgIq6l>b@k_jbicct(;VJQGyzxu9bZ9f$S;P-w57F*5mWL_V)tz9z z`Phl>R=H&HRSfTtd_-7pKEW7RhB5I4#wc`FGW!o0_(>wjkbf{-2yNyOoh1&h&@$rs zoEnU7LQL(Tu|bkwL9L|7thpdd#EWNSQ({TsHL-O4aun)6IhiQ8d7_j3Sb|UMxA<5S z`6eM6*j%7#l6BFrDFdA+U@h%}FTQRV;(ajIf9=l;eU!54(nU1%MxvkmLq8!eT!SM+ z(iMbceGb-dGB_(iOx>~8>v0uUJ_Je*B6>?MdIwoB3dVm#S!dpZG2H>B!VE{5 zl647DRZ57eYbxEy32{c(W>2l z!f}8t6}>Qs54tlYg!?`M<|ZmGUKGY*F?a~6#Bq4$uCqEnsBTfNJ}1Vj9dD4B#yZMR zyerXwc1QX4k2qu`Mj^x=%B{}e93OpqFc z`DgPBm^Kp2kS8k&RKKwUwo}RjPFxCpc-|WIOo@>vHbh!b0?cjBEUQ#XlrXoAxS8jU z!0S;U5>LPxc4F_3{A%P@9>Dva?X1%@7pMbmfDL65PBUCweNxTUu7X)i| zF|i&j>W`HVO8!xMm^$Z>OXf>Il>SgvQQ9iDA-42N_?*0e5t=-ehCn1BLf602dXYH$ ztyl!7zG8S8sPaO8V644@U|!Vq!^|uFjlj*NbKOS}-IvzSL?>21dC38FBk(VU%|KUm^k!o$fUsa%i<%u5^ zOC(8@gjZ^{N%$U|5{aHfB|)NXAn|L{1d!NB)OsWaB`O(hxbTr^TuCeqR2tg!pGg}@ z@I;bNv> zsx6;sJG2p`OB8b^m1+Rp2uaY8#@i8TQ6Hh3>_GU(wONO#G|lOkIyQ9BoYH1nqS`e= zzt(}sqP5wXsdddM%$;J|ov<;%XeKEG$Oy1F( zZ_(Zo<$E7a6iZUlqDa+9HPVo!VxTY~@cB(kiJ?s$BqVZ&BDUDXk zp?g*=XcOxb#nKe5K1MfLEEF1hjKtcZqrpx;8zr0;8|cV7Or>c^zv#H3AvR=+^);2& zS^BSzA|tV3j%=T(cJBuV3R*Y~LbXi3) zTIk4`($<*s)OMkWY<1*)Y3n<9+F}l^Fgl7UoQ-eP<{gN!)nM`z+gufYql0%M#y*rO zKuobOzEzuVG3LlErckjh@8j=w@U6xiU15q6Q=%kvYV$M3I%zN`PAiEf^c3^+(ztjs zr%o$tC-jL5NYS`eF=tP!I3^4g3#ik0yk{<&R*gy+5fwC`@seSwm{u!JKp9)p_;|9^ zPjBr=7!wt8r}3*|X_;1^NVZu(muV7lsLsM9M>Hn%6h+3Lzw<;ns8<}O8;T! zb}jn%>>J9vgNs?8cWNG{UrO81+#NE-8u_YoM>zdwc9EX$P*JwIPOSp^?`a~>x=}G7 zJ9plpUt!<)wmZC-?MLUXN&2<4jo-T?rr7>=?p~uury=)5iL%pnX$wP4h;sL!%2>K| zw4gkTN%X`Ov-5Q69;Ps-i>dV_OtEk1+7nJ;=h&pzb5@i?vP-Xk!kxb9Ko2T&bC>=d z3O|RqYtQ*&j;&n=lN8}}@!+1+DUO|8d)FwU91?LoX`-C^U53I866q3|J*Z~0E+Z`l z84k(vp3Gv-gI&gl85Ghbn|rdSIFEFhgfl2{NcHsOh;q4gnHDgprAs}F%d_F~Ej7E9 zw2ex7CaxfdE4XyuSkjJY>927`V_e~-`&W{7Q^}C@i`ltjW6Zf3_Z*jD*Dp2aKId$q zbk1N}MufKfEO&;prTIByCs{?>iazc@m}Cl1-}3!F`tRSaqial2R7%td9sKx& ztRw&V*k~orgf1I?<}xRZ^NCc-n-Y3*__<=84=^VmS5`9U9pM*>bqQv^@I_hMppRWZ zJk~Xn`QmXEbAthM0r^Qj3H`*s?=i(EPf(G4AH%1Y1 zcJWP1X>w8@rkgMn@^;3lX6RJ+dSQ?Dxr{B|EZKaCEI@XzMc{~ z(|v4>rGrC*{9sB{IKDf8KeanrgEMi)TsS%QxEkw&FB+nI(FrnK0?ktgoVF`GnClSE z-4o=?I{bLMX5zcg!k1}+<5(XnYZ~rdP})#A8(f_FtVGkA{)6X+`q>j5sbgY0-0447 zZD^SdnM$23*%3m&{C-1Z`|L^5g()$u1o|&BBE7Srq8HwlXl2uX^As7LJ*9o&z1YqQ z`X5yyqqAX-7nVwPw$T52FYb%z(QeNW%@>;VMd$L=$C*j6}p zzkRS)yzxhO#3b8V`tI)!{!MNC+a0;aM&i^a_mRXzX?voC*~uB&+-`vMLz3E6#ayW+)r$J92znHjN&-nrlwG4x2eGtliJt=oMG&uLZ z?Gr8$Khcvq$)T08_j{knwD_5x3u_$OoQC9nQ89`5o-|=jy$nO{q)oOG=gZSIGWJp% zZAy~JmB=j5u*ooqF;YpA8kZ<2zZjaapW0YANk%}jygW09(<;!|+(^zs@@iaGGv}eN z#*Rkl``6>LpK;m;ngkdr_Di<8UizHjFk=$IfI+d#HHZD8v$JU`gNl*VfNQQ2m-`db zf_~L#smFSGrd-~-W;ID$TcyS%@;xv5HJROF*fuTosysjR;xX!dT}c`O(r@DmGP!~R z_l+27T1bD0D{QzJGPCbBgI1FCSJ$GEi($_DKQZj;mtJ+f{P|+U%>I82+6*$(ZpG}G zG0x_!jJifLjBX`LnF%xIB8+-TGMsLu=9v&Hii`&RGJHjXqqIe;#M(|34!C!xUXMU)2;GzX3>mQ1fw~FoUU6HdlsZm zDx;;5oUvQAQdY&xfg;8ONphBMHRf3mMfHr<{c?70R|B%@XAZVA+AzpFyVa&;K`sq5 z+8N1vyVW&hwai$*W;~oEALv#;lGWyX=o6zuzkHb6wa;1YGl%{$Ix;B4xHYh6Lt?Qq zIU6aQb8A$}?wzp_VRB7U$Z%^i&mM5LRb+DSSIBd_9*{jeW4nvVlVNkITXS0W6KA`9 zOx{MDBWv7lG-Qv?*f}uyCT(tVYZ=L&aJKhn^6%f==63UQ_Ujq@2&MoA#V)s7%zQJJ zhtCfl*HP@(Z&l!%e{#5BFeqB_v3{E=-=gJ_+Cj*?asAuAe4m~ixiff@YRk0#o%4KO zEggmiPw8w~(7#*D_w$LvD=;L|K1Yczb8l61|zAIs0=!o z`KhfOnTMivlo$*;6>@019R<(Fo>F3$>N3n>*yAL3KE73nU#ixBMHfyM&rv_@ca< zu8%Arrs{guG+kL`FI~TcfaG=89Mg-(Rrb>j=n2TGx>cEGHL4t<8*~-ed>tEPm%gYR zr5lP9P!@OZWzLmVb)O$9&Q&XOA7#$>QuUvI*paIt?lI3?Sgjf|KRlJIRp#-N`SOBl z#QY;tp0>Cr9ZQLXcaW5&BE7fY*^G_V}Ov}8~St=LQ zisqk2<(Z3n8?aQ%ZmpPqR-AXB%-fpfs@K-~`R5&ZHsU_+EOpgeTjocn@(!2zgs@y& z*xEk-f|Tzl?wi2UD68H(KPH;*TIQS0a@|XPcz#?v-&5SLg5^fF`sn>hUkl64 z1@+hSFQf7U#Ql3&T4lG*&rcTT2bcMevfTFC_G$iAM}DaIv3Zue)!Tl~zn;nuFFW>= z<=(=!fAeoh1ySMwbgZ4S8g%cbL<{1|0(e=wy);Z}hIG(_IbMir!q2O6;U%Wju?_qMnovnbgc%U;mQx8h1b5F=-4;LvBM>CDRaa1ppcP$6Bavs5&H7zCoX^AaPQkm(*7w9t=PoR+#=mUq3ru-EVPmnzo-@U&W4Jxy-m`} z@c3$8G~jYdk!`+TD>BdHTR_q9?5SOB?-_QMdVEhSdg2nck8ROtXN|{?hN98gFbB4! zq@7J3KSzotTu%G5ed^!Y=JD%u(d*gM5p16sc6E8IuwR~W2~TDFYP4&>K${m25Fa zE5S6HFl}}jYNKKXy=b{K+BR)|85*x*mY!(MH2PI-Q5kYlF^67^aT< zyEM!@lw|1oiv{#zz0+9QbTnk>7m6{cg{85r>S)VQ7)r$S;?AXUXzCiuFvyl*gv(3g z3ez>0VKgd{)r+r5<7v~ikzw*G!Eo1>#EdB} zX)>HfrAB&5a_N$7dO0#&UZof!HPfY6^@?ST z@+7@u%kx}kYWDwJ*fm|bR(|;>SLEXUe+#>*sz{Q>bllN$=5&kNVpZfT#k}0H z-sY@}IyzMp$r35<_!@KGMO~*V<|`%Y+=+|kB8z*XtJoz=4Y-r!ETk6oN~*Z8lv;Bq zds`?j>UUQ0OP0BFr_@-eFB(i&312A-;l8kFv1@TJRkf&Od0b|ipru}op+L2Ge0fG@ zhMlEpjnVdMY1%8LnVET(2WpIMtL5XbG-YN_SRSr1IaR%dwxTODN6^Z(#x%EDHNN6W zW}cmuZ;e@N^)}kdnaqMbtKgb_Xt=^PXo_8R##?rRNB)(cDt776nL5;3%2g_>?t>Os~)sz3f5w>5`z zuLi}}KgnvdJG4}Dr1k1a+G{geck&K>uW=Z^dOH5v*R1x5Lu)lhe_xHHZJ^HX5VRp* zbrh(Ljc?%0?y|FCzUs8SHj%b*Q+7|D4fj=N+uG##MkVXsCpJRKE+-T&{Akp+?&Gu- zPj<~xxOk$;+8Mdm)?#~tSPh1bM9)4oGBiZ9eZPDEI2;LD! zJ6#`7jyh;jsk~2&?TmfAlOe(pv)?---K zvyX3j9n`B~-U(xSZy&$Lx|X??*SwSG>;rxLAJ;+G`ZW5g3(FP9=IZWTx%qwc^^ARF zc=E9yb?tLE*GAt^AC5K&V5slZzePDTt#UY#>A0lg{TsLV56uQ1zQ7cyQ{N}qD#bTf zb2$4$;1R{4tX8!b^D~DF2ZPQij_|bUU3xEgq%1i&z5aZ61oF(SYJA6)h7OK z*htqClfy3LtXF$hpiw-bCnJZ`-r2M^X?r6y$u4!@9#Ywh`5Wz~@05gS1(7oYPfH82TGD#q6R3geR9uDPZc z{Z#gi_G^tx)3}wGW~QiES`V0x%kFWjHqEM2u^t^bJdQbkqiJ@3ldbxoyTIl$w;Q$D z<0|$cg8{aRPu*^xzw|@p=+D6@TO}s(1^&spHCf-1|*)H>`ZcI$Qb7tHx?s(vjGZ`p49=KhAcAYS#j`f#7^c2$peO9ds@gDZwd$2DC% zJ`NS$RSo5R^mZIt?vJ6OSE}LJkG_xZn)UcQbosAp6z>RmLR;JuhtLF?V?##R?R2d? znNmu&swMs$5w+91=*g8*dQdGnV!Jwz0U)k|yD8su14Vk<)`IVa1uUh#=#r(4(!2`+OmS#0(TWhSJ*G?Sl_O>;<>bJFi^!c5MLo_~4X0<6>n^25- zwmW=0H`i6)xY0X0k!OF}CxE5?{?=CY7w_$kT=zN2a&2zw-HI1K1&@C4iTHSps-;6} zj84c&-Z$Z6!v^)9>@n__&Mv+w4;y!?_wkNPyoA=BRo7(OGMF%~lJEY~x1jF&sg_4{ z6T11HLVo3S&ABa46DBP4z3u&K>u$8Ryr6sOXzzQ`?|MqhbM==YF9YoTpZeWSx%or= z&Ci!n_5n=(U4FMXZqB$*rV0g`_z(ECD&L%+pDYpzPWFG|*Jg2Z(fw7uP{^SFgx~Gs zH$RQOy7TfR&9Nz#JMr7TsK4&BKc#x?9n0M-+kUjX9Frw*}OyY!5KaFYCS$>E+t4em=b2PV(C9N#>A|E>l<->mB- zCIMTAdtYe?Uz!b`Or1TxbNInu4N<4?bW2czfGk z`h`t}<@v#x^-pZuOcEDV3M*a)7t}vJ)wZ90QMa&4=tOz_v)neT#6`=(8v7Hq_0L<| z4$*&bF09Qz(Of?|-e#ZpA+WIi<%v7>FMhWjrC*9EY!C|RsUH)#jhRPAVUvBxQ2qG! z+aB~EOADLxL!Q-7*xtr8q^YpwWyoax%Tu?H(SPbHyd`wx)T&jQ7 zdix~(@=W2K{FC47Uyt9$1m$aC`^%GS^>2RPj->xgUDP2IO1?HFa0jy$&LRvM%-5#3 z-$|tZvZ<&iKa~60jO`svT~vyCUxo@_n>}^s0{vIrqCTNh64&15-np3g)!bpg>XiJ2 zxf@!SetmUx7`k{WQuV_8bFKW7-vUI22Ty4pd-vr|(YtREA|p&;y2svg+%5I^o+|Rh zB+U5Og7V#pci)Rdo+XD_9$U1yi|J0i$mn30-LVhH@7BNj(JnH^blUmYQuy00L&eSje=xiU8{^P#!nYP8Zo-O>U3|Z@^8z=Umqe~ zFaL?Y_gVU1@5bNqk@L%ICHKBp{u|x+$0zdB^54#TE7JeXZ~S{T^5^ou>3eIJ{`>ys z-)!XHNB{lZP2wlfPLU*{Xh%pQ+T58-;q0BZWI!JdK>K%@s;*;mCL(T0V`oMVoJh+R2eQGg>u`Zd6++l;*f7 z>rnJ|bNa=O4Yz1cin1+5?;N53r!9)ZAjRy$F?tOY!A|i|a=Ihuff$oC26Y|j6>^Cq zcV>)L8l$z2d?@WTQQn~#y8x!ZPQ~}M_eA-bW1Rz-Gdfi~=^lv+n#Otuurzh5zo&a6 zDttaRFo5+*r=}*Unw<>HiZIWsZvpVCU@8^`x+gZ8D8J7r>#?W$>P| zaauesF5R4SpRTbD!{%vB26N51{B`$@G3=a{CdZeUbEoQB+Ax}w$ZEw`De=^F9lXV8 z>m(l>U;mk>tII})$wO>&ZG6jT-kGk$RZJ(vwk*ZBf99j^c9daG6jN46=pEq~>2^8G ze6d93K*C^}fchSf_soUUs^JMEN`l6Fyfj#JwnVES@q2^FMEB{lY~MvemQiI8}!j0l# zJr^jPI-3m1Nn#cfsd^bUoMtnaoXT2A*6U@DaUPj5iAz$pkQ&y@v*Gfa!Ay0#h4d%A zqA{*BGyBQsbSz|8^-FEIQ)V!2HL{RZ)UO!hE}XH7J7-}bw@<&uhNo@@b68sod4K)- zF`m0KhseoJ77D5N8x45}OKr81J(V}t+`n;)_r;8zYqHP3-Zhldp>7A?+Iw^i^DW(0(HtGjcGQx@C#}!h0 zzi4cF@U}|$l6a7B>hPECx)0vVY^W4JQJXsYMbmljQqG1OvmukIlZ`vV9xPKty30a^ zFU+QEW$*nKD)QJR?7)Q&={sBY{#p^4b_vhC@GX7U=-xk}8^5?jJiG8GefQ73|E+AK za*bk6qYl*O?4$M)Ws`_8O=AqyQR$;w5Zx#dcRr0XP}j1L(MxQLM8Z&-VBns>KGp@X z-4bV+(>Dd`W%O}+ZQ3t!&NN*vP`{~>KFcZY2GoyztJ&P?i0Q z{X6@m|CC=|yBMWn&N!fBB*S#2L@V>GiiO;Oe!q;+m9p^6G!;wZ0V5+>`72lMWag+? zc@LQN%kH>Rxt3Y1a^T#6g^`@em1?!DO3s57#s^#FY~8K~XIOGjZCkkmS~=mi>U!R?5V!RUy~ydT{n5PCEk=ukD+w-EK5z zzu>fYH}PxT-0F7od-fa7!wDwGw=4F$wW?i;e8+jD!sJA&;<(%G;7iM#4!tI)wr^Q* zyW4!}7w6G=lQXSbe!JcKe(66>N9v&{StWX@PK6x0Z;qRW;;NPSq`F;mSe`m*4q;tE zyrL&FhwGxV<MQ_$CY=^KOpCdS-GZ~X4tBj&G&2N%BZ^43}yAJ z*jEgO=PIeX4-DlRsrpnrypyY;>ajRf*sprJ;?Y{JwyG!N!xAI4L zqg8#}{bg|eF)sgJv)1j~7TjMo=bz*{Hg9&fb=z>4NJSMxtL;7QFVavCWnc>Om9VeximkQgzouqy=C7>1m?z2SE z{p+EEk7kp!E>?a$P&9b)l=}X63tENmzQq+iRt+rJ8x9} zd{#7Jb=rUbr}3S4D}SvOz2OQ^*#Bkwt_RY;6)w+z3oqLLJ$KjhcdM?KKR!LvzJEnv z_uRX+^2dcu#x3Sl~dczd7rK&d&Grv?XE(;!@2y z?R3rFGw7Gd9*D0u=bg|!@}B8Ti4u20uepHc9xqRp8ztMg6X(r0wCy?N$@ZgUH}_dO z3o%W-Bu|d5rF*%Pq%0)c^m0A9PL%HFK4)MdtEpe@$#bLh(D!rBj}>b4Z_4m}EOq#v z9P?O7&ftNJz~(ad@8?S`)Xo_^tr9#^=G&ao_ITT>!L+AvR#{MH>Zrwzu)SYAMV^

?QTQ zJU{bdh~@rOBONapo-1XUnc0?BVa66-atE(m&CF`CJhW=;@g7Yk&dV8B) zlU09MF%g^}_|$*V?4GO!OXW;(LB`WS?|qMCHO(p)gA1FUhAi%TBde8C`8Bxc$`FSTi3yJi|)|0=HVot z<6G+=@U%)Dnrl0p>l1XM{t-``!J!4sBh@}9Zq&cvx$TZ@9l0rg@?-t$AGc#(e3o;# z{~=WJ+T8OyMTfrUIXwRm=63Dl^SkYbRs@gEeF(pD?Z@->*N4{fj{f)%F?a3n^Lwi= z{$q5csg2suKyK*}9HU8g9) z_{q4YkH@k?(ZlOuA&Cf&YxBn|_nULApZZUj4SMaUtNhzstp4oWgoTN>NnQ2M8&&GhYbFj3dfV4s zJ#(W$eRN>L*2KrBuI}!QTk0z#+@yF>f>@R-3ER0 z>rmH^cqa_(y>|Fj``x(F@`CrJyS?9SzgEASKU&`KPA1qN-{Ify*ShuQJKk3n_9w3T zzu0`+@8+kUuex4_${%~T`EK3KpFdyEybQm3?APXdA8-Eq`G$HjNo9|X9FEa(l){ksx z)4Tbe_ek04;B@~d2ir_8EvO%<{1ALu@mWfn<;g|MBQ^FX>ONta(Du*=|0DG;Pu%_V zg7&uKwxx5g8cjk5>&JKA_T2Wd<`vcqU)H}ob30($r-4_kCMOsCU){JJ^6S>2DaqxxSMFrJ`0C&= zlz(dL^8DPL!WZ8n97cq~^q1dD-o5hTdy&Jl{IG+|i*9%8Ui@fx7!x|}w!C!Z?u{2e zUpq|ZpFXktY3}ab7r*{FObLa@FMpP7zyD%I*C8%Uk|EytjJtzpropeTe*T3yHOZI+VmUMeQ5KrbNB3gE5p!b&75& zYNHZ4wSzO1+G>i?H+qW_-EfCsC{555>r(V?B?@cjrcm<5DNf%Qv(F3$ow7Ex*Bp5# zV~%`gO6XL!p?l;gcsSPYGfQvhb{qO-N0G_cGoRV$x^y%sOw(efamfuFySfZD804oV zhvISrxI()2^E2+5mQjeWRN|@cvR+|ynO3Nczp2Fcsmo!7DRx>(A>o0NfMU1%3Uk@C zT5ZA$B_aQA{}q<*X$^(McS;-TyHAF!u+C0v)h7N@+W4tEVug)nMqA-5y|S2MPr?ej z_>5ldSw3ZP|DMzpjy*Gm3Q5w+lJz~=E1WJfSS#J8Ed8mcXoV|w#$4gtUS(Ou`xPtP zWiwbfJ)|t}f4_c(r+damIN9xsLdE?XW4v#i4;Lh#_@X#}|Lz#yf6k7==i|R9N%h_z zi!!d!~E}p_ESH7rK^gbUGJnG_GkaFXT`h4%JF`-D80Or*0Mh%e% zZ*K|b&tkFlX``mYgQZ&=ZqJ@%zA)RURrKKdEs>YAr_W#b*0}5SgSA^5|I9`*r_nTN zi}aCOMTOqRo=@Xy(sk%#ZWY_{Hjz19yh*R9kGpk~{oCa8>8eczuls~s#ZSMzz?`wC z$xx(UJXhk9>&5U4i$G)Re)(L$^OW?rw?#`A zBUQ{<2Xs%#aLtwaW?oRS+&5r+N>+8QVkz^o%7N4Y%Tsbzb2YwM*Ho;B2kcJC2hG(l zW!+P;VI6cnrEqbs$v69virv0J?^Bzv&$TRNzfn1yIv99L@#$Qf@1<~Y4)^}n5cha{zTo2ZK!@R>*6lmqy<;vcyB=IL z{H#^e-s3}fVPjCp=LU|v}=4xH?_!eIDI4I2e(TCifH$8r>6-@_) zXFvLuyX%F=pYY3Hf+9vA{mI?^%j3Vb%T&QpydyMu+I*fgXNoz3W2{G*@^rR&(*G?M z3y#Yk;mXrJ0R>VI#X&9e9ro@e4hSIPrkpU zPQmB1AFJjWyzmq{Q+7N!b@cHLL7Y$ifB&S)uWSX+dho1LSFRfJj7g4E}*TVmQPD2YQ@Q`t$nwkWI9sfuqT=%um@H}gwF(uAd zlQ>DF@|RTK%lu|~&yboQo*_LQK11rCIYV+GMUb9W(vhBl=U^1Laz&8bfIEoii6D^# zB1lwVdUFKn=l%%N3fScmK^pOoAiX*nL1IgYAbqWfAkE*5AT2f0k&IR%NXDS!cLb>u z{G~*ah8ZGBLt>GnhhP{y0*}E6IIa*$3Iz0;kt7P3){i93f@_A6BnQ(-l5-0k>Fm)+ zQW8i8jV_U-Ccy0zNs3L5Bo&^IB-y7$l3relBu#)xaJn**R9T1XHab!j{?~#!pj#hF z+5_Sn@XWnP(r9-iX&lhJi6niVizJo3izF5Qh$QW&i6VK4N0GdN5AX$kz#kj~0f0^_ zij<)gManjgBISbE{ZXW;PC8N?{wIJ$a26ziX}G80e=4|d6-DX=U5BDbV)jv_!;Vp; zBLPt)2XGWD^wN>)Z$*)=fzPmiY>gr<0m(i((n5a}=>w>Liu>TzN)%}pyajWBi6xrE z3}U6CNy{?PBo?`7(kX>#QaI={i6)JJSKy9yG>Ow8nsmTBnxynInxq1D18txK_5gk0 z!4*SdGLIo86vmJef$3zjk#eX+Y1}Z=$r~=g>q!@WFi6P0BA?%765~VVR^z2#;=_PmtUIU)y z7!ohw1N=Y$MBj)Z#Q?gy$lE>S4eaQSA;p2qk5PYsJQ+ix2fUy0EO_<{Wwr``KpW_Q zrCBft%nK_y)d%AK)ig0nUF=o*){;fdr5U?AK5SpaN8aYET7gz*SHS>cIBD z$U9g9AHgTE3_gP|;4AnB82*8`bR=f{X928$5!Y<^&jHv0C*T6yfCumbKEMwIfFKY8 z!e9ds0ULoR5Ch6U1*idauoLV8+CT^Bf;K=$6-%NA6ud;wnpmwYUV7w~}%ARAl)IiLV2D#Vhu041OX)IkbJ1!>?SII|gP1CbyK#DRE_ z0M3E1EwQB2ARL?lksuoEREZ_+0@^?y=&9j9*b5AS8Q2HRfd#m$j&cQkU;qq)A@C3k zgGXQlyaf+5;1BeJ0Wb)pw16=%izUebIUo;I!B(INb^sk4lp`3oM>t1N#-XvKO`tmh z?c@U;>2wtG0?vR45DB6{G>8GQ;5@hpE`c174+=p6C<0~R3a9`q2=o7;=q{k5dipSq z|4K6^fXAppHJ;!Zp5i&);4R+aku&R}3e|XmXLt?|7v=|E z@P-e3;Rk;NAP_+aMhHUjR6(EJxLsf&d=QUhxOy@7aDyj&;0r$lV2wA&^<%vJ880}a zMF96HfiAKNt6>W}*kgN;i~I)%lrc`_7!~d!*^!LPITyKw>}tjexyVC44BvA5dFLW0 zI=YH-fU7k4-catixyojDS6Td{p)A2tEW>hyK5>;Wgd+mcuU)0|gR5M?dsuvQ9MsZ}1lH@B#1f5uflGU+@*f*!COw z9Y0WqpBPU2FY-4mDO=%xl>cG`WwCaXk)#SaimXpIKtnWwCK|&8rs#(5=#4(;i~bmZ zff&O+%*jEpz%Y!$7>s4vSkfBfV1x0PfQgs{RU0>1i2&BEBH!Q>{Kvb=ek9=|vM0Dn zChKG&8=t6u#b3xou0hnpBoyHgig6f6;V_Bo4!hunP{iQ@`x-Tc{$eaNTB^ypsa#jM zfLfGJpQrC1JI*kcXWVjUhX=DNXSRG}JA@D$JR z951kbDYugLoe#Rkxn@SeJJ-uKlFzUCSn#A zU?CP^F_vH{mSYui*xsIWz)tLi6I|d51>$iWCvXxsaSN4byVG3^VFVL=gU&9_4|?bb zeHg%PH~WLt9(Nh<;w}>~8B;I~i?9@49xzlBZ}?!Or@M^87)-<@EJ7$k5dJS&FBFlK zqY;BJ>ak=T5|D%xq#~SUX=Da6k&A3(As=}tzyTaYA&PJahj9eQa2zLa5~V0ZInLlL z&fx+s;US)&2DNyDx6tr%m&RykGP11e)J>2U0R|w+CU$jg4h-&=ngYjzzTYi%s))gA8n)P zFWN&J9WdTVO>{_I=wSl&j-);eUjPkf}=QwP5a&De=+Xz7ZU3(DyWD0XaH3-gc{V*2y^4O))Kk)5RW9JA`?Y8 zjmwyy&RAj*7GnvPVi}fW1y*4-I%RUM_?+!7U+@R@a@?f~zc3`%U7F={UEt*bcX@@k zC_Th|kg=LvD|VObxPdW8xgR*rJYWD;mU0}d#%82kpuf<%#P~d>&p7l?E#rgS4~!qm z5%$SlViAXUBp?aNNI@#veqpXrTjwrme;5a3LBGCI(i(klt(CdtCT{V#u-?DQpz~kK&!W!Odxln?2A&=ntI3pQx6#f zYkZsTA@}Ea$Yac&=OGIgd&m-8u=9|(^&XOdfQ=q<3$Ero4t4gBp19@dA-Az6(nB^Q z;e>}=AEG8VkVQEk2T+8c_=Vs2gTIiI9-;{?6kuM3hb+Vf)SdH?OBX!kGOpkrDsdmL z;Bm`CJkhJtL$+ft!eQ~kL#$AQE4YToc#2xAcj;EDhQVM>jMOvN#+eFu?b7xd&n{@$69R0 z7Hq{fsDJbj_s@(uyx3VEm|^TU4;hEGSdR_Z1TT2Q2jw`0r{6tz zjoX9Qwmo=V+k@A&8GlT`L`=pkEXERS`O8?UcuF9G5R4EQ*Y}i8=!`BfMK{dFJnTR& z%5fT3(N5J<3@}jLQ&wtv$||geE$oqmY@9@46ZWO$Df6)a3*m;F%{;}oou~LC00~IJ ziS}%VGL+*S&f@~E;2N&uKW$Ih)WK7>U@Nwvs}B7~H*|*?dY~8dbUmdb3}6T&bb<*? zQLX1GPw*7a@DgwF4)5__M^ACU4(!Bk?7?1~#|2!(Rb0b$d^Pu!))t-;KAv%&z;qJj^h$m<=iTRj8|7Lm0NQ}WmY?$vUUlw`_53Xe8GR7FTu)}(6M)Q@PGIWclSYRX+ zJ3QsN(o_5*J>_6BV~-*nLNSh~Ft#YiDV)Xy+`wZT&7$80p0cahQ@)>ItV$Uxe7MXQ z-}jW#NorDtavU@Cl9MPwDavsgr*H;maSnI!2px>PLfDyXF!`Mr_kkZ*p(vg8o zWFZ?lIDjG?LNVG+=Jg=*2#(?yj^hMQq6DQV$7!6$MKtc>B^&#C$yRKKRzELkj?S2j z@q@i&0w$vGNH39bUSftG=!srwhz6;5zhEh#Ta3DwU>lEVP2vA!b|qoGQY^dnJ-=v^T$gzH}#f%I^L3E z^-@j>|(Z|UdbE&Z`K+*{`5d&|r# z-tz9Mx0t>6mWc@Y=`H?$yybw%SYnTV6TsK-sdAlPCjx7?-PCG4{Qs36JGF!5Bv~-K!hS35r{$z;*o}QWFQmS$VCC3a{NNF2!~LN!+1vfQSul{QHG1S ziaV&n3)Xps*OcFoAMgo3@ZbRF2J3@9G7dHvk0ji|Jv1ov5q&#eBRk|HHF$;hs3`Uk zqr*P3=Q!sNB|P8>9|Rx}K?p_&67UFBXn(>-R7-rMDVm`FjG`#2|>d=8MjPV!#mzjG+AQFeJ`N&b!-~$Y=GY_!G7Hq{% zjJd%aVH{S&4n;VG!}y3#$i2;&pa2I@h$5uj;o3qbva$Fsb9>K6Dxp(JAJGx|Fn}S9 zV2n<9_K4dKKH@9B;U|7!-edYzX$xPXhegcqp6E9`&nBhiRKEaDJouO=yED$*Fg6un^|`d}R=ZU^qr# zGNxf6A`ppf{e7hrV+S%e7>5a%jv1JbMKB-iE5$>7LUU@rEtzV2?Yam{;6GCGO)99-|6fM*B)n+#bWaxQlzJ#C<%#2ee$| zE3I(PiFQ21V^rfQE-QUyLm^{U;VTp0`N||r!F0^PPV9mUoZ$)u+~EOF_#*&82tg>q zaTF&I_MY?o!13@5&+!5;QG-{g#cRC5TfD=2e89|)%mHR&E*2pGtv>lmYqUXIw1X}> z!T?4vMkjQJ3A&;iM!_1cpE)16!5!gWxF!*aDC|cxVi1cs#3KQTNJa|Id}Y7ixP8G4 zJU?E4_FykupnwNFq5FgWV#QD91W)i3&+rPhc#U_z84rAb(f>FnbU{~iLw9KW z<+h0yXo+^vMtgLC4s@XhGxS1$_(>pw5Q;E_BLb0#!hU2R6IsZ{aja1BlPz$p=O+%0 z{bZ!6pTzX_lXxUy)=)oDqStUgc{bWlwkNP2<|g{dLX;)@NjXm8G|r#`-&6eL52mL2 z$uzu8^ON4Wemu7J9P<<7GCwgvcl5xVVK??*FZRI+&TxS%Jm3j$_`u?OfQ$`S6Kf2kJRIXFTas270UPS$ zF#!`X36n7eQ!x#rSZ_2Is|8AOgql3o2$WPZ4e7{0CR%F-N;_mzo=z@>BaTr2*(6Xl zXa&kfyweGk_vmFBD7{fWHBe6BG#=m?zCxx2N{g9+qK_&(!QOd+(r;m)%*QUAUmPeG za1mGWCX8cb1&VP&pxizbDEH9vR-l;R)FY08r=Qs8_dpr>H&7;H3Z`N@reOx|z_~$? z4D1mk<`{&%X=*ZJc94w3ojF0$XKs-6#b}JdSUgH+d{Bk-jX{!uYRZ}9+MPkN4(qWI zoA7>5kbFlQ-yrb_2olY(Akm5l66fR~xlkA+7t!iykj(iUBy%wj%dirwVT%Jeh(Z+M z5T@qwe}P|vBn7F+L^g7u|2;^Sv)q7OL9T=$<<;aW*uoC>SOa608DTBub?8KSJ=vM; z0uyYcz5$zHO1&$(VKenD=uX)TJ%*mO}jNY4&z~i3D`sXL~;@)V+!`tHkF))eUxXA z(=ijXFdK6)7xOS53$X}}tnUP8ET+B`%di|Pu@>vF5u5NIc405}!4Xbyh64TwL=d77 zjTpou7w2#lkMI~;KZ2wwnxQ!i&d?ghuueNz)?))!83&8GX|N2!U<|>A-odgFoA97-uzVT7 zeh#Y1X7ga#g8#4!{Ran2KfE0k%=>Be5k9RBma#=@VzVY##$)fcVA+F2 z2eyw2<~{7eyoa5BCkM;?v|yQsS=o#o=A!@ya2VAmgQXVc7lY;WpI|wIv$%@uxP_Jt zL!?Ka5OL@iB0CW~GDNyg3lURvLm!M@5h4#(g~+tcA!4>WM0#O5hPs7_1sVj0h+$rc zbV4`uMjs4>IU*1R#fcEzzzCMC-K(YFG!5C>6&Lny`(9L6ym#c`a#NtB=zAv&WQ`eF-Wv73JG z!9MJTBb?w27q~(JH@HIy4|u{0(YV8Y>JJK)P(&jIX-G#la*&67RIq*#d5%1fqm+;1 z7%ot+ATQz)uKi2ebO&<|aDW$bP=F%bAHvw+CrpQiN*#69p|T2@*Fq&1ZjZRup@auK;RSD~UFSVGq%Zv7k3a+?6zVJwC&LhdNJL>j zq7j2w#33FDNJJ8nk%CmDAsrdWL>97V2TeyR}xQE!s^cx9i%K2rISy)`f++!)0VHMWk6FjOzWzZAu6EOrsVS!;7 z4oei^09>Ckr?~r!g*Ir94lslfjL`|5(FG>x3R9S&2YR9xdZQ2eVlakc7=~jT9N-28T;YR2gdh}A z*pFz$AQo{*LI%!b_zT7gBVqcI+akI{vxfUBbc8+(V1({4Ll5*sFZ4zq^u<7!BM(2IDaS6EOu-F%R>x5R0$`H*pKMaToVciTikfhj@g?s6sWK;3=NrIpiJJD%N7; zdv2?UL=^TT8Zn4PJQ9$IBqSpRsmMkSa98LM{x|tQG!yG;SA2>7H;DX?&4fM+HnCFaRoQ=5PzUj zKV0geJ{q76+M*pkH3%0Y)o^ib7%pz`M-*=30jlu~dg|fQ8C_t4R*m@PK(xVLjc{>< z6P%Ht87_%PLNZd2iV~Ef3>O>IPh7<{T*nQ}ZXPcGYaK58^};0@F|h3zE_T>~8#rpf z_+X1^xNO7n?%}cm+wrAuxV*3m7v(hiy)ay!qw~se>G4EOg6zX37?anA%M{pb43~L3 z!sWoOaLIQHmtC&mGRud3M25>(^ePCK-uQErKA~w1|JVB^T-G7~XE>kt440j=B4ihK zV-NPi5l(Q13tUl#U(lQ#AzEmPu5%*96pQ9Y$YLzPGAu_04ChCP5sc9ZozWGhh*%II zk=VK{LT)=m$l#q3G6X|024k^jPlUAF8zD=v49l?sE3pczVUJ->5i%T>u)+v@R7A)p zl)6XAb&m-76B;2eA|s>*uTTqv_z3BZ&Y2N1x*$TbOC$LI)d)F^$cHR{86hDEMHs>n zfk^!M%-`Bt_KCtb5poc3KC{lx2(i$Pl&EiNa$F};PGDrmNEv}=T_X8hV5Gc24Q84~ z%3K`i9Vw2pBSmFZq|`$L)Q2h>LJjI@gwu|ZGUp51VIJmV0TyBr7GnvPVi{IoC01cI z{(};+h(iLV|1VNzU?yf^Hs(MH4|rh^uWAp&a9F}vB}zKMs9qHBg^H54L!zV|+Cv-S z6(y>XQKE*Hu`GvSLKN>Ih!S7;BM?PLqvQ~ZaTsOgQBsam2s;%e9j|aqSfccLlx(^k zC7ZDgexFze;mABUe%XyZ*b8T5_m7sBgQLY_ZM4{KjTSrX-5V|YkP{s(gUX}D>q4}6!v{m( zux=-Q&fYIZhLJI3EQV8#Bjb^PL?j^@DM&>c(vg8oWFZ?l$U`pjQ2|p2x7Z+ zib%?7WI8gCi7e#d1Zq$Vjj$N$fu87#0srdzDWf0yqZo&w)|sE3C)LTLPS#oWg0G!C6$`9M0ncF5(g{;|i|g8m{98ZsHbh<31kXAs*o|s!)w5Xhff$lF#rQ zFHnmwu`y!7vSF~qaD1oDimW4l<1a=~ABjVo*%M5_#3W3{6imlV%))HU#{w+GA}q#IEW-+{#18DlF6_o0I3WnZ z2tg>q5RM4MAQo{*Kq8V+0GoLFhY6U9by$y0NJ4f(jJ!qrq!`ggy%gpJSJJqCa1GaS z12=I8cX1z&P=#u|#Km;3KU~3_tQbBc6~kwxIA_eqLM+A-EXt;gWq8S0Yv;vC2k1Z# zHMAL%ozNL3ctu-RvKwkC_s)-z?ggBCA=fb;;W3`!IbPr;YVZoRc#SuBiylQ>KS(;n zc;Png;4bc=68AB;m~%nKVebEM5nGRN|AQa+iK3&dQ$`uT@EiZb-g54Tu*5`NmSP!_ zt};hR#&YT@cPMFq~`0xsbyZs0m@;WqBzF7BZkPw^Zt@DerHK)+s- zwRnZMc!PJ?$g&UQdwj$ve8FdY#W(!GCf3^wxfUZTsE7JcMMG?1nHt#$>d=5D8lwre zvRsR7ie_kmZM3x_TcS1Epe=Nu3q2Ua1cPCLVHgfeSYaeaVKm0T8slJt32@~Yi*Ir* zU@81>^S2U#2tvdA{7t|pjK&y@#W+~Q2IFz|33C9K59|{PxWf%z@P-e35r9AhW7H?^ z`x?f|P*~u$TCCiGN&8qa#dOTTd@R61Tr-H3zDBXq5B)LBB$oFm#fs0UE}05&fqL6a1Iyn5Kr*OG*0^UjFbKtfPom>D^9F24&yNeQ(+4`*kcXW zVLc*x$4N3$a1cdM?-M6s2uB2>upf>4#)%f1VhmD|fh-)x5gfw_l%fpfc!RfihxhmZ zy8-I*k^F?u_=2zahVQ6@YX3NCh%96y7kS7>0S@3G3Q>f^ID(@%h7&l6zfc(vC-qPt z4WJ4&v_miShdH7Ka;}I*8q)D#2xE?gqvB){7GnwG<}nW27;D&IJod&jMh6+=OK}ov z!LP+%WgHQKDC|cPk}>akoXo}S$8oX&8xi_1zs2#H6W04*oa|^G&v%c-%gt%=@_BB& z1nh{HKm;KeAqYhnt_R1lN8T$xp-+YBtcr@%T#soU7sM{+Y{u}GjqY1Ro1D)C>%Ji{` zd`>=5%;u}}dG|zV0$mtk4DO*4p5qe53o9olO47_ko3v=!stF zjXvm$ei(rM7>GeI$6##Wctgmcu)r`3hb62q0wXaBqcH|!VU2OH!FWu-L`=dIOvN-z z#|+HEOw7e9tcESF;RbHw4l3~w&+!63;a`|2Aqd6NqD0w^J#a(-?28lS%aKI+if^dH z57-?|l(xqcMGtl5^z$bD#6(QOWK6>h%*1TCJxi3DpNaC|7yZB^JVq6&@dQut953(^ zX1^JG^h7W8Mqf<8MD)~4lHTZpzUYSqScI3T!7EsFOOjz2g-Mu4 zaBYz+Zg9uPR>{(=W3m`bOO~(mljY^&WC>c8EWxPTlPo`>u`gLPkrtIK=?L1-a_B@S zi!Rz{CW|)A4<^g7;~e`$vS^hh%c;HU(y}61TA?#cU(V??;>}@g7Pp7hb64A zhxQTVNQ}a0?4@lV95IHv6FHW&hBIXsjH9d|UEv0IOrUK%Cc*}jprm~==|N5*r^1u6 z7rZfzx(_*noDN^gGf6*k7C9UKl;@ChF%R>x00FcwBo`r&au60%UV>oCOUV#&85v40 zCs$x4R$(=4VF!Dx!CI`tdThW(Y{F)2!B&K^ukGYEgj4>HbifYm#4hZ{UhKg>IKm0e zaDginaDzLP@PH@0;0+)6!Vmt4;8+12kcYYlFLKNW^ebYIeBD&B+M@&4!`C~GsU@E3z z1{|;hJMo%r+f}4Udj#V1MaBYZmr_I>376>~l8}Ncc#JBHzLCObQ@DO%3p?1O^_>)H zgSL<=#_~xD|A(6*cWYDR9v-0Q>lC?;86Oxs%!2zzu0I_4#JQsaoxY?HF3A8X1voITTFb}!xx3FWH9KsR(G&zbFc#SuBi+6aBa|UUA zzit}etDDAaerfW!N19Zj8c*;7pD=n_nvB6%SYzA5H1UQ*4ENjEiCx%@E{oH|1l=%p z3G2g|_SH+%#1?i~j}178LiApiCVdc!Fr30^oJ9pbEayB{q{%THN8eRx(jNm*j#F@5 zohAwx+os7TT*eh##TvUbS&PG7Y2qH4CQ5|jQ4;5j0cB}25DT6#COD55sKHPC!P1&E zSq8n2Y4W{YI?oBxrLKU#6Iat^Hs)YH7Go(^K@M==OYVgV<$7p{a_Z{jY4R*ID8DD$ zlD)2_OJA&m4}1}bhv;!VUHW4vhGQ-w5RFIZd?TIj_)V8=d`7>-rxGVM9X`jakIf77KJPw*7a@EkAj62I{W8Vxc;6OGXXT4)L#jDQQKVi@$LrPGJo4AE(?J{IKW?&{}qf|RX%Fw(+hHTKu zkd4>`FTD&=cI3FI!OLD5V&6MM)?*zi52^E>$qd)nEM&uT z7k$U-JsA?>lp&#TJI3Q7w+!BMm?6UuO8t{}hE(`)OjMy7ulHxjJG{pSXhvslkdFc!z(IUrokFq* zhfs{eID(@%hT}MalPE(u&fqL6a2^+M372sd*YO%}@D}gz0iW;(|3mZKEHT9&`p}K+ zhyECVfiTBJOu`~8#u6-r9dz=tL>GGK2z?l$6S`n@KJ$y?m|u`3B?q#k6lEyKDLlYK zJjNqbp&C!{6wmM(U+_OHs^oRh!YpZuT}6y7at|@CID}%HKf+kxGOpknuHy!7;|}iP z9xBo9SeCShHijaogzF3;7+J=(iqWva1nh+qmXy<AEa2`pNpx{govazq4e&YPOU#%$8D=p&X~s zNG)6J9`U?LGg}lmi0MtTWd>fh$(D}ov*j6n=wwR^-E1-Gm@TbGX3MN<9uJc3p^Xlh zO`8s>3q5p%J`7-lPUws-FhN(Cq8qxy3_Z{jz0ezdFajen3ZpRwHRv&lb|vs z(b{b3x0U_j)3$8+j9WXh>FIrvYI)xQZJY;{5*#|~r3?iZPjCsY! z4_reXawLCPjuhYk4x$hza1yPD=kR&L9O;YRmN_y2&8>5!&1ZFKi#z@~(vG}KwkPkA zmAH=wc!)=Mj4D*)37+B^p5q0y+4dz_gIB0U2ijhfI;1Y%P<{(N%J0bc=t%hk`4OM+ z8DH=f-|!tjP=}xRh2QuCeYQ2g|0w^31muVc>Y+XwKot$426Z%o1~kzaO`wIQXolu! zftF~6)@XyaXovRDMhEC%GNxi0X5a@TkmExY?V*hh(8If6jv2x+(F`w9gV%V2cX*Ev zSQ?ol%i$B9!|zMv@cR-u;*S6XA_&0Ug8yMP>a`i zgLim~1+k1-Tn_)QPdPqE7Ge=f6LX{-r*Il)a26G~538gc8G(@)h0!>koFfjYIkE%6 z2tgRG;|6Zx7H;DnD)9jG(z$J+1TI-Q;tBV6r9HIK0Xooy9y-Dl7FeO5E2~gqmMf(w zLpe_2EUw}juHyzS_Q>UX{TWj@!U@he){kvbj#D^|#6h`|gk)r*<=|Xt1(hM38(Iyg z|7e3AXl9iw&CwDkN3cJf!Ly(0@|OI7kNAwQ_>MX>9hEB!M>8IXn#CN<&6P{bI9Ff# zaEv}687W9b8q$%0Ok^Ps`FM%fP&>gKV0B5Z*uoCm5MG)q5r{+zPL*-4xQI(|E6){o zDB%H5c)|8!u5836Y{sq2xpEs-sD|TiN5HE{_ww@%X@h!v7{qJd#!NBo!%0 zLpm~$i7Zs18c*uwNr+~i+;5d95AY06@%&%5rHTAfQz_^ zTj*zzC;c%117VIq7>uz9nU*J^2tzm$FmOhmxHf7e;Vbea0+H~y%aZ`y+)Y~oZSc>a z4bv{?$#l%XOw7V;%)tUI!eT6e$(20m3N!SAIR+v3YMun%$m9D8^5ikz<1;qg&y$VV zhV2;tC{H}K8cE0t`i&E>^W-Gz-m#A#dBV5sh#&mnqmnPHwerOd8$0KV7rfzvhh6d| zyJx=0;C#^^lP~Gk`On8;&-T(b~TeH>;TQ}4hASMm| zEo|4|20zipzpS}mGP~EOaDAOhlYOEu>W(UwqT#4wF6#O!CXT8lqU~7UL7M5;vvzD` zsbcD=S}g7K>)SduDNz~hs8%QX`VEwh%`NNMI5sj@G0|5|b8KBwZ=R#3gGw*`h82$O zE$iDmHYrvy*H^1?)GevM)3Irtilx4~x|4xr15d~1=Jl-g8<{wDDrpeu*wUfi6nzbA zr>>T&8IG-s>&?;Ev~@BoQ9a_=wyxe1{l-eC-j)r|JGM8kZ>!%V&8dG$!%D{v4)r(c zYgIT6vQ(>e)Ge;>px?B{$)ZHYKZGID6( zuiwJjd7NbRYF4Z`(X+B#1xX*AfWOI?FF{Z>lnsg@cxPF>Aa)AU=XInON7nCH~Z zK{ZdmO@;GZOHF$xvtrd^{kApE3rjS2I`ynmEzxhM?y}Ufv8PjS^M)1r?M+-(mNbrZ z>g&+(s=l_hi=AbY45$9Z4e#oAuyt8i(&UKKz`BN2`Z`LNO_o~cod%h!)#&S{xoj)Z zs&pFSp!Pvuufk=AWz$+Gi(<7p{f;#*drF%AbQ)gA7YyjDyE<7m({Q#jS64SMFmY9s zG}Cq->7d@sz|h*&!?L-l^XOvrb_Pbau0AEr2Ro0gQ`a{zR=NgQwy<#?XWq!fpi`P_ zNJ)!%&f^^#^)l#O;TmDt(%yMuaU*ksE;X*vB`tS4Pp)fZX<(wRh_`Iz={(h3!`h&$ zi6XhARiyKD2aPEPrq+sd%hnmrGmACm7<98$@RPwuoM+c*@PoZdMZRU5^Uib4HEj*d z(iDXyZ7QATJ7{h+=ux3KY}vNfd10}pgF(+4#qpB1Kb;rXX*wG8Qg5_KZF3TMn`y24v1m`TerHJq5{(Z95FrE8Q!%Z-MUDm(^Rb*XiYE^g^yIJw5dqO{9T*Vww2 zj)qgzJ*})vG!*gXt(1mSO*}`JnrJH$9a{MtPP6tLXVukIkzCv=+;FO%4C_SfInc67Q&0D7#&P?;1S!y~@k?GJn&u~_S=Ul69_KNJ{*2RXiYdja0cH61Q zt!rIkI7i)Usa1DRMZS5P3d6Z3UMowxM=B0Dw7F_H&)Un*$}B@sSls5W;e1=Kb){xU z6o=~CR2eQ%dTp}mab9uQylsu)!ZfdKr9CPYM;+RJFkDpOwZp1st>Sob+d9L=HC}s4 zd;V0MtZOSqOVqucta@pL3bndTi*+Gn2I1&8*%j8;{6M_Bc>ce_;Fp5Ga* z@s2L-yVLDTU3*I-TXmm!tA3tt*UYu8jqFT(l1uwVy4`Tlo?>Kg?UQcRKf~=-vGyFJ zHMTz4rTve%-Ko=FVzgH2lW#TPyxTqV4z@#*cG>#9Ewwn}{=H7G%4oOJ@1xbQ^X@;)JJuNON%Q+!I;_(DmqW*WAB^@^ z`2DaNUhDp+xMQ8sz8b$jrNe)^|E=pN#*XU#DkCg4lq!St)s3A@{8h^=wUzaE=r=QV zw)Sr{!pc;sdRV`mv5T#LlQOHpO0^&Q`o^wG|K=k`*eDwfGB7b#r1`fl8!=C*xx=8B zv0H_I`w=7Ul}!#Cm>av-`0JL9+^KB(!@$y5sUBc3Vw9({`5?ppv-jpvO=aoZ=m`W& zLeQXK1<fRJ%UzA)*9t!Fae6+YJ&R!%arLu!{0&9j{ zvVF0<)N?3!T40-Hmu`PbQ_5h6%nLSl+hy9Ho-Oqo3Rx0-vS`P*FF~B~XLC@6c!s^e zzLan#XozD|XqRPQVtXTpcL%tEI{d#Sxxa3+$?;}sGa4s!c) z`I+b;UTmRrmP4EUSf8d(m;mzFJV0!QP@PWH6j&?Q7&^ zhlaLH7y4v5y|O>ADa&GSoiFt3c6w`nVYV!HXzNnpr;ARIgA^goV@IHb{tN=bp_U*n z7>ckF24oR14s{%HAv=;J4C*Fe9qN zixlCYuh=d(C16SJ&VqKhc?97L3RwPD4yXWaA+r-9UY3XDN4w4 zDRH>WIjd#wAQdHbyGR^5lFv>I?O+z|Uv!Z=ToIhT#g63_r7*~Hhfev~dqc6YMQK^& zHixU4vk%!jQ;QCCle-+Q&7OTcv@^Tt>qWB4p$l>D3452YD1+guap)$Tdp@+Qvglxz z>!d>u=iD57x3cI^x9gNcZ}Pd9L%Vy5zF%~map)7Ad(HkrSCqwYJ38yoFF*HY=!@y1 zoGiCj4g;EVi|jq~MY-K>Zym1Bo?98(vs83+(G7A`AtbPXIFu-lK|weU5+o+WaW@{lPk!>Nj;)-i|%ob5Rb|oA9dBqV z7yGte$d@x(NdpIq-xOLG}+e3V>uVE8~!@wG*-8OI+4RT%+a>597<-m{L6+~$Sx_CH)d1`Etj&u51P+c1E4eyke!5}z2msgh$e-nF3pT(d!{i3O^ z2soH}YNDG#cX}~fT{C29x77N2rkve_MI#P8Ku5X`WM~ z2sorXb+4NlY5JH0}j9}LJuoqodb6*#>noF5&|v^o7O%eTbo zzc}Z$0a>Ke&%1pkPQN9epBT1=`G^IlYqm*)2|tuHBO6!3(toSSDyYY zYxAVj66eBPK(6xio9@k1PRq#`UJmE>oc?`r^U)cncY+JA1CHoUFET!zby|^Mcr$!t z`t)+vr>~seYc4DX9GySC(*5aMr@zf!SQ$RLboy_LpF#u(DTM{{Q6(@Y3qgPpr6wbM zn-b$}7KUKNl_CT4NF^pcEG)q|MQT2h$1GX3#KIBag;I;aW4sb1)1N>vQAn*uj>VRk zW&2YIrdp|W;PKQF^B#XXVbz>;qix12fmRK;^90HP9>oAgES+Xvh z9YH{GYl(pc$`Y#{b_~HRrItKW&{Oi!5<8B7F07>np3s$8GXs(d=89U+krUG;w%Gyc zgw@(wM&QZ$l8rq9nS?cSwSFTfmr6ca3g8nk$U6T(0jd0D0-I91?7$Mj zT5cUTu#i;h&=V*jtV^j2A1P#(IxPiC36_O*k%2;9DUlf@Cs--!qDO?WrOw$wZG`pO zx*dToWpR=}H;QkXgb;Mg5_XQ`4nB*&(k8A8YHg z0#DDE`t^joC44ehpF47Tsr1vO5QvCFHsl4Cpw9R+IS3-2*ibN1Vsj=Sn}Z>4;x-fp zmXgi{^>DC6yOf6Fky7TFkR=X|XkXY+8hD0xhRfs z^-OpVmrit=Yp5A1%RaMpiOV7qkh0o9vG7bJlgA+vi89%Uxbn=lY+eMB#FZ%m%av!M zdw4NK=M-7XNO{kh&zE>{M3+KYd*E5!nH|i~WFlE1>l`^deP(BNXgbkVE9(wCH-Bb# zPiQ95ZBEuda&GC&o~2Mek%DX-43wbC;+bIrB9+)UIwG+tOUMo@A-Z!LwSg6+vZS6c z36Yl4I5ASeEZe^nCM9|lHr@)X_Cm$va?U{z|_fu8U# zqW4_mj-ZDvKa+~G?Ym{Y&dbS*$ zBKo8>y&S3ODf@nD%M8)Cu<3Q+d0kl+bL%Y8Pto*dmWw||D z-x5EaYg!q(uvB(*X)8oxA?2_jDN3Bjj6jh5iE@)ssg3w}b_9mR=E{*lwIp#tPXv|} zkRmr9t!0W&E=Ax-frWC5pgNwokQqrJ1u5iKqjj<3qU=ZtDOfAF4ysQT7xzTcNg;Fc zjidG1;?ql!ED{H)zy~!5#ih(B4v9-tIE*$_ip#R2B1k;0f*2%IipzVVVo0GW3i7C| zM|^H6DvlIZsGtTl>ckbyZONo?g~D^RaavrJy)B)zMXO*0HO-4_dbVYfw$3U1Mw^zz z7nZj1NfAh;e~=tiUd!ArAVm_DL8Ee;^7`!UC8Q{>k{hHTmCJgzOGw*Nl;NWaW_i=n zb}4Clp)xW^$tzbdqvfP%g)(|n8C%|*9o$mbWuM(~x!$o0CQ_R+e{Ue>O>q?|Xq(O1;p`ZU^Iq=7!k~82+0mXDtaC!j#p2O+=Gn2O z7@Tur;l~$lHy|d=;he6`s^JP=Y86X6+s=TXD51g(4F_sU91`H$Ub{> zX$Q+W8QD@BbVYdf7BiOPoI-4ojb5oddnY?K!a0@Oq6q3#p1s!-8{?do($X^8*>m>6 zQf!>_mxV3uL05HWA2N3)J0DQAbdFw~KKm$pXS(xO+LrF1Yx8Fx_w3Ac{(7#ZfAreY z*`Jnn@}1L>mj;8nQ0Ja7cL|&`h?ho3yKK%q%idMu{0;Y#HmI9)?s?BHiSxmfOB18r z%yTc6c1fMTExdFqsE2oMj=5Xzd`NNW-e^zkxrOZAZO-3mFFg$EO+ELrXLpzL_j8vX zkM?Gt`}5Lnm2)Pt^+`~l@Z4+W7aHd*V(atKzRGjIWq)yW(m9*kIv3QhJol#Oiz(-v zl-8G{{XOS?U;1Lk`EX(D>!1PMxkculS?63u>zmPm>2u53dtNyo(Y7uIU7tU<(zEBS z^U=B1mC@@<=l-^|2Xf&f+hD;elmzAzhj7Uwwwb6^HWK5UIE>3NZW}UqkR&nbjl;Se zPiZq(4>BdImg8_P`Gsv3!9zR=(kGtaQlMzFQV+#S%yQx>E+@2Y*1^N667$}8y35JA zwvFoHY{{DCc$SL**^Un$5lSq4_HtYbiR}*RkxI$BoV^h)LT)=TcvLB|>fIaTQk2q8 zR*&{bK3d)z=OQX>rv|Ha5^JA?WS3$^yQf+`EwRl>NOw7_}3NS60w+_D#A}aywFjCzKVQz5AwIs!}=*s3&?V zyqEXQxKtN*WCY*PRWN+^&$`qoIu5CCOjr2i?0@BQUfYosd~?3SuXq1jmkVj4dXur_`jjih>RZgp zkmY0?xuNh%Y4B}cCD$i~K$a=4l&f#YR)*%JP{@tiD;2?aQY*uIQ|RQTxhpm5JK2?6 zms40|IkK}h_^z-r(kGQeRuDU7>bsSd+j3GP$VzUfBKV%NGP*Z4hTNRe*`mJJQ~CLF zY8?4uVP|{reO=`apR{Chi=wkreSf-gXHHr=`I5G?JNUtTl1iTF!ws@>T8Chry3hs}A%Y=ptX6yZTuDD7)(G zY+*iaembwix`=d{u7m*Kf(!=dP`& ze_EJ$=Hld_3@l^jO!4$3mNi+R9(=Uj&&VQ=`tUC z!mK{IoQ`uHDeSTcdCIFU^vNK&jw-sW#-7Gj7v*G7T-Dkx>yT%u)y2ITbl0)Du8m{Q zva3%oXRur~$ZmYd&%)|bpKmy>T4J}u*w2;KWjWtOxazpw#E|F8>hj)iVqEnp-Q=<7 zJ=N!yzln1lFYKm<{GzL_@Hv?5I-%(H9Q$Rux+>>jy6X*XHzVZ5e05Fl!A#eibKQPp zFP5q=EFa{%P9l5!LuOGmwLaepTyGJ3g2rZTYU*>oEpffg?cs*Zk!obU-%4EXr1XT3 z%`t15mcNy{-Yx8j44LQED0~jdUGFJ+qQ~ZAYnpQowYlEc_Us5*NUdq+&I$>lCs#KIB(nO}o!`8rO%!-lVZ#D{DG(zMFKN=Juw9yj0e7_I@|z`Y5IM zz}U;4nrqA7&A9$h*qagZCtXdq&-b&gj}^U##{M*2)0^}CE7u>jy;&iDp0DZe{r;`% zPjkJwV}D+%xxV~8iEw*D>?;_1WpjQwClllLl-pMr@|tvhv^Nv$ z_AI5ZcFmQOavZI0M48~aP;`8zq;5pMI`enrR|<@tNP z*)eVlDg7;DZ+gx@Sk8`f`?avYJ>;*t^ACM;lHFb^`a8$|I(`08PENYppS1nmA-~U` zf83js>GtQj{{FGwm(KsRoWpl}g&Y_Rd5gO6#OJWU?KN>=bnLCog=aa3OWgj8JD?3& zBwcvkdsyQ3Tgt%1*dp`7i{-;ox4#q)+zMIZU6}L9mAk!B4BQ)AioLLqliTL@SM9*V zkmb}1FMD&l+6F#FK?Y~l zf^=iw@l48^`9VL;s%7aX?~d~+7}StI2Vqu=_sthjEJ#B^8iZ}F-QoNa%39tKmxFY! zb?D2NP}Zdmg=>&LwNCHyr4&oyP$UNxT1)gTkW;LbL(v-4&RXZg1#Oh|x}hB$v$R@r zUqKh;qxqrT8nc{Qw|5093KlgS&p{W}QhiToDAuImBn`T%mUj5WB*lg|oWe10uJ!CY zF-5UW9X_Bj@2&NIcVdRJK{%YjS*@>S_@11lY*Y>((yV?|>vQk)-2b4`VKsN7Ka+i<6z9{{Cx!oDxNe_puyPI1soP&sGE2rg&Yg#x}ZJ* zmTH$eQmnD?sS9~0z)|gmBc+_Rp>x^0Jr5mXXyRKc-ou8Zyy#!#J8 zM_V*jy>*|z6UI?pgrn`8_4>LUzD3DYvU0Rjv;I-t&cj9NR9D?-H|L{;y4`(6nN+v= z(SFTG%XNF+74fMQlzNbZHLH*J6$z+Rl6q8wwXIJ$EGnV8^VC|7wR3$^pGZQbrK%@1 z);{(7--)DD525-N$0oEs#kW{a^;D|wX>4}ZryVYCqk8Go4>`7J^#}Tjk9JYL=hcrj zwmJ1*zbjTz>8PWGhCJUAgu6dUW1_{`HXJ`(f^lc_G)OMqxuKx11nVA(JY9 z?%kO<_b{QB%5~H?RQQ%9yN4^ao?6F84ONHB(%rY{v<$A(LPJeoS*H8ed99z;X}RIT zyE49e1WM=6C78)-eZ>OzNRlo{OR$yI9~PImNAYxAF40*g>k~`dx25XBwL~9T(>t-$ zeY;Q>$t8u#6u#wh_h_XqT1(m~Yd&1w=Kh&Zw}b1PCTr;{?{fcqUbkE8oFi*}SFUo8 zLFwbUE=96--?JL`9VC5{)}>0;aro?{dn`|%!X-D$I{VH}x$jKXAJCF}W!K)FopIkK z)Ms#A^|Ef?bF=QdmHI6S=W5-SW!K+dMBsXmQk?<6={qHw&F z>mJ&u^{pV#l9c1+TKAoe`ok3z+CJTQ1(%lAIMG)@r|q8~uhG(S8gIU3pfZPT;E)g`oVcsI0MhI7;NzG?~WVCsztEyJhj#k*=L?OWlETU=&n z)0}UOoOVcg`_<)oF4y;E+MTTAgcp_{bk1*FN%`)lbQC+8H>1)YgTGm9s<-YJTI_F zZsAwQ@hBwSa?k}<$=Bu9MR*8#w}`x;X1P^=U5rOj>MgP^s8{~cN?n|XNO+6N3)ai6 z{pyoFij}uKb-|D1wz>7`9;bA-7`%`L`NsbGOpnv^xBPS=%kocF>iHfesN4QLj+p}Q z*C6mHCEX6vacmWKxeX;AXLz@{Jg&3Cp}#@mQI>i;T*viMIIT2DJ;cJ>kvv|gg6Joc zdz340N9%Yy70$V`HjlHq+dFunX$o?`tjpuv{O#Sk&>V%^icIApLEVYxg%v5NevKNB z3eue2858!d4lOE5~@&@$MG#BAt~%{c@~led^s}U8IjPWJQkiY!KcpjPOlI~tN@AC!a?*8UX&(`^S{kqSWm3vm2`JQd4`-8j~v*vieivrJf(*03g zjBRs5?!^+%%e?zq-VW#Hr2dN%&yLjl6S^Hf&HGm_NUQ@wf4_2R##S$L z^1&P37mu2Aa$8?{4(J{%^7bq==k~Y0^}Ig+U`4lQx%ucyE99j@O~FFr%r55nwIRF) zNmC~JINOWIbK5XpL%bndZoY#nO$|7`c=*2?6c7oTa za>`1-cjv{T+;)nWS~q1Knviy}xWAq5H8wxBQJ;`=@$^bN%S(fLh!0IHx>)LWnd7A; zJ#^3~R$VO1y&U1C<2@vXCN*Cy@4p=5rB8iG)+hB|JhyT=&TCxwkQ%yAf3d=^BiU<0 z`Os6p@6pAo+>Ugw8@h*#(ESS+Yx+Afy>8Aw^waNOzIb7!gYPwon)VM(HfyQ%yCU$q zMVb!MC)>8v=UyrCy3L#BhNd{T$oj8HyzZn(TAEg_NWJa~rz1mCLt7MnopP^x z%IRo*>duzt+|D+y`?~2Jp=oI?E&ZKcUJvG{ck9z~T3S~+RbEr5NAaOw7PYkdUDbF! zBt1&fe_7Sik$ZL0Ynt~cCGaO}E!S7BLEbZ{AM!%e%`Of4 zbs@Z;kbWr8r`ui{&h5f@Kjr;U7@FaHX|%r!>-{YChhlw(&!w@IE}Zwz!XHXQzX`ph z_3I{hKUe-xuK#A|C4Fu;#rqfC4;7&Y(=JW)chkLJ%>PiMKbUjr=1MoqdlvP$HuT$~ zOSk-bINo!l$1?r5RhRDM_C$Ek^ByZg4>e!9*WVN4y^#92MSrOG(u0+rIPYJDkK04P z(_eb%*PHD9Qu(-3|J|cYk8*p{z5k?p+#UM;!llRky_w#Bo`2l0|9<(>PbTA%p!3A|sEejL?j+O|H+?JM#AFW!&Z&@AWH=ly*W@842?oX}_aw7yvBJ1X`5 zi}1%=q1mCWbAJ7D?>EXH@9DF5wl3uMw|W0n_v6FRoV3=L{rz3uzt8{pSf7*A`sbB? zmG@iJPftP*7q!0j8_;+!l74!wKU~%NTkgQ5_Y&`?xzOC^);IkFQ{Ky|KfToF_O|}M zGBD%)PWaR7&?EZRMZfE_-Yd$V-sq1!YF*B~{>uBk?x)4jqYJGo{ny`m|84%K75&lW z*1xS>hv<;m3@nUq)&|?GLeOE(GbZDF+cx7PDh%BybOsrg=iFv8pu*CP(`L-a^L*M? zy;tGr@S+)uuw$WZ$jyTUx=Hhl)%dZUZDvOXDRfi)jCI)Yv^MjBK{|ca!pz3;<2h|> z-Vd_q2(u^nu>7Jni_JqEI@0-x!+3sG+qxq|5p-1O6Jl6FbDPz`Pz>EH?Fo6jpttR# z_d{`XbkP%P*a>}`_2%JZx_R>x&+!wF+H8*ur_)#KpD@BsF0^eN7|x`xS$N_%esa0( zllQ}XI>zj&f0)3m9lv=*K(}yy8Z<7jZMQozQbJ!F`ji`1=-lowFe0I^OM4nVUg*>A z^nOH2w=8-Z872&ECvF~<)2*7HMvn`3wmTmgZKJQ(Kiv^ll-5oj8115ewD5HIcu`Kf z+xt-!9c%V1K1@{9PTj25(5;=HC5?-!+G$7BlXRQVXDMOD&F!88>M6Qy+Oq@W#l7v` z@6|K(4Moo~!cOVi8Jovu=^LA$9U4FNsNLtt*em+S`e#{Trx)7&2FBjfKUsK|JAQh( z{nPhj5CdoSb6!}9*=7ID8UzFH{Byy0iS6ZpBN_~2Q|QlyVWrNOg9bEMhF#jv#p9(u zmqXrba18sRpG(8egkI)u))E*F%|DlqpV@gi^oW+iaMb@?5muIVIeb7%XE-hVTr*yl zb9w7~EsH@gdtMtRF1j4KS;t`zouA9b#Z{NL9nnQFNTJUaVdc%2qX%>`4Cl1xE#u|A zmp^~6i(|MHJ#P;?tG~Qsvp$(YZhqc5e)iGjok#TP3|Ia0?yz$Umv;~7GZ}6R&-=&E zEnnXAUe9Mx%zhaRlbChHZypygsLsEPj!SGi5{`_QFx*3b(S}txcO(sre=lLs(tep3 zukh*E|9)J`@F@D_R#;_dN6O|2Im5I0mwV%tJ3G>jOtdk)^uIg|t4ixQFfh@@@Lu@k z@px5E$Jg&CR1CV=izi{#MI9NNZ)g|{=NHe%tE)N=9=S2eV1~Y!3#)1FI5co$is6&? z;^lZvZ^!rVZ_F@!i(b4AJFoA^+I(}C;n)1)&G`989XUsCzG7_FzgP^ru+Wh^aPuwW z(}fo+;}@1Yj=sMMFwKkPU=qs= zNSigEsP(yW^8F-^8CW!H5ndO1rEv2t0yC(2)@q_|=ar%(wxD=S@V_hf!i_6 z(6l-7gsk_&%Fc|K@DZrfRZe%}!oKIrWuchIaL$7K#KP31hHUC;Zae3!e{gH<%wy|sCogt^%(-A(Yp)VET zoy}M84NS-Qq@}%Vndt1j`r!R^oX?jaRZB{3zMh@K5ExzZ@G&oi(9)?*URVOj&C13IKFS(^xyyNZ0%wN2Y-q-GW;_d13n-K zY?l~W{RI|gwaE%v3qb-H))?##*cJm@1!u>atcuvWJpc~&1O=>9GYs-yBd*Gki zCP#4s$Q0jTY0QI-ZGf{73UQRc9jcbGDQJs*Yn^3c$5^v`4+!!B<*~`qjmOwnJ4`|t zgv%f1kYwUGG~T8B438@l??ppugjx;pJ@1Gjd9b7~iW-}sLfD}bF!+3o5h6iTs#38| z!t2C)g_H)dyR0Ng^bH<2f^Gs^dSiii@5)~U_n^C=tz(M5$w zIo`$Rq}_n=jS|M%*9T)$(KIv*ΝK<3*V5M4iOnGLj^sun~B|Xf-K_m?Vngd5&Rd zJfanw5{yrZA|_ztZrI>4LmY$)A!vQd0Z9j&Fp3S-uo6XT2=T&$hCaw528|o3^I4Wu zQ|N^R)&dJaAdh3fWsUHu0vL;!XoUZgQYSJBwn#!p@@ns6Gq#yRY;n{eE}#Ua!tNDV zLxcnoOl2sY4~E}PhlRxW5iEqJp0GBCB25X#{3ug7RRoJ87^5RZSSgye3KL`lPFE+g zZbA;(KxzzI;4UO2iD3DdI`p;%%Onvs9}^&^o`4z!#8J>FHITTM?+#&t#4tV~2~9EO zC)v;->|XJv5o`j&U}z|<#q;|J#4+mx5fzL88b5+b5Fj1l2`suD+IR}Vr!*oM8p{MB z!wyah!=At-pq(A^(<5*jOpPb7pwe2?qvPO{dr=&Ph##S{V`z<*ftA+kEBnwHjn;uY zJRF~x@2%RE43FLKvW|ltFP#w`#L*vAAB6K+HxPPClI6jC%#dYVm_=|* z9FKTt2m#JugFlUtsj)9a6l`2#11&gW4403APz06$rgDMA?np=mMW{T%EKxymXoyFY zu@GZY2wWNe{%Q!1<{PCOL#u3zYhmow5DuDQC7eKGjIbzc2py@zCW)hXM9;feUb|Y% zHib;YQBdLW)s#X0XKWbm51W*+S&+HAkqyk`oFqz@FCD_FunD_p6IiyGlw`sZ@0H5L zXgglF1p<$@hzavQfni|}qAgY-kgGjLHpIoVuxwl)aa6pQR~tW$3v9m=OHos#qP_db zdol4em^H9DkH|*Tz|f!Aq$Zp|;7Npfk-G|07i+7^XPsbbHtdBca0_G0NEJ3w07hxA zDU=6nLfMCoYX#_y%K&ss}!6uXP86Z%eK*lX|}lMGk^CKy4GQXy16CRk+lYxQI> zHt`B2EapiyD~z6iX*0#emmFM;6cL}A;zzn!qDYpt=g_vBatqKG#lCSPJGfdLRd2bD z&5?;O|3Ew8@%;WbqMG(yhycZQVWbS3@S_A(#)+B!f?Oz-#dl`lqUjDg-kflHN_|LVFWm92xpHD zW@F>w!2-~O2NAaQmIVkJN`4ZJ&!-MaGV;4kS=fA&6RMnG1>iJ$(W~)@qdZ`{6i1Ot zfbuVmgV<^j2nhImmLN(+7}g+Qm>>?;U(ponu0|V6fEOnWVQa;f_c!G$?rG@xqK;~+ znwmd^O~}ts$Lr)Ejs#b1=ka8OL^k#~MJBcmgbj)8CMYa^JRFMH9*-iZ?&dTUNXHL? z`OHqn%PB_NWDrP|juTB{jLo1iG{{mg=t!I_N}Z{e+*cz{QWBtuU#%0MWITo~rkYx? z)mej*$ssHrQzwpP)ka}yAA-2SMPY%8eLotT@+oT2(>PYzuq0TN)d=H@00)7<&L`A~ zs2ZrwlCA-P7=#*QL6kZ8hM)IK;_p=fmq-TAfd=Uvm|KARG{$O*nvDQS0UCM&LII3i zV#0xDp|!_sz|0>{3s)PWbt=fj5=^mmz{fxb-AzG02?&8@_;!zv8pJ|(X^;eih8jEf zLCMAWZKA9-5E}!j;TFfJjn=^kJgz)S2U~?zsc;6`@?RrS^Y@y9ZY3D#7i1&%#}iZ# zOJh9<$M8X_aR+M$CmQpSMi@2+`vh(kMU+t*z?&#QgUuJcR-4(0YQ;#XL@J34v`7?1 z4(0<(jO1g~kaNCaYy}?%7{kV4VMj^uBmqx_8R8L52muJQAWWTQB!ak&N7eB6LGE_^ z<7ynf&3`QtMU-tPDq+TI>@XiTndOE}L?pXzV5@QQT@ff+d_D#dX<*VU@MaUCK~v7+ z)uO9diW&pQK#n2{BfQe`*!@lO)LGXqX%wMV_jG$TMq?$gt>uyR!zZ0YUCDbx*aw^yyYfP)DZLq1duB|v$z9hyT_Qwh_47jLpnpm1z{HgSvZ8IB|>T2tC?1(k61jMFs&k#NkQon^| zjb7o4{7-kS=fvT}fJp@H7> z-hDPjR&KSPrN#i-0sbNI*I#*n}{W%rXe1 zR=5O1{+9q}E2L*)Suz}C$W1_9HHO8)+M$nwF)lr3L_n6W#c<{M$9TvmB22zNUX-D+ z!^2q+q_#W`NDmkQPzfAEdLiiu4w!|rOY9u6!5SQ%P`5oYsNw;e@=HZPEZym%mjvd7 zOF(DhHyF~@{+ZYao!Dsm&M@l)Q#R0MVW9$?d$66bgc51sjRQAmnWBb1gwTd=FGj|;L$`QIJ0y_vs$2uL=H zh>EaC_|dqSvab?Z)V_{QQ^ONP7WvpjkiNj@-qOX`V=C=o~o@~6K+TkJ(=Cto>ugDC~1 zfK7yd#KGkgh9uc7hU~bx*y0!}egkF~_DLtkG{cZPK`n}t<7tlAS6|T9cqH+S2CV_v z{Y*C=%4xCwjYR{!f^f&NuUN)v$*M{i91CG-(3O3v9I0V){!(#YgoglmK&JeCDq}z~ zBD^8X&suY3Mx3F{n*;|SkeWdr#y`@azWcFhd}`u}R8#0K8X`+f`7o))UJ-1jVm}%} z`re5p21za1tGB6r?^hbBD8>*C@<^zp&UpR?EN~rNjul z*#P9jpjF2F!%Rb62{;Vho{ zpe7L7WH?hX#&Dfska3csm#{dn;Q|@kT?VIyoj05U7zB3KaBf&19)>1C#tty|SH5c@ z6Jx#yWMYNHLB=M)V0j;{0^g9pENd=;(s=tQ2pRG^eqA8Aeqae>7#=hWY-nQ4E`f1f z_JR#i9SJkop%7bd1R93jh1q8EAX^we{oCW|508IC1oi{RkuK0Gr0o}*A>#}lz71?#p17Bn_#71#q40|UX}MqbdG?gdYn$OsO`6c!Bd9oUb9SHhgp zhLMAD=+0Qkq?QXdjK2dr{)FFU5(*yYh;jj^0Tx(=wC8Vt9RK?BKY8=2^l!dSJe2Tv zcWywf7or?T9yvMwI{m?&|DXKr-}<(_72zMgy*AJgjxr8?{cV!r%eUXeCno)Sk2d*m z3~TbC?0@|DKYskb?Z+ox{gWS~4T3Vig;nNO_Yq-MQ?L&VJ!AUq#5j2Eie_C3d6I4UMKsF~^`Jg|cQk9)>7h4-L~{)K zKr{=2!Js&Etjm|tte$D%O49ylRtAhph>K=5q=WJa(JYvE;I2KiW1~ zA<{e!whWs67RR0C3DDkdJvB6lgvPxtpsXKX6`Oaxwx` z!+!*YSb*n=nI}Wu<}m=tIC?ZI^~Vh1DT+A;qL>RHH}h8zg)D_!%`+hv(6WR14CDyP zI)P6IaLmq}5ANF>0eu31BU7MFGZ>5&(J=Bln!}@iJ#YG_nRFkueYs z^kgHrz761vw!lj3z&YG(C6g=;M0gR=k!!6Ime z*Scud`upIiWB>DRzKDH)9wx2j21S+fNwvac`?Xwq`?ZDJ zN#$WpWuQhym|V;=iEa6;`Ce*gZ2Oj`&zhg_?ce44k&9!Qc3;PX1&2RMIaV18f9dtd zmRIe)xHhaPxc=6cu2awc+V%LW(__z`2p@k%$D)JvbKi9Dx-l~S@;jgN%`cWm`6_8T zeZ$(qz)kJR9v>I0?QRPK9m}}S&piB=S+$j5xs=bY-AVv`<(AqIGHt`X)bD#|N=OY~MLJyyZ+`qT zUw&uqTXtKpyx^1h51E3)3H#1vY_C zM}QyzAm1Q7fI)L;0O9})iP|cFEdZYY0J4PE05}7%0n7ka0~mfY4h66SNB~$5Km@P` za0kEvSO5e7U;zw^uv7pVz&Zd7Kpns?fG_}ifDHit0DA!<00;nn08s!T0FeN005$;o z0KNv`0oVet0KNqH9KZ@-E5OG9!2llt0BZirYOBBgGc|hw8ipI}goT4Q!>i{%^b1Jl zKlJ;*px@GerC&Qf3&LSBrp6HXS!DwxFfuBMfY!s|$NV9W4f!E67BK{~K6T7!hKAe+_@XniOi&>{|;FA;{_#8UA03D)jq`6pud7$YO86^hR` z&*Mx%MMfqNOK#-7!KoyN@G-=5keb-{vd-<%>iLX~SLu)C74V%E@y4tbe$ICG`3=ou2NTB^TI_YD}1RO@NooBz+2$oLH zEb&J{2w$T$VAf(kP*i&a3{xb8M{L4JL2Lhu!*tAo0%7T>#I+5_1R<~z&V{w0|0+lv zVzNEm5>?9ipfVjLQozLj_~Q=h4<>^+mw`gSV$7fZWibX&$On1@D=+`a-2Ouy|EtW+_n-4N*gI3m@kBFl+kdPo zuKB}l6Tbc`G3DEoufO_3!rnlxpF}$TSW9rcTKk9Z8eHz*`YeB))n&6VtF}MfR)xg< zLmC1F$pH%TgX9!!|3f*3M6*nA2oBJVP6z^eI}+jY>-Im?Dfb_87GZGwC=>t0ziS-c z|Iok&>AMR^R1+MI3Z$+<&cvA`zKj66(GuunQ1XMQW&d6O0=4M};{x%?K>)pLat?|# z$0Bxb2cj5}Z6A$9;5_~=Tz$+5EdPIK-DuVa>Foj}CDWV*Cr2Y^ff9TRlpc7xpLx1L zgMMPr%bS7V^8|%x<}9E$SwLB40&S;302K@KhJWm7jssFH0=hwFLB8e~ATA|v-)JHq zawI21F62x&Hkyb5VigB@kvZ^B4M!N{Oo&D10K}m|cMJkL0BGSrAONj_fTo$pAT|L( z+XP2BfxZ~dM*xS2pavB*vx*EMwryvb%mAtW777KzISi=bFrZ;q1I1|vN^JqXTnCgf z#asn|gIvjc2n+Nt9kgOcj(|*n;v|8tI)HLGgVr@?ArR55AOcXoI0*9rSjd7*0MNiU zMu&r2;d13bKF2{;pns%rz*pu7xI;9t4O$P*yAC9B8_YG(YM_(X0R4>z{Xv*Vz){h} zOz8Im90&Iuq0p% zTCxQNHQQ38@JqmjppU`zd~AV(wx{qRbD-N%K$}~F>w&^&;J191OIq?iU9s@dtnSD2fN>s5%9(BL;-whJ24J21%+Wi_5aPBPI$ZhkM)*!+YLIq z?rW>*ul_}6|KWlLooyg%@Zzt3XG%c)&%42)Ti!DtRQ4zDF)-uz;7?!gk9xt32iC%j zuWklw&Y%9a3G8-+86V+*PZmHn%=k}FVc<)NkqOn__YcaLi;Iq$~L936pV z30W3k3z^F~2MGj7MhHoOjX(l{K?o2SgaiU3BtQaVV;L_7VbBF*Zfg^JH{EQ~#G7u? zT(;>ZZMSdk-rclmlBU^iE=@LRvYShj=CX0p?KaJB-1mF%CfQ$_zMua3=Kbe=JJQUV zdFGjCo_Xe(IhT3n9FS=w(w}|q7kA%<$Mi40vMUgNmW*#62KK8xMD+FVi-2tb)C>Pt zK)UH|ihj#~%{eBzCxvP5U+CR0ai1gKu{rE>{*U$Qa zr-v6Bu>poPuYBB1$WQg6yDq|0s4V_t@5&ay}TK(S75PMleM0Xwor(geSKau_q#an;q zLh~VrSMPouL?J-&s~^SlasLDoDE{(YK@8v__zO_JyZ?Zk9KE}CeP`u**~d@T>O$@FMx=d$-k8lkU)+{_K~(x>iUeYZReH z6oZ8yu~@iK!aWfYF$@xv1ZsXDOD28@(qMoLDl5cBqsE{y8l}ccqvlxkvEt&23XQR{(j+w*Ys}?Rqp`fWpt7=}wi2#X zT3V@<8Vr@iT4|-Rptz#4(o$=#1o}Nk#B5PTg{89QnDiLtlgq{mU5T}_s9ajEvsPLw zOAXReqcx$kYNcA6Kt@-oue55U8nZ^DK@p>_veJr*O_e5{*@{x-<)u=iu2foXw3e6G z)|R4vgHbEhm6unQ+mK_n8i7+@Q7+_E8cUH|LD!ipO7~q3X+wQOgR!CEsMJ`0R0@FcC}0SWAFn?lJyBI(QQ0VM)R*$5 zl`Lm6nFwtvH8q;;C1x{iE-y6|hz#ZBrg9VhX8Z_$r`OQb#5S6nni|c=j8#>}B1MtH zT2yLmYHHFnX(|lI^v8~wj+w>cW{1OGQiPHktJUdr>Uk@Cg%DVsEZ5R{&zo^!|5EU9 zi`k4K$4xb6lj+!Tm(FZ1GF4dHj+rVTU{g&wy0^T%sNGnen`_L?Ezd1#EP}IJt4>-^ z)|6XoPS!|IoG3qe(t5&f1@NTtWMfTvM|lSt*V$=pZ0u+>THBf{I^1q|MQ5km-PqXH zW;C`ncb#+}b9Wir+M3-)cahtjC(X+P4gzf+93uO!rs7zhGVemZG#^Q$F~7Olo?qe- z^cb$9!k(Y+@w5pLV_SZ{F~8ziex)(LvjgyaO@6*UUt6Bv(Uz~#m^Yp64VZLH5&8K(h@sZ$upXGlQfTI`Q{R7Nn2xEa|!;<`f_7qBiG#A z*xb`2ZDBc1Z{&;|*9a#TTbpq(9+LuuYK%S2CS!?7%!x}ly;LvMSYkCwt=4ALRAMw* zjmqj?xjOc17YC;^9>&+eyBKF!Mqm!5NR=u4+CFObJDYMy7#21wm6!rn>Q+20|dVN!qv#hMgZX6i&Ieh9twXxqYU{|Y$3=ZGm;Ls5MWkaWUz0=#$UuI}A z4)yjbh8zWgtEv*hp@KmpqQyf)#e;)(qp`{_v_WAs`u%}GU=RQyZRCw>{RXb8DqwWn z11OBc{&u6oIO1@K#Z|`s;^HbrRYi-@VSrvjC}v|>L9ie=IyyRbIyednW5Tg+eP2GrC-Um%^`5I0h^hzl9s{SsYFNrZRt7fxoH1-#0N~Fbque`UiS@9RjL= zJD{-m2b%8T6!i6#8GXh+onKdF^`kHQoPP93pWZL1fL`ARA^7<|=*TJmsZ*!=>il&^ zzt@Y}UH;wy1Oo#uqrxB%?dMRUB;4BC%qdWV#ZqFi7zKSZ4qM=wn*5d~OT=O+EAv~7 zTv?g1tZbwVzM#TyDJU2j8N?4FV!4u?72K68`)z~sn?5m2jD zhc25kPFaxwx#|5n>xhG!wEF$lk;#2pBVfmfzOoMM{Uazag4XTFxDifp?cvE0MAZV1 zMu%E$RI6)@+@kL$*Cw0QYEG@5ni7jgp!Us1UD4zu=91>-qCu{Cu(^nvqsDNEdW(X_ z-stq1Gsd7gXdLy-`HeA$L+Wsxbu=4UR?5z?Mk5Ao7uU?OkV{ufi?OA(%V=!%bs1Zb zI5#|xWUJtOUB2_dmWr+lbCj4eVDFwAU1>?QaHF#=#@0lvHO55}5zd$JaTSRQb3#BF-Oop^ zkZ4V`LO+E33oG~Cclq+@YAhZD>Zmzkm~FoxU0J)ZB3)a#u$)+4N#1wg=*sfS$}-%j zA-bAaT^?OtO~w+VF}Qd<5g#2-jE=|7jh{=U5~KMGzZhSvG;z;h026B)Fu&HCnLD zanoF~#gJ-EjHX(-)F|2+@g=6mecUqmM_WhCT&gHh}e37gy91JrmxZh*v%2 z@9pVz^-Lh;jRX+)29gtgBqm%vegV2LjB(l1+q>fGae4i{iXMeOVCi9tiY&>ZfF-hwVOQ9$R zAd@NPi!;T=<7_cp2FS%;eTH2rURfz-D=RbOnM&kd%v4tj`2s?wPorTqpy&mqOxN0) z237fdYin$#8aaKLA+veN>&tYRhfp#2_W3j!U}^Nei>#St1@tbI*JsA{Y=$4N=KC@k zJW-6nO^}1X>AO4OZmv6|7dA-Z!^ZTaMYRvKG7|z6i_|=XsL@;@}SaSu^ zK0t8-@SDRf7wc;Exw`g~U4EC^Wp3^51vXGxBXIt%F8|t^-)#1nTQVNj6Ujt8T{BZd zCDqkKE%7+AySf5hGprJ`#%sFk;tZ8HDjnG`ijs z?HlM{Rjd8}mGt^hW+^tcKD9Dlvkr-1QeH>19YXup+02R`{DD32x8KawmX?))eePDl zTFZ(WJhZH)R>8(<+OaZ_swldcUN3T_P`_tjr5#s7Bc3y!6*iSht;gf&+>)O4fs&Q= zXez!aq+`ni z8Prh{XkQ2E)M9k8XSv!hP+W;s3x{YXm5N4Fsp{&B)g%6De|0s+fPZ~G<^}_B5u3bS4(dOsq}CCIeHGv8n3xWF}T!jZ%w3kckOn&~DEJuGUnT!4D znpjPZe`6zqSvs29h;D4Gtt~DK$Bk4gD_CE()GSO)+2MlchL z#tn(-jZ7>79!4{%OP59=xl5^yOVLZIR4@o0CZm^dzC-{Y2M3o>ID_djb16KUN{p@r z!PIIrdbI~)fFNR51=_LI{aic2&rM%dBqBZsBAb+m569d%vB@{dn+~$?fep z+%~ZM@y)Ht_*QH?mf75l1-G^!$#`bV&_BAh9pB!}#IMHeS5_vMmNN0lczkjbtZq&U zf#BCbU=z{J$?$qe+8$b3iW-Iz?O>ct{LLWVygio500mi@b{w{w z{o9$-r)Os`ZDtI!QN!$5W^6XIIcqR%8hSFDmwI;BdS+)0J+qs$(ah{@W_EXWHj2dT z?rt;+aDV@HfB$U%?pXisZvX7q+}c|ITvzROt$o|R5uWW@-d>)a-Pj0kURn)rudeQ{ z&c(NPH{#o?yW!QUA)7zeOc-u4J0oH)WGdnuEDd4~6A`wrIr)O+;G#rjZYBonVN4M5+%e6VWy&Bow z+>L}I)0;h`qvNBayVJYj-Ou&2V z&Wc;OkS_7L!XEp;V(Znb6(zos65(=jZEbaRv1N5}V5`LE1CGDsYRTg2R$yzfWb5kH zt4lcdLfL(R*7H8!dH>>8pmniz)z`YV=0kdQ6(w4?HWz*I_^v<;ZCLaOWC!9xXn=3o z-1KZME^hh|z-=yGU94ED5VG4Bul6kou!_aSzM-MTmcG7;^^|8~ePZ7&rtwoR;_`3F zJHbuta%*c76Qj7|z1lmm)Vog~WhW+j`xbk>OTB)7U+)rrYis+5rNv(1zyI407Wd=r zb90NmaC4|~x3{;YCD6Ngzq$`sa^C&vbB^PB{Y!pM=K-i?U^mm-chS?E8c3)6)&|zV z&{D70%Pj)j=k>alT#GIc^Y-m06-x?*%iD(wlN!a)01oL)1_je+id%Y@A__(CnKNg= z9x_^7XU?pxDHOQIaJiORa7$ThQQ(5`{-vceOJ{1*OKWRQwM~nbHH8JgCX1rQ($rhK z*pwa{OOMqxA+L#RZb~;bH?6Iun--gz)BCY>%UC*ne^XN$cumbU&8g<*+GZfPthLlO zd6sI|a3&uZt65rVXxkCCNFa(UlEqPgBG9VUNThXTC9t*@!HAt*92*Uc%=XW& zjI4}|WL9yK%Ji@9yN%}N+2+w^oVD%eqpQsub^un###X0i&)}>QTeUAQdqzD2qZnIu z`&b6&G0%C=d3)F%8^tl^nT^FZX6>V+bK&sVSQu9xnM`9XUF3A)6q*CxK&%i87{N5w6S8hFV#*?Pv0MnPS3>`;^PZp z!+1O#UkQ)zoQ}tLGU0`tg`MzRIF2A5jfSJq;n}(9@cK@4C%PUU-r0%HW$=s6oenR| zF3d%*Nu^Qg+}iByPB=U}d%7pQ6F%M3a|vhQ>Fd*bXRj>0VZQD9`i-5PnVB1xZ|rRk z54R0p+rF|lGc&Wdvv=kCm7ObyUZ1(%HgkDF+gdo3Dm6Hs<`ZeF~6ZTswvv$Hp@Z_l1ZB?7X*;XUk&H)k(hyZpd5-0Xup zH)i*)3n;;DAZ=e6+rEd^)rDF9TC{C;JGgOe@PpSkX15a$EL=yS z2SIfn?QWZ$T?qDE+ZhJ_!h^wKfG=HZy9BW>wC#W?kiUkE*RUUG8y>!SEqMCc!lg@X zC>0&vzWKmHFp7E}m^*v+=63YjuTnS9-b-D-|2ncSB732&Z7}%8H$*q0*XHKWpTD_% z^X5YEY|rrA#@YGp?X|TB&z?WKwYG5t*H~K{*RM}ryRs1+>p^S-@o;$Z+Q#7E+S>KD z$+Lr3+9uyPeQkR14by@f6n<~Kc5QI_%9WYv>6z)7-x?q^1Mpap{8~eP()79&NgGxl86jWTJ+$v$Ec~6V zI==J-`CTnLVhQX&v+js>$p5W%M`4-ubqkNe0_%UU@+eeuZ|PB3mHY@nQdoQxN~r#~ z79T-i0{*wHJ_79iUoSs`m$m)%M`5}0Hx?lG+136hmLO4-%jyuw{Kgt27o^_B?F!p; z?p(xbC)Q!_twS=M6o;*XG|f{i#nUR5hZBidp5O^h4GqPl0it-Sl;=w+-dt)%oG&Hj z8j;d$HkI;RE3d+>G0$_gC@2&+n|T_AbO=$1=V|;{*l`UTa70Xi@GQ3F626qDNxqWj zc>WMc%h@toOzP^(Q06ebHi>FM7ptPAg<{3DL#P;B(X0yPy*i#JoiDM3qjj{Xwp`^< zQC=n6Ut4M_rztCCZKc^VmA8%!^QEeBnx+X~reX=3g<_?YiY;SR6mR1Rt(s7>G{q_@ zK13l=O4AgtWX%lCiQ+mFW3`luYG?;9qUcsU9MZC|p5UCkeugB`OZZ-Vdu-8 z`34A^=Hunynab`gk7T13C4CS}i#$6IRcR5Gf^B5-#O`cy@Mc;>v2mcxaph*zK#nry zw9_alE92WdUMPtQw)kjxu8Dyfg5T0oa=(k^c(w%^$*Y+2P@rrv?|fhAP@WhXU@9li zGOU)fWvql3H#vo!dey;N-U4;VHbF2D8UqnSpb#YY4JG*J+0chcnxeSRkrMJQ z=snMtk#D)sM6>EJE8)gj5ibHz#M$H&gra+1f>OHk2wQa`Jx`m=6xMwyZJZ42p-iGu z+JT-kkt{h+Q>ENb)#!8#COTCE`E?Poyfm9Hb4GQil^D@E<>U_8(#5!_(?CijuNC=(4XD7M^rO3rF%Y3R*l7d=gvlV>{eAjMifuA&PT=gX;V zOibnEF>=O<=_|XOE!Ru17+7`6nC1R_^VOeB8;C90|*NNq{nPqdPQzCWz40{!%TmFC_{YEURq7(bH|C9Va3k3 zq`rs#`5gBw*Z;_m$g6UU2QikF{u-joCtmSeNg-*KV+Rh4t!C_huMnxYonk|uLKDnpl1skuSswjL}3w6+9Y%tBGx>HJ0YO1Bk08`3|4G?)LmnVU4ir+%<+-31y zOi-)j+?7ujYI)3*l<7ifDD+M4Cr~62M=p_t&=kc^?nqdXIrSNaHKm?mSk>h3F3!OD0vgbibBH7&0_pY?aK2V=VE)L}ODig8LUrAxJn z3lhpaSx(&OufbT1;vd;vja4b+Qz#a7pvZc^;2OIk|~KgkWOh0TGow{~T^) zercmF5pHCDZK3|~iNhbMDSP94D#{*xcSYfE-aP!)Ta~06ueT2A_KE$6TQsynpp`cQDRl+*S zRSNGb_!{vV-YVITAjqT2KK)$`nqOMT=Lm!5H|*nIk{y0VR`$NHRh0ewtM^9EJyrj$ zW9Ioo`5!sp$$!f|3;8>e@qIh?bJ%JPqxI$W*S2N9-{*jlCON|7y_lmMPYybSoz^EG zd2OrprPp?VUxo>L3Kr%E<@RM9PHOv*vu}@n7CG;Mnftta4coL~_r92;biC~~)A)y9 zGonA6<7cricz?V0Y>u}h`kK}ImDe^_FM{S(mmV9+f4PHv-=cm37S$nGZ-;VN=7LYk zGI5_Y$zr?t{>JjJTa`eh!!9t&WKZm8Zb0qSa*EWP7dY+*q)wx700#1Y<1Ok@<0jrRo20x(6Y@;*k7850m@u$_ zUu23W!A%Ob-n~W#GdWoCJodr+sR0q+$$S4S{S1?K?h-K5_gGA~QigX=+WFGN4SH0X|9s2 zHFBJiJ9<=mP^T4X@n#_bljJ0UBl(w`~2_P~UqAvtRt+D`@mQ`ww)fQ-V2gOGiEk&~B zO0AeBg$&CQg+!|@ASOm!KmuYyhP_IiQhTUCrGZVRmeuJr<)E)23MB<5G^A9nQfaiL zp@1-YeZHJ5=OI@s;&O5&dD;qTNsd;k#%nocgw5Gx!JuBo(OONuvO}c>MVRhuVc}i| z>KcY42lG@&f$D)m4YD;VaE&&QJSF1VGV&NxrjcgLQB2Djtr`0lf+5#OXztgWnqM%%>%VP=( zDyvkgN&`Ipm6h124*DuRr!7!uRppfmT|uQtQ6SSRaF2}I$^|Rm0uic)5LGIfLa)#X z5gk$AX;P{5dKpmloQNyX%k+AMN(Q1Lm53FAS45~v5eo}95i9(mRH91ZxBnNh&QEA7 zv$JKDmDwU#hifjetgI44KwoLh7P0zD(XqRJh6~{Qxs4OgG z3pJV=VfVwaYIC(&56msCt{K*2XDn@leDjB_0Uarc&8# z8iZUeb~h+&svJ6MvsG6ER8W8nS*=QD1B%L43xAR{Hmd4s>uj*pLqQelC?I)^rj{ID zQPs*C8fvS#h5}82t+w{KtN`y(F&H$p1q#iH6S*fyWrqz!YHfxSHk-j<$jwECx#F9V z6ShW~-L4XyIAL#Wl-X+RC+s!$8iT#&1a9OS5hu!ahPu;KTU)QuoUj`-$d%a*HL?>` z2v1lI2CJ&3R%f?Erx4VFF%(kOsO*BCV9Nv?=!qqUCVQ>HZf~l!+wH|9N{-^%$l6Tq zb2K#>0IzAXH(P;fZ>k|#45~%G0d*(mPM53QdGcg^ z`^hSotI}0jUkPBn3*xQURMywG17B8OpRd}qc63-Pp{9lPE?1}9rE?b+x*FRH8w*`7 zx9fPJ+vRo>pW5xVb|6Z|$RRKGb#%HrUGC%co$mV14r^yU=#s-)x4XW}S?_MIcZv0d z$5F1)DRYq%N|)Q&-i~Dqlye^^tilGIFrc<8f=<;zkdt zqnxDFfFjO@!|p@HZg+8x676s3B2KE2lwfy>v%R6692jT=hTGlH)z2-75`Hg5-M}r4JGxVMv|}RU9a8cegnjdt|c4Mz>o=>RHs%h+RVN20Wdp zqUyz2IA=$xUXo7;TYt=%OD-XNH?T$kyNR!gx=H1yj{v3jhs(+xYcdOmmq)UxP=gPXOq$LttLoRuORl3)D z8f-SVjT|1(fsKalcA#{-bqx(Vx4Wmu-OXLsx^0MCdwQ(hq>?l+UYA&1*pTaXHMpz1 zq?vMy3m{mT3{ozYTP-~nZu!oSP{OU#;qP!G?DD#8h(p-AT;zCr+-{4tFV~xEDan<| zES6l0h0D$LT6(*V|=5!rOuC0dH5Xw*$B`Z%0p0gSS4nzPmt2UObI(9q-c_5-$$=gsZ2dOM)4`1jYJLW%x{`hIgmZLKWVe7vJ0*V|wn z*!N!VsZ$;O9X1`Xb8WdGj%ldDYddA@x7m=uq*ZGjunyE#_4nIOox+6G0SIt9kVPc# z_&8HL@a|6++G@FhQ~mv1jzWIwl(|1gHg2<3*<>8(aDzJgpwDL0*=%;3-QIKmo3IYt z+-qXJ~zgH z>mZPabbgU<@Z=!U0DC+`ZVxu=3=R4QFS|SK9=m7=IYXYIKzDbiy>qCi8>U0uLjus= zZ?&82Iy-xY>H?j1Stop51BzOPeci2Sp072xJGa|6G-7MbZEejR8L+oqHgocLDLBjD~VCxRKTml@}-NF&ByE_*F$hkrQjf4cu*W8e8 zzubsW-WKY{5l)DPLeg$R4503EA!Qp5S%^|Aln)8Hb@=*=fGIw#9TKV=9zI=HNAP-J zs*Y56h3e|SCH^DbAuu1x?G2F|cuPjHRKPeQKMme-MVU=?5>@`e`~!KNp>9>^2#Mto z29s2~1pO$!rWTsg**PLEq^iVOrtZ+`&d$-%khQMP+AaQ|ayXPb8WJlDM}XBm914+3 zCRBC$^x=@W8)6tH3TAYaJj#Ytp^M$a!=t0p0VY%>eT2#tyYsAd*1F*ladw^{CWC=6 zN|(VEG7$OqB1576Nax@XdMh#%=^lb{R$avBi$uD+5gQyd7!cM$MeFLu#;V5Z#>dC& zCIwd)Az2l^0dYJqL6(OW>xXxMyPy>#33WcK9(de1*Tr_$XXy<~_GeHBRF zcrkdmrK^! z5RF8lk)(Alk%(F>DQ?`5jzYduFxif#BSgtUc9G=FOvED2VWN>7seUdMNe5@Df-_T7 z(gG>beMEO#3E2i`aGo1tP{zZgr^JU0Gf{Lyv?bb*$+)?v4T%es z)=)4MB`Y~ZoDvfZnfE#y@AQTukvU5&8bJ&0i!MhlTOv`nWjPYF%tc}`Xy^QD#1f5N z9!XzbzP!AgT(&GPUq&Q4w;YWw$ClHpNW=CI)_)=E^77oWSYC+4oN`%YAyPl0mSs^A z_UUujZKYaXv#u{&*WE-dM@~XaQ&rOJLs`mYw|jXR8{uJ>LRpByySxmW=-6;(I8%Hj z6pAe;=Z3F@$iKYwc`_A=ge(zv$Q`r9hUc!VTgJfu$VhNKI2RnX%uQVhTEgL}@D)qg z3X8k&oMk-QmmO#N6E6)ab~_@Rhl_vAOlR5x3(?`xVEPx!JkVb%;A` z33ZMqhJ)*qV`F2(SBB@$CB_rsD~T(PbLT?EGuGL$naJeu%uM^(WMXoxb1Zaja?H9h zGt-k88=D>*vsuPwtUlY=w9h&-b5*c%9Gec0Veog_rY9#8ZueNv+CAMp zGcyJ=v8rizc*gDS?bY@Ujo_GYhuzxA$*^`NydfZ3JAIvF(}{#oYAliP`K;lvR@>R> z)&k<2#_4Rjvojp-3cI_IVfDEM`dR@?aLB=5Z~{6=?1xpjAk}IumV2qNKzw?0ur~~1 zAQuKlgJEmf7Y+jy1^|)4LF7(jKbp^aavEmVHlJaR^GgVMRB3 z+&--~5pdeVfw9PndnFJEKk}8AM(Dakcy#jg>2P>7oLHGWok*PaOuBtDD~Zz(|H|m; z(~t}-J7=^wllM-qtjyF|R}#G*-wI%Et=7E)&`hs(MZ1EYTS=_U1l-yH7;w)BazUej zF%BVxMid0K#DyF$Qfn_O|9#y$rD&+R_#wh0-N`{_&u7%MBjb$0@hE47)7?=<##!OdrE z*f(UG-C=;0-8*^JvU%fhZHJzTHk7nlQL!tTE z%=-HL?EIK}Y%Vc&b#{J!eyj_@{N@$p@4K-vz~&JOxyy;M`DM@Y*fI`d%lr9T3EPFO zgmrmqVQXw_Yhht41gwRvh4YF2^P+_wYvpN|ApPTxrOtcYxkfr98psE3m6Mq^C-5kn-FLS zZf8dzHNS%s&B8nm^ROBg+|tbcVRv^6ktO5+yq~uuuqE*ITKZQ90V;s^F@H%$tSc1g z9bORakQ}kBUObN+f%XjQS=u4Rl${jkfdjCBw~ur&jsZ3?97YD@JjKW{A1EwF;Ct=Z4sL| zkD3=}!)wIvAX#Dslh`CO(k)do3s$1cXQ?H!qF=h-diJaphVV;Eq__|qCGr+>6R-v# z;$nGhVc~wULbY@Tol!LgQW-)l`sw(F92nsf)0w2P!vbuNe+AFDF7%5%{X9onbcCZ8K;t! zPki;c08dkM-@4WLP+2FXbv&2l zQPQ$KZ6~TwuZJ)qU?F~(;=t?;SY9y%NhK?VxzBA%nr1rFS^%9IHk4$kB!f>?qB_Y_ zlmnk0z~I7*fY6*Ih8Kg{QOZFx9juTQu~6BBqNoob7zYGzINgUMls&ycL}io+)lm*n z5=1U&Sn4)K_Oxv@UR0>Yd*iCOfrR}(>q;6=(Z)iFCmh%SXP*PYikfbg0JzK}qey1o8Uv6+v zS}OI<4vDtoI}Bvl0rQvSI!UtVqu84n2im)*J;ZPw53mmO#vVm@y#pVOP*JMGLc={O z>K4s{ve33cg15Tj*(Z{>fCG_~QC#`}9%nM*#|iZeJxFc|P0@A=;yM5Uqge8`L>{ei z2vZd0ND8ly6t@iG>o*cXyh$lft*Izn5KC2z6oO5&M8$R{VJM>|G~+}h$vXDfjyCVR+vz3H!w#AuB5L=>txjGN`X-dKz9p&cP_xdxjsr># z9WA9`qDDy`)jMbx{p{m8DqB>gjZ)NS^R=;?+RhtHM_5I5(ro7*E73s9vnWkgCwk&J z+c>q6LzfLa#KvyGES5o&9BI~(X3$U4J*XY)L|mzr>_I|Fa^qG7M3dCwTqQ|U9XG%k z)d9Vvw9Y-M6Imiy4P$h%6s?uqz|a%=5=u%5_NB7z!GQNPTQ(T<>Uhfq_cR*q0;6uut0522ZH5#_kS#Z+2yTdSljl4=o^ zr-ZVwjvL<&-Ow)GXE~BqP=nq`X-cM|ay}0wJws*I z+mfc7ui}fH8`6i%Jj!==iWujgu&`d}RGFMN8At9d@iWw%>YZ~cOBDV0>~0QKCVB&f zdWq6PO}zhu&pnY-{j@~#BQJIPT||@Ye#kV=K7RhINkV0%9Vx6y=%2GCMeoEI#CXIc z47pN&@QIX&Nm4{ki9<@(p@#Uhcx`olZkb$FmLoz_AMJkN6J!s@{7C|h5S2NSCJOJH zPZNY_0A4WtBqng$1YNF0cw3lVVaO_B87F(?P$5fwS>KUlrB6}glvYl&!i%Bt63yE- z>dR#yf$BUW7Dce`)`_wd<4l{Nbe$|M`l@h(zzp1>q_V-YFw3!>c=O(dT%M6r*9dAn zn3Y9kLV_A%`0s%mG?o(PIz;m*8BOIpopt6m<$N4kpGA~(ZbsXQw%=f+`0t@p1#TtJ zmU+~y1a+aXkDARwBiP%NTo^xlCP5o+phb`}<=BH_3fbV3${eJq@fav1F|w1;6|^gs zWTrMV^vWucy7oFb^a1=!MW(T=?hl13Egnp+rxB$JGw&ovSvfisf_hl%6dKCS=Sn&# z;#4w}GTC{Pk*hG_F`;K6E}R@eQ`>=vQY)PIX)ScS;|Atnw)2Kos?v7m(cMY5TLT{$0W)Q8pHpECk!}CVcJ24G|4DRB)Ec+Y~5G5M}ZXwy$%~Y zd^<=GE^UKabSlZwGw3YoQDH*;l#-HVvFbFw^C+UyoCuv#_J#GT*IAwlHxj?LJbkwM z@W-ml?tHPb?2TWnyk~j(x-0qLYDM?omZ$%fS8x8`vjtA~|Agh~$KHTJ^DE0!VbJ_m z%hR`SRF%DWrD}iV{4Xp|3-j4uT5BsTA`IT}LP$7VT$slV_2GPtA^)-CxB<_~&)WZn z9J19S1VEAL1;odW6Y|(`s_ch1U&od>^4hnK9y@;c8aez%Qg-8JRoRzrfE|2??pHK^ z>nLu_cl?s0;sqIMFqVUe1 z7MHF&0LJZ`$bw@K-b{&7VtJV@|?J?XO_Njh49EHsCQe-G+; zF!gq!wxm!Gi^#w-D0~xAImC2$f$PEN;rglU$qA6oKK~{8S#ZTm7Jxki`{}p5PF)Vf z=1*hAB98B5;pO7cnWwT3Ka*Ydr3b6Ze*VC|&iu!H`dfAA@jnyguivCS`Cqc_KRWvI zbCATuJHGwugy{(N)^yb`U@`bdX}ee=Ia=N&mN4%#N+ps5g^!Ds^5TDx-X+4vtR&3E zA4!#xZ=F(#)yzw0r4pv)9I!Utd_=64{Bce<3jf`|`Vl5R9uiBaCvqf`Pgy6#N~X6S zxNm7dAh)9c{jCUIX#TP@3e=Ci?JQ8g1Jv^$KPOg73h8&F_G}T*p333QenjS^!f5#8xj?!k{}#$`$p0DTPl2F(PhLQXUQ#8i0z?iJW=8|O zv+_E$^F_K7(36J&_(%C53VxRv0o}fw3kVC|w~6q*2Y(+ZJ4b->nYIr>f~ztqP39%< zCHcrr(WA({i54=qyWX>kyw=A6{>y*e1@Whr-UUGSO&~w~0D>zI zB3OG9f_FR&q)&eQ-C~o3pZYk;YF_;@!Uv@c>yKyL1YOm$|bNcmr#F)F8@V=lx0p9p@gOcK{kv0 zqsV&{c}7mEX71+!R+S;Z8+Rqls2;&5k0FrR5F9|SQ!fCw`Z(@vZ=C?>D@_Q#*eqbQ zA-K~C&JwNP#O=!4f%i!l@Ju~``6~Vao%dAN_rwypsl~)_^^C6-1%BRz%xB7-P{O28 z+4B_${<;!@`lAT?K8Eb3Cr?5!eY-Pg(&k$LZ2^?I_Hks{3eSM-`KMOU=Gph(h4y_2 zw)T~OjVt~=#`{@dyzrzPdUh63<{h5{`_Dr=KlCMsm}Od^RIq%PzzSYMU4Y+%e>40a zgLCCKr1ew+385Lx^SJU}N8*0?_X05Tf|R8$A@LsgkHY^g{I}tMhD`dKD4K%=-UW|W z$V>J=d{&ENBYZpjF#H?g-wOXN6e$0ebR5?05}4pis4e(Egnt|!qNe^C{w|OTzb);+ zi|Hk>$Nv?V)KkdF{*KfHL-u>W8~!8k|AGwikI48i65m9k==+ev5E36mBKrqOR3mX2iBJC!%JWx0 zf{-u02&wJ@{2aiIKL$=%K>jHd;qCu~{PaH~82T3k-}o7Vm45~LbznU5bKLR$10uIx zLF7e5e)uZNO#A{!(!1zLTI!rM;X?xwX6pblzMF%L9VLQ)EI=SCMDVJhxPK%l?mYv&$ z0=^81F=qvdyg1>;WRqRyf$|@3!;DS+K9u|D{w#eCLsGI*5{2$RhEdKaIM5S1_!el= zJbLS=RRC)Y2-?OFoSZ>{$--H&m+@lw3B4xylIASNrA-hmhe<>7H}VyvpBE&*Axsp{ z=n#x!dXPNrT@&k>vlEE!eKajrQa?d8AN?2_Qi)+I=mQhJjrKw1nLNmjdJ9p*^+2%NPDZ6(44{H8;Hfk~+R@ORM#>(DsKTb17j3&)WsZ(7e}zp%%A zcMIUJ#6E?I`meTi5~60>Zvb+zSm=fM5-|6Bp#8U?{ZSUQ^94n<3B9n6UJ!;N^M!u2 z`lTU&j`~q@uRbOom-yVmd1SB)0L`=O;PZRo4fObA(+z+6<-I{zXH|DJy6xZ4ItGm1jV<3?N>e}c1V`L^(oB9KRQ69oP_y=4ER3){^yFJ za=-VPJj|@`{mwB|H9Y-J5Pet}iVq2jH~E1-6|05$Px21WpMjx2*cPVs-5lcq5@i+@ zKuZV`dUrt(Xpf=Im>L;YgR#RSVTQNDC*cKw2ovf%k4o#A%TP4xGe~{41Ps3oYD@hk zQXwAtvWdtCi~xVp4)UYWZt7WpKI#DIh8G}Z5&lH?U@MC_IHtE12Z%=JvZ2H8#B$PvxpM8h=B z0`uMH5j?&G3SWi<9^Cm;v3|dS=3VfKsggN%9g#mk%Y^aUdlOy#?T0|(r(gT1xP$pV zG?`I-6X^^0e+)|d1$32UM-TmDdWCae`WOgf2A0G{Ul+6Tj~~#;oF+-dD|6_FgFoIy z_}7a2(Hqwf0@;C)%BZ12*n}ma{#4N9$ThJ^K1;jg#LJ9Jp`1T=R^$++ipU72M-djq~PTw z>i_vB)KC7x{iXktvG{0Pj%w7P?T&+DMCp_L`y_b(U2{plu<%5LP^PrQg$jM zGTJF4WMqeIZhO0Ld)?gk_j$cPpU?O4`{Vn^&%^U|u66R>pZwmR{6Z+kr!61K2MQAf7`HLz1EmA^RZ6H(3_6X7xlAB2-3<{+YtTaqHnaW+v;TB^ zuO_&)WXUO;p@QW!TSSC2;sfQ7OGIDox|R5S$VBL8%C1^j{l_glbwp6)|)r!e8{5L==lM zq~0ic%%u+4V}*XbH30EzAmY(TM8GH#-AP)~coH3gVRHyY)D^?^{eFazxU19ul2W9` zXw4XT>U8XVN)a>5h>EI_iikD4oE8?-M(q*O`Hn${IYza}qR3JjT*mRyF)a&u$SEQ+ zi-|c^i8dO;)REW}rws&{VazS*N@7TNP{<^Oor2QhuT;9EX-z(TT(i#&R6q?u!v+#G z^XGkrsvf;p8Cd@T8qd-EEuqwOf73zN5?OcNa9!iYW93f41X;L-)xP}uC`K2EwCkF5`0LdyDw4C z#ZBcN$BbrXRKksqlY`QX(qGA)(TZ@p#34X+Zv&l$w zRM#B*WbQ$%zGaP_*iMOYqb0`8raZ(;F%{VFCB#5Nd^$*op|prR{~0%)Ymj5+BNqQc z5AfqRx|%75uT2RR$^cgBnAjHB?8HRXA3t+3QR%nE))QM~J8a$Blir>xM4nI(n(s(? zcFAZe$`W4W(7!38Z_(Ddh7|ws7+5X#~lB_mCj14>70m*O?UEw$;|{ zTzr(Rk`3WPqss}mD~_bsE+X*c7nQVnEKQwV`-xtWqSwSk%~>{RNPt~*OnCvDTe-Rg zBAonXKXVF8HOotG(XWj)4K)WST}^hA=5GtU`T3f{lTNnQ^`0^~!tc{L4nL`9sOcj%R0|MR#fvL6y=frL5ldC3j0igZU$5sl_CWIi!zlaYTj7OLbZDK#~~D0Z3*on zmi*E=V7RL4wRICMAJrbn9S4@$;+CX1Kx>FSo!rt;%UQME2*hCuh{bb`5y`V+P|fAe zO({S&u6~Se`L(#^rLDM~DxCJRq>-Pt!Rdi1Vjhj83fjjEcXjGS_TtWE2j)|0smQG> z?kl@t6J7b8EjEkp1ZeJvli15KL(37p$Z#GOp@H3dV+qHy4$A^3vA*j=gbuDmK6J#cG2rHBUmYd-0$Sb#Wti^Pbp1jm-KKoJ^UV+nQF>_)*lB5|?|Lj(79JC7fOLRQ<65DfJb7>7W6kp&_9>%WkmLFF8l@=tpJt z^BbU>wQHL_Cv$#MiqCnPzzG(yE;)Gj|OhY7V1lp#u{6Y_wRBr zrMu*@ou!(!qMbbw%bV>pI`XbcQ?U#%6UzY62ALNaHGP~SDbf-SUKGm433)Adwztyo&cHcKGZ zgMO=!Mq@b0kS*vT{5D>-%9Q!(g0X68FpEMg4Z)Iv52{k6A^nY`=LCkr3@oDicArRy z|7qLxI0`*EZ4x1v6ogU{=MJjnuv;jO3$d&vMEx+LeZ7gJ+R9lF6Yx`C2y$Z3$$)m4vbZhy=S*>zD*D!X_K zp628G;yE;vBU2@LSs>0?YjOVfkI6*;=OHD~e}pi7Ob*n7*xATkzEC;7-*L8}h^nQ{ zFAgwd(_h3qRWDR?th}%#zAmj0R;>xb0F+9IY1D?w78~PN>~G}HLi@H5I_PXC#GKBA z+=}NCRF7yh`L;N6JJuJ!5zhEs!Reeh(U2MZvaJ{mW5wdEQ4r$kYC?P?M5WmIS+nX@ zExu-y(s_q32-Ey7hc6^pZ~g3Db%0kbBs&cg>%c7Ohllr2oF|rVy!86 z@TEA87%puf<-VfU-k3(PtvMj3J?DS1$4V?QWM4Jx2U)Jm=X4Q~bOFV^%``IDBkuIO zxYNsMTiIj|V#sCE7h2Ms6|}NrO{Au?te!>`-%U|8NkFq$n#9{(?0#ZStfOJJ-pBKp zX&R;MBvR=kD$U1=RBn^Xgq{>m4OOk2sm;*rA=lo-cld3DyIAa+{e`F#4fMl2lVxjt zA}b$CbIECG194*AvMm90M3Q#sYE|D}FLvhLhT;^D8k0SiAvV_{rZb$y_D`wG+RJ8r zyY~SSxH=3UTYr4SJkfYGac7>zY4HVw%@?v8u3g3cduJP#V9MU4{#mX~Ak!7t>z~HnVYUwC?A1!2YMJNDX}f&Z z55Lo*5jvkijHNae{zRCjg)+vh1^!b+lO1uz_3p^N+|A}O7{4WXP&I)O;z5W+$(7n@ zKO8oB7vM04fulI~DWyo)1`&CW7h=1}MnhyXN3>%@F{CnvBSNMl98rSG?ypBUC@^cz zGEU}?;QeQJGB=2JzB7q1gGGf+k3cLGQ5FzIekE?%Glt0TwN6k^+qJQ~($K2Y_hfO> za6_zFs)V{2E_z_D7=MjB5CW|)4us`b7HFHV{YgW2)DJ#%;}B?MRN6hUP>8ECIHM_M-l1<;?m-`7n>xR1cDUGlPQH#ptbv##4=4bb(yhC$veQ*HR zCELAmeV8XsRJLTrBR4igv&d#*u(Vt&Hf);PF=o~~&Vu^&Ydu@FHOiZ-jR`XEEAIZ^ zJ*f+gALEEQF^tuN{8o&L7h*sh6?M^m|1OqS?-TB_DCk+;pu1X-ZpycCJAPiM`Rjx| z`z23}lf-hbM}!hB5w#)Wswj2~(d5pi2xEGcoZFS#d>_`N2ZuRMONGN~(jVgF#%)%( zW1M+RoP|_X;bn3hF@+U|bb&U`6m5Kz;>s?66V73@r>3)N(_=CQ4Qpx}$=t}YLHTpq zJ#Nw>QSPahDX$gn6DG2_e=&?YYM8i(avHCs5Xqyx9XXz95{~Dd@j+y|0)3!5O`ao; zyl3GdZ>GnK)rGuoDqZa*hIZmCdkdXjD!}H=S+?qcg=b)mR1c(_oPL$zw4ofQeK=h`gwqAlz1AE>XdXjkqHbiPON6+?K>L$W$yXR? zwsp7!Q;dhroHY;1WzDfZz;l$Cu+v5Rv^GO%nIi%irm|^o3imeHMI7@n2vuh+W+S=P zjl{aZ(T6-=jL71N2uTbbGf{`abKe|1QxMdwN|&?G#By5ih0zP)1Pkl zDrE>w(s~VEBh|K3In&e$I9XmgdCFJ5$&?w62= z^(sYI+rnE&)TPak=ruc5ipcOqB5u1j3^i}Jjd`R@d zXr+IaxwV^S6$%w`gr`bxTohwZ;MXw)QXV%?H^u=LB*hlA6%05h_Ti3pC z;KFu0M6)4jI<+D51r~T-J6}QOI{*B_H5`qttWy6?#ACXodyQOb^|+^&4X|}3-|pZF z|J#pqnyA_*i^{QT$dg1WM|Hf_sS$O;B3beSj=WXNQ&C}T9c3eZ&ShF)ae^3lhohPS zI|f56yyF|^|hMRoGSRzdbKrarWrabww{$NF-M zs%M1%wB-)OA*nLA?>}a1}kqoZCR8W%-DujC^I`oPQZ^8CS4d z-5|nQ_Eb&wO0FE5cipH?OWm1hfLwUA2^DuvjP@!@|`T=%x3D8yYD0q_MLLye2S=7 zgpG}&+`kyx>!$>$dxp6Gj;MJ}ZnHY9+aW)KmutDLad#KSU4XMkcNKkOB~CBI*lm%cnI&h4QTI#CR}C!@!@43YD4*PL zE#lQ)L<2^a+)pI#fLc{HPtRq+`c{;}U;&9zaOLyu9Vo~1cW~=0a?`wrxcLV{Y?76V z%X>MC**xMPV}zBfVtR@3BhR>u{{Y6Sl2%#9jXw%IOq<>~bc8B0t;DavVZ>K4F~>1f zl~J};oLz~|YAm+DH$-KadL#4(BLe&oU*`~R{`|}A7<(}ku*d-tALxT~wMj`G#z$ae!0Na`#!|8b| zW*H5kC;Ke%CPT{!%ph_T)&jc6pU6IHZFS?ESNYN-14=l6lV?&BDkVCQM;&sj<0R+( z8=?OX?wmd-kJZIJHxea!E~@?MLsDux&3+iqU}uOP(w`op5{!OW-ebozKC`^?b&>JpO_SMI|8R?ld z*RkiL@_v%w0A^=d_dLmq#hms1?+CBCn zDcgkE4Xi? zDb%#k9k;oEkXn-1b14PcoP#Jq72-{OCNc!ag%Bu0scMQr>B;>@iLB3JSiG|p2bfU$@BS)parXO!soX34c zJ=|OB_a!wrt-3^nfJyQ6y)`LkNa4c@Ts2k^a0S(UhrsVvQ{7(~zVd#$p|a=K z6|&PVo7cz1Gw1X(;qkM=xbTLLZ z+XX1sdfoAIa>riUMV!e89n~M#ZiYOFdJ^jnB_p;Ks=(fTHF=Z^1>q-Vp`ZH@NB0x! zqx~9se_Wi)c+R6=R%H;(;|=bgc4X24a(DQJ(1(@}6H}F!s~KMmk%tW;?(U(af0%pH zMkA=6>#%@PwUx0y5ADPPZb1M(tj%TL;k4dyp&BzsBU+6|C@5sr({I+O^85+3*!D>{ zzX(T+j6jsnK%9z13}xk`^q*|5c}4EVYDtc}iaqg_1@&O`5#!9i?5V{W?*~%!^ba^} zb{6NDVj@-UJ`u0ZoO{XG*WIBmO>Ac^Tgl|ym0X)@QY@!>*d|bPw3}+7+p9jsr*Er1 zbW5*REsxp8iaA9#S6CX4!y;m!D)^AEqI!3aPNRB;c#`Ws1mvrmKB{;2>Tq}k#8xer z2Wqoi9*u>zQ7xMjbjXi2q3SI-JzaWK4=gmj6c0;5^s1?PM5DS@Zw~HiKq%JK;t84R ziRvRlENn=>yNClL?*E?icWH#9KMrD0F4QO5Y7E8(G)!My#a=<&qPUA<5JND*m|HxC zpzn!w}Y@r{qpUtXPkjlYw|4dahj_ zF^>}K$R09SycegP`8Yi$vClZ~*hgai^lW)DT1$2+Kv)za97+&w6^M?{2ruqg>~?&f z^Vv=-9kq#P<*%lAe%y@k*-GEJ)ku>AoyB{N=~v-oA-geXl-CoRvt=$4%r^SWuS?qM7!7`d{1(_LLNcnU8LZQeSKFfX?erYZ{&my0+l z8t%H#c6;d)szFv>M0I|m!Z?QG6s-=wJvS&wy+=zmL*=1=5o{H{Vd8;~m1v1e!?LLH zZ=0wPWm3;~7PC# z*NK-;c8G@i@FkHJ{V72InD!9rOxMxXb|z{+%Z=Ws#JP%9l(ydv zq`L!cA`f*Wk!shw-0>7O+h`hMc0WgaOAkmkmhh$(-Pnox4{|DtTo-V^?Mt064zMv|jYj;hT`YhImK zz2zXDxh>@7TCwd?O@U#23Rqs=fs`Z*BB-dNzbOS3aLYE4QZf7jE7%IC^^zg2fVXZY~5dFO~ zdPJ6OaoS>s7}yi{Yq;0z!+;QTK`2at`G8ti9jtM@AZzyJEetChoM9-Gz&&`6XH#eH zVj>oyQH3rNGl}s0fTy7go|ag|S&|4811WGDPu`N~Av?NJiX>dPX?d(W71xU%AReiz zURe7V*TFtakz&@p1=NXb=8Jr=5XEy{s`q?8&&N@cik6XJYa z94Cph8&z%845w(~2IOE8X^LxK5O?SYf8n7%f=Sm1i}2irgW5EE{d6&r8Lv$?G3G(})lyw#{*c(n` zd2J&4pdB7HLKkggm7;nDxs?dEmOY4acHea7tUJsg^4kN5j^fn9g1x$&egd(JN>HLs z&O$>ZRAWaMvyB#0VLvhVx!2)cbYdPk6%tZT72dy?8kJ8c7IU1UNgPG*1k;7ABT~8_ zmxOWqB0_Cki)grx0-4f;ODSST8h86v#ISZt;pt8U?uT$M7el9sC&8zoMpaLs(}qy~ zWrj*QNCoMofRkkQVGAoGSHdV0R4CDuvqW3 zsv?dJDppzDDH@I4up80Ez!GDo54_+e3~gDjw!@&?o!-|3S6k>0BVZZ`HJJyZ;al5r zq&*zVB@peZWzUea#u5l|uoK5Vj-)vkLlg1V)Sy0OTZb)EoNL%QV`IR|RvzRhG+PZ8 zxj$9uG=TDM5%qab)L@4w#PykoF``|5vH_DHi9z@&5m9d$VpS417u|7|S(2ROj{O1y zT&XRb!yMnsKaMI$3cw+wCWXqK?pcU~tr$?Z#9Sx&AtL<|NwJ9I^AJ_@5tHK)v&loX zdl&q2>nBhlTV_{fAGsEsWk_U7O2h8onzf_A@6&S zB`xLEw+`IEEDZ~So5?ur+Gxvi5~ZuV#?>*7SG8pBUg_=Wj4Kno(*JK)Of;af($0;j zZm!(B66t@LKC(h>0)LB4j_FLcAzct7x>G*Uz)wIl@1Q(^LNR!+*~rlxG{uT$^J z7Qq+R4}?t~5|>+noM{HjaV zo{2$pHKEttqEWA@SQr+nIcD%2IwV?7wMP_+o!MOO7(<6#frTkWrDuB2CDbW=q*xLY zJzUI%2J=YF3`;bKmBDWu8_g%~Mhx}~2zm@xG3ifVNKkR}!Ev}!b8?p@)WRLnKMh6S z5vRb(^c{Ib3QkEvg^f9gC@Dwi2tD5H6v9?$gMjadK0+m2Z7|1dx184y`{h;$@qkh8 z*8>qJx>9vtM34u`uO;~@{mGhpmt6*sPi6vso+5u+F+P@(kMgQs3@;kk^~b(n^z>_) zL=~fcZWg^d=HK~D0Xef5Qsr*lRtCy3VLpBKw{|NCx8^Awiwqy2gNMvA1Ou5!{N$Bd=` z1L$AM^HyzXjdNM;i0E3_gFq-8W!oC@0>#nrQDo3Pg0LO(5aDz~#UXS)6*v5Vcm(uN z^vm-Fh<;*zZN<2gbHs%CUd*Sd&j`L%|2ea969bIbAHQK4AF?ZT1wSP&yhMoZj}UEs z;n(NUHhzOLX+f`*yvFXmM%4G9(2JA?JtBT1DK`)=KAQZ;qWVozCg$hx1JCYLSRVdw z)?teHzegdu;j9Y9w8TO~){YuP_imE>jIdBdhzy~~?3%I4pSW2*{6%(==!_87Mb)Z+t zc4z|ea9C_=l`9P|_q!rjHp1yEp(6ANb=HtjKg4o#ElNiDGW{Bh3rm+ey2cK=xDw)a zlWvr*b03FCx&+hGBf1+YhLc1KaWoc&MQmXfKrT==xpPBWjr&)<-#U-0D0nHTCj3Se zx)T&ftNl178d6ddBW!dTxnpZ=%`FKqiDk4L z-32jIXrUH%grM5wEhtC%#B!Wk?nRtFfXHXUmiOM{mhNgl;8!*}^k+$zJegKhs~T1` zDqXHEp0NM7CH`;UdW5G~@+@q^9;%EwQ?1^c8POd?bBLy}5^u)GVp$2>acFAH_C>83 zVJ)!ncVzzagUo|i%v%ul2H}2VF>S>SFK$EeB(aCx$DOH`OzqeksNU63bv+iMjpX&| z_=#ycZ9TT38?n*&a>q@iEf#fs!575Vk9km*ws0?M82V=uJ~)T)hw)+oC%fm8tFi-^ zmbh5s@_H{W2Kl%!4a%eU5o4WD4A#dGu3|kI%G9NL7k@PYPMjwAsI#~*xK*#|9<9OU z#ybL5F!?K~;eYg7q+0xxiYFn-g}60u#xEYk5vkml`jEXp5ZK`(9)s21upaKIj&WrE zd(`nXF)A)GDwJveenlTomA;`8tB>lfdXyzQQY8Kcn`+xG?`M}KH)LZV-prQY^Xx+T z{y-cDN@DqYh0j?xQU*70Kp%x}bVr`6i=Z=i`HKj&o!o_AKr6McSJGHb*Whh3ehc19 zLWtXb^)U+h*CjHZ^7+kbwQ{AW&091Y%L6&IC((+Ad}p>g&01`Ov*j8NjB;Pul7uo@ zJjYM1X*yCTZrSSjj{ZC zEykY-kEvI8)RSt#{+Y-N`2C)0~=~HW8zPDgFKDVhugvB@wkJucQhy z+oa*#W$9)rK=gObL^8Ib^g{j6?`4f;RN)wAB=J*Vc`9!#$wSv+WBAKE_EKAgEw*#w zx3W8BQ$1yE^bd)bx6WoVvY9U)8rIXvCvnt-+^ZqslXvZ-?6bZQej!2SoPPrwj3Jb&>dshZxPw7XDFfI853y>fQp=`C`fYbeW2Swy zMeO1~Z6!=^wP}RukdCjM$!P15QIwq~KHAZsopCx>K6Udl(C}m6|L0RT#>)RZG5z=B zCPtFQ|HD<1q|co`e=^j-!1sD|SbX-rh&ue1Jg7?cekc!-G=1A@4t3V_AFCO- zNOO3W=7=iIQ4cl8NcG0Gtry~4Z{pZ`lNZ$s%c>V%Rd4FUdeixKOxyZ1o$JpUTYt`? z`g61D&#S7x;9>nZNm<-hiFa0(j#ZW|Qj)WjlqzNALuIw3mD*NoowHW@SgnjjS{t&o zHXhX4RHe1$q1HA@dwV18Y-{aYAMKqpw0E!3-g8iU??LVSE86=WY9Dy7eXx;EL0g?7 zYn@^roszLSWs7tUW$9E_=^VbIbM(E=F}~*4R`;Z{?y0f5XJ&~1XV>V~Wa*wisC&Lj z_rew3iw|`#zt_Di>D9K?yXLG{H&*Y)482=x^zIzgyLUzJ!F#=jjT$_*Zt&En!Sk^V zUMy;~uGv9Sa)wu5H}sLt|(CCayM3+yvliD=4py~YDrVBqbU8HZk#Kt(m*EliK zI4RY5d4chYTH{q8jMwNlTWixS&9~Y5$Yzl&7{hw7o?|L5st+EslO@QKjGVM7x$J zU0R+R*YeEbmNna3oM+WK`t>$kP7|NPMUy?&dIHf=upw)q;_=6h8fsOV`6IC&9u9psr`6Uhb5*xvrQcfO?#a%?S0kM z>9J{_kEVPn)U{iCH~zb}clT@W5yf}<+7B#j?{&5P;E(Nn49xsY%!ay}`TLpqk2eck zVm3V6Y{Ut(QIE~WsGE;7F%NMypE%xp@)Gl~Z1eCF=2IV=Pgm~{Y0_b)Ylm5W9cD*$ zh*{eqwy?wes~r}8?6AndVu^`GysO1hKa0dDi=-tM$=MbuCoEPzwpguhnQCIW&ebw~ zyk$m|W#(GTjfIw*uUc;XXqjcuF}qvG9M_IJ#&_Jcq+?!o$Gs;y?t9$vfO@9_lTJmh zol3@cDvjz?zP3|EVW-1aJ01Oq(y%&iVs)~cRkf?tsqt24mRQwfTb(~)b@8#)W%bTi z3_4%!*7>?$=eqHoZ!YP4JG=AU6P@or?)-;(mq#XDp15{-HonV?C0$-+cX@N7%e%*2 z{!;Jy-k|HpZe2h7b^RLE_50ecKMTA5y4v-hk6oq4*6Oy_vYWMLh;{vVYpon>Mz-!X z>jsQ74I6z^o5pT7h9Ndh<87Md*t9rl)9Q&$>rXaq8+S9Y?bd#1H}jd@EY@}FSk%qx zTDLBrx>;-3+L+qfy4l)=*gC}9_RO*Eb<)=9iEUqv?k=X?-Q2qO3+diJzWac6-92-< z4=U>Jb*=m0Pu+bQ+xhX=mE7$3n+t(6?Sj_X1?SiW7uk(GX*cSL-RMttV;kF#x3!-z z)P7Qk{p5K2upImFllD`u*-!gqKclfjl&wSbP>0zw9b(ow#1=Wszvi&;lf$CMJ(k$^ zNEq59ab}NY@ja4rdZe7}vGPfe)t`E-Y20(IY0q_TJ<~&aX2kd0kkd16)5)G&p7h+N z;kez@F~`kuM~LIjnU1^HIqoTP%)jQi|C8gv#=Qz{dle7uRXVd*d3>)!IlU@R_B!&U z*D;OW$4z^mbnAU8r1zQl-ZeSB&!6ml@k#H?8cwyQPS@O=>O!1u#yj24ak_ib>HZU^ zKQ#J0GVSxkt&|7tj^wRhHVch;QXTtC5CE7w`4+F9?Zb3@rhzr9OicNfD6E=?0$jMH427rV5) z?$Y|ROWP){Cf!}z4|6q-cC|=z?O5z;b=|egXIJYcZr!@Obsy$tAMMs7&CRjct@m}e zKA+v3o4C7ncXuD=?h)-iAkBSXvAfrG_ragteVX+1ZQpOGdq4jP{Q?vE4bSa2qPpLx zr~Ssr9^=}3gt&W5oZvAz!6Pi!BfQ#U>Qj&Da{tKo{b#!OpEaTXoP_>!bNkP$?!Vw^ z|2TQT;`RgL-3Kh4Fko52faKf(Db)j3J{_=H_DpT>xz61)eS&93g6D=@&rQ{yTb_Du zlLu~ZKQPCA;Eo9ccO?wW%N@A4df>jN0}sf93fd1UavxMOVNhAZphLNXDys(_c{=Er z>~*}o*GYG;Qxm+-BzV>2dY!NKy7<)Vvg}>k-us%ncijZUKV|zfZkcyrkx%UE-r#=hNVePa{KL13TX){=P=De2vq6o0s^utn+RC#kZ}Y zpNXAcdw)OkS$-DjejQ8vtm^!_eDSk39NNupXm|gi_OpieNFVB0GPHNyP^V`@`)Uqz zF&pOAZ&<&H!}>2B=DB0opi{%VpA8$L>F;aiKeV5}|3v@5rT)Wr_>VZ{KkAwP7|noj zW&t7n0wzujn7lL~Y)3%&seq}^0;X#QMw&5*0;Bx{XU__ZNe_%I37lUSxbREhBEz61 zc0mdLL5Z`1lG20XmX`#rs0&*4C1{P|@U?cs)BK08pEW!)efY+b;hXD*Z~Zbn%P=_G zE;!dec;~F(-RZ%5N`mw2g7<$3K4>_i&~8Mr|A-R*5v8+6l&6oVC>e3MZp6_qBdQEX zp0FEP?LYGLtdVEaN1iJgd7*CPr7t6|7>>GXH|o0os2j6J-AW&Ir)1QUr~ad#&l>$Qee~;+(QoTU|M_L~d&4mw?Z$leAM&_8j4=gtX(PEYWDK4Hk$2|o2E`WQ|0v!6K3W1|10i2;))1}07% zzH{P;(upH)OdS1n;#i|e^(hY-}5O4>V+4W zhZlK-mrM#TOAJ4>GraP2_>t$~$Ld8KH;*{!5pil##F@m1nw=5nPe)vQ9&x$e)LNsd zwdPZ=c}%UFH1%fU)Z06!-aS3_{`091zD|8;H0`ndw5I{np3k24a{aW|rPJQtnD*z_ zY444uf3%FS`mY<)|M@yyYC1#RVFnL!>dl#RZW;chK-2-RZ&zadHW2R%-%-%O=_W3r`xoNbkL$rHfw8xz20U6N)%c8w*Mi2fL z?bCFYpTn$SfwKbU%nHhw6HveYq z{%^4do6akAm{%M)uXN75@{D;EW%CZ-oOkrwysD=2PdLo44xE2_&iu0(^Usydzi@N@ z#TWB0*I!WEVZpWj3+g5>xVdb>?OhA*mMyqi<<;3Hi}tnoV2)E=Hlk%i(B4W-1_@sK0IvFV@dmXPLC2^cZV#t8RiJ^&; zlM=&rCx)L*occ0xy0R?NVp&v=Wzj*)X2&dx$y^p&zHI)jWeZ*|i&K&oTO`E~NJqP2Tb{d7H9) zyT$UH0n2xUF5eZie0S#ZJ>|>uZ!O>defdEprNAPkXh2FyXi8a3N_l2VMS05MvnfYj zrW{jN9Jg3;a=?n}pcSWMR-8>*QL}r+`Lio7zFcuxSy^kb^4frvb)hS7Cat`^d*$7; zEAPKt`G>OVq4BE6JytypTJ=0;)yvFPugh1xy|wDk@2lP$um0F$_2;0~Ut?B(OIrP7 z_v*jTuKxXU^}os*HOn;`o@+G2*3?g4qm{Qtr)G`bt2GU^QuQrU8+)c2hNU)5PHmQ# z+M*`4)vMGtd>h+xt*Pf)v#_-tlGj@1t?g8^w)3mCUA5NPSgy16TxS=y&LMeS&%AZL zYSuZuTGv-A&BZd!%`+{oUsziIkGUg;_%+1S) zJ(My3cE-XV8H<`_F6o(>Fg!DHZf4Sk%;kqNSKQ8A^&@jlvkhx|Zb%!xVg1|(eAp|$mq<<=*jTb~Wz`Ydegi@950ZrJ+z(AKxNxBmHK>wB$j zA1t?h^4#`$__nWew|(ER?dPFwziPJqy1nh6SKI#8%2I2drQR`1!!b(^&Z-xirEJX7 zuE^58lhxpLRzvOW`W?47cHG`1c)L;TcH@oPn^$aad2V~FJK}%q*W24@XSeH^ZQ_`1 zIxyR8N_L0XY>VaDmV2^0oy)emlilSfTk9N~jybjibL^(%I4sZUu`$Q7BB%GAoIXEu zoSWymI_A0u=X%8E4%nDGup)QRxm@qpxr2Y^`ZV9+=eT27@Q#4k9YGs+1Xt`Bd1uGy zpF74j-#OlK=Y-&$lVW#-m^R9-0qdHcdypYOYNAqZeU*el)Q}Pc^mfRZ912? z`A*)}pLto$_hdWn$qnAKGj`AJjeGV~?8(2gXaCPV2b=FLblh7Uytg!VZ~4Z(6%~6A z-`RWg=iaL3`6nFntAq1T$L62in18M!|H7U8OF#3kG~ajCao_deeK%tF-P*YCPQ|`^ zclJH_x$h6{{f|2Cf9$yb$-w>3rtE(nyZ^=V{jWCee_gTv?VbI9{@nk*`GJp)2R;WM z_!@iQ`^E!5D-Qg+bKsw!2c#AU)q5S3`FTkA!TKo&wK%4zJg9s3V1vI8#x-hDV9=|e z$%q1@c?HIs3Yu3Iw7gr;`tO3aEecI~6}BHyXg;sdVpCzq%0jEVga%ih67+a@wJ;Zo1(rt#V(zS-3Ar+3oq`UQtY|6c+mM`?>EImbV__Xl?)wJ z;vZfTm{Kx)Z^?-BC8OSyjL|6_*QqpQQ0c_*(#a{MVS7u%&zDYpQ#xIzEV4yeRIjq= z5oNRImCcz~7PF~r?%uN4%Ch8I^~NymB$Y%PZ&|2IIlcuQ~C1B@|5%CE8mo_ z);W~g>Cn1Chtk6jWuzS1u=miW^M|&)IkZiuVtc2GoIw>k!Yg*AROIcg*jrhVf45@) z-xUX2R2KHCEFMu=I9-PNB^#@YH|2Pufx?N4xgTP`0S>`=PD0hxO@20 z--oZXI8xi`$hAR7>cWrQOgVCU?~%LbkKBKA}*te8pKlUE``~0!rZ;t(|Q>A8ArQubj8BtY#MU_^5mCl7K zy|+~jb&u;?9dGP)+%V#J(-p^?>q z)x8s4ex0yxd9qvYlif$2w2wGxKmTNp%_kiXpX`0_WS?Irom*DB_O5pGs_qw2-G4>3 zXMXjd!_{8*st5n7_Gx*_ulK28BTog)KNYn3RPf-Ls3W&c=J4 zT^ezA*^0Bt`DaruoL%|$>}uVb)Rr}Cd)K6mtVxfkSwFufb92qc!!?`l)oewJ=$=C# zWcNOo>ve9&$a6dApWD6p+@8bd^6#D7_x9WY-SY)j=Zn0~mqeT|TXDX8^ZAOy=PNIq zKl1kcG2IKttuCDOx^ODu!kHBpYBpaumw(~>g$ozoUbw7#vDWJ1HLr_x5f^W+xOi*x z#XE;D-n)14{@aTWeqDUn^3vnpm!5iEvU?VB>BWjmuktUwxp3*-+e?4xUjATp`IFb> zFA@2h50uXb2@)pFm}P8Y9ses{I2-Zh)f*Sa}f zv-Q4a$A)~tHHVegdhWaCc;s5|``7yXzUJKOx~tQ5_fgk97F-{&^1A1~>w_*{_kMSM zh+ds<=enWZb^cT90$0`z-&Z%{V%?~Bbz}5yjO%cS`TPYWB zt$cTDwchR2&bQZj-%g);J7eYT4f}3yx_EobyW89J?riUTC&&BFj;VKct-O=B?+)j^ z=3AF{X=x}ovuh;PYi6u7F(xE*+_cFP$3*}4Q`<*Z$wr&ss+;E*bT(hIwG)4_?f>_w z^8fqO+pp9ND_*I2iO8BCvAC|#UaC|!cf z@YY02nzE!&ng+iU3#EUcd|jbbyuMI6n^7p$z-72@swLIIRm|65d1j&HwxLjR-&iR1 zgIegCUnp5aVnLy_rLs`k3S$o!O5dwG%caz&9eW@V8y7WCH?NoUs= zNf+S~T!t%93xB}Q4Moz6ydvr3-Xf_QI`1!%Tq}wsci48UNIG|-NSgJ!NNV?{NGgNl zwp!8&I0@D81h=Q~3{GKx&h-Vngjety-oR=6-f=y{^(=hA{1HAu4fZcw&vE_6^*hLK zizH2`2ihJ99 zOqy6MO@^gYi=|bt22#PLx0YlWQ!Kf1HR9^VwJFzrT>0jhYUv<^ZmNV2*h>*B}Un5ik-u;694$ zXt2aQhU-}9hPF%-x4S@+@g?$p&iO?By7w8JsU;~qJx8>R$?7$uzpa=8>N9YB; z!3p|6UvLH&a0NGThkoDz{b2xj!ax`Vq2%Mm)f>Vv59T@qe83l`;O575C=3IC2!tRQ z4#6-2M!{$p3*%rsgup}yCyfZ0j5!ph2sem;X)slQ|8%Z1AQGZr8g4VWM#C(a4byRp z;W~qBEZ2E3AL1bil3^1>;=hG!4&*`}?1ckx5DK9PilGEbp$y955L7@VJck$X5@wQS zlexu`A=p4Su!Zhm4-U`+dV(YLg5KZ+{b3+@fggN=C8T#cwpcm?)$@v_71)*e#gZ0W zTUabzhmCQ?(hl5r!e-1{U@L5cEZ6}%;UPSN$1rRW?F0c}y_~Xw9oT~d{0Z;j)T(0X zG@OAnCv*VU8aM~%;R0NQ%TNn9;66NsXYd?ez+3nL`m3qqb<{5er_(QBB(&I6EVYDI z&|zD#q@7bN>4F|KfQDcIjUj#~;~yIA!yWX&01Tlmw1cSql%=9rx(Qa5#ZqTj_M})^ z4j(_!7aNvHqoG%u5~(*hK_74iH*kl3-~j_*APjCDK6;Evdw^ zL@I>`9ZRG?;76Ad=_~F(x&95m;5SHJ@x$H3rbIG@@7+qIj(tj`PGAMr;_6x=W%Z{V z1GS`WVI|Ub7!z3{jRnW366tz;iFA8&iF607e6sn*We!}0NmtP{;f<43@D3RvD0$2#~kN}C0 z49g(}GN5H)iPQ>Og9S_}Dv=^!Iz&Pgm|iH6%)lHvfF*PSE9eT{K<844)ClxpGi-r# za2_teMQC_~>mMaj%p2FU?C(x4&=hy{iRZui>1=%x>Bk8-BRgGy)x;-pfc$qv>sX}wSmsi1qwsUq>|z? zsT9f}qNGfEbF@r)2TudarDqTmT`ol?l}k~uZBx0F1=bbik{y^uGG`nqmpXIp26o^8 z_TYtk3~Yh<*b`wHB*Ah>ffW#Xv|P&Px*rZg0Te+YltDT015&9P&O!~GgY&TSSh;kK z>rJ=;b#NWpRF(6W;L0T*SOx1K6ReJNJy9-wC(K8#olcfZlb{CdtIMTNr^+S2C@sks zf=`!A{AN`O0A44S{2>s6U<7_+Aq3)JF|3Ao{JNZ>ZonRTfFtw*7jOl47y!!%>&ta0 z_(KqkfN?Mdro&8_4GUl)#K97H4S_Y~QVLYT6Ohi8OYP2+1{6R!9D%!V2kyZi@D!fG z3wRB^FO*A85D!b?2|R<@7t5s$&~T=fw2|v3*bG}B8*(5QcES$W1-l^+_QHNR00-eV z+=Y8^A39#A?x80*!R6cKlJlc-Y1(t{8zN!d%W^3V)`Q!da!KRQa!H1~zsjXOFsl9` zDF)(U4XlP#SPSbQ4bovfWWqtH4m~8DjG-^V`zeQ{56~n0kmLZ)&=(x%ACe4WwWQi5 zhoq~ZEIlM?fmy*J$sBk+lULs=Bz0KZs6tu?QnLyvy={fG9^SZDNN?dCH1(*EUiYt% zZjP#udM&Gv-ltbcAHe2ng_QfdLbCX$LP}^>DJ_LWNP^T=j1R6Uz&E|5b+7_+8rPLv z)48tVx*FC%Dy)OGkOt|n9x@;kHozv>4C{%zm1`Dkhiu5eEr)9^WMbaMbvNvRJlG5S zpa9BX17XU!9)b#}gu`$Ij>0jhg5$7}u$$ln=95qjr{E0ShX?Qw{)8`3-3os=ZdED$ z(YaE(+@(?)VN)rMge~@jaj2BOf`fOZ)Ei!`W!{)mDTTt~m`Z60G)}6N`mU&yoWTWL z!42G@A9z527zAF>;!vg33R*)O_~Uz}H0(#Ev{3!9lnNu$({A=vkD-3vaL^SSN=U(9~s55ph;_TxUBYY+rN zFpPkaFa|@Gun{)FX4ne5VGra(DI9|; zI1VSEn6g!KJq>4|1bYqFb8rDJ!WAgR{U+C2@C=^A3wQ^eJq}A1_*ZeQ0jvJFK^N!> z*3b=Xp(i*(e+Y&ISPF@-4lK8z`#ld!H{t)Vcjs|UWNp5{PeKSu3>YHBu!so+42T#O z5fm^)KtM#qK!c)Nt)Dz(?CBs}#F-b0`FdGZk6!h4v7I`hE6?>un?7uf#Q zdGa#wJefWV>mU4J4tx#gg3gnl;5>8(pC=vj&yy~AoQiFbcAn%HoF|M!=ZP9{VeAiB zza{6%G585m49mzSNQ2F=1y;(+NDO4Bm61HihXN>sVt4~*ca)Kzpb9R))k9^3TT(`X zPnHqhg)-7nTSn$xFC*KYlo2;-IWd=(6HBlH8<+$^Fb8J)m6H(Iv#gx_vb>yFZ!9O$ z6HMZW=mgFv%b;azIcbH_x#h&8x}2!Bmy?*G6=W4qN^!4(xEiP^uR&Z3Dk#%H71p79 zJ;XyCBtilxAPF|WMxdi^GGYp(!X`+A&9DWwLONtXCS*Z2N zkjbzJ76XfmIuJ3uf{fLxAa=0Z0Mj+9AV#JY#28f1F$rsI1sMWtPzMbd3LMY`E(`-L zprgJT@KEN%aL@r=&;xxK0R}J<1Rw-M5P>m>fq`k5BAUTyFo!W<0b^kTOaupT1SgmT zzAzJJ!E6WuO$p`?!eKG&fGVhlI=BKxmKDSVRzM`^TVX!I00bZeNAQAa;0@DZHiSYL zgoB|qrUO%f*kM`Ru|N7$5GDKuS~IX-Ko|7D9L7Sc9P8E(?+c3{1De6eAIlGimtbF7 zQGvE+1=^Suqy}o?GStI0=!Bo)3H$=zMWMX1g3!Lhb_6Eaz_&3KxR=LvT!sBL0rQ!N z{T1AISCBhDV*A6hD;4Agyn@%T?kbJ}Sk{dF1={;6NZIoWQUR`SDu^2tQI%v59ETwz zlw^U0k}QO9h=*jT-l8Pe;S~(PTlfgtTa{!4=tC6Tg`oSw*Nxp)W5Sf8Gke#I@ z+aMqIzyTPNji28F5B%VE4yKu>BqKopGH?Z>e9R}Dgfp;euad;V4+oUQ=Ae>{gQ16% zWYlFP5yRJ)ald&5kgTGWgz1qW3~c%n0L}_Eik!D(3xonxpaTPXf4x9O3fG^AhIeY~H5C{>l0wN&_e!_d4hcZanf%)B8O&VZoaWzRk zT1|@1R+IfudA^!d!_9tto`F~cSD_h9%c{w@uoA>dbi>XnOyh1fnFG%G8Zth;hLr57Ar0`7#v-DlHN+TkD`q#%RI-~z7T2JYYio-h@> zU>bPCbnt;0@D&6=Ak2cSZz?a_N`J+@Q;Z7_vyw{L5qq?1W<21-oGn?1h7H7D|C?$|8NFjywb!%Hve(hz*QZ zufx9+){!w_0b@Y|<6wq)9hnJo@B@GN3IZSyW_H07;080`JBWc*uo~9D+@p2mYuJ0bjuc$1BfH@|ltTq%*VPf7 z#yaBBR!26q*O4^X3|n9;6u`6oI`SN*zo{erztxdP##e|k=nA(Fx(Dt0!#yM@CSN#JyCHLwFvW(I|mO})rfJnF|XOT{bLOB|`Q2rKiC42`lunJ;fHLQW_ zsIwMv9ju2qh=&BYfnkY=N8krI0ku#Em*E_(ftWEG$Y@x_Z6N7jqTfJt#0^9_v4K=V z>Q^jMg?Is~;Ud()C8&kVunBe2U@L5a&5#M{kO6h5a|P<*Dl|YNWMNnnVm4wkVh&;p zVk@*kJ9NM`=!9Gh-v(VMUx%A;1M<+X8}Sz0h91aAzdML`p#bH3h`n$h9zY*FgzXsi z2(b|HG2+kg1b%_1P=x+F;8&FU;Th~i_Y1`5@Dg4@G5Wnm9Dq0Q7IvZE2gHx?34R0O z&_Jj_0TrNuD$qd<7{CM;3;{N%g9Z!(ZQubPbU+XEVFVb!NDzTNc*6nALnPw25C`ia z0Ti$SHbOF_Kq_p4G}sJVARRIx8!kaD)WH>a4rej_QW))sdXNQ|WT*$1p~e;6a2c*b z12jP^yoG2t?B5UrvCsjVreGf64Gch-djm0?-aw3?#s|}bTDS~#u=}e9GCcs>F96#Q zp1?2g6n=$g@EjcHG?0>z267CxhGW^FAD+Q;cnL1wG?2&eGq^3sI*rFZv=RFbEZB^B zO2_srZ6HI>H4qMH0uT703n|d?5c?-Qgq@GDe81qhdEP*FylWtHv>VBDUL#q?ZzRhh z0$w{ek_z8O{1&B=I3+d`8B7LOco)VZZirLB9X!DUrh*qt18jb!h!Mv`0INVb7$Gv?z_BY6e$-#3yq)F!ePrVeW&UN8+p4VuUc%O;Y# zj74l#H<58*3vUvd$N-#ZYQlHVEb>Ep6FCa+I-AIQpmjBo(w9x->W3z>ku(zz)n?)e zQ^5yjKqRl3BpWr8w3ueH8MeSysN393I62LvXKyoU-PcS`>~AI~q4#hznSHvMP|h@y z^A*kHaaA)p)80%PZ#9!9SRB?u_NTOvQ#)J8X*dg|@DrSa^H2umPysK&WLFD$u3(XH z#6`f~-9m-}2Rcu*kS>^Yu7%8o8$Y!W&GHs94A!={kfxp%@^K@JNcvldC0Kzq*uXfj zh4JtSb$$aolu0s+P{1B#DxwO|U;?@)f-1^%a6s7+)KGRpWFSsLl!7zJfQf#S5naF) z++Yf@(0>T9!5!Tii0UvDJkZTS^hBHrUN8;3VLJH04A4X!U&NWfMOluhg{Td~fCql) z=MTeC{tA(g7=RcEIwM2v`m)AQGY=8oq^<@Eyd!Du{*Eum;w_S`c7b>k;EXh;jmAJS2hwlE4uC zHy~~VBb1X7jS)qVf^sUDpgampK@6MF&kQjQaWiazbl3_RkO`wPJR30!av&GBK^~Z6 zSUzF_Y==T9f}OAfieVS*hCQ$s_Q8HQ2nXPMI0T2`2>bv?;YTQeV{jZ!z)3g-r{N5o zg;F>N7MPbY@Ds}Cp$y8Q0+dh*Rd4~S;Ud()C8&kVP!Ct30UDtRnxO?+p#!eLSiH{- z#BR6?_uxJ}fM@UFR^amFdTs& z;3)hEC*cg-hX>&B2FD|~!4z-@52$#H<%deBf(uX$_0S27cP(THXuwe5fF=wBE@ZsN zegIjJ4LOhtY9BBzFo6ZWpKvU~cuFg=14nQIXOO{UZ~<3vgAfRXFqjVuU?GIVB3KMd zU?~*A4%i9BunSJWNf@EhN_uFmWSV9x@rLQ(3p2qVzJedLTM1LQm579`#NDiwjJIed z=9aBw3|K&uQ!8nK>1iy|is*ybhByPU9kByuqU;OTQ0|25&;>W)2FNk28}Sz0h90;B zccB;V!F_lDeee(-!DILtp1?2g6n=$%cm~hm1-yh;@EYF00KA2d@Eedxt%L$pPytn- z0Rt-G0`x+lw3P&aku%l@%$tmL?tygJ~gCA&zW4g;RE!YHS)?(jCZ6$NQZzXf#csh$b{{hPkF-Kd; zDsa7leFh9}wUUwW*6iPt72iut6Mv5%k$gx6PZ`QYwDex5pKp@P5c`!P;4d<*j z!r95f?-<+g8^$)`Skp#!H@A^Juow2hk8lD`LMc?kOVDm z_G>3|^b5WNvsA;!XLSOaTe9ju2q zh=&A71O+6)21tgDkOHZY2Ag0rY=Nzi4jGUM*^mV}kPF)&9|~YQ6haZ~fD$+cr{NqZ zVP`@+ISl9EDZBtdVmq;gFqjXEUYxGkunfM&vgadSha1of%TqB7 zGBIq_)^;KWQ!oSGwstZxznwUMBRGKvyx!hUzA0)aOJFIa6=NO17MOhy%YPWl4cU+b zxv&lP!amp!ZO{(hQ62bAeFuI| z-$D9zbdazU9prv(2l*MoJ3Gj;z7AZYJ4m7M8WE1ZMl4_~NWdD#0mb|p83~RS*N790 zlUyTbt*(($sDcZy+4dSy@xDgXfC*Z_0|OWd^WiWY1(O-q$g8MpWLoq!vIwH#L&7y; zbM6`msklZss+~j=CR}6@F5)oIf{EzXM&!Y8;DZi0puZlXJ{Z6V7zqLpf+0AfjuD~= zoKQAKG=WiI3SuyW(O?c^zyijC1Z-d&*n%BQfQjG;PB00i;0!XD3@+dbQ@{<}!2>*D zDtLi6OotiZ3o{`I=D{S)rxcc>yb`{H7+3|duo~9DI#>(qAr9gp0g_-NY=C4)fmGN8 zTOb>9AP)*)I}}0@v_l76gHGsz8*mdgt96nzNMK-G@MCt8x2#U`4&Flq=J^BSC-@Ck zpqmWoBov^63eZ3m=%5A+U;+z<02|an1BLhgDM$iSl{qa#+er?>_fW0XNiIXk z2&^LrhsCfAA|V=L2csd554b4lBs~+bZ6{$L^uYcG4A6inGdjs`5cp#rpl~7fpT$^@ zS6M`H2*(cWg{!AJNdt5=;aG0P@;}Epq6MF0yudVIDp;V~3vn7u2XF9!vFPuMI0I&a z9Q?o^zJdSX|UF5(L_K@NW453?Z{=0Trd7rqtr?r!5h=-HbTWNU1dUvibIo%Q@@IoLz78pEStY7LO{#PIRf9S75D6XmcNRp@!_(=DOXy3Q=*#-)5gfHhiCVOe+R6yEm~^$+ z616o=YU}#d;y8>12}9w-*bu=;PG=z3M_LnOOFtu>!^Fc&ko_Vzf|-{-nDel(iHU3w z#s4bdA+jAF{haFX_~$%_rz3`(O&@Zu52Z2un;eFg`qeY^YgXvjX6V-)`%41D|LgpP>;L%)44)fHxJFiAvKK}g*<}u< zE=1nKe@R;S=jjTqjOD(@{*lH3nZ~n@8wa%*&wXwj%ryzIG70lFSrBOwo@ui9xXF?h zlV#6MBDkX>twu%vX`(_avEoaX!c1}MadBFUc*}EfI@dJQ$~4>8G&k~3QWWm>HQOI) zb}-ZI&~dXPEoMibo0V`!AGaEP()VBGC*+z}AvvM1c}=8wZKiqMar62X^M>c<|B{)o zBXUe<=9ufp$J}fgbL;t-9K(uAGdhYV)693#ea~NaHfPFC1GSq zSbyXs;Pgv@={<{eYhgn-$TieXEwvDp3%d(zu!rGzL+UbS0 zbeN6I+Qwz3ja$@UHbT$8OGP+yod1_Rgh)cz`nMSfqin;oY!{!fUGfj7A56}&OFdzi z)@rxqOYT8y`|O$axl#6cS@s1d>XvcgQ-}I_cAkNkmIZ zwUN@~QhKzMku7DNl(O5T8ZV_BEoZKcvzFYM7ybV#<)E*Pt6c8tAMF~D?KUb*M~XwQS$o`+6)9%=JD`qHyRYwB^EsVC)APe)HZn?3d1$*Jetrk1~)s?_qT zvhk{xd(}jH)n68{rvpD^$W=Hn{~=BsNHYwE5BfE|B!M1VSfG# zzV#2!@n3w(e@VOlvRD2Q+FwQf1IY&e>$wL1iHQc+|8p}9>i>sn25m!a{6lS5hT7$Z zPB<}5J!)RW8}rL{xpB!zm+^N zFTw0=f_Z0x#Xy3DpJ-*5XcLfV8uaMa(Tmlqs|8C~MS!a`i zI+NxOBn9&~gxGBe3)rwAW*vw_1)R6A3C>w8rVv7 z(y8|8w7_)ws&qzvI;%9D-IcEKHl3rB!L`rO3e4cG%HZc`=$2;acV!s7%@F8h8ro-y z0y9lkWs37N%}O)PyD}}_W=eFjtn9OF0<&yaW!dFtO(@NB=*n_>ndN*;YCxSFU;7++V2=N)oPhkCS*1BaT{&~#<^=2HhS=wZ1?DbTl^dR) zySOxWNmuT&x498I+am3^MF(zMxoTTX{(mwrV19l{*Jf##X1GM?F;q>7VKYD za4^5%P-(%Du7aa)3rcjhAGhCrGI0CpRol(cQ)UyX=3+`}b~_KzA_dpJ>)@6J-1=*ll+1k92>!d#onxv6;2U zHugWr_2+f{zn17P_CQR*f!K2g)?7cZ?%jbn-Gd1e4k~6H+z@*(x!_>xxr1rf4{mvP zFkSci%n9FT&-y+$_WQho?+ecTf6DULqdS=5uXom9e!qwRndE--)g_GWf646kUrp@y z(+P(gC!F4&km{Y3O+4u``=s0IlkVG3dj53M>&8j%_a}YyPWeteC7*rDfAy(=?Wbn_ zbSmh^sk!e@1?!y-nRq&E_UQ$yPls3XG^6HBvam*%c6&D&mD@Kb5wjnWOL}R(t*<8T=e-!h&iRtoe`9_uag-?!%=x{n`YFT18OphBdXxg|(^YYtwGlZuwA~ z{zvYZo;&8Ps=xWEBzhvtBOPao*h6igJ9u_t{KHu=PGr`0jk;hntO`J~WaMO;pDw+MFi(+9pO(6RWI=-QA?|v57OHnd{iB zHK&=kwwYhltXtNs-`#BRu~{&p#n7=uG^fR6ZHu_5#jLEwyt~EXV~b?OV2VDQIfME6 z?27)4VYE zz68fk#hlI!Yde#RI#bI!)4DsieC$jg(Us}gl|82`cWqZ*QCC4(S7CS8j*nf%Bd+gu zyuNqN_5EwFA1u0lsOysjZZxsgF9TOJ6dz^@Ydbo@3^B|en;BwN_k#@{gg8A2oBLqF zx(DGq9xN_@u;kW*WuG2I81zLt^+nI^Te+?;W=CIadEc5_ed|8;#Th(IaC)ei`*6d$ zhsir0rj|cUyY+C(r-$hVk20MeWzT&ynE5cXL87ls{XO#`J!<}nW#2BD{%2391`DBWk)O%(C6_sQGgYStEo$nJVlxcUr!Em{X>xpKQKc0GMVGU zuwu-*o08+}Ia-+EYZx!g$Q2)mo5>d+X7mN=j@~EtG&R=pQE+CIMkJn4*^s>eLulwjXicu5xAR}#flc`{~RgZ;*1R9@&7%Nl-Y zT0BP@l6b_h+WumxGhftK!f`Q7bFT3JKn7))1&P$B(U-`CT0G>uM#3XZHpSExml z9?V_Y98adPsKZ^nOGGqxr&k3qH1IEio+LYxA382AN$D$N?xOj6CLiN8%u8~pVF`zF zPb2sLu52!qh?Jf7io(gY_I>Ft4fbjrLq+%|iZY!;l}w=v6db8Cc63RlOx!0B`LN>c zvbe!}k|}Xfy~eUyLt08U8N$enQV_O4#*koV2q58O-1(uDl1jc%MwEQ)K{b3ME^U4A z594ZQD>7bnkkNb7eHp13$<$)!g!I4 z5!h&dnm;5ltt7zxaC$ap*52%U-V}jlIx!WdSMrC1XHc1;X~(|P5caF*cCJ=PYpB5# zgGd7#HsMJbX>rwD^CXIzTi_;3o-8Cj2|G?wdK8(rL?LDAnbWX2Xd1p!GoiwlKP0}y zmk}3tjKfz^oGwFRFn+PxMi&@BC({h z)^L8tsd}oZMAi`4s~^V^?_<|^&Jzm!Jxx>cg7YlkzkukTX?^FsF0{AEnz zY@aVLrbkLRp5gJ9H60=5v`}oeI1bk|=@`dTm`34M@y#WDRFY1bXo1EO9v{CLl6l&1 zBqa^b2PH~=Xj~G9DvU3j=}BSdRPl`z+0_BTLW#^1^H$ZsOrj}e0+GT;Cf=8Q++7g2 z4;yKpmJcyaqVVFFX(Ol|LXkvCHJUG~akspj9Ytl)II=4HCxzTu!Z>XOo(P!k$_wnJ z$gtx`XdEMAOm$&dSZZ-S#k+i~?DvRf6zYXq5_k7BO9ru5lV)>gFE4ZDIEADX5q;~OPg`f@_k zGNQd1!op$H$q)3Ub@mF&nud8t6v|Hi^fXOhUZ}CTyPF^>N5UU6PqI>DNW40a8XBsF z7ln2Wf0$uLjf{i!$q$urs`7%(Y5we~X;@D~l!c)UrfC_z0b+9+w)HMcLl?1edQ~bT zzC`IM6qP7D8A66LRa2-)=NPHtIF&uoU#Vf;r?r6{syG(KmlX?X4W!_vfuD;CNcRvi>n^9^Za0FS85i)wJ@ zve%EXtO^p)IM3ZeaHiDfkEJ~i2>w;SwpQR{PBa)6g|0kdn5K-kDnv6uDM*y?IH7`F zTIo`Oab^pb>8wx$?=GhCJyb>h9U);kIh?2?G+$Pf(J^OFQ+G}>&2wsY7+=Lu)LUgV z4`&VLDE)9RyHN@x^c=0AF%84UxG7hQi#aY#^I{**z_CZPeZ+^&HK~-eV;s(C_AV+f zK*%7na5j-Agf2>}<{Oz}=V!A8afC=oM!o&d`Zz;QAx%D>REY^;2^C2`0-sg>KGoT0?Le#w6u9VQZHf5 zKbz}|onH&rxWZvFQ|vgw z0sC-a6DMVJ0?hVGcsK@=IG$ph0b~qH$t7P#+CEO_>a9{M>X5`FN+82Ln|mu^6RGj_ zOxc(19=RmrREtp>BiSGOw@V$S9Jyt*sWzp{y>=g^bHXTLZGd4?lGJk^j+E9Ap$KR1 zO~)>BtojOvRm~Fb{OvcER`TO%#$1NjRMTI|kTGi5UPlT;dt!!IX1WKO(+29ODcM#0scGuD z97^Kgbt|zrhaqqgXz@p4Pu+>A;}7oYPrc$Ry%o%WxTIDW1qs; zJyDb$z?dgm=`E$HD)OWoj1|gx!gwPhlVO8+h$;Kh6OCxOv=D|S4?BUdhJ+4Y7bmmh z;<%opnVIQQkK~i7qZ2ti{t&E2TnYD8^Mwk5g6ql5$>y4w3ahzn%3ka(xH?u%RTQTa z=S>b9xT}fqrPBUU;juh>H6fiyYwhFR3{&>{v7DuQRa@-$iIkX%NWlqdu;g^&dypD> zXqu{B$5i9uNSsy_!@OhSgd5y(e)aX#Nn>VmHIpQ{*bIs$8Y?XcS89{q*JvoR|7a*K zofrAd{OB(Y#p+SKbE8aZneWf!_4>g$L_ zOP0)w{M;4&M;DrK8|d2U^uKMwu|of)0-yDlQ8ArFm6r*W-&=GPE>Wehb%_ckgs5$m z?vWDB_9?HsUoY(zSP)ev1~XM;Vu1)1_(WBcZGyrYqUOwzV#LezCJIqo`{5&pqV6OO zp2Ugs#oz#K{MNFhaU0p7?4F$k=PO8i`m9mgS> zH}Yi>!v&7>b&2Xw3Xcq7ww#ZX%20W5aSY)Q-X|x1_u#w54Ze>61$iu{a2IPuljiP& zx?V&@)0e2t55c6>R&=nJVXh1r(Wt$cn3VtaS38SZMC9_YZ$p3ITh6KRYSl~{1|%l0 z-+bO%{;kdVr@M|wTl~ka!*vxz5)V#s0`>P`6e>t;cs_}}|OiJP{O`gPA4xU;5c1!;09#X>4uA+Ns zZ@@hwmykSK|8&!!tyQo1ep2^4v06um`$?B`6P0x%{G_Y3j!wCw>nAP9$DfX$^sGVB z&ofYOg(K_Q9)X{9u=*6CpVW770HV&_fZ=BlaThy7>?c(Xz;mwy$S8I9{>i3((kC-e zZM2_MV)p&8CW)UE(}-|1^OL?7d$B|*EtgUXx7>Pweh6Ys7 z7=s_aqQZ9~G22h7G3I#DuOr7W1S~&k+Kq6se1v+ORNYT1zqfCDHOEg{F3C6JYpM?r zj=F^Cs{0^uP6QFC7t==gF@upoM?ehKOAzrZT6Kwt;U{f!>Fj^0;paibYJSqewZlTh z5&Mayojl~PPVs!k=XHi<|4DDGx(d~o<te{R0CQfN+hkPHQcv1*Cq^pB zaNE-yZX?RPtnfYNWs>8$&njPTp|vR~^7|kQ-X?|YSdfULHHpuO+n!?q5={NVy=O7}w4q`xYKm0QB$gWHIP7m_&X?`;HO*ma&(fyGO9fKH z?*op=&2mDH1sut}@+!70*s^Ph?e?qTC%TsSV}@+oe(*V~z3j}5rIM<61a%q~MZj6+ zGNCuRwEJ;{RKQ7AxjPu}eC>E?zyB^D_b*E;*1G=7%Cfo3uif}%zV^c3s6t&cjI6BxkG&EqZ8qUSr=ddZP2!m~$$gKT?PjA?uIlk9@vdeH{_G zDD?O9V2c*-Dc2-B-O{8a#S%{+`NxlsKRrd((zIJ>UfP*oPLX@5FO4XJADs<;a5fSc zFxUVGfAF2gZXvm}4eKBi5S|GWe+;($tVp;DHzH2N_HO4F~}t zOa>z`gdlJLe{h3gm?kbCWF9O83m9x%4AD;i>r<62PD`}G_vO zZGNiFiHi#!Jg{(#5&?~+s&*Gk}jg}sq2wDtJ9OTb_FwNj16A#w1ISz zl)4zl2u?m~!97}LM)LhVcA9nvW`28!bk2cT~d2UYpHQcW3lvPTVALB-5-FYVAb1F3-&5&?dj9NxWoRrRRSEV*9bTxM! zEw*N-oOE6)FwGO}{W?QFpv_83xFZ+K*(#G=Uo6&2OP_Hk#vEs{gi49Cx6RhH_yt%b zlj@^5CH?Wm0PD*sDQZ#d?`NcQ99{#*m{;M|n&@cNz=<;P@i=r1TU-qS$-b?2_ zr<F&{7Z8d5zSdP&rfPf(V-ob zGZ|z+ogqcr@pFrq`+I9zRbA{}JM&+fke^%DpX5@C8qENM(5U5$4CG%Lk9Ru_CcDb=!s-^0Iz656ujyi+%tDnU@grjmX1MOtWa$3+x%$7i%gI2ZxF@^}y zNXH$AFQt^Ni5n5hBD9hDXu@-7@8qTobG_f2{=@vF)|lupjHRpI1nxwrd3KBs!x*jf z1GEJuW5l#kex^#o9>b^U`3Yt9Q~q$(hA8iXStErh*#_qJjlb8)HWs zkemA8$pK2*Tf7(TH#yUXlueV6*JFGLZHxfd+z8r7xhY{hmP=jj$aD1p+IzWaE$yk? z^ekr2!-w`tZaP5Ts27uG)ADFf442z@F>e2=OTjm;>t6MheSCLwwG_Ptcq1FcAh4pStecCm*K;VEq6KAJAKtDi(50yM zf8tSwydE=oeT58_2N%Z>4&i+kuiHKNZtDl%M}WfNSWMwA*2>fOXM9lCi=y(>m!eis zgh{JyF@3QNbJhJO8nqWw)YNg+m7<-WaDfsp5oskTWhER5EiLZ%)^hL)I{{}%C3-Sw zTGaLG@k${Bt>{YGE)K&Wexvk`)Pc4sUP_n@T0&f~5VbU&d!M07=B0a@&MpR9F9vh- ze5hKiecTecDBgKrqz;>-P2KNX8ElnP=bGi?F5;$8g)`)~TdUo6#Vw&H;sU>FiDW37 z_9ER~G?eXT*hMqrr-dczM3{~6-q)fNtiNMxFNGS98vv8GN;aL}ydbVpZkyfAu=a^_ zV>Jbz5M~%`_x@hXY^iO*)de$k^SpL#xvY@nyAM??Dvl}iPOVglmwNc9Xf2Aq*- zJ&j`BeGXo_7k`K?Ppl;4IO_bp@$`LCM}wibzT&C&2jynt0**-!$vuhZDG`@9bzFqH zn_m0j`-|huv}qxPdI~|-9&##Jd*;3b>B(6t{`+2qpPIFQ%jGuBJn^17WmxJ(`i{=N z`I$d(7}-PSr~crzKW8y-q9Z+9cByUE5&y(^)o9N#z9Z?T;{v=)vgCCq-S_zpNj1&c zJ?1*x+XKta&)78FlX~|(_m|?#A`u*OE$6=zTH>&&}MNc@9<=*yaudgWk>H^$E8D@Bd zS*1)c%&3k2sdO2mdQ6V)&1H%5R2$81zTU+jOnj4F3|$YjYJSU4>te@uq9LVl7jXpm zRb%pnJ2tjw8h+W&XWN~bJ%8n}?qzl@Cf{A?jwp?n7War&tYh&ow?xQwny{nGpT?-QQx~N_@LWQ-4+{fl*LTcUnWsLn?N^M zX7gCWPt#1Xyt|n%syTdt&vc_!hPPR&N2=WN5Ovv=k(%k3!J;R{nQmB=)fL&bKWAixL%?LNDciCo}Q}>Q~bbn3BjD$vHn74B5an#JRF9fJ2%1~>5sVjXERNnEA>Xy>7#o)QN~=O0H&3Tb6Kid*&K zCXCoQma5B+J#wq(&Cf5_6^#rRaN@knZh7=g^j$j^ZI@U%bl@IagUMlN5mjfwG-;F_%}!rKEnXR+SYIV8 zR_a996_eyB4r{%>d%E?2sv5;7&YHusix(tY$FWlsRl7Lp?x|d+$Wo#!oSahq@WqJD zi7Iy@;`BGD-MbTPW0*k;Go)m=i&SvAz&&s=jgd{_(*)@z#T0eAfXfVL@I_3H*n_I$ ztYYL|8DgA3=ceqHo4GTpn}P*=_J-=k^YP`XM5R7h!cGwJ*$g&at0Wi=b|qDXM&VLdqp0$to{u)}nn?^GcSi}jqn(iaXE~WER^i=iB3oti1KEHS#9)s5Mifpl`ri{*P(0Wr-c;rX`0FP zaU46gc7DAeONVWpctJSKPMmD8dxd@Qxgd=x^mw9C(6Hgc9gMi`ig8q_9li0&eQFSu zYSs-d~#>L_??EmbPS@Yvf&w8+iXwlxe(QxP)a~<-9A?*p42?@m0ptf}=?^o*iqIwM~0o=~JgJNGd5oU{CZ zo{O5woUw;m)pO5Px(-jmt@$QzjK`VKE&Ve3G7`zwAGRm-RAkIR!;V!7Y?D#Nc?l<_I%K`dH8O0|Mfy=^bDka> zpBS<^@h*~M0iuMIi#QNOH7Uj+83BgesS&{zotMV(*9e9;Bwa{YE6E#5H+t*B&tNE? z&12g&Cy(aNvy`Mw@3qFMV`E~XP|J$0-|5P~&lqCftQya8vuAHeujHJi>DNe=sVsZ_ zdHr`{8TMOKzTp;nC1hQ85xWRXvVtQFwQPfiX)LtL_cCErhgB7L*<~>=-7jE_YQ7w1 zrBi~>RIv!9LFQk{D?>B`tHh%w@EP`bs^2J?C3el&8PsWXigyHuQKB`3x9NnBiwY|@ zVH}!brIafO~Z6O0f%x=z`trrO?58mdu+NlQ82og&&CQZzGnTJIz#nJCwBg_ zxjzn1vrtK$C$USW&ndelxVA|nt<23rce4gQ?~N8%a))wLs>MwD#=+p>H7qkeVsW;+ zBJ*L7{Zh8$(K|k3{I8u`Qm?5c>j33q>p-ECirb=K$y(2ysYTB<0Wsl#z+%RXdB6llRLlu;R?L_I z17;9&#tiD3b0nC-VeK-E*Gzoaep1|Gsm5*L8O7n(pfA>gww1>Nta+ z?Kz8Y&S+c1x%6^MlD16kyw9`TD}0^CkNNv^lF`m}TRQ-YJ+myho;Feose z-6qxI>)L4kr5;;Rs(a(Uj8#=EKU6C{OXd72Ow%nAXUz4x&y_)dBadgu@DS3OAv>P8=s!Gp#+Z$?~gL-zK zeUfhJ`E_*lLK(Sx_a1Zh>0Fan@cbqnw?DfzX#dqXrl=-qZSP*)T|370(B@&!27fgi z%CJ2D@WuAfrXj)6ownY}P;P2nwsu;c_~MW9w$NmE8Wah<+Ydq@J>Lr_N zJJd4q+(WbIu}WCHFq{$_W- zLt+DUcfYt;B}6fYSw&KkER#iY9Rf@49g0D>kcv%3Cll!Ic#X%4AttS)Qdm{YHcAwI zp~sr_*s6O~>JGDB0Qp&Hg;k}BGC?=_SQJ030>RuB7xb?ywM@y}nV3UNfnqsgMRuy2 zNf#BNtK4z1VGxIlQ7Mi{i5e+RBo<&pTX@IfDwWxyN!nul=LZ?AcsdiVX!MbnQWx12 zkxH6iF~iLaOsd=uNY#@CX;)t;uIRIOjFFMb1+@Ag|@iq3aZA;+(d6w7Vcil z67Q$RYbcx=VXh&nSwkMkETTypdmH-5Xdi1fSS)@@9#u&RNeT$32rW(a40H7hk@stU z^kB&Om9Fj@Pj6IHE#!$ctq5Oi;RcI74KbC|lg(jfzg~vb;c_`fH?g0>Agibx11d$b zw7v$sYETqaQ%3O%Q&P-WbC}$dnJ1$tT1ul0r~>0L+q-E>GH<3CTz+1tPA=3chDffW zy4PyzriuRM1m<#6DTUH4DY{}%Cdc}PC)KhfW4gL@oFO)h<|}@R8~UMu8RLK$;|yUy z#nY{YX&nNSWl1uJCz~y9eh6oB^9w^oxCI6~d`x}{>W!t}6*QcWLCQrRoq}yq67fuC zd4VJ)U9WeD1C)EFC;`z5zo4beCKQE1P(Qrj+D!|MIkIrDKJVLvn0q)7;I(Nb*d4M;^1`tNLY6P?(WD0ttfXhPJoiJ4*JpQv$4+wcA zL0-kB*aEH!r7(z>RKVoL7~DTEBAvwG{>#0{KHc+#9`GT#Oh9^a8LQR(WhVdD-p)IE zOyXwb)L%CvskVEpHxq&X`H<+J$3WJHMvI$UyMMM1=8* z$ofL4YKy<#FNJUsQL1=ggMS_qk*pi&Uj6SImGaH)108vc#O;f@!r!+g5w2kxREb)y zio6Yq;9|qVbEm3KTP&6TzCNk=_iLsaPJ0}y@WM$Qp;UKjrCTWVi(g%8w5~2IIR%^R z{(XZJ>ZI$uYf=+T(bfLC+oN&t2vTY~MJpxwEazla>N>@UV55A( zb*l9#>)VynR0prabNMWv0{K+Z^3wbfMsD!Q!YEeX2N|Z?DW+qiB#7&5~L@f6$If@^b z8Tm?>l)@#V;I)On;F6;hC$9)MF4@hC#9ALY`Ez;W11DG4&q|AMi!!!hjONumJVnV* z3(RQFgf#9>sajNv@bg^f1St+wlMA-F2_8H6ba@$NRT`H=c$%+eJtq90mm>dk z`hU3;`R`~n@gwht?~ecLi~dJ_k=;1v4CXS$CUk{`Coy)4A;S6!C5rjWHda+a0!_Il zs)h(_>MvvnKb%JQ9aUmm6?a!+LkG4BVpd+EvQD;1R~1VER*u5O-A=Kn5xmV+QXv2P zKP}dV!K&EWgx<3#nzOT=C0WGvWS<7|nixx_(yXsktZ9Xc>ar7R4 zW^J?RDFq z4)I)i)UxAQ@ygSFGjGf5XuCMB&V%T#f$pc)UV0W0I9?4~HKM1tb3-VLdoB%Icd2#H za?W;>Q}RYucRqe^(3i-P&a--Ed%>kq4_=4WaWy;}(G4Z6`HJGvu4$IrF!u^Q~hobI+wVO`GgE{zA{} zz`R90+&Trd3ig~-Z;hMJf(BZXIz!4#u9bRWt&iK%S_NnIESJx*(aKBV zKboY~DsrXnnxd(-R$c75x~NCw%;Z%mt-H0#TbJB5dftXBa+-8&nl`6i`CdB<6m1?p zyQq8h^;i8L4f4p9!n>Bx@G=voJ)C`NHTOrS9u-|(JuTArk5dQAq_r$+93Ng_e9Mh~ z1B08UO+WqU)an9jBG=BT7hWKZ8!<1x0;NI&YDRIx!rQ#?lhyMzoI1DO+^*Z)i#0!W zqG?)#$A2Vq-;`N=_SBOBLDy@|t9PMyTGFdwvk$gPo8#v{`()Q`jVC453$N~Ty;cRU z)lJ*FaYi;fTu+-aG4AxqqQUKI+h%2V4fb$OSezXi>{;Jak1V>UK*@xxM}xQ;v0ZG6 z-qNn*gshC{y%Ej{rymWfRG@)Jyc~^ljG0Z!d0tj_RIo?M`PP(U_l&KrDmysO>vFI| zscYth@fpP{6>yxC)zGv#&+CaR$g6QBF(?0@F*P#>RIsMrd}@}>gUmPUzWy*APJfg0 z?T4Oq_f18=?=dwx+?N7}?@Au!-jI^KPx;-(9$r2*=;5QWH4Se+eRG`z33<*5cN4?D zzokIp@WS3>kAST5om959z+GWI_Hb4w`KWPU3XILT_h$ZQKJMPLRuq-PN4zl0pELGA zq0J{AyfwRJCIuD9i8-7RE6BrlV~>CP;dY>7ke=6Z>|u3)iS`{RG$05r2MQ$)7iRZd z-Z_Mhbvk^v@zHNikyX8s9tnN2>w5h;cD)GdI;qCoQ^^tGde-vih-Pc5F>XS(NH9se z(s)f%+PHjcnAiHTMXxmeaNDjEyXq7~)YTHFW*Vx_+Hf@}))|U-dj@ zNrRDbs~QZe>XsB1|2(F_yX30gr50KO62l+n#5jH!6mBT9u$K3zxCcS$`QI6|hckvp zdJoQ6HQijLWP;`$Yo}`?l;+OEAH1DZv}>ce`E8qTyH(=2!{f+G%}sd=qz>Ah9W}Xt zc}&HJQK|2$jUJ!fFDR$lpqd?DSMYL-%xK%lBhqPBM#lLkvyBNguh`_><23lt&8VO~ zrrt9PTnw@J&Z_7+D1GCXN%4#8<#;)cIdsyp>RsCCLn&`|HThyYKCb5Wfx)BKpN$Ik zvK^EEu4X&FUxlLlpl?&BfUJ|}pX9cft9LZm#-&zpxt4S^UuINL$4hqOof6`m99J$% zf=)7)lgYLO~<7#t`UAcYK%1dcO)~D3VXI;IW?bf|F1$i%8 zzxF~e=0Up$owiphKBM8|i@KICu94Yv;z6%g30WC|L-NgNa3wB0h$*i}|KzfUxJGlE z?aZ!J(Pr7;dj%Y0E%&-JTw~LnOu=5JR_SMzU@yhxj8h+;=T;>3l#fd0^Oa6`#O1%& zUAZO5D4Sb~UAj|r*&@z^EcezfxTIFxn-P^Bc|CgXEn8fXS0$Wle#dA-`@uaIKO9uK zeWP{b%O~v0j)>}W!?xiSQ$qem(UotS%QNrW#48ofKeHX*z3iH18AH;-2Zx!i1x@dM zseAr%1A}iyo$ouMZ&p&jYe%yC?&_X9A$PgOS!<8iH0k={wU&EZue+TM2s!_u++uGv zV|>r@i?;=Y?wU2CZ^gvA0q-5Va#7+LVOv3cbj+q%e?#1|rfUnF;O=7mFI-_%>$ds# zW*(DsJtVi3$Hoflf#~juHzsownfAjGNC%fACC!Mr;ryBw+<_H5ThQ`Xnw+jr* z8ri>8?JVuKCNv8jRj;v&eR`3BVeRJFJsmORl=rsZ^T$-Rt~5r(D3WSV)0ZY@%W{@GpZJ5g!#uL*zi$sgrQ){i3hbu`^#dK_&t;y_cUlhfM#depZ?x=ov3n|tq?^yuxD zR^Ge1SN%A>>Z+aHcfOn6BL27MUq)>7?sH{!`H?%Wx7oG!Z2!Hjf1P*wO7ET5H@4Vt zwtt@+e(x9j+HGgMex!K1v5{9Wy?Mb=aq-!!`)+Ki@cdEtgYEnO-uB1HRjq?cE%E4a z@Om4$`rO}T;BI;2ew*jIynNe#`{+Wzj#w+nti#mALi5Z~W$ znpAP?aya0(>uk~gc~@`WX>gx^y$g9A;<>3>z=_pm0#cTg;V$IAZ=$SsA^#oy@1Hh? zdDLm(;by%H@w48A*dT+Pt=V$ORn5NG|YS&XG9m`&_o; zYX7@~`ybuGX}qIxRm=vnjhRaX8@03&l+St{!p4phL0`pCSn=h|($~yUxG%kD6C|BZ zuBXyOVm7|MTzmut`C8Fs%t49}m?@@k{tfUY)Stg3hXhn3H6bC*&+BIE>n7V-18G`qu*Q)`*!zmeyyfDkS^g;& zf5q2RG5U+?D+YtNVn-$XGfNst^7~L8#%2Qm!3=6=UlVGy5JaFQVd2e zU)-ccioxP*GW-0+(ZE4+ZeIo%NeaM%2sCEZZ1A*Iq}@IiYN1)>Au~fjl|B%Q8qA#X zt8%2SsG=ELDd`&}N+x3L?AzJaY=|NogkZ>&Y9#Gz7B(bRGZz@5T#!VCkoy3s5t9*W zzQhY}vV4^wqE=tv%K^HB!E7O$MoMFS6*o6uUmtftLkkxvV6Bu`Jte=v02li1uO2az z11p0P3Jy9qYikVj1z7GbeE2Y?anj{s%gka4RO1_{BhJPl-4oO5vDL}8-h|jC@mkxZD!Uj?nVF0x+ zL?NxYBU031Q3wYoRrZxT%8^g^Mx{^YOqfw_07T z-2hv|B4NY1r|Kj6H>)b=Y4}^tetBTTQpRXh3wu#vb;YQKEO3WLzF%G6OR@0%A-H)e z1?=4vvujV2jPg=O?m0T`52R-LlYaFvsZVt`?t+pFm_?5!bXNg`-JqC_H;nwqhuLhO zTXoS8!kcRtF+ugGyP4|?#Z|58hk8+qs9k+(RE;In07t&W?W>kER`U}RDTZXe1|zq3 z+^`rzLLFt`t8y*EDT6x1grWH;>ezN}zOJr5om7RcHmP)+s~db3bCg=v5!3cH%CPn0 z585`PYF~qD@o!TLQ<0uk%?vHr1cWN;0C#KxBT>hBN%Is_fU$zf4NaKT=pcjH4=vMc zWXFxZk`4=y3AKy=x7GzGu3!d&8_(QlnkOI zg7CkFr3Bpj{I8cgGLir1<<32C&qD$Z#7~H?T+}ZTCOu*MK|F4%j1w73ETqZh~v%#{v4Cy2vLx)7O5cKrOSKQ-R zOCHkLde~ZjK!a>A8vrB@fkJq!rK%olsRZOq_!qH^UH|^7{IQmvGgZ%dYQVe+Wdm-E zFZ+M;__rgVs)ywtiM)r%w>!&}DOWP6WTl{zrGx%68C;1Z5`MS!c+9VG{Hu%(|5Cu~ z?}U{cUWaUQ!vB%FB;GYpaTxGoYjz>ru^p3vDnGh`E9c_ep>4BVv9YqQ8bYXrPd0mV zk-?4!QiSJHQoM~-fEXU4jF2k$tm7JF>ZFLrFe{6 zFzt`-1r6it)b&(c>|5q)s;cSv6o>i^G^f}g3aS(Q0{Lj681ou})s{KEXNosEQ0NzNLn6@HUtBOmHaVt0b7OKmu5Owtgy@FzN@@a7| zgUQT3QnBfGz@XU8sP2#sR|lp`n+l4JF&VjC_`Q(A_i2VhLup(&#YxpB>&NT@R?qww z%gWU&x?`uVT?g@t41C24^V*|(uGIr>Oe!1j=R~V{{olL)-}zJ- zAw4~7@Kan4Ce7^pp>-jY0ZM_q#^jaqz*}^SI{! zb_o2W1^f7UyGvhu=c``2UEab*U7zjS(5Pz#g2-+7;hxc?T!mAGk3523mpj|&rDc92 z=cCho%2l0IYax9b2En=+_9kUNk)WAep4uey#gpX45Qc?D7=yK<;iNw|23&6mvJdBB ze0waLblv*^?*|~;Rwdmi-d=YsGHUoDHC692>ODd}JpCCV zi(LTkezfdj)vV9j(;3XJy_7UP>4(^Pt(UC4mEu#B(vu*H8AW)EpgW%0hbf0h+#=+F zDErLbtPCLH{hYQ`1zVdioG_Izhj5?p5rTp*>@7+a5{(I+2tx^b2_MMJ|I$8~+f-d? zL6}8&2Qu-MeW0?6#3>RbUfX+V)8Ejrt~r2QZvlJW1BQM^`nop#{iBq4=n845a-5Mb z?(HLGYll>$iR{!tzp;mYTP}d=0Krl4RU9FBP9)d~0%sTU_PGJh{@g-1xmqB%eMF!~b zE3jC96~U7x4>PvXCe0;!J!_QFq;)<*)IIkUqedLQhOeb7MfZmxkjAlJJs|&fmRq8?6QdOwAYXcaBWzN7bQnN_kI7#|Z z=Va2}U5W14yPV3FW_-cStKVq1R87*0mT3>~Zt#r05QR{4??QL^Yj^01xr~ExXb1>x z)kN}IZy@^1?pa8$wrYKWF@9TEQqw%6j+6g9?ieP+rEev<+tx?Td#nssQ<@(3gLQ`hg4QSR52lKv1v307KwzC(a z*!D)uVu;z4AZ9<3Am)C8Acohy4|W^#cg8e6X^%BiY{8~4b{hzyI-v;xooQ-6?AC*X zWK<)_(}g8uDE*mNekbvQg#Q40FJ%acXM}Gc{Gu63gGfk!J_Qjz5CkqsxkSQe5abLr zWjtXHVIIh{L+t}KTbiVl024w;1bZR|4|aVdjaxbjP=Cx{mZ!Ol#fs|*uW>}$LH&rt z!&tiJejJ2%CIFgE1O!Z>$Ycn_jrmQbX0ckAX&7_b46@FXiWQ$#9G|5$BW0LFs@x({ z2T4uaLEHI7qG7cglCi>f&vR*UqtW z+RXsgcIn`6Q|N>}sV_1}f0IF4!UGVK2QVQ&;Ji2Br9Yr-Gc4h7!IH*kEwV7_jVDM4 z8A=(w^!)ovV+QRQWui^JVorIG;lbFCe)NL{v%L=XgfbLUcexxRq}Zx5A7svnNer2D zxKBNJ&KxX%Afr~BJ0H+@0W`lMVD>@;9D^FPW6J<**8--Bd;MeLTw{cm0QR=7i=gE z326-4D*|w>A0-}*s}7%<-8$o)M;XnPcyz z_dogE80c94mFGq^NZguc{D77=h!w+QT7qzqSj-? z>kDQaVKzZJx%qQOUC|fh$fy(7Ahy}{C8PBvkw&lJFD72;HKI*y{D4G<#}+iOZywWSjAE7FfY z?O@TpIQdaE0FFZe4#NPsa1Gjg8QV7}18y*^wTZW2v5=A*Df#6#C410=R+l%EYDcxM zX*{8?Z(EB2F}r1LI&xeqvXd#WmIC=(Q{ZO@_zdX?dNF9x+}tkoT#*;`j1l^QB`k&X zs!=U~-@ms7{;_@#u-D_!Fl&U`w1k%c(J(CvvR++U%FLZUeiyP%@7!Z2OE#sm_=*xmcG0Nk zSfMr(L(<216s1vnKg@^lYNX$88nvxzaT;~4Q#T;@<-kcbUjSSBf1{#Iv5KT%Kr_}U zj-MN7d(<%fVyp`z;|s!fkEa!>M8=ILLN9AACeWPSG7GGcHeFr__^}4?bseDPM!$$F8 zLWBa?Pd25R|IWzkMPf8z3<%>jMiV|#*+rt*b#ix*kg4}A2*V9>yOLN=*hXM&rl~iX zc4f+CSZD#aDAI_;P=X9b*V`2FC((egiSPqt#XA(~K;-dVq%OyzU)JTy1uzcc7!DA5 z|A4F}PiRo!Q$T$&;ENcrRekY{b*=7eP@~=f-eAt!V;Rfi8O!>1^|Jp0ZJG?>lTLs} z(htjI@YwbQiuH# z;vh@RDM;Q(nZ|5a0Ln6MwENhqwI@q&M4(;L)gy4P+Ip&%rN8!xuGVw=S29LvQ33E; zvnEU?ORxHL@x-Sc;rIew=xr}{0sYayI#fT|WEOBJwUDz; zZBnuH-8nF~>dR}&ENr+t->XSQ- zh(l9&zmKH4DXjsUa1q+Y_JEMifIM9RS>kcRj{%ZS18mRGynx6M|3H0o{W`R0$Cjpa zAyw$PW=GQKYlnK$j=|~BnwU@Z@E3+5cF4ju6qS(+=5x$*OB>4cZ}d-c<_)I-)k^3aK%omOVt~ zH37Y@DSW!O1T+y(&ydoNwgzu%2dW=lUOUqXu)Q0Zta zo3s@LNS`VXID)T{?VEM@>bZ9JVzQQDzGe>*4JlN4y|ZjSe(yr|X1YOEs@9q4Nq3t3l`4FC z(1!ed$dBy%mmu26exR-o0Hurp#i~auxC~>CZa&o*tIZxn#+kwJUfFpXT~fY&5<)$O zsD@aiWT7Y@jG>LEwX*M}aL2$54$}P{lmd74Ag=_mUKN zq@FrYesbC`BRG(Jm5)}-qO1e)?ik^kKw>X5Gb;(_L;dkt% z!$mSCkP-U(Uv@9Ey!Q4QJQv-epi?$rq|C7$@ax(H5hm*sAow#pqKwy=wmTlbjrtl~ z23zk6!14^x9ohA%xZptj{IB?2&2$uOJMjf|kCSzNT@Kb_^DEJy8-HM~V{Vew18rze zP=bDF)hDL?hI3`2_L~PbXfdGeT0p^_%+~YUsE0FpYD=bqxH=7RYYspa%7qI76BhwG zNSWBBfVIm2qZu4l?dDmD7CwqcKf8gtj>Os-W3-`h=ND(?n9@LUSdcn1V!V zPA1ZlNXaQgcB6h#RrhJAs_1lxai7}nvhf?72;+rGpk%jt8T5w(ECK&4t~1?Fk9VR) zRmVe;It8$sqM9}nV(}eO$&{ZblTMxa2zGl`ZkBo*;`37Ep%ghc0qo+Y{`6}$K1Z9@ z8*oGh@@59IUjAYL`fgNRc0Wt&gMHQjaJeB_&Q+?R@3vwLb!6!6W-MuWL#UUPgx>kr zFkDUlBHaBQ~cm1EA$wGWw8V_wFx?)4FgLXkDv6`wY_orv@1< zTA9yGn~V-oG!MR>Hr{PnpvW3bxRVkY*t$sI>-3L{6^PQ;C>tiG^uzma_7@H(ie zBVk!?*H{*>ENGafXXFiS^@rg-Hj7Ceo}Uj?k^=a z3s*g7R}w~DWDi*?rS)dX^JFSpu>@V+ zS_yXhDzN)UtVCCRMk2;PyNDK}FsM9Zp=d75y@a`$v8BJ8o@xx!YG5jwr&yRBW-b%B zzIyl$^m;o6+=&qoj%>bgZbN5w~h?H+OrY)qXqV7K7_v|V9<@fe8yuJ}dz z@oW{=Q`#)CkbGE(Y)ADyUU$eD;{K4F`vK(i#y>jt*lwG?B34hYSQGT{o}ib^g3a9~ zwGh37*iazVTGtb@d;Xp96D2Q>VsH4U3*>1sD;)&o(2XA3Ph@502;&u5uAXEW7U z_f)-xf}f0jw0jIs?S%~T&2*i415nz>>283-J?rmxaxkWG=QB6aPUOy;|G0^xm*`5CrxDbdqt_pbx-K zX2R-20UpDU@hGxS8IFuT2%#lYsot=ECz#np${}Ob=3swm0mv3!MLJXDB}MpK04A+{ zcThoa)}OWM$2fmiYA{7=$fT904B!GzTze~nwOMt_G4W~FLZqtN6v-(!f+8mi$cbZ7 zZ)n^L0S@>BZpr{YRt%z)NmCgu*TkPxo(G8k2=Wi(G(15`JJKa|Kr1(x%OUO0!Sk@A zMPIoi&V&;{uf7_*u=QaiZ@>`pwo9ys9)*jvdL!ETpcDMCNX=0$fO zcnvPY;_i^MMS63wxWWDj6guAaAz82cJ}0Y6f3N|Vk3OSr4y;zT{0R0CxR)0hemRDBRYa zdFQ7K>HhANM<6Xtij88;qR08?qx1I3s9W<3pd*V2Ej&nwF%jz-=O1Mu?p+Sl8(Ew* zr~()rO4)qH%-Ep&pZ;i}(@zk=N&p7^-WGyD2TIF$e9rm49{##Coj&gk8F4v)5+BIz zLGBI~8TyB_6_LSsj*J7>0G3;Tq4c~qBpYz>Az<$l_}4#F4X+)3n8F;;Xg=akcN`

Xd3j^bSCV$i2)?}sniJx5$IWVsF)UbSouHP3J&*#}wVJHb zV|p+O3QQxqa{|{p+UtpA9i2q_T+hK`ja^2-tCSd)?EvbdtChE|?2s^s;^JYZGuUP4 zt%XmZ6&oWW$&hJT(p+Wz=lQw3|OUgAHI~Gg#M-2e8$z( zic6vSUCbl15+$9yR#CEDU>ex8B4A@NBz>dJX0Wc^w}E{m6{rPwa$I^185(*c1IKfE zqw)KUX5|Pmzn|%BsiKt(sKZovTn}bffjm-RA__t}9)+R(i3=Z-8!M8eN(Dw9Tv}H3#WJgLL8d1J~%n@`p&5 zNeA2%R_!P!n$n!kAg(Oi^Im}GO)2Vn6>t$F(?9RL!~XfSbl|e|hqPh-mK?W1#hh zLRMRz9ckm#C36@u?r5R?5U^oG0efS}+TJQ3q$Q$Bcb1$VPDoz_zRxTbAw#OB#IEa5QLCrp={%=>ppck z*HD2ZINh=$In+|NQqnyIwpB;eXh$Y>ZngboP6QcsHZInG2yyvNQimNT`M-QjwYTZWt)l(uuITttS8cq$PGC( zdhsh-S-S(!VkeX`ff(5NYmG2>tNcr0yA5o{enibA(%=Bu6AqFsvdJmanoK%mbQV0e z%cycY0Z7BAYQHjuq)FPt>y%k{gVcUfo}Az*asX0Qj`y1oag&3)-Oq)?D?kmVdp+5v zSGN?cf+K~Z|8D1JCn_>KLvh~)U@I1CR|pWt^!3+G7yC)M#@1TaCc-JDw!03IJ48y# zNs4&Q(S$@0dBd1z<>ubn%QzY>cRl#`5!~spakl zO#peasGLH+LvK_h-5}ae?E|lB3&In3MrcaL70=a?K_!2s6NEDkMWvjw1Zo5P$QwY z(P{Dz2dFMckKPf{p5_JgXYHaj2mpL34|rGsP^cm$>rg7RGLuxr$9-|;kEL-wPstFS z4;`!$hB1b_2qeuSX;1;Q)iXZ)eVvVf~Hf zwC_Q4EXC8AK;M$t!*G#Wp5@{gS?_Km%gpT>e9}aPYn=dH-T~Hqf$dkD7qG2sBwf}P zkZ2Eh#Zc0MT>#Op(0Dw}F$U}2?__1#=RMdKAHXUnzT=zcGkW!2j(-|WfkQTx&ysgH zQwvw1-rq0Og}mY=$SYiiMhzQ)!1y|S*t?NPciC$ORPD#Efa;K#b6Tu$P6GU#3wX7H zLiK8{WR20M@iU0@2oMV^KId*H=R?d9b%E}7nvJ3y2`dc-76X*)>tNY@ z_e27@6|=fO=gZKDj}T|nQm!Y~k`SktUk;9H;CHm8w*ix7gxEi%XdeiYk-AUA3}d>s zrxEz&O_9XS^%U)CNRHjL)3m4m%TEYdMx4@Z?AS4`tKCr_lYV$f0qF!qPV`^cfUg%7 zbc{zjg%?J!5b!P*ahy}|)j!%nRNJ*ZM4`E1zPi}}Sh|#*15KM#?kChB{bq?a&`k5? zC|SW!YOwCm2J_PU3_cG#4;i3u69Ls{0g5C87S01a zTmX0@=Jl23iRL7hJ?1au3P&K_~+GEp=M$$q9N8s93cO zuzZWMz|JT81G{-g1N-$K2%MkEDAXSWvq{r0P2UW>d20)>MCon7A(LJKy{^3m9{xot zOBf}Ub*%aaLXNMFd=7M|zT}sWTw@Bqqh%MWsg;a*aCJz>8oKh6Cn&UHgU@Rz{1Wtpy(HiIgRtF%5i;lPDk%yyt!Fv(dj z`rbi#pd0*(v;0(QP{Rqq^aoQLcV@kTsx%ARO3=6XU(*JOvxzQ)E<4WHhFaHQqSpCy z^crY3PT7uKm7Ga--x|BwE{sEe>Gk%b^fA3Dcy1`*3`O*J`wl{>V8Xgv8!msf!m^1bcQ>^9()b$pb~d@$~5>0 z{!>{*Gw%Kl?W$|!EdIt3uzrGPdOWVGQ1)!6d-}0IIbYD<+pWS{?&wr!l(u2E>at^nk{LY-?*kW?p zdDZ0D}xq`&s1f7+?4r@l#VM5Lb3J1+Kni^gGroZg*WZJMPn6Zy@fS z?CNk$eF4!xLuUjH!xdVca6>vkw?8}ueR^bUM4Eyt$ZL3&WBNiiF!-3YRikX2++LUjU@k36iB+i=ab7QeS2OrimSMWI-% zbw?qK>Uy==hjaoli0~AIa~-^a%!6zLWDRJ&X!}BjV_p1?cm_X0Sweq;tZZ3xSeMQF z!`My8Iz=$@0_Ofv;$8glN zA%G@~V`VypmQ$$9Ff_S-6d3%d-aT}F?a9_Rm)mNhm+|4?kwhz7kILH+DcuV6OBvp` zdeJ|F>=tv3y@2s2e$IMQqu~j)cCwH?wHJ!9;mcUd+c@QXia=XUS$kk9I*b% zI~BXxid|WcI|p;9rGKfn74salkhxEP{7J664}GRgS=LXoDb~juv&dhX`z$%X)Bjqg z2Q@{GyoF2i7(?29Ea|IW_eft6i@21Jbdv95(uswMetGzmF>zi5ad->}lJ)9wMvG>6 zNgc;%%5@r>@rI_f_z2qg2{daR>n>#^pS#G;^O;EpyU<*}qa6%JGyNd4kVx83no&l5 z#=-MA1Lb$i@RJFsxdz2dB5O=2vZpI~J7jjBhAmr9Q@Xf=pNqU&X_*vSGAURu;d{w^ z&_w2gnC5^kEx->$5Sf(p4#-)8PPRUUDmIV0n@)XUJTINB6-uVynK|O};$GigaZmr& z>nnN->jRfEeTCm33QBXcaaWs};th^`E zs~oh;D?pp4pC7bM`V+MaEkS=segm)IRmaK+fV!@6fe#W~#do-gz zT#JL;Gl)J(iSozPoo-V!EHngx4j_>JXhdzeJX-byxA@M};hq|y&(4%X><7QmXW7M_ zvU$UCxQGjwRt#b5OYlD!1RZ@DB-+Hn5Ctyrxz93mpuapDSr>@F3DTx?T!G^dOUwhF zW%-3r)oV3{T8E24&T;^hbOhXV23&Um_sM|w#yGZ}Uhbiqq_vp=wjZV_UTV}VHbzIMwc_#Nxd7uM*3osE%KOhP;)fELD6EGL ze(mT07-@OCpqC|MM)6SpDPvZ183)Azc%>6q23b8g*D@T@O(lEbbh0;6E=_Jv9TGvE zO#*yf1Q>ownwTqr{pSP|(Y0ll(Aw4Ih<;nJ0UaOizK@>zyj@;*4XAVv*nIdSWScOa z(gP+y_i7(5Lf;kpnZ7ttBmZNcK<$+U{Fj=(usVo&>USsaf>q^5GHFh|0TZqBO@!Kl ztj7_|Oh})VbZmZE#fbtx0=>@>d;>J3Oq(Rye2WCBH+`)cw`fl8M7z|x&y>0hx;ahAe!+yU_n*yGiFSG(Ozx8LauNO0rvFoL+hp~(d>P=` za=;Bl0sm$Y$)RSAt~nvFYEt&`}PiFt^goU2bn@&-ijIIJSRrUJgxx_GZXj?87T z6f8UeP)>B!1_jA5(Q2+!2ISWw9n{6^~Wd4L#GKI zvH>lfC0=Jg+LH0FuUdaxrn+BYdu1n}_bxJI-a+;J;3g68eMDr!ZgN;eNr&iY2H?<9 zfc3pilQYyYnf=Ew?UoB2DR!wprrI!5Bf|V!D{Qi(#Qp;Ogw*%K21fA_Vk8+=7qenoIkZZ6{Y+H zJw>g6isP^n>%wog4J?D`GGz@sin8;voz++o zUynmb>j|5aGg#Sc<&s&p0JE%hgz-tH9NOw=mqWhlk8a7Nj)R}TaJ70m(atAqb+3LtZ9njhJ8Bd5m zbSg}T&g#-kx^-j}x&39;Fc@*QnW#qV#%!lW+yP9v3)qTO+CZ_CM_7v9V|6!{k2}_o zUcMGkM=UzmCV*i(;1{uNf7+zk9R$RP6%Q9HE+Hm$H*p}BKWC-eN-hPg6yH>Yp=N!@ zx^xAKiWbhjjsPN$f=|S|uB3I%j@o7Xa>8;dXn)aQj$mmw9X@c{FG>{ZXk5 z*#;b}UM}yioN3gV(fGMftc)w`>V62H%#hZJZ`@24^7}||+i2a7@QnmBzfq}MeSA$m z#}dTaN+vT{7xhx=6ggkq(FKMR|N--(UuE_ef z=nlChmcf&y%|A_)^{K2k|FW6oOeQNxoFLqU+9rhxabnhozpp`TmCKHoL=^E&>jPiIyxFV1nK2?c4LThf&_SeE{ESZj>TGL=HI z(w)HAIGM*A0TI1ed0M#(+*9C(j}Cy zr0LNoWt|xLn6ol{xk-+8f8v+409pMHX7&Hq342NLPK~_*CLijSg+j9;^t^j@P;rRk zxW&OGM@`iiZ!k+OHMzIfFYb|dt!0^Zt?Q+^;`^EYttUJ7`|*i?ZyU9yhe?RpvC_3( zt#IpPH4|*edYflVXSbP`1SV%)xM~?#MrWibd;}>T%P5ll?7kDE@-bCdxyjvC%vrRP zdKoGAT>JZP;rg|si$pf$kLS#L}?Zp9=bgL<=WXj+;KOEsvyGW*g6d9sKW zke|GSuFPN{XVAs>Q%`A+@k1ZEhB{ znZuw9OLMZ@)O_-OU0LJzay6k$9dSvxdGk{%G^sTqaxB|2_Tu1j;45L0Vc&n>%pst)ePx*?>ESmEf zn6yW$b#9{Vk8ra!WOIyJ!}5?)XKZqP#+gjRDQY_yxcbb=sWi)tc3a<76lt`TiwX`Q z9UrG0{GwWv&%6bZ*~lA(Sr=!eSHZRx{D-Fz*@n5W#<2R|#_JhwQ6ywPapdoepHMd3 z;1oo1GSPsIn3apHD++^=W0$XB4{hS`r~hWK2T0Y}L4A3cKIJ;ux-r?dOG=50_)b0x z5tCAZi1U6TK1AjophW(|2F%RB^}c?_8L9B}jnAh#?84v4Sc_YM_SDRHPa)Fnwf?1rZ| ztc-}JIm@agwMzz1-}i%CCKuy>(sq5pY0iNDx((ARaUY~*Ri&^Sjzs%x0dxriJZu0j zIeu8xlyVzeQ*K7c85T7+LdiW^lk|(iXX(5uvZ|PzNqUU`b<)c!kp5W_@FN&x?zATy zIHkUPiG{3r4K^>U7j4F$WDQMyha7HcL?7PxjvUjn$(nK>u#lE%Q@;bg{Up1+@&kXj z&W7|!nN`~b0<2GV$`mCwQyPD~6kwaoO3utmT1zU|w$^7jJiMoW?Q}w}j=?^FTi5|- z{v_K1F{A(Meqb}ftH3vFF-!;jFqn5Y3K)GwCp5BEaSID64?SR=r`2sn6FAA$%|YmgpF z2kK>PN5Z1ybXeS_pilfVlHX>Kv*GY6c+?Yvn(%~l{me8ZJbY|3qqvf?6*&vaeA>$$ z5a|Jd--?~|(Snsk*QUvJMvL`CtL(+i>A#Kes8h1)lTamO-0GZ#Z@tTH#wr zkRJVDv~YHu$lzalr@(id&=ytJDtR0!+F$BwO0S z-ju$pLf`2f4*USSKHr~U3l%_wzSw%%K*r)cA`UDZL@3AAbqfFR@zQQHspz|ND#5q} zBTo+*UL6IDGFT`*1GCht8HR&sYZwp0(F8jVr1cHOr=a+Lv_`j8m@fF;bDs0bj;yo; za+DGs4DbY;%LDks3(ztzW$!_DN3?M@KPlO}T=XTiHvoc>Oq1HyV&G-b=~@E(Rx$O@ zGJHQ`0(xr5UCvIkLrJge1qc`j@_e61^kbF5M3;uj#I%?ZtSzcV`b>?tNO?y_{;!dw zXNP=o2EV{{>X$%Pt=CX+Z?y>N*Ja95%W5%|YQCHZY7vED)5s4{syM)<9AIV@vRC=m zNA{JHJpdJIQ^lr6-2^nfg+}FqLQdmG5N4p=YI6A0l?12Sn)@AQfyeiuH9vqh4Dj+_4;$$VPtl zMSMuHm6;fKv8J5zTfeM(lQ~q6uFDUBI97Vt`e(=jt^Ol8)HSrhD8 z%Z~@hbVwX@GwmVT9LONj3?%@@6@cMe?fJ`ZofDLoC8jI;MC za~hFm?Gl%NQ2gYa-vDt>F_Gymkt)AEl1!B)Tp40a_10ev;95voN?1k4TQ__DW^QiU zdz{o1?xy71@0uI8z1AxwMwazDi3weHRhnNOHgFwD3?`rg>nVbZ7hNZlJR6(;_;rj~W7XmR=2l+|XyY2$DQTK7T-z#h!HB;Cd534$z5Z<-OWvo9o* zurTX!hKoOVHvwUde9BM(`2E2{6H+7(dZ1Bbvn$ZQ)9Y9T(uiwHKtv|5Ogx%ThlUnfG*t^ zVAr3#=HN5sYQt*`sA^*=zQ6K94yLL%Al)F9EDwN(CRNAm4IquluSEyrr&Naw8n#@ceqn1zs%Zfn{V)8jAX~ zC`Pe<-)=V};+A;Im(o>Te*u~3Bvhue%9m9 z^CGr#?;FN*W-XNN(fhaL-d5Y+imbJ)zcicnAlKAhN@L&EA)QkfkY@nk z>rlYOVSqIllXcSX8IZ>HuN;l6XZNIiLZ@1?aM_J)TIG%aSGMe0Au)tj7=qr~VKdWg zF2}zbr`?qEzr$!zd*lb`$-JZYQ+G1U)-aQfVWQUSeBw(mTJ6siSw~H8zSEOE_5IjF z)49y@`lK>9fQL59mfZn67K_y<@H;|Y+PqaTu`Y^?w~2KVokyT^ZPR`H#Eo1=n-E36 zJx8Zx(Rozm+uIpmb&*;I^g*yfVxy;}?L*?pT{zrCM^zF>bS;f^!Ljbt(Q6#(=COcn z;{k)QJ^f4{Gx|36LzoBt@O%eynl+H)>l88SdNS`!D3!1Nzy~L zdaUiTXj3bh9H!tnbjQ3A^lv|5QBy2@Cw%m7f3^j=?I+QOw>lx^713FvkE9p7c1B8< z+(dtN>W;zLyy6VSIsuK^JrUBISgUqyFO|#{%WANVq-i^#^$Wn>mw;KX0EV})DEwwR zSI(OzEk^AN6F^)`UJ4KYIS|Y^3OFVMr2J{X@=Jgtm!UDntfE;>N0659sA?U?0*cC9 z5wVhN&n{_fL1YT;(v?=u?@31LaKQZ-Ks8F}!wl=GHf1BDUw_`{IF(O~AaB0-mQew+ z%h* zX)263nI(@fA=Z}}v8TrHQWj~rgkIh8pGe~9G3s10mP)@Vq#rxeV|p@Lh;({_h{7If zVb2k0phYBIk(+74O*G`rj$sVbk0J=6BJVWgc?&I~jk>NFd4$+@TzAkSN<^?Okp}cy z+^4Q4o;*OWSvvKQ<HMKF?vKo7Z=7H8^>jp{+V(zI3vVDT|jvF@bGdb%6}FBEu^&_ZBG{iA1W2*hFpcQG-)>K3dCFy!zZZ752}b1l0J$(eEB&O+s6F`2Lrd{ zXoX$5*g=C?OvLvZ6HQCzVR)@HhRtLUT#!fLM;*!&J(XDI<6aEk=)>^ei}+y%dG!*8 z)v4jVllV#od1ng4K5r2m#0eNJkM9O{x|bil4>PFim1OKI@fTZciD&!drgNB3=_Uei zs#v&Ut7!6xP54y}cM_(fo~zzX#dKqZ7@LGFOlzcXN0xbX5~ot6B2tk0Q4g&)fVvNK zx)kG|MNs#FR#I7>m8x!z#URq(n>yRR=7ZkN#?;yNCKJp_74#o?gPVM?10(pgKV$cr zoiLB~76kvHHl{tJTV#}Ex8f3I^hzQdxDj2>gXp0dNi;Ov3{oiBZ?TjrODyH*28`Zf zi=g=~reXZ9f(^ZEfj0!ni&Sg`b>v$dF-l>h0XBzvOiNy-c6-BZYz?Yn(&;{yBZwSE zhPEImqtf!mG=fAVD2-Fwl+lHHoJ?A;!*rSYh*MLy!lF#6PH%f0pCazA0oS1{gGQ5N~n{T zm?pd~O6tADxPn&*LI*H`!nIRaKm)Y{?F$I#7O@x9MUV&ei&mz#3&_s?R6#PN3R23+ za%{uNix^g*vWy9}Z|T&&y?BGs%<0!KxihubjICG>b-MqKx-;9~fv7{&R@G1iet6Gq zWR4?27#7%zU}Fd(9l6oOk*@`)<@i%KP@H#R)cODncbZYZH;?tl@OCQlE7c^e&O#te z72K0&FpaLo6soH?mgp@LeM`+QhAsGvE%3w(5Y#UwNG|GnPeLAZgrT7zFRP=LIrW>J z$xDZ7WKj+O!OR!?d5|j-8F-OajU-~PFd`6R!Kl_3ve-gyq#ez?N*Sxjtj4GhwFqkJ z5U^84pyerow_OO{3}Zq!rd8P0D?I3BEm1>bOP!pZ!2LEdr5w|k)gV|+E#WFPbKreU z*PW}4S=Qgca4Hq~1{sm8qEgVY7o(fJ^pMDNd(dmN*%4d8MdfyTstK({B$19>M|y zq%eF4)d;Qj?=d7xpBU~ukg?*3J+&3V@HPbXE*L+s7>(%leO2gxynz>P{>=VKjMD;e>>d2J~Qc~&g)IgZlu3CeslrWth~!rbeqYMdhk z!7P<~HK@dwpw@5-wLWcLX#`E>_0YFiFMk*!=r7KrNvh?f+N>LB)0Q7fP*2jR3{@|J zD^mth#401k0jQgr;XqwmDTw1zMnM8&sXVtteK8_l4=ZvtMAThG&6^;q#~M-HHV8`G z5U~0o$j!x?-^wj8wv$a%E`78ffdG}n)GtX07tCu#-EYjMvYo|ML`@*-2|mU~4qii) z2UUtMHXsm2J|eq6A=vc=%hcBYg3l`i8(_F24uN1iB3E0XR-`UQXuL(#`w$bQi1cEj z?xgE7W1?273&v!?Z!86udt@5UWvOdF(ujI7jY=_G+tZZT7;yE~)4_#hn?J^4-gilj4^iiEO7z$Oswq;$dk+8SnsxcUiXMq9u0z#)JD9LT)IE? znBX3Y1SwAC@_wqZ&i6*p){cOfLU8X5HuhS(o17wzGu|DCPzI}^ZU{J1-wkQ|jP9S!8ssuBVDBQGwyf7TMGC8WvyFrzv+v*5XO)<*i=ULP;1MEc@rMLpI zJk(y{&Ic`UrVM+))s7L|*bN%r93EBCvN`$?@va_I>RdNnRy@ZGM)+s7 zxE7Q_S|(N6s_dz!hRmo* zsfXyK>NZqgoF%y1Lp?!ZS7ZxirweBPmyomq*{mQ5Gh7c(#S9KqCZS%)plRv6#I?>= zq)PF?LY>%qq`A{#1zy8F@q)Q!+pdboA79s`bmMtk|2cqpdS z_2=@}xw_Cr&A0seK8rq!3-vLP<$HKfF}J7zGPeHzMosPirZI~;@oj7;2& zDPLep>Xb0w0aNBs?dvTc1bdGm5JTxCk5Kg-7phTG3l#8%L1%8UgDZHebwOK+cLxqsr=4GX(wS z2!2s}lZ2|6W}rWa%h6gaHOT-QTr3w4 z<7hfkHN!_Lm-17YLlNUpXvmi*Fz!txq6Miup`C`ng*qc#lZB~JeUMDlyJOdpO-TAP zh(fY1C%@ZS%y^SJ@%o1fjEt#7pxul>wH<*&CxThDzhrbTg7Q8pO>}GJ0>~eF7-rMQ zl3cOyEGkQkQFVQ%5rR|`1g)kBrm03Zy%0e*RXwNM zk!-Yp2sAUaQ(d_KOZsEiY3kIE8Ob$Y(5rZiI`PG?M(|6R`?yS|X>9DmWUQ!Ps0YZj zS%n2WE4b_`%S$&-W>HP^mSUZ^cSNWe{tsgn&sIK?zkn+fh7en#6XL%xNUtUOF6x4n~Y9 zq9JHuL0~D0;2u((wqA<51^#(<4-%V^E(nWsq{ldW28^428G@9fUY(=G2;)r{nr>YV zCQD(VKD$G`2T5CLgZF6JZ|$+-ld@X=Ib}o;R`#Ac>DCQFKz*Qj`9r0&p2;X&r~O8q zLPk;Tb_GuymQ6h%r6~v`qexSg_;W$bEg^2)HOYL`YahS3y{GMF$R461MxsK$$G zmxf^;6Om^4E;F%t^z|~(jmPTDj(D3^~(jKQhq#};Sa%HZ^hndunN0#4z^_Rd7_QeJr_nLnsG}Jw@Fp4ClmL!*2?ZiRs^Au6NX|lGO3P z$6AQUCCo=Qe#OlFm?iETX8H09QOv&)?83HErzNz0v9q`xY%PH}jg<&K$|0y%M_{Xk zNzP(Ns8(A}8wbM{13Td=P)%aA9-c&zXuqg~NOB&*8hUClgyf^GT3N)HxBO;riXld$MS_(8AUJ& z2NPzAMro(%z0^iA`#6bVr6~-rEjPhFI(&A*1`G+iB6N~V!s=E~2`P!iX#&nSmh+jY zC(@3IYO1J0fHOwgycPAiFwz=}d$1Y7_Z zFJJj}`XP!mf^DM7{F?NW+O zyyJ>kI;@M_5Q$M|mZ;yIY@qT8bu7h1@IzAC@5~NNxi=dVG38+lN;P>T9~-ii@(Jl8 zlZrC|EqZPNyi~UVHQe$UWDK^a64R~3wa@Zv{6o}B1!t*L=U#zeBa)unPZg>*L9F-k zPF_5-W2I8%8|F%<_Nxfh)A9$3E>qRtzP*^%gsT38LlHd5MX-Rap?T2@qWCzoVq)q% zLxl|!dr`IBZEP3yq5|!Zv=C}ZJsAuh#1!YLr%flSW756;{G3^OVix)uA2%y=g&BVyz3y5)Mcw=q3LivfY zG3GcFmKKci7BEA!0o77nv7#2X2|*fi6LrZ!OE{s= z7_j{0nYuvL!sqO1ZlY#5dBkC&{m|WtmUiSDcBAJFb=@bsfO$F52$T1znnd#}ma-qm z^FB~DrP>c1({-;`BdsKpbDXGE&b2`sn~=m zaiYC^^^>|C8)o2!Y2-Z+{G^WeN9uU{;&_uAk08iE86n%K24(r?{5(~aHKSIg$*EPa zj1vdYm%OSPbqIAO5vGE*bE#r=vRK19$Oq~GXs<>UAg+NiXhF#l>WO=ZJ{FZ~hCqqh z%3stmT>jLZa4h<6qd=3Qr!d2D%)pn8WUoQb(qkmW9noueiGx6XrX5PZjdj1o7(o*M6kh@Ei=jpsIn>yrj7W%o+9plcg%kuf)&m(%23nrk4k+o8d;y^bRMN z*HYvq@rr9s7G@@pmh4q2PEvTWe3*)`qY{^Dl6O*xTYw{5jRB?-_Sh@MH5!2=f!scGLW- zaQk+-4O+K=C7=3<4ZmEWAWJt+Te?gg{fb8g*xHvk`s|g_hvo29j(suL5O4GqnTqB# zMV-AHQsq1aY4rh{$~TX~=Zy{Ne@0#J{zg<>KQeS7>lhM$X*(%T7eK0bV-8%;E&mM1 zBM@^0W8ezr+Vc7^vdQ^%xR*$~S3~{m@AqlsE%tzq918fEdwul@tWe<&X5WN4-eHar z*H|2THIq0@yDuRfyYs)T6!qc@yXcUzBL49cRU+@tVh?kuD(WItMX`OsLM*WmSEyJ) z(^)-lqDWnRZuy@s{`b~kIvxLXI^F-{zi$cQ-IV_sjlTc=CrSc@;{Qi1{ufvEhQ%tG z|NUoNDZjA`t2()MI(BeyZVwK0n#om>r>NAP;m+cmNn&?#~;mNoJf({E{H zb*EFwV61Nx7^nAWWQ(9v<6>%U6q01{Yh*u5rVs4$74FrpHv}_)kUCiSW zvoVTgZM6I=@@z40c+6&k)ifHLQ5waUP45)UYHy-!&v+$@zn*^EJgcV(w&_}wKtH`( zFk7(6>K?|sQG$!~JLlP=Ot671Q9^tSKEdo6CTs1P9!CjlGVB^>KS!sT#MI|0VjsKP zbH!CU?RKU?Ptjn8eLXAgM6a7?8Y9Hw7(ygD+URr@nP{^z{Vi{p?GTj@6j zu&kbtYmCd2 zlWlL4Y6IgfF2U($_dK?+3#(tnBPADNcD!PXxUl92#yFwqZU{@5el&~ljIoG>v+WnQqC=!`w0 z!}&(aup;qmjl>IvV*#8WCkz`Ce@L#JWjJ2J`E9~zJaMUbz9}bZb`B= zN|KAQ*;ouW@g(_LV?4Q5G@IxoG1Ey&$HnDw@s*gmB(X(Ht&K}~#U)l^7M{dOCv6&+ zB*ZOSVxE`88!hb^chZGhrNrVEP4Kg{r+I1yw?>R*KTR}5W*?c}&aLOYVLl0m`2?9k z<1zKN;!2kGl1(SGtaxm^t<{oo%+Hb8Nj#3;HulNNUUFB-GwnRvO*aN5t2W8qG0%O& zv!i)qMDiLs`DSblueY~tD$802`6sRUI=lhWb`{B5P4a_rh2gxx(wo|obxjoB#TDJ+ z4X58c&0<)q@WcH4JZ~gD-k~y4SEN~7P~%IacaUYZs8tlQC=KR2Mek_NYOSuMXmPQI zFPGlQm({LTNynmmp6@)pa~!LKy0Vo;r5b-Z{gyJ;Ew#!n7FUA#|D)g9%Id1VD!`(; zhW{r0wrN(6+Ew8e|IPD1pm!nJywp{aENayR+9q8jQ+zL|oMF5cAka1GYMc@vrFx0+ zdWFFAlI<=jd!kkA;%~eX7%p)OPuWkmx;g%qkl=KQdtSq_y<1(2^mjXYP5+OYeJI_PEx#N+2b#$ z2{5$;2ysk#8K)*msmn7xtPm2I^7c$ky`Zkm^yrO{#FS4|YKGKWOQsG*VfiWF;?(TS zwa#tG!>TQQ4eWW`8oq5$iiNdW{D;}kW@?1Bb&U%fw(MGBFXq;aYwM8|v1kbpTTzy& zncar!#I7aKU_}MDR#{trvB;K|-R>)@GPUa4UW|))lbC(yg0J7|s(7E8X`h?QZnCg@n;pq7kM0f22L2)1gls7ZQsr4Himo ziPquTfD(}s8=}ea=(CRGhUpBk)3Kq>9FIeES2NGHixv1B2u|;s&^2OyLlZCY3Cl?D z^U~YI{LV`Jvd_Vq^cNF)JDBH^#3So`4)vrDdg#AJHOSlWC0L@H;ix(1TXn;V4JcAC8IF0Ln!8|lgZcX#iP5QJ zQK#mmjM|ugDz2QJia5);m}%7i=$Gfp4=u+VIDc@jpMHeG_O<22Fy~U{`o%|o$5;Mo ziCp3&c#O$*LQ0acHA;+&%rfR{N6opSHQIoS#$zJePF|4YZ;f&1V$U+sY-gH~6mN}< z;Ns#jHE*X$Ny)Xwo#o=oGIefeb(K;ni@%j2NN?ty#GWsuS(flBLo~)LG>KzC%AhRq zN5)Ee^Oz(qVQKTSB%w^181t+o9#?7GvSgi11$v8;B))uU=dzP7nX6(f>XHNoq}|I> z!ZX*k3D zcs4Is8(S-K%az7uXY)9P+SpqwJIGy$%ccv2>90gxD}@2&Svu-2;J0Ut>>WdKCY;q_wd4|achH2g)edE`gxCqZl1R`>Q(q1 zcYcvKa$z%JW6Z6{5MRv47aMBNWn=1~$QFM=lP__>Ued-qQ;|2m#F;N8)IrV0vR6?w zzBG*Q)PjStjTN_&Ong~3-|0|Cdm9@ErB(44>-lmQ96fDpGnF*rFZJ^khB^h?Z0c3g zk1t>3JHOx*Wn<5xZ0c9Rl~dyFoX+ZKt!(30sg`qT()k>#bF#9d-(~xp%ideAvTp5A z-tKoLIOpo*miw$OEUUczsxorwytj6-Zns{w+poGN=f>pLQC9clRiS=Ydvfl0Z~Mr) zqhr-kzyIcQ?oV#}%j(IZ6606Hb*9QW9*w$tqcXb@pdEyOB|BcQ9SdCZapqoQ z?hommvuwvJ*L<6~KPZ=kS)(Z)K@%FS$%~W6O-`0~S;^VWAJ-tI+&;QYUp2Rbsh$pA$)|yzh zvkOZ*`KCU}G+AqH*`XpV|IGL76Pyw_Sa#Y9D?9mBJ;`ZW>uUKpNO<)#zlJBdbQ)fk zPf~@~I{80&f|G?n%coVs+Ryw4Qwxu11j~1|=j&?hdc$5+sd2EaYdBv&cGtY^xn7N9 z3Ehjr>%Z(G?25THV-tG#M9dBZaM_hOXr?6eYKm<55+G?;mZ^E#zt6tF#wT#~ic1}u z1(wf(L^eGOGUS+D&+b}?b~n?YxoDuq-U_?x zwc0P4hI|WAq1QjXF`+%uHWVQm%)75S?UwMm=}W_Tg$H8y^`+f$T{m}W}gCaiO!ZvB5m)&8|A^G92(u ze}c43ob4AYJa8a*Q&+F9e&XvzvGZRJL~ZKf)-y|d!zW&PAS|7u&sxuB!&?>cOV7dz z(w`mCvsZX$Ra9wq@N)X|+j=ev?*qiEuN}O#>E&C!9k1U<71hWbYU3DC()WKoS6ozI zdgukmP@w+4*B=^-Zpj>;EUl2V{i3Oy#7R-Ym^CR;+#-2NO(Oj zd9JlIT!3>b&>;QwXXA72GDqY&XDSWOyk77;_oVcQHs|ZN2IpRXi8|LKbJUXat&-uT z*Nepx{ilyQcfa#8tbX*h;T)PF?^AR6hSzp|>ya3AIu>;5L#^RW=I_%IBiD|FZ~pYz z@B#A=a^-|ZMAGKZ>PBs^e@LF6E{!-A6L^KWF1U*h~)Y4qatFVFLHrN?h{ zf15BGdGtGCA)(*H4!(O7GpZb;GznmD~jCSlx;@|$2zYj#t zZ~oi6{&(WP#g+fQL=yG{k1<0M!7s@W7R6;x?`X`HM5iUmyci{EPi7hOCed$^WDAQ{ z<6`PG771VoDdw1tHsYeOnn(wbnZ-Q5F}6J{rY1^OjMb6?{W0!6Y%wP4R!mPNMVw=U zdT>6fYsLIQQldXLqKA{-)WnL$azV;G?qmkHo~e~0i_!(OiUk?GM@{V&S*Dc+eX1s3%c0?(!0!cu}AR`*%lO%nVny>l@o%0Z3QvN=GAyVTn` zHMLjW)Uv}$SXIWqKlN-c+T~s=5$n=G^R&yET6Zj8DT)S`?rlxGlezAf<+!3)a%qTp zdS|At-i9}d;+3TbThj+K^^b0tSCr@|Jz{?9U8dok4c}=iXJn4opZejjeqqBunk1ua zv}Oj)(PXU^GmDg*Y15BrlO-oos}SqjGt^;eWs4%h6SsYZ)Kv( zGWfF0`>dp`kROpfn|(U0Z_}i;VTZyW z*JhW2XQfyHf1)r9R(+O}x8p>@Ejt)n0lrU9f{U=%ig;pmy)2LEplCy!$%s z?gXy;AV1n#c=YuCUv`a3IxOWA=4Vfy4$<4x&Y~-)FjIdv&*?zurmnZTCgs%eKOI)E zslQUsRpEVDQH|4~zD+~P`Ue$0EfzI8h4XBlN;b$*_!4%mi}$F_=68XHwdLQY&yDg% zoZP$+Xw+N&%lG_8-V>dhe+91pT>f|Z{9oQER(ty0#@rS3e#IPoF*f$BHYTcyOwWtO z`eS44xou4C6j`@iQ0b3vwimWB+pEa&{DMJ0E`X$L%+nNkwv^cRC!0Db*;rI73Op}y z?@x(wP`9ysswlFhG^jtV*+JK4!v{r)=cN(-xK1*$v09-dy`?O*KU2@qO4(YeQtn|{ zLH6mRj##UOl9JZN3g?{5j$7Z^cvY%gym*5z_nqU8cN?QC)#NU<@#QN!`M#YCgX->0H$5bX?h<|FZe>3h6BuJDrP^ zu_?~aapF|nx#ga+3;z}0=T#+Ju6J%7QFhO{680QtQTIBx{ZaPhuZnwKBfa%ur;GS1 z-<+!K=XKs&pLV($t_t9X*a+vYI$()9QZHt$83|u*O@V}4G?=^4xp}h0$mE#Zp`<3%x zVcS1ePezq!ts2_arnN52Y+h1TaTjX@1X^~xta$8YsFK8AEBm6g#D)K{w}(nga=+Gj3OqUKx^|C$Zxznn z_6c-#y6WxO^||WY%(W8&J*?YJ_XKcPm-yAE3-sA+x7ib@UVUk%{+vMn$?cANb~{vG z_Pc&n;6>+l*NuDjs$PA5z2U`6dN;3)L20UWTW&mgF<|NzxN2`@_4S802G0x~bqij# zucI2Ld+*MS+;IzEwf}APy@xk{of-S(7PTsvCfnhe)g#yMhHv|{;PHShoN>*BRzuRZ z&ue$Iy$=^(Go90r{c^#3$J5704A;EjzgPBhv1G^d$45QZ%;nsxfB9{4$H?Pj$JfmB z-*0`1)5@905k+ejbME)Q{N=s#!{g)k*8JdqF#U3=Warn%Cq~vRXpU07Z z)(`@XE{O=K&T3$A&sV^5q> z6Tj4QNQm>jm(g4zU9F^iYl0B>DsRiVB&k~2Gp*ULcznEVtCDqV6__5DSyhGrgr*#Nm1Xpstn;evxIh@d}%k|)T&I~x(!Sn()qH5zS&h- zu5}v|I@adP5BZ+0I(?*WGgGH&zM`mKSygs^ol`=mW4^MRUsYAk?YeDDkG=C%3jOM< z&J5JKB|HwzUp?g4P?h_$ZYR@|n0z%+|JJHJ;cGq#PqOmYy7@n;%GbTNi|J`ezGk6+ ze^r6&wLJ+>>+-dS{D-Rwk6hc&)YY7?Bf4w4>TLeC0|{My`Fd`<=BkQrUpvgyJ(+J% zxNEWM+`u)QCNJb04eeU0I{))pBvTK4fw5=+SzRn#k8@_80#mnuNcQRry7h@nz0w8d zg#mokC9d^2gI-%;ITRpPU3#Sc6jPsRft6^WY;{?F{po~0#{wI-K$Yr?x9fA6o_QD8 z76xioUmB<{On4SruxTjJpt}5L{duPTm;!sz-R9L5!q-a^`m+ig-FDknSL$9bXL?>z z;9R)dx%#r}^(zU_>k76G?RKxea^(7dOfQ-XTtxTyR#)X;zn1W#uVA~|o}jeq>smLz zy%^u_etOS=w5z?>?`(ZJH|Wu_=ZM{ZAGGejc)2vVlOZVDu4aXH^VV1FLtbV<$#%7> z+HEghi4FP024&jS*=cuf9Z(taYYED?yS7)m>&1Y<&@P6(7wqcOwEMOW+71Pp?X9%C zUakG&#i0App4h#$b~m2h7|@NhTQx(J92*r zr}`6i4QcBZwvIj-I-*zvJdl%dvNb$M5d{q59zM%|C5p zzX~IN?g8K3Jlw<+u));H3F*l1zwPS8I_DU+$#y@JXK#)vky4bc6(_e1}s@L$tcuMs&n- zr^1HQr$P+79{tgg6r7G5&X5VU=xP_&mCc>b9?tR!wd?9I)KwImDI3l%4c*e!>7lEV zJ5xV=W-8R9>+x}2HNn}|;XIiGeqB$BbTxBl`-cmB4(#oEdQVqJ@b&cY+0p|CyShen z4RT*E4xgJkaJ;MgkFK%c8*-#rCM>?IM_kW5_YL1jiBDKsSFfR-mEc?1k+RaToUT3( zJ=@&3nj@E{!iu_{9oMrLd}lsVA#?CzSAUV7bM8Cmk;^^@tGk}x({mAg?>kaedhmMJ zixEBd-1lK4SEml%>w5V|&r@(NZlp%$(8I1*;`+Y1bJ-(xK8K!m4H)VN2!1FVsV_bB zylc=yKPdM@{m6}}LnB>7$Mu5+KemqG(q*P=xJW-N_hbLa9iPJ=x<>BlhYNn19%(2& z9Qn0tbVNTQ_tWCY{i(x$y2k$KM+wf8qqwMH>>d|4h|8Vl8*TOpU(r2bXpkiMS$4Fw zG@QSC(!(G%_p|0`+f=xC_tbHN48aBSQC#lGbx#)=Waln8k9PVTQSF|&XOJiO#dj3f zK3d(gBL-)4zl4o;O&u}pe*MRwSa30J6c<7k-EYJV%W@a9NBew^+I7D*G^`N(S~l8W zdUQ+oI}gLE+^_YcFQ$%qbiY4tSR?qYb@Y|YF~9D)BE$OJZ~dc#KF9WUf4FCOOYr;j z=y2(=gWVrT3>$L4FOH5*9XsCr>5pNf;16h7=ij9v-; z_8prmJ$}9W+lbL{?(eX%k5k9*b$|b3G%mOlH#RSG;$im>@%7WWOW9)!J|~`b|1@0x zM(|JB*kb94=iR?N*3aessUQ0`bz-FZ_wn`ff`41be#k`5bT1XHU(Eg6KlaNf@Ysi8|T(LP7qYSFlFy3ht zq%rsuv!9LDtYI2#6is3{P|PtLZBWDft8pbBu@`fR#hBO7M4Duh$kW9wYjzc#37m9?0N$(WDo^{7F*cN$k;J&V?qUH_ZPeCE4R*g1LB_O#^70Qn<5o(w=*@ zspTu0_=HScT&6wGCQ}<17S#z{4CdSO?lax=ip6k3fsDUk&zElM=)!6@fh)sGd;Y7Y zTVJtyOstBFueBHGGTrXNws!)TiMQTkC9){5I+>lPEnxWp37LUcGfymU1aBVTig zm$ZacaJ-b#4@&$vFW%e|uE_aX%CIW&+q}eNOGE|dcd7MriGSx;(ziw_axqJrs3b9c zmXvOdtKbrpHVaDP_$+1Gnxx3BAZ<~VB=A|f9i0Et@RY zD|^m7JCA3}q>XQ~YOmaV^E@G5pAy@%$r_pReXRunyn9RR8j{y>E6kb~Rq!5`-aMCV z*sJi@yjX}Ye$rm{q(!fyghg2%U(TeX?@7B}B_oRpA^wXc&e za@%KR?}t@+{MSmh4xe<-TowB8>KpzC(k|>N-XSVk59eovzl~UYlV=Y zj89N%`e$|1wsu8f1sT7p)H9#gdRsop6V{sYpH4knt8u8U>y7YwnE>{*k`T?TwmugT zyD3}*S2}22wS4hP#A9ktTw1Mzc9-R#i|F1dToK=PSohI#^p)uGsr_VnlY^tD}LsPtIGsd;sy&PPA3R=zJi-f-%N`ucZ|{L0f; zlNFt=B#UpfW(G5liM$o#ZAoF@*su&v9#cyz7F{VN-}w3rK^`-Is};AUbbS*SGgk7L zCs=XoN?ZFTYi26&SX{E=zb(B@HYF%?^@2rnyO5I1F4?rIOzj2B$#$_ynZvTD<}!^I zHqduSD#<3wW~yXaE?7x-$X3ec%AO9&+O%M8+M%cag5@b2;}Hw$VTKR8mQlt5rFJLoKn_b_p_{Sxp+$~ce>8K;4$tBJiBZ-F3Tl0jSeD>L<52zc?J|gp_oP5)5vs~BO&32gQD>?aH zwLLSi)?*eACe)q$yKD1AH#3_KouEbR{ZMN6Ez}hKr zwcVuw&7xU6zi@Ecy~*yft5)^w^VR}ar#<`Zt`2D3n|(~^?d!6;_}_j5vTnh?d}h(`!hR6o{e@2=GfHis5q7ex*II95`CEgc z3XOx8yPkLGxs|_P> z-z%wXo~X#JT4K4FB;6}}**sT~cW=q2#bnc7MP-X}MgHoNt&1nUdsQx5+*B0&P_kn& zC8k$R+47O1uxhFQVrogR=4H#5ilTc<_bsM1_v$EbcU3zFSeX`f!@`mq?=K`B8ZNwCAW@ zZJ_q^OGDX(p-y;NXZcV5PU|``eo|PAi(nEKV)G043Gmvt%Wr4c0e*Ucwrv9aSC|Aw z>q+^(dTsgPQyOCbKVAnCR^cb{N)bJQfD0Xhz;NCuJ%7n4y)A#+HhzMKAX5LM%UJC+ zNUQ-hP=~dk0avyS5?_4>iN;HVL=zpk=)J9akXYF@NGO5a&>*1z9e)RjPIwGYp$n#A z210#@h@p8_;vl+*;4p;45g6QsaWD*{@ag0b!J9fn@PjZMhq|;OLM9#4of;yTGKPpS zki0TP*gYB|)Z2#$4PbdTMEo8fB9T!0d|1Ql=)^Bsc2(1YP;@EnF=45r}?EWmg83BOt`_%JaqJxqLt1^5Ev3d4kx`Y^Ese}Gs!OuW|}Cg$K11nUhGAs}Kn zObl!sCeHk3C0^}7KiKXZCN{xlDE1sCvVDh%cW`jmFmVWY4h|DfONWUc#0c>de!*{G zpc^3~As#A0oo$3T3=cU*h+wV}qMDA4NJsY(&j=CChxp_XLhaHBv88>4`2K8!I5IOr zL_qDA5kmg>D4__FuSbdBUq*>PpeM{m=!4M8F~R^{Ll8#45xOGi8l$@&MA0t>Cg>MO z*A!g|bXS5I`X$jdM^_46X^;UqkOxIj0A)~uRiFYEm|hLtHJ}b_K@+q<8!Rz?9lE-p z1A3qj24DzAupW%T1U6uLGjvVCk{Sjpum&602%EqT?7X1TXLgAMgb~@P}Ox2)khq1VaeKKrF;TJS0F8BtkNzKq{m`F62QzoP+aF3>TmT z%HSegf^w*Y%TNW?uo3BS72W^f9ymy1Um*-mz)GnxLJFjzT49W6P(r$|Lb`*1A?AVm z@Bo%f#)v;KY%xZRz$lEtI84AKOu-Dy!X`_s4<0}xG(j`4ZWtrDAjSb{0BqZkR-$ag zfY%r?2+BUl7jS+57;yt`!X1bR9V3pz_-U*Uc8IeP@6nxu5AYE_!90A11^5Ds5SWc) z070-9LLmggU>}zA1Kpo+0{!fzIM$sw*1+{^jCcV9FbG5N5n=_#i8zRd1V{wK4dcXq zhjAhpLg6SJg9wO(D2Rvi@D#d0*m0aVtH4H_gCdY~#dvg!K?(gO=qjVT3RZ&}sDlQS zVw@(r`d|QtUK496e>jzc8GLN-)j+2vFhDxnH$p$_gs1KflA@BkX233^}>gtjA%KmbjuTE*Pav;O5GM;Kh!hy;ogl7{Ob|EVCftJCa0l+feRu$k&;-rU0F`7%tlP3I|~-*w}jW|e-9tw6U>7Z;=iD~2-fKT ziS82og+K5Q=q^qX3_t=CFaix&fE###7X&~Mgg_W1Kpa+rB*=g)$b%9ngC^*JBi3zz zt|1tM9XP;N*a4p41>WEbArK4)Aqrw46LO&e-h<<%DZ(EDU^nc6Ll6!j6;s4{To;y2J=#dNh(#O7;LggrRGpgkM0^41hl(KLl)GexY28mI-Hjw#|*=M*sjgD?#< zFbl8Y4ZMR7@DV=2v7RX+0_vY(zJAO7yoWKR#zyrMC4ihj5r%I=Z49J8mI1SlQ2t{zT495c^;5bCW z35bGdh=DkWhXhE3BuIvnkP7=RO%uTo0+Enhj&YCzsgQ6P^Fb1%Lk9R5e!sK6N4}W!!QD)Fb)$i2~#i)voHg%;SGEOy6S1d6wJUJ zEWi>rfE8GS4QvEk7=_`h(*%7Dj&UuHF_b_lltBd?zBWyS!x3P=j$;E{zzcl94+0A}wzrEnyO-;1~Rc zgJTbsD1ag;fik2)R3FMY#6T>>K|CZtBDBIoXal`xI5uDahF}Ei z!30df49vj-EWvvK05)KU zPZ02anuvl}==zH7W0@iR*k*`EFkLZ2SaHt~*02$5!45XTX4nebzy(}kJGg@z>;zBn z0blR~e+YmONP&BBAKrl){|vDfv|%0Sff1|+V=w_TFoz8gDKJBHLl5j1o*{xE1P(wL z9D>6T4oBc99E0PqT6BiEDmFv>2Q^R!*PtG*!wt9zvEnm?fW!fjpG!$E}^;t%`-LUD$m1A2<`4Dm>1hG>Tl zFkU@FB&p31$#4=#2Yv7Y zDvV}`LE{->2!_Ga1oN8WSc8J)3^4*@FbWc3^)xrkOwYy zGX%@#8G;qqfde>!3%Fs<5&Hul;WI427kKN0JPIFR9v0vWEW%fqbVho>Yj^{1;T^n( zIrspdU>+7=5x&4z_y*r$v&#(O0FK}U&fo&BupQjM1N2;x|F$FlfjB&LoB6+JyYHZ= zlECfvZ3e`g5ED5^Bf&ssz?=~yM!=j21`v@fK|nxJP*K2~2?lc7bywH0%C2cwbU;Lr zID_GSx9&f0-TQm1-nvzHt3LJg={d2x&ygKwXmMc);|f=C8|#OomMu&k@+c0giBjGbC_7la2Z!|7X>IrB{DquIU47XjmOlTC;ucbke6^7 zS8x?l>Wwea9x%fgjDs!gU=IhZ$9_oBi0+pe@3?h^=Nxj7i|Ff&ryD%Wpa(N3+~)p3 zKlI1wJdT60@U5U<;Ez4n`(ODP&&LhC7mHySju9A%QMmgLV-?kChCX!$nE5Yb9>$c- z;epfdc&3wI$gdb&QzQ1VydO@K14w7`0C^C>IE4Asg^;0GKzSj;D2HPay8A%>S z6qd2P6w9#!(bRd6tFaEA@PapDc`XL(DR06y?7&VOq27;-CllaLc@Op?k>wyViA+Wc zLRdb8P#j}9jEqDa;&Bv7NW*EI#W`f*GETD1RWcuGlz$~3p$+XgP2Cw}K;b>xlRe3E zWN)$$GAZ{bvq&A%2xDOhXPl>g8Mz)8C~qS#l7ZwUGKh>o2C`9t=cvIIUVl&4q7L1 zGg{D!Hpn1H2h0b)7ccP&um8(I??o*}4t_5-8NU}z!z??qn(OywNrycY=z z--|>XMG}&cf_ZN5g}>i>u?I&n=Q!uRi)Y%E_aY2wl+zJGITBGgj8oLbl5seK1SBFE zDL97XIDu0*gDhm@JT9Q~)%T(cx}iHBUwbc};3#Kk&*pD)CcjqyMhm*YxxK9YsCj4 zefB}*y#64#A;r6&A4K0_Qqd2?F#;o@0Zr(@1g4mZX;=sk{5nc1R%o-$e71o)v|s^i zY(o%2R!YTZbXg@8lhJ#VRG485rs2XasmMd%ZmFn-{voNDjunW(Ij0hf~8 zN>GkUl(N4Pl%X6I_~pOX$Oq)_c!H<+6V><&f8!-y;Wgetidxj+BkJ)9%}{zH6+K~w znV5(9Sb&9i!tvZlcPzpZcwi+~VFw=L2^!GcpL0bs~6gok)qV6E~~s#4WTws1rRO z)`{JckK)_(kK*6MAB7Z^2_Hoj7UX>t3-SB?kNghiqtFud!a}iLI3j;wy|{<_D8c(- z_2RNxy?CTqFRab#g~#}M@rQH0c#NlbhClHfn)yA2ob13i-N=N8tB z72E2??>$Sd1lDie*@il=yma3=zlcMI@pSi^=!-If*=n3&=+$9^hB} zj)!=JKkyh&@E2axS5E^x(k%)m^{!W_)Qd@O() z7Qr1$uz-CnC0Af2Hew(4;sDO#95RuGY@EjhSe&XCmaxJkOvGeNfemJ34(4JWo}FeK z;03Dj8gJ18n+(PadY-Kp%IJmO=mWQNoHN|981HdDt6u!ft{38bz37C_=mM3C^#~rkw6>a}5->esRk$Aga97Ph6k%D75j#Qk$)B^sWjeLt5yhAO%qH9jQP=Fm4 zu-;5^1r8zwE2+CoUd1isU|Vjz@IeqF5si4W)z#blf)t`{efhIE|5X`Dd@ zvJqWUFU(8X2gYD5#$h}rzyg-gEaM!Y4ISu05Be~K0gSL4KF}`bH5kAUMzDY-CSnpM z!v*iE=p!^>#Z&hCf^9MU?|Ko4^{?ys_mVM=^(p$MMsLN?9Ou}sCPh$Px zPh#4{PhvWD*?$rd4xdB{j^Q}u9-qWF9P#}m(!)QAQ^-O#JYIa_z2#40O5G=616NGL zOw5AIZ#~5vatC%|7kuCgKltO_r%z(b$Of?$+prxwut>K-Oq|sq;x{&k1SH}pl8}rP z{EK&}!Fzl_Eu^T!XEdP=GWrjJ; zecDr8ATJ^pzlJu52QW%*5XLY?SxSQ_M{Zezuqtm52c9*EARL6zbN+vyJcJO0A`Ib( zfHAK}l2JH}Xv82EaX5l_Bp?wPQK4?KJ{Djh+^`5A6+R2Co}Yy#hW7d_ zRI#w{XYr}uXYm;g_<}|>p&2b`hYTI~hVS?Rzy6@y zQTr^^VF`&2*M+-!pT#{Cpb#ba15(tY5w^yh3wBKXEZ)xfEdIe(x6fkRqR%31^=F~) z`B@mi5Th{x7O=!bOu}Tm!fU+2Tl|9>yoVHZsK+NX;ENZ>M-#rH87*i<8`{yy`?Khb zF6fGG=m7;N!E`5E6u#*xiZGS(G|a+u%)m^{#vBy0&RlXH zN+_3NKIH{ih%%Pl$VG6+Vl2T@EW-*chX=}8cO_Xtt|nJu4c1~EJmCdztj7jy#3pRU z7Hq{fY{w4l#4hZH4}9SVf9%0t>_ai;oc%23VjkvW0T#jyi{OsMSb}7vpa{h%K`9#X z6>jG~i#}KA&ugE>%Ny+LHv57b7U6XceU7+N>i=fnSl#^#?_tu0C_|&-7txLthF`== znA(34W-xI3BCgf(eyaZ$(F;y{z6c2h?foJKLj|fBieVUz5g3J``@RV4M_XcPnRuv4SBpwK9chBXRX*uxGEaKyB)yzfm)-~v}n#dPZDkkuNE;*@ry zc&XDU8YeW0X6$R?y-e~4exkEQqv(zvP(}c+2cj3{Ao2hXAs7b{ju3<*3=xPzBn~4Q zz1cQ~j71;H{V@P>EDt0HV+d3*6sj18;TVB&n2!Zm0S~N%nPsCe$2e$PH3}UJvSxo6 zJCS-AO>Pv%=xoy{x}Yn3;fIa3jban5?Ha{IT*77CwQm&npyo(>LBY9E44l^}a+WuW z5|kry1?SqrwRtv*MEv7Ln{93s1=|}%A&O9pQk0+!<)}a<9CkK}FTRb8k4AC)TBAtC z37kY4(s2qq?=*@R?fmY5^v7Pv(SdJBz0WogE367q~}+$6w9zx`K#E50lmM9ff(KYs~7{D0bhk3 zy1RZArc=L)t~0-iwVq#j?tK+G-d}~rmapRM)~`aVld>?{{#6)b3%quH72dcP{#D$F z{3=Es{wf~8_10IRo%>aUcT*NBPrr(xP{l9|ha8KZeHHFl{Dxy867PB_3pM2?p^jPo zn#62;pV=f{D=CW~l(Fc9e z4+Ag|gD@CFpn{OvV)0U@E3zE!M#k>v0X&@eq&Df;McP(}t*fS;)o(d|uNe zlavVNg45fC@Z-nvC%dUFgHSXR{cMhy9zyBYYg#EJh4!79$a<*DUPDGz%v<S*Ez%0zcT)1Nimf_-}W^oyLxC^Vr z&B7WRmNbiv*o1@?%_0#e5WcEeMBoVGk%*%(Tf;SB48~$R?yO~>c)hM!G@%(Sc+wY0%zd0fny;7N6~Fpv*-?UKaPnp7>n_kfOE)17P9dh ze#b-X8mcTFk&p2Qp5Q5-;ZHor3smE8{DoI|iPw08xA+JDq6Y8r0aAQKJ+AF(7QgJ{ zxOj+&{Tvfd;B&ZH1mFOI5Q<~CjnUD~A})^hMLcqn84Jf43z&-T>CIy14f+y`;Et8J zd$U@aE(;p1DxVMOC795t22NXHg2 z3Qm;ONOfqynPmyIC~HC+I?#n4TzE~NG=M8*L(&*VFo7w|VTREdgRvNg@t6P$Si%a{ zn1qR#j47~z9c*C_2ROnB&XB+bu9%8xn2s5kiCLJ9Ihc!in2!ZmgvD^j5-i1XtiWol zK@X=Ep#ViFK^eW!8-1V#GnivE#$YVQV<~OY%eh75z+ifdFhtAT79m3izTrE>ycW>~ zUC|vqpa4ZEqc{5CS3Jctyug|HE#d<{q8_biLpx-U;|G31ENBs(&>2_oD<1r3VT%~z z)*@6e6!GpYA_0j=LdoJ5QH}~!q6&6P=mX4ww?~Uuj|~{LlJH{s}FTBKN zZDsMAe1o_62a|&tSFnLCTrm~XFdZ{66D24`8Ol+CN>o7x)k7^}7~HWKORxfKP#fAJ zno}5;C)hux;_*q&4Szu)jj@EDP{!U2`urUAxR%NF;0A7@0FR(@o-qb1oW0Z{GI1W4 za2Z-xX=mu*_I273);GA9FbOfY=uf2M6wcr*&LI<7IG@uZF5n_ABOiBg7wdEBH|#(? zKA{0$;FQn32nk$}bFW3@LZ^^Ehc)cs0FPq!QQ9JQVsIs64=Na0#dv`h%wUeu7=xLZ z^pJjo$|J@phGPUqVieRMgB%_BhVS?R#b>lPdZIV_V*m!?JqM=1+`eY{fQg#|~sR(3Z%?d0fCnxPEC7UX9!@SdZwh^a(th*k23% zj_dzrJBgdPh1c#3DZ+PhWUL@qw!2ju-*MZW>9 zqCW;e3~UvzR9eMryg}Q@R=z{r%J+v``Mz!|-_>p9ySlA>SGQH{hOJJku!B9O=(P$P z*y0l!@EMJ0gA5t^t>P@sAro1+fXldvD5ychtW{`YxH+$3Bt}64hmeW$xP^q#tzyCCR&mVTF6H^bsE8364!|6~~c^12bAh z5Dp?3M`pE(cuby68^Z>+u!9@)7q$vR7{M6Ph(QvPk%D9B=H4p0qX!nk4U6E8#aIH1 z#T)}Ru!SA$u>?zTW(j?Wvp9!t%h)Ggc`#=12DK|0SLi^GRjon+icrG(4YU_xH?@j5 z96q*QxmWNV&aYZU-#6?7pT;YT zs`nfZzvB`9z!N;hGyI93Qu?-*^TaGv)Un*iGJM+TOYDO~N2_pz6S^t2iN9euvQ4;q zwh8^o%EB|bO?bf@>#+eFu?d^81zWKV2CRR8JcwWzvK&Dgkx}F+oW>bs;Vu5bzj%il zyoVH5RU{T;y6+OYf;uU^BL12X?^+yRipHaSQkH8~(yO403B1lQA0y@av*>@c*1|CU*QC%GGGNXIF- zQNIX}C|9Ei&P&_HEG)!wJi`mT!fU*N%CdGb9WyW!v+xxgmbZ&d*oe<7+C@D3UgtrZ z;t$GQJ==vcdU!FuHnfYr=!gCofF+yg8+<|oKI01-@fABabN*PlrCqGTYA9`O7d>IT zjkdvQoIwW8;v6#Z25(Wgy`A6Bw2SRK=v(ZBYA|gb-Y(`K8&w#3u3e;bmI<96GO=Wu zvM?Z*k;`F5*@Ik-g_IZlw+=R<7!|0*Lp;J`RHGK3(16eQf-VX&(Gyd!0U?&EYMHo$Ppck%XbXGEuW%CO$xl zF~KtNS0#U83a{`F{zVNI{~{Ahu?)*0`Bf%nU;*~RJe#kjuC{!^F!*L0haRpa# z9XF7JTr@-FG21~E!>|_X;0d>Cndth3cE*M#+HRy=JizaGgg@{Eud#o!Tm&ExZ*1k_ zEf(6#g&P*Z9jmY!Yp@pU;E8kCH&xEPt}Ft{>EsLqQI5c2%w#!+oJG#Y9L&R9#8MxJ z`IHyn2<3%jJeh!_NJJ8nk%ALAj#Q-KB+_vTXK)%BIEPGR;}UKo5BK0UjqAb+tio!n zg(o&)GqzwG{zWUarptvUbYTQzn7|ZfFvn<&#de%W3D(Y_y|59RkdIpU&!ldaTr8U- z7t65%9$1aFSO-sR!WL}9F7%r(7XvT|gE0gu7z$PR!XJCE4?7pK9em)2<8XJA^Ic83 zSc0WkhUHj^)kr`Jj^PsS(}y#o!Euy z6><@^lJS9P#NZb4@H-yEd?~*h!E4HI@D~5z9b(qVMJ$GS$%US`Tr~Q!pZ(P1RS;u4 zPA)zq)6V#gANUEklZNTXe`8C$RwyRaL+n0QYvoZyVqLdMu?WpVU(+H;Mvu>D6a z?BD=LIAI>lD1qg_a$$ujuz@Y?V2|l&ekT_#Xhj=3U?`P~ z8JLM#n2kvv97f9v5&Cmv9+Za244t&FR{J>9$t{kgdhcJRMOkfH#m}4}?U@XRA zJS<>|M~Lg*A&wv(2}s0IBq15ca2%;Pfs;r>I!@t;BHJPX<9e}wOn?O}VFhcrz!g(5 z4bw3L&+!7)_zUOy(pK0rphN7#g|*7!A}--Fng?}=7PO)b?T|r^4t&FR{J>9$!5v~5 zmSY7xuoA1V8f&l?>);7*cws#@U?XZ#hmTMg*&!667UYw;PJF|6{D7FkwUV94&gg=! z=!PCp!VnCFDki`hj&OoAByhnxwAr$aU56;J?+}G3LNQ8EflB;>U-1CH;Sn+&I>ctD z4zUf};fvwU9byDV;)E53J}d0zO{ob^Y%}m9GE&IbHtyKc47Yt+OOR(Wa9@ zK(`#BqSnPTpjSaBtAHNuLQSn}XaG}my9Ov)bkbAnmKiXpptE~GPoGX^YTYXWhT3%T z4(OHB$wIA%G+;zQ7r%f$?VW7Y6chs0ZMudB^t0$JQB%|l)Gp|n5-`B0^DH$bNua(> zx9osHIi20qdU^&L7j(-D7}DN(g_?3`pt()=s(_&uT|Cu#Wd@Eb=w2N#%%{s1wcZth zmNq@=0!HL?@lop|4V+ZaLmn`yy-R>vUxfpK0u?)%xija4b+z4b=4M8l%=< za=^t#(I8Mer)!eh0M7%{3lyyab=$k9sSON0Fxy7SHBjH8Tc+Be%mecal-vUieY#yy z8(eWHb`8m}k(&f?iDL-QN9;+VIeztv0=@0xd0iNYzGU2JI~9T^(rc z)8mWU$ci9en?7}clX7~rtBsNd?Jejd51i88gAWcX91OJSt8~EDLP0@YUGHFUL0{Da z_C5+sekVB?X4B8$fMbq=in^xf!Ki|MRtKEh71Y$VLJ!8;^mjerVxg#~uAO->p`gF} zfvG-<{NeP9gUL1nybnyzQM6Fkl^#qj7~pqcX1k(|x}HLCy3N4w1G6oZBsE@BW^ulJCchJV1UODO$q=#M>4D$=x+}^7|-9jPctH>HSdMN)l3QGa@@^XHM_u>eikip9)6g1?_I{{YHIaXh@^Y$f_V; zi#}5INtq!n1tY72{C)a-QJ-89BC{D)7qmC0&;EAxDbkSd1*7Ca``h~n4I72fPPS@F z2LmnoDrng1g?1}cQ#}~u)3>jNog`GzR^8xWa86$p4SUbfUWMvb2SeKXs%bcchW4}7 za6K4i(N9mqF*9^fp@#dx2%mmt8cr3VLv1y^4@Twmv(RvshK?xI^g9^c-p@uuq7bHT zs}+7Q)}p^e!$mJlyHG3T;1Qqxvou^KVfwb(*#{GH`nzdN^$asE)XqD2w7vfdjcK7_ z=C(Rj2a_!Zcxp_~3>#OdQ+@E5&wwo&Gb+L?ZFTDorsfRr(U>U>n^dSPKX|fzK!C<9 zg>YM2J*D7ui-DmUv-QFq3-wfkPx}nyPoYb~U2OFYf-`akCTYy|3}?nTtKf6(1Jg9- zg@(_zHE<2ivKW-9F+VeWexZST@Ohs>S2Pw>gfFr+^bWq5Gbl%6p)`DHp`l;!<@P}Z z8g2>^9=1l|!B;H?S7IPr(1@+JCRM??7DJ>OOEV*O7MfHC=lcx#qOq(Z!q?WcF8FTFkamsb(uloUWqN2IVGcu~s-0DzyyNa6T>d;80k99p% zX)#n!b4_L>Gse0f`o(9cndaJxNG6H(KJ*}GsDesjk;<( zzUt5mi(#Ic8#AMB7LKnz^q0@DEt;DuqH=8~)E#=6Gt5VGvoz{%;RN}i*X_dsG`A=m zF0{2!3VCZWJXCY5-r>?h3)PT+eTK(qZj&6Yw6!z{smU3hq`BSm@Pk52tB?=v!_zc( zgdTomYvmeJYcV2Ib7$t^r-fGTAs>B4T+!TBarlL;wRgy;oDn&iyQPO;7Fzp-d~P36 zpy{I!{nmD3cu1qg$O=tgz37_4i76pXJ|iD$`bnZ|Z6{@iwB(F@uIcX?{i$$LUPxQ} z$TymMLZcgPCs&2YEJjH+_hv@76i%)V>F^n~|BL3nifEbbl)8}bIiuP&_e-O{7fz9f z{A?d3v;q`jI@#GMg?6%3Q_u?3i|JNmqZ-=9SFNws0ZELaovlG=w_G(9tsu{sUPZQ6 zp*>`3YFY8oz0b*Lg{sGYrcXs=v#3#|}o%!nd; zztBE1bsMcvg;;evhw#vTmKqYRFuhpqB8QaF0lpfuw8AB^`gV@lp@VWY+_WM*V~vX( z^FoKnG*)OuhQ^xPIaP%Ywbb;~ipq=~SL9S3I?Pvdi`L20m85mVGj4j3 zi&dDeOe;+*J~VE&ovUk@zNL1iRzhan{32KPFhgJMD_V&aaf|GxdWRY3YUgMjmBuYC zn(7y3D$_2|N>VuDVK*&2%-m9^LMvJC$eN;QDPd!LbslP^NRD{fP0tP+m#gz!>zL<} zjYZS*!Y0Uc-e?^UJ+jqqMpc-lrLI&fHS@^Mq8Zg;*1o!5v`$nU@wJ;-7d9zZw_WR` z^vK?#newnHGF_parVt-!H%lqp)>2PFJ6$h6xM-GYxV^7lU+q(p_%OTK2H}pmdMetd zJ>#Q_W?O|j%k9zNAq-%R^#MSQZ|T<`Gdx%w8` z=cMteMRWbaXUg<#v@;bF((UGjhtIY&kZ5P=C1e!MO9`LrYcNYYTau7vH$OXkey)L= z_Ib~Qi$(ME!WYU6R%l-cO}J{epelTkrJ<+x#mt18MGLCK7yBA+(Y{oXkZZTFE_`XO zp^x@uX~Ny2h4S#_GQ$AvD+-B)c5X@$9+pO-+E?`wON-o8BUbqu#b{rXBv#rjGKg4{ zYm}sY-81n)(ITsebuyzg?Hi$qkL=uCBfKn)GqrDKCO$24caK=_YkWofRz>0qyT#rS z8*`0wv~NojUl#qZQ*PTW36I!nX;Pt`t9P`fXh}-Mc3+c++If3h9NuBIhC`ATZnkncM>Lqn6UZENpE?%7ydBk`0ES)MzvcCPA?8t=N z(QZ1wcqSVcugQx%DjU5*=hx6=bNjVbk;#^0JaryqCXXv#TOE1Kcgz-@-8^NYRRqt5$|yQ1@~B4v^N zdhe)zby+s9KHz1b*d%D zyzDn-N8QXF|6J!U&tn^lH|9m%mW_X-^LOa6t@fL$qH--KNOfLj9@|;GsX8j(cfuE) zR~5&6?Kjs&-OZiQuJc-YY;W;qdDMN`1flyz;dr3^7Nx_5mKF-SZ}pA`7jIELT^*kO`yw&P(xy(XM_g(1mSo>|Rhbt{D^>k}8k0%swb3gox zucevp`-P zoT#+-HHfasotUKC+iqiZcEW$LzMo_Jd9=N|phchVKz z)`}A^?ESr?Kjlu!(QT8Scvk)fmXj-VWqK!Tiua^MH~CI} zs4JJ8thL{p9o>>U`MGX~=gCjSd-I~(WRv&5(ft;BveABDRkX}JJDMqf`cBi$iBJZ`p?D|2Y+ zXJnMbr^LXH~otK_{y^{9FBP(*_byyM}I=?*_S29{)gGG^)2kj{b}pHUj_C&R!FjWo*U2lhS82Lce29FOnt?-DWXK9z7 z`M+npHgI*x(s#_rj!(#&<7P0`E6cbvBQO4_e9j7kX<=FBj%Tallda}@8cfg18drL@ zI{uj7+${z(Dzhvd&(+1J=FRmnm|2@Osq~yY{-k_vfWa)qY+J`nrG#{=d7%cg^|KvI zGgT8#`^}3nnB$V|;+SQSkdZep$zZNm_Vm&$tAum%d1(go!m?*OX1gY2SV$m1g23Usx^nG~AeV@n-3*>cqeN7H=`!RCzJi@pfI}%e?=2T5F}FZ>^Su8gA9UR9gDK z&SdSFYjCtCZ%LBjcCSkhN^`A_evmIoGu#n&>5*fe>(N@PrJ061vo1X?&2vBc(QoM$ z!(Ek^UO47^AN`cKG{v$*pXrtA#3PWH0%QdBUQjRwH zE!+Rl(9h*^t>fM7qb+&Mo*VjmUH(*hH}7bheAyerJzl@g;c z{j1t#B`HY*{8!F03U|4x?^K$dG$?bnVqiW##guQ5~xSjA9h8**aAyC97Mn z4mFC^zvft0p_;7ezdFV!&gGhmQ>8(&cK+%lqa$9|rk7P(CF^#qPBV%RyEfaY$~9Tv zdQGNLLe{nUWmWFUhW=}=7$sI-TjcbMcd~K*njE8}wbz!G{o!E;D~yu$udgZlH6?kB|JsK}DK6K&oE~H+kIP^C+~}Cs^^Ii@@{%WXtbJp2JnZ^b zr{AiQEv?r{jZ(9&?=1VRI@#KP-4~-1mDhcpey>ZOl)tXs=w$8ny=A}4lc#j76UJ$Z zHv*j=Dy7(3dny>G>)!}2d#IXX@9){y_>{|yFsDZbDUSJ`D#oY1ZbX$mvPyC8@KiHC z6Lurk=?~Ww7i%v)*bG0{8%O3ls%c&pIqxzaHY>kXmCSM_g|mOWQJw#t7)jPW&>Ta`{P434eI-;iW{-Rsta zvKLmz)^%)1Grkda>ycBn>oG6ujhV(bvu-^tt9C!O-hbm2<6D)tUO4^beQaa?#vJ3@ zwYOfD{pELTbH~O4;~d4?Z=L=QKep9+)BXzMT>aZMWq+p}+wQ;Vp>dwe?OLao*~fO~ zZ+dQ=?{)iA*~`3RyE`_$F}@RayV2=Y)iGb|%~IpLS+`rtUj3iWQGGnndW(Wdp?*%c z@;9o-gZ#JjH7Rn*QFMN5a6CAFi;79HS5B|;w^qkPI<}~pl!WEMwI{ScRaddtBpy8Vy?RLyYS<&*4rc| zmHN5b@UIBJ4^66F^1YltWvAZE-}T(&FR%QK<)89WZ+Gl^WAb-c z{#NIPs?=QT-BOd6S@}E58>&-@Pc^=|&|c9Yk&`FqPh%Tw=n>=veP z6z>E&e^EM7Xzin5`d0r=aQPS26UF{MeNF#yxfABxXmFx5-$%vtU#~k+<&9P+$~%12 zOy7mwiFN+!dZNrO)XSN9XY`1_ifzOTHK?A+vi;z7Qzh3SXdJE`SOekXqK z@U=0OD&9?ZZVo^3$l6b0TC0CIqr5rg#AAQIS*CR^ce9*ZvQIqC_j5D-=ymsEc}w1j zKRf(ZnAV5gz3SXrb>fA!zo+S^th+bMTdPm}Xx7K??O&j&^m6o@wp8VH;PmJkTmwT1YGJ}&f`FoN~o4oEl zD3@8C{Lrx{&9ph}-XmwZ>&aT{y_u#hS@)in%iT|Y^xu2Mw6*fy3+E2+lb`bU=9sqC z-g{Zz;dkuTgu5M|9uZlG!PrE#*I|Pj>k4|6=;1^1jUZXWhx~ z`TN^Vf7agrUj9>l@@L0>VJ4IcI!Q#&v`!NP6wEpq6m+W)!_vC!3FvFq*|k7X(#bHb z+noRvvo78Ry(&6cr}g+2pk~%Jyr7??^VBrOiGg}%-LeY?RdimQ)^kswnOXO$f}xTw z>(hGO3A8ZlQCBdcqKki8pKpOSW(rD$>XNPzY5gW1keDeN6lzy=J(f0L&w*KHO0I?a zl5XeI2HiQ}X4ccY(72*oe%g?42UeIVhZmYly8n_kbYhUFS+DHEaTVSFN*lH(Xp33z zszOUikB@01?gaUm^{FeIRMDd&ZPd4*0JFYIMYa-!p6Tin4~ClcGbnPbP#Bi3x#wVv zS%23e7m1=_y7rxeNoE7Qi>6m7TBqxNJD6rRFuZ8CL}_Ze{>0!+vq9NK^DC4VryK4G zzG61Gs%Vj<=lXQxJHa_-L+Xl_R`m2wH~kh|V5Xu}>>*K(NH?E&sKRWhLGk}ME%kG= zVcx|XD|+RpPxyA|joI+<;;oY2zoc7E43V0R$S&Sl(fhA->pdY~%tlrf`%3zJOrLZo zq}^;(UGd(EJ{{>(zJ&;LHKme3N#CBQY$t{)n5!F<1XuJOcFKNFXkT*;*OD+vKf_aw zcS2RnHN8usD*9QUa{d;oX08=p5-aIH^_0uRFg3yqZf;OkTA?!R^r}6PG3KLP%PJ*94NtGR6PaW_#=Go6#Zc?h z>%K*%nU4)Gdn8evdfIDZRHpg3?6Rj7s{a>zZywjwweF4YorxqQxB~$LMobt30u5m> zC~6W0Ap!;n6cH5!6aj4*v<|2N!XSf!fHGM@W)O=a4z0F;$`FJ)fJ$wvIN{WzR;{*b z@B3`D=bn3R-#_lTpZ9%#zt87J7i)gjv(|dnv!?Ifds$sVT49HytF~|4XGTKZy3>U_ zPCCwc)c51SjIDK_y({c?6w~`Be7N~_N$$J;4}D^~ZpXXbh)6;oKz(_-_37?KRFTwn zfIP^XQJ0T@ld9@B0 z4w@~i&rd5BioCT0#=hnW^@XR4#iB)z24)VLZ>`_`u2>@Sp%3zXE%NJ&-S_y2d~F9U z2Q8}VOVaizL~{9{jjv@}ec9Yc*6~;a(Ca@^2k< z8ML}lUzJv(5(Q`n-F&Sd*Y7`FvPrb$(ct_+>$mj>-j%3D3c6P6E6{DIbuY~o1=?!8 z1_d)3>e5P!L_u<`udmJQhKAFn<)Ywtt^c6SyoRQCr8OdDo;J|ec5y?Cds(w6q*WU- zXuGVTEv>9W6spxO^_`v2(0;n?r0CN}+U0|@w>EtFuB=-WMjwjuwaaff;=Xr4wA6Me zcF?Y>p)+moMNznXXsxe(TSM3Bz1Ky{;)m7^+JD_};@#d`qKLeqBwvT2hEwkQ9*CB= z4s958xY6)++P-I^722WAzMnj9ICFa68_~)~L)!*FdE4;qyM2gLB>h6VuTZzK$Gx2D z6lHrMYfw0&@oZW-=Cn$FVW+R-?8ft_%Q;Tb@fQjQ9p^RnzbofC#pGQm_7yE|9CWV` zIK{SJC>sJD*8x|uUNNf z!o51zDam%^{GfP7)19>Hf8{&UJN_ShNc!}FH%>bqefNHF;oGKvdv^eFR?{z{a*1v; zI=_bMoNjxOtd-1YCT^?2oHOJXX>#f8X7ZUDj&o-GMW$9duUY4P4bNGVcabHREpDdG zuN63FwO%yT%9b_jZmShKXKODS%RLgBnP+On&N+`R&eVErZJzeNR^psXzr>e&<~OtE zAM|nFX?w|1>si%ouWiHB$S?cKeP_2=o~bW)E{VVFul1eRV*S3p#*Wxwd|AuvZ4DjHd$pIB$`>cJ*q>=Q>AdgJ<>lJNTU$PP-_Y$`PQMZ* z_sef_oZmR$Tw!}9R_j;Q;vV!xy)$o!`!CcIj%pI;IU@)|RlXy~E|W_UgENSwh>oGwml`PCUB$qjuTW zw$I+Tce|XVUz?Ce|Evm*&EpypK7Qz%k`kM=e#5#C-}t`!GtubH&qS*q>gQUO z_RZn`6WZr5zvEp*Fnosk8JbOf#5)%iP@sqaX|Sg>kc85yOM|^os=&*0Q5X@zW#M7y zG=?Ta98z>H=I`YnMnHYj!f>pS zzppfGrNT=Z+?)mQ-9t-*TT%rIquph07h!Qyi7Q0lpEWOAl)!rlgK~#naB-F72nyehlz+@QTQ|w8#DmO_lV0BA) zh??Tw)a`U3nt?jY`3=oAG8bg2^hY!WMc|Pu^fV^srt}G|G{IWqRRxZkupR3dwHGkr z847i zj^1Lr^rR8iR@>hlx5|$2Jtvyuc#ac&ES+N@6RJrGW~X2GqKJ&A!DLi^d87uttTvH4 zLPWHX6h%Zh{b4@oMJxh}AiRM&@2MIwtF3QYpYY;!N&M`|QG94lm{ndzs zW?Q`qx?1Bw5=dJjN5p6iwCtKJRRDETU@#)x!50{)=jqhwXV7&jV0cJd40?qCwUbCF zY(yT%Z&n3Jh7YL%gcDrDUbl(rNCtJ1gpIoKfe#U1uwj=YPY^AkKH@roAd@%jGUnA} z_DZE3buUc^SP)y23}sZhCPhM>)Zk^7LtZqiQ%1ES83u^8lDKMJc*mEV+E2wsXf;se zQl@}Cf?Bc>I(1S_AP9KXL^`!btq&YE5~p1NkL@Lo5o59BvY5gnmTXt0dA)|^a? zP@^$m5YQTZJ=+>CNrp~v)iN8Tl+f!|qtPF6?9)J-Gy^Y44N0L1YS6Htv{E5(s!8$V zOjIKob9G3FoCfUfGB%$og8@AAG${<~L^WOpmuN(gRIv02J8CrsazTwoXcDeNW{o&n zks3y5JtX~Hehz|2s1p)SuUot(nxUCcs{(ks@JBZXp8SJf3}p+s;L|)Cj3V=)BB&aJ zFusO^a7h#-Lx>>Qs@$Xzx86d>K1npAk6?bAd;hv1vLJYr6JOvs8WgSO(}fM?L2{cc z{t5XePsT)q8;J)aFxs()GougHS;4o^DEc_Ih(=F>pwF~h4-1?q%W@0*Jy|(LxAP4A z7*1TCl*B)W6zsgmwaFsuL*wTvNXv)WC4)^12)=Sie>ZGb!{Y|&87Mo z)Erv9XIKlDEuo^NDurOOg9xJ1!IS$mX|QgZ1(z*%n~>%Pq(9@P{zOZsQTYsuA{B`! z6g11kYsygxHG_08W6PslESJM49ofafDYZ65vu_RHnj+PxMba70slMiZJ?l?eCFQk0G2{WV8SQ^aRhia?A zs0ayBksOKDJtN`=r5Ol8Vi?_rVfv~78**=ub{<^wCGC&ZzfX`K2uz~R{32d##B*x^jpb=UFDZ1NDXMainbZ3ejncyOFH%pR= z&Z=XpWxO2q!2p=5oX=r*8GWQ^3#yF5QmZLyCplTsEhHB6vk8gRzU$Zk!wE#6wV`|M zY6fY-t-Cr!kAP+&T26Y=(W|n1+%Sa=b-$z!yNzgq+#aZ%)Fyoe8x#VsnWXHwHRKqI zK%;>PD<3A}@F;sjZ8)7H@C5o=~z zoD+}@BdHpxjbX1ax*WsAXqkbGYDpr<`FAxPMiOjS`bc~;8{z0qN?`WUDReYJqSPJX zh-I}ofn=cp@(K+MFlf?nT%Ccw=4uG=Sp&yjH?)=r9SLerOwGdd#0Xm@p`m3eEy~vz z1i-A~=uMZ`MvZWgf|L=P1_t%4TSIj+K~o2s(fD3bkR0lK79vVhWHgZF5u2vcV7W~L zJ4I_VSskQxgOv)w4M_E2QH($>2bK^7yM|I(Bn?z%00(q46=Jar0fz37!B|&#WC^`F zYjPdCIXM`cU=XuEvX7NXN-htpPsUTQi^3z zt%Xdb2 zKClC|RpDXKQXpdl7?vC>&N)zI)71E7xD`{Bg#sSk?Q7~rLXi#!LZr5TeKBPw*Q zGtLUm0OW-dQB$7@lKu)RF$;532Hps>9Q7x5` z9_2he1SvnuiB{A4=;lI=N>3VHBQ#G*G#G zs-uy@zmU zbRCpjomK__`aqXES z{K-=FMK^0TOlgdxGMOzpwSds* zEY3<(lVKbQBFxtjT6jyOj%UwNCu$_PfCtHuI=h%KgH^Tqh(LoN*@D>OAO$*2U+fJl zK*IqJJiHgh%c-u(Mx)5F88m(YiU{vx$y7cORm#O{5Vo9VE8TyRuNK09FHMnye910= zOVw)w<4+#=s5GdT2182jc&8x9d(+eTJ``Bw=snYYR$0m#aD%t;@E8s325O)uwMTCr zNslnhv8+DpDX=UJqNf)Xq{xJ1*!9?0YJz1p2?dTawc!UWUYAG@-hwSYmo^E-E=8Hub}v^QfEAMX}Yf z6Z5R3oMJgc9WZk?+c@oQdX+gCMHHDPx(^+^uea^9mViICt~rz@h(pR-p3@@tN<(4I=6Qk3$I%-OnB$nV7{!h&tZjw*X@X219h01=H1PBXFmJ z@q|aFc%}%ECPZ{kNF-zw+^7ZFo=rxn-Vmq9B^z6aA#4GeM5S5BqtPdId(d=@nW2l2 zzntcY815*~f3No{5DiW*B`H(Gr~kkxzs)#tNPdf zAXhr7*O8Y+&_5VwW8c@6cyQX5Si%m(1S`)i6b+an=gTBh88vUwcwfDDq zR+ryQiPeAX^uH7)HO=r5#m}$^o~hGEkZ#hm?8A#Dwy^(DN}dFsG|kctk!)o{2t7m+ zvAg$`fDt00U#nBi2fq{kgZQk+kdPcA0il9lh$Lw~qyT#s9szWK+TlfLh-5YX1HTU* z@LDOs$!>Y4{1E=DQZhvr!tv~?_ez?C2Z3i_2_km?&ZP2wQA&b{0bR*?HGp9eK|5L0&L@NH3H&LgW`_9P)~(AiYu2)TCER z8wv6|Q$YP)DdB%=5%F(Ii4EQ`Z?4ue_F=rc9~)zm*};3HluW*D zd4Bmhvl)57l=uMu)d$T_y@N_kwa0`~0t5aS!*DO-A0`tH_0QhoP^aoSu9O%x1`7Tn z82YS|^33BnoId?`XBOt)=aqT?Ft1$4=M|W>{)-cfH@TGi@AJxo|Hia}_Ba2RW)<;2 z%_`^|yHHU0f6gkj|L&afvK;-FXBC}TwS>pRs02bf{H0RT5k7{I2F3ut7$d5P2xy{f zCwv-|wj~XoOLv;tmdVe@dzoTX;76jDb8bSVI2mzx`rU@0GaTFOydH%k$Cg5(5YGiz z_dkRZd@S~yoluFnyLIx%sb%JHOdrjwfqU)^h`tH0+be-|EAE&L? zuk-S^v;{XOU(hI-2fmwp@f$TuR%ms%^P-()u}?ij9r&&&%+9FLt7`b_Z!>pW73~Eb zOaHhU)Sa3%SE`cNMt@YX*Nu$o*;a3I-L0OayL105!|X4$?@5wjFUikv*9YxeO?Wv2 zKXyj&iwf#gXCep(Z2YVcUqHry!S5BgVDN692F4VO0hncA48gF#;A??sFmU>aFu@Ff zaR-B+F4}+z1Y-n71I8H41u$$dpqG$2U~Iv_X)K}yV+Uq2n7LqRU~nPx1cNU}L%?hY zGXu1T!6s0E`Y8E2u1f`2)vV$YC%mz$m~h1p`N6 zhzXcrFjin@f%ywf=L%Ylgwx@WK!f4G|IuduVU7I%ac!oK_+dGp3O}c>+7L3|wXS=oy@q$Xe{|h`r=Y4g*sUM5;MlN}*r$ zk&|(*1ihTI6QEym`KyE+BqMVoQpnNE2)KJIin~DPBHzM<<6ZC7;wj3a5Q&+rD3w{$ z*;Rpgo<;&7?Sm@w&qr;DoN1PxPUlIu4Ew&QTV{roPl$=>RE3@nQHEjERhj9xcm@~r za?_o?CS$3^O$~QWyr`B-t$&3aJ?8d5edUu-u+|$-Z2Yrcxg3!_lMM_*qzF zM>64_-;-&-TK9TS(Piew%z;SBP6i=It)An<5>&4^XN^%QM0tAhEw9v!vrFbssU-8W z83gWtGT*CcRTwqLmDHe&w3D2h7J~|m5H>1t7IGNgmk`vE!=RjvN)q!ZnP+Q7pogsv zwSYGJ^FEdSL7Szl-SVe0V;P-eSzV}hw|cP8o%_!t=`R%))sW$5NSInHl5nkr4@ua` zKKTzjSs0R2Y75`C{vTXeJoES$R~E2M`+v7*RsYlK0^jNWpyHYzSl7t(Xsla)!j`dB!yCqxu>C z^kI&BL#dhCAi(nd&yB{^8Xwm8QyYm_$`2cjCrU{YDqz6U*#>0^nm=wMp2O;WItIG~ z2^*FA!Fu}R!v5n{Kq;9;_T~P$m0(6P2q6+5REoi(`BwoN$W$>bVZVW*$gj*u9;}={ z?kF~q27+lSij3B8a4qZVWrODLGZ`zf)68CDcOo1;eG#chjas? zVK;&AO#&YxeQbJcMx`w`G7R4^ra#f?WxER1YBW;ff+!0V1Lrd z6qC*=eSMHb$4FAJ#PT|F$zBX(b75y9KrTRzVdiysS<`i-&t8HI0Egc&Yw&f6mk%i` z*ycL&6*SXzNPd=yAp=m!^Y-0{7ApJ@cm`Pw^RF>3;u?!{{LY=GbJR@op2@i;iopLp$Mq|IT&K^ zTYXeJDDe*gs$0YkFulb_iEq5YZvn;(42J5~A9}T8hY`3_!FZzNKW>`Bi{*u=uFtpM z{{9EJnf9LKKVEyI z+S>ZB1m?bqg4?SK5S&;->z*7^3A9YV-CwW0YV6bQYCgR%z^5ClvWU?}mO2}-PS z1_pK{LZAX1Pa)0QIt3;11h@+ zDD7R<+m&f3`ORAgROg$U2e=6FR0Il5ABHYvV-R%wJ-(ZP#77B0@NEW)?`Ae`f4dzB zeK;Wl$J7ZK*i$EDsLmfxf9K;TWazYV^>W3Jw@&J#v_F<@eeH^pf8mF|yvtf(2koXr ziF-~HP@UwSm)*e8OIs+!6(#<@2Y7zwjuL;k4OMt~6t9>w%KYQ)=dazwIy=-7gvY_? z*>FHl<>>Fsj8szvg!EoBd=rtamI=iqEL~z2874GJSIYt+5Y6mSPmEdDL3*sSRM1Xkq7g(ZbbWuaGk{$OB>pV8jD=qOs4m0o{Lkb*=g zg}75}Tf@vAOlZf*mhd{L0)2{yB#T|1#Z&?`SFE)!hX)$;DlJk2yQW@F1dnC6oF~!3=-7(UAr>AwXPpTxx!>m+nA@t)K*Y|<*b^vF2=ucK)+hwc40@I#Qkq)r z8+s|IkXv5!c)qFJ)aI&g=$0rshx6o~gvvMcHE5b!8+zD*q!5Dx7p9NvI+KY5mr@&-gTGa zOe_2EI(GSeIs9aUQ*GRdtE*EwLu~D#gytXCJM}S6R1B-SOzh{R+zWS|Nxl31&v*u> zy65grOWG&j{=B+SL&$(H?I9x z1>QM4%LkOBHtR$Uepl1X613qz@7-+cWL%(sC)u}Y$LKzTmNb`qkA277&IIcE#!f7< z(BFBkWpQh&|4z^8A#M#pS+WBPaiMKp;O#}f{iZIhiR_5)%8;D6;eG1Vx`8Q(*Wd~l~(53C)u6%th z_^0h>R(yHm@vAL!8~!$D^i}2D{Q*sjZGsmrZygUhUKz9&TCywr?A48JKdfJNEcDS$c7`Y*+%Fm`` zthCEyeL2+qER12_%fe6YJqBLN2KR{;ea&3EFeYi!d*+U>4g|EkHQfH~eGBhI*Wx2< z{B(nRi)?;$nN&6&{?tO$Riz)i;uA)r^4BkfVYX$%y^-IIC%e|Y+A{xqP*SiUyZ5Kb z7n@fdY>5eSRJuEI+K)F$Uaj82x6d=Cqv?S)vI!IKz)OM>-{j?yH;abInPA-~Q8Hgy z3H4hx`?^F8uQtfO{Hn5Emq8iQtAf)WT4qn=eApj=bv8dBiNBQHv6KAl9KqqcAxlzF zsi}~2=(VONvP1Vt@+rZBeSAncKXQENJfX820vVLY%+7s7P{me5f#_cZXgsOp2aw1xf9KvQBMZ*aZO!|`arFKHy3&8Jxun@X2@nawZ z=}3#@Fc?+cX-D)BH9h}49K>Qqp9QZ(b-(j&?tEg!N7cYc6%W z^ujx1GZTt96RJ8EY)3!A&!Q-rK(&fv2wpY#R%_4+)k&7#vfc^RCU@_wS-Vx6s)w_u zPQ)mhS-TTm6KTs7QR;iFtM1MdLOR(s-F`1+ME`i0DpJ=s-DASXqM8h^*iSM-7z{}< zA{HXvdRYTTN8#P8CmkHrv)ME591km`%*dbblb>j$NQzdAfyO zQ@`=7USIpXI30+^0nxr zg-zQhc)2AyqNUzsw*hn*9RBs~^vtO3?TKs!A7}U0`B1X)YpqG)>O)K8l(=R5S{^-o z^4g4qE%P~tbsiVG*!j~aon_9Ij&`j(d)f35*M!FtTh9D+^4eCmDgAt%&SMY@Q-|j7 zyG^T}N52Sr+1HxUs^GLKtC$WYr)*4jKZge1)qkh|&U5^VfrCU-hm&v~D*@AgN( z*u*w9nHB}dpz0fmsu{@-)6W*V%{NPmxb?E{q%8epPppWmSDbm)u#X(TuNrpOlJ#_Q zX0!XadQXzJ77b4O<-WQ8D^Iem;iB_9P03|GWwopO4l0_hAEn|dsAPEkT#KOHqYsPv zEG|DBomc48bY*bVX}@3en&DA%jCyBUtmN*0EgMsi6KBIuSJ zW6BWb#^ta)?gD8$IDf&~QF8Hoau9UBS7TeJbq9t7OBY z<1r$G*}E3vf-p3E^kfXH%wt>L`P(72HhG-|s2G<}rW{2@q*vBZxpbP$aQAY+21=0F#%TzJREgBEA z;(1>1~?mqe8m^C-R0U2?sME z5qv*8F^l4~`lr+&KV6$xqtD8#QZI}e_Ai?m{Ar9){~gE9FuTUSma9L1{`LFy-$J#D zj5^Qv?j5xA>A$e<`Rmz@eOa<}8X^7Y>Vuoje;lY6Ep;?^`8vYk#U;OO@|Gp7znQ;3 z8MnF>;@HcbTm5Xm1`TTKVm_>BQ9fGtsom|o6HaBe@2VF%JnB0!K9B~hKav zIu&u)x^FW1RD^O?+3(M{MSP?=MWyoZ*_kgkmi(6JT6FO0wW0Gv>v3#Hi@-{ghJ?OImjW%>^@V_IZ{42D8A}qK55hvsx4G z+(?YU*##HeU7G)_OT~9y4N%0i zv99OqTtx>icr~m}tC(H}i>j3iV3~O)aN#$*^W{Ijah*}I8v;L5wvNAprkHVawfPHZ zdT3CmMZ<@izl+^ifin^9y5oZQMc;{}z!uR>XCGLajo;{h4ZaySv*jP_`Tos<1BIX6 zIT#d+mjFHo+YM0b^?r@x{i#-S?wfUD9q&Uie`{=M$^?A}#L z<+euoEjQZB{eN7)?55FvzuOt#vd0#^ul?-7jrLrJ#<&)xqOGA(HLv>RoYA1q#?3b5 z%x=sWk9>aLzoGst9OZ&K3~&FWOwJ0nsdn2iw0_%-TkrwSz#T26Pd9$;`}yTsW#yi~ z1;H?C9c!#P3qE;+s%2tbT=q}vPaJYCyBy!JsPpQ+J$eoEoSe({$b+xUd>I!|wk^2D z`QGBDw1>yQ`mH|bp`vo^r?i&RU-t&IHC!%rwp>SEwqv|*^t!w)6Xr$^<9Lwtj)UP3 zKNm=EUX1G;&*<6z+eRn7Lhma3@?@g}3satP(iUX))QoeFv6ijAao48GZ~Hf|wropV zpXl0o#ih1#8|>PSj;z1h-<24>Vo9Q?=xXCD|FjZc*Q)9_#|CL*bDp!TzZpI^r=s3E z{N}ki&;8R3ZpIKq0bre5R9$|+b@}&VbX%Tcf52^F!eBj)c^ksuMqq9!j`$Vf70k z$KtCrhkfIIsi$&JniYD_z8rJ@pkE_-;__PifKWT5WC5LN@D}P$s)zg%)$M37OZ(`cY zM!$;iIWT0nj+WWGibZ+RQ@v7I{z=74%Vq6}p%bq*8y=52dVLRh!jsvLzS^Ae1J6Rg<;uf@bL;bA?9BPwaNqso>s(z|UV?3B zjO*u1tV+gyaH%+S{~hcm{DyxViu|EG#i~TUYEOlqeze8*f^xQ17EG4L1=C9G-fSc@ zD8bz^on$|&J3p-77QXJQ4hQVOtlLhne)FS$Js7^M+%K_Y(Bl5k>TQF|{jBQxf^3Wv zr|0*?xN`g`ndIX3&>`0w!>f-@Cazw5V)gtNiE&lK3u=C}d*z%dIe+x{nCQzZHKnJ< zM41;>-~6h~wQ4vPaL;h1;^oxbye?gGe}lNnVEQPCZPLe)42TuekAtCJi@NB>UE?+v30n)>z zI^=LdFbIX)dPu}#5mA=Vz<`B^NH8g0oD^k4aYP}Ifhe3A6tEfKLkx&47O+5$vf}|< z6KV%V0Kr5mhVam%Vki^GGvcJ6+B;A+IA*bdB`62|HzZpJ}0M$gI(DlF(3k@RyKjl;w5g|_lZ!iZ%pgIZ)IfJZ*r0*fD1QI7waU|zl zJK#JJ=u$9DfujDvt~F;l-rOvNOn~Ih=w(k7;lV#gHWAtf>V!EWXe#;%&@rcxNzgL& zXbzQypnL;Vh5iZ^xxs+8GaxsUSdLOtV2KEHI@lPIMFjZiIxdPiL3^QqVMQ>nm1qJw zi6)|cR1_0JJuu5qXaeXRB0p0AMd%}jEFz{b$4WERv(JVgM4iVxg3oArwfDkNG_z_a5FDA8zm*JyS{LvVuRAV`liV{7@ z{`|R6E-HlNJPCT30dgipS0W^73e?5~npH#v)Q8m06XikUf|wCOw4e$IAK?etAvn$# zZ6g z3vGHxC>mu{_IuxxZA*QGf2$K=QB%pMo69;kP-BAF`$Q`43H5bB6LJp z(sOJB1V%N%YZ^%xD*erJ8Zbz4A}p$i-ZJ!@=~ zpl?#57NB!#jZkW=W7b331=QD)&EGApLy+`FvdUUJ_|Hdulq6^gQOThw-j~-iJZr zB}NuodXsheH7J??4Zn!o03s;F$gvoPf2%?&0{sH|Fi?7#0n-5)Jwn2SgP$@<-8p__87tQ-+{kyQB}%>0~k& zBv1(Igm~VR?~j_0P{(u_PteTHvD^6oWI-h?Jy#l+1j9uNt-{xZDx&(e9UxbyxiHc& zHYkkRcm%1910{}-0$E0^NSKMptEDsTU^=t=+%Jo%k~z8k`)4SoM7kBKhZoHcVV3{|Qubl4evuv-;c7 z`Bq;Y{UDbg1nDm&vYTPp0kD#R53mw|t-u7Q#zH=zN*q4I)J;CZ(II>a&OU?5qnd_D z&{;zI^pFpr5%*BQlDIJ}0MHH^#{eMPJSNpAGV}usg&Xn#P;LmBrconih0w$T_D9S+ z9+m@&lo@BSJ>&yE#5e?S8P-e=Ll1ECFf-7>sRK}WRB8tyQ%HCg4oRUQI?OoIv=Eva zVE{sc2*?dO*h1zed@qA8BAJqzn<8l;A7CY%kPkR`fIm#(T?_zjq0fvY{Tm?CEdWvf z0fDZh)i3~+9)yDCV5&&JD{Xv8ZU+i#(0CnXlzKN z5SoJYTxqijN3iqke)~HFABW12eg$kj=@! zW_VskMM6VqClOuyO^6ATLt=%{-XNynu*EA%0hkKG{taG8H=siXkeAF3C830Wl0$$~b(k2(M?mWQ6_yDxu<{4>1>AMNJW3MJXQ2KTwkL=5lzd^ox_8ih} zr7xZt1IS4S;B0W%ewYAILQ_~0fZ~9H>h&_r^ZEw`#i33Ckf4YE2}|;ygP|`b%>GA( z{ohkzLI1J}(}5QSKzC6A5ABb3tHDVJ13v;vd3F!5@dOE?=1ieUiX|il%|il-*>n=p z5$OygE)x)eBtlLdK7BIa5{`<%L-9P=Eg^d9v(iXAQ#~D$L`C9KNl2{%(!>8SPpsIB zpr9gLfI4G72%`2R;5?L^B?zTJ2FW7Sz#1ekLxu_jaE;glsQ^43*^Jm4brS+nPk4sk zEUG08QS07LUK%MX*F_Y*un1M(4JwXspfc|7DUP&YGc1cwDjcTM$(fhpoT(D8jG_n! zqqA^+9N}QVusq7Kpp?)o9l~yn^p0Z<1 zCrqdS4rCQ^6p;MY_#w29!#-VU$|;`1{#~~(OEHu?kbG(rmP_u91p1Zv^(m*~=FaqO zN_m(j*}n^}?NjIK&onmnYPvEWId{LF^oUSnaWH_!d3lJiGseOHcHR-QhOh#-E?Qa}YCam*6<(|?7>gfEuJzU}Nhp)Iujo)R?j{!p zv|Nhs+Hxw$?26eH-}brYESoM9FQ1wF7JI`TV=L9>VCC@YFP7$d=3R`l_%zo$`BH>K zK?3U_G^WW;@8pJfhv|pzA8m-J4cCUWHOOkSCPUl0f^33*R!rdp0C*9V=(42kcvzIM zE4#W*57Etv?+Tbg zZ**0*UyaA12S8+V{1=t&R&8V9;Jg2D6eo!TJY1Fm1)MVYy5fyo>|H+uxF{i@HvS}# zol4b)`&^c_g)W1v5Z5^t-yAv>2O03J%{YZxoO)>6W}Gbe-0dT#;-;ui@i7LKfh4Ox zRunjPse;OYwX6;JNbCxSf`Bv-O91AS%n&HHD-^7;c=l9%@sdqlblby+@q}F_hw;Lv ziW?8l_ewa@JXHY1g`2n##Y?7AdAsO*@u4DJl^?RwO^pE=oS@J zavV=}H9iX3VO=`JahA1bukA8w4sW+CK_XC{m!7)rE`C$|LBg)EvE)-qrdM5Ud<%5r zP_aU2((c_B6{kB>-eg{!T+d}@^6h=*Ue~MZe7?y#;%Vx^hmqo$0gKFvRk~c8eRcce z^2xN4%!xSb?`ew;!s90f>n|-Wn(I2_q35y*6(>UH+QT9v3zNU4noTI`4R(4TjG8<1 za>lWm;yCLc)?RveI%2lT&!ha}xRd7(!X;nmenU#K|yLg9{ zzJ2@`RwSRGaR6(3W z&SI|=80pi;?_cj2fM=e_$*1C+`|~Z$u5cYhh35%w@$`L*@0pb>cgXj?W0qjIYoGQx zXjZd)@8~3(2>pEnt@-0OIJm8zTe|}W64I}(8THfNA~z-i%qMB zS9cD=-QyywgzF~9v;76|oK^4LCFNXdtgJ#`I1WVYnUNJH)LY z(+^KB3hfxnQ1KQA>?+Nq**C!u$Bri#MFf-^IB4AX7CM8{CfF|<1Pjk_^`R%E#C_r4D4L;evP+6~qr$5YCt$aa#e2pr<`-&YF5%2d z?5h4{Z*%1y*LX=>&n*GRvoNkOFmrG1C8OEHLJN!R+1fdlY6DnI}raWxI+zK(Bh(jE=wZqYdgpW@y1{D;(F6fQ$BA~fe8jeNb zKAt_ytr^EW+e4|B25LNt$@kQ`yEiG|r&l1LWZg zmn#1~-{q8mwX*Zn9+%A!2+#QK8=A7JFn?mx>dwvjE>%YU(~gC{copwjwO}QrI37Oy zDV`7~0!kj%0KsEvXTrfZmUtN~DAg5mn=hCZF#g1TExf!fC*C!%+X}lL-=zTT-1T@g zwtKa~SgA{(zp?zPvumusaR)7tkm+A@XuQ-V9SDK1s=JYKz0{@ZXyYN9gc0>o$Vbvv z9<}PQNg!19O|cl*yxJfu-ZfUByuzVpRSx4Rvnx&giQdsu{F(V%4#obRQ)iQeVXH5F z!_obntJ}w&=el)Ari>+C72~z|&?18+Zkwy`uRn3aWq&};6{yAjmI8cPwsv+UFy;n8 zqbm;?lvvT!T5bPFYj#EO{nXplcwEF_wFRu+QXOze65O}$#{$Ui`rqx0POc8H9>FhX z^#h3hdQG@%K>*S3106^t{oo3y#!E|51Q3nQF5!uI0mQi?g432 zj?OEj;zHagjY(;^+&pT|)MkuWf!>tu6gQ>NH_UCiXme>?L(ftOMqg-d!%My(Zi~h= zYUp8-tV$6#ddE$gBwBi9{1?eG|H-w31+rS*2TG1cR@DnB0e)us9c@bnC{rs#+eBF{ zN3m39l}g+1U3c-(FKj$|+Ofg<@b-(3$F`hb-FEb`%d#`!Z6WPfzmJJuGG=RNTQb<% z1lnPP*NL9iqv7qKx+1$w7|Adu%7%UpIu&96H)U3QQcum^p{(R3p@NYcKZLjcRAaPe zEWACi{aRkwn9xuiHSGKXPD|U$R zFi7!9$>v`coVoesg{{A6V<7PDt1kjpR6Ozh@oJ^@Q`ar}`G9m8GG9;0{Gze^x7hW| zZ`cRl5C0*4_{zo;Z?zk5tZTP2%KvuWkxeH`)}0^x{&VXaU&RdGIC$#E^(R&x@jKq` zQgyR2?1$8yRX5t7eEsA4gd3j)|L_A0lHiN?)~;V3y`sbTrSs=5j-7ki`JmJNBMj!8 zA6n{k@|S1fPD%cDdo4>=9Xa{1cre_zBGGDv#k%3;P~MA8U3U*xB)XnI?qzQike6_J zb#O$&*rws|PUxO0$S}^fF1a1^%hl?gYeY-^?bhe50YE@s+s^Y{m0_pbc7E=UL$?Oc ztr`zF4L9s38pFW1@%H{OyBj+{7gZGW8w|x9-M`8Dr=6d#syH;xo)dd?^}3{l@8fZd z-~VFoJ;S0*mO%fRp$Fc9Q3nLXJm82Z2E>G!K~XSb#tfJ+=bT6?X2h`Ss*A3xm@#8q z#2i*xb6VG&UBkL-T)n@V-E;1Fo_p^7@c;0C?uWb2cKLR7b#-@jRdsd0JulvreY>&a zzUf>Q+vvZ&LO-d{v~=ASx64K}tGG1m>c&gymr{>ptmPhpyvo-)+d4Ol+f;o%truwp zvR}iF&HX#e#p>pbY^VBFS+dc4!1P6d*P4_sI1`>%k>|iBEjN^Qb3Pi^v1w`U=AMiR z4Ddd^z>izJ-qZUO3f}tD1&^7%4rK(CZMv#=!;7no1!oQnxVlP6?OUUto1BjxvNbQC ze@3Al-sP91ROqy`{pkz+#`d)L8Zs|sXL?q6;GTrivGV4f|NQsQtoI*x2V#f!|IfPv zYub3@(G@K3eB#nrDdAm4c>Wy3ORFB9g2e@kDqSN~Wd+C8@}C73raI!yr_r zBGPrC2`VyFm)bgEo;gKJSile9O-Z&$i)v^wcqQ0;lRRCEl;t(OKkrCeTO!hJc2`?u znk~{JDAdPLD9vKB_}SBK&8gyHvo^FUlTkNAdhXlWRyN%lS;A1%;%DkaCskyUC(uNf z=tG({BF$!p4_SDo+vwipWJ{>d20kLq!2A2qG%H^f1-Yzri3KU&jMN{gtkryNbsj zpzKUWx>cWKi%77}^VHdGunIM$=?YD!j)8ILdCx}!(h-YQUTRtBV?+~ODBqs)kdbJN z0ko9sT+%UEYp4TL+@P}?Yz$K8yIE8EG~cJM+Ue}YeiEt2?pTP;yc^X_~ zB5aWfw#XoZ$stTT&yZOSMzT_BD3DezQUosecCCFQGuyV?q_5M(v~YEkg`IXTY`cyb zC;~s3E8~2nudnkk##llP?u%{uwmvkLhM@C>>wWd(icX>8m3sO7e71SN2sl4&NiE z+d|X*B9m6D1X0fV7C*iZ9O0oN5kMM8lFH6Qk}lMsbO|<-&KjxHp+9z=C`?59>Ly9H zz6mzX0|RlkwO*}r$ibduWv);v;gjT{-}@@j!^VsScp3`0TO*QC8)fpaVG;HOE7KU^ zX)uObBRu$UwV`RKrLafPFKH^$Vr$z^ADLcmU3$u{sLrOe)!u1p9+8j<#di0MY;6n25X9&xvgzmyV%PZ=s=6+|G^00x@UWWZ>pd(HRzm?x z>9d~bY?4(AzZbyJBFZys)(C^|b{{8qTco$4f;$)=n9GqbtZ%_V)?N$Bf5wTRLUB;+Ga55(tX0gi;u!7 zF_pH4*j8($*QrQXvFK2XO&Us&qEJSMD@(GOELKg-%NChrWg@gBY?&$Z!{CruD1C7J zuZP}3mLe-(ueCE}WEr&wDx~!1LF()l1S+gTMb0{e3xR`)EusMhI8a;$o+5$XkQFWt zzSxpY4{OGWVHj+Pk}-pkk%OmaRCn})EYCC+nWRh%AS@vw%`beqH8P!9F;uXiQp~-d zKHX+SZ&G#cqM8aNh)B246}&{`Du!xnwLpZ}5Jx69DL%xG53vx1I4t>Z7+`QiUMMtL zZlDRAn!$))Lvb)ThcOHEF*sRb-&qSKBp^w&!fuU8*Og88Q<}&Iebgh}`F@f0490{+ zvrl9>g9}q)9H}BbXV+H|=~MX1d+ZSl!wlsG1OMV)rg)ew2B!q0*=(^z`q>v-XdVCQ zW%e-SOEA_6LL}m)5xdP^F<;S8H=3BiARU6BPbG{#x&WA&y$z<&DHY5~tY7wrA>xFP zNPxJP=EV&D7qf_^{{3cOVyXZ6X8)qi`$#GOr?cz%r_HYS_nZCy2xk84d({8;X8-^C zpIHB2|7QQ=UR3hmZ}vqc|LdFmRom+M|F-pSz5GAC*)Qg0{nuTS{8c4*?=d1){uw!9 z2obxtSCR5w%bNVhpHiUM#|Z$5TYwNQYogaLYohbdJn$dgGWPuELwU=Zc+b&$FVg#; z-Cxf?eLvmd&oTU)$A7hvI$ojw>QRXivZo0Su39muV$Gn6m4p6gHl!w5dbsNE8!`Xc zN4Ni|;PX$$K@P8ZN`5Q8 zT+q~8{U~27cPq-1GuPrlzU0YgG<~f)r;=QmH6`#VO??<&q-+?nI;L_Q85^WATyJy| z4jOezcH+yGhGIGw7ky=ao!f!ponI$VJt`993kmba8)v0IE8Ds5yxx^_=8l+dH zqJj=kuJUp=4OEL0s*zFy$_w-Ah8Ins*=(oOvv)k&wXecPzdX-Hl^ zf7(;$GIX6Pr!yup5+`5zfGB-3Nk692QqvQ1b43JHgi%|$vQkH<-|FeMDc6$duRr8F zuGv+pnc(JBRp+TUg;g^pRMh!)Z|0&`2@KF>mE3I|(oZVS(ND8-2gW!AZk>FaziPhN zfPDD@Rj8q@To<*BfIa|9eH2?xJ~q$D^$Ch$m_i}sQ_gxru+Ax#%Z5fn3q~vhY)@hw zrt&)fv=jfKv#x!IbdT-Xa~NNGFK8RHS?fx4a@37m1zRh<0nzrD{ctXrD7)-H7zb~8zgmh`IXZLX@l$k2!lyYO`0w{ zpQeg5Gg)-)@@d+`4Y|euzLKb^d>cVE&Zn7w9&j;GAEmrTm^Aav0haQ5K5lnyC$;ZC zqxO49^PZb5T9d|yjaFR>6<%GhQ$Zi1Ew$f(=t3BM>D&-7xe??)<DC`S5)b zjSnm`jtDY#GW3AkFPd&Q=`iUug~muz2%lxqbZyC~CQ#NR)0#AvlmT*ZGgAPcnACJ% z$qZ}`#WFJcNy6hC=?dv7g)uGQHJr?5(q|BsmL{vN6q%-^C8Uj{ixjv-p`w0dx{?-? zHh{FYqM-;fok@#GGWf*SrT`V*h5=4$%K*=SW?agp9sMXsCW}mk_GHGB`Grhi2a}Jg z8x8F&FfYLL=m_RYCs?fM47l9|P_ZinZz$Z=jd?i9gX+$MnJS5-9i%*v;W4z{H;xKR zdQxF0=$oKd^rk^RK%+MFqtZhj*7xV(0mag1^8gx5A4IKk!vMa+shUjLddjMgrp!JO z^f)v~^_mR$DHYIcF`)Glz@km4g3p+n6=LI+(^%><8_F=wWKjzfC@#UDXi(*+01lo- z2j3mOWeio(N1>CMuW^=+45hItsN1BCw~ax%8KC=aqPp{Dh-9uylWKx3wT1a9Pi7ct zC212$26=+QLRrkFD1*pxI|##8sPpmAeFtq!lKMM&s6jD}Ip$VG5wVMD>;mE->E9yR z4YDu@u9)ctk#0aaD;wlK9*XawcYXIVqzU@~R^&t$ z&A>Em&74l}1Uq zh{ivn-VrQUd*=7T7@(e4B)=sHzzt+gZ43s?szve0JHHu&RLor-Z&+W#0!gWvJ+Snr z;jfH7>M`c5Ha4a>N&lLBCj*aBhQEv!Rh#o36>I{maixQAsaVvFwzm}kRmK9U1!IZ> zJ;I$aT_%Hdaz9K{n>|WLlY@TEgXwDFsBD zAbwnz24dIAld#=kI173vLe|<1{Ta4LNAfr@=@%%X-o2r$&U+BM^}kbgNQQsOlYCt1 zzu3i$M9|BTjwQ1|CEqM<@e}Fdc*d@E=v@v5TNX8>iFZ%>gMRcDba0_jOg>&zjVvj% zVFg#O+fID_apd`|{#P1#$U349Gy}Mb5}QzBRWXWTPM|e$VkcYSRKBzWT>>RF*L|-G z(&uQqrWjTudP>{F?RmhGaU}DSI{bmcm=y{v|LQPmV(imS2F! z;fsJvmjE>`Q&IfOCo0#!g<|&Rq2L`fzAjtGC~en=nRK`HCum21rq73*($V`InO<+F z#mMfo(+UWF0ihzyfRg#(*a;@LI7ZYGomOK|YlXau%X-FBLgVqIK8wejVmxmRyI@k= zkPDN7%&r)Pw%+L?qDb0v732$isPa5UNe{ZYmBdP;b=Z9iAKH)HJmxi@ld9sIp-l-N zM1>Tw(TYrt_QE_CHhtP|1lyK*Fh9#uRrMk2=~fEc7{ra;xJ7s{+Mrd_TQy^i{AfYb zKr7^3usT(|5}-_Vi0AtDM0Oto$lt{%wOM|1j6UWA!EV(F67GpMf9vPNI2!wf!J^&f zSXk^mNT2W0*XzCMhSK*T>qU0t{gY&$JfJ@_^zk+HG1^|kYHA(y*i2vIYxsodeYDw5 zM?mQfzk#~8#-pa*M;%#9GlNd_MRFa*w{xmb zmq(2^C7#f+C;BgNs%WTHn>gr^QzK|L>7#*ve7pqw`1Lw)|7U_1bB7N{>*#~b^$S+0 z%Mr2Ll*Z`eS>|)nROYko+z3Sa-I7SydoD#B?#lp=meWv`PA!a9ZTzs7aN2UW4FpT? zf&QZJLi;73GBsM3egyD`XH2&Fv%$5VK2?j=`+@4ZxH9xdQA<8hhikrs7ULncrSVm9 zH#Pdg=VDR>e5&~kfc)Dd-vLMVMPO=Ee<~5Za6?y9eys5hx3zaQovm1Gi!nr7ki4A{ zWI`c}#{LwHvi2-Gy?8F5OB!Gc1Cg*vb^n%H`(kz*qg2`u3g!$2jFo7iNEV>rEVw&9jty#t=uq=`jW0lj&z`pntE*l%uFY!RpM8epM$# zO+>1V@tcGcDpa7Th+hgtHT=@(`3v7gK=&fcfL{u%g0bH*9-H~SLL=UeTi#GVM3q2v z+&rh_$;`(28Y{3oa|pNt@Vz>AD}rt@4gnsi8WpaOP@Ah!bcoS^CjCXVi`7|857vP9 zANWqC)u!A;S!K#9)@Alv)dzjHF=z?R&P32kO+#yAf|?>r_+SOCOzejeP<({qVIV@v z6NgY2OIM-Tp~U5mz#R{yR%Q}`?E-O=jMF+a(3?zs(pZwjX}?jRUr%t2{X5*)3jRGx zdQQQp4HSIgVVZ0{z6HC5luLR-k`aY$f}%Sa0wKpxMb)QlCQM`9aCEWD7Fv~9rAiik zeY2fzPS^pMx|hDnB7lHYlYJ1$fZiQ|s3Yp5D*s4%L(2Pu_Q|Du91k}RQ{G%$ZYsHp z63u0vhPu5dZ~1fJUx9y_2Y!nzCj;?is>DOU=|7loFXuMlQEGBg@}2zwzk~yFn*bIy z2P~A;vO_;WHm0kF(NAsV+_uCU#P@2?8n`aRK`XayG!zUf|0!n3&CmghTO-Si4;H5i zxd03-Cc3hav@lWYt>|q}QQ25jRu+9#tqfJy&h281QNcJWtV7WC}fE-|NZx3dBX7t}skfvSc7$=DLo-hl{9 z$3P05EYfIYkT&sL4KOS5Ioi&~ts!}Hx;?!d@u>^2wHZ-pS8~NWH#j^Fc>PoY@Kd2A zDh(!r)CPpir_$2ft3c0g_ZoBsi3E3idIP$Z>30xyMBcIi)cMe}MZbN)i>)W?_7XSo zDA&@MxF>sWC=B|ZmiFd)(2^RAfvJ)r;t~>HUG<(!HQc{eHv1-i^L4a7LLa526jspQ zF9OBPJ+cHC52nCk#_#)Z87~f>h@2jiAy$`)su&60&BfvDR+WU8;|#YVXYm#}fPy2i7Qzx|JJH z+@bO#0#t$+6(3CTU-st|Kc-l7n)4f~*@IO|>lucuRc)ca6(a#|y*M82Vefe?4_5hM zUD}RXU#NS%Hu^wJlNw3D%P(eJH-uq4oJgksv1}YFZP@hBk$)N4<_Lw|9Syf$r8_ zdU5>lN3J|DHeYP7*Da*v5H0lLn4J(b7dv^{9&HW1% zWzA^k!3a8UH@6_H4;Byjvz{8Nj)})1or#!aCocy}yr$K3KTu;)7ySKh9wu`(qONym z7%O4`Ss0_NKFTMK83}|shBS>dn^%Oc3GysU;DoX#gXV)u|P!KWPuhuz|=y_@t4s z52D3WJggjyPz*znhbQ5ht~RLdOsZ}G58X!cKy_7Z6c4y;$Bwr#M?ShFGV3S_AA`BB z8B6s-atPF95}@{E7=55{L;}^G@(_~9gRH?aJ{D*-FPTuI?lg!Ci^AAIvr~7-xi|mH zdgeV7iAiwQlQa{=hFOg6Aer+dLM63(Axus#g2|3=0FAx_oZ0|bz-pm+OzYY!N0m+` zUurtwjYO?ge*mQJr+DJS7|dxMMkHr{THdNST)^4yDX#b!9HNmpTJ?Ft_yV7Tk**My zsLj8DaTD9z`T?*7XQrLJ(#sg{SlbI-MVw_ZI>%An{a?w=cmzdRJ_ex0YOj@y~{KX|xyjNV*1PH);b?^BvT#R|bN@Vz;FRIS=mzhwu&DeOW$ zj0fb0v6kZ!Z;Wt+NZE2KQ(IR2bwPlNY@OBHT!5GG3>-Mf80wh0o`V?IHiWt{dj$5s zeh;N{a>rq%vk&uFA~%6Y{xl%b$4S6nrvU3@sb9@ft9nadlyw=xCAm}Km_rX!B(AC| z>$@B4yLR~VG#uWczsOs@Fyc?Nl~0#STc>JL{PDqb8hG%KVk@$j@K*&t#!+26G>dwk zbXjau)q<;b`cOLcjDnLWKSNr|FWvfua80X5d=ca@I_5~d^FvU^G87Ix1n5sCZSkQM zOza|gj5@O#S7kiUc17odY|G>4W{&L9GQZ}SUhQJt!Ib~yw*Z&-Vb4MyOq z+kGQWe(B&$x?W~Hk9Bf)K%H~?AQXo>;fKwo*G5O-Gjmt=1QoGz4)E|Ganps?aL}W^ ztfNnt)>HLz|8wkrO6cmg(uc~-vo1*V_WOsvko95VRWCmS%82aZ=;$aCL5Rbn*ZPXK zv>-ErB=IzHv;^5&h^`;+N}7^uMv{HQV?c@C+$r{-xC71R|WE=nPU^ z7oJKDRA{-J5LGrJYZn(fQH0zaIWOzsM!!nSUTR8yKx356aRmLRJB1wzf+|GBq#hIp z+>5>H;p?p{28`S5s*Btpb&bFw`{h2rHQ{?O{;=(ot#+bgID%z_!ced=Uf%dg9!dNGi5IsTFq9)CDBBSye%;Yh&zQGhMtJSR>8 zWX%8s&7;Qd)e_*U1ljl1bwW?H!}%m|9Lh50czrU~%cXU!RYiGIV;W1ecK_%+;vzBp z5@PQrY*Q6I0kzD*YS8f~nWfh}^kI9TbOhQCub=WWxvJ-()`+uZq0pX$r?a`sA*fLS z(yh0%;h%nZ0pasN9X&E1;IvwQLOW2uDqXrK{$Q`nf-FJPA~CMquzJP-Ev-4HCXS_} zZ4261*%CUhRseoB&DszPc(wrGu@Eq95e!%+l<#6_8!sW3M^{$VYit~%-b!2)Ch~56 zG!miqzDeGT$y2Ex0M+pbGE9~5mBvlFCRk=qCQN(d)fBTD5y0a?e@sD^5VeRcj!FUN zSgUT9CpSy{+KS2mSGFoDNWz3??*L7OeG`dWi+-hocuqtqFT=F&b={1qj#o7=oM6#{ z_Qasnw`jMl_Uf}Uo#|aJhHX&6ALE$8%mMMhI;S`^)qdPR75LN3=?L)lE2_$l_v#-o za+h|0ly-v{mUcongBbCL_M!*_FXJ{;pX;F8#BX3^gNN{_T|0o0yLZ50D8FdxId=o= zy_>LEHwLbwv+2+bdi=G;XIx%$romjvC*Q8VY!+h8r%&BaQ zR+EnKIJ#a0o&A(49w`#XCBcnKkbS(YRF`*9C1n?MMLMrDhL{ITb8WBBQeo2}L@`yX zwUp1uq^BP?0@||dsmMJ519q<3od&d&Wu!hLQcZvTyJu?gZEBT!49e>XpoZA^6u2#7 zv-P*Zyke#{coa8gIh@YGf@J$8%DKseBq~^E0w&D?RFqg^lnm+I{B490uBflpp}{#e zUx)4X4!#y`M%YRw>Sh@4O|{E&Ph%??$xhLnl?L1nW5`Q!J>ow_QrdtKjo81?{&FeeGx@Izy<#iV?*VOgvQEW=h8czWtFp3a{m`GIZOv!NM@Iapb*s+$_j2T-m2Y;L zdQQ!$*E|Yvz73!v>TT|7H+13LHr2Z{k}6wIG$VpW1fT&(sj9j^3 z+gj;A24K-1---i>teOGb=C>C(eJp!&_18G)hNsffutdJfm7y*1Hc_~=zl>+&0Qf$l ztv`w0we1HN8?EMp&#T38MBni@>P>nH821LSSe&EzBVYiFm~s&hNpH2|y6VOVHA05t zzYlOOhaSC(tHE?F?Ze~5i}eteK7`Skzc!EWp-m9?fVo;$Z9E3-D)OKT#~?jF27`1di~#)5=j)jTPGPa_v=(VEC$T^(9`(BmKUx}M|)(5x2c zN>Hw?H}ruke>4v}y!8}S`iwhJQ*Hx}FcfoAmD;_0Lo_D?W~r?+^alRyoeH!TkpL?! ziE9wru`7px=10$vMp{*$-uh5;>)K&`IOX4_eRwm#w37;WCas&fzcEz}$p@~U7fnR08GuuHu$}y#Jo2j}2jdrJ?I#M; zB0CXCY2}=l4pqJw6j#Rs8YDxOZk~_liwWBe!G<@SXlcg-WsZ)e%Uc=k2Jy0LfR}e1 z5bb`4m0XT}jV{T8JeB>vb~sDUe5$YuF+VM5jJH45cDBw()t}210_w{Y#;l2n)rE2l zurd2K?Ye&u)=|F(T{tcSzRZBNE@i6Ik5+~1K+f(^P6^DuR`1aD>UdT9a2-CQ zs?+Fhr}lW{FWY4ipK^OYef;?jM0>=CW}%HLX{J^mXjzUh))$5OFm-<9QK44mtL?b20XQS@ey`F$-<7fC0tI2Rh z=^aa|+h0WgCCJE-Y_qu1~^|D3STG_vcknL zWaF#V9I+T>+(N@*?qf%yEz`(sCS4|pGFnD4E2YTDtcbJ4)i@;YHyWj>MrH^}bnN~Z z4j2}Ur7=xxuh1lVB4-yrrZ7c95Wx$rf1d`8pFv}*z;>Dsb|DWZf2YSg-cpNQk9N83 zlF~lPi`7d@Ilh#>PWtdIrf|$n4C3o30u^mV(mHxCZgP4u^L6w5e#~^sWon#FI7y9e z-(5i>Q8*UuM&Dl$^Va@6cF;ed>}jjVV)n>gOZC+6svG$}v$J4# zEzuZ8CWfx2XOerr5rPGm07=-V z)_HPmBJ5k@MJ|0)7tQ>P)Vnybma9|Zn#b3rBq%+eEkQ#{woiP>Iv(-KG)SK*i>j7o zk<;#;VoOf_APuNr8u4!mB_G_7cEf4+tL(4mw+5xP0o4JiO#*Fii_{{zQmNjMAY-(y z4Y+%wA$>CrT%GH6v6z<7vfyaPwsd0D_cP=mqfVVfVCS`G5#PM^NbV!wuh-w!vY)V# z(R!4V8!9bWfy_UCEH{jB;~N>LZ*O8b^>PrMs5OL6)PZqx8uS=OCrV-Q>fI>7-O+%F zqP=!Hp^i@gcufQpn*wmjpk@VXRw6Z^cF~n6`^YsE`K<%ol7*{v7DTftY`zVZ7C$oZ zD1)-x1U;3Zye$=Me5}q^hV%}?RbP>Sdc2Z89;s3j!#<32a$LQwQ z>o6Z@0qLAnX_NLwbAa&Va2Ez$tHUq`{bMK4CysuHVvl>533`?9MbIz(sPZ9zvcA5T z;W$U~t!#iJGELoCe6@(!n>5{{FZqHpjo%me9m;VQq-d&S9&XpEy|qD?+*%105kC-2 zl31c6`bKJ}S{kMVRMFq=aK&2MvS z{FQh@Hbxc|_zPSDus=0Q0;sf%HtdNk#6sG|z*xwa1j`A4)^reD%AulKoyMa;_g_06 z^lcaXpi1|F-Ip6XYLKAcY7p$j*NJVta5 z8xhNnw+m%QANPQ<$rT$@{bb2$YX*Dd-7r{Qjh5yt)(Mo|E=}2Pf6BU!o=92lk1QWr zzF);7ZAtzF=cf$r3I1Mriq(Y8L?C@ZjhAh|&m3}P$+b*lnh*1^k?p=?3%P+cK;8O* zD%~~$w(bIK+ynUjN9yc`)W}65hmO>`g{!Zi|I8=noFOPG(P4H`z^IadO_6{;cuTc- z7N7=RT~)w#)x2!LQBk|E1QFUF75*aN57&KRRIbO-^^?#1NBU-}45q`OZS_CUqevnB z(z+>un0vD{23ha2NVdln(1R6ksZ=8)^!w=F7&-9H#njYFvG3o4;dyOE)lpmn&)fjb zJ!mS+nT1#ip_S4$vF*OrGL&8ObFy+P({YwMU0iEye`A2TeyK*I_3`SjGPG87rYzT) zH909TH*MH-49=}8EMA4Nk|_t~0$42`K+_3>0ntMM6NUl&#n%p;4S2%}rN(~C2!HI|jQzugKGdcE zYB>80$L*OW!+H1!#y%EqJHVN=|rts)4jZKRDLI@e!0$ExXb=3h#S3HX7jah3$fpWe)KBd z=bN`R$R5Wvv{g$EfZB4CGGv?WAR@^(cnJPqM*MJrC^W`8JakIJY9m{}cgsV0pi)b7OQ0`1m@?CK5Uj$%XW zjsP-`0$h={wskYAs`~Ecaf}nEGNNGGdjCW8?%BIv>D{3npMjN}M3y10REuNE%}+|T znxxM%cg#OwGCSj8TPHJuR3>f-Nj3|jXBSAKYlF9;jYJ#{Pw{R$dMoO{3tfvN{3Onh zwa5{5iTu-dvh>IrRC+%{9?Fm%79tb>(>W^3eWc&zTFq;T>v~-TjFOP;=cq(*>m zT2Uxl8OEh5$osQ8(V&0C@v5U@4YrX0iI$aNGNg4OMH5tyRghN4cWAxyCmOSqr67ei z3fgk#LXGQ6j-^p;6SL?{MKwMJ+J&8yIjot>O0HIp0gN3BI3*?hwG?J*9BpeQ{o?{=BsmMXDxISQj3CyAS?On=(v)%e=A} zCSjZ7@P6FSj77x#5~}Uz$q+~={RQepf$c$>L6TMEIRz~)GG!ylNMjzCAfHC&37JEe z8DUo$^I|Y!7hPn`Yi`iJakoIP1l5`s;*K9Oh+eF?P5JUWln=fCPc^1kT!eCc#58Ao zpeXq(MR3#p)R!wAp)@COMHIO2AcGrqMu~_ zt|u$h$2WjkA7OxIt7D&F5c37PMuze%Die&9&M;BRV6{rkE3j!{)|H(XSK5-gm2uTM zO(5CO6p{}V9B2lEc2VTskmJre0YiHTmUI9_iceVI5wL@)R>xuh{P9|D$ycjp9HWYh zQ!ZE+klheaqZuWan$^Id*Wn0MLN~ag9*#F6rtZk5yy=CzB57#FK!Wywd2K=d_0zKdmNrcAJw^Et=QFWPa< zMGZq09d}Vqv1)6XO&TDcy$oA9(K#(OPTs$zO0A^~+-CqZZ+~w`G}^<+iKt^AUAhv; zPL(>d_I-bnQ&|2Gx2AUT($R5KqV;LIU-k?P&@)x=EVR81Tp`e+<*u{pV4$gX%(nLP zccWJ<@G@~7qDG4u>170Al*Y%t*ap&ci16(4ebt~ftL|9 zH!BiKZr-3)qW}#~LKryn5a%bUi^$jfhJ5Pi>+FudMHH8))`=G>%Jm?HTV7E3b;C{K z`TSSt^5VRqYI2 z6&7(m{|WFY56}U@s;R%wT)UCkOvL=uTXb?(FZ&dInie*F8j{2vs9iJY)yE> zp+CAGqU8qw{s>(5^NzbhhfgyFSy$mT_}&YYuoKQgn``(4+i~cJdHHW+rt05oqc$vu z)2S*O;nbxzB$ImFgXAyF#k@A#V;Z*|JH-}*8UC6Tez!@XGJo5KoG6*V<2g_}PVW?hBhDu$VCrQ5+U$azK`wZ1E%K5܆#L@ znv-i$W0dliE_jN|&2tC2EJLpNgsg&1HyYeTx(N5CX``9k^5^Cwr$NtjGxa%I>)2R^ zu|A$kO}}5jvD5PA`De4`;a8T}q_}V2R)Do_0Q?-1cccVU=3afh=J1lQ!edixtPN*S|HKfE?MtOb)Cq@H;sB;xq7PN1F?CRC z$EEhp{M(mA;G}j%uu{I_LEjW+kx?*{K;|F`38^WqSmDB3gT*_^H9@tf4Xf!H9#*%d z!p-&!WL5`2OeesS&QKnr@BlwvQkg6bDxa(=wR?llg2NZeO80?-n2oH-;=yu3hQ!ms zsbrWN^{zik>NbFi<7KVvJ09RX0pKZfT?2zvjTQoyEC(cD)M_$QtG*FQNw`#A{Iv|q ziICV|qf zFtFdRfeO41@Vp6L>R+WSgx^=GDZhe|;NTnH<7=MorF(@sLimbBs8|Dw7x}8I;pFm1 z!r=*#gZ6TkKUPuXSQ&JqzBsbkfjEz7q7>NL%deD_j-O4R}HLIIga&F#1oEfO{&~$_~8-teyo>)tv4@xWu893TvWYD*AtUKts3X2N`a+i z=JN}wd&{|6pgez|uUF{n8V)ejnN1KqX6;hdcLLhri5)S05&^C@b!9epkp6OhOtr+u z6puuil@9Y%F*18|;j-xx`f!4WKd0?X(;Uh!uC|^E)(0YCIHc7LJ%v05qi`{52C>IrIh==q>UTzqMDArlQQ9fBS^wpMJ3 zRkodhz6pc!is;qw2*CTM0Jmnen+Wqv5*s{qr#TLzKGvu;8kNO8M666Vkd#2$NBP^) z{A#O`gjsu*Y-JkzuH|uc>@J4?`&%)Z34efCH!AzR^F{78M->zGPYO_QL$q zN}cwX-gXFC%*~Le4G0C)N&2EKxDyGNU!Jz1`w#DMsG)VeHIKr@&nQgs;uo#v{MXtn zr@BOxNP%|BaHLo7jbh+Ng5&c}A1OLyV3vor=WDSmJ3E0Rw!_R_O z)AK8bU##E|=^19HE%$%PFn7D}_znlM_NW|Wt?y7bLr=uogIdl#>2o_q_8MI{GO;;6 z^%*(w^d6H?8h>IM?*Z;B`BjSZ3Zg5O@g23w%lh¥<^QSyh#FF!?~0<698nH;Bfu zSUv=kZx>94!0`Z?+k24QL~+bFa8s&u4q*LZz)5lKi^NO3lc^b%uMsCK3oDRMs0gz= z)NNG-*40DEbtRWvlU&JKo3^IkG-%#lEyveI>YV)t@?l;#0KXz{ePAr0>X8zNso1JJO1HjzN)pe27!b zrAC^+$0#;*<(S?4!?l>zd|ySF?_$zbonK($|0|%%JwVt)K=t4L>2&^=^dl0|g>rPY z_YGxla9^szTL|L8@04o+J47qKqK4`+s-ZT>sN(gpy!-ymiF}j-l#-$0)6^X^HC`0} zwMZu7tSo3hQO&%*WY5*y!$Ru1`_MJABKhZiDb@7@e9KrJKezjfvTg&Jq7)u_@LK_8 z8%ufZ6@VMNXu0N-KE^Suy#tWb&01nYy^3<`t^7_>=xZXTb{va*9ljj_wYHw5sID)} z@$YI_Y4{DnG#$=Hw4qY~O}*n`d@My{<0vBFP{TNMUzi8_7=G|wtTjt0h|?0w zJxn|O6OqEn>Ckx3py_g;#cAtjg4S&Z?Y{%GoROr0c9&OUP6t6RbeQ;Yo^FexIwwZX0ogL$ZC?ZOKGL1&Ql7>Ws(mo|GBp4-Y68|rNO2^%^P zxlu>c0plbh9F0@gLTh;$->Y#1T*}W5Ptl-~4UoM|$p)5XRplaOCGG+`Fd)ZATUMs_Fx^)ngMmdSulGMv6s0dJ-OMl);L!vmFSpv8|O z*;cFT!n#Z>jkSqk1#2WZY4%^E@v!+ z<3&LOF;2DlCk7%DW8r|t@of}Y_LqRDN(k?`Y+aa(d4T7=)bSK7Qze?s+R432loE&+%xv@AKKuYRsfE>V=f4w|5* zw*`!72l$C`s!Ck|h8Sq0;7kzcIA7l}fH1*L^m?-x^`hB;bIgVET?!>fHrgK(@)>hf zF-jeJIg(2z>Kt>YF~6F+HH zo$^u3H9rLeaLUEPwA|vMz6bq%?n&8UFUsU?u7@{e_%oH_!^4j}bn6J4FWqn8rei+I z0@eChf%W!OHRdAqe6Q1!-KQ%Gw*Za{`Er}uvF&>Sqn7eB&$r^joDrp5 zui$Q_yD>(aQq+UqkAHBV-nV&(-mPvIu|dCB1&I1@VzJ|9KrAP3;u5xuqsvprQ?Moh zCbImtm!FWm;lZ9nS*huutHV1bUmTZ7yVaM2lG=-N==inBG8^@&T8TB>me#tzNC9f? z*hr~m%!@WN`3`|hbOGZg&Vxoxai_FqYj%2C^Q2^GMvA}NoCaa7!ABANlkqQMd3yKw zEtpK`DH@%FroEyY<>XW<>I{@+>QCmR=Qo_OibEkBhv}l}6ThUQ>3cd(hXa4iEsLM- zbos+bQ$u>lNl3~N>RKytCDnYqY@MydobPJX1#hVdqZ+kfB&+H5Fh<+75v0M#2N0;Y za+Qn1$6v8TYT^1*sMNnAU=u&SQI}0n7RfnJ02W3^C(w;g#jX&C?EK5PO}|+4nd3~w z2UAMXP;xeH$R@_T2hlc$be)|WD~=e;jb^n8Rk}0dsv0c2J)Dor8P#J6W2v&a@pis8 z;JStCC8&Odw<@a3UZ|Fi`+x~Ol6ebd9Wm?s*V@4s82WD>NTVI>xFGFJ;BPctc|Ih8 z_(5~u$J2JnH!}*4-OW40f%Zg7_u{CV*p(#leZuB`8})S>;;`vPP`zH&s(W0aS%@J z0nKA_`FoK|>P=2U=jeEH{rbWrz8@$y?vBqdh?mF^&$(?N*lXl!4}!4TFbMU?P8%}f z2Jc>yM4;Xi$WJQw6wR0`n%bf+RYcu;$Nqs2nk9?nVDW&n2LWP-BKg7fOC9vRs`i9Y z^Zs})=WD|{UtspG#y$X=hy2AH)EN!+npuGG!;t-O^ea1%4qhQYvSrv<7VoD@iW7E~ z**IAVAd#Hcs8XqwD^Db1+p8$rw0MvgANd81C z_JAGiV?_#x({YihYgrbxK4eyrZXyU(4n!asMCeUjny{WmMKv~O@nYheuhBVn zAKs@7Ma-XS&jQxoz|K{>wTdmIR$~p;EXSSNfien_wr%9Hl08}hy}D4^=&CpY)^)xqBhXc`z{qrvp>^#>>MVS z-+rcE9>J^T944`zPXkKw>>;?1eWI?*TGxh1)O)~AxlSn&_HF!~*2i8C;mk-1MP$p8 zp=>(75tSmvI$V>Q@mL(J3PzfmZ*pr`Tj_~*H94B1r}9`asT+^=rEcZqp2)wCzg!C2 zkD6;c0_C?b$76U^q{@gzN#vxRYY%Q*r+Yx(= z7SSu3s-1^}x$7Htlvx?!C!5qmSuMdY-VgaVcvh9~Iy@}t(fFy&`DCdq-{ULYtG`6e2t6#UG&fbyqurl-qFHC6@?R$s&4+T!`E%uKpvrF0?e1luRPDa_o8>&z}VyP!)2&Z{C>!>ySQ)$0^gth zY33V>vU!28+A$rD9|yGI>h@G<< z<$j+|ZIOouG=g&ckksPm(#o59I&8 z3c&$|;{w4^N}f}qdyO-7=OOnEC7cU8?$_9UhkiJj3zSroA2)ESs0MS2;rL!g@Si+1 zqF5Hsdz1vd$7AefE>wk&Oh_+N`|uO0qM3Y0Y{wA~$r{_gJaWmd07iU+vpD9$%SFM*rUMmA| z@OyYzTFiyV0Zc@zI&~gxTwluL4f_(RT~fRk*Pflsq}szX)QAw3lJm>H?B+D47g)5brjT51y((RN2JTI>%v7Xa8>7NC^_ zB(}J(LaTIpU#(X!v=0#sk%UsnJjvF6x zhNV6{qVUdBK;Ul-L^VsnjUGHN2U}GyfEZGbW!15jEg1^0%IKT+EK+_{+kpkl5&wxF zDcXS}kB#>IBaW zL)#qJ7=48dV|GFzol{(>IZaPo(HN(i-B3=I-APkpH& zDlOC_%Fr~y6qM+HTdmM}CKy^-T_Y09h8ou=cx4-+t#0x93R%YN&_b^a-F4=a#2~Bl zF#SaxN$ql6qwRr1j9YZxM+_ZFvz!nil4 z(5s|wx7{YOb#cy(@v$Vw#hB;lgCm?zh88aG)H~0->=2^)IkZSKr+!iSHXf>$<9sc& z=z!#bIr;V`)_CuH+gdC&d1z?XAlJ%sSZHy~w7j-^OagE5-L*DoSvdZ>H3r0{Ib z$G22`%IqBXT1k!HYgS)dYRbG&k0wcxA*KM|(wQj>b38gFH63FL^7TKFvLv)%zoh2b zrV!sUk5iWC6dae-lAoXX253`OhI-COY8B!d;afIf$`3i7OOx7+agFjV7cpgBsMq?W zcG<4czJc*mHsyHjP3rL8HO9Am>Xfaa-lxKlSMNLJGp6jw@V;#59Pc*hP=)=*Y)hg0 zxm_~cMjono)3`sa(A!*uH}O!V&&C{!k2)MAkLwEi zCHGuxUa&Ga+WGX5!a)rX>$izjMmV2)RXDUkyd~fFiB%UkUl>v(ssS?IkXUWU)XTSv zbaNVzns2MU`WffzQAGzFM&NtwHJ&@)$}2kI@L*m3L-r7}%iXAAvkoKulfE?rraj0h zwlsP8nEZLZwIZfH2`#=pd1Q9}$ya@A$4`5aQ+#jo==b?=`_@UF_Byo0spPRC1s?m> z&7Af}PKoQuv=5;rpC(VtF7V#B{^MzX<&^xGJo$YAonNSys<&IUhKWHI z7lYlV8Mj(}8YU016u4^-JD#QoZl^byKra-Gk#A<)s#hEj38ZIpJO_uJNnX5_d{XI5Jn~ zVGoNioBUeYW*uS9Bf}!goBV&aEjuzl!J}1JlV+xJtx9h^f&z35YdXMG@#oU}k9?ym z7#r4XmMOTE|Cu9bL%d(}%<0t*`Cm`@c1*z`el1Q+uW2vyG-Y{q!7+X#JI4RoGt;f*vY2n+VyiX@}tm@Y1b@`0&9ckVd zk~+0@>p!$YR`{+w?;A;-N4O1sT_GnNt-F`hWr5r9p%wGO_eK?Zmeh5J+vwL7Z-=9T zzbAFO=r(?6rB~qxPZatX*ZtMZNp~uJG3IDKh8{7NSxK=$`JHnkd|Z3P2F;p!C#Zz; zkyIa#9&u5#X2e#m=zQ#iPmvxyhRm9Cr*b{#6Ix-v9zD}$rN;)ha6T1LxLl83S+f@1 z3GU{6Cbe*IkKTE+mc~{Y}h=G(NjTDJ3*$3?o`>z^X)5w#1|cOZ(z>sY~LCWoqx$JI_ln_+q3tttnsJwuaApPx;I!iC&xF$ z+2u|^v8nflSmzvB8RG47FTU8EdqYF#obavb@A4qC*rI#GV&p8wX*wI>{ZsVi})!5A}|Ugv*McAs%M{(t=c&sTfz(xAPwk}ab|MnX2(GkXge z4W&gTQ8L=4j0oAHki7|+@gaL}evj|%fBRqiUHM(O-=44IeXRF!9OrqAI_r${|Ipm} z%+pZYwRKtEY3iS*SYLX&xXW7JMfvJq3axKFT{1@hb$G`;F<-A(--}#2U;o{Qjwk%T z8`wOJ3`^DjG^f)Q|DQc9=WA3mMyWB!!ed{&hMmhW1^fR+|_)%=Js^VbtHgcT{XkLzjW>@V;M8(fHZ1%C+F%5G~hj9}Ul9G8JU(&Gb=`eLd;%>v%*Sg<7YZdM= z$8XIA!}dY`ug^BBa9HfO_O)TBYyQ8_+K6MgUy_cIZ%_}NYC9jtSij`PM%}OVXjttK z?wIVC5@6ITsAuD9rwYe(zjf1%0lIw> z5$;s#mv+Hucu?=@)lDj#4*G3)Z8Yjy@387-;#}dkQO9^(P(VVpr;l^BUwUKXN!J20 ztG&XVuli*K7*7rAQxeCqT*vw1bTyZfx~R<5b#uoB zUeave)1`jG=H2TSJ_&rF=^g9x!Ewu3^n^RS8rb%i_(7w~kc> zzfHCo4U9U+#jIOZVMEj0l!2+|g2uY{sVed|Ei@XGdv0Ks`=F}r!Rw172JML(T;m>` zR~)*2m(t*qaYHRUCghcDT)#JB@U^(%!#t+v?I>S=KxxR6xZre;`FT6X|i zM_Asj@9U2%4ONOCW6?M&Z?}0Gr!S-Ual;xX2_THWnn;|9;K1!tb07vA)0w?hF9 z>Vrm>oaY)`)9vu|25+y8JaT@~kEV~Dj;!AB!FSZz-itk({ik{~x51Y^qwe(P)c8*I zSY?ABEk?iSy{xp^H`U{H4gTyI{k`{!56$?8?JDNtHAW*K+_Sm5T7{#z>fSNt0g-c> z>#LpgGuQMQ>l(1Cw0Q%yQxnW}_l|8F5c8qAjap@txuMs%9s#kQo-S%tx#p&O#|;gL zpX1p~?R2HNh1d8g0g0uaZPd=xnK#-yJ~UwM2hYxGXH_ihye6y;NcL>uuU747;k0+c z#(;HmTJ%*r=V#&OHL);YeQAp!YUd|dG}$|`JYd6z7NgZJL|J%xO*$Kp?&&pI?P9J) ztG$!%1Z$ItfI}WxgcfS21W8|r6OA>ee zXm!H*j>e|3U8lz-mbPns*15)Y)5KHLGy3eE)B4i$nl78BILye`-B;B5*7Lh#tfsG< zQFd|v z!)ocgS?~KCyV~Yg^uy0qD?ZIq1fDQ#s~q#lq+#T|*#?0p`?uAe{J3$$=|E+rXC`~i?Gae*+0OaDy3B^Dk#mCr z&-HKT5%a7plMBZvy$j*(JpG?v$V~4#Z))Jh(spgs|Ep`bdGEYXy~~f=bv*Om?}poq z=SN-QCd;qjh56>3$oVO2t_^D+Q1G(t<^tsf*=ug3w;xzgKXfzKmpyB4)wB;Tcon*N zm-51sYwlQdm{9O~>1fHy==5^kmKb^bYe2-rm`KIC9a)H4kb! zgcZE|zWKOvsM6X;7T!?>@6ETIiVQVc`(&7RLcxc&Th1siwqIM9?7c4L<3Q{4^AGFTlG0XpRg7vilO9C#x^yygS|E0wG*7YU9mtTcK!I@uSTi+=!Ke_h1Md!ML-y64nid=qe?ayJI zUl;r--}+5?#gnza(>s4I_U6NriXlox@w1HHK|;6D5=3u-!ZN_ zv00ve(dUvZ+IOAes#|2&sx11Bfz{NmGhgUdWVLmSd2Y~Xd)EbL^=j?BQ)50~vA)|i ztWf_)R%eyfO3AjCeo=)67TH`@jgsw$`y~__w#)WciM3C5%S-EU*E&}r9*gbl4DADqn_?PonJa`&RrLk5|q+wc+bW~ zP6u;0d`=k?6cE?5MTqm&+>8n9W*P;a>)F1i3lo;D3k~X9-?LM-%OA&WCaF=^`q}qd z!L`#OFDEKBB_(KhuYe-Ac6kLV>$6h^X7n0Z@c zdXEcfoSe63LfR#x;k|lK4QW!Ex4$m!fl+W=@3|pOujU<^u;H!IsB^uSgf#n*cdTy1 zU!yTv0TCh14f9V<*r;bbu2(=@h^J@%>AH>9#uMTKQbJnv&p$UIy{Yk}a{(D4UUTv< z)unebo}$$!JEUcD{U*RlBmbr1jyt#pwHQRS9c;UD?-+00H zz*{fdk1cqeyLtb0Zb53RJ1lc~Z?dKGdZ>NhXVu=DT|TF7xp95TxV~?yJ05lUZnCxR zdRTVfFV&rDU4Ex-{d|4-lfHkdJO6P}m~Ks- zg(i8~gKxx*>)&{Lw}XWZRCC4+Oo;2>Vrut`u9kyxrl%$z>fhF#yPHP)au%Abt?%Ed z-d~DropPg1lHG&4pX<@c&2fEhN@~jRAP#>kHh9C3P-9S5i-9p({hi#q?JG1NlrwkWTCE_T?LC}| zoK5q#4@|2M8oNCpuc)9@cFo5 zp2d@%m1N%JX6tIN;e9-&4%$(2bKl9qHN8g6@R)gX$B~=dc|GnGoaixk(9R1t5Bd&y z*=ytukA*jP-d}%s$&mN%qpFJ+ciUCJ{^+40-`z*QEM8W%>*xC8KZhth#%Pv=ciXL% zR?&W_hR0a9l2ui^Ez?dd8EW7$u4hSXw^H}CszXCpn0t&5DM_pFCzX7{F!nE7f(#g1~P1JBKBw~tu(YUbk|NAeDQ zHG5b;V%hmwf0~>yEmyhy$Ub=0`Pq${R<18Mx&35ZaANVCc013g9<<+3ml2#=Ja^d6 z^Q#WF*zo*raAxtm#XB#n9`fDr!g6G8@%;3i*H;}1+E71yWJ&RYgFA1l9v-*hb;iiD z;)OLk@2xsKcf;GeBaaj>`myt&>XC>I?=44F7Kd8wdb;XJ%7%}_M_njh+-?^)kJ%eO zXN-ka-;G8;jSpMj?9+l{SBCry8vQDM*}i7q zz8rfrlE}ro^}kkhGS|=^ zYabAm*xYFF$$&eWUB|iwtlH7s?AFQPJKB+Bn>UWGYHrc`)RdvRd&joXiFww%(f(7R zL-jw6?X0s})zhwZ}hn zx@ylxvwP!$OX3%oaxZ&wqq)`i2_*^XrC!meuWhs(F@AbU;=xjGY@cjwxM}?Sk~KA@ z9iq>C+-QAod|1iaAEn&oDy7?6O^7NbqOhh zTJ&C6-F%qy`3V`**Ui}z_`JH4g{$_&?10q77D0p01=P58omdpGen*SJx6TFEctlPt z4M@A#VtDuSQ`4Iqo><;^!^=G*PoH0s-t5!F<2oBvy~eh_uzHxM*Q7H#=}uk~_g~mJ z%xm7H%Q_j|yh2)EEF9Kq@1)zWGsfUkO{kc&+$)Suwne@f796 zZKh=rldkAl`u3ipH6zQjEc*EsYfHcQDMmB0` z5c?Uq$z>^%t_525>K)=fBd@e9?fJD)mI3i0EoS6jEz6j6eWqpL`H=QA3OV9*3#-PJfhc+n=*|+2L z%|jVOeoYefv(|x|=a#>9pBRc2@5@*!^~W#>m6dLYwS* zx$o%d+uw(e{xofc?rzoA6|L`Rj2P=RJw~_Gsdd%ttJ*v zPkXax?EVY+HLa~CznZ?~&ED|+SDf#5w+iu?k^iPFYyXY>yMwK!O`Eav&Ay8LcbxA{ zvYJsmWB;4|wXN@`-J5@J=F=HRFCO^Q`jOfFsC%;=W>#G+Z`7tP?SAULx#MSEym+u< zn-^xax%cMh%)EK=(8xBg(`w7^Eqpq&_Tu4XZQh$bsJs_yHLGsNk@N#R5nS7}WW=o3 zJB}Va@J;RElTBfpW_{jqtmXiZ2Ol>rzc=gmj^jTL{8M|Rl)2Jsw#v>E7UeuE7-dF` zn60z3qFuSV`eXads7yu(P_|K^OIB*_o;9<_=hMZcsaqMbCBzr(K^rV$Jyl?V5dib|QGA@x1YC zF060owdnbk;EVzDrmeZSuU(sO&mRS6uA4V+&83^|IxPC{eejm+^Oml;{H0yzZ~rMq zZquG0G4qPyA>YX_bZWD@&W{be+N6E=+by0&nhUKlpZia4iogrGd9DwjVy^)%4mOrxqLxy#1#A zsN1i?YI$0z2)v`)Vcd|{3ALqN7gh(>H0i+GD6^JFm#cwy`*fH(~y(5Oa*LlDX+BEw?J}tDq{AzKhcmK)JZ7!dqt*?44>HhA8-O=2F&uP}zr!5J5_tNKR zq05(i>zl<(2EVHxd$gqB%YN(Iua=B@_bUA8ZkMm8t!q4%PI~t`>u6cQ*IS(Jmd?2R z=4i+A2H)x)+|OCM;PTs=qlaU@y&v`9`qHIq-+k$L{M$Fh=tst3k!#-Nv!hp!@+mG=F5vD4ilzotKYb86YazTe(-dT{$!*h8MaD*Arc z?fhiO?}UdRx-PHo`=d$c=eK`nKK!(F`ITlryLEoq=1<}1FMF5Y*8erJ^P2;I%13|u zwEUs|?^T^YwE25>^bfBU|LOn9>-?qQ?=72O(^kB_{O4%r?+yOdJ^Y=s;>+c~wVi)& z`1kqY-={17T>kfmUr{NPw<#K~RN1YtJi+6e@iwL4l{&kn{RwpqaoDDky3#mFX>b?q zQ1Kk2c4MVglJdeX`rn0*jhKWxB&nozF%DJgKSpa{xJQ!e{w@u^D~%nalN#=oq;{)| zRjBfuF?u(`J0z)p?PBv?Ied(PNkrEq4Kp8yP?h8{Mgt>yCuz3!arv&Y)z%~-B51Z& zP=)&x)m^q`7b1qw)}C9@^p5J$M-5aW$M(}%$sshiH~{EtnO#gwCkihI-4JRg+*U!ZrSTpNDtkjv8~FYZyQ)m>N=xb_kdlS*U=B& zTSc6jZO)x}+lDc9yBlVlnzvf-dRF`3m{&=S_IF+MUGMQ&@6?!&{j4u_T{2YvRaVC% zG2ffp)Sp^*R{vL4=if03Pg||Z@E!)***@N@H9YOyD_5O0u*&Wlw%WkczE@>z4?~aa zZbw#|dpb<5OgwAYA>04=YFkgoxXR=nM!mCpddIqYI&H69f7WP3cJHv*=AO>yDtU&T zmfhz_Y+Fy4`pV5`jhANk{TlcXIGUKn%%Y^F+G0fhbAYh zc<%kreq>4f!VgWKR2?a7@Xdbo>-gm#nkk*;Az0mEY~zIJ56$gQR~DK#a8M`|&X00^ z8=9(Ev@%erS(s~23>`mZ?99m%N6-KNzbLm@)okNEW#lf#~ zFH|!9UC0;b@~zGaBZZyiq+#A#Sr35*(+Hkr<6N>lE^6 zgF?pVDrC@Eg=B8gQLKKckXXba9{aE#2T+cJW+FTBql<`l9@B@2#3Bwyr->ZL2}Hz* zOx`In1tFMZxGT&M<3CUQ86r^H3(y##=k&X;( zLMAq23yN6AR&qO8jBV6e$VLuwk%xQ~pb$kUVcH$oiCx%@QtZKAY-gI{|4oN7hIf!V zu?zbc-c6Qb5BC1A4f}Bbkgv$ODSdTPp zKsq)d1DV*2E!c`}$U!dhkdFfF!CsW%G1SXMG@uDBXhR3O(1ShM zjt51Y@C439I*O-c9sa}q!yI=U#33BUQD`4wJut#5=9xssBLRshLNPj@66pe8bVGOe zqYnbn7yS@~0T_rS+jSI!$iWzbp%{kY7=d7n#3)1~2CET^BqYPBl5K$tT#7frQMY!49hUMhq6@AkDuf({Kg;rMS4$V*@O&aVqagj zV+PxRE4YdDS;}I&L|N=GHc?r|;n^N#sXn1By4RJ({-&~YzN;+0=n6ml_f}cz@d`(O zDa*e`Dx$Df5f57xneEQsL!Z`BY-pw;Y1jyZmMUThBkc24k^P96sv>?1ROI9m6*1qW zB67t@4p(d)b1Z|yErDs!BS=3%tj(Dqz zLl;#!FjQ53k5HA2k*acald71Ns&XDx6}viB8C;{I$a}6T`6xgkzW=8xKT!8dRi5I>M^*9v!*c$rN*?l2 zfI<|Z6sO^IS4ZIt7r4R=CI3`qH+EqU_M!~?Z~#B>6Tk2qfAAMgnD-y4P^d{$YF$zf zOA&?_c!}*wYSPI?O(wgl$rOaZu~tXXVz8Qcp(Qd$smW$+LD)n!@lRHh;0!exds0o_ zRH?~ZC_U9tyd&S^13uytKI03%<0pRMH~!!+{z2)qnkYjR1~7pcEMbL)m~~c7W@8RQ zYt&>J5^B|C`ExZ{f!tSWQh<=xYVzofn)H3ECW?1>uO=P7sLAoaYU1>Q`-TqcQjDFj z_f{7NIAWl$x(vc#_`lXsT#ZwgYq*Xc4BsGc;udbBCvCmZ8#N5yM=c(pA8lI_)Fli3 zsrQgUlA2XPu_Fo5=fI7>ZDjK3q_*3C_x#j zP=OkTvOVgg1~j1!UFg9SX0V109N-8iIKu_5a7Sl!fiL{f4c*ZL>B;J{mi1(lxyVC4 z3b7meaS(@4g;wj-r8U+wPa0-Zk4;mT$(Vw+{1m;;sGAw5gy|S>foBi@yq2n;x_J}1|9|K(ilz96gh?LzuoFG9b5ME8p0Xa zAEgh+IUZPA#qz7^$9Z*Wh6NYZrSyur^u4MsBQXlYud&RV>SFv*T}*KPt&U)DsM%{sdvu1Ar-pp!s38G8 zHDqUmhLo?;km48(DZvix#4hYcDfVD5%CHajv2?YDL?9BY5QAu}#>OvPPmqChY(i-) z%fT6(#Y?=x8@$B_sKsfBH#(vdwBj{H8#>TM3wWU=c4H6B6EwsE3$X~vScmqB8qxvY za9^V#&CwEfP=mQi8Zr-o$r{ohw{ZtQ{x_VWAp>z3NAM?=zI@eDXk$q_{UMiM*NLLuO+RRDRNzpE`2U=HnQ{$FYF=1X+QF)F;VBTvy zyn+|h^|(s?ihPYXc#C)Vgd2?eLf$05lHc$h{7Vk`iC?(IxZ9BBte>pGT_`cE3>BzB z73$EyJ;vWhEwmUmg&E9Yg@?5FARnyIlqaaeYiO=yIz)zRN<0$b8lfp};1AEm7f##} z@n7-858com{^)_8=!JbznzA1UP>w@5j0*H*e*Sr{T*M_@MsL~zaD}=Ld5ye^>$riN z2&DZMc^h|7gL}A-TD*YvDoxRWF7%)e0~ms@o)`XuyV$`V{a9XqbfWG8A9O``_=9iu zkOz2(|L_T&qBSKLCu8V8wj}YoL>96Un#yY*C)TsAP~Ob3fjTsx2`y+t2W4B>SJ;mO zC`bFPn$i)S;frpV1GQ}|2O7|X7RF&BCSwY^XVG``L~jJ34?f{DzThjq;Sc`8GKbd$ z-sftH_d(W;PB6d9xdm^oY08HhO?h>n_uM143nyWrqN}(^zIv=FAD?JSalNMeR@0Jf zZ7peTs3ikTv}B06mJG!(T*u5dS~3gceY8Y(fEHhEud67A0ktX2P{ObUX$dPdgf(oi zgK>7`PSSz2$1Z9Y(iyv{OW{V{1WmDr;pSuuv_V_6M+bC7Cv-*^_`(NW;fHSMiCzf6 z01QGfCSVdmu$OhrAZKC@!VrZRWFjAXun#Bk8DH=fS_8GD#~|8pbFh}w;2!SdSg@Ad z9z@JvfrVLE~SV&S^<9Qm_uGaMaUPtS2{MBhs-6 znb?YL$VLtdP>4e~h6d-gWW)t6@qVVoAH8c!e+cZSgebSi;sxTO82@&Ebt< z$VEPOVK+*#9|!QxMO*CMnGb`TXv-q3K@w814(qW2hfvd0TTGj2%WG3z#T&eZdVg)v zz&mPPQV)i(MLr5ph$3u933gx?c4Hs*;{eKW5Qm@`%(e_;U8C5BvD$KBv9?4m)s`rX zGS^i^lQCG0Sj1segtoMcV_T5BPFsp`EL&U7q8b<7*-qtFb;S)aNE56|u{y`C;BggG^6+$XO87feP8q}cyO=v+I zeo;Eo4gTnXpG7PWfAAZB@ehjaIx+(@@nn~dv^=RJkFM*;);l_KqDDuaz1ETYF1m{0 zD!MWP!5FKmE8{R8Z(MZcK?_~E-CtMskI|KT6LsZ2YViQ`X6VZ4#<~if*}9?&J)C9u zEO~yeu3W%HTtY~kuKd?TR}n@o!*Z;^N`#|jny$1$YxK*~m8hn=idE=O9fSdhW_U0; z1Va(S@GNpSO0fq&3v}hkc3tr;)|F;Q$P2pi__nS*!CehKxrh6BfQN85)|1iZda~VB zPiDF4Nj{bh(UZ5M_2eTQCSWep+|iR)zx2dNNna*<>nbK;GNvE|Q_-NSzDy&hqf>W% z=`&Pc0?`lsF#wJobrsh~>dQ^sh7-fixJ&(Nl)elZt1pR@^+jc#z9cWvmlUKTF@bs4 z=*!TJ`Z63NFcSKQ_2uMseL00nRN*wv;4G?f4(D+hH&KJPc#pK3`VxLiU*aqcWR;77 z#9+Rcf!rBvAjgLo$O)XpW!!_oPy?|?bM!|HPNLB;19=9`5e5<)Y#?!nM*`f(8;A!Q z|1Z1gDw>f^(FD!W60Oi0ZO|6&&>lT76|*oKeRS!Op};`Ais;*319^IkWqvS_L7xnyqNlFnBGeQgP#gRX)A{E2mO={#kH5N!d=Z!JkS^_>V~2UHRNa-N-oUw3?<6MP{Pa%6Q2$U+|Sk&Wl;fuHz=owO^@=k)~*Xu=*2IJ>}5 zs-alIb}u!QYq*Y^xPjZahx_;$#`2(=#x`uG|FD2L_6F)ItjLCF1Z$MhW<%OyAGJN{ z07p2%87^=~V>Cl^c%lVbqYc`k1H91%KIn)3n1T>Y#Ug~^FpfZDi=n7P8%wh3ZvlOU z1*F7KlrgZBW4xEwJ-o{dr6VSl8%nEFhSDDUaS#_Svu!nolG0CCk%|r2$?!(<0r?0I z@fc5FbC>PJ%m;=t1G6y;bI|u8`{xmTerhQ5Khb7nB=fKc|JfMH3-q%!k{~qpFp?%{ ziUYnzvZT9_?DRJh?OsebNLSH)wvp_|0UX0gRN^ImV9FdL-bY5#e6EqiqTM_rX^#&0 z0_FKeye^F7Zj=$v)cmi$Ek>dkn2br(j8uCiFW9KZs?AY*v52OD5Ktoa$H6Y-s3aAB8O@IkS?E% zWh^FO)E8sP!X@~BWnLKkF%}~j!Wbs#pfHh62*Lmi#taP6Fp+;K*ENxYXlrgF?p7vZ z;b9Q_xOO1_=L~+ zfuHz=KlqD(kN^`=Kp9F4o@fCtv_vbk!4M3^y}l;m)6YbF z(G^*POr&a@iJV5G@g`yoN4VfPPU7i&`nkwN*2XeFCag1&;tlMtLKFE4MUjcbA7ecy zOeFN2i7duagyG+16Y;)oA{|li*+dF)?;rhFn2H&!a7WEl)HF?{FZ!WBR$-pLsXVnX zmAq!Al8+rwp36D9xv8ka6t3_G({^s!ay%mnSiAz z#%}1hG?nIv!v^HwIL@J@wW-`|Ybt9xo5~(PQz?blG*fAbR%nZMxU|$%{MVVv+TEs- zj5KV(Mr^|l?8Gi~3e{Dtt~8Zc#33FDNW_BErm_%=5Q@cEf}0mj<%EuzR3OI2OpG1O zB-Gzb7GnvPVpkZ?NaQjsM>tku1tJiMXvAPOViAYk%vXwd>I5WW4b~zF$w%CN^U$wjm4I$U!dhkdHzXpa|Phj1ug?9{O^SJcQSHgSYsAkNAYo_=<1% zjMii*Jg6} zhne)$Y#_bR8_gOtkmfjN-9Vz<8^|Hg267l*J2sHtIJ2^WoJBRx;XE$jA}%2+y@AAF zHDVEmnxY1B7nZvkh!q;a8aCL5EF8VsK#t+Ddh299jP)x;i%)m-SAQGFg3zn>F54jK3xQ$zQ13SjK4Ko*evN_ov-7yaf z5soOVLNxwBceuIK;00cy(+G3%fiHTYH~OM8%lt2x>F^To(1kYJk>+v=|3;Zh?r3w# zLp}1fyviN)E#a3`a0VVI)Rl48~&I z|7xZmPfqw>8}&qT5+-8`LNFE6FdZ{66SFWIb1)b4FdqxB5Q`9s#aM!+ScWhx#|o^( zIMx|XMj`@HScPcBU^QY9hj=6)5o@p($w)#9)*%(^k%kS}h;(FN6E-6gTd)<|kcDjI zAQySaM*#{^gzYFs2`X?Br%;KTxCQg&=3)U$*uV}BaDp>j;Rbg!MLUedXpF%`Ou}UJ zUcu`beGrI#2w`7M!y@YGWGJ~9GpLu4OA&@;SdJA~i3mhu6{4{kv4}%F5|D^BSc_z& zU>#Di0qNL;Ol-ksY(*Bbk%L_1As>Y(!gds+1Us-3yRaLjIDrbBL?xc$KfJ(a{K9Yi z!C(A?Vx_qVl%Nbf=)(Yh=njAMKu`2SZv>zZ0?`-!&>uktp2NJOc_Hn#!(GfoA4ZQ@` z0|VqGbBs}tVlFQ0d0nF=+F%qG;xt;U=l4zNjK_qnY!4beSb(XoS;#1i#)TLQem`j;moRX(g{a3{Nc$uU=>TtZL??7c7x-Y&VGA)nYav#)mXgre zQcmP@ttZ!DEs~Io6y!s#iKVEcS$9kEL??92*Hv^TPX$=Ykl~hcF4$7e;{qQ{bHhgO+roS6XgrX6jO>QI!c#1mwFlr=M zDs&Y;$*bfw{HDH6{vmIW3gbp{lN3@JDo{fWZFitStp#o9qn0*9(g?qGh#w^Und@MjHmSQC$@R{jXld*_HJQ9$IwMa%1Qm_u| zk%}~Iz-DYg4vyj&DsT;7S@wOh4$tup9ZeeX`C9sbFf7M1D4*0-s9+_vDyfEO>KLp> zEaH)XM65wF)*%h)$UqjAj)wBhf#r(sKj;LKn*NS8;KRHVS{{x znKcsU293l8uJAx(Trh7W7jX%f@fMbim><@-YQz4(b=<^lJV!WvTZtFcFHw(IXm8s{ zI=~y#?HkDq%tWA5BME|wE9*yo<3>_|zD*j*6cje&ShQ#)@7pz!4@e%*aYH6HWBgQJ z6XzRA11oDWhXtB>TFcK1x{6=;jX(H{f8hHL6#^wF*hjb%z$viZsZi*+=i&k6n**j}_N}Wd9#afGcoV5(YaE!qO{6%iOwRB3b zmd^Oa^j%0F_@XQP5RDjQBL@X2LLnL@T8llLaRK-72t{k?Kb+QD%K+Sh`xRZqZSoFk za2NMbi~F!jvX+Kugyv|0mS_!cJitRd!eczaQ%qlHEi*6=^RWO6u?UCJmGw5>Xe~`( zve{Z1Y_*oLJFR6Lwv<{+yS>)Z9-&ut6@$vzPZ)vW2*xOkM(3l}vi+pB6r%(?5OR&* zO;I#$hU^VLHy=XFYZ{(g7XO39Eb9h(muHnSV=Hu@DPzCdfw4q8jIL z9v5&Cmv9+Za240^0;L0N_`HXW>_r*&VLuL_9Bu<`xUSpC@k=HUg;_hWZ6kM4&fjUBR1Dg;t-Dn zBtq}HoveS$&)kuQ2_Nia4wfJc%aM*f*oQ+nf@7$_E!@L%e1Y;uJNjlPo*0Bln1i{P zhfplTa;(5gg#Rx;k%&MPRv{WOh(#OL8-=p5i6 z-OwLHFbuZ`If&t4`VI4O4q~^=K^)+WFMo9veq=XvM^E%dFTBMke8yM&g2!?PS&SH@ z;5wc`eT9Q)p&1JB6n$4Z$U-bfBnnW7V%YuDRg{qScvN!ATx zn4&ReLphqh#jw6u2gyS|4&wqY;tFoyew>4}No0Sb7Xr`+{m>stZ1?^(4ssMP);dT% zUg0$+q&dhWOurw+MzufeuJA)Qbca7SZetnPghCA6NuF|$ zV{or#TQ6}OaQTjd6g}hkz~CqQAInr6B@5?~Z|5kQEgeM*-IqFwKYG9`(@_p@bCg#a zdWz;p9K{nY;DwfGh1O_;wrGdfOxvF9fGI~EB?MP+71wYbH*gb=PC3fivyL+1p`%R1 zBuvH>gkUPBVLE1DCT3wC=3@aCVi7{I7)ud`Wmt|CSc$FJhAdgW1@I(uEA^fwW*nM{tliy7Dmuh!B#BNc9Kvm#u6+=7?xoxFP{p1~|!8T%6z}mv9Z+Cp$?oO0WYvu?xFViapqiGVI5~c~0^m z%t_`aI7#pIPI6(ZljP(%Ng*1QIZ4M8tn(y!+DXcMoE-jqJ6>ji=I~v2!%URMnyF&25n)n&snTs ziG~P2|s&JO?D1PKDCD?&o*p01PF0u_-*ktA+hizTtgr|#m^mmb=7>4PUF4F3a zi|D_1k!G0K+*KwaqpPcA4RMuSBgs*&QZdF=s_;I^RX$+ePFIQHT zDed$Wd&x5F!+spVL6jqe@rTI6ID(@%hT}Ma3RI#Br*Q^nQH^srj|;enOSp`yxP}|J ziCeghyQslE)M9d-n>>B)CQnd@s26Uc@tW5Nw4jXweRt_=?Ji$j-Nn@1T@)VfV$;}N zoY1AMyUgmSr?BkoE>?JtcR1S3T_XCp%if{x{G8KWqEp-@2FnWFC9Kw6Ha&2cJHC2~ zOmZ`}U@K~9+eT&~8#%a3TOOH@0u-VM+fj@X?7&X!!fupe5B8!A`>-E}a2%&`89g7m zOE08iJ<_24$X#@>4OuvYv(S6&F8VOQGy2q(^!q>Ty?IC~`3Kg)n+FE7NmPH`YYPGGc+D+S9 z!M5LOR{^!ZPi()>c6(pH=lR|H$9?_oGw?n$XZD#hXJ$Tu%m-Nfl>mJXcn`P?TmjC0 z4P^p>-w4nk;00hEz`H9zT|=FbyI@~_AVAlHJLMMv{9lIvO#{+_L0|~D1lUgtPzQht zU{3_-W10|s0(44+=ogEGs3Oc6K^6;96i|X2T_QwDK*{q$^o>*@`X+D|C|f5)cLU!8 zGcQ9P0J2kvTHg?&ll?+8eGu~c1@ZuRJQ2zVmIjN^C!r#c5k=^iDI&DBSA^z#D?)RD zrw>JF)$by-8pxR=Mjhvi(NZ5VS_bR}EWir282w?DGa{)KqyB&tkO6Wa0XPUW0f&HQ z;3tSXc|eR#0QZ4{9x<9cC`Q)+F90Pe&PdF4C+sVw3_n05kvt z=zufe0x$s&z!UHW762^32S^9j0$YHmz#jnmi5MjT6u=&E0H^>4FaS@$1MmjSz^ofk z79a*BfE;)Z2muO!3sCPCu+ISR16yy3(HDVH@Gl3u0vH3{1$N(pu?Kts%m5*u!ZZC` z40Dbc%}8}dKK%;%68IbteFNhX2nDo&9=HkI0u&QsG!LkJ1kVSk25NwFQ)2WyP&h3{ zcLEP*U~HfgG>{}gH9#n!1q^_iDnTQFC+-sT4`80Z1f37uQA*ISfUkjXfV;rAz&&6B zm;}BDgB^cA3QuLRBslAyc$!3HXTN}vj;25Nu8F2YSg1Q23fCu0S zcmdwPAy!#Qs$ zxTB>=rm`P5dj;5K;!Yh!i~H?wnc>j49NgJjq)<8BH_z*~%fn|+6b1J?Oq<~ZI5SSC z7OPcMS`jONY{414;;??IcacvXxfXXJiuEc?P{i#fH{!0D#Zmp3vdHfdxdnH#6vwG( zh9Z6dr2}{GC|=f2ODYoPQBLDbYjL7#mZ?bGP3guxCW=@0&nhVLe?;lWJ*m4=RgM)! zvH*K4?#0`c(eKz)v@p;9D(+3}%2GLX6)oHpulkBSomW@WO(9i?ye zGjA2I%$t?Nl2}W7R37(>S9j0KWBE^%UhManE?)C!mYF4`mJO&pX}eMb94#yvuWY#A z(|cEXo?|UbPLz$Qyac;4x*Z!?3p2|u_j@UKt$XCy!dhf01C>&SU0DH69jwJ2W!L+? zlXhk2Ih|%bXDtK$Ql?#S)Ll1g$wb-b{c{R-ZF=O?&kCUKzN4C3u`4&g*~(JzcK@xq zqZ_Qi%-uiq&-<@vJKFh(_58%{|ETVWR#Fr|r?S;N3uOTJF4>hwr?WML1yi#GC8gbT zPj+ag#c6=0EZO~tj@& zu*@6qDJa?Vh@oN=)N+=ZQ&CbE;6kwVymIaUr>SIbo=Ys-K$P><+^&*+-7X32h|F^F z0Jpc~z$2FwcBG|TruH2#IT+xY$&UJ8)^J2r=+%5dX?wR@BYSCPMbrRaS$g7;TMIkh zQW2*X7)s%Uz7F=Xj*4Xif~3-udG4p#%dHiOYN4t0RJVIKd&NY>>H%Rv>FbZ&``HQ9 z%2c(eqV!Av)5=cdRb~u`no8fyV_sz^5tUhLaaZYE-OL;8m6??r2E@Ik=N~cQkY`I} zu39o)+7sY0!Cu`_xotpltF$lA;{iL_TA8o*zh8Q>+v5>?%|zvn0sraJcOH2lJ}K0y zBDIuOHW1)R^-1Mbl@3U~%ZBni={{*hRk>OwC>!qf^z=#3tg0T6Da%G5dE!26Emd`D zxuI+DXT(NVQ;K%P|gZl0Ia=LKukLG?mY+2wAp0H1XeRfh)_7L>jJ$V=suNv&>G zFRCcJ7T`_zWbvxo2NpGzeVFGR>tiCSJJpN3%C2{NC-`J%R-YPJ+*|g^BkvTS^_J>0 z>gUGGZU)TB^x4o+{no&9x5_@vo0H?S(OTW3UUI+e^X@r$KAR@0FAgl3F8lJ)9J5aj zwPrvaK-+yMV6Md{msc}95a7N0>%6(OKAVY}F||Ul`)>E#MxQO2HJ1kz%H8)K&2917 zYN@%VRvLCs2F&a5+162WeL$JC`@6h(r+r?u*4$JFns)!tJ+Is6rHPu)2LcOr|M+NL zzfT@@&mDD8#qNgz^Q}JlyghdZf|_>!oHzffPXVz9v_y36{zE_a4eiUUVYdg z;a%>N&#C3q6ZLwHUQo{M;WTpgX4Xdy>XqewQ=Aq~gQY%BV=$ES72FQazK;53gNCGX zVLtaXXTP;RQ4?V*7x!?xIR_@{R}V%Ml>1L{`#Fu&y{VeWigKC4*UCA_+nX^M*;KwT z-}fq~iP)Q^iRvm}+~a$L^QX3=-tr|=zPCBemc6-}=<#xe!f%3exMT0O!RTA%f%$$9 zI7h5|^EEN|%Y%FT9&wIN?A1&_*Y zV?;x_CQeWh*2DAU9?xv39*k2~=%#o$x6RT}r!g8T^a?(Y+uqTzZ_t=j5s}ZAa$m7F z9Mmi|RYdjh1Gpz98V(OGEvSf@;;XnF)P1d*_=<`+g@E98^7ge4#y3?g%@@RSyNG?A znq^%T%X$O}+>@F6P7N;WtynQ7Na4O}*>^^>e7qu2Ae%G`5s?i5sl%AHgGh;Ki&5e-eDRTU|uRNnz!BV{PXyJ}ayl9WNBoCrWvZr6*8Xha7Uy2P+C$|Wxh=6J7R70 z3QadvRrkmOd`Bl5=MAM7RPC9PseG-}gRIcC6;*W#IpI6TJIEbc+f=nTUmoi_P8{Ti zW^`5U>yan;zMFYaJe1K}bzn-K;`^TEpe*!-@v4K0g_*vWI}R=$df`^pq5Oq8zE`XV z6`||yR~_zInCJWc#KGX9b<dNV?=@ai*ifc-_3`{gwZ0z^ zP5RI*L3Ml2qDJ2jGn=A@vXs>)rWUpMeq?Ei3pE+4I~9vNe6M#jEgLc=RiDgXeA@Tp z|B@P_>NoSByXt$3IFuE-p{x3>p671(ewuk`!_bD_>hn|2-S+*=aws=+<9KzCV#$Q> z=N*T(4Q;$t-Iu@Qf$taAL;0bb?pI&zS@OvD%ZWofhBi%CzcaN2@w-iJE(*<|)eI;C zsD5{N&80&*-Zew{0d&8wi01OpTtUrnPk^W2*O|@LL%GVD(WwC3?;A^VUFc>*&6q;L z^Sj&8yl-f8Qq8;h3aQ_>*5-quTTC^VdlUhF_a>ST4{a%^d4Ecw@|&O@ZVla9QFBe9 zB>X0Mhuep?Hr0HXuZ;D(PaN(H-PToey+@hg_g&`UQ$yQ&Yd)D$rucntIeaGc#qpY( zioi_2A36@dHT2@GnosisbNv2cJ=_!e(*2sxdjj+Pew;XbapquyewGg{D0}^aOYN{XTKz^P$3mJwHwb_xnAf z9=#K~y<*QpMTpgJig)zx(DtT1Kj(*B^_wP+PKNI2+Vg8q$PK@l%%eXH?daX}`&7uz z+kTHNM<0g1Jicd2@%)6}la8am4!wM9&rJUF5B#25k4}Z|yuas3&-0J`{+KxWWN7E~ zoHwa@WXrruy9#P&Ppeft zD%Q%U-lp2DE@mslDmR;p=KT(JEqoJXmu>5*1}`(kA+>bc-Q$9Xlr>c`eS-+xuA~Qt8L`DW*v*VRIaS^ zo7T4Q+{%x|X)6qMd}UY%&%N{5vP%{J2WSzhdwyDn@NulINLx#*S1SoBpT%!0y;SR6 zA6h`r`D}e#xwcMFAJ$8F@_n+}sxQ?k>vhuv&gYc3)oJSu^?IeA$LDso?YmT;R3A~G zm-2nb+74>>n(Cu^^#Odp$+p9n_7>E~OzTyA9@gHfZK$Y^QyK_9pWohosiCQUX@Mb@ zFVMGlYWH>3FY7fV@P%3Jr!MX5tzR*1Na2gh+s|nCkJl$EBQp8o&i1!1?Y~vOvLGUd zFBxm^(H^*8zq&UfkMBR(e(}H0O(5oW#=du2e|NZXsLjI{7&{8xrAHG1z&FNmz= z%k{5}X%7naX7olj@)u^ka{1Ch<=%DEkuCg1<*!`RHW~J2DWf|0i#uPreyJ&GZ+1b{ zY5sF#uiVrgGVR^a8`aHUGWp8qmkt%|-83E5&kw*(+|f2y?9Ek1Tlot9iMyAYoAz!g zh`!2K>Q78+4|na|)*HR^20t+C#1EGa_wIdZI{G$0sQko3?UC`l`O26HesJfBUoRcG zwYRV!<^exs?8KDz=>5GrdSf2(pPxMOnRA$>TnwVOFfW4Ti(G4J7#DoSH|%KVVxcB!^e^uDhuMI zg7C2pudw5$hU(t90D*3@W8U!bf`&cQaVi0Ub+W?RDjMpPMna(HcXEf@ni}>N7-Izn zeJ4Mxy{lnguQ5Rok<}?4Ztrb4Fl|f`M3#5T!d@9~IH+8jDTwOqTs-{Bt%gGdOLGL# zW1Why6Zabq_b$y7#7uSu51*KBI67@p0)%y`!#ZgDT9xq@K^(s;Y`DXF-|>R@T7gmD zr4Q>A>}&6hZxk%e>WUifRPH-59p55|FYk&A>oV-?R4(ffEbHuAHr$o8?_|NU(}Lw= zU5Q~QP5VyuF6$PonCx0Te6nEQ>(k5n1qs;6)Ua18_MK5Kw+a&ZCo_g$ZQA!{!Sbtu zB>l;(uv1<8-s)X`L$EUIz5C8jFTX8VRemxz?6vWIJ<1gmg4LZTw++8`YhPc% zivMN3Kc2$$tXHdt{}XC|#$G)b_NHn7<=(^q;kwCJ4-datu>bw(M3pcTJJlL?wqpM^ zWfCFG;-6|CKHIea!-Aw(p-F$LGwiLd{nvYw5`@`Vr%nyO)w}7*3l`tnm}!p@EF zzo}fADcsO`>aF2(xAuQpurf!uaqLu2*!lbWKkr?cC)_l7>f-SE>HS|$uQUsDu-68{ zx@iaQC|6m8x%}6Lhr7KGd|j}rR=8RJ+E`eR;K1G9RgJDB$h zJnZ#5VHYY6JX9uIh57u~?+#yRI`DHr^3JQm0{!cgVHdj&{Mws*Ls*#g`VYewdk_3R zoqSukz5MluVQ-Hgm{P8p5bo%F{nz2QZylH^So1*m^4RNBVei~O@T7OmBjL`;*Pjf( zGkxHX={1PRjGacq|5Wuu6&3MMQ%3r|8|?~H=%Ql%X)Js|&`9Y^@f7XKI_)$vplo!Q zNx?-W<)<0pgN8;dFqJ1N?L6&1GMLmjtI($1XY90B_>ig5sV_A^w0rXOypf@T#@REe zDv<>{!wSDt(Z~o)BShu=Gu)9&O^vRFX|bXT{TY7va95*yUs{5wGV6?ZWVpA{V5ided&3kJ(FjGM@FX`7tExa zMYY%)>ToOVAS-aKMO4RsBW%R#ebA?HZLO$Y|Asz%OmL9fx3*EVH|ve4kul{#znQfy zqK5J};=;!b2l;^+9in}mZ!8-bPdX?p%s4IDKlVmq`2V2LXC|Xx)QG*A8vb6zL0RAn zR?$KJn;9eTH62`7_`+3Dlm5-D@XK8X7x%qzLv$$X%?%@$dk-#|dEvIGx%|!C@GIj7 z6@lv}M29=y+%|IM*1^ESbq_>G#@@^ifB*i$;J$T_L`Nsz+%fY0^ugz6)*<2+>}*l^ zRa%oeFq0~7<)1Abx$4~%T9`=}AJd;L55Fd83hT@C6d%tzTRn13*`%Av#KmppXY0a0 zFf{1{vv}h6&a?YQK1gbcD9nWF3FRSpGZdx;wZ5E%#&JBd$pdCsLTyGJd z;h!5Gx#4{%y>NZ4_znHJvGALMLm7SR8^v#Cox418Q+a6J%=#Ab+46JO!fzQ4Wd&~N z5Wm%V?)u2Bq(j+-8%~SQjh(w0{;BEEhQ1Bm;`5W|J|FqC;LxU-4gKP7?EIbZ&ngb( z25#JG75DJZ-yQj^>Cl$KjaS9J`ty_FpLZSF*0=G7xG(Gc44Es|vGY^mU*12oqi@qA@jH{}pNxDteQ4**CPdPY zb)&l5wC19~9I9l1-%T04?cKbqFo!M~)OTaLJA&rYz8p`yS>c^W;Ult-Yf5s z>AoFrJ{Y(qQ*ybpXYuH_x0(+XZpo2c8S7E#?%i)b+_xo9^8RE`@aVni=A$!P%#y2E zuUa=jJKP$$)grmZ?+qKB@IHLJaBHpP1AVVvHz_#W-nX?;@?lnQ)aa!0@QInNEs~GQ zd*gKX4Tn1ew{=LaclItDy`OaWWZ|~cl8?uF6LsI24xj4V)-CyDvUm09cLj%EpV`(g zxqb|cyd?xTktK=rXFJtukro(R*zIat~OW&8J`=RUbTYWFykbIiew_)^$-oxi- zUc4>&th_H*_mA8f(Z^!75 z(}&-gc?t2qja?|xJ)j*K2+X7U-{D^<9ev<^WT-HY?*EnkLb>igqqXP4zR{nOj=Wo# zFZKU+?7~6a&!!`n`|<<)?@eAfJofe_wv_jPAGbBR2yJGyQ+)y!h7W zZ?}$oT3DFl|BtbYJ-XlTANjnmFwg(T$%_|9f1f_`O*qkCL& z^kLx3R{tsfJ9kGPHy!=C@a3!i)B1NNbx*pE{@VBQ4gZ;}cYYXs(tGsxnU`<-KQ4dg zq3-GU(W$_l6aG&+-}!a)>8+zPg*zYkKOK8#O83Y8qfh#FKJx!#@|`E6e@q|!V`e8J zMQHtSPW-GEG{{VqqJn;k6`9jwx7|#ak_`PAfeKqF7tEegJ5#@t6%A~0cx=X{f{J)jN>{(Tm9(;D*7hQ))PB6*i?GXXak@|xAa%IkKhJ7c*fRTZkxELX4X_Au zWeX#yn2=(E0j`yNsKs@AajcYP7~m6>lP&HSiW8)>Oao#orLV=~adC>&v0^|**uUH2 z6|^f;>eMx`*lPc2%be}Ia-`1V0}8_7yOwzucI8QD-yaCJI?S{zc)ZIjrPBu01a(#` zE2zXGWe5ht{wK8Rtzs~aphdLugGxK3?p=e+thAM_!tJG}rOffc zL}FHUtN21`x76eQ;A-ov!dCysrTtP*+E6OtSlKEIDzi$x1Vb5C$3v|Px0n4>9nZ>yfngZ*%bq;8>_SC?OlTTDWp(d6w&HO`icC~7e1>4Ydn_@i zGE*k*8h*>l{Pft$?Ugw)$@p*&;ql$E)fX!BWd8SuFIqijj;(oIX_iT8BLjrztmCOc zRTh~{FfwfQoO3*VdsVGWZWtLOyoARyE>tzj7MeybTfG90uX|k8B3o23a*gngIGz<$ z-631tHFDkRz4CbW_UhBJ=f+2F5_7VTZ@5ssq+7P+{>bOnIfchJJ+AJT1<*$C5OXV! z=LXeSWeUOQUF+OK$G2>+xhhi{Mkk4RCy#HtP;)~TXd3;&Io|P?ObZj1}0n`*AHU*H%~=dUixx zc~Bis9@b@bA7ig8Et*Qw+LZH%SoRJPRx z)e~~PV2nG)In=gydws0jU>M`;xhLE9U8qlxN0`RMW8A*B1CQ%dtoGKR z28%pSFdjC>o5QuKNP1-~Y_2=$FBM7lTQYJtZrk8aVm2F?A0M-0et*o)_p|r4zP+z# z=KhC2KL4!-q*{{uPiUk&`R<>XrmhSa#Tl!B~YkTW2hQkBGrc_*xRyDFV(8!;B*&Aq9%cdCISBKKiX+E zl1On@#l(lwz%NR#PFNEgrZz^xU(VlJrZ?)L8Xw#wTC z$02pK8U~zuYjDXhztUhd^swUNx}@nmx`CwwB;iwmNTS(MvP|eq^JHszR@l5mrg97K#||S7lbo z=tvu4HbiLXDw?pAPca+uA~%xqD{^hWr^7rJ9WrZgP!Mys)_6i7VoZNlJ;bfp2xonK zi|6*Mbx>noJMNa8)8@vo&x+Raxm zwwWGp>z*prS1r4)pdIkVZM>3KdPT|p7F~`P)6?2M zcF7^!)XcSAMJ3H%d>%rDzYsCZg6yKA5^7EfVl>=C>Z-g5>iHW4VQH|}*zwxo;S`;@ z5z9tl=*?=?QuWkQI2E1D-CGlI&qA+P_#JjdSgf`OwZR3(?Ka_X{6kW*kXEn3SHxV zY+R7W8;0@5f2G0O%>B?&X>H&2K~grRwa_&xmXT^j=*1*WvnwgqfZYiBqnEhAkla!$MduY2$2pXj)TjjQ zvBe6aqe!dlbS#Y}2|62GNW2F6VCaZv`g^i(&-yk?b=34rUc#!qB>jmFvIY@gtV?Pt zsmMrg#qs!9Jx;N`NeoqN1$w<{*1|i@<}0Y-6k{;uE|*1Do#MAQc&~T#bK8u=u7p}l zXO$3XZRA_I6L_0B_O=D5dWX{E3~Fj8{t+qlv|Sl}-8>d4;@!Dtuc|Fvsez@%wpu+W zs2?@cvl%R^8bMkbjG=TjjiE&lvU>O2cv>O}&yB_`Zn=zX~dr}Iuy zT@bw8O;duiPtjolYF25ezP8cE!rN+TJo2R^m6}CzV^xtyX;@lSt&7qDzZZ*#A@Spi zOavkM(A*egK9|>yAQLUF6{?B3tshWJ8jK|fUcIE&+kv&!&4BW^UQu;W8;nK_zxRRK zfkpO?wL5$~TtMDuq?7CrbFC|>M2;Y0#N&#d&m#9D$YIy3##vJ7r8JxY^A=trGoUTl zwOAEVYus-~A~PBUEk))U*MWI=+z37^mZ>RTgqOII$Skd)9P!YQ@a8wQeB6w?sMGOw zV*#y1{;?+&rIVH-h%_Ozgy(Lro|vnpk)i|c6mMa15Ul}4wnp>WSP>m2DQdA7O!QY3 z1g;yWA}E=p;^Aaftcu@0g6v$+8xBV}EwL8WS92STrhba7IgwP-La_O8M(q|Vwl$h` zxsHU)F^Kx@5mpsra1Nb$PlLnr*T%xTrP09GUPv?)rx&}h%w`f&g2Nz%_q+sV2{WCo z!PrTGe&ET_&=E7}xnW!2VJtCfSS2Mc3_cwxV;~&YLDhH7EPJiFmWd!13klJvS%`B0 zLQy5fsmbwW1}(ONR#HrDABoL!eC4B*8nV2E4#TXT&~nI00&TbXik^j+(5dR$hK~r$ zvEhRxLHibSTx4xH%-M8~naaW0B}TpMn2}!G=H|^9M%lG=1J#AZXVF=jl(Fiw8WrBc zByFZv*Qp3~DZCP=C`fUHkY8I1eOzm#rIysi($G?*)_@@ANHn(8T|?c8!_o}1;^`X* z(zxXciXhq&ww8zVN5F_Hp@R#N+U9;Wbc)fX9cD^`&!yWF@CK_$tda(Vs&=e1YG`Kq zpbDQ`T$?$lXk`kIo&L29XOlVh3ZljA?83oO6|%vRZl8rNy89XaZbN(n$<)RvQE^*} zUHGiCEhU~NUMbaRB2!d!ZBc9m>Gs@Om8qZG7VgdBGDsR=+9wKQzrxaf9CEUiRkG>~zY7Rg}5EvJ*IA2n>& zR25xeklN_`aV?fM&X{OhbX8F}IG5wBxreLG27WMI73;&zrMu{OC07`OSZ%Q}z0|GV z$T!2Pf|$d+b2NNS=(=?FHdz=l2}`72$*4;>B!+-wFHOW;6fl| zW4Hk$*Q;E3MO(_~v$)7rn8%TkfUjW2^%j&YGMY5)4Rhh$zSgu^V`)G<$<<5fpf2T` zRTym%-b{+9uKOU(4(a!%ngST)qzTI|BbDEd81$3|DZC;9-`Z!216o}Wmf7e@rB)*v zJ43*Aa|EAjHqT0J{#s?AGIWF?>(qd!9t&vG#M&mr95aI4AaL%va6XGtT)U-{YO_G_s0*kIIRssV@{=mELQ%E7p|MA_$FDrD=@K;<&}eMsmo`0X`_qk+v;D z!0wV1c)y@Aa9LD_-K*N-sjJ`{Et=^j#Kp$|6u zl)+{VUQVb{gE8H4@d)xfVrDeZQv>KV0cuXGk;ZQ)t3naQfLq+Fj{CGwn%dy15`9(T zs%dzI*v#hEg;RNC4lQvIQ`1ZXl>J77m?Ql_#mB!X3H0l{w10Cpie- z>p?2{UlC)NqXI-bDTPQUfjhUm|$Y{xvD&Q%8m`IQ%6-pz{$5 zN@pUNT^R1}hS;Gl2yf|)(RLxVV>+JLqcSsL7MAZiK|v9q(~K!r35k`T%)(p<#B0hRe8vo6kC`Opl> z^=@zivdWcr+Rm9ohjes=lB(uFx>{R-P%EEUISKAjTP0ioNMplJfY|v-!Tlm20q6+E zp!Qpoh{HQYo*SQeQhrAF+j6nxI~Q@CkvUQceAQ-JBx0wQAQToFkJx!U_bkwk^=Btg z-1}{@Z7K^tLw_PCA!y)j3fixQ`*2__;*i@(79f-&Xo(%{@$FbpC(KrlGAJp+N&=Uy zgS?^D@XdH~5Q3g@n{{fCUgV1ounHn?Kr#$+vs(m*f?}G*m|23b)4KM|21Sk#+V;qz89W^ z`F`{G*^vF8jj6vKzZ`eoKgX|&mIC80ciX0|88*k(&8cfM{(jU_o`rc*p2dZz|2v+|j0=L(6593vr-vje~yvo=;zw|q} zANrNIYR^ycs$7V6d?H@OU$tlOSFml#=l>9o1rU?*D)8d1`Y~ShnhvsoOi_QwoF8nK zl=n-#s_7sYh_w*39RD1z`W_ONf-T(5ym{sKcopQE6AUH%#zW=Qcol+z{zaz%ue!9(sk1&Hp)s|Z^?zl&GBfTEml<5g8X+1S)K@mN3gD=2Oam3A*4>xQ^e z_;1~xjKxb16An641rPFQXbFmcp%;^#Gg;NLl zGsq9Q?G%fA>ePsT9Iu)%34c2<=~Rn+0ugUIoq8F@)db;GG2`2@1N?sNlqfPCPj!mq@uqSNRJiuar*kPji}tKtA;+$j&}z~7BmJvcEE zv1HgO0U376L`I!X1DW6^$o=tpggg+>pFl1^<@%vrZ$iqmQL)GwF%>@-uev~SaeMv? zu?oV?NS`&U<1W*&46ULB^a?koTa- z_ni{(Yw;?m+*#`fVk?{jlmJ!CL_QMZwuagsH}9i(70hJsIk|uA^l#Q*PwGD(p(pj( za5)2IRcU#D;|-JCxAwne*kQ2E`R~Q}|9vsGErY;ZSn>WpF2}Y_K@fl=e=f+j-Gr^@ z|87mT?REZkQHC`CvMNJb>i^zV*&*F*PQei>d{mPX%X4#d&}z=4NIlEKkKaiVF=yHD z8c1=puNqFv4Uo>K+uOO+Me-^r>HQrEv;##MDXEBvSvj0`CPC_T*?wqpeb9nP2QrDG zDKcB_NY&xTuUQtc>%hv?KC`R~y=|V>uN4gj?riEe=nj(a-m!12Ta~TR9Uo4Db)|EQ zven4HUSZ4h6tnW~=AKoO1MmJ?JYbY1il8kZtA7jpyb_PJH-7UhtPc-Xqx>5bNn^UpjERHeu2HIz88=XYF%`ulOzU-e=fn zXy5F$T5{mpD1`g5A!%ludCKC@8i1c(MryL_bYA7n$!NT zZkb&5{K0p*^Y^5`vAps#0mM&8{Ma;>bZU8asjU+64&3b0jlK|2*e7&_l^-aU_JmJO(E%k7hv01F#+4x>ueI! zY(NZvOpP#r?*UI>4bTaM0u4X}zy=h6H;@ag1d;$BKn1J@>;M+901yHpKs*ozd;+8a z62Kq8fq6hLun>3-paVg`Vjvt4K}Y0)Ed|to7FZAHfgIo$2#*8%Z)Vr$U?wHo!ad1w zv6=s#LH`Tl3`|}Bm&~BHQCCvjpA&$?goD?`ikNocLl4bb&aMHnwAE2St{mPx5b0eS znbzX9-lbe+9`Zz#Uge`ddC`%UTGb+Ct#p^#oZ`uGDMB1}4bVm8L0)}7V!vxPhVTv| zNPl^Xy4{&%HX#XT>?`)CNpqZBFxB9`ybJKgn(NI@_9q9(DaVjBGaTM~Mz!dTfvV>` z9ZjI2fei0HycJsy-DWqU_C90ZY;B4p{lua>FHw!W^>v25L)hz!t!v$yh6a}OcT;N? zNRfa{Xaiz?m?(O!C>u%9a`q0aZMl}|xF1b1VyOH}-QUZ8NFT0eKZqTUS_zujV|NO%%?avu>OIE>O6KJC=^Ho>h zN2kMmzds3E@c;0p{_P}5b$5((Cq3JKlb?M#_)pRB)qnSa24wVX`m#xcbkcusmi}{} zT>M}0s-v~`i0`Gbes{&Q5OZ`!yvYirh~jekmqW{g>b`F_zK?t8BP&!K4>QGYum z`|0lgz#`o(e~!u)DgvSx4uZ@x8Ee#u%KPLmF%ZN9)T2X}`@4bgEBylj$W&JRwR)H{&xCQ;mtzeL2F zaJ$5jcLA(*QV`Allrisu>`76O!Sg(dy8S7hI(dS8Mvy0#VvWexPW_A&WWp(hJh>Dz zQ$L4@yXnE>&qPGzV<##3T0C~zarl||34%=ua@FayO{AjZgMZ1QpMt>knG+TH9(EF= zPAbUpkxf*hj>W6so9LhYCDwic0@b@98-3;E=?LQ70(1(5#aQyaczy>6n=K%8_9M5% z&`5D3GA>R5IZB0G5qFS3jOS-Ub~(s*;CTro$bKg}2v@dx4@7v;9WCZGY9ui*^dm^ua{U&7C z1(Ne?P8~=mh-^N0;FX4E}r;R9FL~!@B5*z`55w-gQN9}GiQOa*S;UxN}8r11&-{{JtSJL2y z15uAMQM<>s9Dav1x06uHFHh#84v(H*@kM88zG`#c3CYj7p^T^ReE&(z3)Ok9xt+ecz$vGx1H#aN`yK*o!JQ;`{)=7 z3HO-LPy{~JeGn-^?WS`eA@tVM-y2ELrSIJP@tr5}5dXIyw?cd#1U!vJv8V6+4&^#L z-8lo#={FTb{QZ;kr%&W?ESn2TnSK%tZAI@R%dz*@{w6}HI}z9xlSm}kvfHE!gktYN zp<-$d6+r!zJIeWGC-4U7R!emjs5EdH}`*65ij&J8U;W_i^$O1au z*~i(3xxjX1BldJV9A`3_E;ya;%cKi%p@_L)fv-T!WJ)9q#7iIm$w|Nz2xj?9{RQA~ zalvqG!2(>cV8L9Q@t;Na$1!ZK3tNB-@VPE70`^}&xPU!(t^lQBJDmdRuS z$QiEl*>eOE$XGC!Etd<}QWvQV!vq2fX)aqJShx_NU?{~C;t2$D7s@=;OSO>AhGL|A zDWA*5Ft$KW@umuPaw&eOCl%vj6a}RSRn*EM0t5@V?D+x#mnlFN3FJ(##a;s1b4-Lw z6A0XyY&ZZ>1^Ke&Y%l`&n zmkZa?ksf$&`c5R|YZ>2mr%`XQ&7YmOI*n)?g-6p;R*M?d_RL zg`5oz#9ql{2D=1fm?PW=2Qx8DF1NR5%bD!?O1YeE&y)uQI5^0aj&eswdwZoaI2gR; z>|m0d4N=)(!3D0*fPjDg1lb%4c5pEGk%FLCnQ|uTunPR-Y{f7;Gjx7%~jTg4KbGLgW(W{E!ef5)#7a z2eanFqk21Mu<0?#AGn|&^5vG1+x}lY{w9X-~|dE4;~hC7Ly^L z#l%Q_Mli{T!C-_i7LXWhhC<=sz-RFJd?zjz%ya;watKkWR3VBGc(7bPG%$qe08d(> zc2KDJ;^1Jhn8PHI7#t3t?+_fs=aB3Wf^v}z1vVd^k-n4QfqL&I3%V1jHHu}-J6SL+l^F;nMgY)$3Roiz$R?kO|ZF6UGXI zFoHEdECdS*(%I`mI2;`&6ca*>$x<`n!hBX96RM@d!oo0?E{sX&YzcL?Jal0!7C3Fj z7SB(?^0u>Mv1Brq0eXwYih$%YKjwT$ALbn-^OMOUbgVFY1tgNO6fB)u{Vb_2l*Lj6 z$?Od>gMp>cu@o{Lgc(9Zb!r{-p8nB9&C?!}7#}G$? z$q6Ash$sdtiWve^MpOt_Mqx0S!iWflFgRGqh+=XXLMBWVgp9#ZFhZiZ3??^(8^vTY zARs~*!C-O-0>hw0Sx6LkGr3#_gDIOYBVgPh``uD71S{z&|4;qj*X6%K{b3(iAo|<#fGbx5}_nkNT8}%lu)5636v5Y!Ssoa zRz(RTRN$kEh=o@NyyGI!hzMpxL@b6uGFWdQfElM{8sk(;naueriG&#*6CNHPFUCD^ zrV0lWuJw(NTSi2N$Hh}!U8!+QDjq}l5q^<=evy$(9EUbV!oLM#7&HwUw|vD4XfzYU zBaN==7+3WQ3|m30@Utc3tiU59pZ%`D@dPeT5L7Czr4odent;b8tVm$S5z#TOuF+6Y zU)(PthUym+12rJxm~l)*w~q8d_R7})xxRZQ1NB62lzl`AvJl~{%)W0)by;A$|q!po#}bv0-WNs&p0 zsHmvqXC^8+DJprjew8cOhRCQ@t6UQj)~s2xI@;AWD#_K=brtl~suW0PNKJ;kqP0nB z3DIH6Nl7WTe=<)>NJ)x{ii}K-)FwqHdqhPgB`15NB}F9|3`tSxYu84`#U*)oBqexw zWW>dJcqAvUSdpCQ;qgMe#|w$^@h_~)%uHUJto2CtNQAKD%(Yo-Jl2}xvXftkTfIDP zeUd40O?Gyi%_PUIj>`l)B`z*I#T2(;gDE-Fb`!VJWD8y%m$>1X1Du%~SHn$428JyM zJ0&9y+;N-MtjV4qmtivy9qu=n)?~l1>4h8&%gBt&-JH8+ZE||@3#-9p>z3TyjND|K z+jiZ&+2)bEc}qq*1m$e9xpOlzb2mfM%=9(sNlEFOGB+ltzpycLenxsmdhX`*jEv0n zTi54q!?tb7*!*HTbm*Ej>(^&*UY`LC%6uVneMZI$$;mHdgUQIwekm_+TXOb`FT9wY zk(`~Gw=N?)F(Wa5b9VN+yo_upJbPz01~Ep?~$+HvBQ*TT8?2m;F%coro>J96{h)m@ZRx^wLA1HR@l}VeUwQb zY0~QxO>0f5@oDjE*REI*xk?`fWtyV%qn|~IUY?p79i5!0PfbpYPSr=JE{`-DyBb6-w`UvMnX)kj09etvXrfnLA!<;29DFYnA< z{w!rdZZrhuni5m=(YEv{(a|Z%xvtTv@urM*1uwsBGP#-~p`h3llWT0Q>)O~9*XZQr zWwEZQ$;;yBFM|hZdN~owe3o1CvJ{BAEb(Pq5y{JzEladrii%Q}MQfiqz+IG^SX2Z( zm7BYEIouVQa-UtbYj>{QSri%Ro0j5Rq%AVKhJ_i8L~M}};)Z3#M{1wF+{=oz@H;=b zsK}O6kyab+8(E|;ii(aZS_>~AL9E@h!$=Uxdcrq4I?6Yy$Yk8HYKJj=HEak}tBL5H z(W+=7dbKbr3OX}dxH8LE=tmH$l`DlpD)^{`Mq{*4C3GzcGsYU@muXchp)jT>Mk^FX zW0*EZD1?HcB|>3XSak9_uw%5#&}1l8MJNH-Oxm(#7-mEvm_TEU8O9A8LefH>g@k|+rvYL(trHc4y(}wSt)`;UhG%Sc$5}Y8l%y$bmeNlt z0o63g8>q^(4ayB^N@bd|2rVuuRw_}bkuojW=3_I-$wjDdQJBq_*kE(oTwAv;oBu3S zxm5|R+qyLw;)6+3&i^yjUw1YOf@}}k=6n_gm*lOm-X$e@P^rmE+16s1(0y%FJJ>R? z|1?UFo3;|EhG%y~YO?W}L#Nf*G6JJh`r)>%POX#Z0n;cB%`e&$ezEP)x0fJ#v2u(~()Y)uZa$>4dXN=HA=^`M*2%S$%NX*iR zki=whM1(TpnStftnazxdTWXAmiGf8cSr!rD8bXbLI}D462oFz=NQOAZ`H)q3cm&)S z!^3sq;q#5)8*cQ8iLphdQmL?vLr8c?NCfQ5!WE%^8i*7c1qnk#!y`gts4=0`2&y8& zJ_;ly`-p`)4mUy)=;XR9)X83A9~db4f7pBPsHl>z;rE=T8ct)QEg~YKb|WAtC@L!E z22nv#F=GadDCV3SFsH#dC@NqMgt*`nptc^e*REshnfY<^_BU6!0yVS|zfa zce!#^Qhlqqlyk}F=;%_##l@#)DPI>C-%vy2## z+OA!EJB!7ojHO(eQqUU=DN6tYICS) z)tR;F8v3L1EcDH1iWsjLrkmr_+{~ISLUR>SHXou#IjyxS1U&mR%~Et$bVev?O1O8| zD3cOTK^^}I9yj%>q6M3jk``OAKpSmBGh5{`ugMq-|BhP~SSlK-nF7+VmoR~ee)7WK zWP#uQEYvYh@j8?rpZKw(7c&nA!XyEl#niRz(<;5M{cqQHoiuH<+tF zoub$5u}26QV|J&!Vm3^tUb8{bhab`TSQWFYQjYjpooj^209z{f->s6|uaU)8p^OB!S>hEHCiE^LC3vPc-E zm?KSUl&RwcA2HtM(|Ec`Q)q<<)A5l5y$)})F3q=&2jBM52A!pJFfBFM7)OCPAx`|6!LUms>JTP=qDb$K)91>VphCZza8Vv-eMOY?C~@wpn@G0xQ% zsrYCqqm7Y*GCmPzbjYmftU8^VW`;==LzN<|!Ah8`Im)C(p|y;TFcZAZv9`bB6sFK1 z^jOG9Y(?D|=cAM)M@=zniaC;=aOFjr=Bg@Ti+V>WHvdS)Kh_?clHzPd+%kQA_z;ab zt?I9M2^}FaxzsRAc@wG@mTsxy=E{FklcK4m%!*kt`6#G*ywQY?m>XHlie`B@T&Gxd zHUj`vwV5N)6Ta3^9FWUhDJW8`mVO>egw0XtvebDdRhV_U_#`FD5+-~!wWlDg+gI4! zea9LJfe-L)8RitK6KFBTVHIT(=6Hv-v^FoZN<~T0VpqBtEl^ZOPNzpfQ)~)asp;rt z^xE>%=m~k)COV-4!#27<%B?`7FqBSA10}5^?eNj>D z{>49G3!3yV{zuTl0gLY@56T|s{Bkuy`tt0J*Veop#Ti!`JGAlR?V$1>Hk`OLUP*? zuvsmrq^V_;r>e6;ape!5=W38SiM%2_JJsOOBmaEg^3g%sIlsFn(e{77LvoSY{yzmn ze0O%==j?Z3e;vQ9{eP+aUpo44mHkYf{ErET`1)5E<721Swf~8M|0g!y9=ZPQ@2H2b zf5*yy|2u-+x!GT_@mjOLV&&tS|8>Xh*v!CU+{_SZjBjR*R2%sd^8Qayy$pAtyuP=; z+U&0|`7eYo)aly$2LT2$!v5#qp%#Ds9a^Dp#^*PAU1Bxp_2cr!M(CPbWuSf|YPAR7 z|7>bl?t^Id@O-F`2z1Yi5X1 zJ_MPSXD(-qgx)KxWUpP0C@+G7HDfcYrG_^2eN)3=@tEKVLLP35S4{(g#p|Z<7tIV? zm8U_2c}(JE(*R9;Xll4E-$IoqE=T?;ViPXS3~4MyiatlV7i3d(E@{er!Vn&~*p%Br zpOrhrLp*osEgm!t@K7ESZE=%Gbf=)M${#^y;m|DLwsJSfpOUdq=18PHPX7{t3)hI8 z2x}G`%D(i-OGqB8k24O$fz!f;0%!aOo-z z%TEXPQRhLIba70EbzBZwuAB}clAq@cEcPSm$)NE>b37&(V>}`B%fXq5x4!}|hJv^4 zAtE`@9ww0DOnndK0yO7>WE4tbBq1Ja5j0LYOW`ZXa?8cvpZF5`gMZCrjQy#a`Ip!H z7rW}A*+0BMnf|9(i2sa+#UsoAe=P5R$3oa+A>RLR!t6S=w^Um!Q>-zjQ0J5}S>{-?vtw#lmNk$+ z;cY<>jgI3F#0Oi= zrfFe|%Pu#T$e$V(m! z%6h~Gx*xw?c5SSq$6@n~Fn>q)Y1t+jTW9Sb=#E3pj?#8ijZ@7f-1pVo7p6ok${X$z4q8#NVhG?j z)KNVGl)|<5Wu=Xeu$g=pn&H~?8ts>QR48+z=kllQoM=DOk&e10(SB)1JId6Iqa7Qk^XEX8;t}l^ zbHI)qW6vuxr#iv4#A1J%`;Lu2Cr!9mloEkY(eYtd`gxs*{ZnZ zEPA@UIlB`adN(J=wsE3SIejdvZY{-;pWVTc9p&Vp#l`TSi#b3O!4B#`$8*1VPt5z& z<{A)E#8h_)Z2ZR@wY8cb?RefW=A2jb*cnGhhfh%a#;Bcw5p+y^Nagu_*HBU?i_gq8 zIy}40*nG$0Lbn;0&j`zo`K?{?)bKuKmp3r)Kia$OJY!aYBZ|(WLGe^IK6Jiu&5S$! z%gPjgY;Gm?AGL)RY~bjkE!GuuD}LzNfXeejC})c)Xe{9IynlGMwTxSdvEeIX%A}Q8 zI&p|&%D8pwKRr8UVftOS4hO7d>OT*eI3eWsD@G1P z61*LaYhw>2lxY>{etuqqxv|fEChbiq;gPhY)e2+jlM`ys2xvO?O38t(RwM*_1YH{H zn-Fqv+3caa+O4_r?ETOQaitxnxK}wCF&XXJ;G4AO*Sa4T9!%Jr9v-&nY`Y4DPIlff zzuhvu%hJkMk~X^}IV{q5t&maQVZ&ap+~bbi$vT@Ey$#N=X~1IAmL+{qkk>2(K5Nv~j}8 z&KYOiM&&EiL$cXys9z4I&KZX)%(;>Px6T{tC*+fV+L|g{KdetzlhDScrmM{>Wb{}B z?}@GN4s`JEx#jwc-a87*KLd9Z?75{-spKX%dTw>s4cc+DXGO14$&+sPE_FF@S(V>< zkH4sn+drYX%Z(BVTm6;L(Z@GU`mJ#}uc;OHmmj>fNbbuyJt~BHS27?ciTvp8*3cDFY(^h-g#V=3nTrS)XZM~qfOZ5(+4~F zk1U@af4=OSkr}O~d@A>*XW)zgj~nUHh3>5B?CLpssY~uI*Ith-v|#UA!`$9WQ?vJ8 z_lVwl>s6PsF7=rvZZiugK(9D^AXy9DgMwdPUO?%d-l%+toNGvSd%A zUx)p1p*uYT7M=A^NE%-|HLBw^uV1|Lt?%PMKdnxaMg2qPq#f}rJEM%%ytHuH2|L?% zn{qvyW2$M2&-&yfk!MqFhlFnCX^|ak`bQM;#JZ&IwDhvwR$JB#FFPuagGg9+W4imK zGac)CTejSHnd9c2C(P$u-*E3ANl%Ha@OHcC-wPynnKP zF7MO|=l3NToTvHuoEshC?NZ}Gp2R9*SPk>a8Tb6UM(FHIq>0_)uaSt?;mYvY6v&(cShob8+I($IBR`X+{yozBN$u^l!h zOzL>^oD=v-P3bXRmZC*g9y@+4K;#5!kD(_! z=X35+q4ciQ3+eMJWK4R{H>yXejGWFJunh}>9&fl>b$+X5o#P|lUvo@8{&28-}I?_ufO^@%wg1)_Q4)Ie>}y9Xby5u z-0K2H$58{~p0{vvOx?4+Q1f~}TJ72X>eDxuQ3Kk)_*8F6f=sR2KXTozUZLNAjNEf= z&4+J4682n!xNseK6P(5@ARbCUtm`H%E5sb@-{CCm|VjDB*N$ z;8%$xH3`=I$@n?);FJDeyEj;N`a!?S^DAUteAd76uJX&yKMA@%+;NNV%YoOw9@xCD z&HGx{|JZ!~$;W{i6}Ieo((BrdB1=!c=wErhW3Nq5-Vf9oEZf^-%djdD9nN1FSmoxW zrRQG^ygqqA#=1xEP0wm?*>vsGkinO>-%1=bEPY^yb-e~xeo<@Ky(jNlXZ*4B=8N~+ z3V-Z;@6o4Wp$8Jqk9yPpO2*V-yZ-23^+%&Ur=NZrxMNno2d%cvKK@{LX2TcnzE?kd zuh*9Ag=df3|6;RywO@~)Zdua(+sHiQ8e$=5@40xj(ufA9S7yFlK0J0!opGl>b?Vxp z`>p+-zEzvK=jQ20{X+Zixb^BAU2tkchp*q-Zf?Ei!`mH6rGGhmdDV{}MXG0>&K^;{ z$gLMwzV@EHde7mS->$UU-R}6kGKFg_>vFW?x0^kE^Q>v{)|BCUdeQ0PBa57zzTw@F z3*Dx4esn+kX3s4(N)M{?GqHG=N0)~5@44mFqLyVhwr*bI;=n3J){I=4*}|^Y_HQr` z>X_PnPCChgaY9=}wbx^G;`8>)yP2X6r`{R}XElr`wCzZEY6J zue18}FU>~ox|R9%(fMvLIK6#%^!uDJSAEI5CxL4HVn9^?@=Kek}q!cRy6*#X^Hx-ndcsrOZ2Ipd9urS-x-ZoobJ-3#PMSb z2333aWpwq7`b(oTcU8 zO19p+uzZi2TrcZQV#Lc&e5va^W_xF=x9O{Buq@~5yD#&qWuEagIr9PP`GD83&a7R` zt<@?uImX<-@`&o04R60S?7FaY!&SG?5e>SWf7HM7N$=z+vnlqtU-J1UNnhtRSk~nB zyGqAFb_|-M_wLftC%7efrCPjboY5e;Vb-%R^Xj+#{p!0)BQ7mC|D@B`dDV+1D<+Th z28gSC9^PQX(WJKpmFjQ_4xXpW{C&yU(BZq*?=K#FyOm?RCdEve&?T$4-+dXr+m`&O zU**D`gSIStn_sEkWOsVwO7-g}=j8Nv77XE|a>_hia?YcCbDV{)Zu{p;n>HAs-U^xZ z_KR-!qb4inbzT-Z`|Xo=U%EEw7+vYZy7krPp8gbFf5fkrlS@ymU;kU;u{s-K1J4Hp zm9JkdsC}7R{XQ>V(lwy{iBF5`Z)kGqb)`*Jmz|&4Ywp~oO9Pr;DqFvPlk*osKFnLX zbVXvfl}(qf|IJde5ML{=d(waIvGuW6pPYX*?A!wPT=j=U<<6|$zT4N~{v&oLmK{>C zdXw3U`_AoDe@323c_y4#pH_2m<88NI70r`*uA*1&5&Q4F{!}w}lj=o-mpp$lv!%J@ zgJC5`obK}U%kb)hs@#8hwn?ku>mHqPdS5NbJ@dhrH&=(xZdYP<-6ab$GaeMIS1ohJ zqu^OXclMur`~2Nphf>0hzk74`AO+q9oUb-!|DAhJhE+{i;Zbtfu<9dLcZVV>4Bp(T%<*?u-&QMf)q3o5_`z1I>jciK^`*!! zhs(|3R~%PgNIjUaX}bGyhNb$9tmd2DYG$|ETqZNpN~cCcjo^Y{qH}CvgVKYk2~|>@~m5K zh#Yv>MJOEf+y6NN|GqY#Y|j7ozrHpf``_2*|K;Z?|65+0C&^oL)P*ZKuIF5&8gom? z4HY+-I@S1oDi?eXs28_`s%zIco8X=&QZd^&A?GH|Cb&S>pyswqUc4**)46N&37hT{ z#r0YW_g|I?+-q{JD6a;6xN6gspeu^~vdB$dB!l4^k^A8&Z@s6QHs zCqJ!avKqL8qXt*A!Q?}vfsU;Wz2!>2fH8fEyjGhMhB!9GTPaNjzgHzIu?gggR8WI3 zd6TOcsYT7Jv#N@1g2Igl*LRL7!fm>bLBuI39qZbBRBqx03d1#OUdB^i;?nDMwyUV+ z6)I?>k796ExXkylDr%8nch!ZPKt_YH($qsjwT8I{4nQDHv2tn1oyxe^Rb7;>nzv^4 z<`&tkg@E$--)be7czCtx}(kdwuUnc~@@pfxxP^it(N1YIZ;+QxM!* zLdGHZC=1n!{z?`s6wRbYim=)db!|cpn#mkyQq{$DuNHQL<~ysu6D|TLv$}<=g>oiy zBA2`B@(_OEp=2YSykkdvB|bRVY?v9KGG0FZVY(iwPga(>u}*8Ob2GS_t*+L(X<3bQ z=DaCNN}5lUEs`b@O`=yq=$wG#uVdj6mg?{98fo)c%xH1htZ%FYC?avXIlqsRxX?Ss z<`ZVpr%czZsXCpv4>2HI>7!jt5T<~^RfH|}X38YXagSs;mC^nTgGm>5*c6FSwy??} zCZ&K2vy@SCMYq&HEVDYj@Zk5zXq^_;NF$6xHF%(IC6t0;dZPz(2R_!YA%UZ1WCvj) z4vN85EftA^`YA39A)SSeNOkvh^D-+XT@|6kSWc<;t1`KD<|tPmhBcBGlNzTPru&6m z3Q-g*_9aE27HF8EuNPe$qvSHg#J36#qbuc=JeBEHOZw}vS?+SKn^DoQyfinTI#;2- zFnDGOrc{hufI{6Va(24ONk%ixY8v7dR_Oqf5(U(m={d;g7A?&lLBeRI!N*NOZKj)L zBnq?N202kHW-8L4*O?QEhLL0nLcLSmm-u?f`#h5p;*uFC?3eo3EVF^w3ZZyf{6v%~ z`5+MoPGKm}9ygniH{RGS)-X3~c$$bx(OWGx4=h-raGg1M zn`TqO!>YZmVYZ!N@I`oj#hUD6^`>`(wQC`3nuCc!u{mI1h+7Kojm5$=7Sp)gXWn2Y z8^`Gs-W9(Ed+;x64QzxkiQk=!BDf9}vfUj7--zc-hh?)g0f z^PejJFF*cIwub`%l3f0h@KkwKJyl-uTYmCi#L{>C`&aNzm3N=2y3bbqPMwJKn|=au z_zwjBgU7$U-LIP8`j13K`sbpCvSmX`1(&K6T&hg)|44>bA}PaL9P^KM{6EW>>t70Z z{GG6q!)u(w>mL6|?V$5h90u^?p{X98ij(T&5Q4SiSG0&?D3mzYIiHpl+}7&I_r?6M zrYh_a5;AJbCc_1N>|{=?B%qv#l61(=4`r(Dlw2=LYerM7;!tL^jDhH{sO5!&ss<0| z*ELl$22vM6UD3A5s^a93wUQtDnx)F$`kbi7Z~PlED)IxMdR5m{(U+ zK)q`2uQ;^tN>+?o<8>@QRC<#ywL3qOU%-E?F?4NYcsl%@q+jQ@+etckv_K_Zt;PE_s|Nq*~b5?kVgT*Nkes zdQBa#warru)ck}f;M@3yz*C+>fVG?d2GOG|zte$QgG zizaRd8a%{|Qq)x_7%(veP`WapShzt``arb`k*3D9*T;&NjVS(YK^)SZSjX#vB17P` zp8TB3DSDH(p}-g_8F-egwFAe2Fl-x-AWG5{&}(RlaK1s_G>;>C4`XNJjGO9XZF9+e zgxwxPl9GYAv1fQG5X8%1YCc3_NGXGdGLo0drNLh2rEXb5QN~n6x^!r6gXvI(HnS&J{jOT1+ZuqBxgz=)Q5_zgGo;v zPdazQLwXM-l_KsfU^ctOVa97oE3$n$P)h|ehJPXB`qCqMe=+nHMQ+qQN`sF*yG4Vy zm%a^L8T6QDFQ;p@8P*?Uy1x0zfRwN7`-f$`IA0rX_agyQ8UTD7L-sXyeZ9Xnpx_tE z&Gk2E#&+hYTdGyGo&!ZfHsHrXK%T|WzPn$Ssuw>X-Q^+a%PG;wct=W~F?|Um)P4q8 zo#p|iZUVgAOwQxI>-7G{iOIQdt2XV&QS?T9qKm}cQ)Etgyc#vD(sd28*GOelv`z(2 zN)2jgxPuT0l5p&4z@9UJQ5ONjFVP!+&>M}e(6cdDLCwDg>Q7K7?t;?a2le$SVD~e? zyI0g>p5damiCQyBH%SNlkTT|30C1#XN;v9tc5-fgC^0{XH^pEw+Po*<<-bPRRt%a; zC1COA>RmK8VmsvMvCwuiNmn4xoF1W+soWjvn5;loprCzMA{p}stGcx1uzJdb!NB+553#3^k^wBzZ`!;IwDR5axa zna0M3P{mBb!c&K51&^E`wkE`Q-Q%SUL0L*H*cm(@KxoYjW+j3|a~?S}oSK8sc&UjV zi|euUPNSBzvoZM{c{-8h-wK)lURJe(W_EjM5~Y^E1*Pvu4OR%lNR;fvRHUR>lL%qI zZV*1~4#@5c_%RUBcM#>iK@l~Wax_M)UjgaNU3BInZCb5Z?ZuTDbbS>xMf-AjCS%dQ z?5}8A$Goe6zC(WpetW!?{{Bwk3r{HgtJI!Iv`0w+mHisj;5VS2zXP~_0Qh_)^8%T< zKT)eU(lU&)pS3rewxMd9P<@f}nkw6i9gK7zJ@k8^uVjKg9@-2N)`A8PZG7jQ7=nsj zAz<9Zl->|z!RXJ`a-tVo3!`Zp$&?*_ibr<(l1=oV>}=1?$W~x0+0t-T+eTTen7FtT z^dgKR;b5BG4n4Vqy)s>f+2%h0xtYIW({n%$tAnWX9!aM<9@XcFDNdv>l?4v#xNOctNHI| z2WvjnYf|V*QJqES7!`>Kz$SpIZtrpo!wa zeU__WUPe5meOKiu^_Okv^AgZfj!@KhIQMdXptp)0xky*cPdYj9Aww`Dh;#|rum?0p z8&U41-YlBQHr}sBdgtJM(`8-nng-0M| zCR7E?wgN8o1DI3jhSv1;Y@{ofiqh6*2s0*?*2mbVFjH5UCNtVw)|S_sjZc>cKUN28 z<4cvNnsKFJkc1%obdjcoW>wIewdk$G>4ZVmGJ&H%u7M!q4y4V0gX#pl99J{aO{tGn zOj~gHp*~oAxDHj}8xSxXjdha?JWPYprSjXy@$d!a^ZwGWNNn0t6MdC9jQyNgg3`)W5dD~h6yWzAGOzc@v= zKJAF*yaiLIXEe+cs!}w}J$$I%U#UiF5?K}MP~R%=badbxnUj8B#9Y2F9#BR8R}Qb# zEUmp!(88|yVC&-v7)6!ZzkW}B*3=Hg->CjtX6ss@ zT36|a95q>KrBxJ8QM8Hn@Pk*Hc3|m1eX#aEaTF996GbURMmMM`^`^$KgmIXE*Qy&} zd0_a&cX-`m93A?;Zpa6Qu-}JxiuIMoXDr3!ybJzDPda`7yE%?OsA?hTCO?^u7JY&~ zMi?E*Wcnzdz@KtLH}CTDl@mOX+m$TYnRTCb&_{|-ax#@>!4#`M0ir(BEp9t)j6p++ z&*xc-YGu^w{!x7_^3zEDgyN!q z-juC_=OfAlfa;DMtnbQnvflBlLLvpW#KvkOS2z+dtHrqnl=zIj7R#ieD=f`mp(H?y z!2thsz#hzn=&=CsO4{!O8zgQ&!0Rq@Y)}g8uZR!}>0;8B#5V%jxEot zTKTu%LI1*d8f#T5_raeKbp3y(JzK{C!V9=z=^4F&s1DUZW6IXVV8KdE{@< zy#lH8GWgq7RnA7a{-d(W2%Zf*TY26=)U~i7Qu&3%783d*Xj*X+4S3Le`w&m%?{@PVHc0MA<18f8uqy>x~%4*Yg4Cw}A zDac|k^6#hMfdim&jsfaC0u*^nCX2cVmM+HuIqQ61gQ^6o%{!!(4QchBmz~U+mfMHv z)3iL{BdBX-475EO1N6-S>tm_4$*E*?px?)_R0wHduKu*O+(ahlX0P8zNla0vqa1fWT79lsJVq`kHbRpw>T9?uUbFu zfW9n>ls4{L8(9B%2Dl@aU$3`N&73@zB_U%b>D|)c2C!&pg*}fl=XR@oTDM87Xi?A=F# zAmw<7R6&>o@eJd6O(!mmSA8{pK#`94)qz$lT}O3&(JRqn59A{$9C?YpFH}s=Qhrlv z`Vn71KyiRrLF>ad6{c5TY$e@r8|lp-V;GhOUwM7-x;4Xc;I7<7nJcmnG-fsS(OK4_ z6sdH~{Y#Yjco~p*4eD2A7zuHhsbU<^x3>JBWSA)SJ;qc1{y>tw{jkAnMj_WFQjW7| zl$uWQ{c|WjHD4s#^22VV*GuPW!lXv*TUi@KrO7lPy>2H=O4H2)s9601i>fePz;lJR zcQ`+qwM5QvOufVyP5WYmdc`;|-)4TvUBniI>r^km{Gy20VyfcEVE#D+L_X9>TO8vE zbFYh)X{hL36*$gSx{pt$*r+aNDrKAsIXiLVV}*%!ARdRrxXsYb!182qN0?qzdzR-u z;wBfsz%1v*!wb2MpOaYqdm^NHQ4a>Fht%9yxuh*z4mh+Dkg^s~OO6)?P)kqmr?(of zUG;Uw&#Ek)FW(!0Ubo0kp=pJR^v#90m?_PCh8)o0($+VTUg*Pe0MAN@7+S7B{??cf z(y8SECqn_@ zg`PdQM;f;gN65!gi_J%w1#fs6d>oFIq!*dz40(a{X*-@SJblQHJj-galEgp$p79)= zolH*4^AsF)fr8zn=lT8)pgD`8bpGr%HtZe~RW9@%_o=H1*r6mkJzz4*t-P>G%lq-o zC|^x$cySy{)&$1l&C4CkfV7wA5q7Gis__~y{vF_~G=?2UeB}`0)+|eIOPc1l;26bF zcJX={JN>O8WlUI|o)4shorM7t0hF_!itVlZ3Uev9c@Rf`E+GV5zdV6_J)jcMicei>FdoL)j|Xtz0`=CeC0t z#5cxP?AZiZE)^hCu3ioCcSUg&Srj@9Xv**k4{0TSy$N{n0I>HlV9XQPNy#ivVb@U_ z_l!K7e3y$Ka*ocFHq2cz^_N;T@gU&1REe!}GQ43jXk+GW)koSJJ7gUk+}DHKT6sJ6 zC!q$cu>iu=N-vD(sgkcnXbI!KgEsY%r{79`~ZUpVc0N9Ubs6~WtMwxHYXLiqO!FCAOfsU4&_SSM#wn$fTuv|y*gd{gU(9)C_ z&Mq^$}z}!{F@iV21+>Np3cK%4f!HyZpb7TVqrU7GwWa3 z@}g}Zr#r>%wh)wVN8X@bfW3VISBC>KCjhF=2IRq-YQJ=L)^F9ygysR^<-xQYSr}GZ z;h--H2M@U~EFtZth2>995wXi3{CjESobdg$yMaYmgepl<43$ZH?kt7T8*-f2lEp%3 ziWyRE&G)8Qg*GnDsKF^OhzoJ?AdW0#%@F<=c&*KEeIPcj=Jn33e&ns6w-MVQd%BX; zA^kEeW%7UmZprA=2&mq7CEu`n#}{d>|MckQtn7~h-YDN(Ht z-5{Og^?#{@2pNQ>V6hT+<)PO{UJj2y+*0GA3xg&ROu-3gS_*^6Z#`6sPAx{>bYxt8DJ$Hz5zX3V9v44ZaQkQy<7)d;A`@ z^tYiN{Jx?vGrF4Z=1MTAhb7PkyqQGn4?dlY?8nO;fJ3jU@U0OJo5rxosRNspQJ_}Q z|3=&1`~}pm+QaZ-P!5sk`Sk!58baO|a^F@oFPaxqYhEhSvTricm7VerH2@XZD^ zI(oRx_UW8K+Xm@MV?_$6b>OPgK~YCTwi&uq+BG+ZQxN0Y+?_nQ+YdD%#2R%#K|IX` z9x`KtcRo)JE50DG7>yXlO%_8Teyg;NcMwe)Z>a~IJF-2nZ{tb8&k39X7_TSf@lu1u z$?jA$(Jwz;u&ehBa!YoY4ZPD`9^)J@g(s^uNm>Wo5wHc(;>$puRA(2RSD&i1Kdk$C z{WX%;kL5VpH~v!3t1!nzu|D2VSHgYJRuuP#`Sa-7Fu5HZ4Ky~C|Mz)nMS0!R0i&wT z@a_mnukhZWU29GS4vCowoL_k^l8#kEim1J{B+Dx3KQgmLeY^%^6W{JdakXm6A!tz- zF1A39!^B288XKY^cv{cJj6w0{{6X6iI3Kw74pRlkY?H*?ItTCjoBLSQU}N$%Z*P^m zh=r}malM-Yyq4=N)zx=_Td36bq#(dFYS*%gyu+=XRFqeT%&#=RV-~fwIOkD}ppwU9 zrZF4kkUyS|-n7poYxce2KDbrRa*@&=Qp`EeI)-)HmDfm+EB7D1WS;Q^U(V+qcQ7?j zizYm5I_=A;gD)#*5fVu}M|h;!|3D2oKTfy>_Gbc!2uRpD{+eJe;wAWqo3*n_X#aJ)wo8({0v$hn7 zfUY{(PvqsPb5A9wd)Niydf>)+9SPHOWXe(HEMH$ zayKr#K+CmIL$6qhDAuqv>@mU&ab*{8&$(PCpraiL#HT zLFPg~XuHc@r)*Mr(k+-yVybNYKxz3_gpvK@)eU{H@tTR>l3uI4M9z7x$XTV1D}`L; zbPt);jp;r=rMy2@WE>r>`8>bRWEk^>_Tzy4b;0o!vsa_XN48LnK}lzX;WoBBnJ++% z6$dU*YUQC8$Y{Oa3+YTxseU#cpyfUHk)!R+mG75evv*zYx<<7tiIK}A$95Qlsa@N` z?Xc#Mltn)xpBCADCAGil&3ya*V^pAO)i&+;NV^X-|I93Fl&O{?GOhy+8IS)dCx&Pt z9nK;f5Lf!D(nmFHWL7-f1@}r%b@~`>L$(uA-IG#fe~0cB_4u`%hJ%gCw$G(~{FTnm zB}$KN#RQWYO~QWJ^&Dj7)_6Z#87xt~0~};8$jQlW!9FSZ$wzMwq|x8TYBx%Gb-AHiqIYV?&MWpWOnFBIl0>6SlW-Gs`h6xeHjePK4WOk zc*p6!kz#gB(vLclo>*`Oo!6!%Mpo#hI@Lx5xwDi9H)2&kD&E5j7f>egj9!g5vs9Hs)V{YOy+gqflSop z`1;1dsLNnnOKtblCJcPfpjmJ-=aS1!(~4K%HzgES=C=AWYN+-&f>qgg->H0AR=Ws? z;bfNM3aMtar{K>TPivYojIjxFA7IjcKd}Jnx2J zr`|zZ=^J@hU-f28zCKt_UY$2LfWw~O1-5wp80hgpnvJ8+zfz_atc*2YD&^I|VmAJb z(VCK8fk;5MvLg2tc722Mc~#{6;B`@W3-e!c$X$RoWTQ7z5vk-f)JEs#JjQr`(I25| zq;W>JXC+lL>IV(4%&|tEbc(vnp=cY3rP6sWK+m$WK&x0nwZv&23s=z3KeL~a>AMj8?1g!%;btT+ zqGVuZIYygq*4Ad%#k*`V`E7qLE^=I znElQ^6huv)XdbyjcjJ*B#a13!`#Xp|!ws?aJE3q8i9B*Kmg2o7Q(u$X=^6V{E+Swn zhs(Vd!!@urO}tzOAQuiXEQ+z5LuT8$WcHHoTB~S)9+y}QX#_aY6mS72LX2(!aAWYr zB^F})UiUY7s(GH`y*g2R!%*^y%k{t1Xb7ZhAf?IPjI2 zClYqiI|E2e;JFNABHn}yat&!t$;SNFSg2t=fjiGx{iN~ z|KHsD4h3=R_a10l`2{sOb7xk~_;jsBCk&+6(i!p^0{~SAQRccOi(1B=BlVCtzeU(H_S0 z6=&8)D^Ey2pkQ^tVJUC`0^64r4yjGje^h?bw8}Abaxn(XzSI|zzTR@y0`1(xbS}B> zLt%UbuJB_hM$rRn!hgeWo?DZ-(ecvBI_h$9;Ie$L84$^wdB2yUC90Odh#I?5=Rt~z zeX?7-l_*7v+8$_552Tz0dy`_K`779>$HMb9GKC@2FFf)D;P6|i}76J|OaMpU8Sd%J7Y>2wtlH$;%LEm{;0khZg#+_P~mY0<3w@O`!R-4Yq` z7q&Sx>p(Kp4>~<$iQJdy9c9T8eWU`7hPHbz>aHh;dw18)xPm)U`q#tW3>bw|XPh%M zA{rZ$d(uFxMI_cJB=5-j4Db8VY;HuQNp$c1lj)6E3t~?rM9niGu5A3+7?R*KpI|U| zfwc(?Tp!4N+1kNFfz5`Gf*JE$n{=1WopxaPJ+SfjUI2Zs$=EE973QWKSl3$#%VX;R z3-=)GW`xbk!3sw$^rK;;tz}`eiWcKZXg|7MWXU`xy|ax)t5j)J-(C!fSM+RrbTDd>r1}ud3G0;RUS}rVL&x zQYBuPs4$NiA4MXa=NyQx3z^`URB>%FU78=GFMK&H(FTTl>C=T%J<@$K8QSk#f?0@v zeNlx5&)ioP*t3G%yml|$0C@0SU*yj91;RrEXG?z~AP4lC5;&X2FRHmUSGNrdkUQ#F z47t_z9RcroJgN=xrP5eR4V`*Inx~Ug*!)i@e^Khv+CEOSOUg=r*_X5?8uCh)`BxrR zZ^6$Ho5eC`$XGu`L@pZ3yb_<>X-*NURLV-U2dR+-;hk9+)Ot{>yg_}Crshw%rRdtb z9zipYq)NH>ug`qzii0L+k2Wh;S4L^73Ce-_09(AENy*)l>G!qaM4G)@+UR>sdF{o! zX>?NTC~20~;$n*EaT2@t99Y8;ERAp&0XF|gz}FOjB^3~Ykrr>p1DY^Xj1O~6ZPE48 z=Rn+$n}Hs-`jmImv1J#xXsFiOwvbND*iJZuXwTpnQ{(fuL-&4>`$Va6O?sknAKw+l z5YCVumbtS$N@!XaX|=L$czW8(7M$s5U&hLPSWko$xv1ZMlq#KskMMAwK)=4G6GX`= zfCIk(X3qr7WL*|h=E64f$}4?WYj-jvdsE{CkO)ux;CUHU9cA zG#=O0;ZG~o`YY1To?LR2%GWNXn4JEha^taM6}+8)1H^9t^xp`WwG}XPJ7lcAB5?;~ zm2*hRWojN6-@Vw4qS>c0Dl1o0>+-M*cp;}5xh0DpVLKug7D(WyHMeGR*kPQyymAMe zLNs6AhztD+rb7*0cCLi&6N-y=zXNu!p#r(w&E_oFzI~CFJJ&j>i%@hK&pMtRJXB=A zhu*%Pwf+NCl{QcWw?Q0}2gZvw!RZUB0-HfIZ{-1j{Uk#w@lc-0JW`RXY~{RqJ_%`) z$8TdYG?R^P59W8C$6!r6QGPia9+D`r3liIIq%9*Y4JAckERFr#;{=J)^ox=Pv2;`P z`$%-*Wg|D(_S@2;2Y8V-y0^Rw!uu7sPa^Lmxt7dF_w3fB`33S8!{a#lO)pZR-Y_PADphJ_iOzzunMOdRJdjG?K`$rf zrZ`X!;sM3c72(>QOnfe3OeM2LCa7vF0o!S&sB-|2C54=XkXl5|s`@moVgd^FSJYiM zimV+91H2chS6cJQkr>_yIZ>$G;`5DxRfe^Nn#Ee%)o36y{c$3Unin4=4O7AhbaHSn z5K}&l12KE~We}fMP+u)%uzN41R#C^C^`Q1hCEbdC+PAI4wt+u#n6l5xU14i6^fxyuJ*tiKOLva@wzT{nj4n?i&Y8UVIx? zm#{Zlqr|(A@-Ab9!YSXvDjGAav5MZcILff!UGl z+zs1SIa}hR7b4eKa~%0GuDCu*=bWlCyvxl8*szZsW|$xG&Y% zo#lUIYA~pwFBXDY@ywirN8*uU12r;bE%3|Z`_$n5gz<@d3K;sDy5!7#{+8jXjB_V0 zA(k*nC5ycMlmodq=(MSdyvn#;L7uK*42;e0y7f@Ui>A_1Zzs8Zeg?p0#E+J{L!CBx>tf6@CiY}I+@MsMUX0l}BilOBH#fDjWwFN*N-KB|N%G`jGlo_vWq4-r zNNcr<2P+{bkKGhnOoSKJBoWCY&vO3JSkJti;WPt*&e>0cj={2^D=eFWz@$g0g-FcgWfTDj_9b9T5nj%KIV!Kg zI-BVz9BDDxLTWfu$NtSiJ^Av_^Hk-VD# z6*mJ~ZUcnvq==mFwRSf&- zA>xl;Njqen!C?t0a1Q<1QTaT&@}2#Ss1Z$GTCGbMuT}L)J77=b`J5QdX@=Hk;1!tk zkjAI#NWe_&llE#Ak-}o8G_!xQ7-}11hyb?Rn2wE*P-#p?k61F!pMS(8YaRWR%8E+n zO&Y&7ZGKc&(08ihE^9lY`U97_5L;{axB{itNX0HGwZdxO?hUxp2bv0v7jV$$I&gj;3!8|&pi_>5ZWsDHUB0Ihucud@p~=fu z@w#0165DlWT8+M(3Rk@-1u9wrD=Z=^lDq24I;8c!uo>By<0ADU?dx9m8TPx;Q@?(g z;l?E@P~_>@y1)=`3fco3dO@3uAU|a(x{5LE@idh_xVD$c0jp37D5eJ{~!#<@`$^qh8?O(-lGO)Aa(#wn3aB zyOg3%IYoEM4}d=LQj4o^yPGRF3x&&2O)!PJ^SlR9s4O!?CXVym263|-h&Q~{2%#M6 zWcSntm#5q-u=y%7Q2j%xgrjd!uM(-RmFf06a`i4F7yVmOLEe&iOwPH$2Bez6Wh$5w z4dEi4!Nw%|H)j49L6b#jPrw5?rq`wc7EGszG+Pa3z@WukP`}IvI4oe48dV*ETh)=V z7f!X66tHqRJa^$kM0Op^gq@iP z?({&w5gMw^IXIb_=Dcqi`uhA8hzS!`q1bySQ|LU*W>10Hy{VA5z$)3jp|!g~eU~oe zqa46BW`;O>nxd~wtM%->3$enAJw-t*I!+Ly=zN0LCGB?dYNYEFCY|28wM#ewv~T7V(70p6X-$sNr-tMPJ=g8j3vh>E>H=RsGr$G5#O zxpNJH%*I#aL9P7j0sd#%tG}Z+Y!Aln#=swCuO? z!x%gn3t&8|*A3+NC2o@OnhfEHC(}I52{)9*TC3PN2=j3Dpga)Xe+XWs&b?#PB^Bt+ zdHPvgfT>h;xv?q1zQJziRU#wpp}-TtdS3%OnwQzv!OF!hg^LFJI{NV@EKdGG+Y)cV zCyl&5^3ZJLXHs%nhJS%l&Z@AlpaZ`_Syd|Mbp3n%V=eg@rw$_aI9QEA6!T?|50vcU zZo^<4cgrcx6x0>=$SBgDmNqf21QU=5~ZW*@3rb)0kB(87@TKuK>2joc9UN|{IBNx~VBLeK> z#pJ?E4n2j@vhVToGZlM$-fl9Any`cA zK-A`G#goi4i{~DhGxD-MT}I+A2`^XrK7zyy5?VeIg-C26;pj#ph?foW{G`4g;#crT zX52Sn2>{g&^|U(?!`72pe*=PnH$h>#MQeFCH2W2;JpT4><#JAPo&@a>KLS>QhRDJ1GHwz^2+lx z^FRz<04T8)!UoXGKg>NLrSHm z596+GKIEJ5Z?vSyxtP7e*XEJ;^J>fCe7VzT)H+)I94*q8>7?9$4hR^*$;FTXyIsJ&BAOkjnA@e$;ePJNXBtH zRYiRN8gPG3U6rlW6^1HnH3v?l$43kz!$wBzU@}Gwfi{Mrm63yD;z-m(dO4__yk`@B zqnJvSNz~`*&%<6qUcn0&8@50m9bq6u?-V9=DlapoGthi2ECa^_3QT}68z>Pim&>SM zK!wPOQgh0Wto$aS zVT+`3{N)E5F2&@(9rG_g*ubyl|DDO(-yiC`EB^lm%g;Bg@_%2#?eE`jFplE)YjnYi zh*2DS^Fud^Axp`Xqqy8w@+fLvvzpJU=8sbg*wlhqYM~sp@NKoIqO+KF#jHBtI9+j@ zE+9)+GDjD5TUVN2YcM;MvpSTIbEs%@2u*XSyw;&=jzhKE4mJ3JDYK|)6}93-9h->C z67_OK{oA6UqHko@H?iuQ#pzqv^ewaWt#kBkZ|mDDj`3#44pzrbagJSVj)_^0-E$m! z+;;4xIQ20*^|Lw+h;tfba~hK6G%Uwy#BHZhik8fDvT9@Ev~f0VLY6jhtv30LHubfZ zW-?3! z*HhI`RaaL_+t_#8(w_{Vug#&atE4~EPk;6S{dvmj3maB9>8@_szPc@bbw|$X&W6?9 z{cG+%SaYAk_;3T`V_C*0wv5jL7#DIFUp6qlYG8al&iM8{<9lYNkFrdkbeTSH zXZjM)^tF=ddq3092TZ>y%tU~B1sXTmG1CMx)8#VLUtngKU}pTl%*?{VD#yZZ$HE!N z!kx>)dx3?2f@R$YmJKYdf^w|FcB~?StYW#W5*JvdCRn9Eu*!U3m1SX*mt#}3V^a=f zQ_W>lzrdzB!KVFzO_zmTUyj|-j@>wr-87fo`~th>1iSSIc3Tz>dpQnAI}YbS4%b`` z_X`{z6C7JVaBOGc+$q4hOOJEU4oHv$5Rx@QxYn0<_w&QEr!Pl0+*KwS$ zYk;rkAzvTi9}wUl(&N9lgMTD}|MGGEs{{Pk9`axRz(3Bic2aKbl-}Cu9cynTti64F z?VW+OcOR~eyHD0V6j=9IZ{5=!>z*g9TgYAa^1`~;6YJi7SofY~{YSa=i+1am0@p9+ zuK#vn{f~+Dzdo$Tmr7B|<6moUpb7fZ{)xVE1HaxN)bTXriEam7wIH;HF7I>5qc4tU~hgLW=f6%0WV^c|z)q zLYk98+8>2qdL|jgIykor5;I=52Iu+~_g6 zaqGv8+gU|+%8Tr_7x4@d@y-+RZ4~jJ6xsJkWdBEz04mV{R?#4N(O`YiP+&~d5Ic(iIYJRwRsYCjS}^f66ZcjG_XoG%1buuOSbHkY)h2v zsFLg&lTqN1N_bNk0^neylJ3bf@(5MClh*(ys=k-#n6jM^#Cl=(g=^Yf9+Zz@?LB)dXEmfAs<0n~WHaS)WId%s*&R{w2 zd^z4GIsTh+>mJLkUm-6bBrjwjzi}5%ki1yFyhM|{)J=KmPx7*C3i1jHiVh0O!3wJR z3hGS?nl}}+KPl+4De4I+8W<=VIw%?kE1Co=n&vB-Hz``)RJ8u2Xv?N#ub|}UpyV8^ zMg$GzS$lhaEI81#6DxYhG#69J{G`{gdW6o7SX))|7+ROt98$ zzSdlm*8Gsx-N#z@S7<*J(td29{dAZ1^CayT)!Hwcv|kNrzqzUX=CSs>6*?b;bUqpA zeBPz=B}wONwa)h;ou7|&ey`9a!n!LAbyqs-(uC;J73k78>oQE~GA`mPIrLZ+_1GQt zI79Tf3-oxK_4tPM);`f&PpvN?tS@A!zj3#|XtKU|jlSft{-!7TGSmig!UhV421>gP zRFVzUY78`n4YZyZ=ujK#2^$(1{`t?y(a<==(6qqNtj5q{*wE^Up$)Z>9lMdeqLHJc zk#mTVYk`q_vysP?(bh$y?d--o6^(a08heHqdlwk{HXHj-8Sh&(4q!J46gCMqGzr~p z5*A_-QD73)Y!WkN61!*;&u*HiXqxP3ni^u7USOKpY??h~dT7x!huths(X7DHtSH2+ zq`<7K*{ov9?C7FdCA)c*qIr#@`NsX7lkG+T~MSzceX9A~$h6t=o)Xf?gt>Q=JV?Ha2) z!&Y~nSly?#ekg4H*wFgvZtLgC)-P(TUkzKod1C#J+UA3>%_l>f&%15DB-?x~u=&<( z^JB{9*P;!D!uG!At%Ep#+*akQLvwEpa9%i(0NZs>LN{ z+9me0OFV~bqLOQ}lWS_IYkHw;W{Yd~wCkbIt~nfTc}i{tPHsh^ZgC}rZe=ZQ71M4< zpSm4e>3)2pdzF%Vjg$MyQ1{wG_qrDM`f2xbpWPccHaBkE++?)5*=ciY=;rpq&7Ccq zyQeqze%{>A;W4P>G3?}VDb!=M(Bn#r$Jn&T_0JyT99t%pwoEx~nF-x8TexMeWy}2Z zmV2MKJmA>+NNMX6r>)OIw=NWJec7`0_4L-apSQl}*!EFr+oIF9rO<85h1Q&z+*FJH=1#lxW>4HM3KCX{RjbE_vl$iq5-~!*;0_ z?NV>ur8%=pduf+0=Wc!F-G3O>Qj2sr~H!7 zkr|((OFosHzE#S;HO{^#!+dLteCt|$&s_4YpYc7ncjf2GxbY{viklK(j8-bv-XQ_g#5!uHM0>c>ib5{a;e|e?7VX z`=$LqF75yMZ2zyN{l94fh-d(nN&vM>0F74wZFm4(TEMDP0joy>);tejq77u`3SqLj)lLO#v;}M43f6iatV0{3CmLd45@O^PVv-hOb}GbTB*f}@hz)J1 zooJ|oNvM-os7qR?+o{maBcWTKhi;<{+aVgZ%Oq@%SD05?n9r#&zmc%L&%@&O(}o9% zh6kI3hkAvFr-errhex%A$J`2!{SqF}6_KbCk?azY8Xl2e9Ff@;k$o%T(3glDuE;!< z$O4zhqVUL);>gldk>w+iN1jI>qm4Q)8dYr)b;2v^R9e*OQ&DF|qRu{#I!_yYK{UF_ zB)Y{bx-Bib<5YClNOaHh=swz*0nwNtlbDNMF(YX)mruo99f`U2Jmv=NfeFzAH%$&q zdmXrycHs7@19wIa+>} zCvc@F@YE*ojV7#JNLWvoC?J+7sG2Bbnka0VxY0FHBqC9)BvHIJQF1hK(?X*3a-uAE zlDuk?qHB_JM3QPrl6q~D#%Pk(LXr+$vYuG7foZalcd|)(vRQ4i#b~nCLb45Aik(=B zgK3JBcZy4Tid${U=FyZb3n^O{Qnt~h?hs4eWtzIjJJl;a)u$xYw>{N=Hg(@}Y5;dy zpjcXvYFdbETBvtgczRl7ZCdnb+JS|%IJ)!%vGgR<^c3&(wDk0h+Vrf^^n(lOhnLfH zxHIxpGYVWYiXt*fN;1mYGb&~?jx1yxqsu%lmRW6@dBQvMWJG3dNoHMpX8mmD*@evW zbXga;vl>;inq9M6BeL2{vO3$dx<<2l7P9*2vIoSnhfK3Cx@KRB$Q~`pzS5pOHk*BY zIeVP@;H2uoDc6HD5eILjAG}?A@XqMLy9)>J(;a#!cIdI`p{L%5o~IvrQG4js=%F_Y zhu+a0{vdYvlj-5l-iN=WAO2c<`1|PLp9_b7FCV7x(|2VJViUzigvpdc}5m_mlpYU6#3sS z+V{06fTuV}tvJN3I4rU_qO>@wqd4YvaqQRPc%G6(wUT7F5_X`xSPQDkXJX=zzUX~pf*qhCuadCIEP%4*!oPDYm1mX_6Zl+|A@JNu&S z{HpQ`;^mEMSbDDPWUF(6(sWL9y}r{Yp%#b{~8 zm5z$5mn*KlsJOA}$b|Tjn`TF*eU99UJThB)WUk}Loy$k=-ac~g>yi7bjy@DW`q=E~ zQ=g;HGmgGEee~7kqi=2>ef#z3d!Az-)sB5KJNDV<*q4lBUr!(Ve)-tX7sr0Dsw5JX zE6gib`c~3rR<5e6Tz#c-&C5zA`r|AT$Jxw}bNC+TiaO3+cAU5KIRD)7b>EI};H?sr zs1h=-+UQ#)npq`YS0#C+YSYUq8Tx8DiE0J&Y9-%lmCS0jx@wIp)mksBb?9sKBx(%I zYm9trOfqZC>S`>m)L6Z&v7tXD9| zHrc&4HL5nftTwZ=HhZr2(97Dx^rv$rPUoqgE^t3x6m`0!>~vY@>5942N57puMqhVa zqORJ!?u2jMsm!|5b#-U1)SZ1HT)5 zkG_6DqJGG{{-SUFNM`-zy85eE>aV@5zd?U?LgMUA^Rv^wXKzKFoh>^%*Lik+?(Ds9 zXCLsMd!&BuiTk-{QRf!Q&b{nB_xj4YH!sh&ooUjz+NAZW zNoRGlo@BFuMYEA#vq@I7*_mdGu4c-xMa1sooU&8wPnkz zmaX4gw)3^_)M(wkxz*FJ)hnyj=S-{L)z-bQTK9i%4d81F(r63W+!hwy7E#_7)zubr zr!Dq-TRdNTqDFi2=JwR+_Vn`h%&zwAJMD+Qx96nP6ZC@t?O>*}bu z({c2BMbZTU=g!rh zyRUlgukL*)+56a{_o-j+^Q_(%XL?^~p=5;?4!vKra0Gs6ihyMUq_5e@)0N>ca+Sdc? z83qNU28ApKH~J5XW)F(j4@!;=ZhAc^!!RT#HKbrUq~t%Ol0BqWKcq1>r1g48hhbPx zYS_SX*vNm_BzxGbe%NAc*y{DL4Z}q{sf!Ml7oGesx@2E;tG~E;?BbUBi(6k`-1g(* zHik<(q%Q5UytLcnl4r~%?}|&l-Ix5wF717NX+OhApwvi^=17RgNLb8BM8!yC{YdoK z$br`*aSWpgQlm+hqbdHQY1yM0^`lv1qX%D)9%jHF!k6XZdP2Q?4AAe)AVd-a}(UEVwPdF|O2@~_=Sv!AWttQw~hS)4W(X}o2M zuci2h-;Dg5-ACJJ2*=+1MYP1j(baWc`dRNx=uG(jfCsCNfe|;SlP4D256shjfe|N7g_+WMDZHdQgwS zAdMUuf^pENlOtX9a^%r!Ir0o%LI{H#IRNo+2u`n&Bgu?%WD;}aFv*cTD1sSgIWh}# za0jYb@sa2Rr+0-C2WH?%?rbV3jG!X220yI^N4Pwc?~ zG7mFS3Z3Lh5fnoSltLNkI?EG1(1%xE^5i>Q_D3BoPu{?LI36NTs-PN9KrL(xlPCUh z@?qvO?rQIRK-~yU&5OK1z3VLSb+`L!mYiEWEO729NdBTQHtave1flVDpirx!nHI- zvb9i=Bpz2J=jRzI2hEkpAvg>L=NOZ0a(11@tsD>gbmj9GE&w<&~arF3?UE-8MVqJ6SCkS zWP@lujt|>DYfvUt$OU9IQtW~SMU{n!iIaw zL=d#V6Vjj`uEKBNe2jAhil784pa%v}`v&J8&cS(TgqC;8dv9yWjgYy=UI0a;K41yBNI zPz80+0Bz6#UC;vqFa#qo1`{v^ORxeba0VA}19#XAp5O)E-~+x80(~$9_keYy3VDR% zXG5|B2Rz2|6IhR05QIP&Hi8IML&)Cm0?t~caOn1&g+1#@r*=HV{fg9mUQ9>Ozt4h!%CUcwuA2Or=gEW&44f-kTP zU*S8@E2@#zzyNE25v;)mXf)J_B&dT1=zuP`fxVU*aRPVnfLxd~!12QrOhc))8YzPf zwra#~n;NMN$Gj!j$DkTv0%l+V`kOcgc=YdWAY!2{f>bIn7q86LYWxE0|ubPG;a1+=i=)>Lfi>on$~JWC2rx zI$;JDI0a(|)yX`_6{(YrW$Hu(dOFleFT~u&Hm?{da?%<^9uzzp%bvgKPi+crmL%-oXd>2urXG-(mG_4HC?!NkSk74nQpI<<}%VlA5GM zMw6668F(z>bvm*Fj=)hk23s(G99aolQCA_Wp$1OCNjL?ya2o314AjF}I0xsU0WLrz zG(ihALo2jF2XsNBye9Ec&?LU#2f=EZBwRz2L_j1&K@8{`X_5guY*T>y10it#jwVt5 z#z>J_)+DkZ2VTE5i8uH_VuTjSggj{b%}8m7Yl&K93_4JcBPU@BreOwd!7SW{PHZ!e zybJf>0o;d&@CY8mGk6NmVF6yiOLzsZ;SIcnckmHD!xAjRSNIM;;3xb7>Le{f1GGR7 ztAPR7fECz*5BOm%tb_Hi0R%tNDJg|q$QGcXb~&q3M4hq0v)V^)xZGszzA!A37CNe*nkzRaeGdr4U!9Ki{wG_ zgFR|HSch5w1VIRdVIzou1W1D{$bmd4fFdY?4(P*f@C0x0g+MqB=iodvz;B>*;xUB; zNCfe2>;n`b567a6R0B=W0B7)kT@V4wK+}VB0c(H}*nl0(!5MrZ0dnCCNc7?{?Z-TW zT4XD1gYB>rcEN7g1D@ao-oQDmMYw<)PL67kQ{aACi)@C0+qh2P5>(xV`&y(1OdsI! zgjn!=iMhZVd|+{^z z^Ozf!;0r9n$_{N(IjK!N7?~(l541@QoPd+i30=?)JHnLWdX?=n!uRfD3&(r1qi?F}b2cN^a;7r8_z# z>46Se`$~tbgX5q~txK%n23%RGOWxAxlJB&-ghHoF`d|P)!6NKgrAy*CbV&oJE@=Z# zE?vUOt4qScmrs}M;MXNPA#JTLd5BIs*F<%Rtb{JP1>Xa8$+;|D(gb~Q|Ew-~4hwK; zT$hZ(We}OxB{UCp$=%nw>;eKIenPliK~;CeuxB&sk`k{}sU zU^B*2ksin`kbycAvLG9_Vr)AcM12Sj!%i&kh8)zMNH54mod@2ieZdFvv0MlRPy~B1 zwjTl@6v7|^O0X^#S%yqPra&@OU^x|;4(ze|ga`D&5Mm)23ZVijp$5947g}-KK4crR z9r{rZLI>(jxP*EPCSeMmLodb`kT2jRyn;TAT?ENEeR2hP6~^Ei9L87~ltU%lgJqzN z*C(}b2KqofL7(V>Ic$R}_zkp)`h*eMlW<(n1wGIWz0e1PFbbD}J{kK-#oVw~m5Cw- z_8Iy_3b_eRWnvwihC1kj7g_q`eYQSXJfcrN!xDs7;=Ds7L;+ogK7JNqzk~W@26jyA zlWfR=u33yj5A?wx3_;Xwj6cOV^uf#tp{1zdI+kV$_7G6mC6oN7S0>kSAWtbSxb7+?*Eg3cQQq6-mk4M-#ya2S$X zVumDC%8;;IFj34x4T%ld!XAi*0}uzXkN{cW5oSo3q72D85CB0Cf=iZ6lo1$(%V1Dy zNUk8S!WdkG8*m*aU>uCGjVVl`z6n#{f@N25gJ~>pL2iZZumg6&ZrBBTz!SW{8+^bQ z{JHzDewiv>5RyC1|#xxgAsWPcSVedle!W4=xszE zMjDYv@C2U20=$Mdf0k2?h((G8&V2Aj4!#WTDW?m=uAtwK3Veg^3d9YfR!H0TSUcJcTFl42}mH zla3Z+GSOj7?B5y_f7lEAU_S&vAOwN$E+&c}_`_b<2m4{iDiacb41}GigOI_n3-xZ; z10h%rg)j(*2#5qvtcyZMgBNOV@Bv@&gBXklAOm4HJ>~*W&{%CkG(iirK?ig}4-6p@ zx4X|}LavLOkV&`+?*1mEIKYIYB$<#@s7Ny*M_?w)gxmswLncHBIwnjA-5V29xW<$e zL$ZV^`KD${SaeK@qP{7i7?_grEv94wCSiTGDY;*3N{r5%lBiJJ7cfR0gFFDSV1ng1 zq$x5UX@*QdnjUsm;0-?D3x41adto2!hX4qKAP9y`%o&0Vg)j(*NQi)F zh=MF^6N5Yeu@DFGkN}C01SyaVsgMTgkOA3n5Dvj%$bnqQgM2820w{uFD1lNagL0^V zBXA6=pc-o61e}CZPz$G_4tN_(2_LM74Ilu5P=fn#pb^IpaS#tp;BX1Y0bwJ!pO;Mu z$4i{c*Ep9D4$*K1&Ork-L()4_^7^AGInQiHMA^*<3$Gd88#f~vre=iQ+>FTDn2`?- zX5@Mto?9_ytQOLM5Dnv+(PH8Tk#ASTjNgtDq*yjGTb+^JZiM z*aytWri*68bkvNPTs9*ESIx*MIL?@n-V`Q^>^n2khg3xNBbAUUpbBcB4jP~dTA&R& zpbL7S4+dZiCSV5UFo3x%kyc<0Hed^m-~`U#0&d_An_(C10Z;G(!4GCc3}iqSWk(?1Bx0f zNHLVajpr6*{Ywk-e%XTPeYYT76idQOB}s=2pmM;r@T1t0FjeFA$Qes=9IBuiYJhUyk`SPRZ-bWPvxya1f-kTP zU*Q{khad10egnnSiu^(nBo(XxYFG(0KnrxR3bvYAk!|1$0hU(8&dQ2ZLb@$(XJ0%?#18CciA zMA-oAK>!3n4&y>dd87g;f)Xf$3aEk_sDlP*f;Q-a9vFZnxWP8q4)gFF7T^VRU9uwG z&;z|-KZ3chSrIMJg}g~C(llj7nxO?)-eCWqunnk!8mNN?Xo40baafZAD2JCa)Vud;$$!cIjoKbrq_dyt(gxwh1XlO$eKnaw=8)Lq37Iio5MLmt21tudK^2peR zJccLm6rRBXJcnZ@Hsn4Onc0vnmNsN7)YxEO&;U-hI1XsB!+jXSeE<_UzX$8#0yIJs zG(!utLOWcBD{vLY;2PY6HgBwlwfk(y1`q(INE;F}jK?z0hTMgFus;F!9ReT_b`{wW ze{dOPqG-3^Iz*Zw&A|d}U=PMT!3(@$AB?x!kYHpGgg_`vVQfX44Ot1apaGho1=^6^ zZbJ@2V5bd9gLKG%On43pz|@257g)f4$cA){+mKXlTe4-NE$NZ5B@5T_UVx!38G%tS zGqNS-U;(3Pw&ZMvEjbSjZ~>a33yd?d4WvQ_RDxg*#&a-7fFIU@Ao#!}Sa8^pDqcG>A#6uBh}n_T z(stydtQ}czZAa*Q?1)ve9odAos}JwuHEN9=NrYrbhYZMpJjjOvkUVKeR3HYfowg&_ zq4b;`DT4|)0!QH(RKnH^IG#pq*J?-hf3U;f6YL4Aj6L4N!})^|V8b#yjH12_oT$0r z3hJvc2HaS_hP)0pfCtOt$O+&@%@2IA9@fG-*Z=|`2tptXBCrueK@7w}0wyt!BvJ}C zfi%c~EXaX8D1ag;fim30ZB&q|pbl!F0h*u%+Molvpa=S308_Y~A#w&e4MwPMA&rry zU;?vPHb82@Pb!K{y15;V1lp-#}5uIe{Puh7hPwu_s61C>(=cRa|cx zI2XD&r+PT2a2ZJtic9c!5ub(2W)|@unn56aUA<` z3=jgo0eB2yWiajotPZm$C8@Zs58;^ean9jv6`nse_GAJk!Da!^C3rJzPvS@MJbHmY zXD4u7zy{O;AP7EK_5~r-en@}V3;SR{1VA7NV|@@Z7(yTv!XO+XAQCoWn<%6xQUt^x z8q4Cy1IQSNg*b?Z1W1G=NQM+hg)~Tq49J2^$cBS(2o8e;=FCA#B6E>>kPn4W3}qmN zbw`lLp$bkwE!4poXoMDMf@WxiHfVs8`>@pL4Hp{I9XUkJuk@ z0}t>4FYv=!5CkC*hK(QsS-|%R+dvm|!+|AS6A%wS;3xcsE6eueDvZH3xDGd95++~@ zrr{Wk1L31}Agkyc2>U7r@)Eci90(6wUE@Gf7#&C&Y+!OA0-y=HTn=Q%1_!be_CPnB z6LKIj+79G~fdd(b15pkn7Mh?1?!L$GFX9|XF_b_ll*6S&2QmVqa2YlxIpBBI4&-7g zj_H&Gp@)f92QmeWi+C*!k5MxtpCF&X3wQ;u;XQnS-$2?N@VdtVuX`NGYG8mhzyxf- z3F4p!`d|QtU<{^U3C^$;m^*PhU=RAc$HB+E5E4+mRy34x}hj z9K@g#%MwUQ*aXrb1F|3oil6{WpbRRY3L4Ob+v*^V!4xdO60E@%Y`_s5zzJNy72IGu zWIzt&K|Z{Im+%gLz)$!Ezk%w$1Ca;u2RJ^E1v!uhHBg7$j~s|6c!4+gK>!3oFr0jb zYZk`f(Q^m#7@ojWcm`T8aLv7RAQf-~jzJ|HhblM?b#Mk+p$*!h17=_rc)u`F_#pR{ z11W$)D1u@r0r5o#A_0;h1#g$}{P^WSzO8m7zv22CM{)xu;U-MMG~9+cxC8TW7w!Rt z(UA}k0ZEX8aae>Nc1PkP=14^49f>%EsyLD`h=eGJg?LDSbjW}Kcn_apm8v5#P;(^4 zU;@@)56|E;EWv7ZN5TXwFkNr0atnGJL#Y`Ko)(NRa@ zblZ_QgDdQVLNNI5NaARnNFrE^Igx#ePWX4XPNW{r!j!%fad37bLM~1uH_(aXK|T~h z5sbj;NGH+(W1yDoMATt4)rq(rcOtX2%#_hOY}@EW_~x953OzHW@QxEX0jHr3u0Suw z`yl$E6N!OXh=&A7gaNEeLMB5Bq(TOyLmFg47G%RgI0T1b5Vy-g4j~JWg-`@#Pyyv| z1dc)_9EU2Xfod3mYcL0QA^I_n4^klw3ZG&=hS;I^|f z8S-!@HoKh3VSi^Lv>$Ure}pp`fXP;85;E*eG;cbSAMc#WD60$k8x5(HDPq)rR&i7R z`OwyX)pUx!fx<9OVSG}fOIJNbA zYFp-&_OdG-ZC5%6taQy;>E5u?V|?Y-_bazE)9jR`*=$Wb zGi{J8ZHO&xSO9HA4sBEeZOk}r?0edHX1YXtcZMxpY5-k&&OhI7s)4?Koc`Q<`Ud9J zjk2rpxy#mo)%aZGU-g=jWf-<)xD>!Jn)9d8)ZgneH9x-Q-upEVm>D1ab&IKg-(PCc zmT4(~X*q}KTLaUNai(AR(age3CC5ztf81Q^zq7NH9OrI3PR~G2?_5sb3!MHFoclg- z2C#4i$#I3)afJnPMdWftUEqqD;EMgg^;1sj&%!?_ z$3JYxe<_fEG?)L%1^%&#f7ejTZtYCq+S%N-a~IamPprN7VeJE!b&urMJ+WK&Eb#9( zlZyMNyGi8<@HPtYPYSI2D6oN5P*7e_*j`X1NKhic9DvX^WPl5EeD>}-_mo|NqUDA~{Y*NvkZrN$E6dp4_Kui$xA=6mwpx`y^tsUvQhf=r1aa5((hSi zKFZ51+RH2j$t>r|d~1~XF)8!wqYQ;jmg*mG8TCnS1Dm{{g1oSUyvRS?FG^qGUzXjIT>cF<@I)@aYy=xoyHzNyjsNu!_bAMOuzQ|sO*tp{w{j})|@IA}i$ z)?Uc}(;VvcP3^azwBz2h>3mesS#;1@3f5W9*ZJ0@^W&z@uTMG@c3mn(UFv_JFBJcj z-nvD-4ea`Yiu%Hi`XV9vVg>pV&H7SP`qGQ~vg`)(iUx{~2Ff7@ss#q>%?6rN2HJ}T zy6lGfiiU>&))C6Q+0b&z(0b9(_MdMD6{Khq;`skkFDON;DMzcB5Ube&tGQ;Y`6;V= zi&hWVtsf~`KXJ5v7Gk|nVEwY$`t_9c+ePd5>^2`2Z5ADEmO^Zn|6gkaRp{j2;^Z;? zuPvZkP~|FiegXSYg@zu)*t z>A$(_Q{4Yf+o#s;3^Usqm$v`=o=@CGe;PjV&+J&Yv||J3PC@0J!p=KI!gh)k{on5P z6yod^7UmUE7A(To$TzL8s?o|PYs-Yjmmz_&VH?7e(gnmovnV|Gk(2Ge*K*OgUbHH&ifRRWaVvD|m(cB8VLMgAcDsamhKG36(&*d^vtc+6;V%$2s7v0E|Mzr>7l9hg)( zFy(S!Cj7u`@qxLv1M{~I-1~AM?g3ZqBbC@EF0s$TV;72JU$(`*z7_lSOYD2DxQ{Av zi!O0X;c?5wao^hFe%y-t^(BtN9Z#hiPwg5{6A@2W5>MYA&oCR$xE#;SoxrM^!0wvB z8IizUlEB-Zz(1R?ZaHBC_n#h4!vEI2NuoVbYW8onZfZ}`oK4bRPSWL0)>ln7bWJvn zNH#4=Hg8Y1oK3b~PPXMvu~$uTbWL%NNO3Jmac@uYm`&NboU)xeb*F0TZr4=Lh*a;t zZrb!O9h<@;(jrRIqT179X47Jq)8e_)6IIibUDHz|($h=QGuzX%XVVY;opwz}moqB4 zGpkfHYh3@TSJS!W%!dEasHuB4t9Lo8pF4X{HGB9!?b0+`a&WHw;QZ{td&>tOa36Z4 zdgzJkp=S|?7D^7iY(MmR_R!nqL+`l{e^foZ=z4f5;_!0G;cxASf6N~K^`EzBQp;s` z%jJyB*X1eD zS1T}dD=>~MFfA=G?COt;I?TucV2pq${hW@2q5)t7QCE$;^A4RsA@-`*F_y z*nUa9O4z+hB&tfRtV*J@N@}i3`dgJOZ?(L7wW52qa#XcyS+#m+wdP#4_P1(X-Wq-N z8bkLQu)u-yeI6{PdK`taE|&XdoM-(1C5uy)#mX2?XF9eymeLT zbv5pFC!^|W%j)Vn>+0v~&V8$E;62l*ex}*|Ol#Dc_OdgbooBlL(r>Asw|-E)e%QVK zQdIqDS^bsH`mwqC>)-0fdCyL&pPh0)JM&*QTbeuf_S?DlyyrivpI>x8zZ7+Tx$OM6 z&htO!&j0#$p2F8a^$&Jhx^rRO_X`{N8U-~Pg*P{fL^q0+H%fFhO5JIc{@y6d*Ceme zq`0|BIl4);yh**QN%KyV_V*@TzGi)mX2Z?R#?j5D<;~{*Uza7m7JH2r$IUIy(Jij! zE$&?{9(Vq_#Zq*ucX_LCSF8V>)_wmQ{gwDS@-#XMHg^<7ca;3i=1Tu;XQk-w(emyq zUEO1My03rl_8RBwnbhc++T1e}-7{O>GuPEKf2Zf(_nrrQy^l0{pKR`Z7Tvp0-utqv z_w}9Lx8Hl;^Ywky=v&;}w-nvCT;BJstMA91zF*(_DE$3Yn*I2~Zkm{Wx{7}K?tX^( ze#RgD%=`nangi?}1Dr7f+!X`7-2?pd1M7YaY~UXh)EpG{7!-*a6ss7N=pK}sAC&$v zD9b-2uQ{aXF{B(bq*^hg-aVu_KcxL*NSA+DUvt>dW7s%m*tBBUynEPke%SiQur2>Z zd(DfE9v7WsF1l7+bnm|Cfxb#VE^g<)v{Uo1J1Wgz+V|s90RLaLQ;O;yiJ2dX{V@{H zKbojHn(Q%}8Z(++F`C&unms>y=*MWz-|MBsf2B(EN{z>rlQCCnE3VXaU#XwJa_+~K z2L7v!npc}WuC~TpZLhf6*?qNp{%Y@!tNr|AgPLQ*9%Gkc#zrf~u5^!$&5vFGF*eSB zZBp~vl*hH1m}|2Y*XFvf&Cg%E_v6|F{_Br4uRrm){w(JDLdEr$-Pd2wUw`}K`g{Hx zA2n|*dfZrwxv^Yv<6HNQAM-bU{kTC{J5HrFPQ7KE=D;}Jk#YK-afZ9&j6cVj*G{l% zO|Wm7;5;zFePn{SXM+Fk#JZmo8`e$=YE24nnG`uNDRyL1qGwX-?xgh3N!hhG<+W}q zZn>#^;HK)4o9aC`HSgZk{&`b(?UcUOl;M^s;{#KsN2bhsrY!GHS^u1}T{~^BHSM@% z+WEk=>yc^qo@tM}(_4Q|Z(lpJQ)_1TmKo1A6p`KuD}D~De{DKN6TByIU+7*AD$eUl zoIg%capu%gi4?_KiJUua!T-O!*Z4PkN-YwOd32Q8@Rx>CQ67J~-y1l%u0$A{S=;MT zqtLT5!dGL_*&FDQ4YK;~t~404)^jk_v%@dPtc4l9KXjSNlxG8@vB3Nz@t;@#0@UXDb*f7h-bN4&z;!Ihi(&pxqH z_nO5nnt$!cPc6qRxWZA0%8^3RrFe#@4dQ)sl)_vk9PrY9++3npc!HW`M}8nGo92p))6RdB7L6-vgd z99e^uE|Dj^Z! zSrpYdqxAIj_H>Fn7Zh6uhr8*w=8?9};a-*-)~o=3U9JyiDmyUX*m(WD0!hEAcn zwkFp2d#IAIP`at0LhYHLJLiAaDL8~GyiH#rG}uQcE7+updm#8EDK!1DhL3TpfI*^y z0iA4;pejYxDLgn07hGmIZF)db3gtwWYRU`B3SGlwbH|w{p8tcrH-Ty@+1iHBNlxGd z5=fYgfEY#tf`&mv1Pqg)A|jKhG=c)+jEbVDZJ48?;EXef2&gD3D9#|xLyO{o+71XR zj*aust^GX}?eBKq+wZ#nxBm6M-@n#>S?oGB?AleeYuBz_`&5!NpA5f8gZ%>Re9{vA zXF2yWx3XYnsPYmI?Q8qt>5y#^iJ6M)&)PathAG^57McF3DX9Uu+yFl}tBjgO{)-i2 zr;La#^W5B>og>0OXJ`MyWya)$4U34-%sVv0H??=TN~v`7yJMXZ&5K~?b3-x*4GIg$ zEstVDj-<{{-E!^jd}`Z6m!}`vDpYK8``6O>DuGHnqPI_m%GW|zj*I$FgO_Aw`7Km# z@w1dFiY$C7ig~I-HBbdqa@s%sw344Dr)iZHB`cMyloWp9GI)9x6*23=g-q5tB7({; z?=`}Ii=W?pd{1}V)zqBBGrBoB{~o0p@owu2XJ2J%M8vkXz{h^6sewz*e5~=a&@rGp zR7!RcmGZJuRkn~?GWJ5ROlQL2l+`M~kgQy1Rc}>hxuuI!xup*#GAbfJ|6@eh{M6nV zKACJ*-K_mHAkc1p7SYG9ruLq-EplzDv(S~5x1|b|zb;ww$l7yE$Pvc z>$NZY2RG>WMv%WpZdRr5@~id)j!3}_T{!62P_w=1Sz;U4_hC?wM)06RkL4p${NxrX z%6TCwE%s3D%XSy=usv&wC6f^=b5c+*<*G;e#);S425s?=2uPi~<&e_S>PT?^CTpjX zJB!0pGz$D4W>eIFGXkzM%93N5@esdSV(AkWejEuo19W{l}cp; z7DCuD|98i&s#Qy-=USx9m>1HBZ>Uu6c-DNw-wPg1%P~>8QBg6~b1bYf2FJ{B?vu&Z z&MC8;rbD?~P%28*?)Uqsh=;06s`i)&zZvr}$kg1A*}mZc`4JCi1xWk@3}y_gP7$Z| zoi=S~rx_`psJja8u2o|#ESi&dFXPEMJsfwIw2#%Rpf$k;OD&Yg5sQZ}VsL@l<+ ztfk8Dsw|x>hW}1wMOD|Ts%zOQXX>5q7?xs_H5CQzk=C4<7Rs0z$*HS*kI7LbKIjPR$D~G!{0lO1@_H)`1`(LJ`oY2R7BMLUF|>Z?^=^t?Ol8(y7!>PQSMZh zs*C1~PO4dHbLLqLN)0G0yFDi@^+H7N+|tzKRJHtS4bv2|C6#T?RCY`jwd7IuOJ_FL z%XxaLvnuM{h-_Yn-_r z2ish~GMMbuf7SnY0_3)7m7>U1I#Fsaci~o*&_K5 z3JrmQz>eb4Ob(3_W5kqph*+u7G-)M+EKRwH;u=~YD~8g3(UTR3qVPjFS)@V=MfYO3 zxaBw34=E|d^XB-WtsrF{O=)tAwwY19m9#607UCP#g77>;Q^KB?&dw@g4JiIa>2q36 zaZP+sKw8zKG=JG-8G;ycX*x_Qq4>uZ-;)|q^br@~AuW!wH^&rRgyFz*mhgI@S7sEw z!JbD;Qz-7JBou|v{W_wg6!(rE%}ac2K#RY=@0V5;=u-&#no^2S8ywY?fjCA$%P7sU zFEu2ng%-pPoGfpeRZ7p%l=gCcQIR%A!{dNdCV&R-_%kWGd@fgeBE?&?&=h%T6qkD( z;}i=RFo;JuU`pAvfBch}XeLja5?fAChPTJn;oyza`InI;rIV4GE+)l%};Is9AMDAF=~q#0zm1@{#6D5`)bZFnb(&VLZDaHeVd z)^J5&OPayX$Ki@Z8s8`zu6U2a;}63XQ$4kAl-v(j%rV#*&~htWv33pqZiXvvlq7p< z-3wQ2!Q)1&P$9h$uBbK3em1mOD8;WhEAUfjQPS(-ii5Wneo1H&x(S-XasSm(imJAt z{H|Fz3%}`HD3(euhbyi}KJR&`Nf?sc7_LAwX$C)qD@eP^7sC~#TLHkic!8$zqKqCQ zkX^u|tH|?QxWWJwPlPK*6$TmRofFFO^R-pdGvNxikH_P;H3}v832YU0QkX2L3s+Dy zjo)*O!YQMQsui{~r^9U(_{Cz00{C}1{Doalhby|G|FuV9?i~@b zM;+R!0}fKAHk^MSzgZYaRXY?Z_J%8d-!tcRRc*N9)flu~E0jtRw+Z8wEZZl%&s2xo z-cPCyS5yywciWy_HNWy|0@8+_X_ zcpKz@-Z136C2vpD|4kbP+0*{rj-jckR7xrQv?TXteCEf63+YaJw}gTlMW$4lE6=2(={-k!^3Q2s|spZ6jp(J+xql{IYA~4@B>b+CB_egCvVK#OyL(s=}yGSUmObZnWs+=*ucQ^+)!e zslBcCM4f1vEb`|E8EhM#;#Iu6aJ$Q`mHiK0F2A<$20P&KL^S=NG1u$?LVyV12ABZQ2aEy40TKXW05J|n19Smo<39-S z0GI$A0H%NtfB^ve0M!{_126(~0+<1u0HXkB0V4oI0AYYMz)-+Iz&b#0z+ga6KretD zz!hK#@B+YjLsi;Qw8 zN-0YcSnDgPK&xQ$t4y6t?y|t3u0S@R$2KNCdB6@F2#zait0>i~L6&_C^m<+_PIj{r zXj2)xl?kKQS5fOU8kZzbs5NCKc~~W`390Lt3p`o>Xm1JeUcMS()GvnaHTX-y)=tfu-Rr* zU92K7hN80eRLzyT7+ddM(`Yp?I4dW)Ho6DzwA97T%1Rl}DS$}ZT&aN8`oj~f^-XI{ zAD8xBX{{Fq zyh!l-2Ywvvj0=`l8zt$FYfgDyb8~T51nY1hDMCTk;b8wtF&81RE0wz}NY|BIDyF5zT#Sfy@|MTd(oMn_r@6?oU}X4gxgY z3vLs*j!-D2c}Ktap`E?+^JrT0w^0#hG;jP@R}?Lz<+b>-j<08ETG%Le%^mf@faW{h z)Bh@`xotiuAT8?AnxfdBLl8smS31m8Lh}{Vr1y+?N901kcVFh8`C^LReF?`g;Byc3 z$czWS(bsz^G`A=T1-sOKBbsZW2k(x=fD+UEy6hqQkBkNJk5Zcd^_lla87QOkUuCpr zq4<*oweA_w;wxX~9K}_id*^6cd*bt7M!m&QK7WzY+_`|SVA}hcG+iFgeLE2zsTroo zOQX5J9mgPFzWEir`AqNY=x<7E&Q~fFQW48!(s;mHOsAw$1BR1XClxUqjZAO3#FPUU zl~RtMf6r8i6*w>`W5sfbgp)26CAk?3ynUr~zqeXi>GxkLCCt)*5`4i3tAC>8vWPWa z%rAA?9c|xfKyhPIR1eng{;MjF9;?r`y_yweV|2bPZnlTL(V`Qtl`c}%=@LgtpuOsR zYuuVBoAgYz*$-R%N)@BEO8M{#8)UrdJkN2rL62qqOOm}^Y!(mXaw=T(7W-s7&i3fR zGDSnZ1sVm~TSF^iopc4#%({5Fgwn{Y2p6$}Oh-wqo$7q4a;a3>>BPhI`dCTf>DHl2 zsa2tN=Bv6``PviKZxo=inDLbbYN!Gp0Y#4=WI^GKBMIUxKdaCa4T$rti`6iJX+R6? zR;Js8)LZHC89#0dnz99bg_(A(g96TPnRpTAhL)AD``Ex3XTb zyc{$Gsn>NLa(Z!&{{0kA`ne*f#&7mReXLu$PPgQ8frg=w4pWFYtF>$t_4r!Vfctv2 z;b!@YlW#f;WCrIRPCF0|V#?!`x!L^mL9&!J3Ykh#>Ll^-+T7S~vAMvfbkLL0PCxjS zTx!o$z|f|qmB#}{EDySl6s{$EqtRx-v#y5E#C_u|zK|%WM0kvOP|oky#6~5}dMqiX&ui`FLD2H*kvZ*%Z(&9=|(s zTcdg~J9E#KPV?#E=TVSs7xGL zIljB0;QF$8k=uq$l9xwB_9}eJ+b-1TaeMs3w7Fj0M<&dW8ANAmJ%5ui&udfuqAeEl zBDd5ltA|dCTzB!&?wb=`yImXCdTU~I*~Qir_YG#-!^W2mER9`xZTw32E)(q{?7-h( z`LJ!G={G|wZgAgc;EcG!;eFZ>vz<3$?`M>TX0Mp=PIM>KwprU%QX5{8J0_y;fL4#y z4jOyUM$4n@B4k@4?S`2@+H+u{-I`~w0xP1ohc7G*EseCj(b|&cVWxY-a`k9Wt$ub9 zJt8E6X|}7oueK{sImEHczET>>D|}j<`6_K4Cura0NhwYvrwnU;rw@=v#lSFq{(%wO&P^=5|m$f{1&o@U;+%|HHHKeI~nLgE}IGhcIJZDxIjT-q`8 zEq|U|azSD$PzFSa_{#&8`9&9vsw*<8r1p}zbp_sWizD9F&pp%SxuZlfJW!%{%ExP> zt3)r+)FSQ-SHPd+VP>a0B(S+P&ezK1`t_$85j0ta?OcbICP5FI-(}9a<4FqB3Ce z@DQgpjy7R|DeRg|?;ZFwp55-K!Q-}v+g(g7eVr7!cICNznWLJye;MKMu(7wP*N>hfx4TL_emocXEG~0^nM8MfW#oi*6CKSX=`TVi z)Yu&vQJS#*s$KrTN{*>cX8pj*(292&+>xb`<2=Vh>aUNl!J~3!`Lz)>k(Jj%?MBwD zdU36Lb4+&8pwZoBlg@2C6Zy<`C zN;lwjt&44pl*uO5bq(QH(<>DT#ff1WE9+YtBDr45bGf0Po(VP1-O~N|k)D4npfaTD zW{CYdVS=UErtHVX9o5Au-3Rtjg~c0rS7VdyMVol*Uhw^kwOpZBD7{Jc7O!Y4PP*=c zR(dWq+}^-$gup1>-ejpFZ{WO4Z-dVT*T>5?=YN!X4Ary{C-@H7xU8&A;pZ@Wh~?7- zKAp?LR!o&Eisue#HR#n#N5ju@p~~Vl^vS|(i#6y8YAKn$z5bxtp!@ zTEjwT8ExcQ?o!OF8hxUo5!OZN_V*=J)G|LWX89J-{$75y5-3-3|c+6); zh@@)fj@DOY#$H`_lnjiJUQxLi3pBKiVl!!P-kc8AVR}-s)p0XFvmvD!XJapQdHzZ1 z8MmkYzHIr6*daH$r!vH|54Xl9D?QCFbrWl3ic_n)2>SnwwfWdNspj~`s?)9Gvg>0n zb$kA4uE)lz$A%i2^|2mz_1{Yb8*vVNKMU*Z^gBAUh4ovX?=KY%kIV>4d?!xu8wFq9 zN3;CLSpDM46S4IlW{H;NbMk9jJvSY?{}Sxo;q}DwQ!~EzGiKkZ6$4XsZ7-c3{nD(t z#MPv9@NnH1FJF~k8?SpS^Fyqs1~=xHSF_$?K~H+>N7@^6;Lba*cCd7rkA!4Q^;eI#ZsA_ z-hzpl>Ft*dk=wdhWays|ux6;uvs+2kdU2j5NJAtn8gHZ(5Z>~^SKRfO=F;RX^g0Ena)6bdsyeoy zzf@yF{fJ~)s9F8RjKLAqcC&3(uV_X`gJ&!dgw<%`HX`%v_GXiDL8mz z&<7vaQ9bX>+}`xW?9A3lzos3IVN7*W4JVJ<^`f=L-^(n}^yQ30=1b4qe#Q&3$=!8B z`~C6ySchh=#+>?Tm70Q^8TD)*AD#ZP>iBJ=w!aH}ppiLj`@7AaR+X##^d=4-*<=*Q zHV8G3rSG`*d2rb<)5*8i2LE7_@$66DQ2yv~@qhM(KJYMFn|LJ_KgP>Ohx0i~Hi(R#GAfZQ?QS&8@@W`G_)m}Mwq+EAr1!kZ;2oYGxTn>fZ976dX7 zWpE~q6;Uh)_Y!E@%#=}36b+hmAA1)#vQ0Qr$P%0a!+0LFz%)=Pr8yK$(Okbkkf|fS zZT)2oTo{y|ly||6xZD*YDsUMS$e{xDXfrnjhhcrt3OZ^*E1hTuo{l{#;inkU5RazJ zg(=DvKxXxDqYEBMP|%>^nB+)NK0qYpn|HEDMJZn|3mpk$=>nO8V+F>b_pA;5)Rdk{ zbw+h1T`gicj6!MN7mmXWN@?v)w|FptO5)xZf;gr0<}mbyWIn1|@Z2dLM$Zu`4B)p? z;${V(D^+lmw1>T()Lc&Sv?%mNVJ>ySz^JYmONJGG)QL7fPBG?wl(D&O5=}|XiLc}q z2h$;?Sg?RY-2FVRl-@4FjRZZZ?M64SP>6=$DdJ?|Di+OkLthPm1o{l?MDffE4JcZI zndH5oM6`qXbAgo7=FoKpppPl0V_A1fpWbA>9%B}Oa^$~u5$7>2$p0(F+A2T_#u69GfX zDf3?#W&kfu%2!fU-IhR#p_eP_AS1Ysp~qqMM08;}N1qOdg!1MZgit~br4a6t`%q%M z?NBlXR2Ls9?JiPgC?GMXJBEm?6ns5G03y81K)TezT#urcfr(O?9w-t)f!xZ#GD?d3 z8=9FE3jsz)!8s1Y!1c_b10ndBGZAGCnX3VNa(NF7C~wFY(QceNGq0*79N6UHsF2=(_pR_0^HEoWj5BZ1JmnGWyA3ooo zjEve1Y|38*+gyJRZ{I(u$iG#<28NP9E{JHU6)?UI3WI~jeXnWQruk1AmkyQCQk4Jt zhwt~l+213AtL3Y)P~H*xZ~Z+${6BX1NdIv7(B!VrpFKV_?;kE7mp{9FYKB?!5eZJWjI4e5GRjNBz^VWbNm$^SzLbM)e2DEfF4S<31gYZ;rt@%mC((; z<@wYIRFX$|Lxpg=<1 z78X%29c0vfxPk8rJE*&G8-K>tm1n{;iP8ds&EY7zwwnWg5e6V}V!B;_3{Y%0yohCXs>=Tyc>@^-$JT z)M*Ebx&j~aMfCe8_}ecC8=3Rr{4>-Ac(Ko+E9V?kAXo_h_($s0U~-#A%FvM*d;@&T z_2}nGxVMi3@g(9WP^m#2LPglJ@Rm1HwV+-r{K#06j5<&_hVC7LkNk+RS%$kb-#qhb z@ALDXhFAaUef~oZ{>l6Nuh|&M`~P^K|EImr9%|>aGM18xv|zT7<@z>Oi>BlfR)e}9 zD`6N-zs*us#%TLx)CIa|>74SFO5ugp@Rdf{v!Vv6$XqVrWYRuz5xZ1*BDT=pNV{RQ zw>VtDWIT*J5NKqy$dB9|S+%fUx-`n3IevWf?pQO)B3@>Fl>HikdFI)8qecD+OZgry zOs4l{ev}gxBASuP{#+nRjzlJaI9k86N1anCbg|mcx@b_wVXfEqN{FkH9e0q(>>ks zsXP*A=rGP)Mb}`h*0|kKUGvggUd=K!%gYY28}23Hp6d8i5V%E`li6G?UpIJkgBvSi z2AvJPHIW1BWck`HHtVTJbvI+#O?lLGHrb0`ZzBy%mRskI-V67pN$7(57P#3Kw^ZF+ zF>pPVQ_wt-UGGOV+@Gk!r!%WMOI&kLTGS12Uo~0t$-f7L{q%Bf6r_z(7U2f&MZWOon zdOgcy%gV9N9kXJz^G~-7pKZEp@o8tt?8uT%c{RqfBh4Zg)r{D{>AoztCJlbr)d5%2 zPDUv&O?a1fB*tvb4(mCyNAFtw{Nd8G@lI>(6}Jk!s5KX)%g=gIH_z;ObtcwYvY?my z()d-&gKYL*u-~)%cx<2H!1T8nvpe4D`~aKfsAufcfBEJ3RhAEmtDJ&hpoK}ld*3^ zWzuHDN|zDC2U;&RuL`}KT)QkGtDz;ScClU6t;T`Yw-yp&)pY|)wRWI|#gXnC29~W% zSh2S;(nP#C((ZDSDq>v9o*nisw-#*V z2bMalE?JCRWe&Wr)fc<1c5Y1SGa_%_`4_XQ@9Nawe&Lbx!{q(vUrh7jOxb_?6|PXd z-2dj&%(^bg`+t4qwWmvus`I#B^+WQ0eT~#ZQh0)1%7O-y-jAsxuq+t!lM9((pK~{Z`db z9Op(*EHoM#1hakbCcD`q^#CY>ISnmU&Dzg%fazvx?6I+hctmo2PbRxvFwD(~o|<&# zFAs^%bpK1qB-6K{%I(5}oQC26bK`}6zL!JV&d$!M`#D*rC(Lb#p6tfF_p7(4a%(g6 zH^cI5ojvQs`fi1-@1`A4=(3D)#;MKqVODA5du^1BZGYtv5c5+*t8n^(pO-%FV>Ug& zH-GiBcbvj8zrcZxD(Twq|k)Ylq5 zFr(c0y<#V6n(CwvjPKcTzVqC`2N=AuAmCDCk;GW3(8XA;6-qMGWyU)5G%h5!*9*A4 z5~sNBaLjFpsgs+g$$j%Kjg=VZYdqsqGE3a?YA}tTyH3 zVs@Sk;N+%1+C5G#@ISh7ikpi|ZVu)O7w0sJnm$eBq-%5&7rvd9Gob9u8dIC%0i&*1 z4&116FS)#6>h)#58NsFFy5hP zy55H1{Nl5Dv4Au(>l}^_3R6oYBasE6%lBi5rp? zDYK6ZY&8ao=>~*;Tnia4y~2CLk0_-XWVxZw`S03`&$jR5&IVA3{fBwtri-~(rlB=AF`$tYre)ceU zKV8!M{GDlgZ%cc(Z%Etk{c_dq7r5qWpL$3*`M_<5RFgK1{oa$4$pfjrxB`0f{>niK zvvZH#n|c0L=>B6}C#*}Dw10PV%E21Xy}O?ze|{KTGonZ8_B&piMm}|!QyMxZ+IHpI zhcO4{JKC)_sY2g(uIf8)=Ao<4dp#-*JrMn}ps(r7iXSSkcgK)+cmMF|s!ols-LP}> z#_zps-#lPLX#J3J2BynnDnjLjY4Z~6FPRq_YcJY+De?V)Nr{`nCP>e3@LD&xtm$kQ z*W8PVO*bcg-uiRP)%_D(*9{(zeY}5a(>Su7qipY!6xUQd`lOu7xmYpXI9Vs%x0_s^ zC*0joY~beSmD|uO^p+mF<5=e(Ji+T8mPK$` zFKM#7|DLT)#?w@Op9`X`^v-Vh%-UPsIPGI`Ah7GQhppG6=<7?f8H)*F*A0f71W!0) zD?00Mkx!4k+-Jk}^)==?z1Iil2Hppi2hhOPu1Gbt$dwTBb4tD8rk``_t#t4bcXiXa z%6-v2%9(R%iIg;MHL-Q`}E- zCNw=+F{tu#ZS*$08oQPhK4NkIN?zWX_N&9Ef1VrN6)y?h=XrHqJ?PEU1JT`57qr=o z3yi(^*q4)js&3lAkwymwZJKhxo~-G7lXCOfnukYFeM%k1g-sL>D7m~99v|{&bMPci zp?}=e<;P8*@Mqt2ou|=TH=DKwPC6Q3I&K(6ewrqqsdS7Sm!mB^Qgl@#!*r zJEWMV_1wg?6#0}~F6Qc^22UT^2DCmrGa@=D5Y32aF^%RZk(d#S4aAOgd=My!;M_r$ zSdZ@NOF1IK)}=2#DK1Ygu}pTDt4x43XtUTI#VKk1~_F#^QH7SLXMV#DlRh7 zRnc;N^gxU&M*ux~043(qe)`V-k+i8xh(Zs;6M>YzTr8)P;>99IibhuyB9T~bAQtO0 z&W^yKXgNL6N~G^?K#3g1j{0tD>n^+`wMvu^EeMcJYb!Ev&8{}w} zi+>EskrLUaV2tKwjGif7$m^i@qnE*SXIv_$^&R1VrM*}KS{oi)+T5EJJBrNr1krLw zu>s8$`JzKKU*za0b{5g4Qb&EfAka6WhtRL@>pL5W6pp@PF;}dQ0b)|XM+3RSTPzZJ z(_pE$0^gJ842QKyLK0J^pKmgSVyuIEzbJ=GzrwvA5WmNC*B5y$uY-=3@o^ z3d-A7kDe#6qV@H>$AGUG25swSC>AI1(BM#UvLsyu4&W68E24FILm55vh}LFkx%m)U z3)E@c@}iAde49J1>Aj5+$<5bz4-?QBpmm8r-<0l0%dHcLg z?^g_8k6uX^Ksap#c>lxUeYVb~%?v~|PcHVRmkGqUGnWwS2KhF_Btu5@XcIX0@%6R- zb4@W(-V7vcL3c`I?(HqJ)gnWtDQEmH$Vhl^F{p11ug=yNG##&o5$0JWJN*l zz2bQ0@Y}~p=;8R}7i~^k(?bo=f&ne$<KaIo(r~J(;Fi^w?XAx1Gj9 zqs`KrXxjS_cmNe35}VJV#NNA+m`IQFj^;&rul075V9t$bO^WVM_aS0qMDs-En007x zn;rN5>~<#`&p+Jm9i~PbB{m;V53yNt`kUwT&l>-4{h$AFyOVpC|8cwjTetiACx3Rk z6S@0axBKV&FnaF)TerKZq4bZpJvuJLk{Oc{M;j;BLxs-L1ef2Q%b3&F!s z@kfu$Qnb;6laosid5=Ew9***VB;EQTi5?cP`2TAkwf|H>@;#wO2Q3xTG5e2RYjPA8 zXO$ZcnR5=HoF4XHWFXS>y9A27g0#@sdRdbF~v2<>peOI zSzVndZoPui{*db#$PrQ9GWjjMD>818RH3NjY6wD@ASrn@i_h=jWAXGv4=%JO*AOoX zKh23kDR!>FLi2h8i}m3S{sORghBmo|JG1QI-+5*6uU9rSWA zUTAW;`v$jA8bg+3*Aa15h&o*E|-UK(`Z`F6Y_a?^=JI2QT%j~ zHeMSGdCCP$=Q{N}r4)2#AR1vrsE4=GT&|4b*@6(SH<&~|;LgqVz%!oG>^vxg&{g97 zEPX{+i^hxQBOIl|bc!p+WuP#SgT8{pA#|!S z^$WbYdajn)|5~R0R)#zB7{hZHI6;O#d8E*SIDwUcyT-#RfE(SAPIAMk?64#i7iN+1CYwRU3@9yRYkgRh|jXc z`Odr|yfI>#gbjQVyL@dgo&+y}Smy9Lki3A-b=7s{Ntt&dAm`Nt@I?5W1B(|&=+8co zHblY`iN5>sLT*m;9-bT3fn?%M)Lx9TUyz8ug|aV_YMz{ZReTvENjZwP|tZv@i_0RTg)75=Lc-Xy4f_ z=-X&i+yJGi0r(1>buWQa*a4jASHPKqtcusbX+!x3B+TCMB}~`{3}QarVI9T%HVCL0BN3QI z1AjQsCIPKF4x&J0;`tbi>h)}t`~HSXOM-smMd8CPED>&o84SS=(AHRD-p(#rF9$c4 z5@v6syuhBM86ioult`|kBp@~gQ`LI{s#r`!ai()BBzhaVfY0GGD_zjh!-$ymLPyWJ0jrZcg3*{- zrqT;d5LVvz1q(2(%sf9}ofrTzYelnov&e+7ag*kvyOTDrM0fZ7atPgB0{*d;`p1zi zFZl`CD}0&|KIPrwL0+D%&o@tIFIR2=<_9b}=Ez|*a|x7sMK$uItlg8lK)V~Yy#Vqu zCjn*q&@z*VF7!iW_E-!`YiJcuPMKj4L^Qy4_IK;wQ1|?%-%7o+PbdMw45$thH0bIMBzXKG4loB@O~s- zBf%3QVTwdJfQSKE<(+i+GW9Aa8Ml$4(1nO&YN<3NPKY3JJN3|u9}Ezbbpq{=$oJ_C zhCpF5rpAaUfDPei&YJ*xktuLaA=Rl1#zUDQ6^!x|03zd7Qjo7g#K#6Tv22*|Zores zfJZp2-yJE}o=8Qb#$^C$FF=l3laWXVtU|^-XDpR|y?~a8RBCTj8U)c%Gt$Wml$Ag% z13W-+mwu=Xo-#`nD1rWE9K2B>8luE_g&>##^~H1R@Z8uk!7I$FrVc9gK=EENnqNqGo@|6*BzVrGOAt6heyHWZ_=RxR z2%KdufCG#W9cOeQtL$<5HE$NHrTYQ&yzP;(qVrcso2D%g6n4;;2|aaJ#FeuNd14+* zy+jXlDx83DWg8H-ZAW0c7r}{Y1QQP;I6#)jsmBOTwj-dRx|lhyLHNflZqOxURXS`% zXOed!@A?Watb(r#{m}X#BEGIfe2Z70D0uNe6rGqEf}$^}$Zem6AWey)lhL81Xmm6w zBD$qL9)bNtj1_XltWO4co$1JOpNAj;ijf&Zdf=r0122S$f>=@$fahxnyvJQoy3hrs z1ICTU24|8#22?MWBO~-SSua#@5(W;P%d8s${2Vg*wHHuKxu;-fxHSdlKO>*(rpP*&PUzK72)JrUQMz z%-03Y@gf8tjS-A6K|uCM!TZ_tO7AG9XBsjbbI>Ny-xDz>o%b1|FXxXVfAARua~siM z+kF$TslC(!g!OMjov_~71F-6EedVp9lh_}ehG1Y*QM>%aJoL=B3462P_)im8(=x$& z8Ly!wtnRc4=wiktv@#j3P%F`eRt(+n239iFj2yzN);r*p z*9zsC)Uz2o1T#4R0TTqmg^7Kz*mj`BOroXzx4q=yDq&9m5 zJv`ApC@!@Y(C~snoPz{(JUhbT9S~%vt-FP8{k@(!G+B{(|NfpEfG)FzY1?nvW zeuO2MEW`+@kw{H;0PiypG3|-SeMB}nqg5Z0H+b2vl#IP)y5#f?)_}JRFU1CxykumW<8DXqU_csbY{i3sQ-x zU{Hof0j%gF$ZABEnJ1|4`2llxQyy4s-psx|G6>aPokDvC=TNsV5Q!Od1IRyLg{gQt zJ(%al^k0Co!^rCg5o8l;GcfR32hgddmkOb`^2y3b9eoxi-qqr#n`squ42x3`R%lRy zUAKGqX3P$>9J40^qn{ax#w?*L*_lD*SYQow1W-3D@T&w-V<&ZFWBq@-6cviVohr#`ol6^pF=R%E3S1LN~k1ec#7{}Dlb zj{I{S$clf1tVN(lC=CAuakSEaTymt*O&i#cgnKi&(n&^8PPL8)i=ZFIZ9 zo};lHYt@z`!GKOqe~ba~XMm;;B!F2r7eNtp8M7!CK~4dJ0K&D@6$m105uC&9G5yHi zVn_BC+rtPNjvxp+3bu5zYUe4LvvQHHjeY1rQs} zVFyaQkr)R+^UN9?NU2Ro)FYu=1wv$)LjdF;lMGmZf|0xVaw-l9=tE+T;lu5$+Nytw zLg6018zn}DHNY7#36KsTn+xG0Q3}{()S{-fmmfvUXvV)9V_8*$v1shWSh}Lz0g#U( z$$s<+W}cdxLu=6|()Fw(s9+AEQ5%RWk0Iq>hg2C5u?eURzr`n@MPHr-QBPCFpkp_5idIunMxde=V|$h(c=Kh%zFx zPaz~s+g4#}n7$M`laCU%bioF!9E+K-Ca5cDlb#QCVKcHEHRHyD&@S|` z0r_r8*a!CI?LfVIWEBxv*C0+3Oe|}-XgBh&qVsIe@@t^i^5z+O_S3jeP<%IhoGa_2 zn4K`k*kdyy(Zl1}An*kQM&*D2mK)pU>qr=8SG!CEzjVw{8jZ|lZf!h?pj)$GCmqX- zCg#SIsX8ET=K_P5ec!J$!d4q>J?!P5>(S7UL*MVhL2n$?h9II@vkJRLU&}3^*hq{8 zEeDW=Jh6wK9Kgoo2zAPK^mznDLN8dcn^rLH3sJA-Y5zU6NFe^Yp&D;CeuJvKzEZ0O zlz5wPT4V1y9|OJHy->H<>^X*Vt1Bw)+BtV0EoTCn&;r&1V@6KPV^dU6ba&9;v9Slx zisz!d{pj6pOakj2a~gH+N1#H!c#{t&?PXHf4`OAF1oUw!@uU<&dZ%mJLteaXy95OIhP!9}~iPl3^ zXf6h-p2|j5VyCS>3PebddV2~u28q_V44lQJ2UAja^3E_#SS;+!4^?P;Au0rY0-c!8 zAOR)DME?q&eMsL6bBa&!O5JOKM;eYJ=R9Igv?0;iAB3|9fp8SrC*}@CP#uOqjDv=H z$NV7?WT!xMnct@%xRH#YmGG(;)&=8DR^Z4AG$0`f?x@eq5MAkd9#>T*O7*k4NJHg!RosR#y>!i@DMnT zhPAMNwu1Lev=rpAX)rD1)Os`kS&>+h#O`B-1_$AKdEzmCjm4;UWo`Y;}JNZkPqP}9ujL!UvjsJGz~bYN@_c8_CNB*Y?MYL-K$&LR?8 zj0mJI$$`|e?Cts|K=Q<9Be+{9J7O+n$Ew^g%p#~`){+eZMbQLeaUQvT z5ZNn<{Cve4X0F5jW7d$3s1gRIS{$K^n5n;DjTItd-hplhVt8Z|CyS1#g*R{Eir{_# zz8z9}pMhe7BJ6<7=iea5(U@Z5#%7mr5Ao`lGp!)M?lF4ub^Z|y^A_|U>uGWnJ)YeS zn2Uj_EU!n&QJ8sb)&545xE@5MDa6>zBlGQv`DT~By9&lv9=`^8Sv<}5D136+J#d_5 zKRz6ZQJp7CrGzY%X{Sbm+Q8F@Zg`o5!C!i^9SAKbW5zuPlYTrbJw`_f3WvGX!7TN? z)DNXS8iC$0dLkNMGZo!Zp9!a7vd%V6g@WzC=n}Dsvu1r4fVhBgwu+DQyuhHFpdMEL zsmX-O2IvCz^RX;Yc@6PoN=_meNe<+zp>zpXh+MFVM;F1(PDcJPYBO+&P_qtxTM%6_ z0C?-rTGT)=mKb17gRp}41%ZZjD9W(+GnuQ9rQTvor4_s=rnNgt&9Lei2M+|%L|tu- z1M)>6-@1fuKi|9!CupZ|FoInmtAuosQ_~L2FYErI3dhcrIMk!@3dAr>L&I97h6wB;Y7wx~u>#9NW~V`|&HWR=roDP63Mgt;THq zj3Q!KoGC=ni#4D*cpZX=m;=UT6EFjSFc}C=MIhvcM1KIzFpOg{Y~TZku+VR6qwPou zBE+(tvjx*wT?!H*&~@rJC%=~=tEd85y4#R76hv+TGk6D@eMPj~*xd-mR3p%@0Rk~# z*8@%fX;VFf%z!9`sbp#SC|snOHY7pL!zw1#-mL5*=HKK{6&}IsS?* zfnDcpid}M?JJS35B1jHF5I~kpQx=MUlbK=mEp5^6RJ6C+4$OWsrYm+Cn`Gd;Ap*H) zp(SQ|SRy&HZ!WaObh?5)toTq*%x{(2=`mFTH~ng~3*04*N< zh6{rkZH9ik3`DjT)GFJ-@k50W25A|o)tf~yNgIKq0JVq?hNE4?bf7oPVxr%$x|uxa zH)e<)Qe!}NCSW&mS7*Tw#c28SRB9xSHjwn8@YvnDgCIhcz-K<|BUojC;GP6QG|@t> z&_c{>YXse8z<#5Q%TNBzC+l*i#Hh;kn@!qFeD7Huob{2CI zW2T;?B{Ti4n23J!aNJ=I?gX7{0p#7-u;}~Pd-36j(|EBh_<96HtDkRH0 zw15E@;xP^C`A9(IJaTpr+lPqD0O$thS0W$m16N)MTi^>ThdD}?-|j@Td=flt2Hw78 zC}GRxBd`pki;0oiFdnlpEasyIh@;B^hb(RKuf2}f1wDNwK2cv;JX1)%WaDPiRy z?j;bmA$)=Bg1tij4yAj&yXHB%jvdnMjk4=EQTa&C6imnaeIPcL%$bvJKJfOfxO$#; z6O2!pe*tF3!3o9aOg%ahiM@nbiS@}mNJX$e4Z(A)ME1n=5}upj)iU3UbRhF(HL}gu zA~+2u2)5fh{zOX|?H<6}k&fnAx38EUa&}}(c$gh#l@8Dp*1}6T{uSAvUb`KFrtS!w zP>WeA$0#nKePhROD~0ibgIB$~c?P3L&&dMS$84}A{yMy5^=l_d9JF7^4i_xI8I>rx z#34rg+eH!J&8c#{eqqPw#3B5cjhCaWJih{Apo+NZH-woq(v9p9>tB(u8+7*yoS_`7 zHi6zJn|TgKm=9#K zL6URG!KJ7^h)mweEeIM)5s1n5M>Kj2+5Udng{)?>&*|+&KIsUAk2ziqZGROBqHJ|* zkQj&rF^5*}2g$Np;2uIsuMRZC^+*xKb7zn$J_}+m&LfchFUH<8p3C=-|NfkuIAza} zy+X(=mCDLa$R<0YWV9reU1nx>S;=T&Ejq zkG;qH7$06isr>{cbP~$4&rntnfgSm0Qn}_Vv1Olsb#Nop2$>zV(ZT5)Om_5JB8r#< zK)D|e#Rl^}E%giy6KgORg64Z7UJ5lMs-H;KwgnOJqF>O;#LI%;Cb*frHFp8p?yC9{ zsJGUWV-vF$vIm^r3NK+Ak?5%sq+^OiBNnV_v?bOpW?;o?hSVd`zF;Nm@;0LfY5bio@@A)D7v;?vNZ9?=zk7qSg&p$zwOrP31SdM!E)Q z7yfQh-UmUQ%)9F8h=06d6Vzh6Z%~KclKxpc#`6$&UMgyukMO)l5QB%gS$-K&e5)qo z-**p^U5yo49#v7}+FQUq+!?MQ`J>OJ*ISi|BiR z=w`Iy#Gke#jfg`J=D5dC46J5<#F>mLXG>BOA&7G!5gs;%K@lNRXu(=2V(X#!Y=#oH z4SsWx$VwcdvCsEaNmo=tS7bjQA!3VCI(myqK7893z#3NU0~2Gc*z5yX^k$mB2!>H; z@y`f5fw$FBuDc3FhZYg=0{1!$<#F${g>$! z5><-!_3c-K37@}>44TMu#P6CNfLDMIRyz5R!L^q6{2-;lM{6LPKEM1XC!v=6yk?P_gasn`a?{%Q*iCQDm$&61FL; zp^&;`_GwC^YgnS&F%sjlXow5u8fbjw4_c%naLq%A8{BZ5oP(nWdZP)#vERuEV!|Sl zJH4Mwkd4b=dW!s$b#8_;@{b)oxmuuixuCWKJQ3`rHxz!K|K+#X7s0ms!PE#}dMi*f zWNCcn59=TlgITz00aqsxfSJ@pZ2akOkjj{~5@g5jaef&xr=jNs_1GJnFETsFo1p5) zbs@X$I6R|IVSZxv)pn!iF$(xQN_8Ha-)1(|8NU~lx{)4T#Ep7KGrnGa6A!BemZX}hUE?9bxVCF zqODfP(oe5_0blDrz}Gc(vLpHYvGEBOM*$XO5SAk9=LHIEK%suNbHLN$t<9u2p$F5) zF>5kC>kcC7{%fO%Y8zRRBksetZv_ShsVe?w9O+M~cxJabmI7wn@-YT$RMh8ZloY?N zW706i;7|A78Nrwx3`E$CK?pl7Z3JQE5!QRV_X`|%ZbwxyOa7)J%e|s$$kj*WC|eBU zbJyn*c)f;9qE(r`BP}TPkB5lpJXzG+b{&Q*mJ9wR1s`8hG5m-2HoroO&9yVeu%4f* zD@T@?dXlBR30|*px%|8xVJA;Rx!nb&{W27nUbKD=lDL8-#C*N63+K}gMJ@~waoie# zH#qjPk3w=%b=%MgTJ;E1H3oB@doyb02+wh(;`nb0smRvuaq5i2$+GG;UO!%03TSdhKYY4TEob2&0u?G9xv1|;pV0>Q~>y+nqe2Eaq}3E*)u&kXm-| zfF@!z%+(?OidvA4V55PKag&VBMmRINd^l#HeaUKVAOLfMxi-uziPtrU$p%#ChCce& zHN>Kwfk}mZcICPQWrM)P{5WQfsW1v_DxHU(O3ry{*E9b)((gKu5o`2a#>aCRc4l&3 z(0&+oNBXRn(0)@O_bQxf2pl{U=Bf*x=%Wv~{Lx3Z3Zqvu8Tic%=26!&sHK{C(L(!S z)}?7Q9qr-(WdWvC+5(M2t`F&2PbBykt;0XZcGmmkUV0(|(Pj-gpcui^$$@RX2!sK; zjtH<}v!NV=6w8Qa>_F9FQ_S#L6(UFEGMBl~fa`rm{=_>B6@UJ_<2VAgL;cXoSUu>a z-AH3^x6Mb$(Cd4kZ|FCgfaJM~><5-UK=4GzIT-F_c_IVg3|>0YK`zMVahT>VM>d_@ zU@C=sD`-vW#7nVbZ?zen6FPjsrmxTy1L`IZ&+~SE&5a9 z`Em3oZ4$xvf*&})Jru|BDJYPRL!~i?4~wGz z?;e>kg%idF)pL);$P_&P5e#3cW+FMv>2}__ZwkRnfB4HJBZak$*cKl_^WW{@`EcqS# z%uzBV`7k8e6TLUG_g7pNts*J`t3Vh9AsxXMvHiSH`Qu#*ib++pJLjCPr z4E4qLlTaf=o}M~E=~p(0c8 zXAb-gvPfr~{dfSr{UIFkFIaq;8=EVuze)J%3T^oh!QOBuBQs(gq?6JdM2=d+=h6B87+Ylvl{`iVzY0+h&a2w6!UxvEFjhY-wh348KfHnSYQ;eXLkr^?1v=~ zzt}Pf9s!xL#U3K^kS#=H1%w(rdi_0Uu

p_t8%L(s^oT?5T&Gvnk179$JH-(V!oEhb8X}WDgDH?jt`Tt=ZB z>UXKd(1x98k-614l4GZciv(fX;2=C3UV%f+tzyhJ)>yykBhR21p~DdYi&5NG0f)bs ze`(PjP!8>cQj6)8HY!87puQR`M4#_K8+{lztjzN3jcERtWZX4hIfmwce+z!QZ$tTh z7s_cOA`HzrgGg&h*~Cqu#8koZ)%9ywaaNM3eC_)XQqdDmd;OspXQTU{p2jPfjN3Xs zSV;53SIGl)^tX!NW=QZh84V88%~3jkl>3r0N>`5BWln|rAy(7X(3bXxAy!`Gg8e@H z&(cWD6&(;wXxZg(eh@`Y%p&}v(CIH0lH2XHBDnFR=dw52-Q=#2w-hETG-M%C!F!tj z8haHi52T+NR5Uw*Qv4S2eYA_{I(ut;rb43>-mkS_w>F(?^oC`nqMr2=)l0V^mi(uBBdB9@_wr_SNDmOe{X zy&GiJI}?|L4kMG43D12NSK!8H1ve{GM60Mg8MqJuy73J+}f!GIR+B zIP-vNLJqw-Aj8?^Z`NsOhPID45elYO7K_Yub-m;uLa5UZ^MaBgX*J`Xw<+720pqVQ8E`E7b40n zn;|(6X@t4Lfp>cJUQRgGG_*l$Z~)pn-b8qycDzB7ueN_jmY4d>kx~-oZ&c2a##zX{ zph?(m(Vr=UEtQrbLLB;T0ddSJ%6|q5)|P`C0e0VApNrJL7NT%JkeAa%|7D7O4!9MU z)ATCRiI5Lkv;rQh$=aw}2dhu8;;M)5TGYo07-CxB`xlv>&y!Kra39LkQ7BKyjC0$& z6E9C#=w>V$0j7g@2qYPd1viEaS|Sr;`y@;=5ins{q>qzr$D=5i$VtS*wJ@oW0lj%M zta4zbnEE$|Gcqzb&|HhB=G2hUiQ4lDuL5r4vZrTuI()VB+k9 zDR2xXbaPtxHEJaQn$u){7=`9J<_Gq#Ajr=YXnF$HWDo01eiis08mG_DWJ04h1&=D| zC*+QS9wG1KWPc)E@-*@I^A3o^lUQgbYT1>ipQ@)i8| zk%{x%1Qf$hQ1mb;nA5B=$nVWtUSqrzqXC(nBkvJ35J@qgh9{xzMJkLz{bytqA{=ei zfQCx1`i}TgG0U(!Y#+93bOokXnyU>~kRa)E?XXhoLeLc>f01_^M98nILdlvxWSmB5 zq^3ra_&ALZBj~;jc^+qP2MbXha@1^XfVf&5psYk)GADw3uyBnvl7o-Ib6eo|6^U8b zcPISTtbjNCOy$3a#y04^?CaIANN9*ixfz^6jv`k>u*-&_50EKw_8M5Y!9dpgEPw&V zewrZ_g(unMn2m_SbpW#^y=W~20agr>iGj@#*sPC57bizw#<;0rpMH?RD~C4)>|<@q zc+@~5h6#Nl0imuT6gjfnyagjv5;aX9+79JaGL+vrP=1l&v2r)O?uA#fqM4Q=s!cGE zaTHsLaH&=Bvw~Fgmk+1_C7VpOURuEx<{C2N^Fu^oB89UcA|;AIn-B?!oqdzZfo%(U zki`7(Cl^LG0q|{&(Z%rP7oq1Y^x2C_z3dB%)g6tf)W~ztx`rHq)+H2vm#u`kd+H4Q zy57GHzZK|9Om6%q%w~VH1>}NQOso&t)LBo{hN}$ebPslr-OSxKcxUXg2QU++-B*#v z?l=o0f<5Fe4kK5ukt>@DZ}h@vEZj@eCy;Y`vnZ5ESt!rs5Y@t?_#HWuieJFuwbJYY z^zvX-3%R;Pm9wuc`Sl>je?vX3A-A~eV7mhlIsPEH3GM-;#XgbZHNYjO5Da7|kq)pL z(e{Rjq7;S}a2~vZrJ)g;T^@s+eHcYV#T#G>E<7f1QJs%0Zdw4P-xLb(VtB;vhyF|! z)af=bU4+Tm7A7xmc!=_YB1d|x6RB2FR4e_D%%@#(P@2iSc&h*fes5W*Rz|l3!`u`C z^<+%z$|$P727x zzS!b_nRQwMjXT`)VasEU!Gl8tnns+gS+FIuJ!J=5GI(w5kv}!eFsylSvWAV36H51S z8FJk20%e~o+)lwhgq+1%kO8v`jX)nJ4JEP;ju{9@suC^Cz8jy|i4GCdjM5%5S*?c4 zI}B|`F_OGUIZKv>52T%gN$|NF5S(;5{E9)C{Sy2gN_2l!6fWwvE5u+B^eH0jQKb!N z-%D_s#+d;fsfpC@IjBBo?p)Lu);V&&PqTPXD6(>W5=Qm?@hHU}P6oRz_D^6H$X%N{ z>2|lU?$ce7Py{ilvU_(@A{=?Q!n6T#okkQhwxJf>w!@QD5)_-AQ0COagB={d;kb-8 zsE4T--uJ`i5q=*-KWu>~u~t~mhc!fB+L9cWRzfp&5Fu~M-@Z07L&KRP_^O*T{SIUtoXnHC+^EeRPKrqTDcq7NL9IAtvfQV#4H1FCYR2nT{U7aA+L$ z!Tz%Y#-HF05tTij6eeLk9VMby{6RDwGdJQ_B)U&GBi@q`hG{9xi1Y(2v~(p_McQ;6 zVacS=`iP7#B`dZnqNArs?aU^%Q}GwkGz%u+9$laZo|WL4?+RLA(_wmrYUSVIIZIiojJEfJ z3q%TfWFuT2mLjZJ&2?<|#m>Ofu98bCn9v({Bi=LU7c|oV@9n7HXVhgb4gNMEUR4}i z&Ep^??Z=`(Hwhw!#ngt!a)vX&fV>U({L>0e`X&ij<4v?!Uz^2 z0wquMKTmXwG^(6TT9^fCTO8}qQRo=V2D$rqu+~L-#?nJjb{vMH(gqhQ?g7@1s5-PQ z^GXG$Xf!{14Y`MD)P=$X*@6G&9|;XAo*qIUVZS+T_=n=8@kpX$-2N4hEEn^T_V8kmf?5e= zDE5)`tzz60(R!pQc}O#lmf@b4{z2CHolnV@N8}megGkMcnq7j6wK-e{KaOMJ>Ed?Q zq?(wiZMzZeb4(r#Cw3abOwLj=%k#yp)WQxhF#^Yvv>BOycs9bz#2j34(uWMKBU15A?z!J`S6m6`PQYTbrSrLd&zB!+zIR^r`5KVLq{^Vg?1hq2gR%^Sj;oo`8OQ*gaua6r?2l--=gUxh+T0l_U{#{e zyhk$>>V85q%)%85e%kKx59NJbqpPD-Mhn)#8>A2VY7>$q3l|4@Ms%(X(WOh7;%TlQ z>4l>+Va|OXkLV92mmm^XD~JQMCgwlJa8)ZRop05(Z1jU;B_{*)klJaMco5v&_G1 z>^2j6wT39H-&z$K*^&SSiZ4qKI6g+ATi{l`EKx zz~qz9QIGHbIL%fvVK{mVVWP~#Sv2a0z$b&rdEl_?OISB7N9oNy zrs-qgiSdxbixDNHZKaITtwv!;ugg`K|O@Y|YaK*z;3ckWIAa1+u z#Bu*s*~gOg0e&LO0@q3{+v z){N<523b+ABbn9T(F3p;vxY^mqiVrAwPVHsWP2rVC)Cpmae~Be9HeTDo7_zi2)=Zs z8dA}H7>QJ*A0g8Emr&?2ME4QV9r=n}o0sDUlFWG}a$&ayKR|&@$PahYV1j~3t`*5) znrENo6WLHu2;*iH7T>@RSSScXHuQl3X9Vm;^{WWl`I#6}+JTPA7OR_O5z}pW=97a) z0Un89C5Ll!_I}+{0exyZsoq>T5pm2JzbjxLQSF7{pDOBFT@B9YO7yRp@VrkO(dX$x zX>)|~+X>1|S13hT&e&2zJCQSwu7s5^x;6WjnQSNmY&+EQ>5Hglrn~hVYDbUn3TEkT zNEh2b`WZix(jn^+Er?L0mXyVjs6X!duUs+w=cY-_F^^~N zRV$zZmC5ku+K2Gi1hO9t%*SCcdEmeB9rHeBuR16(Mb}(dh!)(CF`5RymtQ|H-f=MIrdlo)czd)grej{q7 zw^4BZlR;!ggQIXJ6^jEGZ3=(-UD--6;;eiB?CClQxmY$lkcOC(XA@GkR(7|X27unE+$Lrm}H zL0h^o2(5Hv*kut`Pmer*Wat`MfKHJ4+YU=KJ@p1k!CNTR9}(bzS0vs^OQsxM41pse z2bq~fIkhX18ObVmQmR7($ft3b;VL%;kuaG@$mxq~@>){$q2w0(KqiI>?UN0UyOL>h z$adwVMSant><7HoS%@r**+j>M9C}jk>4lPYf zD6H)*CErYgv`3KtPf-|7y7J^q@ zcr6LWdENiLB7GKy&S!=tkKPywWy5OJe(7u#JF1t7YMh6xwEq&cN5NCMg%I_29z@cN z4C#{@3wuKL{`?Fq^dxW?UeY>)%ZxosOgr$OJL7|QiHh_wOhx~_4L<>UXM`j2;$wvS za1t|lAsOvY$Uu+pMuz8f=Wuy3AGYVCwLW!`6}O?Q4XIxxj(@xEqT2U!J%qJ$$6w^o z&}gQE5{tiZ$D|{k0;Yv5Uy5OP=>fT6+4&G5X00Tbb?d?qeH;o(&Rl4}Rj9{jWKW%m zsf7Fxnifey)&fyMZ`MX&o^gnVquLfY&VyZ3R|AT13mKrDu0ip>je1%)TV6O-;H= zAszj>g;;+)wQ)s9*yA06ST_e&LQM{-MPZ7_a@`iRAB8y_4DHI`gV1gwC(hpFdUPX1 z7)BxFG_-QWYkCO&1PPr+w0lDEa++=xH>G2t9+YByo>e zkVD)lOK1;R3`2XKXpfnXq3%=6KO;9u76`VT*#0*Ef!TK490Dc%f!O|oEx)@3*K+*o zb<~^>)-Q@V!(M!#AWUfgIej4yL0e`*>A?c}cC_Epb=#YW-;s+F~Ur=UXj;1;P z!MBd%J{*_$-Nr>I)AgH3gC^&2ip{91leq;#09z#47l^UN9gIK+AS%&H7+{Qi>Nw`f z*$a6*AonWqdn3^9Z;4Ppn5G~v)}-7?o%lNd^r~)znU4WXt70?7JhSYF@NT#k(*yhD z$r^9Vm<648Q1zKIPq`SA6SgBBuYIaehKv{5EzJQFMh7M;ct(c`c+$=mb4z{Dl_<< z%iGueF3bxDGE zsRe;N<1G1J&IB`|*VMu`p6_M~6P(*wxP3O)!`2BY?_i0jrdZW5l9b9SIc`=j&ues1eTc5^6ieiA7M^Bx5qa9zDPEScJZYM1vX&Ruff|YS6i*gk zaI$t8*YO(3haA2!d@<&_x4F(HO`qZjnBYq?*AwRMPLdMm42j{-H#czRzMdp)#2G%p zUt^A}-${~j<6Iph&}Kg8HuvMC8S6QtCj>gpO@w(~B*_+Yu8R@8Z9dhp493~|tjkNSmVVM-m zct%)B*)}cHOhhYXNjoE@q_Qe4w^L+(ip_UMQAu@YT7j9UeTtn4@62JX{VrV@BNk)fbenfU^2{lE?}S*A1!~1ISwozwK1Mv>!qs`2 zL$an3*Zv9d8Vk4lY0Hzf+_+j}B-$)IZcp<~p0%Fq@PtICg_kg2NV0Y@*RdGM+ZH~~ ze5;dnj&q%wkQ}n`%ja8{tb3R1Y|Qit3;)}ETasr_ab1|eldC{s{?ueWaqjLIDPhZC zXa4MDeIxEG6Hy+_HT^%pbrZlHFlCaTClm^PVQkv*-NQO!Xg$Hj`;hPO4^sun9>%OYZ$?P zCDTjjyGj;TIgF@_vg0MRtfs}11B_(9vfribBoTcJ>pn)tMPPv4zPb9fv~m-%Rmw}7GL0{)52jVdi^V8A^=8g9pZPYex?L<@dD-jCg%@Z3 zOxyciEJ@j!$uig1V7O{E#4|>m6|<~rH6*#}SBdA1xR_J>nXea+jAZUuc1L@#Uj|k~iwSP58n;Vdgl#Q*t2K$B{46 zMd!ff6YXS_hGws+)j7#^>b2yv5#OZj7$w~vuG5OsUv~Lb@vSe=y?gmg;Pek&D>||_ zQL{(VJIba{)%icnMt_`4KmTz0@05Uv>}|ud|8jLQQXC{QZM@3~YXCD9%1mxFF=kkRu{O13^EK4%wYV3E-)C9PT?T9#9k zY~Y%4%}aV#>dNE%rJoD}t^4w&4O7GJ@>j+huI0YjD?L9Xd?II0zhPp=tqJMHoU7>E z8YQC)?tTp!>+V&u0u2R5#XSRoGE2K7Oa+=mj2n9H*2uVbNBRjIC@?^YM0q-@6e`n3;p&nC~E%y{;2Mq+*R z_uLDV3Cm_kST;2^MqIEv)0lHB$qmK*y72VEAdx$>b? z?zH8m_kz#*7bNWX_+743C7#(eY;KyhC7>)CSmE6zwuT`w#&X>MlAh$yLqrse0023t5W)9jSALetywB}Xj$ z8U9QqgV2oI`7*(lYkCDPD$TRn@m^Sdz%sE{h*CDU+Q~1X>}r+MD`Ku}XO*cfqCQ|% z)hm{)>|&MWD5B-M@@{CT}#gFoc=>OP&H>~0S5IoTWNija5}d^#Q3Q7 zL)#g#D$#V_C6RdsHm_~vj;L&+cMTPo*4cbz6yB(88{Ne!G*hTTCQ5$Q0P_YkYZ<$ss2c{s-Y;pa2bcaMW%+dT0=u|S>f^& zd)G`YZ?yvrCGCZt91g*m+U07;8+Jb|^i6Sy%hY|O*3nSiJ82beVyokpw%N_Osy-N68B?@&Wa)WctT`j0Rry76 z!A>`M=iAME8(Q5~v~k4g8Lw&P%*oI_w~G>jmyPo-x;pb`=-!E3ZE=D%icuED#cR$Dw{z*+Vh)*_K5b&ld$t}fcwmR!`3UQ)kCJa53o^qLK&skEda zr?|kv)jrE^o~DM{zNX@mE?1u{hs~OLT#c8+%MF&VvR`^cb1v8ZQSs`d%j4~rebQXa z)kI6w8o2GWchT3f;c8YbX{dACJ#Be}meb2-Q;8-~_l9ejq}(sJc$KuKxF5~(c%-hD0DTbD2F zs&@I3V{0UD4tSeh52ti$mmJTL9B}n@m7xh>ABn? zB_$RT+Lu$SZ`{-KL`t?OWPM-Wu>QiHE5D@_#X{5i@|6sh^z_O~s}_Y8_Z0>iEbX~! zDy<>5vZ1fI*1)~znxC|G(aPg}yN3-{^juGr))Nc6)K{itxU#3OO4_I>>~3F0kYQBM zjSguOvGCEps#?PhJvX07n-+yn_3arp+}d;Nx3rnqD*6Vd;Pjr`vNBdhtHf{A1sUb` z^qb1qibbg2Xs9(R=^5~oaV&~3y3sgnRNZqYQN~#;(&|Q&lJUNtyHzr7MUif~EiT5b zVfWf(JnlsL<+g4%Zd1R1slrQTb=ZvqwZ_LA@86a2UAKCzz@e+gr>!5nm+`-|Ix+X~ z593bths=y1l{FbR+LY#8ZhRB;^%kHa3vRIYsy)p6q#HFwzhNls;~<=Shxr}gKKaX)RCvEkU-hfB}K&i%;!?9zuXdjCl~+$>tBTgGzjw-G4OSZ~wdXa)&=}=yZB0vLIx~`{L@OAscTB z4%#i)9vS|jt@`APjn4$1Brk}aIetm*wBe@LPEU_4*u?X3RPNldP2ZiKe_F7eXM&c$ zXc*5d8`d}7!ShK~{_?SSsbw!>O?UB3TFGBE+^n(eNDDl+29 zs`+#Ux-^GN%`~@U?R>5?W>OlQ5j6srlIQ31Oqwa+a8pslD8<&9*T8%x!?jOQ;&_Ue zGhd{+PA}KFn(0$1fzJGe=7u}D?$78AbO)B4-8ub> za==vfHW9sP)?KTlMCyY>bMizC?8zdnS-rFTQKE-?Yx=ORxM> zmBiv*Q~irZZ2t5r{832~&!-2luyb8ilv7PB&KDoB2(}ftsN)t;xgCearVo2 zwU($g8xR2tmWODAH%? zfo9{=jSuL`;G#7ZOOH3tdDQqgtun4??cmam=D9x_pVL(-MKLl?mzqta_rFN1DlCfg zaq4THH*fzKU0qkSzQXBl^L+39@6xJ|7Hu4KdeXdL^Zt+Yp01*JnPsERrsexTr|lUi zO7L0sE_30Dnct@N3@uO0TK36q(T)8-toFVw+InW$cg@9b_W!xG_h-@epUeL7nsI4R z#x;yvvVk+tG;=i#t`jvYKW(|I7_~@2x{UlsWmFj7<8VIWw}8^ z=wpoc8ZI>mi5&ph&qwQxo2EVv~1oq z$z85bDBktc#Z1eVw;6IzR&l|6S6eMR&1SXB4Hd;j39ims_IAyhOZK%Cmz;6+(sGDs z*15dzOmXQ?*FY`D%x3*1jn|6H=PzHSwY0g}_;TZ5ab?2t7%iu(&GVM*e_LFAW_i5U zvNz2OFYo_Zy!Yqw?bDpOw9Jj07>Qafw~V*WN-b9Vnj|IaBi-`ex|p}vsx_-i>}zq` z{nj<8#c^M=(Y?kIx9V%lw`w^{v{>D1Qg+{W&8=F?t)#{6UQ4k1!E5g4w7eu*L+>4^ zb3bv-n5(l zBPAUbo>MLUk6IyyO_rP=^!(it@S}A*R~yIfP8lz{HBkCM>V`I9k1kg)zU&}UjUEg}0`|%7faw z?;Z8Ke?7%p-y!UVc7^+~HTQ3HdCzkQ|De6+-m%2{w>W&v9aahG)VUwex!-T$V>c^8 z^WeVA$18T>>J^_4d(shm)pX~K`WZ)al z7q#!;@qH)n-yaP2jk+FnPN!Ys)ad;ub-o*}uYIc1QF3Z>_p?FY#MbB^2QP57bCeFr z_@%YRNFVA>Zx<~c@$t)Pjh%Ps3fF1n($NaP;?_9tL)X(!8)D%)XODg9sPUXC^uLpx;6D4^hI4J6UkCgjXK&ds z`{S2$ot}^+Ut}ki%>KOL{B6<^x0|x1N~&?aTQg&)2!oBeCUg$d8U0|CFX zw||-a@5=?27fURVGbc%+jU%j6@FBGzkkL-oY=gil<;A%=Q1E^7LOq82qH-Bs87O`r z#k-9!1F4(agt)uRycn?{m7KH^J+X~l&R)}&2hGY!Khh@os>@%BKRd`^ zO-6s4Oj!3?DZ%a_6YU*u+90GRlnMO`GSlA4dqm0lQdXJBf?!+iOwA)|J(nuV#I^)G zYiHRV(X_tYQYLXW*h@P*LSN_P<#w;>pM(A1XD1)gQ}5}KlAaZ^@<2|xzTw86n^H2C zA*-}=n~y-S9rT*9E+m>i@3Q{9IagkL$sG&X#J}sM{=&v9U!@e@g=~Aji#lqi-pehm zG%GaYeZJCBtHxeQX_b)B+`fW^2DXw{RUfI=hnDmet}t-SxN1Iq^++g{}YfjQy4lA1-OPY^DGWIE-wPj_S_U@}ky{)fDmg}5d*{)ss<|xGE zg!0+HR(5Ka@g57Z?#n9IUl7)-U9NcyGIK?_;g+!5+7))kBCKz;lpCK7d#GI*aSQ@< zyZ7ABVbA%i(vQWc-|Uf|H!J+r`>NVw>l<(0e>6WNeEWyK>T`zilD9@5nbwDY>D%+v zFmd;-sYi=mg#YQ=`^PXz@-{tarm~9bMva_N+V0!pgBBsH1a8zWG}pQDvH%*R$NLM6AM#HeWFfU}SN@`zcv`%V~@ejNz(aV&~3 z%x%19RJrj^jE~cuhy}U(zZmWPdMC-pSuD~rw@Jdde&gMIAJ^rP4!O<7#{0kC-6P|c z9qAg?;&P%T?A{?6kL!^>QLV8j4!yc}UdD@Wb#T;yf)hu>?%$O0aabK0b@0fEldtYS zlkv-59T#<|-}sEjgZG0gx>s*rdiaC!`TGxk4+e0qNnY9}FsIAoA^+nb%QcxxkLb+l zx&KhMGGxn|e4V3qC$CyR(yCl}c1@Yiv51p5dLGTM4F9#JM(23u$$slW`^ty~QB680 znor*A8T6@Ky(Owm=j7FskE|a@Rz{tTYS%gS=H%0!#|f3uzoI&I+Idew)X%DnU9h%S z=d|XjmpxA^D%WjUdt2v>-KjU$Pg^QCoL&1+;Oxp%?;D?y$9z+%zQYm$E#DFYls$JNMX-ylh%_jOf}fm-cDl!@9B=*JEVXcE`43 z}tk8oHO8+a<1y_*7+YiPh8Xx!Nvm^CGe;_iU_*?v*#~ za#vm?RPFi|YbMysHBV{o=uW=^!#Eqkt7`MqPK}oN6|IX~CU|YhJk7Z;n`BE`<2=T% z1)bL3|MH}4>AN`p@$1Q_^=7{6ku9IKE^NH7_O#LdSNCNrSFc+;e&gb4lbK_$WUE`( zZ5+QjeA;yX*jL%T@78S_zeSxfoB5hsu6EY?jPct_XRP+WmXxbsy*_We-~5d2%s1+C z`&!rU9v=uggAK|Yx&80f?-{?7e8zd^TWh)ISsNP1@7A7iD}C$sv?XN2fkXFBpYh~* z7hHXya>KE_`}gMiZh9B9;?SK9XY(F>nIG`&UDAp+v5nn%4<#0aYF3(#pU!R@`}kL`lP{hldPt-rC453q zzRMwgTJ*4fN5=+oP5R6CWp6f!e%0Sm5&n6t{LSl|=SPoy>ew^(d7J!gz66Wt*COZY z!l&}&2OJXYqu=PCYaE-}BY!tL!8Q7=%ej{DFNfssUr+FfeiwW0(Abyr@(=m81V_Iw zICnJs>rMGVhb@uOAC8_0$4}?5t@=fjqXnae)uEb4jiUH18l)WrPRlXf^q`l0%Kv_7%q*5ryseWgE)p1*pL zSbgjB)w&{xPM?3k^DB7o`^v4y4}X1h{_&1q zaeK!fZ#}!~+m}Vpzx_%Im=N369sOP8!bte_Ir;4b_WB|KV2@o z4gYgU;p_EngVDcYFN}}xH^)g^f)6sv9ViF!}B{yQIF2*6tU%SsNpqD*3jc>>6GZ@@U1~j>FIG_ z>y)3e=z?4`?~V~^WO%3FX50#+l_x4h<~}s z5@oS|XHipImZ+@qr155@uP3Dp#%2h! zRPa>=uc24NR~^g*Z(!EIR}*M~SwI`;0v#|L=mC9T0Nj8(@BkH{61+tmRruC`THsUT zN%;alumbpl01ybS9`vNHf$N|T+yFPh0I+WJq_Sm&SvkNCdVAmi9KllH1eO73-~w`q z-y@z>F(?6HM?I-Z}3e8>0l4c8Tjr1J3%JM0@+|M z+;Z^E1$m$b=3V&K;adyxp|8ic0W^U|upjILg>WkZ#h?W22Bn}3l!FRT393Lf*aP;0 zT2KQTz&_9j_Jd~71X@5VH~@sI?#(#DG{}-t0x)Z-rh{m{oBD^$V&%HP{1Y-}IvNfIct)O^>{&utC%z zmp64x*qiFq@up50cvI~lWwAGv3erHAxi{4fE`c6!1@wY`FaYiV%F>%Evht>iL9@L# z<+9wH+V1X6x#|nEI4is<8Z3w24e&tE0AAn@^E7<<06!1_f2PkU1jJH4q#U=Tb8Pr&pp zv}rf`P7lfg0>KF|*pL2s-<#3|1GEq2F6~2wfp8E3)`JZo9=wt9q27T{U=n-=hxC1@ z)8H&P4>AmVDE&D;l+t4OiS$8V@S$oC;tP&<_)ze5e{pUux}iUn)q(ml{;{ zrKI$HDN8G1mLk4NKpCh24WJ3Mz$~B*bbv0H4LpD+@B&#N8{~jokOy{wd{77qKoOwz zeJL)m#>AH@UhYeMkMO0ouk)qmCi+q~z!nrF`%)KEeW_cZ>!>ev3G{#~U>tk|6QKXD zFLl!%b1}XT!60}Ho`9#|8F&tcK;=DOss%KI17MQ_ek=acmkIHiAtc9{2?K zQNEzi1wXa~{?G@3U{DNm2?&LLB?trIU=`Q_GC>x|2036CCEE-a2T|KBj6}F22OyJ z;LWiWl-8LQl!k^sr3q#MZQy6tp?? zy`U&knDud&KQ#f|3;ih%pkLxo-Kg}Zf~Wi`muUP;FyIE319xC98bDb9OJD`8feo+) z_P_x+f~CL-ECbHK1-OFczzw*Al^_g+gIr)Qj_{y!P5@Qr8948;x?ncY z2SRZHlrTsGcAJn#a3dvv(oPScS4od;$^KOf%(Ko9zb_?m)6U@J1f+KaR1^>tEJuok-c;<0cx=Z8 zcx>lbD3+L7zW2@C`^~#=?p$Yn^Iute?Y-Atd+$I<%pxzUc9U0dERsqYJHDH| zfxqD`)K=pph7SNM7_L0$nd6f?Z}_B*WQ@B<%1ai4f%NMfQMNDxL>8 zUW5(|5b?Z78i+x5zZdyh;zh>6@vs+hg0*nroENzWi!XVRdjkBgoE|T7AK8E$Lbf1V z;SyYhYcK*qy93X;zj<3tyjH>g^)@SAV;owkqUPOuJ~VG~3`G+e#zMIOW5yEqnj0K+g%go$L3 zYaU`7kbuXDkO2o_*`F8_aE2}50lR?%T!;fc9E2>$gIc%(S7xJM14na9oKqXv4f9sHU;4bi=c#%dp4N6bFh%%_a zVpsyl;TW6*g}=PW9B5C&e;;`7MFu}&f8$<6`llD^knkp*&^E`L#434{IM@$-I1JO5 zcoT}DH=(cdCJYGs#lxG3L34*U(apl#F_GS6Hps(Lpn)o=LC`*LvIl};JRaqAl%XQq zn^eN39PAsq;2Vq`_a-?l-o&ugo2XpyCJTWEt{1(@={EGW-J95TV*NZS#UAMZj<6Qi zfirA?jo=1b;AfXNS<#I;a0B(b-b4eQJ@F>FBiL$HOJ)DG7&;W;ic$0FtC*ecx z!{s?X}c3X8Dc7dk#fUC)PTfF@{xHsnDu9Jcl$ zwRS$l&e4b1g9H5Qg#E$-XCHEHix0U4_aN2Jha3?35Wj;y#2?Jds1%a|AF>RVLje>) zArymAkq=n``-^=D_mmIOYxW`f@U4PM`3^te-#b2J#$6vG4KnZ){(@%^eGEU>zF~Zk z7l_S z5<-=I$=PZu<*}+Sc@A&k2gs@U5_xEZ({KhZ!cVlN)Zk|y%5BgN-7pPx1bR@Oj_gHB zASK~C$~WK>e19I{6YU&;~jfK}dBT%isEv6_6+CNAh9AX)2`vSqMc?3?)zs&)_-y1uq~#%8&e5 z>_;d|{0M<*Fdc%_AMFxHueR0L&M1dCw_ECm|u zMH^M58mNN?Xn`hZLkN~LkUF3T1`vw6Wyo-(IWhtn2~psT#~WY=>;^CJ2R1x|*Pyi2 zkC;OP^uWI~9IvV$NrwU`hiC8xT-5x?7T5-!;04|g3aaYpH>iUKYy=P34#B|F@FUAK z{YVFNLcEqAsWb2+r=S7$pQTb7k?}}AoJ08!NI1+Kz17=&AJ8}7hexCi%P2p+&NJb|b17d(d-@Dg6ZD2&11@DAR?2lxp8z-Q1i z^dtJP5}1$<8E_CXAqR?}1R8;6dcK@EN|q zR~Uy0_yN=0<6IU{9yhXT=%dNOyL+DhddkHr|{0+kE~jU^9^+_*e(ohLVq`7 z+_z%f;cs{gf&LgXHpUEogP&p8ZloVs4X$uB7Uw({>lfnsg_%vbmX71vfy>YiSK%$Z zhYuiI?MGC=0&Sny!%esYkKqY~-u5G5&~+F0+I>H==9M3@1AAcq<44wg@*`$bxKClVq(Au`)_{$S zKk+p2CoL}i#D23sF}jTJLIDSq9l;6Kg8deM5+CGG_>cgJ5GnR2QQ%*IcIW)@zK=iI z0^1;K$e(0G4&*`}6hI*qK{1p-DU?G6RKi&}2hGs=%Aa&W;TP=xt3SCg?oXb5^Cu_d z14!)r0P;yCfRq>oke#dmVu268|M~#@m0JNs@Fai~jR%lYCns*| zv71F+!me9*9SyHg-i`DEPwk1-`;KOu;w!2|wUFP&`@qa&Q)z29h8FGe8<-Ko({L6%=4TEC3}? z1{Kf-JurX}?Ar)w0>8oUzyvF>0c!|D8(X9u*n_Xp&-pBJrGZ8M>}HW;y)1I)DvLb&#v)H)nHHO9>9dK;@gemv|Yf&fRz$WXE3dngd zACy2DRA3=2g7s*#7`X&!pbi?K30j~HbYOrE=z<=IoY+JRL+}9pgeUNJ9hh&pW()4HreIQCS%~dlTEh6EqIMSJwOh_L--RO!3gRf zBcH%1%1@Ec;4gR%FW@D-g4Zwtqc8?<;T^Q?Vv}}=+|4G@5Czx0FuvYw;_U-rY;p_) zF*sgGheaGVSqwW5un8{<#|$<(Y~qlQa|se9Y$6GbrMPyW2YR6oUcqY^fic)y#wK}? z57ctBJHaNm;SQ{;W0UYF_)6;`HlBOgcIKQPbTLso%56d&G0rdRDDb&Y#S!oxj8 z=_i#U^K}nVMykLUq(0IBHlpkXn_vsLg9rEm8zNyp#Dfvq z8^doP#A6d=Ix+)tpajaH0**oz9EWOXfJQhC=iodvK{K?#Fl3EmJCF5F0QZUic3nsdt2Zmq_zkwxK zfi2jBrZw8aC#PWY+8KSDra-yt7EJEJeJFDeCWoLLGO%pxj$pjU7mW9Lg30coVB!Tq zAcS47gNZLJH{VNEz)BF!RG@fT>?LVPG5DgKf#e|%B9oAr$Se?`oQ)JC(;)*6LMCKG zF63Z&KC%Fcpcu-b5{|)fI0cPx8qPx#v_L-Ay^QRG+i(va!DILfp2G`x1^>W|ReOmv zEC5~51AW*GTc8Nr^F|gUBalL zf(*#QEHD5e9EY=T4*mc;%e}-N9KaErU@Z(f>?POX2Hb>OuxjmI@;lgpBb;|eKVj#F zy(AKm)~A91+sNKa_7(0Wp(T4s7(AM- zKnX`iLIgxXG{it0?1y+r06ru_5+p+kq(VD%zylbDhwvwyX~l8FIXDl^u=LSh()t{I zeY==%`a6`yWs7?qWVqpk5 zTMn%Nid|VDNv+Vhmsi}4bf{tNetYw3MIGU4s5dxB}}JKVhJIeu#KIe z#0f+i3Y6iXQ1TG&g@=;+U?D~phmx%JP?8NfkPG>63l4XLk|WSF7D}$e4Y&_IZ$rsK z`7q)Kf9WbvDi?*3!*B%bmV^;|aDZ~HFme>C;26Aw7208BCA_1D5qU-!Ugw7qsg+^G z0#;dvk&8}YMuS3M2J#VZ<~g4DX#{JxCIS5$WtO zQdf#)a2o!Ay2D}Qc@@?<8%E4w(c>_(7=rA=NvUr*DT7MzVTBU`JDd~*hm#$#;pEQY zaME`qoJ{WxCl7|hiTtB*a_&_)5srov=HKDO>{B?IITcQHBqB((Tm+#p6)2m`Bgkgh z0#u6#q5yNCAIrEKBFKIaLk3t{Dp0bKIgkrhcx(-MDBB?Okp)l)MNkYSPzttKUWPmb z-q zegQAx6^y_;cn|-;XZQkNVH_r45~koAe1{+K6Qo=sh$74fHBbi)&;)JZ01x&79}-}` zYXngS6<7!~P=&Yf4!*Xde!`P350_C5V7+d%RpWzEkz$8q;w_k~Nny(^=7H9)`9YLnSbXYVJ zLAu`HdVd!|lAUnhzQ=tF*Wm`-gyJtahOZGsW+H;fgM>sRVJ?a!Qks!u2JBcCNsf3# zl6q$a%5h{h)IuGcfO>cae}Tttw1=Ir9d^NP@Pq(hLlEroj3gH#B1y^qNKy`m;Z1xb zX%t72mD!QR6t0v;5{2Vfw=R-A>53$sJ&~jfx?%S7NFon{Z;|BUv?%goP83cAD{jo=2GU^8riZSV@b=uyNQ zE;6FXC8*PjBK2_781=tJ5et(jvI@R1qe!2B6mblXBD=X!#1ppfkHY6SvETkEa(R~m z#Ra(q+ECt#Y)85yw}A)jfSs@#IZS3C4kP zC5jN34zocX6kra_1sbS=8mNONXn{5iW4i{(he$)D5tzU-_!D)@kt<*&n1UIYg9WUD z)$lv40VY_26ft1uf(B@W(=hW| z6qyBbP&ODv4#8mIM!0CHPI6fRjYT*Rb!Hy46WS*A-Wt2pdF}RBIHSm{? zCIP^LK|H<=Y?N;x1Cc>+6Xjs!9=L__UStS_LKuX@ZPZ5~qaYHZAqF_W11`kEK8S<; z5D#~-9v_(iiI4=zkOHZ200bZe5r`oT(jfz~ARBTZ7xEw<3ZM{*pahDc6w2Tb9D#ah zhfY`^6HSa^6#j;Jv!clymXSk%H!CePyzdb63Cw$O+G6| zlP_>iIhu4WjV80zqRB4U4HNWevVj>*URXzykG|356MTj*@D<+d!#J|=yVm|_;sUO) z5!~Q1Tuh867ho4#f#Ps58b3p$i6_d=NH3%>c!Ll4Ljbs7nJa7pH`oX))NersBDW&l zVHGgh4nQL0ts08X1MW zha7?z@Ek_q2arr$XAl8vvd|y!glIT}_1@)2lMnD7LJM(S!sa@h&mi>wF51IRc!S4( z!&_kCF&l#59Ui}j5AYH8pza^!zpxkOUveU zOoLhA2mTNMfe-|HAQ&Pc3Sxi*T-bnZ#Uc4n0;Nz1HSh-v!2@^*f5J@H7$OJrK?|(G z7VNYc;W{2Z=qj5|z z7$*p?K>r~Eq97V#fCF6MK`iWpIM@&IkO)a|{#UsYWk`k;NChE?Knxjh5HcYfa-ax~ zLp9X=%A+wvzZ&D)g7ZFrWl#+@PzxuZ4(j0~oPq{ugz0y1d>{!kKn70yjpaa##cR$5 z9C8Ql!abOP3>^+xxDWUBat?V5hgWj&|Lr*Bx*6KSUAPCknH=)BC5L>1&+rAl!Z=L8 zBuv3K_zpo<9I^+vz=K#g4@;~$L>KfxAASQ9aIxVKSJ()(a02R}9!z&}$cQ)k2UmSK z5M5Ssd~cDA^oxD2GFW`3jUv$2g=7I-nD} zpc@Vz=a5Xuf(`ZP8@R$oaD(Xy3Y2B%Ib;PahyErG8GtLmZN>3IEWB>x5I?l_>);T7 z2mqOW4v~eKARgq9p{E>j2_`}=Nyy;h=O>r!0hvrLvCF~O!MY<{vI#bWKLh|9_Q2^HF8Mu8fii@A z0K@PQ{)9*H7@ojWcm{vLb9ezS;T61w5g3Ircn@y1*dJ_yE$|3BPI8G{6PL^ed7#2P z&}`w7mR2rlh3%IxR`94Dee6Jgu3@a-aLM8?TylFBj~tuLBh^qohetZ+^T_U{JmLw9 z)Oh$kL4mSfi$|Pc1Gs=IYy>xO2OkEH_<|q!!*v55S-*uxj)w6_6>Q{S8Tdn0HueX{ zp&Bmd@bFn#9%+XT=!7olhJPSAmq+Fu;*o79c!USNu>TB?B!U2hAc8b_2JKxuLha>| zInbG}K$(l22VE#DBD;}2FdyXw(2KGXQU#P@AuNJE)c3<;l$XF#pn)o=fd;69CTM{+ z(18IupbL7S4+dZeMqmuTfe9=FCRl>yV+yFbIc8xc-1g z@+q;T0E(dm%HR-`Lj_dAVK@S>VAix)A_ucU9_GR-=~!|Uj=^!L0kI6$gZSC8WI8pL z%z=jrv7~!ZEa`z>=z{^c0$1T048l$5T8etO27_=5UcqY^fp_q`YApGn7fU|Dwhggl zJA86QpZ#KqAR?Az#Ksb*w%2}FcrhLrqR(o`5rE|kWSC1+#tH-K2oO1zKM zK|M6U2dRDJBm4uO;4^%IuW)I>J|dyHk4SQgL_-X4z?8m^ zn8PZt1Zyy1>?4AZeMAWAq5Fsy%;aHPPxq1cf1y9{5&nVC@CClY(HHy3Roysp4F=&R z+=AP%Zh0J857H~*$Q!FTV!b|&R5{0yWAJT99Qg^9opFS~G>`--m&P=xuQ z1$W`6PaHWF8b|a~;)nrUs*NLUa2eX+6}*NKkZFx0lWlRN=y4p`_#uwCLG(u~Bm2qk z%?gy&Mo3r#Ot1uXll?>uG(iKjKpW_w1G*q%y`M;~+fU?u_LB{J_mjR03Y0bR*aldF z71)3+*nvGbKoZ#SQ4iPQCftF0a2M`Fef55F5>7z_G{R{(1Ci(U2>i$0;E6|qHo6&$$Rm{c_^M7xrWzkGx_8n_!mCGXZQj~1-;PsD;MxdKV0_U6M0WQnR)Zqdun{Vr^Y8nd-%i{x}gWQ2J?wK3PpHCJ+Dja}L=z`~oeDVS+Q}{$86Medc*Rth&LMr%V8kkh^$ud|D zZO{%0$N3}?7FF{}4ZJcgJkOaw)4$H5i|1fliPhP?Z zjKXL53KK91UH8zJAwF64l21}T@d@QC`U+tad=d^RkP3Mc3FH_YhibS4Go%toEbN2- zWKN0|6uQK(MM@ICzLW+3{y)q+(MHVqGsY=2nuLI_z?|m9moX)Dnq)O!v2B_MeP5Ojd$^n`8@r!T2;!+AKkWF;i-QuWUMQ0GWw(&jiM zIx=N+5;U763Tbm$iR;^B%oDW7B`Rq11c|OpS*HYssbn=xu{v>co2*BIE=RI~Hoq@% z8*^r0f_{@^6K%mr;?B02yadB>$u^pjbdncymMFp4RH~1rOi%J_n^l}(!jZZ`Q*lUQ zGv#U%mN!Wa(H63jg4^U;5>}2&J*6!YB;gyh1`^CnXN=GmS0_ca%^ps$;LP|yThf=r zWy+5w;Ct1_X-h|v;@adV6V{B+ps3QMlM|R!*+ffIX=znedU8tJe`$6Jx~gV%a$egU zkHobc86#D#zT_h2+`z>3O){pc+9SzjZF6~v8^&drs&wg;O6ELKqN}N_gDQibQq?xE zIMIzG>!PaTkW#}`tWDh9BN*+-5qHrK(?@a;|Ou zaN>5(%urQ>zLaL>g0aM%O*1*Fh9fB#+ZIeF?jE1XS2dDOZD%UUCV8385~v!}Q@h)g zRFZr+vockGb4cxHD(fWqHO(qiHDRS*Yg0B)3K*YNp}I_vdXuT*l*Bfbt5#iJoqD%T z#Um+*BiEq1qA&FUb75doaFbk<>dKMSM{NsvNg@BE2_+7UG8fk-MK#SHQng?mc-ywP zB`Icn_EXhWf&(9!O9qm-rt%}ItE&%uZd)>(6w8tSp!$2?feGf)v81>r`S@|wH6sVU zw=JDaiXWG!s4=Al#F8ePoM1|oR*<8y2!Sju99+uGcQwZql3uNlA)7Q)ikeA&5b2m-_9^kIX*tGLT!^kcSD$~{ds>ONjzj>ry;V2ua`}c{9EKLGauQe%~st1gS%i2wNsn^Gqnd&U*^h(QR zqSTwFDh}#wdU{p+vf|X+92FP!K!@}i%jLDHcbin))q_~+_3g`BQtywec&hIaq&Hfw z7)X6!x{#$FT%CTdeZ_Fnd;#V8U2=KItN}hEh2N9`881MmM!(?J}3 z+rGNxz}N95Pu2Gc4t})!ec-@^>CzGPxaxzS+kYQEFvVFK|3Q6!-@ysXHDd?9H!U4k zj~_Ysy?xE(fuG|`DH?p~Ok%~96%aF;v_=9wQ?i4pBACvlQ8f}BGG(kRbp(>lG$oBB zR;FBsrMY0n1dXPVEXY)_vT_p0n5oh=QmQi*JFGkeGr6ip8mWDm%2w8a0=Z^YQ;h>7 znTtEDc>?(fRi=hOI!o2cMkG)$Q*+P|(z7%>Y>EYQxoR#NB8M!7m2Is+v02SsBaM}% z-(maTPXQrNHdAM5q*rGx@30#dEaa+(YGm|fnOWJ72^Kf2b2JW)WUcP7pA;;eQ0Hr8 zN@rVIImilC%`^lWS@dk%4hI#XI#(l8BikX{(aKRrsM)MhsFB0UUf29ISCnNn$;S4)!CanoIHfOT+IfJ{J!jMR%-)=`pue68U-WSJ3H3$goYEEZ5oBr zIbK%lL_%XTtv-z+dX8Vmx?-UTSL=pGu|p2qYJIJ6d9&7#MhPn?xMO{baOH&7Q;kwV zPMDSRfY8iLdqktGIwz{bd01${)&8Jys4s_WwP8%Sx>I8wt=whPyv+0jnvL}Q?hbdA zG#{>Brsipf{C=x#I%$5*dWD*2SozmFwwb2|Oz2f;o)zTZwDNFDW1Hz$Yo4plzuV#A zkru?&Z_xasFaLqn_Q15@X8k73^CS6>I=1uDLMHUvG@GOgo>}b>rG=Rp^l3KJ3to2Y zC{Byu8r;xqaVQwI+F6?x)od`N*~%(-+p)7HEoQ>tspbVi!AGlI18H0{!}t-+i`506 zJ9Z7H#c~ZlXkO|on6TPCmKN7+IIeklq~LqU?#ZEfHwbKhF zJ3UpzNn9hUR)<5OjJ21JIHlQ0Nvo4pDA(y_EO#d% zZx3-A*Vss_r>{`i+9yz)-fV2D)jLwSxYLIxJ~&~_)asKiQnmIKiL=apbI|Ii7io6- z7K?MZzqx1)I219g{c6Q|&A+*8U11gJclx!63nqT^)VeAtGPd>~5Eq%5u(YmK7cKAf z9~PHzO+vK>`-;r01IEN<%_bbJ>mx<0I|C-gcX@->y}sgY)k_07whwC;}-@9YfXrJtJkkLesrysY<#(i_cI^l3eym-uz= zDNaAbU2#Ke*r9}N9bB7!u6f0f)DzpyM$>?r2RnmUWDoyVUGtcOqFr{g~5R_(FhdX8To0-wIUsjjq zb%uLnT;ZA-X}{_#EwYXX%(&KUW~%*qq_nIvf|qf9!i=dsB3)K#9VyDVX=d)AJxVXD z>WnPTxXm?p(H?UstFex%&A8ia?ymiYRaV~_)sk_4!rW8)Z$Vk3b@V{S12YSj_S@>R zbDhz{84tM@q1x~I%9^cX#xfo?TX3}BkCa{PjG4@MGGW2j{vds*-I^nN@R`{vf%ZrG zq3%wO%E9N{RhioVI2`J?=IR`L*}SSy`(M_fYn@#4gRdu6RcL<_9J*=Eb2>O`wz^vT zbM>LSoji|&Z@8-)w7>KndSD$Jc<^oW>L%^4BZnS!#_|rnpIF_dJuY32?-Lar{Al)j zpY{a3{AK69;)DNkf4`wU=}B$u7Q1=5!vDN}uLfA!C!ElPTH4RH9F3|F@|bCYdz4gs?)vCebNV#@v!l zm#nE!>`L^=oXN8^qD%ExDBC0jX3Di#n$l;CRxIvH;$_NDS~BUUX8IWGV|Qin4a@~lJYv->N}Yz~ZNE^e{r(B(%fS9cwl%v?HY&8Jgk4qMs?WV2MwZ3J`$ z#$nqofl8J-&nA;T$MLYEjZh~`v&E*6K9_xXeV5QYOMB9$f<8}p*wsell*KT&t)?s1 z9Nyd|^2pNV**4JU_aEM7lNOkz-(uTDUod)jXIC08%W%@Rjjkkf#LGr3$}%>$>!T|( zj`($ni?d94b~orMjz`!w>9twQTkMAD3)x44yV6^-R!-VIr7sd53A4!<$TBmxAE7U< zITF>CF`Q+=v;RO}(tm_&b8swcb&LHted*|txUPefS!*WkDGZv-(FB`J*=$R52Wf^X z<7i4(rb@Oo&ws7#hN(SvJ{D*^cIpbcSZl z(Y&r~kL*c23@AA(k54w?P~7iz+f<{ zs=9KEv)y=3E({&Vsv4WT+U(6OPVNj{c2#{>UQ71YNheQ+p0KLXCVwD%oB3K6L%*i# zTvz^Z_IBRdP=-N&RkKaOSoY4AwH$`wXw}88g30XNlWX}5Bbj6EHifb|Ugqlr3}eQz z?yf?W93S4gOvZ1H$NFuGbaMP!))g{L*vGDQ6`AJ*Os=b7EE68PX;bW!!!}=E%~)P@ z>~2@FM@|rLeFI}f|FH))C4o7?E$f>YD@Tt#>MG&ogiNk)W0=Yue`ZrE$_X=f?qirS zj=$_GEzXJHIp1KIJ02gkDXYziYH^MqVpy<`zwIh($%&bCe#%%SJpR$<&_E8?e8UK1 zbq;lG<^fuh5dsV24+vbhQ7F48)djB3g53YFX> zo(omS%CTC;wo)fIrNu=_$C_O&*Ij9zdtlOqrehScX}(FI1=182=1OMJ~*UH=aL1$P039jv_vAl~d zTgP>FkDiF@J~f$ld2%a7*Hfl0!L~s*zunwjTGxwFm(ty!lHbX5r|Nn;)(LGJb@IDg z+?8~F*mddMjpq5ilkPNKUtwLA?P;g{e)DZ~UB8;TyzbK;`B!+`jCB3`>xyj81m<6B z*=DL6Fj`mEeTJ8ReR3O9mnBnQX?s?bf79HgQt|kBe zq=%>O9$|f>?H>dA56ri-bc1W^&vpMXod1xwJydsZe|@v<`LX;*E!#P|A*1ydyU$PN zKbhRl*A109*>2k;Tky<$hd?)sak9I+Nu}U9Z%3wXxZ}xw+h(1Dmn}OAbtBj(uXQ(@ z7rdU_QK1_tJbBZ$#i?M_d}p<8RL#k|-7OvkZ+JT!bff!EKCo>KEO^_pvq?8*^yH)N zR$jsT$(?Pw9GO$kY%hokKAP|9)8#Twz3jeFT<|Y%SNsiKp5v)e+l#dYpIdef>Bh28 zz3sl(Qt)+h*Hhhn!c!k@FAWq-nC~9ZjjK8Jx%<*^!4z-z2i^VsrzUJKj}?4x**&fs zKYHqW_vOigpOd>OdVHA%V%H{HNGv?1^%58jl09uIh0|j_sd|Zy4KjA^I)#$0o=SR2 z>;}1>cJsm+Q=T-vWMPAXU58VljD;6nFQukIv8ThMaAvHRkzQ(lgR)&`V4+;Am#N-? z(T2r6o&VJ-6B_kw?Z#hc-qx&z^RB~;a9DfS95xE&%poMi4j_gR!;dp)hiXA39}m< zC^EAM7||=MITO`0I9y~A8}LEzQ2!aO-Sx4e)vW>JdgY^M;(D%67Oj~Ipy*e~oK3L1 zAzN%|!IIXmWSmXuxuH^Q9m}HXA9g$|w7aQOY}?9G(m%pJo8EKNyx4wpx#p_$yruxT6&zAMv<`r+4Vl(xtWzJRF z-4PYLS_C@i*D%gi_1q~gc8d*k(XVwpS7Ud#ws>=Epu7GF_PP3=yDi0Arvg3o>xAbT z?d}Z}Z?gzu>DSksJJ)k>xOjVPP^kXN{&UTC_s5ENwgz$ZPmP|t*mHlfc=uEgU%x@- zk9NBu*%B{{Jp%nk#vk21Lnf@xl?*!B zO>(`@%u5eUh0+YVgiQ+ee>s&3EyCyq-8KI+(;pc0$uz6lzYvvXS%fG5AWD&tKxK`7=y!X{` zX-RBEsKH==vzh(tvC^{E2#&$^(dN~?uO~~(ry}?UH)L8Y?MGzGDlH-f1~(Zkw!I@N zWk+HoGYxJzwm904>XcQrMiv^}X1A>G9W^gIJ{4JEa7WnUYCq;wR$~!WZE&}yWpnSC zN7;$ks0M?3{Vm(<-vpM`w?;J?+#hY(+53i9c4{iB&0t8T)yw{GQCXu!bf3WkMyp@% z-^FETVxw;u3_G^6?cdgxookIAGI+>t4eov0Qg(hS`l-R6!qzbRcLQb37BM3Rk7`<@ zdfyF~wZ_JLFnHYG%C&z#R(7#9X58S(Xlq>W`^mD)Q!y07r!p53>_5mJYPaA>8$M%P zNa_8ca;P(wLpA)%@q*C)qt2o3R*sV4bM}Sw-jC*odZ#!v!xzE}S@!=p9qPB>(hXnM zT*&MF$K%kISgw)btNsf`_WuSRy4K1yHGDmKp{(~`-l6MLT&CfO%*9IkPohINEqD%w zql}AHy`PE?-HzqC7>+q!tg-)Gd+2T}&)x71`(l0X=axhFr+A)*e+w@*+J6~1^uQvP zW%#z{;%Z7+|8?xpqt;lC;rr2x7kj@>9(pnr%QyTWbE(~aT(`T{rC(O%VPwlHP{3N_|(|*#a zeAFVY+VFGDrMtb89_4Rh;~EUV^j~^lKNVR1wl%KF@ayQMN4-^8%3naj`Y zzlq8}TI}yLoM2pj+54@y{NLF9Hw-5oFOS-PuPy)Fx_`)UihcQQ@AsDSuT%S<8h#UA z{%HSWpnSq2e#G#5&E?O%KZeVvV&gv;{^-9vVgGZi{CjKsxZ%&y%inu{PL}_i`aflo zgH=f^Be~l4<$Y2QD;Dlc3N@NN(2g1J-c&5Uki;>PA8TLTH)E<|>9-`l5mmOs(m{G= zrRu6=fsukvhi#wq!bO0|(zh?lX|gZ%hLMU>C)+{pMCI}esY6B!13QEJYG<`*lnN4#aPFwyT(EB#No{sMDE7Af!+0eimivYV%~`t zZCe&8PWx~3PDn-gh3$T?+AMZx~r_)MgPE-7w ztW#9`k2xg-geieHEiE@%TS_AB9Gy2>N-EIS+qzH3Ln{x-8PXdzS>waz&fDB<$$TSg zH+Ly4aJF=_v2-QMvuthb-HEca)i!rI`FWPpeyt;^NO6=|E8*fMIa|WnO;X)vz3qB; zSu9!aXz9M&#ny7;I=qisr>ppfuHLVyEvSl8ic-INKHbI5T}i?c`$pq?il6Sv{_4Xe zTOz4QlQxo|dQ)8}lvkyRnM!trS+|rrl{~K+YZ28uO3zEri5aOWT5VLmT=XoH9*SzJ zF;z5;pp zTZ!G=r&r~Ar&9Y1z1FdNOkF5O>gTp3(~r-~qqw}Rh?b>PNRm_~rXWj6NrK{&AK_QJ zS-j1UQ6a1JT#2&L+J_&|V>Y8W$HPHQdj1ZXp2D0C4?lgW)QG36UWgWIrbkFw1k}3? znAJ5cld`B%@)5J!3#TP3_000TYW8~8R;J1^BR7%G@iI5D*@4E5SB<+Dp;W6BvBUV) zq8>X&)jS_(Mpf=Kk+IC8h;?MJCfwPpN~v?mORtJ3dH5Cn$AS9v(W@{p#9Kg88C6Pd z8PuxW9uM6ramOrH-8>T~WBXa^SBNS-F%g;dD0z#OJLjn;8};B! z8A*yw#8k<7CIe;z7=+D9TRWuae2E))lMc=(M{Iq zWnGk&sh-AWCq{Q+wAfgVvMiBey^PjHudSKRqBfBB8GPehs$}(@ z*}SUUznBdFits`Dppvhhb#Oz{kSBKK?$~X##mO?sxaS64QqE1v+wZKunB7l#I^e}_ zc;!6JTY&MbExo6tDzJCVQir=H8PEie=ZPHd4HQOh3agWq*r;N7DG76|S84A_T3m-CF zXMF6jsyshHY%CynW}#4?wwb*ae;nlc@ZE+f0Y@e8SdFaqaVD(#k5?D2mo9K-Xe+z* zD1}@|&Q*DTv6n)BSK9N~jUrCGVcN6EQ-p6@QA^%Pp@fRo*5$T4QJECgHrMB3Hz~Fp zh3}%=zl>(XNHspnr>5oIQqmPKr%BMO3cY8W2Nj*kH)c{?#mX;B+hl#}8oWzQB$a%I z9AD4#S;r6?yNMa~s-FiLGo)RajAQeX?wRkYZR!XWA%CUJL26U?UA0Nm1 zNvn3bv1ctCbY)$*OEIl#_?T)HqLI==_n{dL=*ZW2wQaLq6gId$NW8h!M60Z)bf)T2 zN+JIH>T;2{pE$wY_?VuHSfQiPbDdF7;jxY6^&$$BJY7b)VVXkj=IT)9`P}oa%1un9 zQlu4nXjS5AOd^S%haD-6a1N?3yFHsVgMUuE?L^VXT{`a@{<;^)Zn#}YO?mcyT$~I#XEOVL|s>&T{Lp*(S#Kg zdh_ZK=OM7(S=z>{k_6Q%S>m@^Z<+2}zi8+LHC^N@786Op!-3hML!2uwuF&pQUW}bG z-l%Mh7nO4?biE?wCH))Qe-^6S4`3XdDL01>={5E?&J=G>S|&N`wm)oWuK0ZmZh&TW zkX`Vn-6c_Dc@x7EIgCU^f|vx+ur~R!F8f5;#7^p#oGPO9Ok!tz=jMs$v=0YN6fRCt zB<4|bH9!Zoi=Esn z9T60=b$~{{C|Eo(NLx1`F`FdHl0?J!Br$Bjj%}WN#qM?cuc!lzmyD7+5^-ABTXZpQ zwpsDK@R9cqXr+s9D}QVs=3dblpliyGq9sLCM3oOd{vU46$&-Z6t1GuU?K6E~+BwhL z@+&1n>Wq$o6Ys9GrX_lY*GMHEsOef*w=OEmwX>m2m^dac_mY}xOgcAHTx3ZknGK3Y z$r!5l%FIp~*CORi#q9!DL3z}=AzM|6r)O?Ki{jg47vqK+Z8UsI09AX5s*}RXwN=EN zcHLDD>rtdlY&R_3QSKS)Q)g_J4QPA|Oxv+JA>6Z0 zl~JJWEr+q(B25g{Vk87{+NTJ^CTfe6-?!pWq*)lf+I=`h34uehY!um{ zFf`@b&drI0@JX^T*x#rO^${^zp%2)o(AG)YV~1>&Mu*0?*L#KznO+p#{#Dg$KCp)T>^E|Xql$b;dHlKf zx>dtO{-?^LghM+O8CQJo4&;r;`YE9(@>~Uai7*QOux~|UJ1q@N@@b!wvP7d~j7^5c zu!ltnLJ61QHcB>W+GWG;YLbgE{JB>RZZd8s)(Vq_E^J zW$^!>m)yd{zaz+m|0oJNdGw=2zc0TjGmhR-PPmjU;c=M&BCyQ+i5pJ1{h|T$CgK5< z!v&g(LE;tO4y3{e57!Zi8AT=95v~$nO~dTUqXu~rZj+vv|D5|~FbAIi%M$ePE~vTH z3cbFO%NIq^VG1cwtq|-8mFmZ(P|^l}!|M6pVo~{HDxS6?0*5p=Am|9!OoLwq5m1fP zDitb|T>+pUW%D!)BzQa{3Q{!|;BrDHnBsZ^I~3TALJvG{G7-d81uYy;KAGdJc!Ba{#1G3)zZU zk=ha7^!}_AC~ykC%w}d%zI|{mua$!bmTgyJ#xwRqbwGdJ3laF8MqIvp@h4?9OG|$* zX+h%e+S)3xJtc7Y{&9@U>c@TB`d_KjU#hOkM*WBCS_QJ3&v^^pn?Id>n4cb>H2tp? zme-djn(oSZ`yVz{G-rB7M*Ix?Y0gDy(_dTGbpqRJflJrR7?+}# zeY*P3n)H_%Y82X(n}yj_+3-^5(W^{pPecg`c%V>sx-LEFX=D*5`Z^ zdBg$mp2gBV!NLxA>UW3>KkC;lCa)&sjX?bB%ew?NDJBwEiV-~2dQ|2Lrb;P;3F z6+lPpsS&jLm^fI`5Uq~_;x`TA6nbB@U?IIfTA)|#i`G|F;$MSU#~c9iTav~28Aeg$ z6LFsUNVGmKY2;ULp%&@`T6kEjBS*y%3RH$F7E6zd(?now5Vf2f7pqB=I6?&elTo-` zoT6xr*2iqzx%+AhDm)b}IHfujt)Is-jl?5zMqEug-6ehDtk_7Oi5|j`)8cBy$!L8r z+vJn(4Cw~Nu4p|0#V+wSBp2ehlw5MpV=hPI?isl(R*U;>!I$fSa0Zv*4w}Fg^G{gyIqbq zp0`Kq)&0y5ZfOIJ7PR(+_$oOJrVd8y^ZgBXyFNyJhsBNLpty@16F*a_AK2a>rTz;rXuRYE!+LZa>Zoc99lr%8XFCtv2V^ zM5x^c6b9|@Nt3O!$fZ*CVwY}VRLFn$7ovXf1@~jM2S>^K*8e-jI3fnllBZjQ` zhF92O*gIf%%>HY$4owlSx%l1tpL{X*@a@NaKR3P$A35=SZa_`M;7d#($3#@9k8>cx z_*Fl`0;OMP<7eAN!K#3os39`RGNhcMB9c8TW45g9IAFCR*rjK7u&S)UUBB}(hwy8f zdY@=N;tXnT$nbMmE(!`L3r=z?9NV55=CV@iOb68H)e=UPALyY0qgjEDgM&Niqw5S6 zooy`7w&v!UO`4>tg0ovPRH@!;Ia%=<8Kl`9!ivG9?|L8~a5Ay?`F!8y>l-cE!Rzv3 z9n8G&k3YfOw-|UgEq!L^*r8Fx88O=%%|Ndh`|2J+f{;;7>@i7en(G!JC#PC`S3W2%io}zz-AXq#ka^E7t2>)x=o%0NW;F z%rqD|#K9PCq9P}eY@v( z8u>XonkyZsGC$%2nM^-j7UY9XYOv|>OIz28wSIc9SeHP@JKA>Hic|9+Rz+I z@9iX^c@$|J$~cVnH}Q`V;qvAj?3HO_sCk4Wg~@DIs0rgiFjypqJUFOb65-H)-I+%k z8rwC3cF>$Jx<>4>RQ3moMxmgKxAa&^I0d)Puoebq*?FnN&Tkn$`-Q(0nM*jPBwAe1 zm*bMT4>t4w0$6#crMYc5^usU+?DjV=Kk}+duK|yR*LC*7kqw;#lP+)xhJa8SH$hNaip$TQg{f zgW?FcN$f5T6<(dFkD>%3RD;cxDu&%7V$+CSoPpU9t>;Ckz)0WzW^c40O;P##o^Yo) zg2BdxIw48hjcp-yl-?07D5ZBr3sT57vB7Q2?|aH^*n3i4q*h$bYz6v5TDk|5a1AzR z)nXahAZ{de;;T$8vVNn5Td~`!a@UZ}Aik!LQ}r%@DzH7P!v^e-SdR^of*cWR80=A< zOR+Oth+v>U#ugCZXY9rjd#h8V#a+!>73T7ucSb=La?LQW6M`4`LzVEjc0E~XMLl?E3pz52`vPr#g^FKMhaWAHq1{Vekw~WmA3c2f zGm5`h`u;EXMsH>XH6N^mqLN=8qTH)#NU8Aw1Q6n#f`GWM(CWp0_)VhB?j87$rE8=p z^H=VVetadR!~wr9J#o9#8?E~0#5WqyKk@6YES&=&{Sm(OM5NBc8D7i31fZdKgtrD9p#eNKbwvpj+Z(?$^GHXjP z#wz%v-GF_EEkiqYBQG!LNo4L5L?3#qZ!oTt8d|w9=*waOP*DvQvFBrWBar`sp z<$3YTQs^{4zeQ0iEDj_IKmg*v~L>}99)vl!RjlU5PF ze1*U=vfQ(7VtYn_BHSXou}6~R%`qirRjdoQXi6StRYW`G8V)EenZZr>vyo!xTjd{D z`#E23Z#cRuaA-ES;dH%cWl@WzupvN+{l&I=R~yTmu{B1{F5-MM?En@W`bI0PF-&@e z-aocf5V~nauu6S4%c(3_Wf9^ZnWj@&1$+ZaYN6Mu?mHdSKgx4V~ZR#raQ%pFCKTQTdxQl+M1_7k{4tVI6LQRc4&W24e~t1 zV{0vEdD+Q$TTD#`J{M&o4H~|PKdce;bEw)zwFVo@EPusR*&5F6W-q^*W#dgsiAGq& z0Z=2AEfw}}zngwkBkSLuGf%>JtT}VH`N-70Ys~o>@$!l_;~!^!I;|}G%*wPQQ?=uo zd(w|MWPR57BdRQ^=(Nc=I#sad%*C{$(=6WZyc=>P!DIROd+D7C9&1`^gtXjZjg$ZQ zlv&=_!dz;Z_ z-b(XfiN}(Q))eY19!#~6eV=CNN*1-5kV@8ZEoQK%4g3;+#Z4SKSmX}7DOHqani3^x z^yj@Q$5W9aqg%`UQs9OrsJ=}9;~ zfTO+mKr+hiZ10ojOK47WZ(-Y#QMqr}rafC?vGM%PC(k(+gVepX-AhJs;_tN_S;D;A z`M6N2vB*B(Tl##7wz6|@VMmguD99y4C$Y$FQLcQRbo^S8{*f|y*v2<(SGGwSD$e)n zTa%Aho@YtZnf;V&+Y+?Mdt%9G2PE7}(@B;N%Wdh&>rNR}xm+J%(8xBnXj`5w@m+sz z#FMkhqXxD2UX1@}^!k3cTUrxF72!v|83L%DW+Q}4wduzmi6X9K}# z4_MRaDj)We7ayVcUOG`H}&$#T{hMx&1&-NT?t6CPje*lmU8Edb(@z?vl8`l4+%|QP_7c7tY za7?kC$xm#usN6CA_qWZrSbr}6>GO3X)_!*__3MO&DL3E0h(n)>+8))5B_qeVy+7M) z{?c=LeeeF=>%?25p`C)g(o9pH_Ek}k*}b0| zbxPUbcI8@UtMIMXV^qF!i%k}_PQD2%-*zig1XM>jW^cRkz{LWiI_;%=bY9p!g+x9w zuIR{(?{L8O{m-{M=KF3a*?PMp$=9JDr~C0jiw$Do<;U}V8523^ucR}>w+_?KiQrH; z?V1w-V2%~T$7x;5{rSH3(j86`MNVePsiCclOQyM9DeO)jeR_G&^EP=y^WNTPZ6A4s zemkNz@}s7A75B}3GaJa+T!C-3e~Lyu|5@gVcO}Ej+cW!Z)!uKU6a8~r`fuiCjoSFe zO@^=aAE%MEN*MnziFGTjwW!jv<_)KY&ld9r?sXX3U*tRYjgzzcA8X2#6}9v*$g^K= zmSxF|TCbE{;>CZgaC?^Y{rnLaI!C^cYfg&!89I$Dx5ZTHljnQ#Y|RZH`RH4n^M3KM z%D-_|d*gv&#I3VEX`jnWYHoWU`&GX5@K@jQv%X-mKZCG8pIc%d_Wmyev?bY(XOy2U z(i9{vTQoo$B{G(HM%PMcuk#zC|UUb*6Y7*8~-H zr?hmWzQKw2zI`@1NR@01be!X9Z#72IDGI(L%5S*F(3@LKa>c+?l`Xmq-Hh<96Ox@I z$w9f@@qr0NL7ThtDhHIw?uceKD#}h#R$2azWx*mVA2f+QA8nO^rd57mLUM3XUpcOs zRb``WV0h8In-v3!^Rrrl=XGlKx16UYbghfH-Y+ZUrTXxc1jTq!lAxfar&^qZvr7M_ zONN6omxft7dC8M*6_Xo3BAGf*LiGkL5=CYKXoo!)ds$TJ%ok){i(oOo}3 zIup(lni>s^>tM4)uXK&8NZ+KC7)VZ}=@wY4u;l5R45)s4mrSG{YE~DKCKcv%Gwh^Q z(p0UHMap?64fE}age4L2cTk`Yk{ZecUg}8;9GIt`o-r>?M~!PV>FF4G_|43&Kud?fMt}c> z9O^BFqzlI{oTla;T5U2qNhJB~5mFYT;K)l(-DDIJ3$xz&0aN9U9A&9M?^zkPtl6LT zlm;$q@sE(p&3HXGb(<=Q%8pqVu+=z5WtF0kSel4Q7oNA$;4ER}rQ>UR^3v_ZCwFu+ zBK{`IbW<;~D_%C^uB9czl&M>`r%9wGi5dngmI5d16fe8V%cuSQlk7N=>aG+oSBprO z3|~v8xZPQGgpo)?Yn$^tWi{kdQ=Yf&(&CA=5|k)Rsg0Bs+llXQ>o&$$NAMv{oYLRV zr{0|IIhfNkKXU`8)LJT>t{>p7zLc2ZWw59a^LJhj-zdm$<-i*X9Q;wt#9=?T&K6$iDL9vd_s%g@}+JnDBaj^bx6uC6DY z3v>%Da=B;6)L!%Rwm+L}y)YGi1ieIp6MT0GuAtKXoz_!$3X0<|nX<@_iB&91({Z#} z;+}M#UAb|*!z=LBneFb`>GDHSE2tJ$(PO!JR@0HG9-^Sz%@G=p&(1tqeAHv_XWv+*>!$S=I{&!%(`kF( zzBlP_i)K}II$%C4(<(@(3HRN<%%+AZR4%+RyXW~U=7cP?sn`ZUzGqqFeuV>z? zX_61GO7^e08s%VLyl>m(SwkX>&D$2CKuu81CHe4eW^p^43?ntdoSXA>Y7O@icY(fU z!*$NrrY$7mBD-OLObnE;~MD{Wbr-pZ~A6`G0u^ zq_Ob`vk~n$5$V{@4FDTl~*E zKzIuG?*Eq~1Z=%=1Tc-=xc~omh`=7I{9imo!082hCSyn89DzMZd4=N+9B%ZTB=o`i z&Jy|>{_6E{0?sU6y>?9q%AP*r@bs)>4JQg*3LXl=g;UOo44^oK;lP7O`%7qIRNxGX z%Ix#!8EMK$Z~_B$bAiGUa)yu7dj&{H2#+J7^@MgI{RB`hj)G(=D2YPPQ4yRdTS?SJ z5WLfHA6QIdul&qVM1|P1(#vfc5fuk20iNBHPISM9= z~AQN5`R7OjQz`s-hG|I#h zk@hhR1c`!mR7Oe}@QX~`!2@yBfyR<@@HQ~O2z-g20|`Jt8isHZ*+V)-AWVZ=&cl>EFm_KIee0O4$CUDf>7X=L$mbJ zY7Q;|+J|Hk@>=r1w6dZIowouaHKH0lSVS%K%ash z7hQ&{dI4y6CPcwQ7o#mMP!)pW2nr|$E2vBrkzhImq>l#R_~WYR4p4?vGk@>#CZR$v zqzT^;TMDoxhy)u|WQ9V2ezC&sW0qjP1a4Yxs1+**wibw`uix2c39E1h1ieB}67-CS z)K?q0=t{FQ?X;*5&4ciSmI_b`#4SR!hE+}!^!JtH0w7DH)c+S1t}JrEG*Ni+Y-s4^ z&l+4KRIZ!n<~&`XdO zQLJOgWD1KN!dNL_1C*ucI-`VF$H<64IglolQjtn5B*cdcIl$j;vP2I-8mzsra>T{b z5D8YNP_cdJw(fj+FVH# zOfKP?X!OAJm(Hua*#1*c&B#_@7&}}0(NS2bgHt0euyUzVm2z%usRErMv4W+*$XDU8 zHF$-}m6B628kEBA0}H)TNjOHAeVZ^-|KS&8*qQntUcoO1+8PJCyv&Jp*_RV5|JNz= zFDGNEjqH#6okA5jAW>|jzzn<@t{TC=%)nHXi~=)c1@Av|?|$`bt6a*jh# zA?%{uu@&V~dysdI;*mX6ET-)tq@LnYvnda^QkME=Hc}Ip1A8z4{wj~;sMki5z>^!Xhx{eFuRZ~FD!20epoi4?_N zorJ&8t$osK?;|cjzDWu0pZ)!+=|$Y!WSyCCfc5r?ZeHTT$f4_+N+3&tZ+?%A{ zkgzvWaXnPzGzlI>6MM0;?`C5+oO&H%od~x#r&2zmPPi5+fdV%^`}~)u#Aysb)CpR8 z&T^Z$g26RWo)P}M6ibPOieaz3*05Jq&ojc4(E=m+SR72g6j$SVXFd5!{D}S{TA)*0 z_}%H+CDzf`qld^u*Wq=g^2r^sk=zF2Re1e+#A)0GbD}HPZFev zZow<6%p*6%r|6z&!84%i$UXNIau<%ni>U9WdnxKv;3{bZxh9Sv-@v(eQ+yx!+sJ*m zE58<3Gv7w*osW=SF&l!(1F?a8FRmbu;c|S4_TLugF)%jH_2ikmo_q%<<@bHg$UHbT zW#l-Xo51QZ@gulFkC0E?jkM#WkTf_)Q-H9I9G)Oi|LIyUOxn{?HeLVhvF!66qnH21 zWBKo3@bU3~JeL2fJeCK=xIs$v8Y1FwIMqZWm*fQMt7GLdPLrWarxq1t1t>LcoV<3! z%``s;GaS+hWOkOu+8Wl;W2TS{(8(nhW0{;*oZ;d4yQQZzSh!q}Dk9V+ zO*~FzAjkS-tLaFDn#pyd=IID&bRvy`8e_1bbRMyzNn_7KnW$vhZPe~iV#YD*z2pp6 z)t#buz`0PFDT!lw_FlD*PPhEzy~k;Kxt&TXew>D*qO8_oyoNkQ>6uA_x44q{Or%)+t;zs&wBZc{mO-X zZjPg=9EYMAx38soZaCYXAu#w2I5$SSXyGXN@^KBY$zUV zeC($&73VhgOqGJUY9`gMqJ?k$$f&blKKKsYpYrfL>sxB?g2I410_2HZZeB}uFcq6Q zb)@=Hoyq$x7kRHOPU&k#H=HaB{rY?@d5nE`J6l6VW1r_`L-7pjMzpVZ#{O$lg+R#? zuGUK*EzRLHRzC|?^>4dfwooO_FLIoBROL~A@^0CYB#-5-cYiqIN0zsat~ z!m)S!wMiaj=F56|_AQNbz0kwa$uzRh zxIO7*4x9VucPIGC+-+_*rM$`KE_?8!=g}z9s$f+ne1gWIw$`5p?rRj?o4Xy28GS%wg!4h#{^2K6k`QLw(*i3 zue;$UMuekL&DXQ5RpYXNPa}%-JFS}zV*8_r0%>EcNWwiJ+a0{~vhV=eZJWi2GyCLFe}wTXSe**T@dkkXcLXtmN7ceHoLgd7%Loyvb( zWjlK;+!U%!v-79Dp;v7x40PNZ!@4Qv9%h{u*a3VoYfWff)QB%!yP~#7X(wgt9mhIu z4mKxDUJQvcTrPMVyeDtlJxJ7J#hFP{j@p_LMNdK7)Xg%%*n*mc4YD^--(qdMo2{ZP z!6Rtb$+R|E!?o1*U@}#-;!KVskN6C5y|=F|@RQbAHv7(|@9Qc$`1Q5LH5C*37j&dH z;68g{Tf)+Y0kf_Ywk023=PpI^P{iCM^B>cdQ{1)|++DnFTIZT)+mjl~U0r_Zh{`t# z)hj!!@~h$3ZcEq~&5=cPWP4V+dCM)}d)b9XO@h6W*?&2=ttY~}+-A;*wGmqi1i68? zzFX~jy$E;kZD&-gwylm2_upvHpCVg4qPj0iFcb9CbDh_bX=Y1 z9DS(Oy)0N$Ixfi3)_zAq(Yn-DIDbWx-A?6o%oPQETX9S|Ab44^DbTajhkv_0c(v>k zRigAsM}qf_@0NY;4#~Q;KT|H<9jkAK@ebQyipmC9}cc7FSf|s+ zsf=rlqrPM>xu*{2zfm9SmQcjYYGNM6L5K~StD=V3udNE-iE?*vst>M1IVWeUcWjHI zUE7Z2d8`@xb^2sC4^jK{c}Lx3qA7D09CeE`P1V=w^qDhJcA(ot?%dU;} z*EmHsi8jox7^>NMK}em)cA$Jr$B6h&3yx}H)|lcCdhVL#xU4vGzcwaImNn0S;#G6n zB$rc*xGn7lIX`#R?)G3U-*kOP%#=3Sv2$*#Zsja2x>odA!htv_PW#QvGC#gkQsS<- z-II#0%{mxk>N`b`+kN%^r3BBo-ASI;LS3fhCta|G%sV2B0%tj|+yOKR735>u0B1)l@=5OHG?* zV5ljsBPMRww2ol+DA!8IhO1}FuUA^QkXb(`G)y~4W-d#wOeHJx+dPj>bFuD_&Tb$s ztKGeaHH5k34{1xN4BHZ06kk>OzG-rEYj8bh*vgJpjb)h2+K%$6l`@xQ9pzq?c%z}C zTvU-Pn5fXM-JLviB`z*FPe6gBP&%2O_``Hho>jhaW*eOQE*CqKs@MWUG~-pAVb(3J z0Pf90fwKV4Om?hr+_JeNDb&RYG^Lf5vX0Gc4b1+$Au~g(!af}KdqGqY=R9HP%6!k* z%WCNbx1ff$SsmAR{UoeNE{;p=ieAA##LrCdk97+jztgf%>4y6{+LR#^-r%;Svu{@v zu1MIYkQA@K02B#3CIc>ENwFthJ2^~TD2Q)}4O<;#9SG;?x-_qw)(+HO0> zym%QDRy=-7ybnfvm|IYO8^?AkPV`&*Ou~Rz{*hpkyMMUdk)W2Sh9pldtJAWm?b35A z(8N{Cl=pUd*$tRleaEkDopy$DWyj5g@U8o!gu3;*Y*N-dT%zrr+ncIef39L!gm!ZH z>Vk_>#kzHa;nNSRvt#JyMd=${sjU~2e0!8|4+og8r0%~vWqJ_4Fbx}j;b>Y3?Z>RaL7jK<=_ma>0oPaTJtEKDC8OItL zyo9SVJrrraqgS48KYBIQFZ``K$v&#{C)?j2?TS=(*%VQ*?n9?Z8ajI1ke4(NY5qPBa;)5+_@cb`wWw0M2g zTO}!N*Sl;s4gSX3H)nD9#fjAsi-NX(d{Mf%D#o>RNRpjRn9s$O0|Ac=yVuPcZr2sP z?{cVq*zPbNIHz_Lrt2mR+-g^xKEF6+%*mO7_NA`b+{$pS&0uL z89N%{Hckr9Y1tm2s|-6>W%Ap&;dJ|eZ9+dNYeZ$Viz%!wswaB%fG@VP3(Lv5TV8bd z?zkLQ-PJYS^ITZnrY^PAD#ZE8_GqDL*6@huQqRnDQ_b&!6?<6sq1B5D7zhWc0$r(pbdn;18KVdKrH zomx&-o1^B)x3iWOtPhwMFR>Bw{F@@wVZwK!wRx^PE>{KAY>SZ=G%7xhh@}g*Ioj>M zEEn>|C>b~Dr1b%o#XJE6ZqvlBMIC&yPV9B5$fptOpgEC>S*)404~yhr_f%?q5cPBoG!&=88=6o@3P}e*zMXib5>Q=mMz=={==Ct zVd6zp|8GA#FaHCGSKcP6b_zE|SxT8nA$Y|uTD78}I<73*c@;E!d$g;SKM$)x;_~dL zKRYwyGiLwUnR#lw?XGbyTMpxNXfO71%)f8-`<$8o4*u&4?RuM0VfZ*2d-SZJDI9uF z#hEfrllu;$3Cj;3fj!1xU$?NgDX{yOBUPHs`@2)~l=wdftN&XMR^imVh?7+_6G`Rb z%|c^5KF`qt<*1*-HO`T6c{dNuxe+6Rqw@w|1b*$|<$kyT$&Bb*S2$W?C;)J14UNH3A;M=y;D0Mrz{?AE zxO|GLN`fXBDWn?fSF9!tA|8Er-M{Q1g09Rv+Jjrb4PlK_f|JyMRy&Fu*7WnXYOqli(&d<+pm;*H)0NK&GK!kdLy zLK!J&tdpJ)B^5G|^_2|Z$KM|7FX2Mpy?A5c4dq|;gz)esg=KSY9Ez72r`s>1Za_Zx zJCxwv8+{|TCkAUGI#WLf!>N&w>wJtxGkfJ{{?m!ubZg={wgOIe9P{V1-+hdKn!*0k$MPQ^BQCD| zA72qd{(tWGJo?A|p8wlE#+Ms00RHY{WCy@s`WR2Gn(R`(YI2{a{r})&l-nr&@UdHX zVfd27`7q~r3x9qZ7KOGpSQ4Jn{pkz4$d+*t0mWR+BA(Jw)F~Zt*|F(QZ1IwQ|8CAH zooxwayP9(8ssTGSU!2C(+ar5OY z`vo$Ez6r$4&apX;2)OieP5P0suLfjrQ-NHAr2@Ve#?2~Vv9ejOcQDxHDCXh`S6(PO z4Mv#qGvRjS_3wRus(39~65 z-nR?txMqkNod1%pj>MYjH5L4x>KMs=PdF%e5dea zv@Q&fd+4b=36oWmiqfvNUm`vo_!Ey2o&ROT6e^hh)Ch1$CrYfr!^-o>f9DDEML!}+c@8%4aZlb&I&K-#svU@a0x^2hRpi{6 zXvb62-^U`|HXr1E1BGFW1rnxxs68@7o`Bs3tiVSg!TZPzqdq3M#J^hV!c#B{>^=oc%YGMHKYIu*2Z@~syAVzw+yisN zrAo*dgNU;byFQlfkf_)E~ zK#Sf5d=TK{QURX~^(LuE{ZtOY{JP_^GkRtx8j#ZTj3>dX+zdUs84`bphm$QRFr*bU zX0jbx^$Uvb5i_FR3!->s9`oauVIR3B0HfG!1Rv zmu^4?T>;F?bKXN&jhG7@C=fII(@hYY;ZaBewUDXsFN8Y3EMJBAz>igkSMNNJg4*4{ z?m@+xTwx-mW5&pKAg|vWyC6^}@LA#wK2MGSiyR#=zXNDJw1!EVvxlb>ZQSmjL`BfT z2IR~4VepKYJbF6*%Jzjg07l0nicA1wAKVVafO+Tb50YUA=Yjo9kO_`O2fT;GY6P}N zREYuw8IOmnb5ZAfmd$RT7w3k^uLSv zL;r7JGL8fYX2Lctn(YYMJ3as#-C#qp5ItwYL;E6d_r@xKW03mcG*Vw+%+hvOLG!(p zqj<5*X8})8^a0D(A{Zx8)C&15%E*Z3Mxv5SgO(ukm?JWcPKXu{MidEkV}?I8VH84Z z>8(GYvU7MC^91BeF$m}x>|ji^NnAhS)NCORZ8A>p+B z_aVv-&%+2TYhNPc`3`}Go{KRujt^vX&b);(&oTi0YzfM+4Y-G{WrRG<@drVY5CV+&C7a72f`1@eUzzPw<1aycG(bHiZ~ zzW|HwU9lL2+jmZeGJ7ijhS>eZCXhdg8JpSk%P$zTOP=*hrRr%tCOs03A}en=W7?l~ z2Xqry#a{r4u=@86>NPyV4dHU9H+i}~r5UI~W2~QK@MMhSqn#lB^60M^;*Forn1i`w z09zUqanp`r97d+_6s##EKO$8iLYY48236799&U4C=StrIPtH37z;NC-AA-pYwpq7^ zuLB_PToVA!=Rt7^_^9ea`)^(bi(g$qB)W+xAD!994kPkM`TfesD34)vY%1qd-Ga(24>ECS76*=lNj;FuOX+)pSrxK zBuvVq8E9x;S1I_}`vrdUU2!e5kw2?I*%ei2I`y>vWo z{S79lgS!f4i)mD@GzYyY$T%kkb&Um5ok-31Ln`-8Ae2BUXmSEnr*PKtV(C;a9`A8$ zpj>xBfSG7u7X~L3Z)svd35Z$60pwJo5vyA{!A=nZCZ zGnNQ+5r9ljQxgb2fmvWKu4u>O1xS?1YCefa6N~U&<5@gjZNekB`63=E7WzfYH9S59 zMbZ2voOERu)BFLeOLimL5kEh75|ysJj`}n=z}YN_ZSb^9sMC~&6_2(S~9IpylK7c*TTPF&|}KLwB#*U^5?6 z(KfCv$~E)@wl6zmF@zY=Z-XF7l|X?iSwoHW6tgWX3`0$dB?bLxA=;lXm@MEK$T*~@ zu`37G5WTYptnGa0gz8$Kp}KV&-o)5uXQHr;Xq9C(@Km5T9-#@)c{P}NuqLA|cf*{q zIhT>>w+H6Tem@B74NI0d8~P> zV^P0&0wC>?g=ScZzDG0I8EX&1OH|-$3(Ku$%lV4%xTJ8k!dPgz9d3dB#hai2((HiyEu<6HdApOpu0WcOj*-8eZ#eQnQ zDjXqAI*l#9U+IkEg_!Xff5)MCbYr0=M=c&x*~jZcN1%hWZd1^s4vSbFKFrP&-X(~{ z%OSq0VAFPUGiLPtutq6iZ+<}Cr^{kdbn=){5H_nq8q8kw_O}3gchW3KVAjalAQwJv zIePusDfflg1)129(+*m{<$It3w?LX6Sc=Glog_cql>m;*S+nk&y#WlxQMI688G?19 z$akh@5oKT$j$^>xcsG=pi`*>-9?>UV zC<^OL|BCeyb8}27?91UmWVk{>@o^=rAF!+z-N`2(Qn5^`w+^JpuKi#n9Q94$W1O>= zid6~v4Kpl3ST#R_#AI_M67fJpP)PxxSjY5@yZK@$YaS9HOjkH{*C5*$DB560`;t`O=QJ%NaS$i)hjKF%soW*JDbZNIn<6-=xL0aJWEPs2>} zg!t)p@XS0Z{TPbyBdY^)RtMM_AOIGEdA#fxY{!gTz)HXkM8T{Qwqc@5xyM_IiQgQ< zy3dZDNA4|@n*9`$e9A*ho~CcEhMcQ5SxM$)!Di2E%)*aP+`DQT760htW&?QDBA> zDk}sdEUQ>I;AT9wUn(qa58#30jOP2XeQXIMKug$;z65wbSHRuT(ewc=B3pEF9}x^^ zR{Lpc7>rX0JC2sWi72=R(YQ^B0*`}{Tc!7TH<@4V;xL8ocMnD1K{4#3L?33_o0LYn zBnvQBYT{L7iukX-EH%V_;{ z)cM6Hs8hurUPQtCum-lT!&l;$?Pqx#Ivi3WPmq||jv=l;3Ea0%Bg#L6XnrRm)mcQr ztmPWpjmYdGD7}F~*f{?E9O!SCK2t_wQ@^hO z{7C|W_Ev~YtZCcg0Mg%l--~_4h^zO}7vsDi0Bi6F6ovhzN3z?iR6 zNS*Ky)r_K`+@h0rsa*dGdMc*;zM}z*Q1T4e!=7MxSpNbj%oXkY65NRhM-WWMTcE$* z16CH`=M!M?My#`h!3*9A%ASwhp#WARCxMs4FQy^Z_bdfR{cj=DefujcnU_~d^Qd~} z*c2PUeLq44Eg%%O6r5uD)3RK9ewO%FV2LbgGG7}E5$j0{;^l> z$p*jGFuJsDJ_@)ZYa#GP7NEdeNMNeL$M#?(iZFMCA;IqMDvOcZh=(lJhHiohU~cnm zoh7t@0gZvk7{$&`i-G<<5*Cp39KnuAi9 zFqn>S2SYRmuv}uyi$-HEge_&%``?9->hcpIp5H<;5T8GWT^RlGrz~1aE(Y03b^=a> zYBT=n1?c^x<{tD_4u{Fdmw%>m0pFW#1H77I;0-N7+V>|r;LUFS40xjEi~{U70+h!= zwH}5-v~{FIA&z=ie4B>a?i#U?VpN;z(2*~|&*Z2bAf<(_ioW{FqX;Fl(S0@iFfEGCF(XMYC^AO+J)_{2M(bI_cr$Ko1v-i=y>rdTF zsWke7AHl)aCIcGKg`qC$d^5C+QqY|&!(W1fn;`5y8oB{1!5);*L-j}`BXI+XLVqMS zB7v!d_6tCQ9bRkMv4$z0ZXSz=UOddfnFc#jXkiGXF(wqmVX$crb`dcX58)92+e}4k zG#Cc-x}%6BpMbOa;t9NGeaASTp%!+j5K{u}-n|@oVcQb11lxq^9$)Z(JsqCLFFPfU zYJgoki3H1J{xU@h>qBUhtU6r zvA2%OstFr@8Q8!kq)`M^u)tQr4v?@g5ep1LEG!JlqEi7CB@7V3R#X&3RKmo@uE%bl z$L>16&HH|9owL^W$8q_aJ$vHXlh<4``{uq6Qot`tu|^!l$*RVgeFCT1YaY+gk*t;D zkpT$0%#GK3o}yGtT{%mef+w7_G{s0%T1puHTi_SoTJG>&8{hwt1n!B#=k$L|s&9vnE zRP|E%^496}{OWB4{A#<1-rj_TA_r7+s4o3grdW5SHcgmkQkkNqt#8IMgB=ie`#$25 zI6{zDUrJ`_o@kQENP6zM5qSefD(M4=YY>{xE=18aih6jm6!qZUrjX<(-aR3Cdwq1W zN?!(XueK^HBN=jsR0-bE&ImWSsTuCKHqCJ}FF7;c&faRWijz{aJBU}ii!=W|PCZno zWTBWK2$CC~vSOrPz0j0$G`mOfSD-bG9biw0*-v(&`rC3Z}Ny)QK#zE6!jf$nSH%F+}g8O{Fjls=DDAQiPoAKfhyjvX;oY zJtlQpLm@x;+VSx7iXQb+^r+KZ@|&hQpfnTHxZl1S#|^7T>XVHm6tQfV=7@>&NTV~A z*}hFNV(#7Rijd|1kl-{4ZaxY^6N>sXDXl5pqSAmzH#%9t_A8g3mY*)-NJ3uz6C+Fd zS2^-36^~h$)OUKX6*rXwTB!}YWZ7zjveO-_tl{h z?O{C0Gb)ucg7vi+-}1Fej2c~!zB@B{5AeyklqW47O3F#KSol)b6&fHk+Br+u>IOTz zW}{qF`y1UZEtmGZe^t1dhQrT8oc1^}Lb6J6VF@&=h-8)rw>liP? znU=au|I=MB4CHstn~9jUMt=i0yEpbBt!9Va1Wl`IzL5nWbD%qAnWJ>^u3veV z>!@cccD}SzIsK8e^qAkvsl}R2PV`&``RZ1*Q0gglBd?sD@6MRc+>H0cW)IR-7vOEQ zX#`$N*4*i0*SUX8el!-TaFwQG_MPBl`2ptml zh>58?J*3%II=K_IRbKhYMIPg6tQ^wncIxJ{jmD}RLcV-2@|iA+<|*|P;*ftSwu0QR zY#MAA9d*yl8nc7$E5+=VG$N3Ml9jWwgXkvoyspRkxSc%XiDLgOa7?7+BRi9563rO_ zXW$-9vb+oC*zO+0C`&f&tGwEfUi4(O_=!iq7meMS(1Ab`)*aWk>&=?v$c-|fqJf1% z8CZ2TW@H~$o(yi3o%>frxH%zf3^$7Lf0%&%ZmfI4ok#j}*HT$f=i}Mtfu|f#hoN{% z@n|S(X%9s)t0>Ck6@y6{5r&hdEMF02NZ3TOHdy@1Ixh9UizkX!NDD}6Q6zazCRM;e z>TlODnJyno%z*O6%xuQKUP_ zqR}3@{kIL_FKlsQ$jA3wGwhYku_M-+6slzENhZ1CPGjB`lsx4A6T1?r!@mR?*Hg~a zuC&H`$czmrdRt5aHlO-Y3#@cUHFA&*oV5tp_C-a&HPHh-Vp)?_HUSarq9h#aB6ma|-=gET=OA8B2J~DbD zQ|24VwvHiL2XAs_Fm&V-ix;y8GrGmepYkGD(!Wh@49V33lsTd7I3_|2T!CwV-l{v! za=63GJK8|$nY#I*Jiz@f6Gv$eMX6nc$Do*8LHBrDxD1auCW^{pJc}MumXODkWsb6M zWuH`c0OIfhEqIh0rkvF8C7VC(S41m%ajRl1X)0S!%(Xfh?KrtoOtw0U_XA0!|Cowq z1kmu(;1A?(j9n$=vS3QTm4ks_Drqf>>WtH1Ris6!5%E#!-9C)iI#K8i$zL0}A&A+) zO%Kleq`$hPF|3CZ%cz#@umB|;#iZZU5N9F%AhlSCgn4ADB9mSy63`&hkud73G>s0Q ztB`IprX&MJ{VTeslcL`rqrW6=#p3u((4ONYug3H11aSr@iTf9WNwU3!v%40@lAXV@ z;gF`#Thgd!RBFc5bpC*1@ibz)B`HaiiBR|*XZjDKUze<8T;?!iWox@t4Ch|`5HXCA z@weV;M)L+GV?T6{>^vz0J@9hV8b(>1qCzeyliRI=^Gv)%1Ah)#W65$b-x!Xh* zMSnM_r6rZQG=c)JRax5tZKkN0dKNf?*5Rzm#R=Ml(`Yx&z!GY5uU@9QleD)H_XFB< z?>Z?91vx{Jw}ci}=H&5_>rfI4Sb`)&6o9lBbtzr>Ok&e<>oKTXk><d1 z%Z2!+t~hNJOGFIDSvQ1|vfYt>DVluzaPF86QZe&PZvBk6t)vyb*E5)G(^>f>uSKMZN-?7!%~s<5 z7;i;CmL%YvQ_+!}$^8|}(rUex>35- z1o?Zc5IxbFLTA&ytf|uXW{mY&%?WmIL9jAmi#U3g>jk#pJz}9(OUf&cBF2&o^susu zDJE21=bDdZICycR%*2Mur#2Kl;si=xD)Xw(*#Ft^{K-Czbul^yH(#R%KHgxpi_&>+ov+qAHitosnl zk}Vax6FbL)SUsi3PBYWwz2{nS8Z!6-N#-Lm2bi>VhQWU~5gm4mL{a@)^UZPF5D}^D zgjXrnPN~xKzeW&S9M=njexw>(AbE46A+(EmV}j4{o0kT1;&!)WJj-!w zF1b%!+d_xVem?jxZ!Yf7zr;;wesMX5bISb#$n|@GT$6WtM|dG}^b?P+Nt}sbORWDo ztv<3*KJ$@lKhulkz^EeT*mm@_3HiIgE74OqndL z>3_L__eaLD?5BQIWXf~>lvb#Htqi+^e#-mq<#R}=4BxufHc3?I4HYt;hg}qhuB_93 zSClo&nl(#0q*!=03~?zI)hI=$^MD$gCF-Hl0cHJHWBr%xJCaT546ih@Wv?MgvDQ*- z{yT*2)232h`pQdB_t(mXqfm1NZj$Rt%3|nF+S!(h;tN6_$a9>xQMU^hZjxfG`8OKS zCe!4YH5xLD*eHGFSgF4ykKEsk`~2SfkeYIwBN<8c1fiQ>P!RN)l=_l~&WSqD68gtZ z*}{%{+o*!wp77OuDDSMK!^vcqbvtqkI`wa#*V{C5fi;8KD$y%C*%9 zCk{m;SGg=^c7M@S!7hvaU5*~iMRcR28uhTSF0 zf_pgKFo>jvEN*oQB_Gkz|1ldBI~)TRsP5Y+ z*Rz?gA)Tt&h03mfW?n1G!Y4E8Uj5&!6gvkw=_s zLJHNrJ(;2Y`v@G25|w-o;_N$0jPAG*Y(Ro&EvaNYk2s6ux+8+C0R&kqON(NO^;14# ztikFbPaQdm*Oq6lNnkq)JylO>^#0u0jVJl$iAZQiL)NWl5hOkcHAE-- zv!*F_;*bC;lZfYlQkh)LG}+X4wEEV%?UP9>B5jC#*b%K!G-wQ3qc(gJP5+EQhX`W5 zl|Ct5MF*d{s*It`oq@NURpjz)wl1>nKWC|bCAvqlQ$})|_lSQWpCk3JM23SDk497$ zG2Kx;TyKkE@}7vPWc&JEd3U$+Ot8{FijMaPjN=$&dqK}Td~)$0zkY$Bql`NOAw`w_0*Za%I0Da1QVdI!OE`ey2o>4t-NKQjmKW5DUJ}zNK!t7S680Z zTD0*aVzp58bz@PnI2nMFdOd`A;%Ti%j8ZBa13W{CzRAsCL&PfP^a~^UWT!EebTFm( zJgIK~W>;U?U^F145s4l)Cy|WdBl&m1xz55N87Q_gMV2(CBKJL>&oqv7X$f(^u_dI5 z`6PHjxXu<5D1{unhjzIfxU5Bm^mQsJWbA9%;ZGzwa9~UZE&k&bS(IU)&XKdE@`<}4 z6zC5jMxQBAt1pBo_6~zm+WVC%D0QiO&FPEBUU|)#il&m}zexW0W{rB9)B$r}zNWI4 z=@=x|@sJs+B>2h7AQ!rBz{>C7#(kPnB`2jyb=%DT2%0^VJ4LT58nJ9R4^xhw4l18c z%u^-?YD(JAD`D9*aFcpyomFi4CLaZ<%>VuRipk)@YWaW6e94*5OUoxN?7)J4LslC; zkSxgd@hCFiA?KywBl1dcFHS*jt1n9P=;QJGW0f* zh=)j9EorAN3%%5*O5<6=xq_3et-)bHRF3~tOQTKvj26EjIzJL!r6Mc?>8_%Kt*vMY zsS8^NX?#nvo{yn$texXz0(`tDlH8 z=}!)M4A$f16_Bx*B-c5rtSkTYH)|OH=>%!=uDR~D#L9#NPg$G}(^S&4JTfQ;4E;9| zdxQ!Hsg|L(RuME>jaHeqMppc z@-Kffm4E;Y1o=Pf&sY^7tVc!L45t`csmG@w zZ>h>VouO1E>bVcD?lV>)-I%^t-y!70KiJMv`~!rH)pUQ4kP-{>w^h_YIHkyKM8hwP z8;S~0*2C_cXUqE6Q)uWE#jFcgrp>A93*5V)`{Z{HS6P!fjK1_y%+dW4)$tRPZchfX zUXppx8u~ShYILgYj663gH5TY!dGv{14EV6Cb3XHt+hNy`)PzrZ8OYC9V>#6g3cU>G zUqfYO?1!j6YZ2xCzXFtUX02oK$Tj4H$V4zFmDhaACrav8Y?TeHIm-I~AP*t6imKPy zj3n%^x(|7?7pXhU?Oniz(S@FtRJIVUL2!F;hEd>$goaz$7`Np{*$XQj&YZ0~YhqDV z6mi|>dJVSK<7ss#9h9x^Qw55Zm<b@e*NRjJ#*6h?1{)*KaUZfrqWC;dUmlmzQxJM1Olf}L8!>i&>nA}^Nes@jDQ za7ywXYPps)n@M9{#7tJIIubBol(U_>zthO)S_9um6uwuhztUtbH~p<$!#K*y-Gf3*_>>NCrNLjgfK%r^!_!TM}9J zh{z?63F`ZVAdWVq6;BBoh4H1FLCWDO7h*%D+kzDPR0%u5GZ;V8T^0aoq#?my9dKH^ z;JCPw;69OQZj@MQA7xe3?@y(b<-n0MhPn@@*bl@LNyxH6c$8N1d)-$#+@er21@a>c+r-lxy0#D=q%#80NH6zg+59ryl$-ardg0yWdD?ryL-* zy+h_*OlA4Z+pBD+ir;a+QL!!)RIV{Y=QhGS=c)3+=6=ucI@jDr*C;LTd6w+W9}#Af z#I{80mrTwkDcqYrzE4gS+Duk^`jq5KnVvp;jf_i5`U#KU;~k*H+dfv5t}$Ne>q{KB zuLSBm_{~cE1x>W|vD%l)OVUy$>3>f&NNW3(R4PhIzL;|qFMCYXLKiigM4h@&sHVOY z%5gS#vw3FV4;|KmTzs$A8j^W^KAB%_CbRA{cZ&E~Q9nUCV^JDGx6$OX(}W_VokW_{ z-H*7_*2)^VzbW2k*OZU^wnBmFy1Z8VDjvvxYh@BAFl6L;-;@vjKF0==^nP&f{@RhU zWgc8eg3Bd@<}+SpwKtqlNHbVbpBBmG1oD`CSGh`jkLS?Z!)7*zgHLDs=?otXVP5DEpyuy!8&UT%_z?op99B%w) z%Hf8zpaGHN4&7lQzoXal`G&Y>%^hkQm(2(QLVjt*!4sZWkIa|tB%5s6WlvnQQfvhPVypU3Qcna>{ zDwa;&d{@gqm_LzQ+LO|02ZFu#Q*=cseJQ4O8XX|0Mj6(_$_Q5m?w)eOE6#HFoC?l4 z*Nx-i0S?~epVpng2$CFEx?J0p=dPGHnSFXv2m*|-M5M0aB$-E&rtE{IjKu_PSwfI9 zls+vb$YL3HeHm`0giMWO{F>2hb+38z2$|2DbE$beA(h-&F$$%-sXXUN)eMH)R|*ti z`h=n3Ss%#_=p8A}0!Ozo&SGp>-J>JT{#qwUR$c_O_Qok!bc7@?WMob?!JGPTl+seg zv9!^(vO`p>Eu%^@QD)y^YAt;_N2-@EmCq+HLlfH$^Kes{!XGxLh&d#lh2gBbuGm0e zD#Z%XqExIbIR@8o>fOL;!jXY)wU-qnP%eNP*vXipHIBQ918>_fgphBs1>i z_kXI9o8@jSi3B;^b2zc?{#%F*3P5rhk`7I#C9Rb1IKLDpBnHQ0HBKXjgM4LFsd|6i zH_pNQ!MbkRiKQAGKaDZnfSB3q7`sxZ4OHjgH5W~aop&97!vu4gv!I{Kgw{ih@|sWn z*sd9kY~G&uzbo5l^WJqgQL=s(+ce$DA%k@o{Wb;tuPLeEj*rJD=`;H{3VIQPhY~Mg1F+L|9DAr?TMi}=EqCcnhJxQkZ z25$-8%xYa{aQ@^&u~O59+zq5b$HOz4qK`S2iek)YTcYEZ5vr`rSDP^Knp*K3RfbXB zYl{6uDOz|^b28*88I+Ts#&|}z#&Zr&r5#VuS9Sr8LvS>e`ko*2PW^>^aL5NNSL`PF zocKZ_L(YNZ+ZJrqq(_z{9oq~?t2xf5mOM`}AS4^owYKH%HtXfcyQ&7(%jB4`dSvN8 z4#5vjTToNi@y(Gm=TAq(E7hr|yah;*JJE8}n%j|fQwxNysz8j=@s^bcwNOU5Gm2Fb zClPdTwmln^Ps}o@d?n7%Ed(w;=Yyi|?-Gpk65|wHpVozlvu{IWRCW!IF4+H+fKkPW zPDAt~#Y|I0po$4Cg+U{$89!r(r;K7uSjohcCY>>Z2W*Q*hF10mR&eEQlMB_GJWSyNu7B?ru`r~HaBDce3gi>{cY%z0T^wI5Kv_OB>%yVvBI z*!DI{lfh-y0NswpHl|{Z?A}zF3()}=2%56a-i)uqKugMjej&&i?oXx`eu$JU{cez= zdTf`5%B)ic%YEdj*WP|^0u`bL533s(>%U3MQsR8F>yU& z=Av+vg9Eh76(lTlAA+z+{gAC}Bqu0`U5!1sWA`C>{>Si9S|#_wP=@tdWxe05m`Vqq zB|X;q}qIRF1~>jm10p*W<~g z%X>?@;TY2L$XZPgex+;60+8p)i1hBRSZ;AJF4)*KpJyY0(VT0gInPOR!6eO)Ihvtq zn&Cy7i!Ny{d7!yeg&zTvBl#gOCpmVK96v`+NRyL_Zl#rOwUh3NNxG+&>7L2eJzJrB{(RoZFcWqL=8_Vk5 zN~?FfsNUU6_5OKR?}35dqmFt{hUq<)Xdqag|AUd{?9eC;eVu2t30DNl}7Cz8g+PY)X~t` z&f3`C+1O#Yap$?lT_cUVr5kt8Gw!k1xJRXNuZPBcgo(4YiHoy|o3BZ~NRt71CW9(X z20t_z^4?^aq3MWDrXz=&j`B4f9cel?&(yopbo@ipiK4!*b$x&5`jdU@Pl>EQEwBEJ z%K9@O)}JjJ%(ZSX&$+<@-v%L(4Z`voL{v6d{IJ1NVHRm^7VT^n>uVMtX_lC0mRxDJ z{GnN@Xqax@aFuhz4Bv)pBO7MrHO#4Oxc*_ojlw+7+C1OcyujCdOQiX>Jo6ou=DQx6 z?-3Sztu6LhTkLnXIN)nh5@~TL&*E^U#gT^=$Ao32wPm%lh?}Y;-HH(Vfah_Z~L7FB(5EZ2YKG<0r!#Kbzb5MS9~`dmF#G z-1xuujo%qId2ik1qjQtbzD>SHHu;|3=HEo-eCf3UVPYTDGUY4Z_HTLw3^UD>qt zzNT$!o3{VZw4+foJG*8MBbs#%Zq{{Wv+nzv^{j2y`$ID)qvn0>n!Ao@?jGEn-<%q_ zuenEU^C2Ia4>M{p!mh>05iLdqw-~ds#khSfd}>=v_|RgKQA~t#XZ8Z?bE>c|_~N;MQAL zw%)$4_0HPXyFaupGHSEWu1)cXHV1>-l&)-3wy#ZjZJVPX+Ef^|t+H!-oUe-yZhLxV z+nRlC&(*fQ@PQu+Xjg03?&^qk*Mr;LT-omSzIJzO+x_#Q-F?;e51Y1s+_(KxzxL0g z+P~b?{&iLRH&yNbeboNH5AEOn&wtep@0)h`*tf%HzYbrcI(*;M;b&Ec-?bh7eCVKJ z+)=ezM>Us@l4nPC|Bjl`9kud1>Qr~E_qe0JS|>x}PDb{fOguZ)pV!H3RVVZPoh+|( zYW%U2m9bq@d%NbIb}jwwY@+R2<=eHXwrls;uER&Wj>h(O_Vx~*_KyDcU83!~<=gkD zw(ohxzV}CaCu4`c_71L|4({_D`mb^rknb?4+F|fxhoNeY!<#vJx;T3IJC2Ta9Gmax zUF|sjvExLw&c4k$``LF6@a!Du-+4-O=V|$!XH<8d`MC3JwJvj;b(!bVWr2T}kmxR9 z`CTHayDWa(WvN=%$Yx!mUAo5lca4wknwZ};xw`A}$6Zs^x}`Vkw#ubjhJUxU(cQB0 zyX91OTmQJ*Mz!vF&AR8ibT9Dlz9qW*(rx+OcT{)Z^|<>UwH|w$_1N#y+{j2&u9NW zU!(hc&+qfIy3g;&eg3LBiRMmfu1@L!PMR@JTAQ78jyu(R;-oJ*8#Z?~c6BxlaIQb! z*=)75d9kzQRp-W^oUKgyHg)LRd}QC2^ZVMa?%TS!Z`-SV+kfiY(Zt2h!Np;ui(`OG zml&6>t6jPmyY#&3()*K(lZk6z2UpjTuI}?)`>%E#SnTR?)pf`x*I_1ZBOKgDj&vI} z-)+olw{gX8K3Cl)d~%y);_m0*9x&29aK8K0)$Y@a-DezkpZUanw$yKK^M3PO`z;9Q z7ZTGiY;(Ve9Q$_jh_bOnhf0JFmUt8frax2Ze2Zad-1@XR|oF?G_c5I&_0Jj z#UlqDoIj{^^`NriLFHEm9eFb7*r!1iCLUD|9>+&|oCxqZ72|Pcvqw#_$GNK>=bw07 zlm=gJKKP33;A;VcZ^R6~wR!OE&4ce8AAIl0;D0_1eqb`>k;9NDBZoYjKjg*gA+L&u zytz8$zfVKnnGF5lF!a;NplH#>6cclaMhc z>A;w!Dc8oV_&g@fbZmNyv8&w1W=tNtHg;@Q!PuM=W7j_&yHR~yUW;-0ZsQ6jkJ}PE zZd<{)9Vf=^dOB{8y7%4|-uvCW4|sVWT;N@r;azsXyZoB>(a+u$rao1UKGkkMCno!x ziuE~D;B)qb&-tf57uCmKZZZCf+xTmf$KQw@f2&~pofG5lJsp2veZs>Q6CS%wcshB) z^VkV53nsiiG2zX%3IBbb@XmDN2giw@ye58GF!5W)#2*JH{<=2t_tS}g)hCJilT=$y zQgffA9ym!eZjx5vB%PC!>OGsJuinmj3qc{*HnEUE=(^75eu$>EG*_e;`@6*zfx+~l!^lf6$)9{+6eM2$e-mVy56 zfs+FRr^E$LD-4`*GH~Xzz}Xs8=C+(N&wa{*z$qbdQ^E?TM4X(m_}P@D8dD=%PL1k3 zHD=V**ubgraZ?iurzW4Ay8PMH6Jx8zNz!{{QvpyKD{7tdf~$9Th~nAesKEE>(h6CnO;} z@W~lRp3OL>5mebSsMIvB5mM zF7q5l&+8mAuj|@*-Am^6yfLr$*Li(3=R4cXcj-6ZZOZ(9@$(04nLp^%{K3!X57k^S z+-8AizXe`X7L1NxFm}rV?^6rLKVL9WbK#^03;nt*TpBQXVPMF@scRQbFIgCLW8ti? z3+FTl3APED*Dqwjlo0-ft_=w<30ZU_WXbc8rJA9UHlfk|LSv_d#)pK)$A>0v2~8>q zO}P=e;%jJHgRqrd!d8zCTN4tNxi&1jBy8P{uno_{Hfo0F*@Wl!3ojTQUKkR-b#3_e zlJK23!gqfSFKQ66uS-Pn=!k2Z$upZ8d1?;QB{{k$44(Z8M5f~+C?=b zi_YCxbm8lwOAQv+c3FIN^y2Fwi*K%7e7j`v-5ZPl`MUT)gC&o;EO|0|$+M6pFV-%3 zRkGyGjV1p*U-DLS>3f@{ANwu+JZ0(E_@&>sEd68~40|9oA_H$kg*T_%lLrV+YK z&RnKlx=i=xGQDrh49p^px<;CeiL4(QX_gsjUK(k6GqUlwNGr3brd^|&kBMp-8fBXq z)w(pQ?aiq6-=aF2McZ|ab{G@gIW)R!W_0(`=$<#Ddw+{|GK=ZkHO6&JjC*KIzl4|p zTVn>Djv4$SW~dxH+&0#;f9%LHv7?E@|zpilsW8wlsF+t#EV zrAa$)Chh)~RAiRCuWNGgnB;?@$)%aeWu?jGHj6uD1pZQvb?nqE(vOfHd{7X&PZ^n*3ef zwltkHY4u*F>1(AMwn{f1kZw9Hy+LAn!)@snXVM$JOmCvK(z?~kW&>8X7`w7%*h<^1 zm8}o0YoaPDR6Y zRo&JdAG_{k*t*kM>uL_IJ9lf{h41SwHC$iYZT;1;>#v8cznQiE_M!E6&aA(CYyCej z*WcIL@UYc}#{)Jzownh5;)a*oHoQKx;mxfL|9#)^uHnWH-8Oz2yYWld#&200e;nHQ z>(<6U-#4n5=c;zkmB!_2gy+iHx!Ps9y0>%ne&iaM=NWa+GZ~jxKRnMYJI}l<&+>L& z;~#le=9`*!-_&g2rWVsT*(7ahwS7~YnoaFqZR((%->G%J{lI+3>G@rf^1E%%?@^QA z>s5Xq?aj`uH@ghn>^6OKzwpidlQs|7zIjm1W)Do0+na~{*gVXs4O@Yttg7L2kCj2OvWM1gkz0iMPVZgY;$AqTmNd? zM(yo+t+(e7++Hw!`<8IBZQr(i`;MCJyIyVI^J9CF`Hp?vcN7oYabWt6lJFg+**nV0 zb{wwRapcvGW7<0_TkotMxbwvHou`s^p4q&&LuKU`%ADZue)Oz>hfxDkh-~Bvk_si|OU)SvZ_toyV+I!xa@A=Su&!=&F zzJ%}jmc8dk*`8mw_xygf=dX5=Xj7y%s7QTAk!EtyQaPtc`*4x&og%%TMFtjoje6`g z@!nfMVy{`wUh~6yE${4Y{By6B#lEIJ_BHq3*D_+CZO*>dhxfI;v#Ia}JjsK3sn1@X?=#D=f;ZdXyjcEtKK6abv7cv;{eFGyuTF(%TcPGrp&nGBv8X~`SD{^Ap?kMN z?^lI^Wu;NiN)w;T`im;f)>WF9S6be!Z2XH8qblpRRn0uAS_D$>Xh<<&jTRrh*R-ADJhbKB!C9>?8+j`vGBK49nZLFbMS zesg@N?up@TPk4Hq@CrIHYSD=?>rRX-KjCxt#DrfbCRv{J>v=N3=j7y|lYxs)PF;6$ zdilwqyC-M;IyuMkRB+Ez^L5B>)UC;DTa)imQxH_MC8cKD z&YJDzH9POt?EY0#WO;U9&$GooXAdqqTe|LSS^3%WyJwI7I$L3RuBzv`<38t3E;@I5 z-MO0bbLZ}!yYTDWCCl@*J_( z#kvcx$}haRd*Q!d7aG2^y!fH##ZNvLzbv}=ZQaEmvXR2Fouu+;!RF{N+afUT#va*1BD7v%$43 zX4cv)uWhxfw$1t4cK_D4|6SX$(G|O1R~*J)>Ad(#*Y#JrAGy-=-j&|JuQ)Zj+PBwL z*YQ`~7hmna{wjy4{A-!E(#O(NYUpU7qS4SyXL6|jRNq-sCQqFIzu(^c&tJ9rRZmr0 z{&(2B<%OMN`8}Ke->0?z_iuB4BuuwI5+2GIlJiRdDrs{+rFPA-juaCeBgG`}1wZfy z&mNJYt4E~h27gCIiimlU;>hAiaTHvmB85AQTpKC8fFGX^=kFS-?A#YAcEN7g14XbG?(dHjR^^eR85rM<6ix0$3jJru03$E}W9V_;P^J6x zNb$=wO8kaD@E2+x7^<9w^KcGS>PHCys-OlJ2)hUpes$0QO^`tgv_S`S;S%ZUan%ET zFaSd^0%I_R`p^JcL;nU*A_UgM5x8j?C7wdtMp2?Y^o6R%QQ{1og==sfa+^enP4K#T zl!&&C5;1Vqo_P56&``z7Axc<7Q)mWZ5CP$^2o^&h|0v-E&d?WJz#aNQe;5GK5D%@V zMv0;YQQ|b5fk6wS#KKg{4q*@uUsh7)XND>rKw3pxfd*)T3>^vA=BfocP!DuLAN0Tg z48aJD!30d9J~RL`Xb9$D0hZ7R8bb%@2%W$V?7;yXp)34ZLtlgTdY%osP!IG#9}K`0 z%%CAyt^ zVllL8A0xhZM?Q3MjS(+AV#G@b7#}0-C&h@JK{4X-tQZj(93yo88uAk%F+vs8KmuLD zWUgAEt=xkFgfEN{VX!PCMnpk0yw8adEAwN-xa~2*8+>3qOaQ$dF+v{MQe1i~hQ6aGnDeZdd>!I`kW z-~s{M2f}1<<$em+sW1(u!wi@SLEuKbJM@QsFc1d7Aecq`Y?uRcAsFVtd{_VrAp}Cf zgY;os2XhVQ8Uc%7F+@WY#6dhHKq4eTG7KTla;_<`0)}#*$~6trVI{1B)sO*eU@c@q z7Gy&Xtb_Hi0X9M|eSMVBw{$VV@Oqd0; zq3AJf^oBY0H%7eGG*a1T9xL`kF&u;vCaGMY-LLdxE;4IvP&)_|gI7oySVB;Gr+WE(dis{IJ zN*FskRz!xyiUSF;;^>N4aSSS;68fYfYjdpdI~FVMoQxHB;i4&0!%5KmL3=}=pRvLTvLGAE;Rv+;6)W06 ztKazHGSosRy*QE6DNeNN8Hb%>q;i>SEnHFP6(?@NZMXxKZgHXts^K`CfRk_vPQxwt zIMLoGPBiq76Xsw6gXhMH{qy3)x`;UOCn`>;M#l*?kU#^rB*%%ZFlBk1C`gMFOS9rc zf`yUFMy|P#2e;P6iKp-kUciOTapDqOhM=lAF%#xOFwBGZXX3;s2)uw?c-_cIWh&P< zT&HoJ4l`gT1i>7b3&AiC=EDLAg?LDSYPbcr;SSt|>sR8$4Y&<=;P;I%PZ{^{^S7e#Z%C=nF3J z2mZqQKXKwC`2UR)0k8o!LOqpuQJ@hodRiH&m>R{424Dsa;R!s2XOPo2UgSZicJaav z?7;zw?c>EGnB6&EIJ?CQ7jOkPaEE>{0LBlE7Zc#@#CUOJQoLv$7%z4$ju#)7#*0tz z8NR?*_y*s>Xj!~4j*J&3px@F+WqWG8*a16X7igu$3vJMWFS+r;v@Bk%ITbHv{fZZJ zVLmK?g%AcsdI@4L)WQ`o(N7R3nkERTeS&c6ogkbcr@fKNI#>@$E(u}-*Nw0Oe=gSy zu4^C<|5~n_xMp(Af^5j=elx7YzaDa61LVR+C?Kv7w!k*n4qG9gxXnsD6v7tR3frIp z&cJ231~=h8ynvVR3SPro_zYj*D}00R@B@CrFZd0A;4f&nCJ0T~>y{w)K^Yu|av0w) zL9FbbAXdR@$bc+Z2kT)2Y=m6M1LF=xDkjitNP>tNo*-hOd`yC9H7-Hi^+^!sQxb#) zSVALc3{AiaVx}jEc0mck5xn_HA?KM1qA$3D3%G$d_`rCW0Q+GdY$E-+g$d$3T!c$- z8D59dF7O8ah5w)`HbIow8L2GIP7uo=5~3g)2Iey^;3`~!YcOOpW#2;Cw^DY{E1^Hi z6U27d0XyMAbf1!v$a47r>ru7X`{qBsF9t|W?< zU<0<$3WkFxjD!H#1ck5#7F4nehJn_7`UK8A;`%I6oPM4te!fZ+zhL*b zL?Qi36b--(%%LIpIUA{1aJ7U+;Lm+yu1&xStf471gBH*dY`_*;L2GCO0px4TwH>sF z4$u+o!2ukh3v`8E&>Q-|Ko|rbFc^lwP#6Zop%_Zx5R}1TxD9vUF5H8OB1uewcsWTV zzz~BZVQ-iuLXDC{lln=b88kKL3O3NhGD&3HB#DphlEi1o>z*X?VKaF2N)pCCNkVN_ zlF(m_9|BX8#JjCYqFZs2&^??a>Om~TLDKakapYN&(9=y8O`9Z(W{}{TELOmB2pU0q z_D>c~zzVFPDKvxT&;nY54WtiD7As-&pky%y#=)r%-lno(d+}c_ToC5>m2BfzYomC z9}M$gJ}iI`SO}pI1`)6boJj8si}5dkzWA4Mjf5zOh8T#2IEaUA$N{gSWHAax!x$I~ z-e7f%_JU^69R5M(J!pZyCD?#1w1U>q2HHYSVALc3{Aiatf3WjhaND(Iz=?FO%W41rHC8dQ-oLV6fp`cT~ow$w-hnc zBSp-D`9o8L!$cz$jbSN522IccZO{Q-XaPyXQ$#W}pOqqPpe5KsD`*XEpi@kWumgK= z07q~GckoL{5z`?EX2L9(2lHV8EQAmUh3&8l&cjW(1-Ic>dW!fBf8a0lUY8VqY3UOoB z3UL#{;#UZx%oSqILL-&5ycHrHR)YPu6~X}=;T;Uvzd{^8fXpjA59}fAo-aS;Dy+{?VX{U%(jrV__Wlzyz2GlfW1J zU^>i#*)Rt-*rke%kP9!nq>5MY8s5Oaklr;_tb{l4FHGx>4447eom0h9w^VW3Jyk5} z4`aB8We^EFC!~s^rPw=uslo(IVds=ou?uXLrHWS21{x%$3XAkqF|d%jmZXY@@kT23 zzNQL2&~4K!VCUMGEyn(m?rLX-OF`96vII{437wZ4CVNbK?PL7aj1q9 za1x%7<{3PNGu)qs8aNB*;5=M_i*N}pLoHl^t8fjj!)>?&_uwD64-eoGJceiR9A3aH zcnN<&rBj*^UgxQ4%u?pWffZp*pcymE|!z!OHoC>Z_!uy^NiQCw@@!0Yrz(KJ{z z4K|2a4Tue*Hi!y}rfCoX(Xg}x5tU6vMFm7cTw()_B0*#q5Rg@I4Ja;gML=9)6xZMy zx46Y5?pxIR{+gIOxp#6WGw(Zh=KbUP$v3C=I#qS5>h!7NDF}tB5CzdN9cIETm<@3- z2j;>&NPt9`56SQ;EQB=J3;WQ;|kMBt$_p#K1I|4l`gT#KLTdgE=r4 z=0Q9pKqAbCBv=5SLNY9drH~F^!9F+$Ezk;w;1N89S8)Cplu5V*m!ait9w|URUm|sT zhjI@!@AF8ljwGMd!F_2y`2ilnBX|r?;3+(V=kNkvLV#sH34|aB264Z9V%$HUR5|9O zjW3^E0J8!4q$fzh94w$0SOE*~;V&@NDWCLr&LA`j3xDxai(luye1^GRYZ@=}4k zz%zIOFX0uuhBxpFyo2}f7Z_WWPr_g#OoDKj3{&8n2DJ6SRCGr`Bt$_poW$^HNMm3+ z%z#*!3A11}#K9bx3-cfz5+D)gLlP{2WcU7=w!wC2 zg=g>_UcgIu1+U=^ya%cp+l-VTgxP}Q1&^BY zN##Bq_XGLlCn&s>kKgn8WcYXaWIr4LyG>&1Akr4N1ffs!N&d5Za~e3AedG_^uS_P0qF&fFaSP*5s3xlo5cm>415m{;JpU(E-xU5 z3JVCQ0@GC%kSQ<~A|MK)AqHY$Cd`I7m z6p%sB3&@SX6c7Wdkc<`-5^vxM3yFbwA?YVCB<2o`c1RDy;hRs;lmO?V_%RRMi%dRbLg6h6_dH8Yw0> zVALr*PqQl~`A`7q_QgaC86a^fCha-JWYbwORasU{)&HMs=* zt*eP8SV14y*0Gvw2kwQ{ga>?ZgaI%R27v-bfD(K`4U@pNYc&}KAAu5FFRmuTVFb9t zD9F9Mn&d+P6haY{Kq-{L8YqYHx5U&U$gwRY05rP4Tzz~eUR8m6B zpeLLdQbMMUDj}sGm5|=b5@H3zy-LXHNhM^)eKD0DUP82x0n1=H+>Iz9_uxJ}fS)m4 z=*$u_5yD_iTnQk_7Xgv*4*hT8FF>s+Ap|(U13n0V1|b+i z6y}MB9_VJk7{p)#5-^3HAO%Zcp)c5h9PGgX`ojQFfD;S`XK;a`Fbv#bBq+fT#)3ag zfDo7ovmp`YLlPuI3Va5e%1g)=*a}~Qu%d*_t1TgikpFn33DAn}HfV>#&;f}UJ|B`` z0elK6un>-5{89KE-9N!Acn=rquz#To`qX2)K?c@f1Gdl??4TdW!5$o-KRChw7znM< z24o$M1$>9~O+p$DlVJ+nhCA>*+=U4ZCFI&hB80&t2#1odupB6ZHBb&c z-l2Wdpp1mVL|%E$pY2)A6z$oFs? z?tsDEG9m&)SRG$RN?_3ZGNOQ+$z_CDScZ0>GD3Zh^_60M5M76Px0R8^?Pa9+Vj0=} z9oBcFj6^~)Z9--9){vEu1FIkx@*p3YU>ooRYX~0%pa<*3Ysh-&9KD8I5}8mp;j`Im z$U;~IHgnbxTj&e1YuAvw&1=ZKLu-fzyciQ|G%Px@hAf69V03H^F$7Z>dwdO<*|~;H zdx3Sn!#crOVnWRpmXkP`19PDZF2W^MX2Cs7dmrM^Jsal01N8p@^UysP;voU%Ln1uHxFn=MBVB+r89s&2 zAO#k|LU@GnOOP&xrSLguAPrLCF~+AOeS%br^eNJ3kb&-HupFMFKNIN-q$`kS!55GX zFEQ*DyoQzN&w*9&2K{g07sy3_KIB0G6v8_UFGBhrX))5(KsA>W0)N4<67-h>7yTUI z!5Z|JK{-@FC9H)is0Kc!sX4zb81veND?l2NY z!AGD34^V+8c)@ zL(x49mO~~yfu|6&vmEb=mXj0k4UE`@eFY<76zqfjV7t4V^o1Y@1~q*A70z{tf@qiq zb73BQ1$$t`9xM+=0{I$c8lHcHV{;m1`$jq8g8;<0%ZUjH?x9RV!hLKr+y|ovxOM>p zIX{<^g)hp3P2gbs9_!t5p7}O8~ zVXzWvpdQvk&%PBz3g%!1hwZRFI1XRK32^FHK_anjRYVSPgbv3Fas-aTF*pwE2UL&-*Z>=06KsYhBd~m^gL=5*j^U#!$RK3}dF4|< zUPCW6_VFYvI~>E{HTX=aAl7p#a38E7KYv<5CZ%Edx!7N&*ca<7i0aD<;t3;nSCDDY z(o#W|wO5ekkO{9lE66VoE67{0__>0VKdT@+VHezcUP0!*tRPQGC6P!f$#J%ljB&Jj(;1IOIG1%{k;cx)5qAJN3kPRy#2UbDZj7qWw z%Ao>mQ!9xB%>1H~+ynQum1Gor1S;@^`YPlLI`>wR!o!thH59}1+m+-6_>#5655|H& zjDx$w*OE_t){+z`AHSAl#;zqjm;lM2_8 zaF`6D;DXdtc5D@iLE9T^Dzt_FRCV2wAI9Pk_okYMKv))x*zEQI0!9p2-=`0 z#|O;q!?xn$I<;YoPd+i38&#KoPqOj4lcmAa1n07ZQx{86DL>-8aN75zo;g$ z;EZL@L^=!NU=GZMc@Pg{S5^~oPBk%siFwr|B)=MeyHZWuU^t8bU+{zR@G%5HAOt}$ zd;*~m29qFtbv5ZKsV3zW)#Of1HMt8>>#B)kxCzzKfbE8(a2Bq^O}GUH8>)#241s~Z zActWaktcA2s!i3T8fu^x{?c4c-UGFxnh@Xs-C9k4If?yuwwm zU{J#Z_yi(=GN>U0#G)GfZmS^-7=xx~4M_zpY_O~$8(|Y{1~ayXoElI=Y=_hkJMi|b zA!Fe5_!=_p;~EkRGvP^C4S5RB;RU>eSD>1Vbx*A!D`(e`qD3_%A+3g3y{aL7;F&=! zc@BQ&wM1rBOXj%Ll6m0eR!iP|R7=+Q)sn3r*OEsewd7!EElCTnCB36+iE|9*n_Ei~ z7u1p;lWWPDlv?s`Q7zipYtbHFi}vqYv~kxGEfhm3l);heT5=o?*42_9l1-?l##+*L ztd_k0rj~?#WNB*amnEf+w{^58~g}lJmSe61o_5TEjZB2CAV3+D+=nVK@&Lz@kqb@$Fkj zzFvZQ=b$>W75)P6p=d}QDT6f-F|>|E!c3S2Kl{~@*x))c4-#NLY=Dih32uN?T}J|- zmj=IEkluxRa36d_>WCj4gckUND0M?AIZroA%G3QZA^!c}MxwLqBvy!p&v2|29NPen zT|7sg%W-JrICgRdKH~fufX*d=J2al_n#&#D$aU}Jj(WsZGCY-n=M}*7j_3L0^86Zk z{++z>k9YwLKS;q>2k=AU`JuV|u-`(@MKFRWg&-zCFg;!nn=6>rD2VG6%zY$4q>n^} zUQ&Qw^6%r$J<>zaxikf>4WO6(X~els{mMuBRgADkA*>4!u8S8o}a! zG6vfe2F(EmJL3&@=Njy3G}zZ^aNv0xuWBZq7$8>laEB5jNuuD z;kf|A3-N|s|HW`~uX205Y3%W~v&Z{KJt$*_fBJ6&$|abF=9z{ynTDS>o$}Z;!q_az$t)(&Y#Q%uDjj%bLu~Pn%ahHm@?asByBW3$$35V9}6gv9ZZw^J$B%k1ZOFdu?;- z)g0JsXF{*tdA;^D_1bs3*MY~qT8u4Qoh;h}Ejtn{kLFn(Z?Zgb+VbRM%TD9oXPkPU z3+#O%p?6na?@LX+ubl3E?Q!q_Ad=h@r#{aD`@Bf#v-DMW9J#lr`@DbLhZ3_KXOw3Fa&40)=q9 zLUdMP^i;w8X{5OM&hi3hhwaXeXPpQBE==6e`GZ{x1`pr<$1&pkw!8SBbs7KECBS4z z(BL8J;2|ONhlCak3EMs-{Oph^PlrU9{1zZ?{?OQhp|iFRjXOJZ?$e?1Cc_d34@(Lj zmOOu0O2M#&+lMVaJM8nP!!#zYX@gz0!LG~ZyJi-+W^H%PKI@wE)HT<{Eq}0EVX#~A ze7BMUx3cYSV$b#w`_7Iy@N`6riF@l{_xArFINbb^T?HdAZ6A5%?8s|RM_xA>b#w5j+rgvm z%pY~HVAO-{qaL0e_0!W)OCOnh^kncy&w@XCG5@1i1s}cH{?XgBAH9G25hYP_T$DVu zQjnyi3zfoVrRbc}=$Vp{c!*s*BZhSr0_#f zxXS-xM7RUL3kTPczyCrm6}FtMa)V%d&~<>x0>KA%`+8dftT ztZqWsx&>hkMPVCvgl#?_w)J^fqv@n=Lnbx~CYf?Toa%5NY)yk~NF69vWpE5@nYhB`=P0 z*cs(`A!^`@D1}+H^U!FQkm#Yw(XPeO!*@ozUx*&{B3fw{qZ%6H6%yl}9OF|QC=;^#}-eYwR3vh zh3Ru&OpiC4kvMclQpk+t0bMF<;eXw)x!wYkNdNKEr z*}Nx1=RFIV_ab@TtKxZYcFueIAIEmI94Kz;0GA6WE@=i|V#I z(P3Aj~QWlK;0!<6kBP^jr`$Y=Qcd1tFg< z2wlA(Y}bPDZx>8?x!~7$ZhsukZP=$tpM0A9`&e!-Kh^Y1NgI};{Ul}Crzx4M|165z z%g?HMF02{0uC{9^XQnI3K_loka6_u}6R7taHT(jyz zv(}|#HI!s+{AYx2bIrLOnsX;5=Uz$9gWWj~yK;VdmGelt>WS;BXQ8WJq^x>Xvg*z5 zRd2giy??ceGSB6>p-G7ym)Dwu`Gjf4y2`UXtckqMcZ>>~BN0<(ik~yOkDBEG_=5w4}5Y4`RzN zmR7zltuimGaVx8vShnu7vWEYrIBiRR4b-NK()POIx_RYIx60cSEAM<(d9Sqc!9OQN zn{8N)-NG7qS&hS<8plgD1K-prENY#H*Sdt&4qaI5T2?!JPp$i<+EH(6l@@iX;dNeN zb>4{2R#xY?M;Dy!Lu9t$y0B~^3+qF>W3nM2+h3!xAr#wm#9=G@kAko*zNCx5hVW}2 zVy`W1$SmuQx^}4{=S{<}A=e7K5*-&hy6aUFuKm2KR( zXXEBe8@K*7(puQ2oxg=y+jnWxfj664EH<|e-`pOyxntqxqh*_q@7a9f(&m$IHg{TV zIWv6Axv(u47H;V(+j42omMfRGTzj+SI%2F1-+DW2>z##L@0D$RuxIPTOIv?>v-J-H zti6BpCDp5uGoq0(?()vLzwC_fwJUMNuB1u3 zk{9htS+i^5-d&3?|8GWCTZ+(Xe-l@2@BS;7_h0*E|Mgx6ZjLx``|pCP{c_+@uY*rU z9DFwE;EP2EU#&U#X79nbmk+-Gq1)lQ>#olNi3;*qB%lTVv1IW4U? zZL$Bf<<-+x?@qJ5&sdK|hb-I6o%iZc%T&p2NFFGNuD+kejg>bddn&IR;7A2jm3 zdh+>@CFesc&WG(kAAa@xly~PNdS8edc_C)V2tZx-y$yYj-Troddapm~_D<`gAIr;8NXYZ?L zMqWKP`RauwSGy{%UfO^4%GIma-d(-k``XQs*KSX~c4x`8dllCn?7#N#>b0NVU3=8~ zyC);Rdp7yI7fZf-Rq@@M{olR4`rZ3?-%(cAIis%grd$_%ex0toEHE7fy6@O5#;GX>}ge4DAD7M59aIfd`I*3r~rK<`eU^x=}(Uv zJ&r$Ggt@ z&z&Ie+<@RvDHbp~6cb-Ees+=m@I@2;QkGWun0xY68n*~WMls82k4_J*Z6eS3u7>c; zqeaf%_8j>RmAqaQLfOn|s1IXp=}zIyyNcwK)-jHs(9zl^6_1LhiH0%IZ_09AUlc?I zjXmPK6aEsd* zo=i(CT}X@c=NAQcGNmZiDTJbU zR6IiUQA;C@j|gotoT{-3%f3{AOmg;;R7)u=`^PClGNsnE>?jje=4>QE+QJ}?Mt8_4N<&?D!GiFw#qB3Wh_wH0T9(83F|sfNt?dwrRFC#(ml`#;O8avu zRSva0*Qs5qWe$mCH9LZQCJrd9F0>eJUtbuME_&25`$x<9m;ou(HV`cbpO6?A#{I@BBX`U@Hi?eO~bS@dgY!{ zS$(6YicW6|qKqR7cc|<=7!R#HgBRu9OGP^sGc}?oR@7`-mPJ?RGDi9~`Z9kShmU8^ z)t<$X&uhIlVn(^mQT<5PI+Imd%CyvVNlZF^>lA@Uh`)#z0ik78`Wvt9g zzlQb@P#h|rcGEBcO5cdwK#J01tm)`U708TXHBX zz|YbqM_#qk>71LEuC|D6#pQC=xbiABY0W-!GBBEEwbaiQFHbT`I!Mc;Imxu~22vX= zP39Ef;k8ds=}A{Vtu(%yefBuZ8DPtkVb5u$;~81|EQ)fvHsY?OqGRIh?7=+h7A@m> z_^qhB;5LRL9VtIEp1RALi$KhE2~fJ;V@dW*ZBYo*>lPi9Mny5!+0xpgK;6dU{JT|2a_AloC8iP~(E##nC_r@5|Qi2@u= z1#&1ZEiPk_p$=nR{#-S59>pL-TGttwo-M8%+PZ_;OxscB*+pDTLgAEHp2?-CxDlMX4w!Cc56yx{#SAlAR>J*$K-gZEkE0wa+o_2n`&Q>C~=F7X>MapZ)m|gVEKs zCtbNNE+tMGJ2)+&1L%ZI4qrb}8adlIsDt&A+MJv@BirUAiOQDosEFnl*Nmx5@66~r zMQf<}>}n^ve>T%0H1}NTq>_3T9TEm-7s;rUc`k}}sgI^jIJjQakt>yH#mA+7MJD)` zRACTJR47N%Y>BtDzZc~su-S|SFq+0zoUctmeu1M=?qxg;kCJg1RcIs!V`WF(LZs1r z&3xQCoI>&`866TDv03ZUfva+mG`h~Ud+#Z`PoZ|Hn7dS!?bLyrfQOdVGW}?aSgA@7 zID<-Tm(9v{>Y}2vWk+?l2(2_OTXvk?v}4DmFs&%^gj-C56V8k58UZdWH>S9H6e)5% zqswhh7{|JNdQ$0=fg{G(IG>nVpUF)Vj&{>q)+;)jER!~rb56Lecfo_&XiCu$`A~OU zIGXcRyixbHT6g)UG@KSzqX`h0&t6${lupf+bx7-ZlH*b@d$|`KU6s%g$jcmK6_KVd zYiBf!HIF$$%e?LDci3sE8l0Z-N)j)iqK&Dea=r5E7S{+(P^Y+_VpPU`H6~`plx3C5 zwQqT#pAThuH%mT)`BZ9?ooDTxkj8M7^3=T64*Ab3=bx0}K78D@ugK$MKvgzZBh_Ti z_co7aR04bN!jLlg*xdRo=@9FcFDZ6LxwFum5#N+XYMoR**0(K{`1+ZKp%^bIM@!eP zbi%S4=WCr}ToNhc?3Ds<5}loBCR1fQ7gMb?RmO~MXyWOo(ecvA(HduA}mrDQb11-)ePpA`@p%OZv4| zx6#h5(9Z6}sVR2*zL}z{)BUbq^oLbCS5tR?Q(kw4&N5C{(TYx(KQ(@4TwG*gcVE(P zeW-R73QQ{ne_QPmM*npJsqo)s#d5N^&sLBRbugl!xI|3x54DNW;S2*QZhAPyO>jpy zM#`KiQ?k2ol=}_(6}L)CagDt(!iVA$jTOb^yHk3E(HoCJE`ZVaJTR9G1tr*0`nx3D zkFYqd1amOVuhS6SvZqk|4V)0(e(+-c}Gv8MF6N+nieTuv}}Bf%1s z4ixtrKT4ml%0pO~-bvjUYcQh>E=oSvbguy;xN(@cItU{Yu{l|~skFEtBHwz1!_Gxl z2)cOjW|%aH(tE8Ta?B8cr6cYkm-wScV}TB5z~12^niqyJRR71%U$1@o!_Qxauevom zduhLnpvrrx&1T3@%XQsm2@-qhYVHXO z<|R$jbtEN5#!UO;a>MT)C*^lf`>)dS|6OS*%$WLbR~XTpY4hht>H?U57C-IR>r9oN z*;YNfjqj(}-FeqtWd8Pm{$W9B)HD5w?{AvieX;4{va4F3%fk<Ub*)yf3S zR*U}KR+tmxeQqq2Y{m3trTBNu&_{7xt$ZQJ%vfgIbD}R&E5A8GNUU0^n=$?ywNf|3 zm}6??=hM|%iV#q3yi>*7O zmR`gf7U{Be+%T5iuU0<9hGeuEO7R`vDv(lp4I9`UYUPo!!&Z3gHY}mO8o*N92k1Vf zoq}$fY98Q2Z8wzCO==~-ifT4g>bm;Msm1{mZ<|`bi?>T{A1}fZrXzF!o?a7L`fl_>9az(X#S(NS#o2)4LvPh{|FjjQdW#vZtxC4$mSaxaoG_K*vUMBw;3Fm59QSV9yl3l{ zU&=P_-TBj=t#dB#oO2of$GCOzj%^Bq^)t(kD@2=~^v9PVP9Yl#(Gi7V+cVdkTK7|f z4Nnci5cji5BbQehVx!iwnOAFvEyPEqDhJt1$ewDqd)ZWcUg1Aq!2j^VrQw-o^88ITc!v?O zku;n~f6+%KbZt|5fOCP;WlC3FLKT?O{VsimfiXy@!4%Nl4Tk{@6QDO30)9AA_|qjy z1m>W73$h1{fG98kX9xrQ)};o*P#6Ns0NJ5*4PK+c0VIH5^VE7|?gUaR=mY)08U{m8 zD1d0N1zq)h9E=ASXopGA3&z67;0XO;4tRs^rYs><|;0qMFuH&CDUh{4F65n>GK>t>e z=QP8f(+w|H_2mV6O%n4whsJw$8n1YndP6}41lp!=TVYdLSmUcOR(NSk0*;m19_uS1 z(IU?X+lFHK%|o`n;ySx(yZ(!ItX^bztT@HjEACjaZQCP(r_!#SX^C)Gv{fBDaozQ> zLNs`YDBVfqd(*YeV?<%_<;y>j^XC3qw|ycodnT}Zvtg>;nGL^|2HipY!&13MiOI3R3E!5 ztrNN{$n^GLB^kslY({mL$f!m`SJWp%QEya;GJ=)iB$L`~$T(uwU#qLS`d_5{z=t~Z zcGL{}@xl)p##1YeW9jbNBR*B%z@pOJU0L3R>SqFoyJF#-NTHMDFtr zsr4JFcEc{}up!OXf2di1H9$(W8Y-wJ)MMKWGj&+hIJ@h-sOQpK)JoYPo^Thcy_-<2 z-Ds%6qybd5;dN@QVLWex+CG3+ueLA2Ms!i@2S|Ba)%I1qu3-BXY>Sfm61iQEMb{#4 zYYcOF6{z`An7BgHSp#1zaNZITg-nue+l=!nD9rri_zf>0||F=IN|2F zqgz3^*PMyz`!TZ*wOPH?dplK1xIcMggb(4rUus3TZ`_IA`=4Kp#~>HL=r?ktLqQ3) zMBnF!w=G!QTM6d)nfY~^_qt_IA^ebcQwVpIZpC(3t_0~~P!Vn!x*u8-y|)JsVkJM} zZ$!E8Hxex2pabDL`w{*3-yIx%}Fni`^ z(19$Rv>^Oz-|6;&6_!u*mTD-af|ZDtDR0EnteDd#V|-}wa<7|JK60bx{5u(O+7A1%F;$xcZNC&Q4~}mpYgeWo0j|c5mDUuLO*$M%Z--00DNRr>)>I@6nbLe0rCt%hC`7b z!{f9VI6&Ske$-rpsEj(A1~iFI8g~$SI_0NS;uJy^5eCU&Ar~(~|*j%yaMgoG?t1qR72`(}IW zNNjKsE^n*C;WqgqGNY-`QLr4}PAPOOGfYpF7K+M_wbYcvinw0-vOJc-*A>!o3mNfI zDoyFgPgIch0uNx>&T0Nez_yWd`|Ld&vDPR(4RSEVK5EUbHgK zh1PnXEpZW>@LFph7mE6}#W;MLv>Cq$fHQrT&b?O2LiOu@bA5kxsroE{>|! zbVUR$G+pKuB(lv+)$MFame$LIl}hrpwlZIwA<9kD6jH{@YZE`USu`NykkZ;-XuD$U z<1}AJSW+`uv72SIsqxNi_`vip9wmMy8OXWuJkeex;2!Fn#aJvAayaXBWqpOvTjMKH zlpLR?U_=A)IV>lsNRZB%u8@j&U0;9ijJ~$9H>}%ZY|q3^FBuTB>Dvd-!w!oCp101q zZkfW+qMc#;L%uxku&hBU`trQrV{43SoArEQq*%zO9%ThmU!KoWV+Z5_(yYg^U(`HArFAiH* z>BkE5(T8&}V|3((P%D=V^aY6cU-X@>8@S?7M3dYxOVfe*L>Aw8R98=w*q&*vJss+T z1Cgebv&>5G=q9+HeC77EU zxASrYwI_8h%apggnk?Dz?+FzLltC0dzAbV{X=(0lI}PH9Ddssk$q|z$&7Nq7+3 zQU`?~Jy4qBZnaEzZIbBnqB$fIWTf7BER`S^2^g52p)1N#t7S(b(5H7QH6cQQ+e_Mw z)X%J~)~MEZI{8?g`tf$w73<+W)2kya6P@&ZTU{+}t@P6qzIL)^GQF%L72$hJzOD~o z5=DHEyR#UnRmaL3HHTvd3fk%;a4J}Q^L>;;V$ApSh{}~31$^8;O*^ zBK?q~k5uHHs_Q#U(gSq^^>yP+)3&G@)CwuxzK)HQh)oTJM&2GzBIBfed_5YCS?00| ziG;}<7un{h5T>ovq$=aGtOT{#enz@ z|-DC8=eb38urM8P4o8jFzI9&gdz{<8F2-g?ywzf#Q)bMa_2kT@p*gL%=f)3kmo?2C6IB9Fhruwtrgbi4& zT0h?&S9p)pvHK;$im`gLcc_PpeLo3QL@0{V?w$E6Vvkwi`Jk4trHv=?IpORjzg@+E^R3VK}E^W9@^6;Tv-iEGOcSe0~M8l|^I1h|eK-*5HT6QsF1Ilryt*U_1E^vZlhy%eLY~j?ctxAgT2DDt_~EW9!*|<^e5YbV?K3P2JV{w zV{+cn@zt3)e=|fJw_`DTGox02Q67_(nbZ7*d_2AiVmrcKl6UoflHdE!^S+y#e5rTx z^0OzC-&{=E8+^MN^Vy#H*f{W^_1^DrnR3YBREs%*KNWuQeNeUQ0N?dwrtf3&hS zLg7i3Ppa~G|q58yK z*XSt#T_dUPrzIKgLq_Yc~AXFRlNvUt0J6;CB|Uhpt7N zqdSZ&#VWWgrKC6B@{A@s(gLy4j<6tTTTbO`ItbV99b=YZb*>6Aqt9 zIAThV!xDQ!TsRy{tenGAqmWS!5lbkJBc<<5;zZVX)d>L=IF$Z(d_r@IiGC}Mw+u)t z<%$;={ad(m2fb{a}%qicCo5i4E7Az?IY&GLIXTH|0N3OVvGEu;%jk=W!MiJ6i>p>!hF< zXUc}`vgI%)3TuV#;Pac5T&#gJn2RGr5d~{)!X+9U30qPs#Ud%LvlB7G7XL&h$wuSl z3M>}S2uUDDQVI^s*ndF{*Vl7(!K)4wI~bc|gEu^!$!-34Y5FbaY!$j)F_m7`-32@`SbC<6~;BEmBe*4dgO zm2w42n;sXLpgd(345gG%y~rTZ;K3$@OqT=%1WcfUWRW=d6!B=slNg2}WCStA1}XbK zqp`<)Bgi|7jg*P>am+l95M$YH9!ISIl9GBHCF^OHvY5F-qNEJ1IUIgp-4zV+Cs=YB z%d@415)>*~G-0uP6I;?nf(e5HM_HTLczld~gPk)N@yk(0%V( z6+{qVYmEgeY*f}NN~KaM@lQe-l0b^%F);EQnLkGM4er}0;UJ&p&cuend|2@Y?5zs^ zVIIYae4%eZv1C5Q!~53W#t%`58}x_qD6^UQltjkSt&k^ME9t?ZoH^XoQ&^tINXnWY z?t%QLX7Kbl9!6xM2v_tdVFmS*Kry8}N>wY%~oT2%k;qW_1t7Ky3!x3@}!wm8;QknVph?jhy0=_|dz zyN8eqqvKnPx~A~m}ApmOjU5;?=g|Muy6@vUg; zI9wn?>J{NrC8P(ddy60*9^qS@cZh=e3F8pYPO=d1EHUlq!G}`O5Ir1|ydWc~9L%`hcRZDq`a8T%a*ygl z!}(RiO!`9i3WawKABiq^KZ;;G|N7cWCUwa$g~8V8-jFejppKv={#!#DX#*Yra?MT*P*tT>t2^RipK*ikA&EVQU?cwQf&iRJQJv*S`3d;`!V|- zw96kFKv9Pb<+{6f-!AF^9trFo;7aW?>_qobIo@UY?YRlg^Dbx81Ap6M-~B=lcHQ5# z*#G;%Sp4RHwb=j9wbMr;u4mz$mCLJ zB;)Xv6v53{k9VDA8A&F;yX7oZ8Ck7cFRBZ8HG}3UQ+;e1VVd%Kyt@?#%^s}8X!((J zCFRa)xtPq7THfZ7>f>&d#k-lHn~%Ewh+&O1Z8vn?Sv*P^U~7b0Vy;qxwQH2DrBM&A z(m0Z}&hqlGF7>xEX}58bvD6B^8!}@{CVx4Vm%@rAX&PjSUZGdpQl*x!ICASzgW762 zn&9NNLfWfZ#WLLu;wkuj+gcmYC^0Hlr79aGbk8q1TsI}n^ONBLxUkeejmRXr#=s!C z!fKT&l~gOOa=)(6ZIFvaqB}Jawsg--PMbVjC>?z>F_jVK8r(_rl`EKWcN$MiTyl?0 z=ek7<6rQ}1U@u|$yt;rc`S7fh)$w0R82V_s68mh8kv2U=>DrsgsjhWZy5<__waS#P zjFFamqdR%(%9E@af4wtM!dM#dR4gmBq;EcUjc_)nZJFyzHfx;TnLtYpL*FK;N+A*| z)!kX|@H%d!(rEb@D*rxOQfXeAlEvOwszs5&{?|1;%9M20$#u1^yzahKmL>Gn;f=Z{W9y_V#o4D~)ea-sK71QIWbrO- zzmezTu6s`7;%Do<%GfJLywbZi@1C>8O7VK*RL$@e<7-dbCoBvXFF$)f$+beX-sE=D z@cJMdQ@;hy^+65GtcQd#^rnIbNq+U?9g4DY{bb{ppVfCk!+_j0Hc6SOHpmdxb2ca) zBtCW4mTHH?kzP!y+TpxMow7k~ZF=)*;1c!5Q_H6XZ3x)h!=pZEgW4g(Pa7a(95^?4 zoMyqsRDEL)mmoZtd2YVaE9*2(;bE1HnP0uC!J()SO~4zJrbQdmlxl~KSswKp8yw_C z7--KF8w;leW;5M=8`LY8VPG{g*i@^VLtW&Tp=2A>@*-h|#Y*+Ytk$$xwYO>7=v3u~ zfT9+;W}}*^4XjF|vl&{u(OJcE3RH7s$ zNd^NTl9UV*M9D!6Bo$DUAVD#Jh}o?eF$c`sfH~{^^_+9w_k77MbC0Ij@DHDV#74xq=J5A97 zbJNM1J0!-;1Gw_EK-Qosu1|zse6Zj;)8!fFxKR?lA@u26v2lR}?i|<2OXCcwd7Y-{ z)n}O8^Y}4YRmXEI*UiJ!4A4_{5|g0Fq@$}unD5M; zWb;^AE#_QxjE`5f6Pp^&VJK@(Dr*@D$2pW_hHitQZIMtG=CtTkn?OT_gSPCG;V0y~ zeHvxO^i?=>bmaDUL=@54C&LZ3Hwxtl1Yef&P@gEaT)$?cZKGsB@T4L}gGG)9lL2M{ z@b02G8)YyKGb(OGeM~;$mnJJ9@J^rW*C_e7M^c9$d^#)Qu-|@5W@rZof3Rh*vq(F= zceGU{Mchx+nr+^W1^=2D$E#yW`e1;OH`7TV!@jPa;kRGz+(*2Zr*r(lCrh>vV}M&( za3m<*BB0K-g)z-TM^+F&oi8h<9Dv&(*}{;33tln<8;9Y?W-1g73p9l?(ubuAhrMPx zb6pFs3Cy(E?N)oiPiKjaY;A{ES9P61ZAYk{<1?AU3!z=3(Kx>k)w`_|T6;mKtES$y z=0d2Rua2xLCsZ#p#8~xwP~~W}tlQy8)!XOw-8NF$H`%bN&~6`)cm>tfgsA6vhZ+n= z8M@MJ{F5!OMbU%jb!)Z!R11T6)$yTSH#eGBUx+4>PV~~&8vI+WT{39TwQ0EM(yoxN zSH;NnI1GPwxT&po?p)873!#*yjk5fa=!%=s#^ zMy`jcf*avY94=aTW!!aMD6`-}uK8hV|0YIEt#WAJqDvvDP$WU)InC z9eS%n&e%g+1DicYZck!OCokGC^pG~iG5ye|~GtJEJz(%F9dZ;V|C74<7JcDv-> zy=2pX+tu*HHJQ6!cC4c-U*CA~AeydtWQ#K606{zM&qVPDZ)M@y3lv|6cKr$$6&SCw z<3;CBnviEcN{djO!0^%NF2DWES4PR=T<4Mgkz~W1wj0lx83jtOuQzH%7uhb3X*5b7 z*q(`wgwnqInY@q5<^hd?;fCo0LLA!tPWTa_;$7{`4AbEvTendKaRG~%My+sT!S<4H z&6NkJg@KNYl!2uZ&ClmkEiavKQp2Ku0rqNin_kLRPXH~^S)mz zdv7mPK7BW__x9_`z9%bt6&v35ODgoBT-wGWBmuO|L-0aQf;a{VTcdUB);X9@|2J#MC6SrwpOUJ`&~`u}-{!#}3~pV9yMEj3*kLnj$Fe;+Hqlr+!8L!w^t zABn8VNj-L4f62;xm-&C4tWc!vzfR|Oi~Zm8mH*x6EC2N#)VTeB*j5S3m>2Uzq`S@! zrIfP(PlUshW{P>CVkremi(;OI6i_()8mIl6< z>rxtQCtzb&iena@3JX>7_M2Tgvq%#!_L-OJ%we(|OO><6eFK<+=m>`?hhrr3Qii=0 zLj>({coykg^K>2qY3LhL7V~sLS%<5%*_<+-B%xv0b689%mW!pPeK92}~mM5}5r z44530Y%kRjD4^z0ic4={GEVxiD~2>vycoP;V_Y>+AWN-bEmF`ZkQ`WPuntqgITL@t z*#g`BQce5PU1{%+zG0TWn)X4)9Es9i({*FvSMebRL(5aj`(#natbiFmini zS1IpIDUVUgg=OO9K*wSp5FmM~!aevh7vOOVeq4p1q}%T`7kt&83U2%>@|dQamO`D4jx=^Q72z znYv&sDFvNyGo;j1N_pv-m?jX7vga}^OLXAc@8<*5ThIUFZ1dke_kTR${J%%+|EInR`+w{a`|;QR ze#A~J@IM~0Z+sGH&-5F-OGxT)-J8No&EF^r+l6lYif66GP|KAnl|C!;F zL)wo5ALE$+5?c^A-z=s8`39VlXH1kcNj^{I>8_+z@V&VbLvb*TOEF)LNQVWfxmkkp zoNy5C49T?z2hS2bxJYN>8*r8-LrgEd;IQNX6Rc3m5@F(+n|$<*-%`OHd9%D9?pyJN zJ1az;+-bYwA>WYWlXqoPhM<@Z^0<#=;dz2*S{}JgAs?dCxX&MB1~EjY9FU}~aRn$f zFhIVXCs!_{aok%n>3@8>F2=gbO_e2_Qt^=Hc?_`ylLeURi3~xbSOzQWCWj&JLI*Or z>EtVZE)f+1;Pd={w8gLEfQ&AKrAowU#7FYXbS6W1a1JLF$$1Rn31C$Sg^-z*1X$Cq-6sEvXZ~yvl_8}Uv+CMNjI2PYD<9iPn zNt3W_r?C31cLD0Z$N5I`|2>HR(P$og6_9Cui{HiCIFlb^w;yI|G8Tm&Wdzp~^h+6j5ng#Z!m0CNUKXhwoK&3=DSMVVWh_&COhMDxOLl-dHxMNbK`5DDTF;Wli#RmB?oum@!zhM$y9V7= znB%m^8h;i|?;ALTmM5JCZD7zB7Kfb}G+mXcNze4{1%CQcFd8I`x>bfDXa13sz^)$# z!ZG6}i$lN9c*3IDBN>i7rX3xbxeOc!gR(*MH5fIgLt$+9#9B2bjt!H6@UI49h44HL zraZm=-bT==KL&cq-#~BtW~|BN(p!~^z|l59NsGy0d&z9jX3|uoEXqrRX6P_E^z-Ge zD4$`xa5^T4xj0S9n}~3zBL!gdqY&WoMgZ|50HIQV`DFmZ6#&&W0H^9PMsHB7>WK)4 zjnjA#wonTr8g(jw7NybPD_AI%q7U$3F2Eyu0OOSaYkmT7eu16{I0$-)-y-r12P9Vg zf#wyb9AGsw?xQvLuidbt4LZc!YNn%B)eL}vSxuEjCD zX_jsYvM4K%wG5uXWNF6ap$9!rgBipB7K>)&gLKQjH^?_Rjboe+W_ZP|pOW z!FUb=hqx$AN@3S^82hq&yFhllT~v>0O2^$~1GQ;T9Af=E3D%mr8kku-XQOt+9OOOX zXJTeWfJcDsdd!=#C~uR*~7AS!jpd{D-v(>{YvMf9H-25p9 zOlP+2LlZ+LO@FK+jv=ZMwPrSfzri1SOgPEI=YTzV3fO|vk~CwCCC?6C%`$>}iTeO5 zIff8S4F*jIH?YywmqKzNtyhBVE{l!PovtZUjhW`uDc!-@H$8#Npj>;mYe4S$iDLpk%>Z8OP4>5ric+&BZ{)(uBatbn~cy{7Bi02TV z3wR#j`3NFK4G}xWdSpuQ5aa8^0|TIhG(~h7laNusvl$3^EfFrm4Vlw;#(@yj2EqlI zjd+O7o&r*4nusQ&5DA73dPHVXJf7`9teFn-7f7hiM2Ru5#O&;J(F06pDpLvZSZjs^CLI+!2ZwNH4O&q z;s_Q;{K54S2tNmaPz0+3V=`pyM(!$esIL7kDvV0b8n<&fF}nQ%N968>0`{(V4U zeXxirB^Lr7+Q5(@Ar5X~IF$Y`GkOW+FCcjlU>b)EaR65$(fid=sCN{YC&2hd15-lg z$vQGm8i6v$JYj?(aUd3wj>4N#FX90<-2#|)8$jR=NP|H!@(2{KUjQ}dCAz!UQ%9|t z@e~CSOThz{Mky=dn%{AuC} z7pw*H+;S@*?-B{J38f&Ryb@>+*irM*wbz z?@To3qJwME!5OwlSR%n%0O97_f&R#1fJ5-UbpKpObnUNs$j`DtK6PXv9M*YI-@6Rz ztvA4ptSMwY@wCmNNvMf%YkL;^!;Dk18e2{X%o} zb8inA-7_ELnWTrSNM}6J8G2;U2d;YTIPwpN0FM3zu>3ke!Wf#r3BT186u0*OD_05?XIS zJ7-cWWcXc8y=t;H@F>o&h$sDM9L@>rTyUmxiKKw2NJ7;AmNaK z`~g+nksHH|(wc-CEp|_!wZ|Sv?i6{3WRob8sqbGwA7o-$AWEcqKB6Lt%8EWAxA8ML z7kmL|{|ee$g0ES2)SF!7o#3>n7}CpS&rx!e8plh)+XOyi;D!W&9i#xRDWNF*@>f`M z=oREI+y$774)Go2+52drp4evH8_=aNzeCc~0?3;P1BikQtG^75(154q;Vdoc-K#)r4k&?d5!+>M{G-Cypu2M`ZXBI%Ru!`k7;I<=gfiymI&fXxUak*806riCiiQKM38xK8Ko^Yp492-biEj&aUj&cEC2>bJada zCe5BT(`_Nf`Mn`HPclL;;FkHzn=S%h5@phD7T#E6emE=zPZdWP&1{~?V(fv8&6Ax` zQz8tgmG{+Mz_Uu)1RH&Z40uhI1urtVbEcplF)9S53KSN7N)&frl%N)JbCKJw4O>xT zo3a9^?+}2NE1HF|imF5nyb43p-r+Yf=g5wSOwGXvC^sh(bxF{x9EDLJ7J?w2AMtUc zvHqANvl0(!ep8GHmzsxhV7LO41k8>&aF`Vj%ys0{)`A0esZ?wN64)(Jn-h^BONwC< z5}8Ouu0w(}XR#iFC#It+R(r~L2L$Ix-%^w3uNhsJ?n3TivcAdR0ZAxGq_FTX)S*OQ z%SVU|R>0rT16I#Lcgz~ela|=(Qp3Pf^AQozy_O#l8FVAhwyLJwEHr5eS_TSL2CGvN zXgimfxt=_@M+3Ef5>QzP?)Z^Vs{?Xu3=k#j0$%T4xIv?Wf1?!sk?v_=(e)-{3-4^+SKq<Xk$-Em*_o#LcrXoY;cUUW@%;xYy?f zKp4OIDB5Ni33Xg2Mlh6>iLzCeFl2W1VOr++96f6OkjBuzMh#fBzo}BpX6X-~r|Yc2HEn7+NSRjAq%9 zS%y`f@AeE~L2OS!RWD{WTdP6d8`c1iLCne=9QLYI z(lu~)xegRIaCBDvILnlsy+d>vg38N4%&!lE0ZBB@@Wb)o0DC6PtJwYmV6@m{NC0m_ ze=&cGF*3A#1wF0PnCg#{8f%NAAmN!IKxbc=?+}2x-d53|Cr0)+44GE4IFDs)wWUBH z&LOupV@8P1SQCW5QSi4hA4*m|zC~{kWU?hGc zvFA4#bEXE%nLZ{k9pbNC<%p_(t%el0){@XVDh+kW#{6bBNF`P+g#qrmy#}O9O|}E- zDh;DT)>KfDP5Cb^sCSPJ1ZrFbP*ez@M<&lvGI?xaMU=}nFdy=`g|E#J$I^1vV^H?q zs?0EDE}sgLN-%vBsoomY^LUtq`OuRlJm8ZJg;&P$zFnuRVG`i)mU(7 zGgY#sQYFliRn8$9LI^ggjldk;b|C~qH{ZYl*Y;_LTR?>eQF>U#1f_q(cB8cD#xL~w z)+nrH&K9Rg1WK*SpFn@&?As`;cIFNfqp^Z+S|zsvXMr9nBpQj3vqKdjd2pKcY8->L zYx<#;u;4kcRibr9Cv-GxHgwdXG=yJKeueDNR9LJ=VcaUFIsHaC9$a>+d_Xfz5$x5N z82(I8urv|`%TbjOuw1NiG6sk4!}ouKsA|MN7PF!Ou{bUMOr7*3#RLp()2 zT?}v-YYVmB9l*i^AjA_u%L^Q^lrx<1{Hv3*y>X62_L?MUKu14Ui({o7h(Ue=S@ucnC?1FT*b>NrQTCINsU}Dk*05(elFNf8BN;@J(O=b& z-1!a3OM&J{J|whlLAFR9`vrg`38~Op=ot|S|7g^CE0&h{f9fwE|6&0lyJr4?4Wkc? zZ6f9jX&{{Nti=P}GX9{zY&(M2g~<5fAsl`I>AwTok41)@;yKiT_g-YaBXhJK8L|vL zCd<$!;xXR9y;3qQ=mD|L{lsmM1Ex5*4RUx-Nwgt>jUu&z9A54QqesYG=z!koPE-xR z#Njt&)DM>2!$6&fPoSD^0k~sNO}Qc^(C2$dESsfe`rsl}$mW6wiAg+&{vJ34Q^mjmQ;#Mf;-|Kx#j6-(+Ve z0ryQABUGR!z@yQT%%3)W)H(s=@%INH2@Sv%17L9g9P|LDlf`8PrUp4Ir*>I_ff?oq zCGZERlNjTesef3j{~nT>1DBG-Nn{AxLIr-Cc%0nsl@HbKa8G_pz9L^S=K=;djf`yuWo}b zR9W8xy2f6BO??0%#1GoaT4IA?ff$kMClR6E7t}x_xInYPG?Kzm484&eQ=^7P)7Kr5 znor`pD_hAVVWU-ktgiqo_J6xQ%V<~!4(ws7QUG-uY-T1pIv!?vIU4%a4q-{ zWn40rEA%loSMoI0TBK90`6$y?CW`hszX-%nF`#rS1KTIcfQh7DNfOF-bs#DsXYA}~ z;fV?VydIh%f^IaS`(Y=hVu3n_InN)>$`Qy~P9}ddB%gH)eXL{Y;9{4=+@aETU~M}Y zawts!xIxxBGc~ldZQCL!F`f(!)%FeEIq1x^L}oL!kA$~pEGWF$^}T?tU|q^ zYJg+406}DK1rtMH!Vsug7eE+W>yOW{aJfN;7-XOm;+!^g&_fzUBYzR~^q)Tk#Cj4T zE?b1NJW82(bJr$Fej8n#`#dX&IY2FbfpV+swUB6O*z{zkJmrVoD}R4*J0U#H>1wKgRUllhB}7X zNh$vT@E}8w!jQ(`=37E!x?+$VXQUrTb(kx)a|o^8mHq&oC(D37RsrLAB3f%b_W!*> zIKid;m}aPujTub`7hFQq9fhb%A~$t%QW*$GL42SSIjO|82K99tq%a_w|0Hl3v0TW! zhZt_GTI@${YN^aTN@O4Er-8C*KkA=DVmgU~Wx7yh;{>n18u2$ z0u8S+M8z+l-)|3k+hT~j7W$=rwN#0L!OH-nvJJ!&O91#l-q3o=fz@+eIhJ~e^3!e_+xHV2-Ds)J%tk8BN6Pov>l z>I?$ey~Wrx(q#+IqthOCpda=INDKib4om3Fc^9zmX@%iThiVT;$^Hlcd$O`i$Ahu# z&_!6h(g~0wB;}u+qz_|A;K-?YXV8l9r%RYNy$0jjoayiQ^`eW-(5We2fE0 z%O{j(ZN^OJzkIRWaXlKll2;2JEt>BDU6{lI7sH_Vr`CA4dP;zh31|^P>P3XqTn%Alm@sM^1e99h%tpj5=nNsCg+P`0D;U^diqI1K7j z5`b_ripe6aTZZwSv`rP`3ztAws1S#=;q*DMD2E4QC*Ky!WCqYTJ;_XFK&9kG3);0E zH%0W&8}epKE@B0WGY&A}9g0<8L96kQpn;tJJV7Jk?-2$`BBO?foEEjaoj{)f)k3q->N`T{I&?qE&J$|Mczs} z1WpgZDfJUL$zgqESOG(P4Yr%|l-f^VYLOEsqO1bDL?Lfh0RQ5agaXdOm^sT)<0dkT z$V(mKY}7#f#J_|&D1qV!DC$8#z`xEhCT{dp;!;*3Ka|i+0vTbLnbbSzs)=YL%KwxUcFaME{e2#KwOsj81;5$q4h)(@;@m8@ZJon z#1nEGn5LBTFsNc*AjqKS4wHSA&q+}HJ_YdMGyv}mfCStY|ILFG^q$;0Q0EX5zPN*; z?0W!i?@9$NTmeMz<@w1*>23EtJ zu-`r88+A_m1FM};PkBo~?cnHZVhq-zcZ z=!v?>N0u`&EQ%wF9p90_sRhhL`uim=xGWt3!57@ga;U$c8)_PvK>q82v@6Vk>Me3` zHcva&6E{^C)B|+Q@!fLY0>OT30C+5GzL=0!qQ_KajbQU8M_Y9@`ka>qp=yPQ1k#Knd z28qD#zyobj&M#5N7a3xh2|%3q8v_yj3Yx*X%YX3%8sc9M@LpqgLbPO!hak7$8Nl-q z)g<2F%@i`j34tXZ$UuT^vNrHzE|s^~36ZIf+0B2a!gd@juS8WZL>TmV=_ zKfr{;$vWByV$y+nOgZ9)DCQT?Ku-P&Tk#cG5~P?BC((-eM?Hq8rfjfM^3N1?F%xNF zMzW0^VWm}thrDgIPzdA_urB_aE6Ru$sr$stSjIpRs(5nWKO+*TN$}3Z zWT~WRaSj7Z)<3r0xHF+-U~GJW_BSE73cEAfJaj$lH2-pOI<~RoaGT!mf~#@s%lyhhH9c+SsI0BzEg#qy%>pCX1dcf$$5Q=7KUgk1r%oygyhN2OL(R>wBl^MwO?cw-XLMjYWQ6XXgDw(m4q#G0T>_##h|sC= z1W>XV$6%}pLFD*i!wt?RW;m@GM!fnbvNRmT(m?BPcEVP~2EC}(BH{MSz$~yO2W;oV zF)Iq*7HwuaQ|d(T@PKSTRjL?=AdfTWL*P$27T~3b8ATuGS)uasEBjHo00T0NhXj4I zuEG`;Ap>_nsa?Ye5#xh>Q3Ka88|&{uj%^PB+8+Wu8$&ZcPBfhvE79C6K$fU$gm zLz*pe?W|^M=_24^a1_fORMljDj$>-k+uOQWx@6Bxzj;JfOwVCF z#b_K?ep-%mqGKwkx=00=*X%8;Y$_1o*R4iNl86fUZ~bAfN=a;nqE-{Pe^|v90hVzy zHcQ05Nc_r&{V5?5BvZsN zV}npmM$S7F<#1%w>Qp4al)8kS3Z;+rjDMOemxmlLA5|W10w{sC^WS&t1IzQEb}NCG z>MTRi@^XNu)c~zE0AgDJ#>VyOz;p8w-QBd7@|GTHX*0_tH4sFYfPvO*6_@Q{gRu?J&~ zV?4e`#wbt77BmhAkzz{hdm(&38(*zKn0pN~lOFc3J*gOYdcs{@}oC8pb_5f}U07uD; zTImYl696C`2Jko;;4QJwy8{4G=K%J=*eJV40GaS{l>TFMz!u_)zlBYCYSgPiUczPH za)8ii0KpXiT4Z3=R{?Cpl%emLG{SICK_a|wu?qvg^%RnIGxs35Ll;!idf?juzIpmk zgNhMQWu`#!qY5*i?#%*f(rkboa{-w10EW;ny6t!;!Z9o8o35Eo&h!rx4eSI{{uMNd zLeA;IO%P583G*`sD8B*blo`=b72#=V4lo-U;x8al))*6PmNRZyL7Ng=FxDqPF z7eu##3iJcY7v6(n5#3SYz>@)YjzFDTNH8NoP)SsfRdR}ls?C<7+TP6%(*ZO2K3Wl|mP+&A2EtGq4 zp|~h-B#&3lL1TOVCac8B$NqqRkDCGGUk?D5j9&(1yncb2x4wd0`|rc<0l6iJ@+o&e5GeWsydk!b8w?Obw$3J@04l^Z z?jcO08pzbpfi&bkpC6}buYy%}A#)16@?(%8ijazh`H)cd1)g6(IL2XJDM>=@o@9U( z4 zShqkulS?JRYtgYP_UL^s+yeVXnpOkQX6fL{U&l?+`hskjS>L06=&$C&aj^EQ!)XV;QKd1aJ-#pgRTa^A}JmwF|`AuM8Z9aK~{e3 zV)?gBS$I>K*4n!m?;2FMgX-ZRs7SDCbsSWPALvhQzF1vz_g8^&BhjJcr)uU}Tda6MGZdO&;-9p=6jT|Gg%x(I%W zs)|7UEvVmu`Zy+#r*jLp;S{9}*{3+&q~#sABbe4fRX5Q(%pt6zwB@`*V4^^L>}cI# z^l)ZAIo4XX8AnmKIr9SxfD}N^}dv*F$3cR)}A;ok)j3hteiSvb+sI z3EHIZabt1Bwb&8)a4a!6Stjr9zLCWRH&r?n*u&F+o#2)T?AK&O_4Lnfm=2w>tPB*k zm~ixi-#7}P^;?bsT{saI$riA$-NhWGOq@`EJDId@fuNhS{uGWO?O@XM$8Y3F!Rd_d zZkz}VAR?wEO7Jol{Wxd`m}CawqK&>a=OtL#Zntd4tw$!@9$Rs~M+*W|k9B!ydeVB- zx|&o2K9`CzTd^B<639gQ9kg2o>8;tMY;D)KH?@&078~NMHl{kvwYkmbeDoTfH zrB7eSCq(gGAJM{;*>m>(k; zV+~Z9AiWwh`$}IvIxkmr1mb-0QEJERONEo8PYUQmL5y6~=I?2pHegt@5SmdwI|1}} zfhP9EO3a24)<)E=b=%Z|uDm0TFd6Xxe|!8U7}8h0Z^R`s6hbfV{|i=Sb_7*7%3}LK z?U4g@mOMZO#Ny9i8x2(H6L^D75sv-nva7fNC+jzTzxf?>pGk{#GV7>x5%5@%husSu zC2$@cEEWV43kujtP7k3OY9aD2ivUtFf9SXyJz$QYiV@=cl6&PcY;I=ew#M!7=@$~olxnk1(Ds@3?*2ug38h?_kdp2(z^#6 zph<9|lq|7oCXA4rUQlhr*oi*3T-6757w#Y&r%LXEo%F+-=!U(D_(AwvaX0|_lQlU# zUi#w|*cJX{^ux4f8hL|W*chztq3c8DDG+c~5H1<0> zfWBr65Joy(v>0I21wg|UOs+$oI5#NPYW{9q_<6#JQo(nhfIXX=|iUm7F3%bMi}U`r;`BBy{rT@ zsJ)9(x;xqFn6f;@)E|x@X*)7k}@Vos)CM-An4S4m`7cf-EC(J>-SM+m|0LX5o2ZPMJ0StW~kXyBB zk0u_+`KRm{>bypSCmus$6#;3qWt%038fgnnQ9~8;lvWwVrw-H(kWy#{tV0y>)vSxJ(~xK+VWU z?33t6B^bBW-VFG@=RxW z*RgyDSj!%;{24-lT%qOWLLmV{p{s?$i-jWFg`x(8R$LNVIWDw{AsoXMjx!gIcNUHh z5Kc%JPO1}5ZWm4&5MFmlICWe&9d7{Z(%H^*?rJ)(m@a6iH(sJQjnl;(ky2fea%Yjs z)go2tA~kg)wF4qJ|JuP2t>cI`=!!O(i|z~%-JLGlQYX4+K(y_W=)Q5$c8*vlSFFoi ztUExgH(l&toml^X*pUZf#~AEEE_=wFJsiM3mCim>$38c}zVLwk*EstUNBpv`_!V>U zYXRanR*R1oi{EM&zjI0a?gR1rj0q1p6CUYKcFNp3(kHyAoACF*gx3!yycwS` z&XIV}mH41b{{QGK@p-kxmvo7*#S-7!C4LM@{JbRbdt8Dc$q`WI2wHGhE*zm4j&KG? zq@E+z!C{YZ#KHOP(<)P$Mrj)X>l!~4d z&qYc-MoP0pO1nc!XGChoJ1JesiTZjI4P7Q0$4oRWnK-Ls;+&C*^WIG~mz1_tmbS8x zwh5G8kRfefFYP!ez3`#*Vy29XvW%OBjJu1>k{B7U5*eQknWZB#eh+2*nX-Y(vdi^k zgDqr30%gN8WFtysBRgcHM`Tw#lwJ8wcD1Bjte)H&7rC`Da)~8!$sKa*M&wf8$)!um zXX?pkyU6Fp$m8cQ3p(UCj>vC%CtoZ%sZ?)Lxyz)=m`T+oleTnB+B!06`@2bXk_rua z3QaBwyJ8fYOB7l=6!wlN?0cusE~(h5r+C0cu_s2cuSD@shvMN8#iQ>O2P7vS*PEPn z!e#Qwn8~M0CZFw?e12r|U+*SglALl`Z^~7dDc57BjFwEf)iLGH$dr5UraX{TdZefH z*hT4SjMDQGrI#H_uSS&Kyi*#NocdmG>PMHUpJS$eEt&egW9rY5slVS%Wk_-Hv-y-O zS16WCmvTirx$Mi_3GcZaDP<{rWocJs*;r-yQe}ls<;jai-C zr7GH;Dms@{X1rI?l~UE$S2c81HI7v^EmfV>sXFJf>b&==_!SsSeV&y*&)St|8%us= z$)S_ybeXs4JJbg< zQODI+JW^jJpb?{@5of6pAEc3xsgYEwk=&`V?y^SedyRA{%}jmGY*)?PSk1gr^0%Bf zUe?_7Ub9$At5jdB+*PYGR;#*HYfGor*2`Mk-)q%LX*cL=H@Rx>iq&o|)o$(7-g{Yl z-+S$LscD`1(+;>!>xrG#S32!b=d{C@ryYGiZ9q!rxQfn@rOt4W&Z$hDGYvZDj_X`_ zq;pYV`iRQ(E0)u*1x>$^IsInC^xMa$-+eUwzQBx!Dl^6`XFLg-@ho%3i-sA0AD{91 z(TukOGvBGq{9rlrQ_##WnKQpN%=~eD=C4OH{|M+ZRdofebXm)Fg|l=;8g<2nbj8PX zB?R>(RrMws=t;Zj$;Ro)m+2{V=}o?(r}ROOJ5gW7K%eKPuO6qbS*EYus6TB;fBKmI zj1T&{6AknY3=G{2jFuaiWEq&28O-W3m_1}Lcg$eEprQFhLrVigYd1sNI77QKLx(Oy zrz?hwJ{USrG;%dCa(6RY5@+O9X5`alwDgLR-J_3U^V~K^7&t~=6`FP|6^$Wud(@m@P{{e=7QGdtYCBDHRg1= zx#$6N_Eqx5((V?rYb@l;Effw|OulNN^wEMVZK+~t$#b_xGZ47gIJaJR3J_8;@Wc&ukm-CL7-qHp?E{_*1rlJlo~gwjsf`VcE73O}0@dY*#$C zT}3U3;Vp=>UJxI=AR&7}QqzKz6ARWqUXVuFW$^5>tnG4w?KWiFanZQ+;&;5o zAFLOD3SRsrd-1oX#Xn9g{&{ur?~jZBV7ZrZ7BF(AJe-B%o#_f^(Qaq@KqLF7g#F3f(S~uem6Fa^cFjsu;QQJY3b|T{SCQwYyz)uDQLOCr0ML|;_z=&89M7Z*&*W~;b;F+PpLnLRywX`-8ERfxHeNX)UK?_}@^^X_bbF<3 zyyms(lUK2fcd3zgxrcXUymxhl_m*z&t=GJ_fAX%A@o6yfY4Y&d74Oqr;nUjfv-g_M zzE3{wGQOQgz6U&fd*XfjDtr%h`yRgLd-Rj&^mN72v)xP2Ut9Xu zr=^!4G`HCAl4JWz8)~) za{xy+P|7$^dP$({+CcfrK!u*b$=3swJ_mATgH()zcuRuR*9K`;25I*M>0A$*@i|CW zXt|#Has%7tMxo11a+jO!T0Z;a^0`l!&ld`|P!G1U4YmmlUXUAXzbn}BWbneL!Hb>- zFBS@MQ4evm4e#8`)qS*|a2b*V@SD%E;E9$i3Gi_kE6RmyPN) zjykX;s%LFfUuD#xo~XmuqmF)#8jy`XZXA7LN%YCJ(Wfh;&-O&0zaIV9=jcnaD=r(a zxVmJ;^|dQTD_7j=S#jt3ihG||Jdj=a$av-BB`cqt1x&DyTP>8Znyy+c+Phlp)N1i(t0jbEBsF5B zOk$)xV`LLz4{Je64VEOE1NQi(=V z*@C2su%xODNj1BZYELC?dzQ3AIJsUUxp6`A&amX&8n;kf zAJJHUWx@JuVe79atRJmff2()>?NjUTK3jiZIQ5}i>LZiX$DXNA6H=d7rM~P7sq)CkXX$8;AO6;id~T3wZm?HwXku=7b#7!|ZuDsG%CEVr+8l#@`VV zT~90;tuDINS9E8z=-$_&2lAU9nQj`h+w>%S)3dxyFRC}a?A!Eebkm!!o5tlgzc=0d z(QEVP#LZu;H-GQj{Bv~k@2{H~lZpk*iYf16p`>EErdae~vDlen@tegHUKC4+lt^lo zOtdeNLDZjABA;I}sij2mOv#iNB~wL8m9x5Q8r(s+(N6|%D&ttqI^MqxqVByFUl8-RJdqW zxY<{DL{xa@SEPBjRQR5$SoWgAU!*cnt8%%0Wk^J2Sbk+hOJ&rV$`vngja3z+RTXDn z6(3QRkYAP5Qk8P1YW<6YAD@2Wz(8tl9pprf$-f2D2?q-dlDhZE3FA(t2>q-kV$YecRGLskYOs_JDV7 zPeg5Re(k}Q+Ws@OM_$w(6WKbbwROmT>u|)@Q~6uZv}`?hX6uC)TQ7=i8`0W!#eUnh zh;29Wx7}>ncKgh>yDzrg7uo($Yx|h}_9qeBpXG0VQM3K!!R@bZZh!M_`}m|C@6C36 z^xp9~X~)-^9p4Y`_<3{3?{7Q)h}1E)>jWL@Sdn$Y1$82=bz+C=#Lm`XFYnEtY1rf8#{cB6qq zqfum|NkOAoYvb&*jdNc%&KGU6&~CDFXtIfHT2Rnr-`eDOwrSzZrbXYIoE3Ju&f4kj zvvW!EPOmLHeGcvPJ-c(+%botBy8^X$EqB-z61gj^U{^%zuBfxSR=nJ`N_2OO_U<@` z-SLsT6AE@GweC(iyL&o&pmY~C!|Qli~b=Fn0R z*-}-|Qq$T}d$wiU%a$FYt@YZijSj6lBU^VDw6?Ui?m64q_Of-q=$;PkJzWlax+C}W z7VJ6Lx~Ko_o+B^!924C;sJ(Z{VefF{-ctp8&$RA6cXscEmwPXYwvA}FU2(vlu57ze z&~~%6?e?LzJGa{IeQ$f9uC~+4)b8)px!pPAN2ji0m;UT7 zL*FjrlrGcSu37zEb8dId`_W~tctF*1_5o|(1GXs#>}n4<^dE4#ePGd#1I~)wuCu${ zeY=;WbbHly`}B7&z1{8iqdP#cCunw0uy0RjN>5l}PsE;{sB=9l{_a^N)*CaeH+FXK z8sFZvDZPocy~+K(>u&d^{^(6t?8}_pm+jk^o6?t8+gH%vxAAu0rXPL9iU&()A1wDh zSebILy7u6f{)1a@AKd=qV4dQjhS`Ujd=KqPIn-QxsI~vl-rI-v{W#RF*xxz3zss?| zJF368u>atm{{D0QNB-_VCU$sm+TkI`!^2_>^`{4xPnN-#*Bpk>Bzdur#O2|EE5kzk z*ZlLh@%Ek&l>L23K)vThkoxm)UcIz+;c6I<|>0m-NQ0maHi(L*StMxC0B%5Bnidcc zl`RUmp#tI>7u?V&A}SgrhzN>kTmcc4O=JfVm8fw=af@+DqKR3I;%?0Swn^?}W-|Br z&ph+~@B6;bD4+hF+E1N2)ts(sPA$YD#9}d)pbkgS!aUDb>=w@vm9SexV=^wU+%1~% zeDiKGW-G(_48!fvZZV~Lx5)isx7dpVxN%{(_z6G5qItJCiQ+) zyhjX0Hk_vJ5nbo)5q5YNMXUCRksJ4jqq%#;>@W6+Ik;k!E&5w#i%k=y z*2mf6HpXSPlg1+)AAOQ7CK6{6=U_G_F&v2q%w;%=I2kdd=V3mg8D2n~N}Pgen2v>v zTZCB5WOxP^lU{<4NYBE@q?chf>E*-~NWe<0!fGU94dyb>TH+^2LJY&}h{;GrI_5EM zKDLqm3_GxZ;YC=8T@1$(7h^Z+eaJ^4mN4!r@fxn{u#LxaJv-kB{fWoHY& zeH`~w*}~v-wrCF(4rOud&SZ-TxP!Z}Yt9x^wziX^F%{D=9W(F|W?~kq@31|*cQ;#f z$C9==VoBc|v2sT{DVg{wK5)qqZcy!PC*AeP5nrRzkQ`x&y*z981IAYptMD8HhUSR= z_|`8+%=6C?Ss&$y;N9({t;7$B+lV8HqY#2ne8#x#*nu#HcM?Yv$6y!f-55)H4{;nZ zo0x-Kgfl!I6R?-zJWM1#3HwOrBZBmP;sG2)6vL4yAU&B_NQ@>H5sNW}bP4egrjeeC z!=y_wgLD~jI-W%z%d-h2{ht!oWv=d#uqq)S*&9= z&XS%(JV!i_xujz-5A(4A7Z`sLO}K>1XvP&>#Wh^V4Sb24_zJgh8+ULQU*j8mi+i|_ z@9;ey;ukzZt64chA95JL5UpVZWAuO%reX@FVLE1DCO*P`9EARC)&oNrpf!wOj70Wr z4cd@y3k6JIigr-K9A;<_3v@t7bb=*1qYK`F3RbX&4Q$aBc6b+S*_J)A8@l5?IG_i5 z;(a*63C`$+-spqA_=M$MiEj7+{V@OoF$nI^U@$!32`>zRH+&F;V0?_Fs6-Wh!6Q6I z3!dO9p5Zyn=1>-B4-0&PB&i*q=SV{GRlu?d%O8O`vTN4dZc!x4z21?&qx z#d@S-1JaO=3~a(?Wa9cl>N}pmcMBe9#|;%Q-)A9XIeL`mW%6AqYy~SRiU; zj@Y=0Z6E`iZ~~3UT*q*7j(C-tBereG5uah=7WQpxj<|pq`*Xxg`0Q;bb*dr{#LmPZ z%&6zMoaVTo5Bj1XT(J5KWr&^UbHpz6ZDKq=z?w^xD|TJxy|Fn*gkI%66aBv7Jr~*7 zhf_F>FK`2ozvDgeHRs@OoP*dY<%(U1ZksEnVAcM1Qf!x8u^7ux!SHn~vdR^)Sd1lj zX_qTr;RRk}cSNr6E9AYIIEwH7KQdQLAWlRm=`omy zNem|tv#<+a;3{6=4Wy`CA;UXR!3u*W=ZcT96sxfo>yUz-@Q-GF*o8wl43jCjVgTH+ z5g9O=o-0~IfyFb}ZxnvSzMvQ-ID}FhMj6Uci4$nTWi;d9tXy$(POkV0w{Z_c=H?1- z_`nxK;fG-uj>(9|6imYmB%yp>uBb*0YEg$HsK-$>;24hM1TMgQey*5}InZB_E95YM zAzH%-#%P1KP{0JH2w%uHFmDORqlh+MT&|dkX_$c)Ji$|}EN&+~&*1prC0?O@X0EV6 z2Q12_Eb+3CX_!%#D-zJ>Qm*(ChKJipr>^FT(`Ze4&-Gl9jU416gmF*4Wm_0{k7JGZ z#a`hECpe=QT<}D9uh?m@SL7J)6}#H+6?t9u3QyHu;e{cvQtuVkut8!)JIR*V72WY3 z%7XTa3i!k_Zwm9m1}U&F*()xT?-dvEdCgvN5}GE~51%iY|H)oa`D(A|?U*Mp_*6vD1LPrM6zjHt~M{#e1h0mKAi zAaNxzh!~6yv5Mi<7)g2*LJ*2DjK)}u$2f#z0w!V-A`povOhz=OU@E3#Cgvgru~>|y zScX(=KpHk89U0h!&CspO6EgT5%@e*DeJW23Jj43Vu|CvZ%;VAT`$VpMpKu+xPb`?T zPkgs$pV)epb}7Cm{Qy7XA*{0Zi9vh!2{pd^ZlAF4nlIi*+kW{%Z%V#6d$*m`il`4c z3}A@XFoH4Kq74)hllHJ^c&<%_GtrNm{pM*2G9NH50?(qAH;^a|W0oj_cP zRalKgtif7*f+Va%GCoBL*5fOdO(kwX8a5&w8Q6r)*pD)lqf;DtMrU-vJ5a$2n~{l- z<@q8MS67gCT*rV_`C=f_R_BY-HTmK&%2199RH6zss6`!)pdS98=8FIXB3v6Y^2NB% z^2K=kxFcU2{kEOdbzi>l#Yl|C1kA-eEJQ5Mq6t@U6}RynWceI3bi@wq#4eo18NB); zUraocFD7B@xqPt=1DeR^P4d&iwh)E6h{3z^{bGpae&GXO48;VTN5qeO)__^h{bCT@ zp}}ByBNQv0_KQ_mjWt+{PmqLlNXB|Nez0GR$9ez#;sPpuqTNJn!X;crCF7cjS8x^A zU=^@mOpe|!qA>+iF%8q29@p?uaEB+zv4I4+}akfy5xXZQhq>8IC(%MoI4;^VioLuJ|I>q3c0$lt)^m z5jP?mHDeEoTGV0wX9vXsEW{!#Lmc9vd*GnhQE*V~!fv!PRZ4q^N@6xK2f4_@K6n%! z6t{{GirbKv9ux*JL~C@x;0oriJt%%=dB^&L;vvzQ*bBdqeuT&PfZ_fafKQGc6iF~X zaZs$ddQc=F5o^&}sg#WHDd{%E^~AQsjl^_hU=ubY3)!ed6>3n6V>pfzFuKNe&<1T` zi>|Q4yRb(e^hH0o!teS)F$^Ox>&t^;Hs)Y1%)dG)EZ~3ppa{TYw7~1mK`{g|n1>8( z!fAW~x4Q?$2T1wmph(3#-?AJgJz!lASr-z2IViq;PPzShP&|T*szChUQXpo;7l?Cf z3dF(O03=I9gK=@%8f{hDB9HJ}=#e+_Te0!@< zw1cHxp%|qu6n=vX#Q-~{6g#3&EW%smC@P8y#oG6jQp=e_@d+^r>+po(r+9|vc!6Zb zza)N2Ou;MCsl@eoP5M{7!3KtZBT8r4Pv}4wGNduB6;Tfc(1#p`XbmG6qYc_Z0TY;_ z9h5MGIoiVl9ncY-V2RG?f_I>T6>MOOuCT+qut#^i2M258$R&GQ21dOhGPW$ z5r80!#2Ac49O99HmDq;Qa2Dq<;2e3vDy&8#zD3FTLU9P+T;MogqAV^`_uz;==!<@E zfh)W)1m5t0FNWerJj8p=YzqSsfk;GQGNLgBQ!x!0xW~S?y~44E38FCt3$X|_sKrq< z;24hM1R62vI`bkHYj6f3Sr=@f}*(;_hiBQ1)=D1;yslipWKksXUf6ec4YQ!o`)*%_6A_cp?DiV($6p0o*!Bf1%E4;?# z$3;TFrATypQY5;hM~;R6iBN~W!XnzRQpNKYfa8Bi=X3@R3e z>SCT+SS&2i1Ftp3;vRBu4KEf-JRVUj9>F7^Sa`w< z-WZNBjZ&HxSS+Sv24>{Y=SUi|g zEPj|;EYfE%ZX@H6l3py<<1ub$7K>&3ibYg@v6zf#Ou;lv#|*?^9#WBptGJFE7`VS! z96>$il&~!r9pV_{7Vg5blssV!#$p`CBNFc(E*6826pNhtVi9n(SOg*nVHk~t$BM=K zLzI%Lu~=Ba8q?ACH06rTP0W9zSOnukd~lC4zyJ)yvY%PcFKioQA%9UU3}A>HAEnfq zXar;QgDW&`OT;hjO2lIurb|s=1CAids;rB~KA&OCgLny^zl%X6Is6-Vq{FKrr zY(_Q1HK;`p-w`iTDshhm;6E48w4YK1y>5Q0#IVKl~IEXH9x!Z868F$s}~ zLJTH)GY%1mL=+}t3Z}upw?wSLT6}^etb-z`M63=i5s8pyln8;1GfTu~l%PHGah#{03=gsR+U%zf!R>kg~--#C^!$U52o){Qc!C{Dz)m zOT`d)!v|kRQ%>_JC)B}qL8-7o;?h##pTM!0qLgkWvR#Z@TPnum#5&44g|bE!eygOs zt4f6~W;b&@zM`JoqMl&v?NSlYLjDkp4>1y>Fytep6haI|7)E0PreO{iU?CP`1ro3l ztFaN=@ELYsCw3w13F|>7vat6l<$-sG){FQYap16+ga|}o zCO*Pi=oB0lx{%?>#lxcc!C~>!i^HN+uS{I%T_!H#$a1ALc}kgBjs&d4Dy+dLILh>O z#7~ie4cLgy$ijB)LN@jy9}PH$^SB7j)G{#`-td7R8d+}?aRC-%2|mVBEC*jh6|1oU z>DY{Y$U{C(;uOx{=(I9nJH1RKBLy3fhHco6Jlw?Q8D&CuW|>%sC5Xdv#A5}Duyq#8 zVJB|D|Kli}U?&2OE;0Ks(DH9fO!v`3E z{?I^gl~Ni;3_uXZU@W3A8PhNWF-X8_Bw-zru^C&i1ACB-92B7#M{x{|_yU)34ZolT zvaMwz7|U=0b=xQhy!RP@IF8=v2N!&ZeRu}9?PX#R+@Zl>%*NxLW#S3!cbAE7=#*V1 zI-?8TffcM_16y>39o|JZbjN#W%;ESTlk#aIUczNG;|jJgJ`2}LU&U6^+wd8-<0j+2 z!Y$m!9o)qZrrjgn$9MQ15AY*?!q0e!U+@Tz(Sj#=4FjY?F(crWWg z3!dO9e#z(DILLWfKz^~TiZX4WUZNg5j#H0MQcq5mi4GW!5r{p@`vgMI^S*SIa{Q4z zJSr37DfQxInV5; z>6n3;_z1HQIl5d7pHMDFAOewy!em5aCdN%F7iH1qqIyQTsDVyQxzI(`ymFDTrCjt& z;fc(<%Y`0fXa#*3!i8zBa6@Z`jqm~K{xBxp1_MYBL|f7dn7|b6poAIB(H<7)fR5+{ zOLRsTEP~74a^VU$^v3{v0E7H;al4>g3@RxX<+bHv-O+N9jDr_g=S9|u>&@ljCQL+y zFohCk=zvA072;yM3ULQ_@io4|x44J<_zvIWq;rLsKBz);R9A>jSP)VnvO_DxsL>UC zAE|=xBUOmP#TB9ihfs>cC__0aP>Cv3qXu<2f_mgEsSuCx1pPBfGm(TnTgb~+%6wOac)y}TOxmiHoQS>A2OnS{{1AX(M4?k9d8n!o-D)d@?opOU z60*_dbB^^F6=L=orZrUvJ=k2X5Ki|hL=)QUR|*FV#wh5TREn-}v8)u+-m4VT5$I4U z#%n6YIKN5}JG@fV&8QTQ;wpt_YNZ&lxl;IJY&M@?%PNKGu}aYnQ!iGE@1InPAMhi7 z!bALmM|g}LPb?6<*_4_?A|Qq42}x$|^Ap&dk>f538!gFL;C&JjN3|#WO@zvkXFNszfNl zFdwJts>Es3T&NPmo7e~V!}eB{=n6a7BlbJut19vBZ&l(R?qh4~YH?CoExzeiEh?O= zMI{yts1^(H^RQ}R<6kX$epoFcCRU3`L}4sE`Kt!l*Swl(4loWW7^8u3&28sYL@jfjJ+Zw=RRlv1*$Mxsilh8;B`4I7bxHalxXC!E+# z@8gZ^7yM3{E2Q7w9;g)9Lw8$5El{?5Nw@ zU$^fpU6&MHw*p=MsD0o=T{TYvaFluY%e-d&hdq;or09he=#4q2H}0WcxS|z*SR3L0 zFWr(%f7>U?EQS6YNBtOo{rR)>7p7=?Bl*X>BH5H;kXc}mb3-zqL*rY2>3igUN}KNs+Wc_t&yGj_>~iGy{zm?#vk@mA!maITM7~~O z__mu7`WX46gOR`VE@FA!q{}ZRDwF?w*CI|zj{v3DY^C>lrEj6q@4RyOFG_zCvp}bR z*RRMgX5l906X{eWz&vWUdGwz>iu_;iPUOG0H<4xk{O&~lwl9%D=dQtlT}RI88j{*I ztf=dl3th)O>KblpH__QHBG4{sj$L%B-P9tx=@;x~KC+u-`tBU(cVhzIoj>Q@h5zJ^ zM4TNO0v(ReacE3+I9cRy`hvrmM-Jyqdt7kt(G=L@@|+%5QhQu0>T%;jkDHHr+%oNX z$GPX%fjz&S)AN36&+m(R{&1n^Pmg*&G=2Y(^ZPA;OJ>M|sW$~cb7YDt0JV;Wib$Y91!Rl83?Hz_R4IbDsSgrKX^!D%w z_V9}F@J{paE%ESc@)+LY;ji=z?ClvG>^U;VGbGJ3ti*Fnljpb=&v2#J#NJ*J!Cp}@ zUeRe@Q%k(2H+ju$@tUO^GN<>DnBXDvV}>kD8xmVGWJ%MIr7c6^l-}{Zy%U1HSH*ZI zrg^U|@lI;;PHyo|QTn9z_DKu&NssZ_l;)FJ;*-_nv#rHvyV7@OZ{OX)zS%LpxoN(6 zCBFGhz6V--3zS2PdJioL9$FeRv@C6CMaj^rrlB=0L+g}&^}YQXg8h!i_%){aohMvVKm``%qsQpH7Z5kB;vfoiH+b)%@th^ysyv(Mgx1lb=SX zm`_RVJ0)%8l=S&iHl_c6sV5Ml=2>l?%~G`guXh9Dk}=oq@ITN8NWXc(qvnlVFfSxy zUf6%81CZbQ|5(zzVCl03aqSnz_gk1SYT>E{3llRIu06aksd-`YvxO<`7p3-Fls0Nn z`hrE9G8SbXUX<0mXxp=(OxRBZNw*xZcRyu-2i&9Mjmb>|;t8QPvds+t$q zJX>7Xeo1}5B@LsN9AB`cF=NTe!%I##FFEsU$+`9)U+DL7)2M&x_TxtL$2XsSyzJKh zvD1&1QOlk#SoS<)*~`PrUN>5 zx8D@sy)540N_@}f@s1WNoLyG*4*CCQcOMNQNyir^HEv2eS@sY0_3?E`^0y0<@Bf!N z`Y@uGj~6M54(m-_*DFKccJZ-!eaG_kmRHwzd9hyAA=TP7)iyNMZc(cJ=G5-xsSa0D zd%j3@?6ASvbwlsa4Sg4FaM`@Ut$aiOs~ZNs*r4u^rg2U42u<@^l;*uT&9^+w?`qoc z7is<-HU_$G3=Z8ma?!?+%^SnYH;%cwaomfI;T_T^x~4~jrbjJGkKUX>Xw=pTk1Mw)w^aj zgk~LIl-0O7>tuP>>8n|1USyr?u=RrL)~3*{mlthac4hO{Yvo&ST-|!}#nxLLw%u{v z_I2pCZx?O5{}1%+A$8oYDY0Xv)iuTVY~Xq?sCc8 zb5s6Y;St(-c6Z%Gb{FHUE90u<=*Wb^LD!B?GDS!j?K%> z%*(6D%fFU);ALJx$9+X^`%1$0mB#KX%iLE{v9IddzM7Z&>N@7vyX7~8mNDQy4c*!@>B_h0)zcIffb%L5PT&BN_rOW47uu?L_3bNli5 zFL&b+y`^+&W$EVRY5; z#Z`@4s!mo`oxWam=2ex=xlYv=KB#UQU440R^_4Bv*D9-TT(7?Qs`^%^nmZrVd_B76 z+r>5ax72)JS@Xm7nx9_PJnU5a=!4pp(X~$(*FN7;`?9k3_4V2}uWBXBI-UM?vN3h7 zmek3!>I|#ujBeDmd0nTlJYw4ah;qyk^Cd?tvW|4DI%0X_NSD_~RF?JD{p)SV)Y~no zx6i8YURCdKqrT_sdPmEn&i#+}9&@zslA|tJN8PH9_P=p-;OnDm%LYyV29GffUP~Ih zvl@J>8vJfF41e9=Z+R@R|FPgP$3_}ScH2)xTAJzn(K$)0xskJH&YNMTV}8KN{B(+r z`K=TkyWH0ic6FtbEdR|lt-tMiBt^&UDLs!^zwLM=QCP~j^fk2pQYeEcp9p>X6GTz_QJ9& zUFNosxw8Rgu4&s+t^Y@N!FtP7iSDf|CMAjEyg2zf-AjmXbp6&c;iT(eBgf+wYNhjW zi#;h$zNYbt-HjI335wGp@n#-poW992_mFp0nT7?r%L5GUZ)aUR6QHvr&Qhb1Cnz-c ztyTn!yevgr_uJd`ts;2Rm(^}>cWY&Uvir%b zxx-B~36B(-ds(_EuJ?M;&T~vB;rZP>V%ELpwGXI#o)7`Wn!7Ho0oFd%>=FSm;)dYLMG+cQn#8Y_d7&Iz(xukvFc1%W9{yQcs~t-tKfo z9l6~mS-w2U>4yByfZJm%?xi@%tvFI0r%X(8IUCkTE*)aOj2p&1}hRw;uQ%g zCdXBi623j%Ehj68m$bRE?w$z=r(IVS&OPmVPrJnQ0%wgygiJ$8q?tC#yCyk#b)TT} zIO%F@G^dUGq$HE$@`?G+EC%nx=Pos$T+(^Ob ztatR9lDjhQxOa4d<8jx!mLA?6WJ;BH%O_7{5`((GolmPDz5s7oo`1^M2r_i_? znP}v`iL+0-*ef*k9deA6Ivo;{OypK;jZBqRw#LrNfVD|G6sk!|>jp(m*6iUO?xr~% z_nwsCbkd@Gm3u3Vhs?@6Xs=ZZJFIf12lzwQi55sJhcG z`@XHLat=8i=MeU8Wo4(k%y^@-N4Ke}G%xuP%YgcBnxylRg(ksdlgk|=vot9_%em1) zSI<<3yd>P1d(^7lJjl18w9CYVT&GnwIZ2#7lZ^IgUlQaUZ4y&B>WL=tfxdfpt_}M{ z?x>#Py)H-U?!D33aL&(%f*P$83l*(2sw5qCFRJ(2z@7Ql@&4Nr%w-e(%(g2uA4!+v zqHQ==6_Yvm7Q5C3)T!OgI2<+!j&*a5S1Rf)M4aNZiKjBg&|p%6x}(M_!AU39DltxB zZ=c5bV`#RVy0_WJL++hrs!@B(ccdvMB!v$T$Wg^5DkN1xisKN)Y`RXenB8EYyTj32 zK1puk?=R`~Ze=R(HhYzem3*bYfsL&9akq(Ccm3s71%b+ry4%}n?Dr@WbUN-NSBrNk z?j2uzkL?;PqmqZndn7n%m+9PXDVh5YVJ2zadj7%ub zl;w#oo>@+skfiG723OBCMT4QnQId3I-Ao$fd*U>55uPP{-Sy(KeO>iS0@ilFZ<*j_ zXtn*9o9cuFg_nU<$}MjTN!qQJG>6RNO}pBg1O9vXU|Uol*<+``>ZtlfU-rp@#orXd!Z<$>2ENwxf(Yq+$1txjZOprN5oA7w(2 z=cc*mTy;0iJ>xz}e#EPzubz{KLU%{Jq4n+jAd7B;9lRP{SEOC=R%>fU_dUYOSK*nc zs<&7+F>8)*hkHtsR2{XvbfjGFY8!76UiNI&+@JTav}ipk%X&CDikEBT8%@3B?yBWV zYfaWgjeJ-)6OBcpnPYfBR^`Gllr8ufik(*}z@4tv5TX!csiNg6YKOU=phz_ssL zRrRa0-{tc3rs+iEAVqB3-jc6mm11Qv*jHiQ=x=BgpJtyc8);~jm=@@(=$fE9uF%=> zu#1&di6Y%XY^;|%Mdh1W^paC17Aq7U-XZxakIcC$$x+j3xr-Dw*qbMGs1oEFL#zE9 zsScA86!G$e6eo=;At^wTw#Ug56;?8}w|hd0mE0u9GFWa9A#;?^wMh&bu4*II8Oq&_ z`>5sYrAFo6ZBL@VCc&{mv1j|kF5MG)n@XJLcjRt&(lm*^gv4(es!Bt*Zl~nKQSjQ(RT3P5O3CVG198UvMB`H#! z?xpD%y5*?iTiItT=MHz(PjmLQw@Pq)`##F=fh3dqZj+Pd`bu$03U7t|dL6@#y4*9j z7@QDSuV6){3accwMn2dm=47{fX(k@(2?=DCUpgAOP0};TP^^rrmrtsiJls{!Dq)Pe zm0s{k*8^!2&oHyKzu2Ra0=3)VSA&e-eO5-!q>>aG3t9*|@(BsEhFjcAQh4!XYyR0~ zeoy1Y=+DN>j0tn5MaKNzczI-He#gr0%*jZ*vc^bl)8#L}`~SA(;$qg{&rJ5V-C{Xi z+nAU%d+zvIQ>IU!Fz4-1%%4LvE$-`?J<|KvO$)Q^~~l=arKu(%(6Ap3FU}`xhcd zZfggI`NokRzQKEmf)S>yORkjk?uv<$LFW<^xrbyROzY~FFf9z)jhk!daeg~cg3;(q@$mh;h(v>+LeW3^+=9 zHrnj$>dru%ko1rAS45Em?wlytjwB<;lx#<2!ngO04lD27pClHP3- zZ6Wn!2ek#(QL?-I=r6THf7YCjBIZr{qk3fNo9F4x!v|w&^;1-bXQ!+CgV{6@)>_1aPwmHi zT7M{5&Fd5Us$lgG0cBlQRtBqg&l+IZO#I?B{}}(y$_D{uHNolwLvF_^iQ2&~)y4|{ ziePn>o~d#W(Xe^ftO3NbF2Y(KtnS7i39^OE&HO7jb}?cr14K7%m@N#93WC*25o%Nt ztnMX*QE{+(u`mf|;k;ii44cRXmrQW}(6KaF{o}U&4o#)TYI$|AS{yDIQyZ+lIBcRK zq&!&dO;!#Z4pzTt``zJ##P95!OiqWbtG7Q^oe#?jT3pu zbDVV4IM?=Auv$A~(UIV83fa+MgIhd5u)wGxSRF-%j#D&r{K7oioiaSpLy{VeQzUKt zyoW;77_8QAe8>^wM(sRn+{3>)S8uH50kN}qF5h+@2U?&i3|8CPFHI`1?2*EwRpX@6 z9*OqskV)VE)qzCL!b}6R-3EVopXOn|+QRxgSZyJnP5JMlkj$GL?EhvoC+HWlmMLPo zkfi^4)Bb0?Xw5KRB#LoYfe7dxDa?{O-=^ReZf zkIk`IrBWpLM{F*gy)`2;G`+|8r;%5mef-rQKg+}At9H1ghELfRotO9B-t66(2c~7x znY;F<+x66*uS#v?d@bdumC+|V?dvB_mlrKJ(>c_8*io)iIk`K2G+H6+RqV(UKaxBy z#*Dq*!jmkL^+$ARnm^;gD^u;25WAO2QI=+gfA$3X_7}qVS#zdL`s0n~wTbyTguRn>4;T8u%{x)L#B2QT{UmQW^)pDqIp3(r1bNr%2x2tyAT* z@=b*#JJQPilD1O2E5s)rvI*WY$z3T5%Z!#gJTOgEtED5RRp%}0cGw=%Kfgp$OUunA z525(>grnuTd7Ud1>T~m)9jW)0`f7z*v0BZutgU<2=}V_fWkR(w(871eZlRW9#%WK1 zXw%L*rq%)O#tJLyXI>WP^`93}4Szqc=T3?JXXgQwLT2ZT0{CBgX&yruyf8yb#x9l3mtr4*Z`VK->O);3s`F zGh#~2l#gcq-kLn#%DkzS-NUU>c8yzqKl<8(_P0kpPHuKu&(AE5Z;@wwuFwiPQT7)Ptv(qbBMxATncrCehd`9`+XULIQ#Y>+6= zBi)xrZ5x*v?R?uLl$8b>oKw-veA`6cXB^IBgtd*yTT-=gG*8~mH4zNCNR^DK>R~UZ zA=)*O2h97+YJv^wq?#W7Qmt`{>$Uu9=Is4^qR@S9FLd81FcP+i5V|il zqV<_&^9KJ^Cs-9z^`5@5WD?6f|JhpTzZrFJoX~q+%W&pZ3nm1@i*&S+-E!p}+L>>tt=-%*nK)(7oJ|v==+Lj5X*!?Z{dl$1o~P=ymx? zTSz_GK`W=Yj-nNhca-x+ucMOUBso8lqmfW3Je`c(CFRO+VfOF|QxhcS;mz7oQu8oK zxs+lnsqEu)q`D%>^N6HwC-EEO%yJH4|2B`Lvyb~rYW2Ct{m&NnKl|9hB*1>HX&)XE zcD9}1C$+^p-d~xj@}ONhTxlgKgo$EUKyS~rfyxa#mB-q3^9``~4e(oQtz0>aul@Q2 z*y;r+Jx!cF&-Myv9PDa#A*7A-fVR$|4H0@rT0&&*ini{uG*AC^eN5KQG09NzeGaRH z!K%c&s?90|QxjCmU?&3-GN&n>Hg@fFq^s;*g#ka!1|BD7eV(T1GTXpMqd3)MvaQ*~ za)_VIeU{~rSw}q-qL;;NgZg=6WwSfz)_v*W*f4Kw^Ao3c+{ZL{s+x}}_1wp_cOR26 zMA7W&D4RQ0wmhbJo=L+zgJw^j!ZWs6uT}U_LZ2}iDtW_7)mVKV`e*Gqfm>#ZWg?=b zsg=7$hH8xLdj@{)$q_Yk)79cm2aDDw$f9vIy}_luzl-AJwE9Z;yEESbV}%+@EM~6 z?IL|bMo!td=T_Hoq2u@r5gDrV(Y~ffmT!#oag1E!5E|N(LCqR@Lbpwm=k`sH+`R7U zru5C>o)dObr=n6fWrrI^hGynE>U|rLnXb2arGI#e0YzecnsaoSt)dUx7!Q% z+;WrnIFruxrVg5ba7C~~yyU)GrB;;46@~^fnR|+4;H%iKuj_8jVlwS&) zApFWPN1IhMY*r-?FIcxTamOxuQ;#E#Y+q)zHpSjF!G5jEdYNT{o^`u$Ho02vo-$YX zFLyGsoikc;N|H~=II2__I$ig4ky_MubluO+bvJcSyzqFII&rCA<5*e3fV1!VnEs;N zmEgliuGE$t+Sdr*Y+GM5o6bfyzNTv@-k&G7lufYgZfdVuXX>b9sxpxsnGx3^(=@|# zy}ggxR;TW`kW!`_#M$|$<=s!;J?3Tq2NegpA2~3hAWv+`{voqy>iVK?TZ+1M+WkYt zxT$Ny0y`f1v8(6#3R8~)uYC{39jG)tFwHYEV*knqHO&tyqF&UzZ#3TVwW;F|N8a-* zd+&#eQxh|DAJqSp8RJqnqW4V}(0&GYm(6G$y3`j!a)4MP6Rs+C7_m zXHzll(*t*BoISlJd(ovs(@i`Anx1rJqKQ%oa&t(h0x-gE5uA-p?yDn`x7 zoRWDtAgnoR*rxA16%C`?bu(#-4AT$DY;!cF)km4*d=_ogozSXVKpWll>EHP(j(%A4 z^U&UB2g|;lHt@(`nTL-;_m`<%-K+dVP1}B5b61@f`NCB8OY+a7l?|Ro->B0LhNge> zAoHUa6*f+3Hd2-6>}tth`SYvfwJ*B->{xbkPua;Qb{{=5of&Ai#>X}zFjE$wY`@-C z+^*5qYA0JCwf)FD-Pi0aTf4Y=6cufbndhCd6KciqT)lKX#qvq6?Pr_FCQPt!XZBU2 z7IeL_k?(by2ouf^TTV>#_Or~VgeC_mmq&)a<8yOax@T7#uPF26c9yG0YtLIx#jp*w zp6>o){`z6M`FsF(^RTes;g42Ain}P(>N~b=WM4QZdpIXI(4Td!PlexA%d{7T|?U*57caG&>8^T${ExwtL= zczpeWn?*gORqwk8gog~V{JbE>&CB+R#$jkbpp(=vC%~{z4P-H_M6XD5Sw0(g9BpTz3+18xr?*1 zSF-2%bu#x`cfFJD$0$0VUuQEsM&_RMJTdvM%bhpgiiU*>?J`cu&s~y@j`=e)gZ#*x z=G(X{8nkIsnUi+R-Na8_hAV2Hf1({}Ki|_mX6&=yf&*f{beZqbyiPVd$-+k0C`d_0 z&20=iv#fU#8941?e(d)Zy7L=Ivdg~sm#Z=c6Q@M{^_on9qxqvAc02b)*_G^x;+o9A zZv*~EO~b!!;yar4^fNPh+rGC@awxe5V;~%KNoX&?XfKTLNpGjp?11tw4br|eK4scz z7TTG#|JED+Powjn+URWAsY~lrm*%EgYA+<(oicTrkjGic)efSSq-z@PuaL-Qz~v{Y1^r-Npnim zq7{;@yY}Lkp7t7?4p-N-w*b`)Dr^nn_;hH)oV0B_@ERu#E#N9$xIA9+PUWJS%11|K zpkwP(WGnb`s*bHli7<$h8Kla%$tWalo9VL#n!fTPJ#MPe2(y!H3?y4F{~eW7YDqO% zM{z`wd7H|H#;MfSWK~i)OKKKTNCw(G80jix?1MsjM>nvQq+=v!Sz6hql8>ai8qfR@ z@;H?bd!dlk#mRgmxjIF=Q`YQ0mlEToI!UEzC3z@%nv)ZWC$4Lk)!x?A&Z8aUElZUr zYWGoBQpuPaAIT)PPMNa_?^Ny6p~;m=oD)UFHyl_}#(yDe-HMD1(hjkAWKk(sa4AS8 zXI+w`Eh};7@@JeR6>0OPj%QhwTDykuPSQ*fZzLR565~^~hgXtR7I9KzousFj&V@z; zwGkyOQ?RDi6au?r<((Q5r#dCqmV@moiDj5y#1}e95h9&7nsg+N&B?W!qtIG~bBePR zpFu1ngSW>tsg6NOrM=_%wlLXIDaFJ>M`BLhI1!R6^H!*+20}-BZIo)Gtrg6~)=edf z&PO{{TQ^ivTa9*M8Qc1O469XZ>zkgTq^(`r+o45#B5rMTSVbOGYiwIfGI#B9HxO|$ z_c)1}4YWBiaMzKkb!4nrSHX4+M5;;OENLkk*=0ekz}E9Nz~gTY?b@a>`~SCf4LAR*S?ki z%VoGSGxK^gyM{|qb}yT_5c6-|Q~&neT4`qS=Z#)NZT*n~wJ$2(qD%D64(R`zw`dV) zAyiDiru#o{&i~Xm=Yozx%%{Hf$4Tv%1(Nl7(SGINfcEPVss>Ifp{=^S&kFk3k!pmV z^0F=iPx%VMQ@&PuTo_7gg`qM|kCEF%TW`K&A!4CDcM7ztjF#-4KK{$aK4Wd=`Zt&Q zcFN7S%k9?w5M_7y!S6Nf_bTzXb*n(%tc;6&^MAiPqucxKwYsx^-VrvI8=cl(cD~zy+Y`5@sHcbXa2ao%sp2NZoC@XbF)|y!drW< zR(oNPo3(FmOLJpYw3P)ka(V4@V{aqwIx9thiW{`Q-;6ePaS$Vazb1Hso6llB7vWOE z>;Jqn`^RNSE*rkRm0D-4FygN5r{bZUTgall{VZ-M|8ck5AWqI~VzRu;v0#G`>7;Rr z>gpet22UERj4r*s{wG~F_LjadekffuE|8kI^mm?1fTxJ(dPveOW4UyWtB+q9*GU(Q z?@MQJ-guMjO0dEIV(-1fqD-2F@0p=TZeYM6g9L{tC@KgjVqyq_3Q90XFy{ruoB%N= zOlwv!t-9v8y6TFem=n6jRWawBR$ce|^*qmW&hx(KeBYnvkMBBr?Ok?PS66peS65fp z-S_ZXbp@j9MEP!5CnT`^Ok;zmDfGx|YF6vgDvoRa&j{bY#QA?k`2H&zhG+jb!bkA! ze=x#Vp9o*Ld~p<|I})rqLTu0MeKRH5r8wBVaf-{4U>B{I+}O~v zG#{cnvUR|X#{F-V_qmbPzjSE-NqcNd8uu^#&;MKAaPokl!<*_iwhbsT4k+2_aATV- z|CbHzUsBuSY~%j%cl(#@_HZljpTFA{%93riZrkEuB6bXL+vd?h<1lT<2-U(Mt>SrO zn}__rd_wGT_eu9`d8zGI92{TXN|!eLY-mzxxX1M2ol4{M*Rt#Km*QvCN-_^g(%A-; zZVNsg=6J@KSelh6LToSCCfSGB+8dKfvfM>hqQj8nvXCVEA?~Fiwxz@Y!#qljiQr4J zJto-KO(}P1em^25q|?mMq~on(7r8jbIXISP=}WxqR>maSXC;;yk!Qr&Z2jC&+p<{0 z+1QAwY2I z{erRXlzWK~UkFLcw~xCL7H6NESQhKxalx@H%TN;L(Of-b=HrNx@FCi`q)QGdvGd$* z%408NxnHvNh%L`9^-3*?b&qcoAxCAK6dRp#AuB2NZeq*qluNORE$2)zH##(b6kA#z zF)7Bq`NM4OY$6m9DGn*FMSN>rH?~~gEtWCQAYmKlakgB4DJ#M}EBS1eyTZU^t%yrm z`k90E6G9`ql}E(h)6a}Cs5hZsG`MoF4xa%@w zRf4efFfXc~pVi<@nSOPV`^gItX9+!bbDv~Khutq^8J(L{Z6iD`3~~1@*Oy+1m>CmM z8SbJ%GDy74fD?MED4ek>I z-AmUgt1}hf!0A(q3MLt5TnwLc*l0Ov#PoYpX3n@2)~hVQ&U##*GNqSK!fHdp)JtJ= ze8V4Q1?2fQNi9ow6xhVNYCzt3WfJ)8fINdl9`*`v>ZAN*2#fwHpwwuYcX7xvgJM0k z_Kd=?=(4c9!T`%ARhtro^C_6YhciV1kFJF!tqOZw6n1>X(kj;jP{pJ&ci_~EVb&vR zCzY+6aj~Fx8Tf$iWeWHW7xhn1T57KE9=OVLK()dd7i(|0?mp?dXZ$J8_-&R!)!gw= z4_R?9VJn_z)X)0|EZO%~*eBY%nijHYj zswyhgldqa5MGl+hY_L>qd^O5_${eQ`*EEOrY99UR8NBh()Vs?MtygUtwDYakk*zCt zR@YS-c*7}hX#jr)xoP`=sd)i8hIOT0$|Nq-oVU!&lnHa4RgEC`yaI_C$9oMQuj#9< z@9tLZv?j#!MMPu5D=A?cx;exhj+kki5*rw6)P&EB4)eUQ&$IMR`@?G6f~GUjn3m{) z2uVr~mx}4H2k~Jr6C@-e;VEO!x-le?auwwn+ru<^mmH6R^&YS(uXmn!#l+%Uoi$(D9*b4 z;i@pUy8OXb5Y8(d64pj9#uVEY_Li?j+)Hy;@*c;0YUd9rSBx_*adI_!xcAmO6g!I2 zZSId#SDAk(Y?dsp404a}lx92-RJ*S^T#T7AOuBp7BV&Piad2U=UbJBo%JhgCy#I&f zjuTf|kGFL=J49Fd)#it>;g^DS#;m#%2S-@X?>A|%b#a{ak@M`aE=Rw1Hm?q-fm*VDZUc7Ii`fBbmC#!G;N0ol3)9j)s%%c%_xgf)w`! zp%F*Ip?6r2<$9Dvj!jIK>*I8XE3Xeq4#B$PSqIL-?Zd%WrmW=r)#i9Tomr%NagNR7 zPkWzR+wB=(JWH;<;_7B zqyNlRz5eqimPN+ke*Yfxds)*dr-uK-FFpNdevmQ0|LPT*Vt)S>{olXV4|WXukC@8MjH0MFvlmtxy@7cn1;{F%qKjjw1dO zRp1*{K?XXACUIzqQmIf3Ww$|-;}?*lnk`h5Wu(V-Y7Q|pgQA!BAmN-N13QB7T@-lr|OTDBuv zq*90Qt&4#UgsCbL5TitmeNw|)?H8^JR_CjA>Q73JP-$IU>{K3z!9smX7>w#}yv$UO z9WLmf9Gso61tQe=%F~?Q6uwfax^j?D(dWp>mpVb@2<;dUja(E%00@=3n~VrLs2nW{ zR0)17dEKvSqEWBV(S+i+PoYcdvBFNJj((iyt13X0!a|jkOTaYGH)W1GK^1jErCO-* zO4g{90s?1#>N%!}+Z-8ji^_>Q-+Tg=*#^GMP(`YYwCZs_LF3E!63j6%>8KnO(4d;* zrQRtAtk!!ioY>v0Am^~1LBEoSpl=ib-2#Jpj&Zr*!>EA{3B(Ciz8OvkVZQ^lReHVA zRc%8g)>ucF8TlI3N{uKo+OiDXl4Zb93)93o+y*sc9CCpEbM|6^|8-|hkrwzrLv;m? zPFo%Px3B5te|L4S{|(js|LF40|4^vz^kJ0iU-$B?rE>kZp}IT!diihM_YV#G{|ME& zI~x8K2DS+^!zgvj&XE5M$u({)Dg4O@tN2`CU;PTImi5y<#X7BHqzF3ppHrFeP44umF zzJ#wzs}+2O#TvHnv!wR@=IXJ>Cu5$>#38mE^9X1({wi}=Y$ncI_1J@4BOjWnHB@e}0T8cX7(;U4(ug+=>aJ9?CZ20Qpq<0+7Bhv>nztEh z{c;(J9ZgDCTV;N#swLmOsnu*=@c}9eudt1L7^YR{xU0-_O<1+;#Aua$R?eY#eU72x zNZL^woQRRbNak@-ndQn4+jzwm!oNv0&)0PoeR85TIXQ|PgM3=kwCdgtjY%>S znW0NnnHN>iFO?FKk#Cr4>mxLH5jL|)gj<8k`>r*p8W03;KEv&VI!gG@v`};o zOM_0~P}nbn%AZ@WnspX>5%Dv*zsNP+NG^X9Y`Pq?P?Unc)wKzVLu4E+Tz)@HtB6A% zK$ln#*XqS6C&;db!_(|Y*lD)d!t<1Qe`u@KE4iW_#Amf2iql7E^}6xB%Fn8Ul~vw% zVHH&EfmYFJ+IpN*yC@ghMSyCt{30m-@q5pMx>J6F+PCgfTVr1iyGeInP`a?S!v%GK zGW^z2N~hkY^{saRlggnh8go)>&`lZ}c~KoKhD-%rcN*yGDe+nt5jcvZ!+0*bsJ=7E z!}%6^-Ry3ixagrg_RN6U!^#j$nLZDK5(u8mKni*(t!0h0m(&qTKh4ij)igs&?~v!0 zIzgA!2GOG~&CHGgjE<%L_o_9uxNCJ1w2EpT@>dV)O4lozGZrOx+8@BdG4EiL1+nPe z2GFQ2)h22_P|a@TSE?}~I=&F%r_q5ZxD35jR5ybW6npYTR zWGaPq(*U1a11#Yt8g-nB&QJ;`qdBTA zB-(DLE#pqgM(zSQ?*=UU1@L4aVBBxe??XCsk4)6AQ!IX&&#;5Z+eKYK`cGl7*$9Kl zNl0HmVbWEv;iCRTDlVT(&rTN2KnT}6Ekl?GtKUXIFT*J7S{u;34j>`|_UW)|ck-Jy zSUEQTFT{0WRHbX`-irUH4YXC8!6fnw=E>(N<|*U(o#zX!`+l}akjqDe{>gwObrw%9 zkEEW@v!24*Uo0Y2abz}-(S1eCmB}y;RVYtmo;Ez=c-HZp;VI|&2yxsuUf#4OvzF)f zKMI8VUtB$xwNFFg_T9ooS)u;fFF{;E+ImB~; z=Me>l3KV#gku@@T=(gx!P8;LNoaK4I^BLp-1tc%Wgrx=(e!~BSTmKFw~v8H`n zJ?%$=x@5=1ortOU^=GdZuZOM*0{n) zhs^bYEm{{$AOJwHDo1Ugorf>ht8*FmQd? z3EGv=ma6;E>W)CDo>v7pRHLVI(Yy(xr)gM8v4BOY42T~{cNf2(Lv722E8%=^HE6xn zWsKyrX3;m)kCoqdW`b|V>QPOE!-Uokq~W{?6cg_8rL>r~i4F zYI~k7zoQNicRqpluCjvh3<@RLB>!)!cGWoEr30(}Y)CV?>TS@E(D>YgTf3zbK(#5m z3~YPuI-F~=5eUnh)ZN|n2Et0zVBB=6uj@U);tpCK58*ZQ!N8G5*^v)zAs?Ip=-v}h zPm21xG&AXaU~$|yP8%V1M3YQegJDvYGJ4Hg)b$#2?;AZqYY?+A!pcG8G`hTA5?(RR zCV5yUPZ$@FES1ULG0RA{#QZA3v8zZbsH_rQZ#~IM^-1motv<| z*JE|EIQEnwZF@!)xyMX?j>cUZ)ms~&{CN}63Bw<>wRBy@C&;L@jt7`&7&D<_lTlvX z#Q2Nlx5#&8-;)d!r*T~w`LY~v&2z!Yoh^GQOv@@vr4J))C=e`Bxu(P%7qn%QJE$)n zfM=cnKQ=va*8nK;1^nUc)iyFnUZ|>3YEPy) z4;pFOXqE(71PI^iFu^8?_+Z3>xm4Zb`2?a}2u8)D7R9mTT*B$p&vKAE9;Tf)SlD)eleLE*~=@v z*t2Zqx_uvAfUM6iW913Y7o85y|bpIwudUz*mHD=ZAk_tWLtV6pfN zWtYL-KhMA_Umz##t=tz*U818SeurP@8-RRP51-;FqSswkfBoMzPcw$AhS5l^Y<2Ee1%4{#m;=2sqe4PlYV{0Eb1%hM0UU&A2#hJ6vU$3nXo zRI72wu7E^<^f~rTrP)c{r?J(K`eQm>u6-Xw|5r2U$6VS^D3pSKw+aruS5vZc*?z?8 zB&TQXr!y3pv6NKPq3l)TEG0$kG)nv@Tm{{pzf{k>4yoSAQH=9ICjAF`G76MI+0>>o zRX4PQ)}v1yk{z;14secwYOHy43{ODrNZ^i$LSWh-TUpq%M}$3BXNu~hpmZEBDeLum zOoyAHb8>SP6^Ek>BlBJ&uZ_81sCWF-H_F%Jg7R?{s!`5FwT+$nLfu=O#<1x2jJ)#_ zPy2-A7`FBvUH%@ZdBtBd$d{Ysg?KRao;JAxEU$V8l3PL4k7A_>!4|A)$gtxYmZRZof~1SRHsK*eHRM8iOXqsC zkaW9BT;8Y`iV9d~RSl5Vmzb)9Bv!FX5 zL+ZpxNj8K!bx3p^N&@wl+R9ZEg{vis>Q4%E!|+1l$^1xW%}BWZJsl2>=Ku7~$ksq|hn(ljPR5?%*A9VZRYR1A;#7f^rr<}9*yXyO%`4wDZ|G!i4B)b2^u{X=H|0NBSGai+X?tb$WduoX$mW425g4A z!A8o!(jD$qu;Qv^nH7Zr$|r}K>a#Mq6UNVSWItyw*9Eh6r)Qhy+x_^C~91H z=W^88d&(aA9>=68H-0H$nYbN=h{;v0=8;n={ZpwvJ&qyw;bi3ATlF}Ge+rpGo~{2x zrk`N$us}k8m3~UY)$a)n%{4o!EfjHG=uAvHV76Qg@or~ny)Ua+QO#zMZM7GX=U+44 z(3_vWu_DG^sqi*8M_FyJAmJXPD*?wQx=}i-JssRllP73kMhE7$A)ABZaMM#8tO@|% z6?X3aK}c^bIhYHls!*-COug;owq8Lt->y<;Nm)(ixgrHher_;>zm91a>M0Ng*ea^2 zj6#l)oz~>W*-gXr`H{(nOT`)R)XCWj1@o`XEvEpIvlPFTuG#T<6u%-jVm_=8m9Sby z0#y=im&+*#4AGt!kE_yXMd>Rxh0{X{4+g>8g)LI-VhxJn_zOy?Dhw0usU}&RTbsTv z$|HGYe0`Fe^GVjsjV4(kkNP{}QJ6O*<{YFKL*$H3hV@b~2`891XiN(-_aqsNrEotD zT^}|kR6If3TXcO$(*1A}49$;F8>La298SZHJ1~6^Gm*V!DD!q0Y+=_4%-i|uKY{gj zehah@`HU$YRw&*XuZpp2sLd3Zw27_ZKMaG-c8g$xNcicev26<$?xCh{T5N4^AUN7j#M*2(|kU8C-JTHiP7L+cSO7KHYm99qo_>cI@t{8Du6yuwTG+(!3`d;Db3(paNJ0k&Z zD7~mYn!1}?tYWAqCv2fBYp(7E#*Z#z*qqOl>aH*7bDx71KJkKq1t`QJ)a1r~C$_!h z15`HeCy(X5`@`&6Cg40vT8#ZliEpDkB54tHL9YK!QOxW@GC%Xkm1u25F@GSFPbMdZ z(p(ajaj2skSWKv>-vsR6Bw92Ddj%{G;6Gf-W>nrx$)!Z}ao`9~>={TAOaKh76r9d_dw?zp0E)F~aWW0k}ALXDnM{KM} za!%_*Ou&{-6wU2M6=~n@bf-#<9+b|uI?PO+b)>WrYe*?7KhFHP%HAHNqv8M#sIdA3 za79l=AU&0CrqV0!B&XmdD^9Vf4i!=ow;C`Cr(QYxi`*R69hS=izf&2gc}xcyn^Eo| z08OpoftxDwZRkY;eKd)aX;tQHpKB9z_1!d|i*iJ<9B;+Sa6G1|fbp{5$2KDDeg#o$ zOJi7G+XX?r-Q-_u2ofrGze@u#>4iM)Z+d&9OK<=zHw#%+fGDHVucl2Z7o0Q~*x9CIwk`mqy#AR8>t z0sSdsBNlyfQ0LF=twJ}IYK=h;gwyfr{+L68*|cwC9|kPivyl-*pMx@)wXGB#JI&P8 z&V7z49$B~fSIl{vC2|+?)!=+9O)mTm`HvUr=zjl(d9)^W;LFkTjtZk=Dg1Ie4$5o%9iHIQB9&u z>C^2P4d>p&_YoiNTPOit+qYKrCYt|j1MyyE=)OKQcWFzansb89R+yFVpw_P!@?bW9 z=w@o|gpFx>Qwd{Br%Y%jka$kQT6!%VNrX-#fxjtTW0NVuM-g+UW2O4dfalV=@N^11 zpq;D=grGY^Z#(c1lX_zirE&9VK(6!HAJHH41<>XXDB%d){3D@k)eveNp~m0WZc}6W z1B!2pmq_}JmdXQdmv?c5MQ&4!^CS|M9AZ>@ux)ud^RU~QVxQPl;hl(0uPMo!!`iY1 ztjoz+BvYpibW#n?Q&7X{Eh$22g(n-A;@@w#pI*3RGYTc<0HgSv$|yEV=N%VE^#r<2 zov77>RjNE#`I|Pv#m)XC!=`hA|+b=Rh{Y5uFX#!=E?T zM0}49!S*X9wTl_mK@$5)=*R4%aNf4G0ezEN5Q(2D&KJ?6wW}zIACbV?Pcux0L82en z@G5Z0gfJlhE|HAmDSv6Do1;-Ny#DpDy-^;O@gIEDl8Rh7#8fpd%srbtFD zbd9%ag;%ghI0)a5^iH%pMTuPYKk;A(l4PYZJ&3Fjts%ZbfHy(Si^&mZ3g=vY%ORrP~@!1#paz_}Gy zAhUb2BmEe=?$tZtR?=hDq$MusjhdOU@{&Y+7=T}e<7r}TsCXuXx#XDGpEMx2LzLMIX%-0}2O9g@u zSmaDPj2#$rj+2sZYDaek&vc>7Hu!m+;6;b1c9K?;nE?@Y0T7GnGrfCw4^La-OLBdv zVtC|&7{i0#VLB$ay3FF~LuNG3NuFEO82LN)A{mERH4V%QFw?HVLq^q#Z=ffaNz}QC zp4gT{^9#8nk3e66_&b>;k0E9aDf{ie`?-i?OsekFaJ7XOq@s=g#z zje^&Tv2?t74j>|z@jDxIoR_)EgywbjM||zqgfrYFr)o3Y&SxO2^^`I@#|VymzBG1Y zNnNJ6m)zUU*xQ9|Da8wiN3#H*6p$RQX$O}h2W7}-(M_!-8_LiXa_4zjK{76hg|X6j z#Aslf>SJkSNEBRBn#q>7Q2$B00^Tzoe`{~0PFBY4BmA$M-=hP+nLr=D>BK6X2${5Kil`dcpib`C@%WSCC0b%fU+>oMfI z2Gs`H5=Y*B7ptpl8XaJ#*6Wgu73}5v;>h}NUT=&Bs-t@XyN@gf?rcchNUTU?ntal% z9C+(#Jhu=0&yz2~V{FhPN4BKd)hYhL5aUb?e8~5S6t7BsNwqHWNKjiI2|6*Y(xx=- z$h6A&eIdQYdOawT#yq1Zef0YKqqeQwsFWYezp~>l=^n<9jf=c~Flop6Mj8;YYBcp+ z$B-Kd;p#fmV1BL-evlW~P->44pRpgT&fS%f#l7P$_*(GKa{EB^XA+H|3@ zB`ZyGrV|wNT#(%OyD=P_q*ZwUS&eTA=uj(ia%IR3V)tKoG?5LOVm@#8lxu$7{-r>l zVcQYNzQcj8UIr0&g{C~@sj+A0YSbS=z4{{=qff0^vUSLoHKIXp)JRb>Ut`M`zIxB( zZ=n&+t|I;o=AsdPvOGSTA{H;cV9i{9_@Iqa^@89STZot=50f2HV0o}pj$41!2J7<2 z@HO>$Q@D1dbOnkiF{=@Yrgw~l@FH~$^{899K21lCjb*H?OJ#{-;yNJx86fU8tUB}dKSBxmTiUUn?{)*#nMYu= zX($xd!vVdSHBpU~ECYAqy7UC4`M8!tZ#L(FmRl=!Lg>e%B!fsiWB_8|M5+b;1aMdf zQCIM3+o{X^iyJ#2Xs{DB-ipcJy!$JH)3X9hA%IFPO0;)`OEQ(0wxpBuijE@wb<~_o zs8CdvPT!uqa|maYlp?ZL-0Pmlm^v7Lr|;u1wxTcFrSwZh|J77^_A@BkjezbnD~j1Q zM6*}0Iq!M}n`GZaA~F2Jh9x6f;x?#WQ+Grfbv1NBOl<{9>SP1yM_pb7YQkbb<`Te| z6@Xn*#0~KRm8CIPwUhtx_H^?m@=dN@r*F>&-$1$>=G+EGzm<#Br7DY5p0O3{j*nUP zUI+tUNw}pWNDR30%q zwBJ{}gUWW8mxFxTQ9+sV2JX7Q_WOA7;!4j^K*>JuONB)C73KJ}5H#p}S}m4=u1dBi zR}pH-O)@qRaIFTwqc(imb(CM4JmP>Dg!xp0Q$E4+*B`8)fxJwmv1?>@8Bjen=naAF*aFOVugm z3p(BaR5H&n+>``Om9qH+m&o);F29L#&xF$qy%zT zrh7fYM7s1sx2hOhZA1~|^&UQ;HLrIFj{~o;`oQWq%K*N)JY?3xS;xwPJ>Sp^(>2YSsmM-_`(V`X3@U_)F3@T$H+i7K&+Su{uawpJjZKQh8@1A%~2)VW zX{_eBw``<2tD*YA{3DalR~~6~dq>;WQhyd5r$Q!;H_Z#fG*$%ej0lqNW@~|E%?mRk zw<>2c)p&aHV6ROJz-cBCZX9p%LI!WW<>-b|B*SNU^B6>hwP0Mq%;Gr#cH%~Sw5F_8 z5r$5frXy}ERsE$!%7!4TRE`$cNaNo*$!MwI7xSE zd1bnNhJ%c_J___(P%5@mQ$$-v`s~LbiVTQlXY+C)Zi)Rhz1B0E1Rawp1FrJrgGoZ> zlkvEn$TZ}U+oUwa1anMN+hPh1BnI0Qfm+7D*gaD`(SIs(mBLQqk$c`#TC<%7t3)!3 zd9H!zJe@PZ1QOodyfTqW4U&udB1r3*kQ>Ov^V|o~eipL7J%^#W%8A+y>IG4DVZVUf z6^LXbOBOP|&1B?e^B0KDi>UQ+DYZsR<6EB@6`ywj*6*g;Cn$7#Xx$U?eLQzT#Qs8+ zcYC36+Xu}k#66>tXWT-eHiCSRhCnTu|8ryhrQWvu%_+@iUl)=;UEJ-)6|;p!>mB{dR6*1o~^9!aep zMzHHE8@LjCYA9Xodm4(Tx5+n|KL^(BQ8Lbgt6?!>jI=(jeyA3%PEgJ$7a8I2NmxhS z>Ap&BZ2Ucq4JhX&gA-)K8W9&t1eFmeiD zZ6Ux+x(0RW-wKs!W_Q@am&KA@{~g||M3!#>v-juxloqS!J!mSNoa zjpXPtygV~)6er=ja}k5-bgVrW(E5?>D7S^tQm0;>0XUumbS|UT!MqW;)tlludWr{f zp}J0y+Tx!@bW({@odY`0km7&R;4-lO1-wET;}a>om(czEU8!qqE5iP!VyLwR6)$Oh z;q=oD5GS&T=qiWyC4(6MD-9aidC*s$ByZfASfQ%8ssLulQ37{G55 zEC<8RXEI%{%{oyk`UD`k3uB~VosSQo%sLb%$6F0iDwTsufM3f0hFMRq3iZ{Q%6V@) zP!2wxpmr{3(SVhD;XN*oNN0x?t^S0=Bh^N_e_0yIPFg^C1;9#kz)35B7He&aPwr=I z>>SE`w@sMDc~a3c>e#shH|Jd&!%P7S-jAzN4M98+!5HB#2n5==Kw(;>a&38A`8gy-|kd9lnWR z9~&z-;^1DRnOt@&6CpE?$v_lNNfQ|Hm{#%#(e=e!;E%TnLF!JQUl4`0^SMS>7F?$* z2kAjS`JSs9j*ICqtKt5UyiKJnJh4=p@UNw$ICNQwwY=0f$;fq=cRx_SQYIir8n5@` z0oT~DglZOC*pS6IjzCIbqZZsIUwaHSG+i-OSPDw~ukv2=kSzR9CE~brx)*K6FzeP> z_(!WJD^6SLfa>ho1!!o8%Ios{%VIbzd=G;E6U@A*ENwxB7l4s3Y3?}9y?hPl{gp4V z(bbnW^@{A)rA^4=$(YN`z~P!$X;T0zVhw}6ge#zI`|Tm7^K>bc;w0cZ%TsCF{Vh|7 z28+)!9ASyWX1eLyo(aw2kvq``N-f6W=6I6%S9Fb~{;#(ma;y5+Q9}ghSR|{WL5$_W zr4_w$aeZ(sOiras9zF|BmoVeCV1*@&C*c$|{+1g;gtV{D@BG$@xInV* zRjCsUN=+bGb(KV9F!Z*epdXU^fm^Ne_KI+e0PPY9S{iRc5Mp{gT6jq!y&*-#jo@}R z2H+73NXD>iRQqc(Q%~kNbMWV^iRO)~aEndq>Xh;W5Y8ZJIf+IBX45GT{ z7*^gz%G+E99GB|rgz1v+qZF&)_kPKyNof-Tub1HzE~*P?Umx&m1HiMpfZwG(pD69Q zHC4sU=YSnji++2{#Q(DGPj1cZ*-o-7_dDw=2I1;1pH#=@1}pP+nW2tr=5d%3J7wbm z(#i(auZYd))QS9?ZsfzN*^)nk(NY%vBrmA-3#dwkzEZxCH36H?QuNNV1`0W&em#no zNnwqS2V1i`!6?P^QVZH~(6C^XsjpIj?LX#27xV>^w4v~8|V(AuYcy-&7!1tejFg1c`pY->IZyyqa zm@w>~c-WT-8{ZH5Kctl~MT)Xa)*MrzyzaA?*9+D9!s+=gO%o#Gc6*?5ms{^pX-E@a z_h)H-_&|QtXY!Vs0UWaX!`Q-%R-`K!AIDDb;{&XW(d!}+!EUw z|2)o20(lt@O>5$3ga=b6lia*ou$di3?l!qT;pDPeFyf+A#maKZ-!%uUZUN}BhT&a| z9qiXz9EhWa++NqocV$*+g_xEI3P)0GOQIWplknrJndx@M_*96zx=?$AaX8Xk7frHV z+fhv4+IG}B196x(AVCVr);;4{|q@1C2{z$wu? zndwQaU&Z$X8;-blnq+FiUUcF7s{52J*~qjVTwI|kH-1_#r|?228|W-J5h-t~dXqeg zkyd^Q@uNyuv_CCXu;Epy%7u=gDALn3y$ix_o9>&*Aevc4izpGRXUP7}06!Oik1OD_ z8$j^{1Xp8JSX*Hp#Hi%Eivlt?F+NJRF;=_I^exJuXv!{u@7s#R7>H)`$Vh_Cr+798 z1KK2N{R!wNm(fut$n#v zCHG9_hBt=;am)u~#mdJ>p1}Glzn*@AAf0c$XDd|uxHK~+m=0^UzRbS{t0G_4i1G5H z11l__>1emoKrEb5jhoCl9;9yggRb-h2Zg>}gcMk^qc% zZSuQhC(ioRrxX0nEgQ8r#y~Wc*g)?Dl2v4~mk(~H=phLT{92Nnhw`YF{^rC#VC$=z&S(bUPY7Wm9A{d<(Q%b zz2=RJvLX|O6`4bzhNEaZl*yF}{%tG+)sF-1!!(zHDVl`gsVZF&c`6YbF^`H*|0OJt ziLXd@{);aHOt&yOB8ZOi!y9$^3U++Cw*JEN5s&Xdb~CaNy=Yhu4uNbUQp8Hv5k=tA zM}q4z3Yx|xYY@~PxpVD01}MdmP(J4V1kr~)l3(&RG1!be8P|;4x{Lldl1|>Th5JAk zZo(f?I=DXtom+?6D`i_=dm)H_@eLOCc&54u(|nV>7Ak#)RD*bbZV0f(ed!)uKm02d zcbtPfZN_oVydh;_&GDKWUV_WWJAj-gbo*Dj9hy;>*BR5S*>03AGc%|*U7DPg)u<-A zGC19>iJ>jjZI$09G=OAS<%W>7U|y7~6PH4=_7Ax2xkdi+Sm^=wZ&N234|0|CvZ7{z z9y_%mc4Kajj?{bK6T}~CWGsp_B!4`$lEDT{-^#T9`L1}Gx_8QJaon`=gddXWgMZKYN?n6nw{X*LNYo2tBhj3t4)_ZNH zk>4B1qtf&J@~G51>IsFtj8rK|LJp|C7$t8BluQKWCzpF8ho>0A`paHJ{$>ddvaYUo z<(BGSOuMzoU{v+?V}1kCbD@$NzAPz_{8B8pGR<4wE}8E`Bv?ssC0MN+?h@Jx8RodO z0X4@v7ueE!E^rNNnu`X$@-$cCeFOO9;aw`6aQ~wSY51?ES9r2~MSwLHuQvgH!*vnXy8u=)yw!#= zci+!C83voVX>+;MeQqFymOlXYV$?;4-{H5xVHt<+hmItB-6q6gdiU1Y^$d-)tpw2% zZ%Tg1`@+ztqf(;&QzmNkQ%zgKualpFIQD|ZW(O9X7qA~Ss>;>CKb!V|_bkmm@` zcc?1AwumxatF24Mmser(jhX4pE?o!d#$vrq-<}M3ZxJDaSq35!qiu?EuB6T-UY3da zU+GnUjhXhf*d(`#32YUn58|W=aFJKQ`=BafKOTk{Exn4F=$DxO8@1*O!S{Gtvfpd~ z6)RHH8xb8kq#%d|EKzY!s=OyEZ*u7I_(jZZI*Lav>ON$UedLgF9Fz<%OI~Z@6^ICL zB)u6_p_d+Oas$*fiFyudlZ_J#`Zw8-l+yw|EOau!mgYo%mK!VKwk=~*FrAww$52bU z<~QZ-OGmU#-uO{|?q(niCLQq~rN4!jpoFaQfvp9aU^;`mqpZ~$M!xxHSby9C7=lx! zJWcTBP`M+6ec0rrFYgLpRX2(Yr8?9a3XRLsKqAs<2!147sxD)&g(TwTTd||CQz~Ty zai<{~0(p|?&1a#CW!G8^G!7JcD%y0T<$;pnbBwf7)-{4bdC`jLfEh-gMM-~3TW3?6 z)U6@S;hYIyIT-1bwdx1Oqd^p2SQ|&x#<-eFv!jhs{u{gkz~RQ;==LtD+b-9jt2$+1 zd){qqcFtruP4kj(Z_>PH0`GVe$r6LHr^-UF#nf%Gcsc0W=Wuj%i*5|P{vb2cND%R8WPz{tt#3Z^^jP5GDY(ByjA?BX1W{+V8*=kxDD z9y!lmyD9jQ%z82d_P{ll3>IC4{6c0QnX_bK_L7mWhEfh7${i%O{zmWo4$&qzqe-Re0BxtZ4Dkq2H{G<~k*@q+%ul1bnh z$MYWQn3p)heaT>`MTJ*%r4@;r9L9z7n}4)nQo0v!5ro4#FhLMYUtKEOP@ZMbXTOK_ zQZmv={LUj|dHRpAmK`Z0llckOChf;l6>V|dL`_V9sKh}<4F8+?KA>&UBS2W1(}e6C z;t^vN8ZEGZA}J8yj5B3=;WJG7N<+gz-D(K9lnU4@+N+|%ezI?5;Wr$s0Jjp+Dd%^rT%$04M`j8wZy$CB!U-W;t^BDBsx!F_{T$4!euF*U16vnZD#1US_ z3O{KmSB(XXmz$&ObSgJWouN%O-M{|B?MFB^iUHZ$dkGx9&J@A%?m}ERWnl!tJFx~I zLaE~Xl)ZyFR9+;#rbaysm+r8*m5-OLAzl$GotXE7BB`S!+%x z0~W3MN~IsS!8}Kvp#csNNp1KPT%h+XL!hW)03>61DLcJx2;J9-3u@U(5*(TIn z&)QHroTWLXRWa^`&h@NYIxfqxW!Yr?ifFoVa?7a=8)PH7 z6_t6wq((H?H-#3liPK0zd9Zgn!yMXyLFw9@Uf-2lKfR)H41~4?B#qkw=1POJ27@C; zcL40{1Z&RMN}Y?_vELUv%VSwI236^@YCl3yzo^ws-CMV)Yp?FSZ2naWG#J}z+N8P{ zqJEQp8^V6pSFZW!(^M}TzcG};+u%I}0Sg?0wz*#(p| zAN@gla8E%@$8w70Jpfm65XI0Oz!vF1wnybmck&6Vig{Q|Wsl7nZJ~_an{K7C_Dy@H zwj#!$5t)_Hy&?K6A{?nH4XHFFy?7S!{L82|apL79Cvnpy!haf6)2#u6GnNK-L=?EO z2Bew3MMmq6b&-5#k!j>XCY8(#G7UV*$n(|I%FvgPus4v9ae%F47J1W?d>?vpkzBBh zC%hx!D#Hk=RT-aCHOkn(MGMZXqE~HDUz-3#Q}7H@*fgWgcoLo*@I^MdBBD_VRWne# zlR-631NFDumLlY~ErVk1QD zmO`{=Ib}VH0QK(!syu)yl6qYsNS7ZZ3??J}k1q0qgt5<|DR~RcGf;=$!Qi$uzSX7w z)ndchxYtTpN0y%aa?Z3-eU$HGx?x9^R$Q|Q5FgNL#ee)YmehtS@_K-T^vL0I|CiR%MP% zN{cu0cRBhr`i8cb4YBOUVIXf9W-PPYf&Sd4dr?qnAgiJJfp8dP{n!(#L8~VJpmx!H zS{#$fa(U2bY8C_|LogjmVXr?B%XS9Z0xN6U>aLPe^+Ck-5EkB#NzBdr5}H~}$=)%H zAqbl&CXWXkngBS7F&3#a09|EQ%Vbxr<-AJo8oxwt{PHTBm_MJJo0nElymvKV*m^+1 zWXdX84j%(dDx(Y+R&4l-o*ej2?mqTsviU8) zwxTWDm}z4zhu|01i#UOu6MA%C?5qpe773^v1ywv0&*NbcE-hLfmcsN3->L&*kXE4` z{|B1XlDavrug*}8q(~PFPsvC7E8rrO@7&*rDliTEk4*c^;ZtGv9dq$^$M!Q6z?C1IbY;zAB9ELwV`{@U&N`G!O zLb&{lsqQPCLA)xl5)-1>i;tWp;e7Qp&V22d7u+(CXRQQu^+j7;gCKfex0x(OI_58#YO zo8AX?eT<-{hpYK4>#w!Nh7fmd1n9`}7x6}D*$zbSI5_RYvY4(0N#i%C z6nRTo(~#--5Ph5_UWJ7yqwAu}IhYTk=n$FO=aGE13o!hET@oIsr)kSAlHc>bL{G&F zM5T;8Rf}7oJx-I~a*q7Vy7f>T>!g47Ayfu^<|I>^fE`jCvl3D5MLKM^qFtSZP*T@W zX1W+OU(VzwTCTK}I$K_gm#K0d`kC0#gWTc;w|MD|)cRD*h?fh>) zGsoXn_-7%hfBy8BlPci9!SbVXTWh}G;{4Cw%G2@ZJyn`ul}J`q$dtdxX}MLUJFl{N z&F@UA?Sj<~$!f=RwR4WzwNUMLNbPY)U0J2^GH86FG=AxtDmj|KLQSlS$MixogA~eg=X~*nKig$)`&lVV-PV>A}(FT=kV*DqS+ylbVsyMX;Tc^v?y(> zbZwg)ZM#A(f0DY>9c>p?g$zT5Zc!C_q*v&bQ=v~`g?@)B^gmx=;Oh#RdhOmPLBYmBE&)lP%X~TCOXw+_2Sh;~~pUcPux*w%n?> z+8%7RGsDu+Rs4`%@l#I4KMN~no~lZN2#6e_O{4TjyL`*CJcDVq1@L+sbM?FE2YEqn%%dU6owBz#_Y9#dg)p?P{v+ zL%i%mjrQRg_H}aY>lWG9E4FV?Zr@1lVDxf`F*?L$IK<~VG%0duR_u^e?$AQ*nBwJ_ zW^`{@kW;k87`A@U8WSdOe=PoQSLHJ?K<1b zbxw%uycE|Tvs@R>c3o8Dx}?~3S-I;9b)_P&N~?^OR`Y*KrL|d=*3GW8VOyn*7b1Xa+yoM+Hk;jcafB3uA!pF z%hub=KHAH%tCw@0m+ML|x5Hi@cfBfWybX5V-nG1aQ@#BMc?Zn#uDabj=%ROxx8A{a zKDBE3gr)k_9^?}-$0u^TPyLHN4d43ko4nDrd}C968xN9yeyizr-^7c)&ENVa+xex| z@@tvu*Lsj&+c|#ixBGRx=-2tJU%H)t*INGFQ~i4m^6x#zzwdVc{)hbs-1YxKQzg@` z${_D5gQKftr&bx#waT!(DkJ7p8M(d6=!;dxzO9mD7m({6Fg`k9Lf3#vc>z;a2248~ zFyn5(thWILc7bzh1*D+8At4qSFIaQWN7Lc6LfYgJtxU3E>@sz2pb zUB9yG&xfmSx?6RNrrI{UYTIj7+nHKz_n>OO%&E3-d$s)+s~vn>?T}s2;aWi@sX@mE z1s$IgbaG|Tsl!2M?gpLHRKMU|y)3%=<*wDQxd*Zg*|=J&TXRrbM} z&|rRTQrj)qe0;FQs$eUA`MfOH=3TJfEX39)#6Bj(v0I4q_z>4sA#O)PJnn^5Hml|3 zQ_Cl&mS4A8RmRr}Tve;uky_R7)v9S08sZZg8WS4cEws+~(7LNa>m3Pga4)oxS(wo$ zEG8x_u3K3A_^>9c!kQfkOS%`Nl85CM4JFU*p!F7hutut~*ozZ1=#=gUN zM&$ZLjE{+!&@E!p_=qX1BBmXQ;14s-GOIh=r|#UCy7Rl$T`<1xqE&U59I3nPUfmUD zkwrd{t70P8bc_6HaOAqVksDS;{=6e{V_D?pcadA|>unFMw==EYxZQ*6?HONh@2Yyg z9;tWWUcKMm)jMQg|8QvilC=8A2G>76xBkf;^-q`8KYOqKIkN^Ad>WL+G`QTY!Ikk1 zuB~cteMf_vWeskU%qSj+P={npGNOu z8hz;2=+pQ{f39lubw{IbWsSbSYov0B(u75cmQfY5qm+43mOG<#m!fRm^JfQ)c40<` zmPV&+qsu&FrJY9iOGeN4MuS7Nw{NshY_wnZ=qmZqfvcmdl|)y+A6-+#g!smU#>Rwq zkEz3-?pPgDuOz0y{g_4~*614>6B`@XJvKf+w#n+)W+kyn_hVa#xD?;GwAi>--TAwn zaqU*ebtsAJbU&_(Xq@5OxLa)F9^D)F%5U6fb>n^|jR)Ls{DX+k@{J!H8$YCb{ILA^ z5v$`zmBf#^A3shcgX##54C3&%IAP?~rscEa_6qq~EiX zuFgxkzBB3OrKDT$lkPY)zZ=&4eoOuqO7q9_nm^sy{Q0HkFW)zR?a<(;9wAi*jrk9o{mwRm;XhS|-eI*>qRS#LF$4e`uNP*eb=ZRa#uDmaST~ z9@47q{8sJPwCZrQRi_87x@cRcJGSl`-nx6M);))`?mfSC-(9WyUv53%LF*s1ZL<8@ z437JMxVsOisJ49H_e^sZ6%ZvDwh^-;B4WaYy zAW3r0Ib#l(6K1{N{rdJB-M9Ok*XNvj-hJaa#%KKInrqcuu?nav|7+bo!*y1N*V$ZJ zXLoCzebqY0kL#Q|YCB77ADF6rXsx!(K5e&C+U~csk7j8fd!&85qmI{bos$c6oK6|) z_#M_ceM%?bmQG-nPEeIj@JF2xY2EWvbua$80lJs>>4u%sy>dzSYL@P`N4nQLuIFdn zjj8Kn)~=7;w?6*V`rDV*C)`?}RJA_kf-95#4%$>4dG!OKSmuRCsd zJAA|Y1sgsZZuoR~!N8?fzlEEmv^L3DZj$xgG%$9PeDx;9$D0Oq+B{^$=HC`>9=>h!NSDo{d^an_ZXR2` zd3@XEi85Ozt87u$+M;5)Mb&qUTI?3}>MgU{w#@Fdb?%6*^A>Jhux;xim#s@KZ(WwX z_4mhH|LA16a)jaPg@$Xl8Lo3N)VXZ9KHE_Lv7tem;YOKln^m@L)!Mera+|U5Hj~(G zJF~a#dc19SCnK{FM&=8R_H8q=axt>GY-E>hk5yl4>8Xw+f?CN5C$@{Qc4uMr z&f>P6r82w9Rd!Ws?W(riRqMN}K6Y1Q^{(c&U3X!0A&un0vnS70z!V|MWo%aqIx%an4dxsnC9qGDvl;2*ZxV>X* z_KyFwcVd6@$s^6DEHa;JWIoN+e0r$)j2!csPt0d`wwOE8Vjeg0F|t_XYOy5LVp)#G z?@uiL=)7-b|9z{b?OU^MpO)1=ZNGiGar?M$yuqh^8~a;so@Tjqo#i$wOJhGvlQ_$r zHI}BIEcf)c+B?$9e38{YBP%B>S1X%PE4v&khbLA}ovrtev_81V`mm9;tE=^qQ0t>P z)*dz1o}a8w^tbVzX5+KS=9H0*pR3L3P@8}_o4^{I;3qa`JKLV?Z+m{4?ZtJrm#u6= zU2U&~+Fs4Ez4pZRdS|=nk#;v1+1)a-i*vQR9cq`DW0(BI?oMa>w2}51i|jMk*=JkX z=la>_huRn9*cUyqFX`-1HqxPDkwcY{LyfCLU8qAtjziNEhnCKc_eMHCSmgM~$nlA* z)i9HbMG$u`;6M(Z}EO<_``a&hYda--YDy`S=D8$w#zna z7h``HlX#b%wJxTgUG~Vj?p1ZQ&~~-7cC~VIwFz^z%XM{l>gv?RZT~2@gNxk`8@su> zxg7~}JDTfu?5W%FE=Rm%k9eyd@zFlwYkkDu|42amk-*v`!Jm(W$hx0bb-$?Xe%abR z%-=mc-aVq$J@T`Al8c)?+8)`~9=ZM= z`SBiwwI0QvJxXPdm8%}B)IL^ieXQ31SbhAl#@b`epO4*@^}Mg@`Ear4BV*4eZl2G= zJfG)!zI^KWy36smqmI8{eEg&F@lS5YKl>m58h`wI?eU+Vk4p?V(Qf*Q4mu|~*_`Nd z`b4+eCwkPK==J4<YfxA3A)~#2TjD)@yZ6W= z-lMO0kIC~M_sn}j*OQY*pPaJfc4-q z|G_2xhqwE?9`QeN#sBDSe~&tU&oBNb2AuYue%eRpw6D!+|I?=fZl4aUJ01MxbjW~! z^V0(^>I7W22?#qK5Pmx#qAnovOF-0sGdHH6iP1R|YjY<4^qGX)XOim9qaGMe<68*Z8m2=pFaEb_Sx@sXMcV#Q+T>h+_j3cn&&lPVQ)oRm=U7nkK<{Jr(!@~;+LVs(X2x?PcSD%O(kzch+Ct)q2^q_44j+p=L^<=1W8OnS@%ohuVaP z+U18jw1zr$3)`<0c5rFfVUsXd_pl@3VMp`BjvY>B`BaS5BE+@pHd&I{eC+ z{3}7NSI%||Kc^IaVQKg!lkiaY@GIfrSM$TK)rUuZ4UZal^@iHj7~QL}wpZiauig&7 znwWnzx%KLuZV_op5gAJ(vP>d!+#~YBBMR~(idrK|x?L+%x>m9DT9wJQ8ry5N0oUpi zt~J(QYyNue?!d_VYLO3?Mm{o$eBvJYEIjgge&oy6$k*MjzZrP_o!a#ey4TxmuYV4> z{x$sixBTlrTCe}=7A3|+wOiImXclYQ%W1{;liBlyE%ErP384BRqSr6 zp1G-(cvHRMrqitFH)nT`nL8$C-m;hlJ7N~v#VkG(votYAvms{rx0n@jw^q%#wPyV- zExTLVXKv{x-qLHhW$^9RM!DF{Gh(-{kKJY$YkVfwBr$epL#*ky*gbM_duPO1tdFy_ zi?coxXPX#j-w@~cEzVgk{=kg*L+j&RcEr0Ljdwc}@17X%(Gc(XE&hbuZSNVkeb(Rh zwY%+q=5|2h?ZAfH!QXC&bWb=ZmvDYY!o~Fomvcxy*u+|k6_R}&Kp5|f`N-szr{HYO=!SyI-Hq@1Hkc~_GP3X+PRCzW(hE*q0v zu`Ic2M{>>4m8~*WK^D9dqaX zvO6Dl-1&6$&X=oqz7^d0@%+xO?x|vIYJ1JpjyqF3d!%-aNbO#j+Ve$f?;dG=#-{bt zOq1T3*54y-Kt!5cVVc5=v_ap~xTD=r_4HwS=_BmZM+K%UC8dvTOdtOueFAsdlFyi| zo}sLlp<u}Ajyi0mCn**hDvO}}UF zkId%Q=U z*Vw$1nt7*o=K0y@`3L3&B;^G*<^_MxJKH1w+}QjJn)#P@=7)OZUx~=ST9|+BMgH|3 z1<_**ZfX|X+F20iQE)q=AhEC@`9;B<9))RR3o|qevvwBdcogPE6c!W~7QHAe=}}ZR zwx~k0sA^|XjYm;kL{URwQPYc}=I=#!<%{pD7eCZ1er#X-G_bfesrW@>@vHB}Z@9;s zddUa9k~aI2&w(XhlS;lfmi+u)BB4;)Zf0o*{nAbjrJawJcD+{Gy{NQDQ)#asrIHF| zeP@6ghkl*tB_4NNYRZz^+A{82Vop?v7f@?rYrBOJ;{1(ho$myc~KAOE9#qC&;w znH9?V6)FxDszDXguT{(_s+jq*V%Cp}ISQ2;Gb`uoS1xp@TpU!n^jhVzqDsxC%H=;Q zSM;n}Ij(9o_m|VJ(sHQM4yw{kuF`9&GWb!oQK5SC%<8TB)!Q7ZjgM7tzgE4YsCw7S z>fJqS%*NH2|6a3iSB;fJjdf6sZE}r$Q;oyR8mFGM`^VKD{Jr+@u3Fb)wMVYi9!;+G zXsY%6QF}t6&U7jKI8B8TYgX4^j?3@djlfx$rayIcy(`3ulqyB-~Vm-{o$tfM|$2L9eH0V<^I^_ z`{RG!pQ!j?@~j8S1`np1KA7hDV0z?(8O0A~zIrgb*TcEvAI@9;aDnN=MV=3rL_Sf$_;~ZI$6E~^Z*zQX9Q=5D zB<~{3~kHIrv$7lY*&jM1O1vWnm{`o9KvGx3{){6$M zmmOQff?LB=S|gfUBY(C=DL%h3>v@d9^H|5{@xjj%Ql2L@KTrAjJXP^U+V~e4%U@)f zzR2->ksJIXKjlSX^NZr2FG>|(md|=wY4Ebz@nvoB%led;4aF~;UcGGT_3GaER}YrI zdSv?QiRY_lk*}T?zk1R9>ebIzZxmm@oAvs`^4A|tUw`s^&71Zk0`Hp0OLrJBRZ^nU z0NL(_)?17>nHm{xu=t-h5%Im#L8kP0`w=&sO-2-!8gVPVfB6%Ee|l4qH{XQx*Kb1e z&s*7W%Q1=14j(?v%DH|=XyxA#qU?^?SdVHc2@7%`<~QCE3()Vu9U%qHIw^_K@~J`z zV=xxuFdh@|JM$-!%8IFC3Id0xiZHAgkt+7%9+u8Y6`HuCkt#}1hUDK<#b@_a@$G1; zh&`Sv2KuIoxS&)Ka3fVLDM%F^N>fEgXjP|*(pr{lOcg$FQpG9w!Vl^1SkD6~iIv|| z#VS}!q;U)KG?65kCX$hYf^li0^Y}E;1CP|w#A7_cQ+O;%69;wD#38ui@lz>@fZb^# z5W$eylO}$%NE5?wY+srvbxITa4yB2PRw)S$mo%{e3$YP7$i?`hX+q6EP0YZ2yg--J zX(A|}F>w|lIEVANfQz_<%eaC&ucRcx$yD+x*}gbUbVLWtu1FJeAybtmX1|mcO{6GbS&q;~0|!+>j(RFoO)!=HxzDpg+?C zU`bnnv?3MBK^Tl77>Z#SiP5lT85`Kb4)$=s80L+|IE;rQeG|xuaHj2qNwgV55FKW zFkJ{pq7N3rN-LjO7GLKXus0$UJ;=s$CAy10p3h(#Pyk&e%pFfUyM z&rcU07NiTgg&ezO>EhYSbRna~xRAvFY%=8dz!=+Mg8pV4D{CHOmo8R&r;E05_7`=D zT$@elLbo|xtcL+wAEt}%ZRz3%e&QEwMTSTmm?5SrW{4jnGDMg08KQY|hFCQtLkv;R z5Wis@CgIV{43WMwLku;|5cljegk3vniGy&#s{I*aHP&D)mLA9u%dj50(1#ujumLj; zW{6oBcQ`{##9EgOp@pCLh5fD>;vnAR11#J!#6E04k|9hmz&%3@gknyH=vkB@mZ)V4 z`JU1eUb8cWH%`I_`6z&?MyA+}j=iNNHf_lio3RC3VF)uk-IFO=@eG4)GsQc+#|J#! zPd{4m96<*&MKI1H1Q%d=BvV+!3T>>%hP1^e+Mi)Z+X43Y!t_@-(RRjud}I0mc@T&2 zooN@+4X!whBXGx2c;Fa3aU3V$1#g^$4^F`se(=X>1mFyQFn%BzgkYRS2+rX=LJ@{E zWFQOK$U!dhP=QKRp$65c#YK0HJ@Qb1{*uxX^NwbU_0WSp46p$kQRI~=Hu+_Wo4AF` z0jwWkxPl{r>@z%Y44ycSnjqGPI@F^9jc7&-?&2Qq;{hIG-o;F@0Ixz>&#g?cysxxG zC>e$;2**`i!*%?@{3tR#mg9g-WI_Kn+m5}=-$ZVPIqj|FHW*#cu3@ z8TP^)7O=r#xWF4Hp`MZ{W?&X(Lneb`ibz~X6mB90T{APqr96%wipn#EYDK1)j#qe% z{&kt+H*{}cUmG)p=Dkdz@ibF>Y-QXZ%!kcS&Z~^HM0d$7(E~lvu1}V@(JxCZk;)Q= z*ajnX7@j3MV(W7HaSDf5X9*X$!VNPIWr=3@ETMHGORU4ape(WTLY4>&%M$)kS>kRq zeYlSYc!()Cvcw7m<19jO4&9U39`u5TqO`;@^rkI|KJa8(hLlBr9A|m}=|!G|HwH89 zL!QDA+C%XhhG95Hz?XR=$gdfZK<1{8S9Y9VYl`$16m+ zVlT{Lf$c1_kF_TkBAV%59L$9V7Ge<=V+odG8I~gzX;6L7dZC6N2>h5O zq7aQsZERTlDIiEk;RYi?hR|B|>lx=WziSF`9Ww7=wwJ4mFtJr);+Ph2%lm zA_cEmK9x*EI))F<79%hkO6Wa2Tey$P79MzpPm{7m=9FylTs2#~z)MI>&lUph&>r2< z_D8ljwkcc4Z0E5i*}@ZXd$PrCOgx<}CSfwBKp9h^f@x62bg02EkZlXj7IGo1_hPol zhFoa2Fb~TX7T5=^E7@Wlj$CJ3aTF)ugUhH_;%6EehARliRYc$#8kiqRHj+)aPCE+G zxPfN+Zjv#$g;>NP9tpUOL?j^@DY%0)q$2~F$U-)9kc&LzqX1>7LN#hoi#pV!0fAAR zZ(NJc7LmA)Uu>85O^*K~p*g(4jxslw2jrGwsf#A-N6P(S_-qWLI(* zX^P$0gKqS7hXw5(WKY=8?nU;7B>JE)9O-vPKid1rgOH{z1sNP>`Vd^upXno{J7j4e zB|UHq$Kizm^m~&h5rESO#6bE(NjdT=DNiaOhITw|BLRs>!X2a|7p15~3-00`?&AR- z;t`&pT@1$zozVqd(G7!H&m?j(&fz>R;36*JGC~oCD;UG_cgR$vAsu(o^A^VtRq>p2 z)Swo1s7C`D(S&b^Naoxl5?xcW#rV{0F#%QSJRUWu#Y>FJ;Q0$B9DzH6GP6Z6&cZb- zTeu+-*HMic+{n%rH_;)7eTN+6F&lF*7a9n`Iov`lo}m?8b2&y>4Y~Ymv8W(h{DBo% ziB(vQr9~`T#j^OLnqyj%4 z@c|!^o5|xUb9gT&M~uN(gx<{&_V;px-s>D;fSDh2gnNfvaTKEmv{~Tb8p!1np~x#C1d=VgY@K#C0rMm@5`z88q=b zmctpl7v+jQFoWf?TyY4aG}%Tdp;97fU^agzBZu}OGMCK5 zVcIUpr|n7>kZxold4wz?i%|l1rc24AWEokG3RI#B)u=%&>QIknwBR_8S-Lq_ti&p; z#u}`}A$T$WFzEtU9DzIP(SSxYp&6cA7!R^rbHxZl<|EZ8SG-0Vi44H+rEJ=LAcz6q;C$-|+_q&XJZ_L9WCqtcDzY zXUIUDx8d4>5`8ZYaxIW{cuD&eUgHg%=yS%nL%Cu8SUQ=p8gP{A}zhZ<&L zHfCWC=0XGWFdqxC0E@5~5AYC=@EA|9+=czYJ9Kx=6(8NWCXaD#V#bkXxu<7Rz77tSdBHPe#U;c<%$@^JTX!&PuMS!mT+60CywAKJa7!2aA2Mz zj?+E?Z+PJ(d~ga*%=abz;E&S?z!?N02*Egu5S+t#T);(K!eux!PAD0U2wcTAMB+N) zkOhl1d14(zmi8+$8jIa&Hct*b;>5tP0K=iZ%aRWC|ty&;zP>UWL zr6uafHvGcN)dj+A17mKLmUz04v9O)C33gy70;~(f83ZC2#vTRYtD&^SK(7Mvjr@)u zkk}?Iv4Fl^&^%cnmSZ)x;V@jFajHPme#Or5Rfyh86vfvla{@^sEBMZd2 z+w|uYh-~EIbGNwQo zQ=x)sP{nkpVFuJO6SFWIb1)Ygn1}gTfQ49u#aM!+SO;zBKo{$A{8fSQf;U3(;SFQ5 zjdSt%E$0ON-xUa1Xwnxz2I3vYzGvGp1+$@n1z3coScXW%ApuFqM28QI2aS(x4>q;2 zy`MSnc>lFPykFo=}7Hma_?uDW&x}z5c$rXyh z*gK+7C{AM<^DrMT(N48c{McJ4e&QE|Ielo4-~)x?EHVxiiv3}Q;s6ff5GvmliYnx^ z(f-aDKMF~40 z83aONWRZw4E)s`Ki-eA&v_!L2k+=sZ>muO{l>Lb&{4id%Q@TgY}6bVS~W| zMM5^ONDRP0l++iAf~F$TdcR0K#|v0KDiUGei^QUi#o}%6V&OZuSZE!VmM|J!ER3-o zCfI?UFjXoRd$1d37{8@hc$*cAJLbjW)Ba)+6j3bNl^2V74aFkASBX%SDG{DZCE^&4 zW1xD87`mxMyt60~AMg=Z9ZH0hONp55%l~U5WzZi}m{!Ih+Ji9!LopoVFdmbj0(Go& zEfK4al!zZ5CE_Q3;mffSA>mmf_Ia0xaK93fl2amXoZ~0@EGIPUwQJ=!Wj-fu87v-jGCJNMis7LLLfG#2^gDPz=E^Br*OpQWeuN z18PvmOw7V;%!LN#p#ViFMhWiX5k8^j9@hyTq3}tG82glS*jge2FglpOgSA3;u<1x9Z`tJ4V-FUDtzGwlTM|= z6qc}pHEduDJ2=1*S8)xSyOfIHuB9RaeY=&4Nxe(OWF$(K3I*9xu?J?juE6?m6DJ3i ziiES$5S8D~nVXjo7x8gX@1 zsqolbD!kx>Hj7d**P&Esz{-pLfj#yImWl(=J69@}MwE)(2`pPuDkhYcia~Xy!Vp)V zm5K;Bw{oo7SQo55aa<4^DlOqjdLfQ>JYxTBBObSr0O>EK;>^!d5r`lJ<7@je@g3bd zmx=Dk?OrAf!+8CaEEC4K)`$7{f$oZBqBkTF3Clrcye^gr>zQR@?wT?Yx4ul=*j*+< zY|4cBu`;m_{%6X>?>G5fA*@U&Lj}_?GP+ER!WZU!#im%0w*| zrIv}!xSm@kq7aQZloyqW*|&IIBCAn@S~TLr{W4+nuuN=+#nUq3{Ju;ybuAYUdY21b z$#T&oRW6!w4^~6Ug*EIUJ-l4V!*pD^xQYl|LnN*v3emWMimBzI?~HPxeOp@M)|_$? zi#Wt10f|ULGMeX?ix%9)iIwHT3qJ6LAI_q5O}Y5uP%gGOmJ7$j)3bjLi*$JgE!eC}PrXWbQi z9$g`#`&RH7V}-bh7~DcE;t-Dn=t)(G(=rvJ_oxbyI;KK&9$O(=rm#MAo5}c4URfbZ zbSp$Dly+5!QDzn5xqF4U=2;j7IT~Xz21*zQE9OlkCtxxrVG6A2S0<-I1=FC4 z>6if<=BtylFdK8Afq9sZ1z3nhSd1lDie*@iKd=HTu?nlP2DXg5mej&JXhR3O*o8Ov zg$d1!izQfw6z`GM z&FK8RQane~+e+~b-*NJNr8tK$1bwI!6F*jp4L@j0REc1m#cl8|x~T75B_5+=mn!k2 zXO*b!T_su&D_JE<233htOsVFxk0DjU4$jz*1K{S>e1ED+bi$G`RbnZ+tz#VRD)EO- zl~{q@yQ{<=v;|g)PuO*-N|@s9WgeeZC0vTDge#6X-LDBw4Yhn-@zc+es(@el`SAH*RX#&h~!pyQxwes|s~9wd?SU@VLk}CU5u30XTcGN*7p&r9VSBnl4t3^j>Dp!l&@c|$IY^zp_KM;;@czT!L+wmMP;G@p5 zCtu?YKHweR;v=pzztc>PHRSOd#$pMUVi`2?J62-ItZFe7zhM~0VjRX}0>tcU(GKm= z0hX}BRYV{Q**L4gv4r)!YGH%<^Q*-IlrE?iWhh-pV&5XhK;*J&5smG?R|^x2Sb*;f)Q~qAk}V#$YT$ zojI2WS=OC)aJ7gFsTMvD`92L!A?aeZNJa`$k%n|+APd>ZK^}5Zh(4Ei&V}-vLpSt5 zP8jo%hkTTx3>7FxC5~OG7M?f`_eatan&DgrSdNX@gv}_w$~7Os^$dMHz`g4{E|%w8 z9LwTvJm)Z_T0}u9i|2G6*F;MS?c8fbN0bKCh;rzk zsS(qHYxoXX4c`^3;rn4VLNTpIq@~x0Vhs3MBNSlPx0dgp)r#_dwW0!*Sglwq!iUz1 ztEd`VD;!m8MGGc;;rB5l&Z-qjNJa{ZP>dGHYSfAW*u`=yb!){ctVY}VTJZ^=p{`#m zW+K3-R+yUA3XQ$BLVAC#=ys%5RQuElS3kDpG-Czv*btUKS1W$ctQE(9@STmET5$n> zv~%-oMILq))QSW5YlRD5{N#P!N44T9KA;UR>H9=}#w*(2$R8N>i`Rd3kq73DzL=|dL2X&UOCmS%6_FeK7Hc8ZpE!YY}7-1WBV+G4)kst91 zZTO5;^h=96v6@_qHPAv#`#KSaJmexD1t>%jicyL(l;bCUL83#Q5RmUwCloLg!?B=q zomhxPSd1lD2E$%#CycQjCfEye?1Nz+_5nsP#-P4+VlakaD1O5*495s8!cOdhDR#pQ zdy$Jg1b;21nHq5uJ6RM7NBJCWHxm_pPC)A0$ zj5?80Qz!0VcsCh|5g3VFrt?sMLKI;X{i9Jyy9za^h7x^sWIe{v9*cXl$C3BR@#F*Y zAs*o|p5Pgtq7~2a0x$6juki-&@E#xV5pDQ{FPOl%zsPpA92>}B6vki)R$?>UFp*`C zl9Naeax!_0oI)NaPrwV_@WCngLYet~q(4q00A~=0AOzzqLU0b}aRC=GmB(HtFF}R& zG=$O)!xe<%Dk5+Vk+_a1MB@f-A_lh*i#WvNHjW+5r{%Gx;NH|9*}`7c4H6B@H_kLLM|u&z*X82 zxQ-}9;|6ZRys1uD;3p)Sd9K}Ko#=3%>;4Jnpp|og0GvS}f)I?e2*Eitq6wKV*>{Y7 zT_@z)>O{|<^hwl@6d4*05fLAOStM)r(%}jSoik z!q2!~H1Dhz5xeU}xOu&}iqlT4%ekJ}>DBG7(FIGS&t6u10J=}BZ#Zh=ducTh+ z!vbT=>qQ*=+v-J^Z}mbFQjkG^41gRIunCTE!oKhI;&}H4A*s?J#$ysDV+x!xLRLoN zyhek#0INj}!W#E98^nb54I;~!Wvm*+8>a@Le6m4&xX>U>${IvPU4sahY!r=s8pU%2 z4v~>~MZU&+e8D$-#}E93=-Vhdpc8(>a42C6#z7e=QjMaPvF?y{WIa-8ry(5;Oc#(v zXr$eQV%lXWML8-^i7M2f8g-~g0~*nc7Tm*qJisG7!Bad#D_-CwUg0&~;4R+aJwD(g zKH)RI;48l22Y%ugB&6AY^hSR)v)!`f01Sj2TIiD}?~)4Seexa@X%E6+48d<0hT#~A zsd&J0D&#b%Vmj0?1L~NKIWUAVOt2k0unVTx12fpd4)$<>Bb<;7UztYX2Y;MK0D^E2 zPZ8O_QB0C;6q7LpvoQzF&>hey)+1zKqqsS^QN$n?ak#70D8j}y^8REa?@uxgt|JQ3 zxT(VO@SVZAoy&HuWgMKtdF0}u9_L@bQ6z0(TOohBQAoYwJoRYeefTDkr`05$?rsum zlbXco)Fv@4uSrbDOw7V;WGTx?Xpr+T9}BSAkj6-if;M!Zi}ldM7HowP9B}|gP{g)hAVU#`DBMIW;t-G9 zNI(*jk%3I)AQySaM*&K(r=UsL!Ue8y!x4BO5ElxY#8pHf64w!h_#)N??cye(4@*dw zaO@ELXO^+QwM}AG1Dcw|@&`@g51g1PBe9b7B3F}ZVSr6A#5Neg7xS5`*8pVaR`Uu z0v~k$&@6hOCwf5=eb5&JF%Ew~3+teZ_0WSp>|hTEIN~fq@C$aInTHi$I5yab9bcP8 zEZc4Mtyx&Z26l*}&z^L{5xBz>$6@<}Wnd3SIKdeQC0c|l+~AI*@W3$y<11r+!8e>` zIt1r%9v5&Cmv9-O2*VYG<2#SJN=D!Z?Q7%>+{8D0$4_{-ZxJWq17DniAN=9esYMv} zYY~Z3E&S|k;b&xv$Ur8tkPSVV7NL)vDJ|lVdW+~gyG6`h!0Q}j=Cp|Zki`JVK^}^T z#C1d=bw!Kt)M^nLhArZsRNPgO7}ep=S+X7e{7>-w`wv#tmDwX?Pl$n0LcUZez7c~Z z+YO!AZrF-;Blff#b)ub8RJ*aI?Z&@pN9oGR6Wc4VXs@!Tz3PefYEkXgOWV(S(|(R* z2aSmx=CA0ma8HNDCps*R>Y!QLVfmX5DhDVGTCC`7xu>)BKdYzfvZssNi7xI@T|7#=c)sa!Lb9v( z#I8Ooy87i;*zbVL5Jl5XkWl+j($J@((Op__Z6M}8D{ne99V6$9sA`J<+Q*s@IFsUa#KtdL!BU-NfD>R`hP$)BE#@ z-e047e=qI*^G$DwK9cPwNp@H%*~v_@i@3579N0{{;<<(ayy6@PszT@Bao!F<}!<3~ zPc6EidRf0&Z~M*ZBc(A(YW_;8g=SKVy`+{#OKFx#Eq^PuqL1{dNz!XpN^6-(YkU1! zH&?Gr+TgA9#y&EeC&_GGDYMN?#@I{7BwA)?nT+XMnLT~_@14}&Vr73zv;Njz{cWTB z+n4os{AJ<4P~-^!lo^LG_<1EL26mJJAgJ0PUb!1I&-Nv+(d zK5{oE$;GUci#3yr_mWG9mP;y=OL;4o+DATpl6>Y$`D`=!Trc_jzZA-SD_`12p?s1; z_;97-V>894UW%>JiZ99(U%gd)(`V4TNrOJD z9Monu$mz4!ps&$`zLyR9`F4;*-@)xB5ALvPaHqY4yLb=oc4Kgl^1;2{4VLUXr0?V* zQmcl@>>VQOJ!IgGA@b!z6yFW`KTsvNe1z${5r3A*o&0BgT+6*9t^d9_?t#gp4y_vH zvUil5_bB%pqddw-dH#K6Tz~J;0XIem{%u*@RZ5rlDusC~h2Kz$C|8Pnrxf)s*2KLr zwy=C`@w>65eaDqg9#^?)T=m{@wcg|EZ;Wd!AJ_bD+}*z8?@u28@P8?Y`)>T3z7yU} zp73GSgtomCK6_92`uEjvJFK4A$$Vm$lM}n${A(%Pev|r6nIyG(l8pHz*^`q7-kc;~ zF-h_Lq{00r51lf3*y_n6%qNdJIa%rEG}do}ywtMfKzqwVPAa zE2hkPKV?ooWsNDy^H(b`G*@1HQhDi3Wle4;_+FVhxc_^K;Pz8JFh%vyYE>6=RkxF> z?l)CEDpWn+tDfjL-FwP(pViZS&8PdHoE~sD%|}rTu1>PnlV{dSjx5^sB)@Y2d&=}>Tp%kMrwo+sK2aSnR^Cm0LQ(iMq#bTbS&pfr5 zdFqw(W__49M{2%?^8ER0<}b9Ezu0H~(wO<0mGhT>n7=}5!7AkiYt}5#vRI()vp_dy zfnMbTgAWTfN-f;1ym0H9h1)C^8v86XiCMU_a-r#mg?pqH?Nwf6v1XB_#Ug8;MYb`E z>?;>JepuuzwfMka>)pEfEOw7s>`}Sc^TXm3QcJv*m-wt%;%l+Q-)Bic%#y&$CBYw- zgh(wtue|i)nx&U5mWKH(4UbtGQMoko!_p|JWjBDxZtv7}7Hij8?ptSlYMt$0 ztK0r#X3)Q$T4YPSEO zQnt0H^y_cwH&*F4f7HJ#ZE%08!Nau%kM|ikJw0X6dduKNmBFiz25+P{yqmh=!`cmP z`!;+&wc+cn4d1Ia{QS5Yy*u{5ax7dw6syFs(+bAissjtdE6tZpG zG+1WyP?gQYv^I~h{FkcO&XL)wp|W+p*4BlVTNnFoT^hSpvwG|Dwyi5<3|FZbuF*2o zvNY88HPnqY)T=i9=Ot{bx0$wW+aqJNSH;Le%gEBw$lBM)HrB|#+Q_lZ$XUktfQs=U zEn^o;V>e%8_gG_(YGcnf;}idXtY546$!u`{y+fz%9ky=o2&;c6UOUZPd7Zh6mAR^) zxmujLdX4$4Pv&#_TWCzPn7__qp_RpAKZ~Vt7Me8{%RgDH_)k=>on~dR&dSot%G%G$ zHqOev#>(-Nm2-dV1JkSzt+RHqvUc;cc8~k}s-_)xO^Ro?) zvyG^+jr?RA)!**MG`pB}cCl7=@qTs*adt^Hb}66iQv2JdPy3r`VJQ zluvW0T<1`2eSmV(A$-(Jvf5-dN93QT8d~D_T)X%Xs&hbT!~`Y+Y>`@R&&SrDPyIbx<2_&0dcOMX`9}8mJJsVKw2!x0|7R=I)_Ey@@ftk9 zd+2oUVLIL;Y`jOE_Ex&>J+{t!{1@+u15Qq!eo|TIq>9Z+)zc@{Zl6@IJ2~si$vFdj zG^YE^*YR0seaB9``Q)_hoTAudyDN57+r7rE}>AqWae7D*7 z8lUzxx$V2N&e!yd@16mEd#C$Z==fRM_*tL!v%T$SU+3re#m{+w|AFcLhjjd1Z2a9$ z`@8?|E=rp^ATWJ;V5UxBwoPE}>A?KkfrWK}#a{wT2LzQ*531A&sI@yMF2?*(y5YnSQq}SIF$${tks-2V4Jtt#(PB!4&z=U)1 z_2(4-oqDu?Q;b$M;DTDh1@-z1v%X%KGw`B@+Qs?07Z=)ITpVz5X~IR#`u}hxTGOwW z_Y4f(s}^dZ8)|7AY8?=2n-FSWAL{ru)Oldo0kyD0x?wK1VQv9o?g?QY^V|CFG*o&4}u-KB|*lRF^YR-4dgEG(`3K79}Yc z-FL=+we;+)Z#U=2#c0fknZN$OzwE5tzgBbh|1LOdmk@R)A^fk^W~1a1Z_G%HS)Ukd zml%I0F(ENAsUb1tTVkqQQu>Uf%=Jmxc1gKslJXOi3LBD&za=@9$|aZ2NUmI;Ty2+J zdnUO)F}blJx%pf2UAdI|Gg2O|PkC&Y^7KqfYhucahLl&|Qr^hjc{k(EhxK>b?CyL% zbLVT~o$n2Ietx?nA)ne#J+*^gYA5^DE`h1tl2Us#ruO=tDk-1VS3ONiFHOciO*SxX zU{acVW18Z>RcrQp`hU34tbEog^{h2|Sz7j4+JRZRe_v(xf0mdH%r^PQ`m**p)`2;; zNjdh7Iga0RoaJ*5sOKKi%XP8ObqmaOPs;UZ%=P@9dqO_XTRqQ5FVFYyi_3<{=bu;4 zzo?gg**-rkFh4vgKcX=|@_T-ie8CO%f*8GmSo?zbz=DLNf~3ZRldDiW(dLtCeND1eJD6{`bnt{_9m` z)sidJoBmlz+2p_0lU1l%HS_PUg*-nmky9D3umU6d8^W9!Q?@B7(>pSb7l)*h2$9uBD z_XeiilW)GK`19Ui#rs2N-5+Lfe}v=xQNjPDf~<<;1J&RMYAFxYn;*>j`CyLXLycJv z=Nmj+==gAP@WZ7k4>g+~F8}#(h2o=C|3dj#lYd$}Hu#Bc$`kwlt8lEd;?o1Oo*w$| ztQwm>>qVx)i|qfw-n)P`b!>g(dxss^i3#qVgcy)&b_i*}U=yGeEyW}RL_~`i6)#n8 zT2NF(YpqfxkSGW!C?c&FE?(-jf|shbDj=ZNdTncMYrWQLYgN2oYyZDZ+jDx(+xPj- z|9!sidEW2+4m@k^Ju_?8tXZ>W_MTaj*|Yx_N5-B#cjMgi8yC9WycBu!`_VUln0@ow z%9}s!`^WQQN8frd`_`kCx1Q{~_3YfO=g)5u-P;9y+Nm+^?sMAd>UQz|cFFm6&tKb_ z?zd%qZY##zR?fMts=n>B|F-Y>+ueS>t?qtD)8~$M%pKjFJNoK7f&1?m&fn?z>m6hF zyTN_#hK{)#Hs@}5_1)h4??#-z8~N*9bN74JKKE>6?#0Zx7h8QVZvVab^Y;e+dM}~- z{lq@^?PKl_opV2_`u>Rh_eY(-Kl<1E$=x5M_IZ##=E1l*4>GDBjNktt>->XBzdp$B z{xGM{!)aq4PM`B|M)kv)`yb|>e>nTshk4x}<@b5y9P?=IoJWP#kLK@xRC4~&f?pq% zb$`65&*O?Qk1OXquBv|g{{F|y&p%%I>*Jd4PwM(SSv}^-+Br|wRX^FV|H-EFPd@zh zNqzUHTl+lSKIZ91bDr+1e){qLr=OgEibZ?fp}iA*y#&9Xs_Q;=?377wP4p5dpG7M7 zeJfC&`c|N;JeQ?=aB;lvKP~wG<)OJX0xzc^-b>qYVs06!;y<$Yx;={Y&_@kUq7WE3 zd~h_`S<){$$(`)XSO(iHgGg_qWk7V25UDZ!1}0iaRx%)ZKoZW)O&FXcK&(~Rhqfd} z315&z+xjMvFHqJI!Y=&1h3rfZ9GpaYP}bo>r7hk*I9d$)0sSmVV-lh*iSck-{H*hO zp!?H@0#OS$@KsTKnIQg1es729_iq6T$EYcjmA25VARroFEM-$^+1&7}d7P86IOPKw zD?KSFIuxaae+WnA&V1fBn>t?ZjDVPUt3id=bc}~*pkx}%K6Fhl*txX ztN5s*w_l0+5I9?DdAzYrk+#X$rb;U@Sqn8;ngwU$qu8EsRrQjpg#!1uHqN zF<3D8=jEoBl-P>kL#jy;rvqmr7DLRWNZW1}CcGL%yBP4i=}CED3fpaYbud*!lXH z%GC6iXDkpmuT$p_O4}{`*70-E%XYf1-g>Clyv+xc-f5v%&rO+JwCzmFhpsN8^{!H8 zbpJx(qJ#;e7iyJ$^Mj(5e(6q=Y`jCEO!7WZmZ<7F&}lxDrd_Rw(U*8N3sTm6Qps>Z zwL2PT%`a=E-q99%*^I6wj4dE*v$|zuanO{3^!0$4(`>wHYjRu~#l@$zhTcCna>5G# zGqdKcZ}psVYxBd?y{{WTIyd6F;la7YOO(=1Z&3Vjj`>VzUZXfR74OH?c9t*tqqrPUaG0ogX;gBYP zl6B)sisUWg1!WcP!za0z(_$qV5Vd`28B;7}t#pd^i$d|6BF;%$%Mxt?^UDU!*Q_aF z`zH5g{V5U0G|*_8#0bu%bs4b85r{%O+^E zn6c*1&=j)NpmlpcI5*3u^PFpYAD*L~Z22k1=B!Ifxn+t`^oR~WE#O4856}5I6|Hfa z4T`zWnYTpTxmf|_51kpCUrU>x8m;G4X;$BleBUWya+HG|!a-uWrEsl+8^Bua3QHAO z8IX7Ir;653hi_20SBU)z{V2{B?mE}+^&qV?xtO(%_HmZg#-_ML5X|uc_eC%1YpV*G zd_#<4d?2kUWXR2CPtLmi3mLjbn2&lwk-w9f4}~UJ8FN))DqTx+F0Y|tc}&2dq|D9! z8Ygp;wK@kctY~(MHMFHl-`eZVIPU_eaAg15sWAa*C5iDVS1OtlHEHVxiMy^+xGU14 zgq#guVSyP0l$zQYE5$igaTS^>rn&r0SO8_E`}Hd^bxVnt=#~epsi-_L!XtpQTQ3=+ z4~>`;$2r;((D~`EQF-dZfcai)<5D=vLZ?`&65~XlRwTyJevHn^c<9F|(pD>mi2YLN z7WE}b_ zlO?L7yrU$nEyX1~5a%6L$W)2iO8g6r8|r2G7#P-4!?jLROSl_-+pc<3Vl7O_S}`ao z#pP9dn$2I*Yzz2I?QkYq=_}R%r?G8xz=VdAr#gvicQ{is*nCfUDqAZU?pX1b%1?w7+WO$RNfQFR&h(07tnPzfGzQ10tT2{AKMAdjJgtPYC1;eFRb8zW&6H-{ z<~6cG8&)hiEfWN25~Hkpo8toF1M*I%PT1`7^vj~J3ObdP4-)rqNyFMqYwMd!WY%?R zTLNcjt4u9q{M(rMv}}{sr-N`0|YSCTA zTn{IKkAy7}-#+kUlWOXLppx3s@ufNcID0yCqsZDiYZ)4TeeD$=Un)#xJ zW9;=p@H?`3>M_xc6_(S=ROXkXkrw7!j(riIOnms16&>8`2YD>_RREWE}YMEwLU}>m(Tv2^R{;bI*Tyv)b zto4N7XRld(-*M6p&oLIgjtY!!Serx#s976hEv*sv*q}Hq8^2+*ms1wyY|T%$>W9py zV=FXe#yk@m{iew-PFrJasnAd&)|tQ=w^o^pMV!00RlVpGomO-4Y^S(&jjE`02`A{6 zNonRL*aA+=38S?koSdYKh)eq4({erZW_ zx2QEn7e(kAui@QQ^2fXD#FU%~W2gQ0?s_Lcc_BdOY?`RMux}#&=K801zQ25N4fl$O z_o6ypTYa1CDNg$vflB=Mtl)Fe?cN3Q z_xDTEgQ4j#!i`iCLXy%FLLp^Dh#};e%fh}NNX|Y~s3L@JR3k~ad4z*Tpd&oI{RA2i z=>U<02T6)X;m%5U_h_n*;__0#s?4#&=4XanW2cmEh^|iQGtYqL#^gBLOKyxdS~IDFUE%u9^h*oMNT4w zw>oDKZe0_hI-U!koCsyoYNCtCDiaL4L74^g%+E-$j2en6lIsKli0_?4xJk!@*bV@1 zzHLe+iG)Y3JF}eE&_Do~3L~IJMhPK>P_8A3aO1RKl}8y+2%$TD=TD~Sk)1O+{jId$ zZSeuEbc%EvewwH|^<#%E{%6MWmxihJ(*I$YG8;cfn)24LV*xyFQW3=)ttvhsYqRx4*!xI0|PX5yV z%EVsn?(trA*aC=;;1mkgANIXCQ}L1AtRO}D?RN97Zm#QtcG=COdqTGzcJl-MsD;hA zioaTw{o2QN^DHtT1DBr%fo_l8jKur;PweKSpWzQz@KYzm@@jX^46Ly(@>AT6ZU+XE7)b#~DqVx9B?@qx6RSS_7F ztPcv8d|)?^L%D0ECx|uDdSX=vJR<6(;Z&_XfFWw6yXYFbnd?LKt(7(s)j`?BN@*jt z!X8k?-w?~CWkj_!LtSk*AMF21k!4-bBQR=5{54`3;C<-@^?P>n&FhW5{MJIU_odgU zD!b?rQ6)_#mP*eMl^|LgMCdDiuij$28N_$! zUqoXB+3emv8b943{trIE$e!(^F_!YZ*vN7uh+6VbCWk5|eN%7enKiN^TQEyg3W0o4kxf&s*NRIUkqj;GTIE*TZ`+UobOnYubw*);!yR3EP8KFMqaoWL$(|gw@u3ap4^?onz zgU>T}r0z&EEIo2%W>d1!>oc$7hS6KUy59Q!$&4K%j!r*tF*|%z&!tCxoD{bF=Jah} zXYDxKbLr(9Ij?n}{P5dfa?ENY-STDDM!L?*f(boI+RWnxu0+pNJ2X{p%g@FNfwrq1Phe--hNE!M!L9I|LyD zya?czPSXMW%V$?WH$Z0qzYNR2KX?G7fS~}K6iK)P_;+0z@CLvakP7ew$N_plAAk}t z5YP`04v+xwXduP{3;-=)5q}S1`r|qsFbD8Dzz5(B=m`i0ybs_vk9P-{0WSg6fE|D? zfKC8^z#zb@06#!9U!uN2#5f@3wR4K4KN;H2Rs3A04qQP_!%$& z@CqOefU&Yr>hga zBg#vyRTa+O`fg4SYHOiZTP$=lR}n-7nQVp$x>;QqvzW?5mvbD+67SNcvad5F z-qH`Ec1??51Szmm+Rgf+?D4eTj)Q_0(L1)hLp4l41}moyTi^@CT8Fh(B|&c@mn33L z#W+9Za=1$S#+X2N6z7_JK}X;^H+0mGNiruzxXnPIr9Jhxw!xGFaGvY^DocD$+hl; z?oK@n?2m<^&c7L0+T=;;nbR^SPx@^xR6{FQ)4B)iCh3l?`_0ODPK%9D7V@rd_o z{A~)9=T2Z+QMhfenV*F8ZTNlKQfoK!Qywwi{yXMCuEAy@tse6GEBGp@O0op=7P8mx z(*t@fra67!PLal9%3;LRBSDSH3>aRVxm{|E#{}!Zeaxk{+RglwY19_Xov@I*iJu&$ zD?a>vKC}M!nG~WsW?0Ll$r8+;2q`JlQ}5YDdD=6B&HVffQ!`N`u|pb8Zv=;1OvpBV zdd1OTL>^SZMRv1)JE*dWJW$V*77}Hc?!0dgm_bxw zCbm>+rnU|aI7BSK)NQ^r8B))n=Gyx)gcHolF=umO`c;C&(x7&t9Bk$XnTbMa6;T$X zCUSAj3tC4PqmXOFB#2sqX;TsAjr?Ch$F%8-aIbyg|1fR(W0r~!%>MS5r(ElJ!rXTLw@Z9mr+bUA{>}exOx;+X9%`ZCA zp3 z;W@sf$Lx;S=Ngji86jUVr_ZiibI!NBP|Wl5jIe}q`-Gn~ft8AMo2|vwfU{)BV3Z zbm1Utk4K-ZN4=g8`{fq~TyjXaJLAF0PIgS1a_%OQ9-|-MTFz=6H5XUYOPN) z3yNKCTUFui1+AK`g$6}_%&L=WMp@iicQPZwrxYQh%BS?;CACN)F2<~{Dj~2WdezAc zgOX3fJkAll%93eN7RXas@wJ0Vku{=*xId{#VqR_xj{nwm) zWa|j6PclgR`=<_v>~X0lV+}NQ_{GGsT%XcH?H6T9+C`sQ6EhKsDKN(vO6Rs}$`w}G z2|@aBgI1Psfz30Q)@vg@`|9mX@%%Z%GR?MKZi`*S2 zGD5YBVhYUqP}((@I>qbX?01q5H8_rDd=XowT~M0bP*$RMu$(2L*a5s~6htJXQp*i>pmov|3$iul*uJ5w|9By*)%sF>e!v5fI?#p zLVbio(zezsB8%HTxR9`3CqLYln8eW1;kYIsX6!a|ZNjpn?ykCo zz;#E*1zQ|S>4~*lrJKag(n4#}>*j5>zcF6F}w9{I9r9lz@ZQw3m3z5WQl==D1q3+D_CaG&SCt5mDjk zvIfkE=%#cfLYu;c2M-`}WqwnXdWN+Vv|DRXVo>4NqKd7x=2|B%85WD0rs8Uo8x-O# zt=42SPM{)PsSPw$D6C6hoK{Zkl8rDZiPLojH3^IYBkqX$Brubu1~IfCg)<}j>C~n= zCqMKNc`p6tp_C(!rEA3#8t5TnFY}tjhL%8=Ad4KCmu?tJR+!aEE^)j?Z4cb!9b_4- zW=zU_p`anjC1_U1suY{V#r}jtEQ($u2xKy-p_>!*dE;L$Yf7jeB6h{qCe{ZeHO#ML z(oQPeD~s2z^U5$)4q#U`h#Rbip~>Ep+zT!a4N*#<;0pW^A`MUHRCHUy~bF<0TeHH@Oa696#S` zn7-$`RA_8%+2Lg4dyeUTOsdIsLwUq&5+a!+t>TYHS{#EF*>ie!8CX78pDnJ6C*=AZ zXID#YV}Qa~m2Ym`JxVMz4?A%ZGEEwG;WkaV1|Th4*m`ux#@*r%mtCJe)k`dNHJ!MA zpWc^${QlElX3y>BNOt>p)N76-S5N(zA6any#i@@a?gu`6JGmj-P}y`fts&Q-yx!%R za2A!n@XNFYi3cL5KF;oQZsEb7SUM6-k$OMLpe*)(+B6%Nx75r`k1j>g&E-jvvcp&1 z%0|_@9lLt>vBYqo_TZhzGe7CJ@Pwbr(=cX>$5fVnt$O1RvYVqmTrsxrfVZUOtBo=0 z`lOe-pBT36#`qm|6CQ5xlD>TY$d%w-(?g=B{Ltgc=$8tbUfK9K_oeFNiFa@0ZW*py zeEadJ*3>CGmyONcx8mZJBR}VE`|QZoWsgVo{Az%H)6vmeM*Q%@-N(5*zcmcL`*_@z z__nwAu=JK?w}!akiK)6XBlmHMY1QrN{YQ-6d8S9^VSPxP;NsW@Vo}WHBNz)8@B8Zd zm65tPKOSyyzehSe1l$kBSR}ka&M<3&LbRt+8$K(US8y^XR^W&nb86A@Rztu3!Idbl z;d^fB;q+i=C&#EOZK!%Ft>M|Y&LIMa_qIo=$|GFZvKLZg#@%6x{Y&?sDS15W=(x!9 zKQqNIjNX6#)p>sqg- zJt?&59Lcv^bCvy%Ufno+eC7MchZ#;AUaoE(miDBoa&>FsMiV~X6*M$W*12WG<`}hc zLcDJA;Z3IZKJ!_OdQEs)S01bWV*JbFu-0mt@UrIQurwLHV?=3e`swjIMi_@~Y+5vL z6g$-KNp4*AmdZ^p@I0bZ%#Jq zI$x3$Z0L?S=HmmJ6z|U!;W6c*_~@#ef-Lu-J@cbc6Oh-I~gn2 zskdJf&7rD~zj<$WZqMo?S7$|5AOC=-AtIL^e>;3vstIe~cXm&DZRz#Q28KostR4~j zS9Y2=?f%SpNuPar_wneLw0AD#a1M_yGIzJ;ZLv9<`~<_4_OL zjW+ynw>39%%!#WS4q^L*)63?OB>?j|)xcoVl@j z2U8qZ9=iVc)#8{{do#E9Ma%8I>{)d0ap9h{TVKzz&f1<4`pqik`n=Jdcc10Tw|v)W z_bl$+$nG!qy8pt$FE?x%@!fzAUaEd+=Dio*-S^$?M;p2=-16PU3wIx{ZCOA0;mp0$ zTZX;WH+S#!mo8mja`wjbec$)UEY&T3RnX+)KVsFEuWs+?Kk$Qo8z=j`Hgw@o0T=h` z&5U7L4dDYuFW&e4cGU{qs>7e%pAkRsZEpd$yT73B^Kty^rad9v=S9!Hlh z*De0)$BMWfiDUL@UViW1$=o5`Z_Kzi(w`Q~qeQ(IPC3sQEj2yFL#F`ft?hD-f z)5dY@zczUNv}zn&1xr3%9sJ(54VC7a#CJz*R>o}}y6W4kio<(HZyC1X_}kx&Uo&XT zNmXgw7E$EJ9pcTSw?tl^Hq$kF&*&{Xudk5NU)|cyFSYAh?*7SPAWF?l|FgT`QMB@7 zi_ZD+B;EY2li)7+k2B2wY~uNs%NR*sqvE}k9gAd!P5jK2UyKs~+yH+#1^78F??B+4 z0lb5NmlBI9iWLQoLMtzjz4YfvtLZNdxS;p;$VmVGm z&f#7oG-NMPZ*rn=8l-1PFBzxhcH#_0PvK2Vf8h>-O_7sw8ijE$Nqk6X-cG(jzAix< zn8+Y*80jSu$Yk;C38E7@44V}}qX?ps)+$$t{XveF|GHVX&r-+-;7gIO%RZK7!+Ng?Op4;|wC zEF%bwV}>MHrqOWUsAX~(Hm5PE$P`pUlX3_O;^dB8PDpVIa^BboidAHWrjUehz`GJO z3}^9>$WUw2pJ0VT3+ad^&7{;nh!k1OBt=D#DgB}cTe?6UUg!=f6#?(S>b3rZ@LLt~ zU7YYNV}rgX6vB6~ACV!G2{XH!{5~^{=3Dh1r3oY>Tt`rF%B4`~8RiB%fjlIkD=C}& zcW)8RKO_I{D{`up_k46^4<_lV?qe1IpPov8`507qss7kdC)VFI_Cdrttr=3L@NiasMYkfMms1PIm%fy($*@_cML z-9?JYn?wyMB2JM)B35|R*X?;Tg~X3=Z2KZI5T70s5>+He{0!pl7z-vCxAI(;^XL-+ zsavoN2K467!H6Y%`-oAM6*zLW2e zU1X%cgjeWw3BP@h9HSlAX8+BF)GAFf?6&{r^@5WrmK5b3{QJ(njxA)^OxEGSYWdw6 zc2GJ)a_Wz5c51 zwL7_+#)d%hMQYT3?54w4#A}Ezu;t?n9B3D$G;socp3mV-`W$|)lc0NBGUyC0=PAW8 zxS4iIW2s%dUuv&ZP3^NI?f`a0BNg&))5;9FmR4F zVmBbd(p~UR0fJ5tTj8GCge``vkyeZRwYaYVRHG`(;m2|xP=j!lei`;X^6k2!!x^Y)V(((|I!%dF(?o2D0rL%4mCFNQG%wt z1iea}pI4i0^d;tEDHRSV_a@#qC3X{}U|)#9mG4M4(_*0|*=%Io=TT~-nxfs{dtmfR zhp@n5rj^CbHBG8eewD6)Hb{z_Qcq$*Xo0z@-K zcL=66sucW%Og6ep+}#??dN;~F#BA2KkNt7k2+nTZzRi&xhzQ#tO11Xn9k}y?ETk@GK_rg+3fhae8S?3?JX_cTr zntIog8@cRg9`Z$YJ)W{EfrwCic6HU6+{lXKS64mCjVw5dMaL}nuC3qSZAj3GPhgFO zitPR=ztn;ipR0F8FL+`RsQv3uJ=TBPYjY!A2czy)#cDgBjJk(K^}2&mcOK>1st;bh z{Wv$Qz(bRwDvDiw>Oi{*tdAeKqkh*=aPZCK21%s$6nLssmB%lGvK!Buc{ftKNFO{S zDYE;?tK_33cSM>H!m=$&H&(q)EanD9I%F@lV>jjUg&v|6?HjzHeXTXI^x7A_dg-zRT z>%+(amoMz5+J(mx1gELYht!Ofe@&xh%?qymlaCZ}= z#imG91zcW+igD!CrZlP~4l*{)sIp571QeDLOWm;-HrX5~b*B>H-{ltoafX{pN=RsG zs|wED7zc?SBGD5*T0xhVzH3ILL{@i7&7;voojctXQ-eGY8+iFUpM! zT;jMrMM~42%?@*}Cwa_#@4ck(>V=N9?A%DVe4F60uRFHk-pdK^rYWc03(S$m@m8$6CN64CeNT^-({zE!tag{A8yjMcRnD<$HeRBN zZ%!t>bt1Yg)uGohQfD2TWYZM3CRXXijNA8jALmJycWJ?zvf168sH-}v;Fp&3d1qR! zx`&x4>({aMxIi@4>K%{B6k{#Y+AyGniskfJV*pP{274#y-J(+L2@18WUkcoY;)D%ck_TR$=%tn)~t#r7WZ4t1lTkMf_YJeHY#5*PhMoB1QxBPun+^D1v}k!`R?fg z2d#7oQ9^cjCoE_?;&sKX>9@V(b#J@5d(Vo#%14wb3!*bp*15zxn0bE5&hb0muN{|Mrra>%Xw8W5tm^0=$MsN6+9fNsGU7Vdjf*LcysbD8J71K&+r z{%!JR#W!wH)eJ^cVp--?@2pH)@C1hEZ)~ty7Pk9q)z89{IKUc zaY5@u)2{C(?d*CXXV2LgpP+=U*jTXa#={=ZX7B8ND{NWoq}}IlJ%sN9o(xTr)eAq)p@2{TRKJ}S+fAw+GqhDsfYQ6a|L^XSEHQE03N0fEE9ZTi=yI*|R z^_g#~w3t|mDmHyZruQ*p|yL3GEnFYPf6b0XV zjOKQR*7Kd1%f~Z9HRaxZ$1`lQ(w6SBqOMGNgy!{Of$Wt+K3$vmWA5C_BV~z2ihzZmUJbq{333#y%y~Sx=hD6D8S6s|EBDvj zdL$1j=f0QimIO#+zrRpuLk8wfxVdMP=49}8Nl?_06aR2w;QV;<{-LpSFLSr@{^|Pd z5gLy}|3EV5$XL3_*0ZR8YDSSH2xB$TFQVtty2K2TxtoVABXjtOp8by`;!;_85(-VnMs!_3ohqmk3#lqIhjfPP=Bgk(NuY4P0H?3mn290bBadxoRYVO zo;T`}dy8Lr;j5n6E&h}E$o{^bJ6nRL?DC(K9pRaM@Wq=&BYm>n4_wG-ytu&a?9^W2 zmsTFX!v6SbQgQrDkr0%Pmm}kO*0p+LW6rzFyUg6>LspBs%`%X#Fd;X2mC;N}kI#TTFHm9%NbAEv!JK-8Db|GzD zin=7GHeu}D8KFMWP1U#0goc&YUE1>Yb?-_0SMu*Gl0%xCy3lTOG-DRN{p$SL`)jUs z2|b(LR6{>}dNKc0b?uA6XJ@`P=2)`v!c2GHfj)*CtLrY@co1^Uw)6bJd-7lM*Ce;E z?S$SrVqiG!{#H|RK-Cf9tln>&z*9}ylw?$W@uP3p8-{hAzR78-8TckB=a%;j7yD36 zV^H=_3m`&xy|-=7M@=tfJQ5bng+!s)^jm(di^L~C4~c6A6nOSc=(7Cz+NKNB0)5V8 z6=Y`jUNP{4yyWcpiw73j-alaSANYaW@|!WfvJWgiQ{qAgmbaa^zZmRO8hdHt1{*BY ze?ULGY)M?*sf%ja61=Revk!FnV;_hjaB}Z8iNh1a=Hd4pye-G`n?ri9sLQ;)4wr#f z6|Ync{9xdUQ%&0+=f4fzfe|#%l)Uieu^> zKiw0q82ILpAGZ&A=M%^07p4Yp3|rQ;{`+3<4E$i>!HvD&`9yYm9J2eT?HGZ9{AAjf z{X#Lwe{V`El`U)w+&+Ft?~OyMC)e~BZ#g#=Y=aZuZ=C_$PIymw54Kfm0i< z4S`>1$H(;#Hk=;+R>2XajhAP^pY(NB*pZ{@u4cn4N50J2GQ8)` zPZiU5V=xZ67PIG#o=dyFFm3l)?JEKCIlIrkxOC6Y(|3=$S;_nUMtwNF78lW|51CK- zV9zOCZ{+MbV%fgu=SiyydcN{0e*K}_^&J)^{`jeiC}rw|KYO0)qm_#+x|OT3AhCQh zJWu~|I@#fQ`j63ny@58t%O3Bg=W@F#j`_<^CVoUwQ3cn$S{0EL-zy{#YS*gD->(BZUeRDbuo~lI*Oo zSXhP7Q({wKlav6I7SKxL9|dc%*~n1VhO#JHPjwMR8*F|yImj#u3(1mGdA|ucNWy}W)Z^_R zTz$zP$i^xxp_V8`s8C6m$n~U94Nn;hr=qF|#eLN66+70sqh zBQ+8QE0>ekgm3!VENnVMzMv%gA5ySgqAV<{L^-C=5E}%Vs#Nr4qflImTw!M4RAjAASb8ym$%tHIib^pLYA@+WelgvH)r@(T9?q3C6xSR08$Wh|sS1IaBmp@H)f ziDJP!GHBE&C4ROmN`U7g6gAl^97Tq!Sd$HP3=zIdc9lSbOusN9 zlq^z1B9ldq3i-EL`lHcIyhd2Gmp^MlBP+Nt34;T{xE`b*)DI7fn*u&0H5tWl@f5Ux zA1Y8bi^v0>h;J#2f^-oUVKN1wXAvQ!+VX}WN@3Ez9I6OK*`eIE2rCz5vcMZ5kAkUi zkX8)5=T=iwSRnFuN$ZJOtI0!{1?vW{BX4gUL(6Pbj!0-yr1NIz1E(d$L4Dg+^H@ z#c-R6H@OJQ0JTXGDwiu*TNG){wLpDPlqm#h3N$*p0UKiRbD5zIHj|w)Hctg@rFgT- zu?Lta*s@780+Wi1e=wSuaE=CTIn<) zEfPxn*(e#8OF864a|t$jw^0t8mxtmDBv5gb@EY=?S;#~QtiY5~AW>KtIabKg(YetS zYZjiCz|Ymgae}tkLddO-VUj3xUT!5};kGNF*QgK+=R;`8SD3-5a%{ib&>8CwMUZ>6 zHVbKynW9*0sQ*+7#wsA9xCkLbHV~{O7rw?=(NYO38x$&%$673MS(Hp+g40+IrZ}&S zCHET?5EcDcp^)3;a9_e38EUaWzZ5za9IEK=LYU|v;h|}kAzlSAW>auflx2|B&RP{5 z`W`!l71IN?{?{l82+50Jp63V?T^E4 z`11`G;rJW!jG2!o@rzGVfP;73aDb2-J&Eu*;Fg?^1WrQ;?S@y8{BD~t1%+&m^Bfb)eM7k<>>oS!r` z$`^KZkAR%&SQesrRPX{ZgziE4pG-oWnLQ>YJv|2ps^G_iu!|B8WtE5ScFlX+K{Ap(=6wYuO?9=3Xi0YGcbz0d2mAes^4A#=^Hd{r zkt%wjBLp!E^1-JaY`wk-6{;x3w+N=9xbv4q40Ysth%hrK9zR+P=m(MXaR>_zJ0Q|F z&}=*gn!7g;=To;xs3wMh&6u7`Q8{&gP*02piXH%lSs;!Y2;wQj5EYt)9P2GBgl4+2 ztsXM0xDKhN#;fO&*C_8IP?XLG+FA@0Re|iYk=;;Mh`rxsVekP-G3l zy=#Gb)dBU`h#ZN?add|WzwsXACk&@2p72L8WuGI@ck2QmpXYi|hVYDD;#=USZ9rqT z1C87Xl(-ixz6A^AKIl9E4(w_KR_qtyaN0xQ(J3lOm`$yJ9i-<@1Boty-UzZNC}4EI zRH2aygQloq1A!JMAjjACbfKEA>YE8m5Su3hUny7*kzSaK#9_@lgpA~?SwrV>Wpv$$ zH<0>l>ONr0XZTS#eYW-s;KH(V!2Ukxk!!+>;Co&V^j!eZ*+8I@9+2TC#1t4Ha{`3< z8UZ3mq#_i!nb#BQ`y8!Lbq_(2!$0~-ID>B%_`_+5<&3;g*ht(#YTA9I(hh}CMX&iT z5M^bYL&Pu3=E20Nq*9150ri zGe$< z%3eJ_8MNjJprEpVdSwGirvknEHqgQuKxwmp-pYj@>QUp3b5LW83mmo&s=!R5&uAp< zK#k9$kz6}h!d|c9uA;F4>=TBxfc=t@pq??{GqlD_d4gi{5S=oahM?`uoqz}7NJg4! z?2JM#%YbGnLG28A2X&*CyO8uDZ-}q(hPY-el1A=sML#WS!tGe)Ver)b)Luf)kgUHr zv=qa_Hxq_|X;j}+$hoaQGCko9CE~$dp-OU~ja@(*spCfw-SZgG){{t^KWs1<7K}pp z7%D>tdE24vnLfBpyjX);96J3Fjbgu>wUA7bjJ)VmhGMEFnNUE%WK{k!v|@u+=&!wk zA@o^I7*YbXU>C;Q(kR#x-8W-8%BaXhxNaiQ7rctArU7+*2gxg5drxSTyqo$)Iax?w zzE^-6oEqf>J~O5kIDF*yIK|)@OsZRIr$wZZ&L8#`qF?qahg>BdstSCR^0FMvvJ{X9 zrbqWG=g<(#a3@cVMir*LdjlMb&`gLURzTwAvqfqg_C({nD4O^Ja2jw8@Dmc*IiheP z9DyeYaCtEH4YRJ^DN52O`*ZUwmg{URV5S6zaQRORvv}+I#EfCL< z)@v>D&BM*cbs(Bwk92thkjKX$I*j74JP3#T2aea{bvl*zx0<<{_C^iP@#9*M*@j#`#X}rdp z#>Ov!ns}xzJhQNUhNdPzhn+nY-4d2jIp`S}O7R0z7#aQM zc@k7(c}+!J1iFGE>6m6ASef{&!yCk~)JA?#pq3(nDm)5Q{soc>T75y4&li}*7ufRx zhmmwZZ~#cp{{jr7CzJP_^~a1exJhJtYCrr%UV)oupiAwI{4 zD0c4-@K&BZ34M>m=qgFPA*kf?XhT;D89(61f^Yd@po_N(avN*B!SrA=2b_E-9JtUB z4LZw4&=vBQ{K7{-2~Z*(89GF$rz1DaLHdd$^dNeoj5&&Hpl5kTRhU5Jy3I57E9&ugwa%vKSFQlK(Lb6RQ)iZPaqS< ziMhf&;wzBUJOD|=L&PoLI3Hu>-ue>M-n69#wR@NEy|cE08Q-gZ*bPcq3qm@CR(*z$ z6GlS|Dr$wXL?GhU@q@!!-mdqeZpRRljUj;=^#Q2nT&Y7#%!YwcnOA|9pi|PvLRX>4 zXJM;Rm>)U{HGKupJTxe+D_(=7&+qMlG(V0$3Y@a^YYdqDKr}>xYnD0Qo!KN+n6yl&-u zXloC6mF6&a6m@>71FU*hA^y|nZ!aU&)Q|jN`;?f0 zJo`!H`BDNjR|@ooCrYS-b$pIeDm%kEta3!VDS=kNxTz5@0CnpID*nNhsfHd7@y3k- zfSRv`RBQZZW2oJ!1z9h=4ieMQJk{|oG|vvfZOHL0gOp!*raf#xZP7qyVt{tN4(hvL zvn>&pz`sD6MuPi<45S@`IY>_Q@mbCp=^pR)m=6|rU4aMr8?Z{rUGKn^m|ZLk z9E>U-3%!HqyllqFY9zCKqle@Ib$Syn*QSaX|O@2F`_&s9Em; z`L6^DMj^y-NHTOaL|TIzk2Mg<@d?z|R8=C(poZ~eKWqnDh^Hu>f4dZ;?mAw*NyGxE z_n|kEj9r0x`T}wOU>FNN4|U)({6&QD?m=8a%r5~@_V;K^YELM#r&|^ZGw8QcmZBbX z;pgnEEv}yKrY2&-wOdB zKpJ2rB2%kGdj93KqrX-yL(wlU=Zog6<68|B1v8=yeEkRW3|E1@VhutsBJ|^0u>S=& ze*BBX9)1j7#1Hao5tWHSp6Y{PmzwYtXy`M)@^fyx^XjYol0XKNOgqADCn=kgn;!Xdk z5u?IuiU=ef+}sCv(wkAh{fME!w=Db%{;G{rQ0BcH+z!xg2M!T^10J9Me~i6%Tu<#E z_}^70PK1mKS?$PZD3Q@rN>o;ML&(es$!JJfQ8KeCNrYs}N|Jfs_FfGQ6*6z!ZolX0 z@%jE9zdycz{CYf|*SXHM=j(c1*E#215Uy^){i?mqh!1Ov7W- z7$utG{+JA=4H93L3;Kr9ul@swX{}7~Gs*z2B!*<5Y_gP1gk0?q&R*}R#X6=qQeUN- z+O@FMXf6M4`HYG1XKUD~ao91E4{cz_8g#q`7R+7Hr~%xHOKeV3iqze1|4-#|M~oMDzJ z_Q*DcqpA15cZhK5E)ljAgwS{%a}Y_=o@Kc9Q~G5Q_oo-OpG0nmvtt??DpGZ}tRv}|R`m-0~H=|Gbco@T68C^?{ zVK^!iFZv`A(XAzKW&JL{jMEKPYgzaDCH1MAFFJZ25R0TACb<07Yc5T;&yEngCt-|# zP0SUT2SCLA;nB~%-i6o>b_K87cYlpV0GLgHZ6hgk+GKV6@bRyd8sK40O_ zWtPJ5`mt;md0bjeyO&=-myN``y0VG^r@z?o5OPu_bP6|JOA&DGVri5eE;*ylZF>(yHquy+XBV zj);^8tFp0EHeiyf8eQZ=Ybz2=*?cJ*FA|oTNg5XQ`dExi3!Q09re?|#v$Huy6+%L~ z-wGpH7sIP7rD4}51>Ga;;}1CXsiW|Bv{tC+$k{$C$$o{>>~EDNBQJu`ifDeyE>0fQ zKaSIs1je~^+!bT@K#aM{*xulc(andzj+7~a%I2$XRCkx({j;4)#cA&*)e-Fov!2N+ z8#m5iPHM5@DuYKlt3}#9RWq5ksv3AtZ+MV)eDdZFQCs}Wv4m!mH(ieE@0U)O;u|`Y zHCwhEeGuRC|FB_~HCsI-%;8_UxomQO`%UL&{DYy#IC`rqA8?#)w{>_n!BWq1T%&os zao`Eg+twcEfthB|u8Ak<`@h~Kanq)M{3-Fj&DIVlrBa9-+1+3tM}eL0jmQ&ba+CEN zNN9>HgG;)Akdlp0W>XkL8`LeihZ7=A+WehbWl^iBe`wz?Ka?bDsu<8?YDsb>zM2E4 zv^*z5rKQTMo*shneJ)1cJPd8xTk?%13Bss`m?XHEtJZ0z+Q6f;DCpQ zq8Kkg+$mGGerFaxf_AeTiSt!Ou0PJfSaODJwq4Co_mlm197Y1nBXAs4(XEZco^VhT z$x6|3x8VhGjgMgvBQ>WHfX}9wUu1kgTgb{UwN&Qts{~R?*nge^A9sJtKn+@1NqUM3 zR1l{$m{SdPo9fKS+~~yb_xZ^VkGl~pW$RXw*uIU}+J3uej=<5R)_V*E|C_iE70t9M zQd(~7w3|)u*YQ^g{sz$@tJ*#$c&t*-OG*!G%)}Tu3!k3_B}@fONwb`)ydwGMn;2Lxxly!W^Ud7!mlE%h$Trk9>ELrl?XVz~tHNE*4QfpYj~U98 zAIrd&mM9{8lOn>8(<_ozIEI0uIVCT_*vhC-ltSrzJoS&E$WruqvP)|)iH3Q>J1Oz5E!>s->9{?RM`5tUmhy`I{Jsu&2kAC%KV9sR=Fh>$*~54ly=$MVCGnJEFpWZKSj918;D(* zgwb&$h8NOa**)`Bp)75@$t*LtcuU>k|D43iH1)rGVl&!pU%-L<^_qPO#Wl{NgLAL^%EtNH_56$ePzueD88` zyME{%szVE{V}#2PXc+PildpunxP9+^HbPcAD4#(l^<30)KcY2z=Vi*jg~dzOU8jh@ z4E&@J7A|?npUwZ5G>nc*4|pHy%G;jAi7zM2Oqc)YlcSGupVtxrclqkaLUm#_R>?HVKX!95AY|%F=Lx5vrThDK5}|S zA{m+@Fr+bs5Xa-Sv56h4 zW+qvo(aO6o&nIALTlZHy9e(+*^qSsI%w>x*V1*$w+O8B4-Fnv#r~ zK>!j9=_$X_pTcN4ZGhul{h3(v6f0je;E!@I=PD|kw8Id`69&t%?oq6b3}W#w#=k;e z*#uuuA}!EQ#OkG3o9Q7$lm{_PwR9ZEDUn7Spw-Cv26yXf8=p?F?Flo-&w{{!9ClE$~s)%^xoTiN{<7WK=!nRHP%(JWJGlPr8e(Cyh(vu=d5A{$P2l8Jl zXQ06<&1a#gC>`d{rgRPeokxLkV(ws$i$XZdN3&}v@8zggC1R?)G+8NcH5*M?yfc7)sjFSly!8H%i{Cur7&yLk!pG|48(GScftM z)rKA&N$xZSaj>BJ?snKr`Y?>fQPM5svOheSl-i-U%S-BJQD4q)<(EP^AZe|NqkeWV zj)atEaV(^1V>#@th_63gJ!eINH7p*olQbGx75&^55$fy-pE#^=}Te z=TVTn&5*;X`saX`C{SL>b7s<{QuQiy$9HFzk$uH8<&0!}C8_j?I!P*39iQWT`8bP> z{M1{Kfyr(z?OGH6j@mv}Zy1mtvTv8<3IDWkb^ITY%T_0_E)jbCgY4#j-0_e`RgzWzBtu0*=uo z4aG~d(1s5BP67)q+Y@Mr64kuG5v!WwUzP8L^}%JV+LwnBl*4EF)aB7QCJImB`+_FfgO|1T$DZ2 zUehcqH7sj|o%E{_zE`*Rrba2|&iJ;o^rAhS7R<24L zkM&YI*0pItSa&Jb<*6Z9N2OtPPn(U^T5>2&_BNiRDv!qsZ~(E#(1A zS?V^-mkkx|!a#%WSN5Pg7&!Hs+`LwdgJ$W9=07+voub82wCxlvB8|}?g_DQWPbri} zS}~(zaa!~D`VLaC;@nqpPEOS&*m9**W7AQWB`d{sO{yW*1&ZsdZ4)WV(*rXoMB}u_ zG>u1ElMk$#er5)gC=t~|?wW2%n9>)+aA;liKl!1kQ*{l8Gov(Hj>4mX5g|{EL_?JO z);uFM2XkdCS}4secS!qA%F%PW)S*EvP*k^&rKGcgst#X94d-a~M-XkLZrLO|k1coW z`lG=IO&OG;89|%k*OK6!v{<^va>E=|p1Mf z2`cHZm1)n7BL8N9vd>ut>p)o-RdqES{@-_BR<~v`y4;qsdg_rF0;m3Yn|phlzjnk~ zW(Om6Gr%x6#JHkp@ej-}%4xLvr}Lc^+2e_pCfR#o{8TiqPRST&Zej%fgK^{s&6eMA zs_}kln~Zf>+%&qTYdk3+^`)%!3CitD@aANGm}S7zoBA-VSj976gM0S!UEHRlpT z#c?i$YQp4K7IfMD>EFbcR5{L*x26AP{2>A~pI`3x$?>DKyMUysW-x_i_029yh}4Ie zZEhlc_~~{8ZjGXCIn8YJSvW`y6cN1meG@{*xHKi>Y!0*J7AoU#NVu^OhZ<#aCvm z6%lFooae}rRSi{9_GgM(rBTb^alx|eJ&p4qZZ4`-1%4<1(wunKONU;m97FBxV*L&2 zEGdT!s@*XDQo2xslLqOA9oa_b+H%UEaU82=d_PL?zQCP=9{e>9^KK3+qdfhza?X8B zQR;k9>Lfj6k+QzkZ)6+Xp%~rBv-MqyaXn`>>-nKDgn@K_0bw;4k?@|~>kwJt+WV;5 zTB??`un?iRj}(;6qv0Y3;nZ*fyG3K<#bA^wN_~5D$^R2del!2Ex?la7Ov5mS(Ox3c z(V7>dnta#ym%|oWQqZ#7LFDVAh)Rn);ilolKnrF5)y6Dn|kjPK(4br*we#7h&F_ay{*T>6@P=PPqLTLtAva#O|X z-kwJrn(tyJbVe6P1a*JPbI;djCQ$I!G4C`{i|4= z?%$!ge>;96=+7Y}6FQXejNE$d;IwR(aYi5>hRTskEvchFu4Y@ybftR*~zY8+)MTTx*~dvdc+7f?ha> zUz79XjNp;`mrQ3Z`cNE$zh$&?x=@s`g-}aU2%wy2B2uJbJ1E*>?JTqjWl>Ls55$dK ze3ZDOrsWY{Z#v<-or35wA&%??r$)1aP;b6Vmz-z5Z3QnaTn)_RaxNEsIqwVjxVh!xr2J1UTxp=4ynoDRH$|@zz;-v?vY6MQs$}(|M z8RyGSVeCCkH1Djo?BY!iaNnFRtbZZS>mDUj1cD)N5J}@6;^&5tlI=U`@jP!Zqe?C~ z)QQ}-A11~_0%qS}&|Ml~L~E@vGNyEEEeOjHUobl$luUk1;inko)Qak*wO1vI^}* z*^)a<<p}rk{vQWVF!@lS3*YyosJ`N|d8nlv9!1V-HianOL?S!Gg$;M&(e&&aAb{ zLA4a99H3Y$Be0uvm|nA1VxP0Y*yDz=Xg07!xxv(kuN9<|6juJx zyfgX6S)pD?s&h#H>or`{ZVx7vbtRl4N{OYoKL3Ex`XiZecrNcv@?$}JxC6I|J8>() z?Y$U}gmKU1b{t)}8I*PC+%bm!Sw)fY`6rOE+xG|7dqUaC+)X=cv@^w*8IhXe&88gB&iECY3nWA z|C_JuMCK}kcszX|y}E$oahDFXx%#M%G{o*1%8A{fP@Fe(;PWfI9RkxRccUO8SmJhg zI^nct&kl15!{1W75XU9IlmFJprM{yVN!{5#lK{?X1DXr|n~Tlt7JSeUWJ_@w(#qrB`eiAs7gO&z<)6n} zEcHFf(j^V-%aVT%%fWusN_qID(vApdR`t7_eO%7~Q&Q2O>KEcuP8#k~D`!HIwDOLW zR=wV*$V?^qey)s-cgk$pF#_X)AI2tTaQ)j!+WzFSGXSS#r6-0VQ>E)PrF4f*liJV4 zxUb03mZ)%2)oKjQ1dLh8YH6afE*V}TB`dNCLr;>lDsWO`084J)X}dp~p~g8<1Eo6- zJ7H{B3aPAX21+4Wwxj_FU&+Gwp z6;~c_6V4ErrnKn~l9|`!_nt_1snIF{N`ZMn6)a5j2n;HwZyx4g(PKi#=tr^gJX?l5))fh_f#F`Qu>4{-%gV9D= z*(MFZ*yTW&zDvg=Ku*xek}jPtcQ%{Afka!UxHcRbp61ylKO5zQn2-;+yrobcN_7s< zfwIqtcU0YMB-RUks>$a$gIAto@*V4COPZa=*39ocu`TJe%~2jn9qV%pvzgNwika$s zLXVP9Tg7RjG>3s!zZksXwKS>QR_@sQYd*75X~!dpRi{-6<8_AA0}WeH_jbu_*d*6I zl%#MUu3CI>(OTY|PitzvvuMzsuaX{80NJx@1#P#*)T{#ylE_Gqs!rhiFxinBHkY4t zBqq517ycgn>9YLZ`{U0kD$Skaj174E8FH5M@Ss8xjL9b~= z)k(x**CG$~D&=;^5WHvlbN{4`zPKaRye8y?>B?!&!x`3;#=Lh7ra@pn?XW@_wFd?& zdya9d-3>T2+`oPZWj#goM**CEX@&*Z7*f^x*S7NuW60mChzzE)CP@bO$axF3c}t1Q zcg;!tOv+=t6sS%xL;icHs&r;$Fl{@KA{I7P zp584~Eu-wo<^8%a}3Ussjuc5q@ub~2H)}cJM zk$jnOQZ#Z>E`QEb1Zm()?lY?iuRH{3ue|HK#o(&CndZLPox1W?l%1iXvU{*3%IgiT zQI(U_Z|6sbo_u%tYqDQDN0D!7$a^`dwv_5j4>0S7{INOXiEqMXA56CiilSl@LRM|3 zVRTg%3sYq|i^4KD8p9$P(LoHi~_}dZ4lqaNdN^wku@P(rqkly8`(o zm3drz>Xvb}I<3-JagL>(RD{Fw@j;V{1uA zZbD_1BK3&twa$)wI9-rVC)3jEo5`Qk57{f$3k7#^N0bEFvibz(MWa)gMMss*NN1`m zH_XVVXrrO1m5w3yWMAX)PsmfXMM9)}rp zoMu#1e;bb0|5}H)UIN~Uc&pjD;QgLHucMsmy&05kxakUtZljzYvE^1?&ZvJF_>`+6 zBTf5gcq?xm*Y7=$h4qJ0E$lhS=*`W1K&!6blHVD?mqJ!7MD=e~CFA-R@s~KG!*q4Q z)dH6cT;97Pnif*$`g0NGK#5M0hPV-K-XKa`p$x6246XXJnjiT8i+^J;{MQV_ODMx^ zC{8A`=_)TQt0QRioIf<6Wa(WORQwAsmho zqH6FLVl-n?%I&j$FrgatHna`!mmkdAfvo>UlNny7$(kX11BfRJv$+9N{G6_qf4be~868%IkjI8;D zzC-K3(O2Wp<|awXo~i0I&3=@q!Zn`fw)1(==9%ebh@nblV>IJofQF;Fb|DX z%1Xp&oP?2!RwCJNBcUdI7hoYho5;PQAz*fDds}o_A^XyUhZzN>{_4 zPv8rPGNV0JJkx}#SPFTd)oMPZbwCH0|%u?O_2+KRKvB9>T=;Hy6i ze?vZ>2{N^Ud^m2d{{~VKgX)zyT*pC4AiRpcQG|gOM~YH3!&P$rjM4oIM#NVPUl!u} zB>^P8xdLAIm%k6t4M$iNqy@Kub_YlIG>9B&eB=zBKLLl zlgIk$5HAt|D0`XqVeB@k89?p(-VdVotj^L9&SdJ}OU!;uA<`Mg(pp77yWev%>zBlq8PlIgiqA@=&Wt(rx%}66rQ$PtMl=)^Rd*yiJT;MeYZD zKEYJn@C&Pp1Ix2iJO%5cC+~@;R-)|7`vr-9`YO_jskkq47MN3Pxun?`hg-)nw9Vy!E5p8osyE~%8N6|@3 z)xbj4$XnGURMl*QYL^_*=j#8>(ivL9I`YTHiZr{XeQXXh}{M zl8d)AFjR8iAbI3SgYQT~K1$wN>b@50!@boT{q=&xB6 zsd;^m=FLl*x8G>~t+a|UuyF5O_SFuO^Z7-C~dqJEo=j%_vtwAm8XW?O2Tl>9b3i`(oL zZBzBzraQLH2xyxb)ix`&?ZN!Ehl<-C5qd}U^^Q5}oe0o76{UA3QSaoZ+VNL@ z$KS;r|A37eY4JZ;tn^#?>g&waZ=IyyHdnvh zJ^c=!^gHqISgZ_;d<{(I8ki*+bjda7de5NyCj*NXhE`UFJ$(&r=Nj528TQFF?0e6! z|0hF-7DkQ+M$S$~u7O5w(ME&Pj65$Jd6gIqRWID&NV-sWPUc+{MlLT2g@Ij*qPt#C>w5EY*IW0x{`INrofh5hS#^8h+wIZZZcmcBJR20cDI_4pdt<9l?EA89>) zUGDL_q{kms3t?!X=4_!JWYI9jLL=Rxae+mXQj2D4{7)v!md=*iL6*8PmTl54^$IN8 zms)mIv(h)TGIX{w4zehWmJ%?G_&a<|QvF@F2ZC_yB zuhe>gT2DvAp8P7vpRf8)O$3W49{ZZcTyRx>CFKYP}N;dvA2^y(y^omYCk#(tD>A^xj$8 zd$(GjRKq^$&V4e1`eerR$x82Y@K~Sh`+W|7M!?!1HMBqGY=0ui{#16sEbfnW#D`l_hc83<1T|AxD5H?;@!&Cx2Nmy;jSa+yZR-&jydi+?t$xsFRuQr z1_t&VIBEF6$@2$JO&&Py_`n$t2G062FtnB1+@5aphr2D9@3tt}ZSirpB@f(|eQ}Fy zligPzcVGL!J>iS{hE{`;dJal<8MJBApe-u~ZQDC2<;tL)WrKEo z8MLRBN2-xWnvqAki$}&JkIWSwS$jPWUhz0o=5a*wJZj{5%*FG>B+pYTJkRX)%)8=w zzRdHYG&tXAaDmIN}e#Pr$ znb)f?UT<0rDd{<+Z1|A!`9ms`hrByJq~^hp_g{v5Y&G;#&!Jz25B)ZO==T*vf9xIl z>&npIWkdf+-on^h&DC2y*t=n@w?>9{<3jHyZ@rtT`!qN9Y3b^t9qgkU>(eH~N3YPQ z{ac@o>c0BMzJ{*8#=*X(vA&%%e9a4eyS?@8p+3w)dzh8Yu%07^*@g|X+cd1tiD7*o z4(tDQn1l9kC!66eBZdzQ8}7boxW|d%gC7nb@^!ek_6T3&5yMm_8tFPFBX~?^?3k=gV-B1clU+C_`{9_wU&rKVkIl6idwj&$^@z%C~PyFLo+{w>f#C&(n}hnC4C?zRsQToCNHIe5&;;Bk+FCwvR`*O?q>J9*N`$-$E+PgyxRWZ&fJS0~RbpFF$4lsP6- z<_(+@HhIdzl~cm^O^LWVWoh}8_O zn0eP^=KX;)A5Nb6c;(Ee`({4BI`d`u%-0QO6`RZ|9XRXl_hDsKsGbShNy zacI-;p<23gTJ)OJ%59GJlsUR_bJ}Fi(JPwMzG6=8 zbGuc{?a^?arRh9tw|O>G=Jkr3*E@5bebKyr74rr(obPBl-`Q=x>y-I!aq|ad&i5>u z?^Q8>Xu~ic)39M~VI!u5jfx8!of$T^C~SO1*u;hl0!$YKxh)8uvS3Qwf{@Gw(~B0& ztXMF+;leql3+K5l44blWVcf#-%!Ls}3zt?bT;6a|l$UX3sHKk9bQyUp!v=>ig0+y30y>Eh`(ftbF0J z$}P*@WiG2OT2@=J>_fxlb*9TdyDk4ZW%>8G z8rd*DQe%H)<7<&kDkGc8QO(VwTDnJRPmR)zk7~0&O7B`!`^u<}a{eD+VrF z;TFGQ(Eb&k*H(B{t{5uE`k2KIvx^<>7dvuMtl!qyF{fk4J&B$0Ppp6Im4S9EC;6?M zylCartt+RUUOD~R%9)ibXUlPO%;M&`$AwLeTNob~zCSMFTHMmgxaI%EMYfKQwu@ik z7r$~*eEin<)u-dvT#H{<8NXg$m1wqVqx-5&Q&(+?U$t%js+4Q1c2=(1Ew4^BTb=H{ zI%DeU%=pz=`&S>lw)#-z>Lc=+qh@Q4xvx1fbYsIQS|^m) zC6xIklrKuC+?w$2bVALOg!lg>d~Ch`lim6+e(S$2TK~`1^*>Lq|M$uI|NdF8(q@Bd z?+w!E4GqFK$lEqFI4v61*fb?J@0{3jP@;B7qVB51Hd%>!*Av@UC3e(E((jyP zI4H?DB*}DDQs=BB^Xp07s*-yANU~_N(W>{xo})L~hHtdnwz1EdjeVbP?Ehnq`sSHc94Ty>(|OChL0iH?wk%w=B|K|O#PuyptF|oH*c#P& zYs{dnv7@&xUm3D>W%$7bo!Jr+5Av=mz?YN$`QRlBOPc2!^BRa>>|gU0T<&bvPk+Wj?T_xDx1e`M|c zb$$2ms@;Dy_J}Tf)I9d6PutUQ^&X7_dm7)^)8yTrW{pytcS&vOk*Yl{Rd;o2n**tO zH&WZbOYPVwO}|T;p+}nWv^3MzX=dBgx|~hx`Yf&c&oqm+=~jKxdyYxBU7T*WJ-yG_ z^uEv1`@Ku=|1;g8?OvxodtJut9k_U}`}VyaXZL#E*z5If@6bjWK3y_~d1Q>3mN9B| z#^?hXV{c@Pf0r?_(Y}B#`+_|71y9>IW%a(01N)|(-8bXezF9x_g|^L{+a+_JM`qZx z%!R8n!w+Oe+{j$|E^~RK{ZU=^$9U|Iowh%2_5M`{_OH3If8D$N>l?>okuP)BMwmtjC+3Z`-vj6&-eW&fAdwmW) z7<1^+;zLiiA9{B7(2Hk>Uj01uy3yg{E{97!4!@mtxMKC;sso3sZyc_Dclbl2BXwPl zeD*lK}3$6lG7+JNAr13(_cASdPiH> zA8qA%RD1eS-8Dzs96YLb^Jx3(qa7RP>YL{pdgdC9%{7V0HA~6ul9$`{d2aV#94H*K zG(To-f2`-&W3~~;>{5>P$vf8f`LX`LjydQZcd|e3GWPhuh~w@l$360n4}N}p$gktx zdMAACPYfS>Vr0Y#zmyYW@=lDsd18F^iHVI*2AH1=@;n(l{p6H2CqoXNbew+kNC53o!O&zHr4!Wy64%9>1Q+7oTZ}< zvFf6%RMv;EV~w14=%8oQP68`LGSDW`&$M5Y6=EuUUBSt#d+`* z*BMvb)?OKueZ}+E6|b5rL;k(u-LBBLZ{e`Pg(GGZj#^tdI=he~G?RU$bK7Z4ZEUnu z8nkKMICxRe)PPx2f+x=Z-w%tHJyp~B=WkV$giUiz4sQ(QUoZUsKB)V@{?O>hTSEKi zEiqX6AzYOfDk`TZf9YkNxBRuZ8~Ix7k9#eO*S;1d&~x2uF*NzLXuj*UXxsOVxG?LD zNL=_vSZ{kHY~d+9hZpb?{I|amEeqZVjo~^f-MCtTHS`2quz?+gZ^Rx*g*0#|dLtgb zej{4^dLype7mFKE8Bi>&qKbtrTm`3f#ljg}AOY6H21tY?$l6#e4!}V$+gB_)gE@4E z+5^SnJuE#)JlFt9uo05sG@OIOImN>Fc(JfNQ7qmS7K<9Fh4=6QK0+OQg3s^;)UOtc z2G9^>XatR+30T4zI15+(byQvy6^obf8s0!Llt3wz!CNSY3aEr?n0l>PguqOg1+yU( zPTnaNr{FZ40kymM!5b)kO1g0MCFTE8EY5!|7RyA5*sWF~Or#RwpAQ1L9C=pq30B&oLW}_0(9W)x3h=t8c#1iPQUm{M!O;~7F zB3yfuFL=TdSPIKvIh^fZA`G2NgsB^8c$NsO(IvtfdV&quLUvGzI0c_g`C1ML~g=Uc@qA6%W$Ji2aer1Wc0NvtBM9I1m(LSL> z9E5c#B_aXV!v=88C=rkLm59gi1fD`}PKme(kB(BtrzN6Q1!aVl-%3Or#6!r>5^+?N zid;Ab$H7daRCI&0qhqOfW?d@YyO)YW-%?Qo*P#}&hm{J~5v5`vxPd#g z@h=r^K@ZwNd*}e?gG+^PLa7+Dw^S^Ewuei_kMpJCC+J-(6-{53iXmmC;?)uzmGPa* z#6<9iYj7QIfQfOLD03(iZp(C3-g0&4ItVJTSAqxjDz2Vf-*Fwx)eDBed+uxD1JuD# z+`PdDd|@~YgYWo_;5rii!Tt;W169W|;fLF3=!D%IEWrw_!3Jz$EdFD_4!a{b!8q>S zxefv^@Bu#vg*h-0e}4#oKsboo5txqs4cE``6I7ha#0=cjxX$D{3qoNw%z*~D&*eG~ z=7Y?A7}v&J8-XThgATL?U1$d#z!*%x4#q(tT!ZWI1o}FciGJV$Lt!+Gg~>1%k|CbF zinv~bo8aP7CPu<&7z05N1KVIH9EKy114rQ{baJEJU?6zHV3-L05CThJ8AL)9G<7c% zT2KO|@DBckhm=L#qf9h}M9(sj1ZG}kqB8`5KLkP$6v9;~g6lABC}RMwd6S<{nOF+T zAsU{HEfY`S1=vh16YKrU#1_~J+aLwZ0vH$26J7*UFVLP{CRWZX6LAm^>mUKv!v@HP z1&hi=dQ_R%3#Z`>oP|6%2j^j1Oqoc9_VHz+3z$P!=muu1=|6B;LtiIQmi1*~H>5%u z>;CiB@hWwFq`^DbB%!&kOUhc88*Qdn7EIz0mhk(d9a74usoac9YW^hFpt3#<{T{( zw~o^;XUPMGH+zZFJc46C}~4+$=B#d;XC zT1O>e3U=54iI4;vAsIHoX4nE-VH<3R6xabfVHfO%JzzEUt+0lkU;~pv-io|MZ^eCh z1p48W1&pBwSb#P31Y76>8Q{O{tvDC`R$PQjP@ef#R6tmUKv z!v;u%vm10&HgZjd{a~(FF2cgg#R3S2#Sj6xN6JO-tvV`w;0pGBT>C;H_9D0jH{d$l zg#P&b#q~Dafd}vi9z!|2gHP}UKEqe|2H)Xd_ya0A=t(2ftw#p|c# zqQmQQVOC5&Wz_F;xoH0bH|Pj9;Hy$0M!*&I3gOhMLd>+U5VK&*ZXJ~qV=KfNI14pU z3*W%nze0SNTp>O}9ejfAiz~#-B^Bbq(h8xOR3Vx`Q)mWS&>UJoOK1h|n=6DD%ulHh zVXzpsLGxU~K}&E0cW9VbA--SZ{~g0mSbvjpW$37MzgHo8fCX5>a)^W|$bDELboU|S zpH_&rpa<=sJbcY^b0hV9| z*3c7dU>3{<=Z%$OX*TsgMcq$RZb&Sy6iv!2#i)0cqNo;kxk~hEQYCISuM!UiREbCM z6nZ;Wi5|{X!Ww#lE!YpL5`(~UNR?CGlIaVbaoTw6CFII_fu=!S%D7aT8t~{cxo^fAMCDQ(>67Sl-6SoGv6Hk5K ziNF2ciPm%8iR$_9g!7JfV#l6$;-5?J#1Hrhzrp$McOrd6wRk_aT0{m^i`8&$Mz!cQ zho89!uNEyASBqAl4LVQ>RZtD1uIs3*i>ek0upTymA8v_UlVBsvSy3(K!aSG{K?&6& zdPB8HgG}&B!f#`>$VsjiM`6hJYN54{Ji!KHZ|bP51Y7L(5Qn`l*M86+;<+Ed)d3vA z37o+NTwx%%fjbNW5AcM+-~~g$8+^bQhJhc9hH)?(wnGZ+fK}wTlj|NN$zYo_)uJ&B}LOcoxPm{ zgsV^l*WfzbfSYg&p22hY1sC4Z7x1vMMzpQ25qi)L+QZAoIx4TA1NM&435>uPOu!V( zpfmIU3$O$?aEDPa5&R(l0^v331#z7O!7v%7z*Go<=`ahTAR0D8GUUCl5r4z8Pm~>A zz}C+-VjFCS6o~y*BMN@kh`;_Lf1O(40p7Z`!UufeZ_`@w5FUZ6b*&f#n{8{wnIW|z zd19?NKDAbyfQZ?(;?lfYQNFNNR6r$EL2h)dNLpJftasK58?c3AuXI#;akYa!U=Mwv zAM^)DaDs6#9wtE2u3E7Xs`u22A!)T@D7?(5725l2#mLjOBJWJCn0>ZZ?7T#o;SLz& z*NUT0Yeo9=T4Da8R=9vG41`~5@5LPb_u}8q??pk^_oAi6dlAv=y>NAXFEV`Ji=>eE zV(_&0BCDKp8m>dZ8+^bQ4&XkFYscyDMK7>}-rx*_Vd`AcpZ{KbTJ>INRO+ZSTK!%$ z=IY0FG>n0VAZ>mx)S&@%+wq<~V&UPo%m~7owz)+ zPHbgcVRhouf;w@wp{~kju3zCBe20JF2aH@;Cq{$+ zqB;=(=N8wA^KdGnPMijQVoYd2FiZyfrFEh&OoLF!hC}cao`J!#I$;Ejmy+vB&+rAlLd?lJaj%iC%9I;*Vk(5dG+2AHPOO6jSP$o*!!64FjQYQ-6P2&IR#2B} z+T=%_7z3JrNDKNk(N!6x@`?Yj_esow|A)PMfroP4|A&8HV=yB$#$ZS_Q)8l3Ln=x& zL>Z}!Q>a8DO643n4Gop7(j*~S5;{wgq*lu*mCi?*oTTS7T5IpM_g?$? z?f-B8f4}GdJkQU|d#?L%-KXn5xaND`t~(Om(ObwbV8Cc0{#q?$Q-1@>aAFHFf&pL* z8ys55M%V<}ZY`ugB)elemH~CiqlH|C?-#d_YN&^-gcjn$HlVVR?t-9r3sv=H-KEoAEtEhHVjF)*OEAyvS3 z$bcQN2d=Bwi2g;Rx$%bM_WlmWh;5WGoa#-#zO+Eg4M7N65%mC zfgbn_)TvfNU_SUkEUbh$_yj$$;B+hThb+j3GB^$aHLWBNavEAmE<8QoihDh+xYyH) zdp)gW(}h-&3XkCl#NTKooF}d1B(se)!ZmQ!Y$F%7+lZk~8yPUDjTplLK^r+Vyp0?Y z8c;`}1Tsgqk^0eXZvVIanJ zARUZ!5OkuOhm;Rp=pKUfBhsNryO9czenM)3vx2fn}_gLYD6)J|#!w3Aw>heqFa@&oJ&ZYND^+sQrf zmSMWoc4BH}K)G&iCsJ?&cklpDP}$l}=%5N}z}(qRDk|H_NvMSH;4FL(kKqZ7IMq&^ zs@ut`tL;SZemlt~FYp1H;a3P}bdU&$geZsxrfvt>Frb5|PU#?OpbileaL!LhKBQ5Q zgYI1rjqY5eyWv~NhdodLd!Z1D;2Hb~&w(!KASy7$xr2xy74xt|Y7bMv0UTi(Oa~{B zz*X3SyccJ5kPFZVm*6s7fvfNcet^gD1fIe(_z|ANrCG=iSKu042V-dm83@DNI*2Kl z!EhJ>b>QcLrr<(4h;|Ihb)o^aA_e6EF%S!JkN~UU0f;tq5DTz`RM1TAAX-WVm2+2e>KuIuol*V3=-ik#w8(5h7{Ni zg-`^^BRk0t@E9bcJ4v8rCutbdNwlmxQIG8;M8?1tPqog@$T!bqP^A_5BtfQ1kUiy#PsVKIb27!01*Nv1C7Bo+Rh zxIw)W}_^jqp6(n(sN722R3I-nbR;R|R##k#<92!|CA z4KWZ4DLfcL6Pn>yXn|H}gLdG&K)J%jx7a3Vf_u;l z?eB1|LN}0KaGt`zU$NaFfc4!tkJP$I5lk7{MP@;qivbl6t6(*Bn{<)Sum;^?VHdFn zN9X{nVO?Y^R6sX)nRb!oa1MD-n{|4ZEQjj=)j!k(5F` ze6RD7R6~%b0VQXDBt=jTC*TxR!C9z=l?ES49K-{|@FQV@255mcxOf>*YeXN(I#>^u z<3ExgBR-N5ijO2T?<4AyABp>&kK|#~NAj(o0X6yUM=}NcVP5A);!AatMOxjY&9|HQ ztm`K7joswKrf#xW(M?{KbdzJjXeJGgL(S!GQVSL6J^|%$66!GQ2lx)%PmrEPS_9Sa z9{s)W1;~|dQiowGNa>&o_2^eassZQFt&LO%bin`&VF>WS0dk=nPQfkYzX`9=-3(e+ zyU74BfhG*U2a)LB3fo{il)^{ofhQPmaIKq61WyQrZ{QSgu6GkXFaRzXf(eMg6Xw8t zSOSM`bd$r7d$XJDh9c;Mk&98+M*0!D;S==0XXu5=x4MZvc!C!^xzkOa!ZY|0o`X(P zH(`S=)IrpfZW0Zjo_CYmud!|&ST`69&M*UJ!O)N0L;zGzH=zL&G(Z6Xv`?f!{S(Pz ze^_(YiDpGe8dPsDh~Co&Lz-1&(- zho9gDyn@&82ENI{G_VzJfTH>n$*cWDu3q^>hF|?eM#3nNT>nIzVFiT4Aq-%`av0#) zL;AV*5KWl8poh$g=php$dx$LzjO`&yi+gY%qlYXm=^^Ft;$#o`?oR z45&8ffOa_Wv4b%Qm zase(vBV2;Za22k=HMkBp;3nJx`pnNn71Tf-7{CM#=m*@{pNSzD!2mFZnQouSELa3V zuonuU2y*9qCc7aIJm!5Stq|#t?FslyqJuw^cu0VJ*aHO+9P*hghGj54$$(n0>@)F) z09XitunAJ3A^bBr2j}6(m7hs@I?4%Rw|yp`t3H$Emp_vgu=>GgGNbu3sqg+wx=1e( z_3I`18x5$CfxUPxu$P=1(Mu{e8Bn(^ddVHQ3r%njPGa~2qz_^3m|n6DWRM6+FxRG+ zJf74`+#Gv}J1lkUC1G%SK`-fC*h_MQd&##@zqpskSN4(=c`ph62GeDtA7*dGaSR^t z4E-;FxwDrTf+I|WF38L2C7HW=$uBTqZ!a-{VK5vQ?YA1V%I=nv9 zODxa!l4~`++@?baRn>YFT~RK3mF4ePyld({`>xc&=qftm1frvasr@`Xf0 z48+13*be!y2TI|WjbF%n@Z9u;c!4)?Hh&>{pbu6sKkWd3x>^v5OmK%8j5rYEQQ(V4@2sT zR0?je4E^p%J-`#ZU=H}eT$l&*!4DRIKLo&XOt%6SqB{^4K{)y&APC*Tuoyxh6qZ0F z#w|q}2FqYMtblNcfJj&gaS#s*7=9W10?Z*7cEfjY7EG?7EMOkY2R{gag%Ah{unJbg z8jwRWT!E`_4c+wCPjMzFX5g(MSB1bAuG>- z!fiX+6L<>G;1viADB4gk0pSRWMxRE}Xwxa$SU-wpyw`vlhjcvHf;~k0Q?yAK9)mO% zCZl^L(kV#ekj6s-h|#|a=^9uI>p%vHpnxX02lpW=f}$D4V>>osJD?J(;1ryOkRqH{ z5DI6|{~er#@1YuMpcd-j9MnStoQDf=5gOqVT!t%f6|TW`xD6qjDOxBjfu#@z%V0UI z0IxKPmhufnO9KULhb8IA3t_MvR>0kDD2we_CR_%$Vv44{kD>)1p=gUC1VUj6guyb{ z30aU0Ij{>ZmY|%EqI}^d>?@^c`{7X;Mf(AszzcW@C#q0(;Bgw;^gTuUTuafYI*R7h zfU;;rnLq<<+HXMZ`hlY5!fwceZ=vxq))g+n71;R{`{Nl!8vwS?DOxIQQYW-OX-H=r zWlC3$!W&ock1+W8Pc)X3{{ORp`j3qw89ao3);@w0{(7|KzIuGF1_S-x1!wE4_brY5$(1l|V8(m2M_yzzS*ITkt`sg8bx4tJZ5 z(J>w48#;r2)#2%|hgkfzb)4bH*ycCbqkd(JbaX8(bgiVi<05pex9LterfeE#@~^sL z9ge*P$Km&Oah#8FTyAh?{mOCG(Q~uV^N{L!Md*2N)AK#1H}A&3*CM?HG z4epj-x#>EF3Jb#wsbOY>Vb(UooMVQ$Hw^QBHO$vBDzGprlo}OB813I?bnuwb;TuLJ zzZ#Y53@Ec0P%a%%5iy|hpSOt9_^&mG^Y^Xc{5q&vXK;(f;5O;tj)=kQy0#7OJ~p`L z#^Byx2UBbw&5}oVfs;buTqbb2DVWtFaAljgS(eN%S`6oH1TUO@n;JcS_&7r34Yi{GF|JW zG1hW->y#+#joYnLk6WkRvfk2aovv%67-N&+Zj%{hleOI@=eSMoEt|Ynn|$2~1!E=@ zx=$#Mny`QSgoA%+`9^nQ*_est?h`AbCRT2rSap2jnOhUjwoa_pwXGdvTkme$5M_H| zyKUof+sn6XueRD=*PV28%%t1ylkP@My0?AOgX5DP-J0~cb<$Ja$v=*n{FD3Smr;{n zZ=d}0@yYLQO@7}xd0n&al$J45+T5pfL`~`1KBfEkl%88tdRwPZ95KyGO!pA0MT;33 zVvTaK=54Wdo0!G1V_VsAJnZzN?YJ3sM&)+Kx9tYC+3`5`L#*rt9{(?C_Qpe!87;}m zkmQt0a&Q0E=1qZ>bD@WGakTUP4CjO8&WCS1m$W&Ta%PlS%_#SnQ4u|(GGj(n`HVBS zXPj-DQO$9wwQ{NVaA}Bkxsc(~SnhK9w#(Hvm+PFFH?3yg_LzA$dgi^1nGecmKDzy{ zwszB%F}u5bcF*nEy=}88Jy+UTSGuRGT8t}WhpWa3SIs-F+U>3^Jt=#vl;bJYkCAeB zNR3WNjqgYYwM%(=ZbQbp2|V3|F>aIdMDYB%GMD z`p%rS?Q>*$-brJ<<(~g?J2!g11!H{+J$;K~eD{BCVFS4cM4ci3A28Uz@FR2I)yBoZ`BRE`tapbtg(O!#VV;9F|E>5Uey!!6q zwH=FP`XNc=ht%zM(nYpy8V(FQ?OV4&Jt=12#9T!&b71j_N zb|EvYu_Em9-LR`2Vb}GS-5j^w8Na} zX)B|hc1Am&jCN^?p4A!cY7pZ#KE`8CjMvH-@0~HeCu8O{#rSo`_#4D793Q)APHgbX z*pQvEOHRgyHN`IPj14zf899Ds^qiHkD_6$tT$ylk zjL&U~&+ClOH%KTLpHMg_p?GD&{+$U2PbM60N+{_}C^cABHhxw4oK+PoS5@v@RrPyA zHq{2JYsasypR>AQ^;X{N$HlF;&ieBPY@D*>&XsbPd-7Rb za#wD$+rQR&P1wDZ6)~A8@F_AOy_P= zSZ~Vk-jo@)DJyGJPUWWDdzjA_1acLK_(i$t%F5mlaX}8938{1|Z$7h>< z{5Ed(HltJ9jPGw7^l=-{P%*?tA@ETM;}xdais7de=Jyq&{+Al9xt}rXV}`5Y4mX<} z9zHv~;&*sw@9;gfW8VE8ejj)E8)h!F$z0@<862M(lAXEaRA$)y%;g_5!wq*v+U$(> z*%=$ZGcJ2)!l|9B@9$juai`2ME6FBH?(;XT)ol5gm2Q}=u*uHw$eQ|?_jjHBxU1SQ zx7H@N-Y2&qKKDX)ZsV!k%lC7ye$2gYxcjEf?%O`Q@5b+5cQ1SQgHyX7-QWHAfXaMDi|`M zK;T;-O!$u*s`=AaYFr)^&gw37H7atOP~_oTKJ#|F{9JRj2|8w^ywIHK-rLq zf7UY1fa8|`Ub8gw9-i>~bi#i?#lnddi{@4Yuc`>yRk7rBMcBiN<)144yh)l*CuIXF zlO|Tm=T@eys@%A%GWB$2+QZ5%pDNP_oKj3Yl`;2J=BiU!yH4etK9&3MxAtfXCRP>B zttwtswf`R)qbZ+zx?|C@ABxcEzo$( zJLk3foOkXy-!tdtJv!&tbI#wmVWDlqqInI$s~bXc8*rl=Sbh0I?&ZcamoGoMtZXbtCW|rFqy5%cj#^Yu z#PXPBdNlpqG5YUvX!^2UG+};Ov{3dU()jP1zmQBe7OCX>IT8 zO-C}z$K4SxhH`Y6?oA9dXM0RlBL*xxcLzHu(VH?%t~CoFSlqAs@g@z%h&nG!qi1+I>EZY~HT z-X5&?VDKN>(t{kfy=dA2S20a!Zr?~=a;9FUy^^t=_)7_kk`&2Y z)0lGO{4K)|<*L{-`E*4R^PHTWLw8^EeX^5+l$Sz$UXAP*x#PPH>B*H%!*RPJY=KuH_Oju**vAEGc0JV zGJU&I@@g8t<^+o^#aJ4D2;Csba*2+EzP;I=#hTQIBCXE7u-$r-db zc_G{0iJze|E1u5{%ye5L#{Uay`t+e_%o6@E+{Ow zRJRnCy_1Gq&PuXG7LDOE4zTSRgzwhq&8O=*o0=#~hc9<^w%}*5lb8n%vt8T|X0hxw z4$=cNh8NLK(ClUDlw2Gl&rh;&I^gEaFd50fe2HxvPc7YkxJi+i;UhnD!jB=$n(o9e zDrD!X^iSVDi&>NI&)`2R473j!&SKfm;F~+LEXW=QO5@FuC^x59<@#Q{`bvK%hEQJW z#i-%QuKTMSM{Dz*4Gh0)!#l#F@y+C7A*rPQ3?KSpr0_{b7xhLB+IGJTn znW+ZSBm(++>@A%nH4R~gzl%nqjFraZaa~iH{46v}E#I%|Oy2K^qKz@r+vmru;ksUB z&c?iwENbMXcjw~0?DkHKZ`$%$ulWjR-UVf8Y|IYVo;awtyP8og zr!-Ap9rk=YB0qz5Sw)`FVoyyo54=jMxq)dGLhjUB`xVf|jfrovqGvHo4I3y*VIs4aU4b)`Ra@V^!)o2E2Iu*+~4Cag6jH?#sO1ICNaGtx!nPIlgK@-~;m1gnMI1u|) zP2Eme*2Rp#G* zcN5jKhTt4viP<{p@|9vUXT4W`rhK_aDkZ=S`LDPt_A@9-EM8J{z>%S$wO`MkzbPGO z3bykmBT>$_A0ppP*QVYW#R~y7Yl1ozDCb?V)w6dJ z4D_$oValTH85gZ zlRL{YR#p>=p(4|`n8k9jR;LefVekvxycqpV)amBRW0xtc6w~-`59m=8pB}DFNp`E? zNXTd5YAJ8d@qMatt8345-odulkQa)zt!wCk&H-vv@l`Ya zwuLStVFo+RkVvD#ci7xbicDcer_3s!}+LF3VN4sdHJb_NK2m z{w`+ca5WU+k4PG`0kgtny4uzaxhg5l2FgFpLY~dKXG2k5Y)ajD3@!p095<2B&|ZT& zm+MaB6>)X1N)BXT?HjUOH!}E~KxeZCnb^&&;Z)~cjdS!UZBqefVXaNy1tf?bS#6`9 zt}=@!l^1fvJi35#(~#$Lj998X2L^u+$KKUUmAW}dBuvJ6R*hrGqJg{6CGx8B0ItC~ zH%9qKg)K9^PPy9Wb8gNM7FIma7%h;?n4{(C3g#$Q%VQe~mq&|Nvb*};{EVAMLlQB9 zuWsyrmDFW*x>%em%y5!jr~5bBP+3u83BL$0xDqg?QVf&by7NRYzoWDHqjyAVGxp$E zVDQUXPJDT~8;{wL;p$8_rqkpJ7jV(0%$3&*b>(glUfqjoj^^*xoMi!#OM{}nR&yGM z=vNF8maLBvW+lcbD>;9tCH!d}$6C+UMUUQB!!Zt1RyzVCq85aQhJ^)0_Vq>o-iJ!a zbydB4s()Jv(FpuY2Cl|mC&heT)7rGjZ&eZ^q*O>8SVJ(Zr3^^Ugq{>K^eJFy4hXI(Y4h~Zl_2uXpLNODcz8Qi> zv;uipzUlxqTdXI~Ne8Q@o}pYJ7A0&x^oIjNJv?&7qL{$njtIHBe!9A_vM5IQw6O1p z_-DK9PY(})p6Tz02irkeV~vPe5gnxLh>i?c6!hl@g=*h8u4-RiWc{B*;lJ@vNGbor zp`f`eC@LyI*<5IRc+l4?#3MER*J{EakHiS`j`ST5f4jH-^e9MI)9a$U=q>#EMM#BL z7(Pj(X}gDb3?bo21}DFjR%UvL&E{>bZO`%$Yc%Z*pOoVv&dDak$wRz~Q!@P-df!Pa z2PNQW8J%vL>mkOt1WulZ_{wGcpx+Id-e!Aqi%ASySI? zRr0VDo%2TMU1_eUlU_ckq7fNeXj@ex_Oa@xC6_nMH);9RjhRZFS37NY7skE8i9OO^%O&90= zf9(ZTbx7F%-Q`r((EN`VQ`H&Q|5KMzGD6i(_8(qGRqF(Qyoge4GDh&%Ybe=FDKDYV z|E)`?nu-{eLp#nOdjAwVg?2%BWN6@TXZMBv`Z?Od+Nv1g!4olP4ESF^w*K_Q zf2yMQTE#^#uJ6QeCe0rhI0hGiE#NSuG66MwiiaLMq(eX%A5qpFl(7zV&;)H51pUDf zlz%}~U_K0o3ornTKnK)-1w60>E#LrJB&Oy96DGqr=m)dF0}3G)X2TX33|w#l0XV>N z(1np;3gclaVrhCx*fYD)hHO)Fp!L8~F9J7g^ zk~CGKj#-K7N=HT>*i>Vvsq>nyGM;`jpWbf>y~aho{kqoFA!u=Pe8}UyLNAbUPIFd)$ zxefRKP`~Q`b>{}ISiUGUI&{VIuT`uPHT_&Q;mupP2;cg;hm||#Pxq{hq4!SJMNdZh zcCL)>-&cp)_*;EPf$9%xT6<-ERW06ecT}nHW240qpq8MlRwc+im|Uk#zty=?f2+PE zyNgMGfyRd4YE@}k*8b~%t5GHXRv$xFWvy+{4lP%Dh6mHyeCKafuWhKU5I$a&Dbhrp z4V5ZQEPWE17Wqa?EK2^Z7Pnbh?IJaduhpbXRFl%RczDgP%p8*|D{xX&%;pgbDpza2 z=4xeiZI1v8=&S2(K}Cz&p=C;?X~`Jr9%2&_51GGKDMvD>Y{=2_$4i>MF}0H>7FkEW zrI)!g_oEUQf;R$t;|<2HcsVjlc`8gIuj#uz%+fKe8kIUT9<>KsRN!G|O683VL9Na# zpI+!;=FdQt&LoQ}8p)*>dzgioqORtotYRX?$&_!VmuB`im9pPQqCNeuKQjGCDyEEo z9E`EoIlrB7M5Q+ltRWcI5`*5Lb1Lz@j|O)ny&i=j<3L96Z2S^*cz`P?2vP4H_Pug= zub8Ozel|r0V>~?Hxtyrm_I#U!5s!xu)pxz7%1T-9WTJAGjW(2eSwyAV4S9_)p*KuZ zR?51^VHQ)A3)5GfVXThl31*?23@halX7A?@otSa&69EQ!5tXh%NX<$A_fMv@&p6my ztXu;ga0zN%P{??wJ#n<_X9%50hYq?>}MDUCYrkgfQ0**8f4DsOCPwf>nR|Xp0qma`HXasVjph z+I~SaRt;|tW!UrRQz^XNljA_R>I*s3w+X})3r{!5b?A0dt{m@clX8*+ILUYuWC7Pv zhn}*>l#WsU+*CG896+DJVyQ1Qmg{i)DFl=b-J#SRH48G8MFnu}SjKo?VZFV8$zs{D z=tP%G0_YAb?&bgv=Htl1Jh5Ev45{(?Z33B;F4o~XO6gOjoJ1*ovkvln{pG6pV^)-1 z03E-MQp%BK<>;?j9bqh?aqRplyJuEnzO7>^=X`{C`XVQ{X7g=Uc2a>2izCI08Pna& z`k6?#3LLjtNp}clGCUll$6Vb^q>iedsrGg(M^6dOZbyOZ4o6Q^I`uq#lFR0K;FW9W zo#x6V^RvZLd9aTDi2z@_fMkbdD2sb)%Q)wx90xrQytq=0x)>aBA;=5u>`04K#q-;R}; ztcT%)+txhkTyw88;K_8S`e2{>&NWhjWU!s)T#loc!)p4k%CS#69d+#dr4B5qTsrue z4oB#tpD6WcbJW)f$ih-}eC7%y*E=24c}_Q3j$ZNrg>oR~jo+^sbfg?|V>(M*5#)1h znN;rRy)DRL8jI4Ausy{0?=ExT1o+514g4hdghqF~!A%9a(} zvvXE>8!XXJRnmw zv>$z>0R?)yy%$Vmy6mWOb4Mq8OLnQ0V zmL>6)DfY4V55qf~3QUIwg!shH-zU5{F5lESKW|^bx+R72dHdpnPKEX>4((UL^6|>7 zuh7xA3o#0gA7mGnvnas+O@O4#e9+4P{BGS4Kig~)H`wP|YjmcmZ<5KRB?|LAe0S)K z+bWy1)xgp3r+8uTLagGgMLshWS#tjzzn^`ILo>ZHa=4kr4sY-((45e!q60xilCna- zs-kV7IX@4{&Rgsgx+Ld@&Tj9V;z60QCLYD19LKpvj>43HH?e|?@%GAX&Mr1fmYP}h zorBne+ZJX_l?FMpxXBS_b~>SkVFMOVkSL11H1xRB@gg^S_W20z;)r|VNTFTU&5M$G z7fqNKBj#U>P()`gQe+sPkC=B}!ag7HDBmZkEnt3Qgk7@^7f}4>QDube)p7wdv}0;p3`BbX`$yk?W!Yoz!5-!r;;%J=K8pm%(Q>OtLTsOH9|R8?bW;-n-NgoFie^4bYrBV8o@eGc~37J+Q~}U`9*P%%!6)XD?!3x0-pY-K%NroTgEpAtyp}V&?A+O)UuDCJ3J26f^S9 ziSO=LEPLqT5mK>CoUEqfp|LL{wE(N=$i)i_A9jBC!^<@r?82XP?mio=VX`THaHrP5 z&TpQKPks6EyB87#&Wl&^lT$wG)X&X5A2GkFMo0Ng|Jd0#69%4-So0b?cW=qt?s;z$ zDz~04c{?%S=kBvxKh0V9>B5Jhvzud<37Iw1Z7sI6qjj#Dt`r+l(h}OwlKAm+xASoL9>n|@Czn3iB^8DJ{AoWs|wt7Fu)Io*n$bTek=*1(w$ z1Lr@C33xbk(w65T8v?wWByLamli0~NEV=>Piqz4uSK4jm`$9{03HDB`D(SbCZpZRP zt5aiYhicY6*#FJYN%+T_)6t6C*SwofOnM;FJiT{Pttf13=_IxYqh@Z6wob9yEx0DS zY4zC_JB~l|R)F=Ep=4`di{QEV)^jJ_fj``^`r(_LX}6>fw`Myv1Jz66g+hRSXW%SnPZ=I!_Yg>D=1RVOQ zj7zQYldp+z5*-bFKI6OFS%TZ5maco9pZABZzS$c1p^9^781v53GsW$bo}8XETe_zI z(L)P9?VZwORP@pIeft`P>)9!t-+lOW_QSQa*E*4|`SAUhuFkbjIypUeC%wqAw(ULp z<;M zf39f!Pwn@Ad+j$Jb$fM6hJP*P;R^_dQbjP0q$7-_uH0G5$e3U_#V4B02PvL7oku0n z360(t9iuaFBbp{)B=H!S#?rrcnlkQ2(Fale$T;4;6jg4;GL;kx%fekM zjIvYXVJWy%hr3`{Puvt^%5azv7xXG9Q&qj@bR*1uoImoiFb|3nBe<8&XIRoGOi7I- zvPE_>EP-c-b*BUastcSbo{uF(Fpx!O$OEV(O$OFQ=1*s19qCwcB1==D@WM~!AxxuD z`jp;%ibg2$JQR|GF2QVQ%3vLp4*1t!Wmzhhs|@|=+EH|d$=GIzQxbxgbOy4pm19T} zVv>EUo`h0Ry`Q4V;8Dhuo^m$I?ODX2coOA~;iI=76?Q>QK$X$ilopL5Q^Z&X#@7uI zI8DRas1q6goF+PqR9nTgbV~U>frU#fcBLxTUSx{mH)IR2bs7{IWGbMTSY;>7h$1R1 z-2P@##8;D*L=7>St|`QxQe{_SZ3&yFy!FVYP);JMpP|egWu}r&cg9Au>0GKmq3BGO za&svbnWZdlh75!C6m&7gW+YNh3T#F?Pi$w>oI<%KX^JTxBMH02f=~hpMVH8sgQ^kH zE-1nsXq4DVk)Mc7rwGj-2Nh<_!0}cpu%q!Vr7UCU3Y6cGX^U_;F;LbriqhvPw+ban z@to2L=Br#;I*LUwnbO74RV`D7PMz3vWvCaMibIj&-!Nw3Za?}JcIF<~OLJw&L&t4e z%n^4|afBF~sA^K_0*2ifE9|I&b}SxpAezh#5QSk5bZjO*33d@yHAz)MF)*IWq!W7_ z9{MQRB&?A#)Xbo;w*vfGDryunoqxm@#b%enr&t0pBFi$tAk{%swW^dF6|~lyL78(= zI4OiFLk!xVt4>vy5(X_`k1Y}Vi*W*pC~-P9mCd_A&T*LiWF{9&NrO4&gWJ+f6hA`t z2Mcgr|2-VYP+r&n8Vsz|*MFoZ%>O1Se|qWW=y89)C95G&ZlIg;gNJMN zWp%xB^}oAT6E`m5Vf;&#|6{Jw{(p>7^o2V9pNvv8Z_ZH$Wd5^p3Pdtkgy3ldqI_ha zoQ4lPH$a}!@gY~I(oe8a&0S zT&F|IJHGfa^!m$9R5FaEyerBf5_&mqxN53-idPXG9s-MfanAi(u3~imeyg|dW>ub+ zwJGkW{(e`N7&zem?AP!hZr6Uj4U73=AV1SRC9=bt0j- zaT`LFAd*?BypKFbvlKB-VrP!K%6)g6abuRkUEWeHhVlZYTFt=bkkRz;BU&4&vXQ3L z;gJmGeO%T3)N!q9>gY%oby7!?8X%U&~GP@=gD6zH%AAm*3w-9shAK);H&W zqK^OjqmIpI5Of?X=4q}MG4O;cDp5%+&NLolLpV2)#go``QhAJI9>-}2J()@B*fEm zQ;escyJ&i{hkBB=c9LH5b0;?wn=MiJpC(VEpE#PwZ9IuxI^RaMj+5-cnX3%Sg{MV^ zIPOt~8$I`M&OH4fa7VhV<))%Zx9Lr1^e(O!Juf&#X&EL>TYSa2k<1XhXE|27yFD zH({kZvr!eJA4u1*8}5|Ncw@oIaOE&D#B8|3v0cgD>3e(|J5OBiv`cUEIr!_EZx5{= zk=ec`yL(OW1hZS6F+VJo{;=62<>T6;p^NS>UYpsz?8#zf2=v=fe9bfSXwp#&iY>O| zZ?wbv!CjAjT=Q+vcTYNf7SN}M8hz)^$^Jp-J8875hr@Rqy&7fsbL@8<$82Rs`&oS6 zT`J`ixq29!&n#jMbd+j1N-Z3H^f_i4jx`*;eT*X0+bO0gj^`{+6frP~qcqd5FNUh@ z&K52lj!8IrBL+SjveA6Rg}J9Rg3N9w25?ip;{&oVOg!3=ry}wJNLNemE%F)cWtk+#suCDJF8<{geYKMQ10C@vu!Jl?#jIx z$qrJhy7R_Y#@JqK)vRmQTy;FcH#5fQ_>jSQL08{I7zZa9pC^M$7xGhLd>=*@~M#L*jB#Hp%6f=n;+9z*OT=7rYDFQQ$+IM>p z)i+fF#po{?0{)#X&}y(I*toe2Nz)q&Q-sl2n0Y z8k?W8IQvLIg1%#snTr(NdDvdZ8ev#1N3)LYB4w)%NG{@}+^G<`_;}2y(Qu6CxVVbK z;!Ex0GaSvPJ&R8!BTAp?`2|#$+M97+#Aht7(RY+iBXy#AT#;rSW@02#=cv|+Txv?~ zYeX(m5k_g&aYd?nnsq62sxcvY*&=s+?8LCRY^?OnHj1T18^=v1QL*X5v_J^sY+H%u(GR=9`uCu$c2gIac{Q zSAu`PTK3Ru`7UY7yqwHj9ijT_!UH!e=F(PdP;mK|BYx3Okz6utj;JnOCBB$n~S< z1CF{Jmu~f%Id|x?-K8;kP6o zzy;CecuT}oo6bEo*ef#FTi+04))ZsWeZ= zJ@Qs{K~sWG(+R(~3G*8h%o;JI=rDQ@7AQXtEv#&cF?^5AMf*j2Ofgks%zWp#`Fza< zd!G62eCF3TlT+&&iz-`+s+%##1jEvPb&XYijWK2=d#c|aG)1>^F$wdVs{Gy_Jd97w zSM6QN&S$mtG4n@g`n^4Y42Gi^wW*rBNlD9EOc1GTBiZSb(b8XX!;f+Qq0k5lemX)OR8-evh&gQ;Qy+(v#>06^xa@ruO zR`X2Bu(*<8nh!Nr=kYF>;pR(fcWppf$%uX7ZnDVaB z%O&6Ych-J*c)y}7a;bIJBkFYNrQ}EXrbjO(KWa1y-DDJ^5SSom;@R*cXDl^GIDa0_ zV`jb(9iBF1STXuU_+`@*%9RPGWB5QsUP;Bdx_7>BR68^FO7~@ z4p)BB6m~!SWBf26YUv;2haDpQu2I5!WwFAfBe94d{$rK0FMjxsga7=w1+m^_7d=+r zcLZMFmr%qL5N-G^hNy}jWsFey^cU*6Y5ICydVi?r_I=G&j{8dtF{R)CG2c%AKXeCu zC>>G6{^CMYm0AIhk}+h|j}&4il5{abGAcs|o=WBunLPqCiIkX6CkTwWB6cFC(oLx( zN-RTEfyow#5l^8;`j-FlK>vvUL$(?pPwIt8Im`Op7F%E+;z>V*L6B9fJP4J?0-`y} zAP|j#PcedkPI$jD55XadlHJ}Z`!a=*Bp|GO9*u2BOR9FkRBUI$mZ>3D#-@oWI|~}+ z%A%$gaFaL)?$Hs+qGU2A4RIYFR!JltsUjv~4JL&}Bq8J}2*HhegdWp)3`$%yjo48< zUOKf2OO{bGGJ>e(4-h@2RZ!v-tRAI8nX4qJ&aE(cNXy4__5;eas>f+FSYizxUCc(H zR3KJ3Ia2}zft>K*BBEFbQ_1LJU#uG)Aw*3XU0HMrgw-fR6-4p`(jj!JfQJBplN>Qb zipfUgOkjrVk72wEE(e0G+Hv0D5%=Oy{b)Sw z6el*8AY{-?B%^5kFlS{*u`eJfqwEk!R0cYEC=TV`LYx-QR05y3!Gf}(;s z=bR7`v!Y_a`2Kp%dEWQAYyI!KU+$N?);;S?*-UEN(>Rb6`zgQ3Ovl#sFIu~NcL zr50mc9ygX!dL5IGUBII87N0H%9Y_h=$Ggw5kjhmiN(@9zXZZoo5&$2F&En%A1&E30 z3L{Sl-Lti_nUqSWqhuw?5-Nlk0`xF|cp^(m0@+X&H(x0i`pvSn!hqNdNRrB8WfJHE z2f0>=iu@&?5*J@IL75L<$WlUgn#>0oZKACBRasVIEGd-1pKcS&3LA}qSoj}RB`O={ z0)fR?)6@S+WwX*zKx#G>Bp^U1Wje<`R3RAVf>u?m0K~?BNNqz(hR+rn?nqcTmDozQ z7}W~ok4FAGKS}+@|H)6b%1Bqrs5d-De)7@3`N{t>-2Y<&>Hp79A}5HqrT$-fi|YUT z`N_sV|DB(t`t?8NCp*8!t7m@wf6Pygms0vq`dicx7K#-Z>relQi;Y64AuT1ZEw$eQ zp^1Q$0Pp*6XqM#%0EvhuCERbpX79IPsozaa_&;e=PSC&KklAk`y_qdt%2xmLH(tH( zFT}xj=>9j4|54#Bq^$lU(ZpE#GP=ILp^mPOsjiNm?*FNpn4$^^j|xat3jAMX6#1_X zvj0|ib7@MtGhhtDdlm*#`hX?9rC4}htpDPao^IXhOFO^z!edBqZ% zPx{Bv@4NA>W^ptG)KOyN!JHg&%KXZ(NEz(ons`2?OE{L=Fp}(44!;=?f~+hN$kPC~T~9me6|jUpc& z^ofW}V)1VRANF{Bkl^NUdVyg>R!)YzKE9zx+j5Z&kG5d36N3b0W*FjoZbkw$NnS<2 z`zJ;v5kJJ{1Zr~FT$(PY-{a%-$Xc*C_8c3QP`85(rX_|aM?%+)s= zY7CnGNK3$}wFRJDc-VUu+l2U^gs6YUV0Nd3#JCD&k8IgW#GwJmYd~`M2#GOeo*y|% ztfnyx>)05EJG}L+9s0RZkz1+A0Tlpqw*lPPj&48DZO=|T!qlg0=CT_=j4l3L5>BZ+4kSdKrnk*+&mf#bOPdSj=_X7EycG-s^W*q?I z0yOmlqunAT%Q}b3AS%VJs60UB11fxNs3fA&hKfSFkSy8P0kI}^qW4yyF*Rb?1scjY zty@S8Up-@3M^H&U4C3%3Al`BmVCQjwz+QlJ7XS)}vA%~@->LPJb#qW(GZ*!{uixX3 zlZeo!Z&QY$48qzds@Hr+(e@yy1nAh7&=BsvJP~_q6K0&ICWm3&!EjyEFew{dv}+nd zw!1*h3W7Kbpg_;^Cv&WbRwca@mF!TCE?LfpDqVytpfVVX z{&6y(T7%Jv6+I@4LXtUFOj-YBjA%aq;_K+U#_AKa{Rxz0TLGVgvNzdRNQL_=2I0bY zDnRufs?9_P8FiqY`<`G$qECRMDAmCXjINm-;EO>Ti&E}3}tN!8a@kP6RHb2vdsO@yC6oR zavR!Y>C#OsItJ;igRTNy^7aPcFaDc=l6OAgkX#XBoY)P-Rv28$$AMfs+XGxe3nI|p ze+P!9>MIU)XQ^QUuKbSSXtD-?()|OJHwv@<;sh<9S<2DmYVT8UW+yTod=t>Q5DGxT zy1^kI%iKXHa0bh~@jNg)QQL3<40eF0cbGcImDLTRi$Dv83&}FMFllR5 z2o{p^2{x!A6~+DMK6AvleDSmAvYUxy0vP@Usd^$9;sBD0B=j-63Bn{ZJE+5Bk!;NO z^5O=-mZMK0o5nH7Hiqs@>L~z)GXU>C|HFq_DKWq?Ay=>%L|76yNXA_G`vwN9pZyk{ z)n9;fcE=}dZY*V?KxecDI#HuC#)aL_c-&nAs&5a`^y(3S>SMISLb8tI^AqxoeIlg9 zazte@K2bm{>K9TXQ=WqCDr&4(n8!RcQKL2IHBPM`Ds(Yo-k<_b#P}^psO88+rSu(^ z)&4z3(#4r%jR0ly5$$1+`_-MLobTlLT{JF*h7rdhfWmK>z#2K^EoHef=T61ew7Y@=(v|Bl&ec1jsPX79r;K9)2|1RkP>i#LH&smLn?ENRu zoi^(M%wKCBuu?8ypNH=8RGt0igQj)a3yv=9G;r6jU&J5@xD9phvAQ>(en-DQqv*E^ zo`yvK0$BMQ?adCOkX2(Vm|;zryQh9|WVtOyMy}SbOo+M-wsjrZg1$2u5Lc|(3CX(# zJ^&$0=e+bau%qA6hiUTv37!8Kr8}SP$^>zPPbb0O zb{?o0+&GEK2liM`wiwai2g4m>!SGkO1?FxQiTbkT03;dGRbfkDkD2#XvtVNat3kJ< za2*uIs0hW^{vV*W8|;O=@RN*L7pG%ssl_+Cgr(K5xdQpl9HUbBU(;KFm`dV0IOc%c zH4{q=b&h(WW$;4O>NCkD)>ty*a=1sHUflqrMHNsFI5n<-*F$f1D0l6s=R!8aEWYUl zq9G8>e1MUfb}zt4Gh0_+eZpY#Sr^bEcML79`x3GJto;!6h2VCG`eRDgBDNp%UI6EZ zh*1+!CIawm3~bfL2+m^o;n_!$An}*K%}d#0T>rgq{P8EP9!*^uz-CCb4j7fb$ino6 zU!kb|xd_EgBj7Lw`^&s)t-v-4Jf9HAwqoYUXF*8lHaB+*JBTgINDCW)ajcV62=*ZB zqck@8-YyxyjKzU~cV9;VUKULNG{1Eia7<1gV7A6XstD8QT2VK%$QXb-9l)GAFn>5O z)(K2ld>&57QWV5oP?(N=`;Q~e0#eT%P8jE&SbL(CI`L1aaJ;a-Q&CC52Y#58F2T8d zwiJB3{lNDNP?-r(>Qv|gk%zi6w=X`y`OCVDdhjg({@Xy7M-E_}r^I0uzYHpcDu*)V z?&^?wked513qav>A{cwLGl(kn@Je9C_W(3Dp=nsK6`gBaP@L-b7Mgxm@Exc&Qq{LV z{0qBwrpYvnok;fWLzj7o0EikaG4pMQBNXP&JYNuBbpv*`2e9v0)o^Ik06btra98RT zskI5hCl;Spw7BqNG1j9nQ4ouv3@r7z_6!yy9TggkhzOx#hYGZhx<}S$AO(ccZvrZG z$?Ab%L}2$6P??5`4i{GIuNVY*Ee=p31+ZKi!^0^PHwBCojDjXL|Jz}ZSZ`1}ULN6=B@JGeG^c}T8y)FT+I{TteC83bE<(W0 zqX$fl9YoXbQ13GN`6ty`b=Ro#%7`THyc3<`GYlJE@w!is8NTaiJJn7D((5BmKnic~ZrEkPxpAofUIZR4;?{-wKr%SBB$@dsR zC`a7CU{rr-4%LZ_?@+c;6qH7>8&pdVpi=w;SOg}luc!$9M1`te1U~EVf%>tYqCw%8 zkS@yxg)P6)_c7*ypN`=)lQMz@T5jRCNB(0ou}ehRRC1b;#5A8KJ5 zyJ8$>Phtcok8Ig{_6IQb0Ok-dZo^o{v#7u|lF1)YnSlyrbrG9JI;bj< z;!h${3oyG5Gz-3b#HtB?1#4FlC0 z=t#j1z?JD$7#((#xp$))qhEN4E-L=D=ps^72Lip5yTR$=DWFeW$LX}Aj$jZE~qK@Fa589-kZ z`afN}1Ire^4n_U?*BCsCgPS;a)2xZH>?rcg8+2E{0UZ7g(22$4@%H6cMEq8Q#i7^7 zrJ`6Ir~vO$+(@rcn{m)dx)wmeRDeg+6Bn5R2swZQ-LszgI1A0b7&bmnos&w$H=#ii z4P!TBfmOf^u9)m90_RRt=%v7|6r9&n$<4|^d>g3IN^m(>gW5yXNG)oUp(JEHtPpD# z)^eH!{ec4FdPc@=DyNfP!HJYRSVS)XWf^MMyiq$-gvmaM z)`j+3$DsKBsR?%SHJCFO9?V4X5j248c#C>)pbK1)RuJlSD=_;M;1qXy!_+oL`CvQ8 z@gry@Q)ndMrj?3K3yCTLfz+9(sA775n#sp{Ox2JoZKV!@7e5axb) zF$ipb9)-eG-7=16J8%s$G!qbQ2K=0i7TGsb0KXP$1FAgH0bDtSUW^BjDj?@*VD>-{ z;G_Wn+YsQW3BVPqCtI*|Ow=YTjsux?14Uyudbn<@!38nrCerX_vJ%jh@~g3MOJQYc zJjdKyzY!|G`O6ImG$lehk$sl+9_YdRdFcGv+XdYOeXu^|G#b}w!#K6=;3D=n6=&!^ zC{MA{&=LUy#M8J#!dS--)EP^R1u&8T9#=NP9S1xMqcXZic^p^rVK$0YA2*@zyH6+% z8SKG;?VeZ|vV9SHw&19emx%!NYXGLL1&~e$Hw2N?6N`~vdKfeRT*4(78pF#ol>Phy z2P$i>ouq6+@BTC;z$!!sOACZhaDuEPeCTbX4j+2Ao;(AN?H(%cPzkie**S@V`b?3MVqUd9nbmLQ@Qd8ZS)p8l6lyfb?InhW9|^8hkj0D4>j zylDWL>jN+c=a^(I1bDas(+D@1s%Ap!XQCcTyDX%ed9!62JX;q%c8B{gD7C6WC*W}8 zhjgKYQnRQb{pAkIXB)8BWb($2b$PZUA!KPb#Ya*o6v zMw{me&@ObDh;wi(81;$}01w*h#wzrpHqhoA7IBZ1BJ7Hy6L2)2^$hr>D=_huQ6N16 z>rRBIU+@OLE)C2(pnT+920%|9@YD@e&Qan-k+^RdaJi*2-25I8@0bl@(^B9>rcA>A z9SBA7;aC;SWwJPWTSF0;tzxQ}%R7A(pPLy#^FPiv$HFY}2Es$#8?>r8@r;w>BD6i- zOk=tEMd<8O3?M_}3PGGeC8jii#xr}c989Ke8@fG#o^mhl7fn(A&b+FZ!)7#^$zySQ zAQ@{GB=Up^U5}p7_IXY%R|9& z5IW#%%Bn&*4K2^_$gWyU%9VjdEXD303rq4;eUiW$x| znDj(N>I^(hzXJAJ|AI_==_Wk4=UWG91|2|XZNs3CHtl0)FM>WZO+`WNG6&1Ze81=b zxZK(mP-|}?V5}lijEuy-Q7m$5l1M6Y3E59_fJ)p(Gykh9cVZBx1_Hw+5gYh$=0;!@ zbnQ9yf2lMsC`Q9m#0BIE(jUxuuWIbFyJ$ChudhL@%dJa~&%j2iUBr201TVG&we@3Y zCT>c`wVdut0V0+G%%dLGGzXxC9+0n>0Q#N*WK&IgIr%U&sVpCL zWt5P!`Y@QlH&F$n83^8kHpR=3 zhWg}7|1=P9dM>Q5q92}j5y5NdM*};$V2`dtq*J#r+=<%&Kkfne+y^C9-9FfO!hM0- zO4Obs2}SaLfxsLJN9WIEu(`8!d1<*Zd}titN8@4z0K4g+IP&lCktR@~RB)(Altj*( zqdlE&_VP#&K~7>83so0wmc*q@^#~MTb0i`cUZ(Rv6l8@LVl*=*z+G>7t%&WOznTVa zL2JNbFazX|WKD5=GpJT^p@ob?f<5ZL(@;!&PUCR5Od7;Dyzv1%puHTOqi{WBEJS0` z_+BLmh#jivEvW{qh-wD3@9=r*BD2pFp=~TwompbJ6H0Q0>c0csB71BR?^qk0f`~pbFAwSxQTXy!~rBUy10A9)&gb#`bVG7YXL2*`Q^CwLmv6UL&65kp<^7M`t;_rs`bQ1(ZiuA&|ghWZ2Q`k!EnnOUXRVBuj% z$W(;jOq|Pg&^+)(v;0PYszabzrg#IB`>ld{r3QeO2S5g;sox1;Qw8~+fJ4GA$kzrH zVvd{KgcH6(kJ`DJ06Sa&BwYdCEdsc^7&MODTO1SK;T4~Qg~|p4%avf@f;&CgjBVnb zO?DYjyzECa2%2d2gT9SG%60%0)8UISJW)V2MKtQsi8_soG-jL%LGZ`e5GBy$#(}S> zAwFdGD&FRtC3-VJpKJ}>Lr| z+B6xSgBM6}CysD8`M)IjkdrAo!PX$1mtgt)vL-aV{6QCyB1d;&8g z)B@VHEel=VC)uFOiuLvo(_oOBNX+xY^Z<0YD;OK(9VryizMj#y>Vhgy5F9(af2kQ}r z+hhX=cNP`ukElb{n)Q!EuCdZI&A-n&>mLA1D;iXJJ0;(56W(&Qq!%*-%E34kJo@V4-=YuJ8)T zIv z(-9}l!31oSVFSjXC;5c2ne8?jsOM5nayC;zNoetzHSWFSGCDC%Hl`>VQEXzcIf@A( z=o=6UX)3`^dn0BgT{#19P`E<4cR;yphf9XTpjk!Uf{)vQ9hi$sEdfy*CKGPoTHzGW4zw2Wl4$4df<}c~=5`xcad|*p>?{pwY z<2^i1SuW(jQ!uv`=q$Yw{r=L!r%nyn$~b_c1b|(M0Ew#s;^0DfD>4ZQ30|7im*!V~ zQU!T~3kvT-t=f(@6Nq%;INAoUp{<2RML!+_L_Y#m2F6E-&$fsHZfcnz_ zXsn(A;5iSB&iDmt`J{Ui>Rl=T0*EijE(?Id8Q{}}Zc`SbTLM+7(g1*^IRNw>BVw5k zu&)?k;ROIgs#-WRDrEH1$6Uh;?P|Z z5z)CXe#==4&8GAgaZS zj$zEkH=sN?`4(E#r=lemEiPVoLxj@&$Eq+>ZRo@AnUI$eGJc7nXj*gHble0Kf5_+e ziZK>v6d7oA-jQziL?NIZlD9;Grmy3K?Q_P#C8fe=RxtA zH(PFCQ4TZ7owKL}p)?vBrnXVK1VEB%gyA+wwUBO16?WG# zZ;0c-{k<-(gzZXRregeX@3W;yXz2KWb~jEfnY9cftPDGccq}F#&tbVQZgIEb^%3$K zjAXG=n;+0H{4`DsNq7w4^a~yS49rD`8Fk2!akHO#mSGW92UlSl^#kiLje=*OIs6og zLOKUf)KF{0GOivtfVO1XmOpR?h5~NRdvid}Tk3v; z-ueJudQTsyRlSV{6n(n}aQwStfV)4O0yKGd0WjztTrqdY{q5WEs*UJCEl{2BaBF28 z4j}&EULOdoWGgX8p5j`;M4MMbh|e?{s6`;e-8{hAjt8I_G%B8%p@(UH=D_y6D;=FpIAlzXKOx(eszVhCu0vHjD(h3@68XC4{*Q^%@H+!7|3S?x8n> zOSeF<4Tg+ob5(FDU<{GY3MF>6XyD%i5PT4=h;o?OMt2Y&G)TX~P&s4A?S%4us7=IB zN>E2~VL!mxW&o`g$a}#z1@Gal7Lu#N8?1j|j+h<0Z~(Y(aw@CYvRv~T-5NZjiL9Y7 zB>3KV43_&1Y--V>_kGnaytC6fiY|-zaUUZ}1WdDd#>^Zg8)`0j?~F*A#bKkSA<$zY z@5N(za{6i1k{Ep*uvqm5L_4H`M(=*Ss=#{!k82L^f$_8n2EG^KRQBZ{Z}Lp|yyn z92UICDB)HsL{0t}h~xWVO#vydZ$Sa!;q|j$f1*_k z3^M~LjjupGr&FDVSu#&$_;IggWKq}D0{Hb8$Tcg*Vl1W7Nf6^$4X|c=fn9B-j=@Ui z*kI4AUo8Zb-xvq@x)-X+OwXf+Vm<5`)7pCyDpIil!7I1!8Fx3kpM(yA%TJgmCU}hk zxSX0e5!3;Sz)pa-WKIQvH(7xog!}nvOan~QX2mp6q*JZ(ithx!uckEA?%DMb=sz@| zF+Boca2#Fg^C$0NI}o7)5Z}8`!?d^uC@vD9uM}_6o`rr!KWQM8{tI1}V~d&dEGG=O ze?5w?#_bF)^HA z(7m0U2Rhahq>XveTRP3&Bw@N_GTrF>j{tU123`oUi7HzV6oN?N=<$+RM}GsAj{P7a zSbtt-32M!pWfgI9d7=9k>|6mGDvc~O@4hL>hl(aDh<9n$jpy9y-plbL1LY0`{5da= z)6T&OJS70$dP3km9cX$2Ux2n>7)|-~yzi4n6GiQnf$CgGp-TZG<&Q_3+ynqIZ-9U` z0RF82bJ_p`j{;1GdebZ$W0GC0rpxS4D#aSmP{oN}ERUn`nWo-5;K+EdTk-PZcQdK- z{wd=)(0nU%Qmg_yFbhh^>|@hh&t@tPJ(%<4(bq!-ecM$5<|B&Zy`Z*39r(d1K!xKN z@^bU5rvX)i^GIjSi%sXvL#&qP3$B`cyEysGAU}2Ta(qqL&BTf58=wTBri?Dz(B&1z ztDTHxJ&(dEN+x4tOZuJOUoBcudbQkNHl3Q-LvS*-29iZe_W80=lik!6FQmlv>OJjM$`ysng`9o*@R~z1bDAA4L3!Qt7j2W}!qP zo{bD@RiS%0EFkX-3Tvifj?}zvp@{)iItL*d!SbFr8@Yli3XM0z&}b5e8eNy-{iuDq z04h(aJas^s<4iC-NDDcEKAJ`{C;4ZlW)4 zH^l80gqd+0M5C}7B<&G^-eZtIRoaFJY$5AV52nhSe;>#+O+2gO<=wjb@YW!+B#(w6 z+I`eaEb0A&-0zoErQWAC=Ap;XIuogp_-_3+@tHzID zJ=1qBZeuslL?dq@cGp8oyHOdyVifL1g<8Z*s0cRTfL+6BAOqAm*EmjU4hh!S2MhD8p9e<2cd3r#VG*a@uk zJ{0Ilp`2-GL^@P>Sy{1 z09i;tmYoB|8Hn)?Vi2>7sL>oo1)W_l3X5pgMhR-m3%-i+EQ1JB@y=5vuDJy~O*vWK z202nqJTrFeAx>qHBp>H4SwmO1M-fyA-$(;2C9`A--nM0~cM4*GZb1y533f$L#)MzQ zx6hb2qj;r=mksB=wPN`ToSndV85k)4#@;7jpW28v{&MJdpoD0aa2*QM1W=%6Wh#A% z(}->`R}4T7BJdu+z&{d8X)pjMdUD8Q8B|(P(4@}>k)0t|X~Oxm9(t{$My+Nozz1p! z=EE59zIgQxRCvW!mMqK$`YN@RUk?C`Gy^2Hg9;HN4P?pbgWyh`DxNyv`|1$7D}%l? zlb;F7m^^H*ySpAvKrmWj`H=J^e*8d2B+)dJS2BwDN+JRiI`2G!<%#5@Uk_M7iH=V~ zu9AFE0jK+Lk<3fBEt+odp}3G^hbDai6z{n>pxEMyVk_GT#qE?}Bfl$(2?V0N8e_-kR)@UA0%V%;XLr<8u!7AzS2)-OeQm6|| z#z|$yPAM`!xcI!R`hyE^{GIAPR_)#CW*}br%%+(2UrP zcMio|kzRwX;{|g+=3{t6H!w|e2k`a)FK9OJy{>7VXbbZKh90>?3&7_Nny4)x-WWy} zl_{w3`=FAD3Qgp{^+hNDh3NDKwKo7|fdDhnk;Zd-lF?Bg{)u$10r*Hy@n|-{U=Dy>Il$y? z0KPQU zoG+Fq#?e;BiXa6}rr^#<9?&>uqHq&JfhF97ptgndL>v=k^2-lnm`4?;H`D?|)&V@O z2k;(3^P8w6@QATkXvSaVC?@Ow>NrkF-Q7OKWf#D|<2m?CVHKISnlrH9{lZ8(kZObg zyy4%8qFGA;&Dx2NkP$~{M|RNn*`7Loz_?yiLFv_2WiI6Pzz)&_vIPjvJg6d7BPcU3 zC4CV8en5k?tB@{9T7-U3M&h~{{ctFFY4*QMu=CTYdT5>jSa=b`o-`Q28@o_AUbY$M zOP1!`$xtF>sFwYX$<2@h?F0*eT{F-@ZTu*vvDF&)YvzK(U)Tn-CehyBtJE(t+V^F! zN#p_O3gYn!8M&X1enAFm}m z-HrE2F&0_t2aM>C`ABFj0Dh(0TTo_^rafSw$$8;4v#VVti^{hAb*{Nm=|AJM?}Dnl^0xg{EznQM3^Ki=xt2#2=1A5`D_eG*USff(HZM9|415brnzxei0FtcQ@i>=A18Y0|oP zkdjd;#%zm^B-{4oob?z9<)Lh2kwji_&`#?o`BAdYGT=Ht=?Y-{HNxll<~8 zj6Mz03~{Kr8#|0n;G5<%a}fjcGTt%*Gcf*o2~1QCF(C81eHJDfa}dR%ngqxic%MF* zn~Pm0!cPHcZj^WP@q7kpJO+W<^cozKsf8;oE=BO82;x!Zzl87Zz~Paat$q9Bed9|PXj8##$W)2A+sIL35t^xa~MZB-R_$^a>CEMY7>$NEwbD$;byVC_q~Y z@S0`d^cIDoN^~~b4)jXYs_IY++yzRSo+3_f01V#(EWm~`Lc(&hmB#F9!Dj?pyhwt+G0Txfv)v@TFGil|qj@#;;q4eko6bt`1X|lNror?! zO)|jN1s$NtwCO!=&_)Ed%rqOktH~VB4Z?$7wS>@RE_cB}&zk9c+P{;z$WTL;v^olnVpMUj zhONP*i>RpyUWxjp;SFFKIi>()`@CqbW3U6Mb8-zwof+@_0>tHhAie}c!ma6%KLhPt zh&Ez<2LuY*K{$p1dMHL(M}SrsM~PYVeg}$oe_%F$F*pC8*imddS@jDAdP7-)JQ^dk zrX3f8@lXdC4W_XAEI6&HL3ftaV|c{%pp%gPrzh`gWX?49N1#J2>7B$w_6e^2A8|Cf z8Or1=dzkU9mVoqcETkcQkt4>&^^=Tu0*7$$_Y_coD?=uMrk$9uD=L_ER{-ogSMs9B zIkq2pP5tdBYelT((3ctD^qNiMR5hGe8uC)P8KEtS=(0V?+kK+HFQqMrb3e}RG?Mo+plcH}W9 z@t>oMyAt=vN|*DnhMy9G!Cy8BlyJ$sClCQG=`gVKZy!d30n9P+fbHWs@nF2W#64qn z)AD@tkn{iGww2h_HFr<3b#El27$+t}{RYl#dv@0d#u;_(9Q_4*y@AL{bUu zCc)I(Wx-x@+XDD5vWLlrGVt0CUUwnFJ(Y1-U+RH~o(aI^!vI1@KtZjYU=Nmi8b}@S zq1Jo9za(B{DF*KRB{_wXrf>aqr`jhlcoXo;}rptxf3#$RsrmW^73AD zB2y5H@dBS^yFtDYr1SRBb=wC5?tZlS0#$SXDEVVRY4rky3qAc_3~B8H(-pTcCr=}3 zwM61h*FShVK#n=UTn7OEIRLZgqL(*%(NN+N&7qR{Kszn~@bv-MNzZMIKR_&9`ftGi zm#9}u2?yXq48vQn1FIoN>p7rGuK;v^$Iw$Y$#Bl%W+n?=SPbdN@{_=Z-^lCZ!;{QEr z=*1S~VE3tx=u$fxM-3Z4e<8*m37{tVRswiA(*vL_SP7Gi<3omMlq)L*oYd$6P3(>c)dg2v4IGMYeE?66`0K5xg&K79OO~@rw z47tXaV5H6i086idARBmx8z8`;BbE-<(JZ~l!2he;K4w(@TzXls){|+yY5g%AT z<~e=x%$0aH_6Gg}$^L;3EQiS@s`po@0s8XwGzjS*z952>9beN!bmt$S>-C(PQ6*ed zxXm`fH}U71aA76q=`C3X%1@-Ivk}QJha|QIAaWYexQ)DLz`#`ST|Qt6DAPwxIrqVA zz-!5NfOoa+!Fk(M^cUAb|4f5*|7t`dO{hPajfPr;ER5aLG$n zRf!v*f*%?<2S&4L46^ne+J=+yCpfr{&!634A16yb0^=$294q6&#hN_=8);}9w@ps% zE|T606JpVMH&l{~pOU?YFGU1hqgTRkXcuEWK=U3K=8Uu4b&!^SQr+t-AXP8e{j9*`u-_VucIEvpSpWm#R-{LC2` zfdD0eKr4Y|egYw>0%27G5yu6V_Y16K2}UakuCfx0^Ak)+6-=rUOgS#NreAO!{`{#D zlVQbV`7t@E%=#)O?>MumpV`6^Do_$CvJxus6WW?8R9+>t?YPkPexaQ#;c6w}8eQQ! zE8$(P!u3(Y4f(>2&BA-H3hy5lZWa@1)fH)X6*&|o(wQ&P-7IqCxJXaG$T1eTSBZPV zihIhBdnT28u8Mo%IQLRN_X=ywHKj2(tj65(8*?Xh%)P2H4~~y{)Ia74OZ2Id=rb$P z=YFCuQbk|pi@t6aeS1~({jlh;*w|0HV@F)aevKOYJ%8+v=CQx7j{P${mL)F6rzb|{ ziwQ)FF$H46En?hjVxk|##KgrV^u#6Si%Um~%NB^sw}_9wCO+|_xRSWUBs~e$`4Z~U z5}E}PS}hXV*Ccd4O6ZD@)7KkkIDedx|G4RC<4mi^nfH#H@pRlwwxqSPE%zQSF&ZIm1S00%f!x?iI0{^ERacVky(9BX6;9rRB_pKJ=skBl54bVZkp`+ zYT3MA*-cMnx3J|3l;w)71&Tjf6n|Y){PR)qFI$N{Nr@k?Hn}N5 z9|dESm_j9?8YPiFr7;6aWBHWDCn=A!QI-l&mPuEZt5H_yQ=TxOtjIS>dD0{mn@MVJ zlhk7-X%YMrEgeUi?nNxBj$`uZw{ZYstxDkg<0X00j~*HtV(saQ#<+UTp=x~ba5 zsNzr0I<>0Kxvo0zld7wPnw!3whnt#LjGA|$ns2MxqU&l)KH+Cm)dTd^gWS}EW7I4QOBCn|f{1)EhQaZv{-flRouc&C~~dQy&dXeZr^nbdt_98=dEFIzur! zFAH^Ex9YsTuJitr&alL^Px{kF+@^hvnfASK+K<+0zphXF^JyAuoGzb%E^*ftSf$Gp z=?b^$a&PF0e%2Kmrzc^cC+V&yy-H6uLr<<&PvL~#glBq+{QAl&`YN;Z)dKZ3GV~|c z>Q6bLKlPdZG=2j;6$67=21bDf(=!ZAYYoiX3@mOKSbjFJ8fR!@U})=ZXt&DHp~%pw z&2SEWzvHu^>o_Af10xT2Bd=9P-bF^fZAOc37%lm1~btr@1}wWixnm~MY&x|83mTE(n(mf5aAv)vhHduq*^PMGa` zW_EzzyhX*lZI=1LK=Y0a^R8O+!zavpo|zxxx9C-|=rgc5>27g)mBra2i}P(3>n`4~ zxcu4T>bM!#4QAYQpK&X2#+{5A_iAU{Z=3P(#*D|GXY})1J{@N{U|>1uZaK8d@@0|b z>o&``H!R%wIQV{`owUC27THXhl4%1Y)h2Vk_Zx zEACAz(Ge>#Noxs1Ye^4l=~!#oVr%(!>+v_OCyrPvN!m;@v{Ci2QIECJ$h4VUXEWub z&D24gX=Ik3>MR4>Sw#8P4|bnC%ri+q-zSZ~N>;H)k&yne8WO7hq@?YMg!N9=UgFkuc^+x zVLSI$(A+zjbMF<;z283f;mx^^N9OiR&KoeCH|Q~MD0be<;(4#z=e@l-@BPTUVM&)y zhAtx>E?;9^z8AavXm|N_)8)^I3rotC&&ZW{x(dX(G9|9U2VJ?hTt&aQib>6vFq)6w zo0g87FIzHS{^0!ax8_g$GG9r`ZIY3js;8TJoSSBeo7O=$?OSd-U)*%1-1Uvz4L#kB z)h0@~tuQt@HG)kMnIP@ohZlyZ4sw{x81GQVUy+7Orde zTzDvMVQ0z0?t=@D+*)|_%fjPQi~5Wfo%CFEI&RU~l11kaF1mPY(d93Tu1YPwZnXHO z=i=LOi|>{!zJGA>!&{3Ve_7lwwPe6($)M+wp|~Y4OP0Joxa94vCGWp18J1f5$!O__ z=hClnOTU*a{c&*VuUkw1d|Aqp_Tw}5BVK+2@qSFHpYS0+?rlHOuYO|E{u0Lil3xDO z@&2-<{&MyH3a9-i4EZYx1SqQqsMrOl1qWzk2TZOH&^i>LeLFzsYk;nFpuTaSp;w@B ze4t5oplN-e`RTwJLxD2|f~?hpX4wVJ4i2)<4sxsyay}h2cPPk3VA*{2W$t#%JcE}l z$X@1Czii>@Ws8TFEfom%R}T)f3tko+9FiRzRv#R3I(Yd|@JfM@X!Vd)b|G=WAqm+b zN%bKqr$g2Zg{%_@O;ZoeunWx!4$a99U0)xXcRF;_Q0Nwcumbh4BD=7X;IOUPVdeE< z+fIj7-VWRGHLOZHyv8`Z&MUk=KD?ncyzx-@-rM2(zlJwUN3td=OaqQGMlSyOm#pSANT0Iaw(lO!R~o(TajG${H~$_AzP;V$>62G`GfRb;M}jiP8BMqbsvYfBGuJ1*?n`R+(&F zW!ACE;?63|Z>y|iVr{0!+AfH-ONe#Y8tc>%JLgX9yl=6tGI4Ix<2)9`c_qYoZ;kWq zh{Nx5EO`;PR50FOBRHFvkJ zx!X_q)$1UKJ{h#G$un&GsD0k!zeUkdTxekLx%a; zj2SO8W-^)9nwhg4GG~Wo+UI6EHe@=V&7Au((}l^JubJiUkmVVgwIDakry*vqN&T!y2+9&So!vnZ1(9iPp?n<&YESofDs!lUSCM+?libZqC~8 zIjM+dHFGl@aUnH{|A>&E51ecMG$=Ky!VO!}^lY^;>h-mp826c6R;tm+N;j z8}Q5UwGJD0g>KlLyJ1hmhNiO{_PyM2fXQo-&1=!jYjemu=$&^cF|RW>ud5;NaA)3; zyLm^y=N*^b*k`iwr1!?ti5t(BZ9Lz(@#5W$m%nekD!b{r$)=m$oAApqcXBt~YuNPQ z?50OAH$7oCKh@m)%whBM(9JJ$H@|Aw{HAmB+q;|Jzuf$R+450#%O{g9Bi>uSgl_qk zyJfU&%a6`2zwU1N^L-0TE}zddpZMepB;_;Z`NCcK+59?PD7mdMrdwluw#FxIO)TG< z+_iP}y{&6Ux2{W-D@!*m%k(MBPAbbSFWb;nw((xs=Fzfzx$;8O@?xLz(xmdT^74wV z^2&SVJ4VZ^XRxO$}1YXD)!#1*gsm)EVr%IbX&X6wnItVI?K0pcWpaz zZ`;w)ZO7#*`%EiO`c$4ysyws4^4y-v3+E~?y{fz-wEf!T?Kd2^-wNA)XZ`kjd$!;2 z+Wzp~_Q#{!`{i~FnC=+#*)f!~<7N4d*Ihf_-rMnhbjPsJ&X1FKes@21tw}|#S!1pF z`Pvz;YiA1AS!>nJa;lphUT43d&atu1`F!2n`*rhv)Va#fI;k(N#d6##^F5m85 zi|+4Q@?)2ue0_jfeUNW`aB_WUMSXa8edPW66+h~uqOq~Naqs=c{XZI;<(pc~n%bP2+I^c2B{y|eG(n+6>i4Qckw$nb0V4r^O}X4nnf-&k9pHPR-{FIO3OIs7O999nY!s6T6KQ5>MFG9o3|M*Y%@-2GuhT=cDT*r zL7U~9wwWUB)>GPT%-d}jw%et&J8WxrI@~_zLHoR)?XC(3-OLYqI3M(kIJhA1pik4m zg%=JkesgfC$RYnJhXS1sEsHo5l6PobSks}13-s5#R{T5^rO**$-VwX7BR-`gaa%|7 z;f~c8I@Y}DSSQk%Hl;Jexic%GGbgWeeN$)Nh0aY6Iye99%vb0tH18^2*j1X+Rkp3G z;&4~xgRUJvyQ&nrYs|ar7IyE7=-!>zy{D+~$1vV8r2$yu)2h zhYw#k-1Fw}F_9y^Q;wW)K5{DJ$eFw&=bDaOxNzjsnU!m&^0$38nB z8(DblYs#_j+m8J>eC*eQV}E`gV~sz~XK|b?IxevKI8%9Cc<*tMi^u;Ldv^jAMYbmT zK5~;lCXkRo0?8<#g*X;CqN0_e;)E6|Dk>rZD$Y0~PN;~eXyJ@A&O-@>6QTlzh=>Y^ zsDOxw^K3blT26KU)H&zf)4l3+pV!^@b+7fVYyJFoMC^#zv13PMl8|4hp5>UCoHlQK z+S2c|)wt6(38%H2PupdlcBne-^!c>2L$0fLZoQyfx7oSwn{yjx<~FX%ZTdOa!{JOb z?=vle&a|3+#%uGLwwY(VtIl-zd`9Px=i{B%IVjI}c3!v5c|9`o{HpSLf6nuFINR6z zY(UW20kh8rZazCW^K4Mn+2GG-Lmcu$z4OC^@<-0jkJy|)=3xHV!u;{i@+X>{o7DK+ z6u)!R#+{pyaBk+obF&K1&3Sfio=L&{#sv%g3KoqkSe#I>h1uJ;Ey~+8N zjnA+4KEKBA{MvEn*C(9caPa)5!t-06o!@41VSD2XJN+)~9(Q4H!iD_@FZ@_|;b7H; z)Xx{v94;RAzIZg~V*2ch8JjPj%)FRcbus(%#TZAF#lpg( zXN8wciY_-Uy5d)KZCp{=?4pXzMU|OFRaHgRpNnc7itl(A-wP^!FuVBC=He%r#m}mW zUwkfp>2T?__ocT%m)_03^daHWhs~EhW?uSSb?NKpOOj)WXkQ{9T4FG#M6spBD67Qe zW{K&S61C%Hi}sf_LoZv;xvbrC*)Hp{!_CW1UoJa4mbx}6t=p^goAIUfmzOq3DQ#3# z+T?lZx2h|iO|CTWb*1I_E3M~T@!E2wZPpdeK#e=b=}9=UnaIZOhdj zSy%mTUhVzms=wp4zU{9C481mB&b7cT*9N6r8&Y&_=<{pCRAnKKWufiM!iScPoKqID zrEE-A*|?i!6TXy1I+jmvUp|#*rHwD2vAld{O8Km!@;T4T=cy{@H>p_It76gkip9$- zmZVfHE2>!jyy6Gd^_5MoukLky?fC2KmtWtIa(z?L^)1h@Z~Jn6o2oL|v2uHpN`8vo zHMDZioXWk+EBB{V{#aC*^1SjV)r~_vZ|lmtbXyO`laKoS50oc>2-@|R=!_;>$j9!pNejMd4B76RgJ7^ zjb85>{RuUOD{7RfHO9p?suwk8rnk+T-nQ(0+iJpXn-#ZhQ*YZB-*$X)yN>A{m!@~> z_P+DYggfD}hN@3x$9xAls|;t@nCl-}9@!*Zb=|f2aF>JKPTlzCU2@{lKmF2WQ_8s=gom^?r!c zgU}8S!h;`-oLl>hywRx-#uh&q|Kh>KuMZ-f9!~D?aBA?w>2n`OZG9M>{cv{m!?|A{ z#yCA%(BV;R@T0i7kK#=wx75UF2P?U4OP*kFt2G`G6&5jk!l)6$rvCM@4O>&>)<+lV zxn&-m>GpBy44#hmPk$=@r;l;ScqOySeI@g&eGm!HQIe{n-VX6r+RDTmJDJF>BNL6C zWug(@*O!TpklbZLU_^?I)UB~hbVpD0K>a2%;ST?AWnwr&(a}RDd~g&yn#;s4oM=J0 zolNY+F6@C(FPV7lFB6}J%EV`UsWnU{3`1mMXoO4zPL+u|(`CYAu1sv1ClkrAUnCRi zIGMPQi(6$P@vcm)#44L*os7ZAy#4)h7A!S1j7-EFoa_SM&iZ^8)+1= zig**%h+z5_aWru(#-N7jam4YMfJjWlBuvH>OvN-z#|+$NohafR;!I*RW??qwU@q>m z>>lP(j=_8^z(PD={(Zz!UW7O-Mm&~aDV8At%drAKAQ3CE3Yj>A0-VPqJVlcrA)2D& zP$9aY8@j`Am=MOpg)o5%rZ9sV_2G^_=!?Km+K+oU7AC~JlQvSoXd(Jz00v?>!jOz3 zIErIPM+Q#d0xqHuMJUE4JjYA?0{K{uaT3RfMEro2XgEcPMwl>_^MNkYI0rL?*fxuO zB4h>U9bpK^2#myNjKOAH#P>h(L>BZxKlDWa`eOhFq7?non2-KY9}&XmD1CQ~X@sV; z%?u%&PYU6J+nMxVHvI=R=AC1_W!p%l1wvfGRb0cA^Nbx#!*t9*yCRPFXB%l}1;(*Y3DbXPTZlq5PT(Y-!Mur{uz)2ru!1!jp*60w&=Wac zdg54nJ&}&%$iNAlLMF11jh~T&ZXNVQcLd`pUg0f%MYExL;_zseowJdS5Rc**(s3La zIDwNmg-i^iZWb{cKO+aHk&83P!&&6x913tA7jO}UC_*u=;u^|OjtYFhNBo9Q_zagZ zdcqYcVS*)H& zSf(db33|dD7SJGWg`PEl&Opd2m zPYk|7zr17Ll3dg=ghnpP>&S&!1G#W&$~5ZY!6h3>(NZpyFoH2m@DoyzhKhD_aUCvQ z<)VIfx%jr1TzJ3}KZMDJ`*^t+K0_`-5r)xGaxoU;FdnnWiTF`oegmfylrmq)sysmU9in zych#fFWx{jT4Epqp4mtvmKlhV7=;M*XWnSy7!061mN*XMF#!`1iAk7@DHuq-sl;iR zjv0u;Ow2+wW@8TKB9Qunu!!<$WE+U5xOUn=T*VINxfU9Tx~PY1Ma+lWB?D0(?r{9YMtXI{K)l8qI5GW} zScmv4@g3g7nduM2kN6Fr@EKq56~9BeY9M5ArLGI=q8`419`gmi(>h{8-9L<)vg8i)|&;tZ^B z7zi6IL@f5>C0?OPm4RrAshEZnIEmbwtcyG>y=x%0J>a|}1*ynFvdmE2#{)b;urL&t z^$f)!Q$ukCbG;0OQya=ULlNL(D0=iX6tlt%MZ1ZH!Y^2moNTYm} zcnxJZ#Png5Q?9^uRN@Aza1+(Ig&N$(9o)q|+{Xhv#3MY$6FkK;JjV+hVIRK`U*Z*B zW6&N$F&HP2!SeLIhT=FfZ~`ZB3it5<5Ag^A`wT^YWaDT24%vP~(IC}Oe1~FOLgY{E z8*h#oig(y_+)!j?7>dtXhT;o`UojN1JmPW*e!xn!5DL-HRw0Jrw7o*)VoyVb*zkqd zc;Y^6q`V3HDIdU(IEWOa;wPlx5Dw!Aj^Y^7aU2;qfs;6ke4Il8&Z7v$xP%g1Mk%h~ zDz2e@BZcVER3Um|g$KugRXB}YjBKG0qY#177=y7Ghw+$ziHL+_ONH2pUGQwJ5W^9Q zaxC>yh#PGb!nT7#sJqigXbE3*gP9+F){A|8&%Uq-aX2i~O7eaRp^qN}=p)<}TB+M4 zh3Jl`sR}Uzo8~Ko-4cbcM?Ew~6UdjbE)MOapHoWegFF=A zJQfwu2Uv^;Y%`v?1WU0D30Q$2uo~-d5Gi;_{YN-N`7n;)C?40gjaY_W*pF+4)IkmQ7Bhyi9|s^x6k^R~g}6|r5T@k{@&1-V{CZCzKH?L;;49qj z(+4l;17so_KO^B6&iOmW7he9x*fLd$M{1>*W~UU>F#}NubySKDb(F%fsZvz5Rf;DeYlTqE7)eDQs|)Gq7)X; z;M-kF(KS^m9%m@U6RbX~6zi}ayNi`Vd5L=1uI6ibSCqmBJGa& zLJ)>x7=|MZ6EF!=a1K!_BN2^xn2+apfnV_s@8M=@BraMQi86#Za%@trMr zVFiA`O02?ati?L4M-nz*BX(mCvXH_yBI>Zsy7Uv`(W0J_D8@lIBk`ubk$8(=@g5i4 zjl_9eY-J=0QG~udSN24(MCcx%SarCa<-8$h6y~;96I=*6FQ?Sx}z`pVaOc%XReWWjW_rJ zpLs^&I6h)f40SLA+wdbEE;15xm(p%@Sjl#A4>s$Jgci22gUu%TDcML|!hzlNC4SCe zT>NY#LJ@{=jKnBRfbVI>!WknGTxBG@YK+9*r;L%;94nTg?Hl&_&Pbf@WGr%_@ii7L z`WuT^cn3RItyDhHSX7`AH&BI}sKzbS;5P2yF7Dwz9^iEl+ro*V#^PY8u}H!DFk`V} zys_AYhBJ*tBQ!=6G{pn<`4AqI+rb;1&>3Cei$WBkezdW0#~~cXy%=L*n81GOY9;kb zV_}X?=!|Vh#^oeqQM|!eSZp;GV-6ULv>%Pdv2G3Or+dfQi%ar zjCkx9Dst(oM3ARe3dS&eXQ&cqa27A|UZD~?rAnMfJj;)(RN@i7n5slmGnHtLj_||x z7=k8hl?XIv+Cn81mP7=ik(Ej`#t;Z=l^A2I5<9RL{Tyit=NVa%}E{QtrBXO z!vdDjKnpwA!vU^{!^Vy(v9q^I>_Qq|`%~_t5(NV|2DBNf644!)%pGWUo<)QEjwR%Zn;85mPY@GZ2IM$U`-5;Q_wFzDyFh+SNQ&EBP&ZeRYH&KmSwb)Kg zEy}lX2X}D~_wf)9@Cc9b1W)k{&+!7k;1yotHQwS^yu*8Zz;C$Cem)XE;WMNzv>66a zKnWwLU;|s&!2x$z*O7Racn?mL>%bZJnRX%8#RJOqiSBqvxgpU5EzlBPc*OiR#Ey7O zxf8K7y1*A*(GA_v13lr7KIn^n2ta=fz(53HD1tEzjk}tPCRmRoY`{kB!A&^zqWy4& z3tUkb<9eHl@kqu=7<_Lk*7h?M>rgn*RG18*e_)EqL+N8w4l@-uFe}tlEJX%R;uJEG zjWZ}jF-mY5rMQA`!%amqSdBIn*06yVwy=Xe9N+{;)PW1mpadPJ(JymM#kS?N8ztZI z`ir(e1W8ICY<344>ZHC24>Zhc*IP^5IYdRgAN_xgZa$wMC^jj z@I_ZFWc~tlqud=m&=Y>>h2Hoc{^)~#2ta=fz(5SbV8pVW;ly#6h6T0run>z7hsB6T z0+wS1R%0#JVLg(t0UMEw?Rbw5_=r#VjCM!Ogg3Sy<6I&xgYDzq2{S&=;GCZ{6I-zj zS*OfIHcms8X(mi@4h5+GnRAOVIc8!k&L9tGk&knbPMZlC1oR+>J`7+81(Yy?F-)L> zDa_!3pOA(_IE*7Wieo581zP2ri5s|y>e})J+V;Ixx{X`4WxT-$e8gSmeH2RK`i!QANC_1?w4o-60s5+@ol-8@I*7{u5*2%Bi`L){MK+yJmOk_ z%VX|as0(+TK_0$)!g^@>f^A_Zs&EqW+pD-GOpn0M~)Ba$iPKI zwJ1cKv05xfJdRqcMN1pz)lrLtx@xfklBZf|VGEO1tmCB?YMom2Mjs?~RErI`*hwv> z`KrZq%s>=oA{w(W2j6s8i%Y%LV)yrIVbf17>W^a|PSYT>_4&F6V) zQG^@N6sd(Z>RnNbRx)#O&BI&>Pjk^2P0$qIq8XawJN)KtE;546#R&xMF&9HI9HDr> z*IYC_%6jRn2M_#^X)e|w3AwnOWiFg)sM}^8iYSF7b6h+)Lblr$6Isp6G9{lFbAS%{g9ErfRy3-JYQ+FJ11VxOhhDlon@cs4S)1Q00yF4zJ+LT-a>3Bu@D=v8C$Rw$Dq7rA&k)I zjs?F%u@ndES&BW)Ek&@Gr8wTjQXDvGDK<{w{X6ji^iEld&yZ(Y3LiAjwiK5z>1Rvs zZ+WyUKezQFPxdz zo#;w*K@ZA3;fG%6jql-)KIo4D7>Ga&LS5DiCe|YkCw@b8!$`^z7>)W&yJI}%NtlAE zn1< z8u2?8`LoS_>?c$sqA?GPach!B)L_MAjrakp@Mx+={5(w~a&ZP1@Buq#Xv8iojM9i$ zT#D8RqlIj15!;975{)ofrV%QfC)eVKEtI!ugelCRhB+S2<>vvSB{Z;tHEf`TE$mjWjj}6#}T;!wc4vpvuKYWit z2*OYVV;I6Q3K1BKafrkuOveoP?9_-?d+6J}^eb$jg)Qu04+l8H33cEMS5(4gAN64i zJJ`bkj&MR9IKu_5_y!p`ftdXou^l_G6T7e*d$1S#upb9-@&M=LN6ra4A}@_|7sEM3 zO}a+h#{(Fh&l;d}NqTXk?T&6BYV=UTVVJu+uHLlGvjrf93W>(@ezThiV!$RVF%Ur*idm^vqF%a{Xb5?=mDs*iE6sgqB`RTiKnlugy-+HU*B46#wKjRR%}Buwqpl&Vi)#cFZN+S4&X-| zL<)YwY2@Mz@^BXUcwt~IUg8yA;|<>8SG>b}e86w`h)?*8nF?z$*4SE1L?k9*GNxcE zreQi}APO@PjairtcN1$-M`bN;+FFal-&>0#2p(lE=Ehr#mYZqsA#1TRNh|dZwGrRL zAAQgl{Sbiu7=VEY#2^gD5CmZ;f-wvs7>-bcAsizx5~DB%V=)fnF#*%D3<+3{b?6&r zBaYCnV>pWRkv8HuGLX*niP~x6$=Z3uQ^ZVU;b-LFG|nIoXOWKroWprsz(o|I2*tRB z5|rW!%5V+ksK9kp;s&a46VNbywjIj}OF%P@3 z2Sq5x@UfgnEQ+)dacDltMzp|uEWkosMk)4e&`LkdwGk_^3ae|&3n{~Qv5okI8wvCo zZoy}xR$924<3<_|p>dLp__Y4cU$kes8?ghiJFy48qpU+mbV3*Sq8q-4Kl)%024e^&AsPX! zzk)b`I1q;@2NLs$gNQ}Ms~AkVoH&Gdomhz~1TlS!cpG@DA@0O#Pww zK=~tnhwPD!(8n<58xRelfD$U0!vfCu25xACZ_yU*&=FnG6+IA$!5D$jh{O~`V-DtG z9%8T%!`b&T;tC{T8;&9a@9+`gF>OQ$A}|IMFcoq5@hQjmoc+GA5v||Yh&J#-2Kwl0 z1?hiZ1wquedpbHty0&{pM&z!d&*XRx9k*Yxx^Mt(1w!`?aFo0gekg z%)>W7vJQq~B5YC_FSw8TKT&rtx!g}^#YjY;$2G0!2|pO$V$7lsJoag&bGNmk0GW5S zA`4TWa{ibf@S3rLff$5948{-yVJLzz3}Np$j~IdXA31hmEA}7_M{x>ysKFt*t*Fx2 zinlPavK6K{(Uv;+?nkZkw6m?4+s#(QVkwM<*a~Bq;NePJk-W`T?7(*H#4hZ{9_+ys$_H)v+fn+J<&)Cr^F#DG ztdH0V{VZEy07Ep(vlY#ef>vj3MaO(wVUwnn6z6P(5=JnF2|iz7KZUlUKLRiigRvfy zi)=+wv8~vE;1c?}oW8|1L_T8sFF8N&ZACU3ezFyf(FEInw-v2rcH+FDooHfXCuX3n zt)1wJ1$K7ga05G$jx2QbwiDgavy+_|iRg5#WZutCSU>|SSi=Td3>#o4LNFYmm>g&) zrr_96wu$4&zzLkhDP-XWZq-hYv=i&c+KET(CvKdbSd4fqf!=sKzNlp<@isfLC_^i~*<&Z(;#a&w)P6g02#0Y5$MD^cZ0`i`ZSrV8Y#?XamS_ik z%JxJDIKl~a;0zax#W;+|L`=SGCyXE32^CDCh8Yqb+X=5%cA^d1q8+^P!Ny)#YVAcw zdwcP;DY3b|__c+-$nvol*%;l~UKsk?3uCPRSu3e}*b7r^pqvCV%4(Ry0vc>&z7=s3 zaWl4HEB0ZspS{>moI;$^%U;Yt6lS9OdwX#Uw{Z{mkxtzQ#D_Re`4RC2enB?VKcmXu zUfjVw+(%CBI>gh&hr~yCj3;=CXLybm_ysTV3a{}7Z}BVM;RAldM|{F(e8umO`q&E@ z^q>y|7(xLhj9~&5ETMrFtWgKfaD@x%;v0BkBBo#l=3^!DIhHNNt=NVgIDi}!;TG=T z0p8#pKH&?#!mzKs7={o`!p}(R$9cp?oWM!E#v9lN*o(FO?ZrB5z$Vjp6YNFs6#6B~UYx^u6yx``_F~CFd-3C8`t_u}XoyjnY#ST1 z*j6^%g5^23e}?z11@`_sYmg7Q7<=4++t4IRV`L?ODZgGlsq5Z@H=c{a}Ub`T}_?t2IEv%iCA zG|)jbMiXQYcHr|I2XSGjgSd!#!4BeWn1dJ+=rhYf^hH0+o97_f#5#z! zXb1024x&9e;5+D0wckNhV|+F1-Et7J8VB+Iwu5N(*g>3s;vmL7Wf>;m+&k(Q@%IkI z^SFSE@P5xWp@RmHqu45U6x)!DCHjuyeO*WK0jBjFg&EW^$4qxeejnv1%HYx4 zQA``;C}tx()RFJjIf`$mJMwpAjzYH3QOt{T6czD~!g{Hr_>;_@tt5AO?INwb_DjY8 zAbY3WUowc46p512|AEwFN6BSF- z6bU7Y6)zQuYUL^qDN(CRJyciyRb`Q?ibPdqnyRWqRsB*`qc*+cVS3Ns^g-l* zPp-~ZlPtYfTDCo8>3!L7~VLd>m0J*~t0SdW}!9kJ4S%pvP>m#rtfvW_&jne6%RCF)$M zU3y5Ha9O+Jl{V4bc9o~?nm+%DES;C_vtQZgm^?h-rwivlq{S@GiN!;IseGaX>{1xq}18;wX@p7#iE&urmu_j zWEbr!7rVnQ4y7(muU(uiTwR;F*86Ww%lYRyIVZcvuX0~{*gc`teZ_0{M2iNinl)I{ zx52u}4U$$h*!W+PkaJbz3x^vQmNqVa-MGY}NolhtSNk?8o7|*gRg=oYO{z+pRKIRg zW6|_Zv!?g{YqN2#_Ov_V>2Sr<>5ZqeWi!|2&Fb}Q<~F67`|4&5k2Gt1rCHNA%{(ld zH*4O!MZe~)rZo3j-MsC8Nh;3Ot;QT_HSS8Q32$0OTDG3ty!F(6t*1|E9ksf3^pV!H zue6@~rge;^*MjC=vHiT_rg+7#_F8(xE8&XQiZ@<~mTguwZ?mRfn{`v#B&}|<@kpD^ zSK4fS(cWmDQ$tp2+koc|yJ=e++D893e7_%uA~)A*`S)3-hznoiAH zbZYTGFa74asoj&-bl-Ti`{t|Nx4!M3tm(0%MUPzpJ@!oPv2RU}14nxtyxJr6ZI3if z&%-Tx9u4T3KDB4Ynw}?*_RPH6Gy84N9F1RY3%|Sozx=6w1#A2+9Q7-_>R0^MuSCPE*lmUNqkSBXn;z=E?jI!ETo=|hJye?YMuramg|Hgdwdk&)J; zCbt?jb-<|U(?>h0cEH4U(t{qCpE0}qUuV~hUmvyfcvM1p)Qa~}i8eD=wVt_V;J-+%x%K_bWSi(6t)q7hjNUUN zdf)o!1IMEemPe<)k503hb-4AcqXTE9&zP05e%8t3vogzPWxt=5V>3Iq_3XTXv-4-n zE?7VN!tvRK<+F?5&n~fol`_g9K>(=w$4xIOH#=H;f=Y2dr z?{oRQukYtc+8E&#BM*!*h>B4p#TaG8m{i1=euz-aLFaMTB^Ig;rAN~iV(DeT!e`dg6l4t%WGH3pOnK-jx!{!SoHWyysT>RVS z65B1MZMIw;w54q3mU$H${)x1iw%cB}+4gqOws$kPeb}(=_;5Dzk0M)xjxc(J2)hQz}oU zR8^)_e@v;dOTE)J_1@sr2hpjIHl{u~nfk0U^~J~3mv%qBZu`^Q!9Tr={^`TUpFW=a z>2u{zUqAjN*{6wiY4Ray2D8!>o6?L4?t$-%OG@~yNuW&8F8~R;x}b1J(ZF0&t=ElGwbBOO(zeWI(hKM$<$9L)9g9fy8Z9W&Bd2V*qxw)Uu#W)l!@GgiA zDu|n15Wl%#X=XvfAIUHi9nP;J6K2r)b+gYWZ9cy-^Ze$j^IJckPj#N#dUo-Ujx;fXA zwp`zsb$#>A>s!BEC+Fpkza_lfx25txR^`E)m8oAU(;ROcZhzzG&>QJm+uwRS^wzsMw?1sS^)c(#=bN{_ez_$%)rbx?^57bSxiyNd zHAdMrCe<~jUu)D(w=Fu{)&$?So_kxn^|oF1ZHMaHPG4_3JKb^ZaHn4I9k;o6+_&Co zn0=>l^_`|)?|3-fZPwv#i{QJh=HB(%dbe%%UGM6<9lqYxIoUO8uFI_t z#$-PjSN&kZKbh!q>!YRFj}odMt@!#V(dqH34v*IaKVCQYanjbu8?zs8u714r>*Hjn zCp$Vk*%ka`&)g^bwmvzK{p4Wvlhm(I(wv?i?(pM9v-)ZF*QYs7 z&vH9F%L{&%KlfR|)@K*8pA}X=EB^Yd#OZlyhv!#=pO?*jUa|FgW%l!`>gUy8pVv6O zxYOaqz2FxQ=Dv8e^~ICy7tg9+y!iU!rPD93JN)uC_?LHcfBCTWmyg-Me6IfG>(^hT zIxod{FXh8t8q9mC*!I%s=a(k8UYh>?QeEej#dohX!(LgNNp1&j&33R7f6d-$FlEG~ z3Dfv{P3u%|>lH=9dPk9Pd%1A7Tl}hMhkx~<_n#)`OcYihNzLi{M`q4=nS-3q#(NLy z*F~mq@7T9f$J*yxb?*`=Gc@VcFVH~dr0&?SbH_gV&QdR>tVs82oL?!EhV6cl{)9XtE?3#{E0_5RpL z?|y-NYM=RyRsJ`B_dG5RK#z)AdB4uaDf)A0f%5JZ+4|vWG{q#--3x zo*+~7@R8K{{*uWd$vHMHv_}Kqcu(C}rQAm$b5q~d>2fmVybDi~n^?H0g}!;LreB_= zTyie6aFsj~8tEUrAFPN~4DZ|a?uoNY8^kFjDKI}lic=@AF?4a+)y79Nx_5%76tO%; zojmBG)isa_ayaAu$9z(oMgL>c`DTN)h===B<9cn zNvF`y(TT=Sz}87NgaZTudS_whzmZ zTw^=Sm3oR+I?vr2Jzd{ewNBw;Eq9U3T@{VktBH$3sUNS*Nm?UUyEW)0NelQ+Q3JYL zC&^+~nz+OzmI>z+PlJR8`sz0c<}TlaDH`q8jO@|i8(TWkS+16R+&9Gfxmw80_pcn; z!zHC-MrpS*^r95DrkR0#Si-MydLc0#bF_Mi9rLw%37*wjjo!Qo37+}di3#SSdz^_! ztdYx-7P%fFF=_*ahuo)|bKI`7GLJA@7scq5V|^^7`2 zHqiG`xFl;7wFfKdnw?j7@*BI`TSy)Tl80{tXbDtP_oFrxRHIWT( zN#)K7GHG-#NzyvsX>TK`o5a3%@Knk<$FUh@`RY+pj=rSxi4TochsBgh4cy{G5rx zKLcAAlU+jTYR(uB(JshW`^2}fi}2-_SRM+Q#MKt#p^(Bn@`D;U$H-#4#b%61Pz#H` zJv_ow5`(80#7g-dsV!x`3NFUnv8EEE^SrZMF}jzgkHq<*8%&eBEa=`@W*}*Iw)8iW z_Gm(+2D@$h8kr}XyCj%PG3L3>K5JZi$l~frrHY)SM1yXUs(DQJaIMc^J;qn8I!7sU zi4k)1*gzkx%Q}0tXd|sGlXv@N!y1M5jO^9ADW}!0CN8J-A8s{{RUcAJl451~5k5!D z;+$jEukxj2WhWo|g=sRadFYyE8>fUwE&>4f-k#w9WIi+mpCLI6np2@Y-udFF~f0=BxFrT^6bh<*eyppk@XB z$$cc3*o+#X4Ljyrr5>w%sPNpW)phPRTK8y`XPnllCqx@3v0Ss~wtBY(`CQZdYB$NA^>`YiG@+i4pHX;> zr$)+KK76SnN23dpQe@^jDZGVGe9zIl)>(91qFoc`>24d+yMcbjngsijwfajvbF^!! zA|=^s=h5M7eH1N>6VpthdMaeJ-pe`GL)vXqHOo{ka}ciPu0B%t7`3%ak-3i(cR^|6 ztcG$fM3=@cl5|m9H^;rMhOsnX@^Nt4WiIm7J-zD3szqZi#Sn!~jE+_GF*0|Rrv>Zj zC3GBYnWA)#(~}ZBYfB!S6z(ArB2ktjg&lR1Ym?`nOX3zN8E8|K+=N16WI4{ej~WcF z+e_goxqm}X%3>u=OkG)mtXo7(qFY|KFv(Nr?3&;voI~Z#a*Hs@IZ=H&COlSh-siGF z?HnufZQ#i6d>e$tb4!&Sy=-6}5>qDEc4iQ2LrhBT%u|-s?me-r+gP6%nOX{S&*@pAveZxv9MFRw+ag+CwT{(|75lIIbm^xP*@U$M|-M zRr?eU?wlwInX{r}tU5obOJ|SWF){<^-U^o-`#80Ef_k^N6w^GcMQon-Zp*F(W97EF z3nuN}*LOijIX8ul3+uI5;1M6=8>4n5NnfMW8cF9q>?{|Ti=GKLg@f{bJ`D_Fh8wU% z-NREd3DLUvDLO7~usdnKWGmUN>WT`jzzK9z`5R-E7d3zQ=h`ua3*)2=G@812EWLt0~z ztKFetxaiD%3KVgPtqinbCLWx%81o*sVex(Q)jL(q6fWV*J#zxKP5VUeT9jT=7skEyADo8qlswt>}VgMazf&nWEwsh~Cb->b5nZF>87^CG2s%VZXk zp3Fxwlf^U@y7{LAm|7r{^kwcL{HNF~W7=C^USQ@Tljk@qxC<)#$ZITHu~5vG^y?eA zQR2?5XSvw3Zk&In)`dM0UNx!SUkEA@=&47Di`dP#1@EB|iD7)DN>7(OlRuVd!5q4iZmx6-9^+&(UuQ#)q<(MkQ&0aVY*{ci)v zqBHMlrAgB!P90HOm^yiQ_=rCrHu`@o)9U|G_x}!-|7n9|-^RZiEGFYeOqnvg_Sq9N zBS-w@E_1=q`kk8&@;OQtKN<*EY3>Hyt0Ub=MwI@2bHxBS8u=Q?_g z>byY})s2RKJ7K3RlSY5nGWcY7TI;gdJ$g-mF5yOmG;wV}17|6Loi0;-kd~?nq(!QI z(vl|5QoJgX2e5yrEjz~t=#nN_EX-Nj#9dmOG0=ywg9vbHfdO z^;e${Z?>g3A-36;z76{~Ts!#C;3nl4!&$b+f|XbLZ11~2pxJ5Px@tx3uAS8dmT#iG zlY7UVc@-7jtY=)<>8T@=weHr-7DcqY5z&H%Z1;E9oLW1f^_8OVW;cBg1YEd3`NV+6 z)+??>rv$gTIbrXWS5r?`7r*}`Ti9XP%lDQBi}P;%^8T*X%>23*8+e24R+++$&FF7W z)%oK^XjtUr(P4kRuT^TSS6H|`*fGbgaMK)~sq@bd>OZ~ZebBRtk^5QA{o{`JS@!ai z0fRpr41THhCrf7(p!Tm_X`aMe%G!?uBo%7keKo{)sC^@43v+aV64P)1R%nJW)V}IQ z!5fY+1`qR*tf4_DoX{DQz$YS7?PqqTXaRn5mg?Xrg7F4Fpg#P;`~3gK-g}2dm2CaO zd(+jh8yegV4K|3{jSaRa*aQ(BLlZ?rMFd1lWYIxIM8|-Eq@ti;A}C4_1u-E8%wif* zQBhIQF^dUxbWCHM`>S^5p1JQi_j}I!e9!y+@jb_xT3xkj)vB~=)v8+64Kp6W3W1)y z+adgf&<#O>AVz=<&DbDVBWMtMAt(``H7<)xxBgvgxIjY?T8E6#5y8Kwzh^N1 z_cX(+H~*eu(Dl;Dxm1lbg~w}UxqAsWh*xL5(fIJ5m!-;!9zkoYNrqhIW6Wht6EQK2 z+s>%eG;zw_sfFXY+Rn<9c#%v-)~LC>)x<-!?1oi*O15i8TMoYof7~kRoE3L`U$=1@ z{g4!EHe^*wn0-dPt{bl!D7Q1RDN0_cbkEj3w<63fW$u=EWo_@Bjk9+r^9hrG<1%>e zMq8rOHhzel=<5o*rzi)mQ10Xn?(Ca+N00pGcRhLTpcLudSv>ctAJ_5D5A$9L2gaX= zJlmW(jAj4Vb-e!fWj}4g>_1oiZX>fbMmCqL=i2Pr0c+;J2<5+A*_^4^55Ge|7Z;&I zPo?{xPDfy*T~Rm+{YR?*ds6*stoH99TLgIue zlfN6d>-5aN>DnwiI@jjWk?$fwXYyZ)!Z8EWt$Kc@HQ(*pyF%q(<|?%l`~1V2{cZyq za^1=THQIdzre_T7P#COiPi{al4Ar>w;ops1SebtrzkeCUFj?g=S-%^rhOy#auycW` z9{!izs)pey)4=T2n$s``zZ^|H%wSNSD3Kb%qp0*3k>nYA9gZkzF{l}LsHR`s4`)DE;Uqxfr?pfr(rU{Y?0k! zCI>}QVD)C&shOYb7}%>t%wju^S!fqVvI4u*z=-u>k_~xgw&6@B9(3x^~!{YsGX7H!?zW-ZbRzOtTKckU*s@-=DzzV*( zvu(c3Z}@t2UcW%3Qdm~_?Rpr%hjlE{+KZ6e-#+)h^I;Cpinf35_}m4VUz1p2(?=V` zzG+y&qa`4{ZOQ5-ee17c^}b#FOfN!MgKzUcefbsn-{?gMK6qB}egWlb%L=}HzX)MP zv%mfRxf^gwmR^K>qZc7x%#g3W2-*MQ=0+5FE@g#xzgn=uw{NXjK?3-EvqZRn#(wGt z%r~@EnAGCP3O>=i#R68a4EH>^2zhe+oEa-T`_UQbA{K4?^(b2M^7}=|w>LgOd$WQg z!o$)&tgz;jyCLpD(5qj0R`0#tSqs)6p!M6u53F}=FerPYS+1<%w|QT`?wBuU1?y#C z`1u=qzMC&A{BqPAsc(0Nz}_7y7JfTw%nBxZvWAZgKHa&{Iu^yzyl){0HmsnZKR#Vq zy>Zke;l(5^<(@w$BV3v$Tf<40)}|DNIa}h#jve?uBqh02F4xGj7ZxRJILl?V3-}CY zX?B{Z=2`wAad)oT6p z`|(N(!?d)Bfgv(8-4lu&A7zH8dq9qlrIAR-E#FTi_C4ud+(#u@8mX4~D04WkfC@k3 z3{NY!{XRUG(PCWeqg=zKyYCN?i4Am6TD2nC^ZrhCH!-ow$@Wo7^-|n({H-Jzq|?;V zO5;VSAxbyj&nl-b%6Zv<7`2bmG2bsny*SP#zcyxNn9?P4aLmeqe!6Q1+nBh?jWaTp z)pc<$xrXD9HOF&n+2ravo@=b>ZZDd=lMx> zQ)|Q2GTp2+MY!72N;g$ujGz7E(iH4Kr8vvmD*X2(e(}l2sTyZvQEIqTlh{%0-Kil= zX`#ytuP*kt%DA}5IK$5-`*e|mPn0sXmT{8Fc`Zvnl|V0Mfw{jLNDp%j>Or1ZB@@g~ zrZtLnSJj5``ok}i~OQl1mopr|C2PfJw*!_(-Vk{98YjvY*f-vo)ep>YVX?_lh&=U zA@=IiOz+ZKCRvu%K{;Uam8VJG8!kS65TB`Qw5ZhNx1@flL(8vzOtP>Zj&o7Ittlx9 zaFfe63@v^AG0AVkg~x$iR$CYiU>gUz=?VgCNpg{|jh_B*iX2PMbMu>zMgAREop|k# z?r!w+<5Kfj--4mSvIep9mWy{i5@LflEPr6~DJf(dtuHCKGDx)>sST@7zy7$>x$@%A z#R-Qkmo+|nBT5hJc>S@-(z*RMTt9p7_oRLu7d?3MF*d?zK)K1~B%aGUUG*u+M;Wl_ zp+@e!+*2MOn(o71Ub5Tjl)IvSi>W-vXX7PX>BJl#qeY!~gRPc0v!!WbeUjVO-J2=& zwf=dcDYwefXn>r@hvUd6rkrbVb-s_(c0z=cSy$tXRldcI6Za%|#_kS0doSLV@3!bf z75)TwWng*m%3d)&%$D}>aCM)*Y{dy=POQv3vA3iupv$!_(yrTrD_5uP3Ejp!-#qI! zsw|`|VA~^~)gji4I;CrxJol_BHygDrxa+VjYkWb8Dnc70v>&U3cicSty5^prgC$O= zWdU8yE>wl~LJQ;eEZG`jG{AO^Zcp!~>yqG1FWv zd*_RjXEsOmzIDNRnydZ6Th_VNDE_n8sRwoTx#!Y5x7LT+jU7~$&}_LZ+wWRyub5>c zeQXRAyyHlfv>-jMxB10KEBZoae9E4^bFUt$it>=V@4Xavjjpx)PFGjTFNGC^z^wy zU*}adPTBLUm*wHHQ(73Dxd%+y^Xcnsqju_R!z03<1N1n=i9FCq&ifQ*0XV_^%v2Uer=Cqs^%TpFlFY_uMvjscZNPW zcq-!fhD-P2PeyoJ4KrQZ%d+o=9V5h6nYBeZKF+5jPIp@2nt3Y1*kM9l(~B9N7E^j& zw<|wX8)%!`b?<2V+vMh(S^)YXn<6PQp zm92D$-n72U&(6Z=(?#c^^v^~)ns>9QYz#Z!5@2(#Hmqu`gUTu2&t_AGD7ARB?eI%i z`Qhp!%JE0Pm@K*q{cNtBj=JGL=;~!NgOVkO*KfUkt~g31_1)y;Fx{qb z>v+efiM?;_+T6NrP1m){_S+s0JKX5o_Tbat!(+GAW%iigqjc%hH`_+n$5fkLZJg-0 zscX%bud^Yi!2*jxezL8)4;m+yh2o~8A#eAJ9d9cY`DK^wt{rR~y?tD_n8f{=-Z2%r z_=|>p<(FE{RZOlLwe{hWQxVnqm4OBO^G@eab@|aH%id<%nsern{bH(5MpAk#?08uWz|1r%v9as_TyjnlQq`}j@F!ue4dZ>AQuTtz=lEG;u&9VE9^)pUBUC@76 zhOTEulEz|iCu30$t1MrX_O}SF3vxW*?=pB`X;p)>cU%4FtFg{jrQW6C#lGI1^wSe^ zRw)fLJa;{v*CuP>#w$5!`uSL84ravGGoZx1oFBN^V1j>fKxLq9vu~NdkNJu{p-{-4 z5vzTz11cKz_k0eaJ2nif1jrZRNZsWwK&$OCZ*y-HrhV5wygB|X#4h|o1HX9Ji2O?@3j{()#h23*1rD9 zX;(1svhqTV z(+1xbr;kZTHeXbFSq06%oF9-tWrvfPz%_H}`3 zbG7->9@EZ5J1unYk=3j8qDzn7YkdN{Jx);M)Q4P|P~EF=$oMP03VYL!r-_f&VRE2V z)>h?Rd1Bc+$aYJ^6NvfovU3v>4~MB&V}$@%3B@6A_Sryt1NQaE)&;+AzoRyV}{VqR$5>HNEt zioh-HjrRvu1oqo7<=LB$N&Pmoc{ICrhR?&xkKfqMRbMxFbW@Qt=+4!BZ$H9%Y4bh> zsSOQ}M}30z((p9#Q&PXuhNs$cQ}pN5W_mYGUgBv#cTlIHWuf*#!%dAVCc1gi*4j2Q1?o~4^QvXim7*dq1d^G3QdYU9*T#JXd)KR?l;gDa!$ z3+MggrdSj)A?ok<#4kO}cJ;S8djbBh)AXMBpU-^%x1->{yh89X_4YG0*4`#5rSv9L zhY_)(>7_5d6sDKD^x~KvGwB5}u9~)YFg*!Q_*?C~sjjK6r3((8gyzd=0s1eu&3_J@ zD&hasfwSq~J8qITm6GHmAOCT9^r9EYOBfb^K~ew=z0t>m2bN`cQ70VHO*kR0 zb-Kz0xTkL~fa_mqN*FcEIXf_n6EEOVgkd#=?aZl77}koY0gmw+WH>TTo^Hl?4!|-d zJj0uyETVV}Ap~y*#c_jWAmPDGhhbw~c_)bkwX+})88bYaf+H0i3=#>e78|h0m?DHk zNm$`Nr9ECUpp|{V(2c{J3U-~P8K9hDRBWt}l}hXxJpm(J3u+Z3WnYMSmX)xd2qR|$ zct<={kO=3q;g)zyAw-odJCCzOkDbMM{D6AdJER{YoFc`G3I|3loWgb>Onclg$yqnF z&Q=Omj9S7TOOMDU9UrK|wM9utY1;wC!Mp&O%S z+&#dT_`CTFT@)k5D696;P9V#!S2z(o%Q@ky#v>hx1Wjc$#GjyBjHXMnjORTx5Jkl} z-jiX}yy67=MNZd*+61y7T;q0+VZu8T0qa%9bYU-39p-jb*cFn;u)T$1!b;paE7bzV z)5*yM?|71p*avuHY6sRqz&J=0<{axFcavN44#b%0;$*`9F5EA6au5ijpD|8u z)pEGc&}T=9+U+)&-;=7HoPbL1!#O#52+uH1iak;YCF3AqEeNj;Z)Jqx>!|23mTr(r zmS^R>2&1Ynm1BH}1kZzr^=&ze^?*mB#8dIBVyM^*riIniVnqX6&(XdxI53ItG1YMlK@X_PbQ=)z00AmNJ0X0~r0%U21L=R;@ zvOg0dRfJ=V2*WB?h#6L@V0+a5N*L4V1AQQ|cyZ$o&mHUF1a690h7Etosnvq$SK;kt zA`=0lah50s3)wy_Gz0M@U{5fS<`RX}tEb zwx@r&2}(_6f4w8pSJFxXD3D{fTf&*%(EBgD9UCBH<;YL}(_x6~z5iUSYA;r0-!E2K z`oir0_08(5=znjgWBs7xUePyqdnq_h0baVX3-0P5cAqA9;k6uFL%}MFyDJ2E2F<}n zAwYSqr`VTxvbK)(WJ=gJp7(WuSWGDo=*@dU?p{AQ30t3(pTbjKh;BwmXp?F;uCl z{&dq+0yT~wfFxF6x1E2!7($gW2AFEYB4#T*8T$>f@dt~-wEo{;iD(eE7ISSdPlqjv;$ z;hD{L)cf;_*<@J5l))Ra(axVKwUaZO4KHzUDp}8DO5o}#!n>qmI97@bW0?(T)_Qnu z*26oq4)}E_y%wbl;7=(;x&Y3We8ZhgEYY_N`w{2TWQDBH&e_i zNh}l8<(T9c-2|+Ual#;t7tiF_-Gxuchg=C7$its>3{ImaRpy_M!Io%UGvz%^7xes_ zevolmzYQkpzqn@p8I78*{>L?gH>&?rmh#)N|E_ChKBLi4<3Yz=5yn{(T^%MKd%6yz z$;JzIYy{t5nJj>DXDq@&PHxE<4iJQ?V|^_0{WeNCPv`V34KH$L(sUQ5R>fH?7^wVhCqtw#cO;xu7LM#V zp-GtqR`kM6H~dO$qK%0M&g!z>I?FfXCaHJUg_ua1=T#bc%-_P$qcl9vnbE}-4P#>! zuE2_b)Ra0X7*&dQDYkM8U60a+-<_EaU>(aaS-M5wg6&$MVdDr$*x!?+h70aq0i~f% zw0jcqMjsNxhhf4jrX%5$wdw5-%@Y)R}(m;K2yZuTBWftgr z%5c?*M{M_tT+2--rWg2ehL`9|GNtP3#OPQ)uv{36al#cKZjlGubh{9hk`tz}F}bUB z8|Y_uO5-LXIu`DmJ>-d8VfKVm5x6qW8dzeHrIMQ2oYcwdZR@`1R76z>qrdc21S$$% zB+n@iWXueXB~*nNXH?!x7}(o*$w{w-fpTk3XqAxc&eMyy9--azIi2Jhb>{^J@|>Na zTVP6x+~jSz%99a&L)xq{xErxwxnj{EL6-^6_GYcGkA;oymTT|;*A#r#Ij;qYa(j0@ z$ArG4?cK*eOsVNj&Kz%^@K`~{$spsyT>HF1I@!C2sPkG|W*bJRn+$Y+ zOyulkTN76t^Ht0IbhB&2=BqP2>3z7`oLeyKOq49+DJ#zk;}?o_a_VCo+YYi~&qq03 zp5S6o7ga5daw%nFoid_q2D$k;o8ZCT%z&-tlf2udyKk2)l>DX%bjFojg+pwIf<&*A`9>4VW4%;2PR4yta7A;R>CG)O-ZX`g)!QPf!{hdv z4n8$;k6Ly!GiJEN)~?;spsHfI?dYCX?|&8#V0biH*?=;5(D zuHSr-)OYQUo@?u8_8mLr+Jkp7wwfK+Ti?w?UgIZrs>-a*_*RxZYWeEtf|i6nWARM! z>+G`8T{lOBqxi2qTi-<;4I3mBZwWm*uCft6n!{teyABs79}9-7COl=nt?ACTVqK9_ z!W;`SX(sY!`Y9Ek1MLK53fjbzlS%M zM(Oyl{zW2!uh+HHvl%CI{bM7m0(V?rs-lYKo{@)E-^935KU;?(u_q3VT{^bSW-Fth zz1)Q zO16EtY&!iw$(Hij?-IR&h6M$h26+YTC^t642Td*A67f0+uR#yo9}(kkN$Lv^CRgv9e?chbeFIV&p z+Oc_;b=KkaldcbchvD5Z-uY!_9c~9(wW;fs-#-pk#qC?)P;>Le*sb}4ulNRR> z@ci9DyhrQ1Smz+9+slEP)xN^f)ET2+9#FBs7I*3R(|#H;FF1)Ww2Aproh@b>(@Xn? zZ-dhXcf$?A&f@q*t=20wH}tOzx*Y3h8vl5Z&WIo6gUdoM#k#JxZ9cIvwywjZ&Vw4% zyQPDzTi*rlXuoOSporO9qKvN$vJp93YBp`DPX4R*w^5YUcu9uhPa?nb>CF2_|IMjn<&jGPSe$!iDVf8F_(AbdO!~Eo+|q zVFj-8+YT^_wW-^wl-g|D;DM)C)P*=U53-4@ zi^|LOTZm^0TXX$vrqzvh%pYX4;xN2bnO@m-QFueqQhR>Y*8G5lwP)0_%t1Dz>SDUJ z_+?OsJ}B2t52-ey4+^5}^Zi^}cgD0A>voE)8?6|*#Vd9XS3PpPi$S`27dDzL(?fcg zReHoiLHU!M(3M;ZKQfu86V|E4F zTCXg`D5TY5a}r;(!eYh)hOU@(<0Y4TH-$OoZ|wxfNVj~yg%SHhx?Nk+qxC#|BVDBh z^)YtGak*X}3Dx6!J&9-cdsUB&# z?@+pX<*4yNmA59R1;23dXq;TV+HUvgviLgx*q;p2gDY=NobI&Uz4+z}-GjF>Yy5qJ zPc+Btluk7b-5%hbSszkqKBDV7$zw;0saW^}VjDC2C0~qnUg+-aS1#h6MeM=ppNl{G2Slg@9-l_A(wsC3wIvXZVkJfiFhl4MA`;^3& zhfZ>-tY49KW9#_0^)Y>$Y@1u(aeJax__sE4b;{zs&l_j=_1kg%4PBsZynSp$dxG$e zcqZ1j-9yof*}V%pU!{J(_1j0ZKAm_tXxBd9_p=YL-@duEK4#DD^N(AbCmvXDbZyZ4 zn7-nzP|358bf}GOm8=-(-o5$FySyVZr)!aW^BiI_o0mF7_KC?fk35}M)!}?>YjY<2 z#Z8?Y=TK!4Wt`PlXM0^-O2PS#IZbhnKh}3x+3ftjV6e{iYjG~~>mT8vgv;6<=a<+0 z(vVj?_Da(^NrHjVd`^;7Z?m;o`Kn;QP4@w@+NsT$5K)j8V~W^vtHVh4Qx{>6bbunh;fYjOo-_BCBb?x9M(_GVB zQfuYnhk^RD{H4c9iN0lhCh+=wU!?RI>upDN0n~Y+nrA z0GHKAOW(2|DiURO1KVE2<6Hc*NqO;iPfhr=AOG&D`QmQ2qQA|8J@af{ZJP&A%|9Oj zwVs-PM*r)_p8toZhT50xzw*>jFAY7!;_j+MZu-34-_EB0?5WA&|Kq9opF4lP^7!{I zut+qV3UGh$_zyl1`oZIp6PKzc)XQK>a3#;eaf2-Cv_U4^61+sfN=zg|SKP_BXFDr- z_&p@>R~S1Oij<%=agazHOe7+qCF8+6NK9B~)<)>4MoELhUhAd-@CYZ2J0$iDcKQ8)B9VM(|ABh~rB@Pl`)o7twGy*Q9 z$F3-Xrn7C}tv~~Kwk_jeIGiB@To&!_UmHPV88Y=Z%F%LO-}1c(&5oS|d@j9as~dS4;3ed@?+n83h~=U@LKx zumV&oQA;E;)(%}wWF*#`5PD)j7@&C*-hq9M`+XfDuH!{T>==|~Bo4d-FX3@{&%RT7 zfR8f*TsE(iaE7oZ#;gIH1vl|pS_-TN_w`oz`Zc=*UJ^HnoE0fnV6-L165)EdPMp~} z3NN-5_uA$REBwevt?@-O>okV-vKdXCAtr9T#LI&x5{V-*;c@%T3_#87LqO14Jqa@C;Kj=nmt<{G ziQCkvik)ITegMn%!q5xlQt2?zNW!MF8Zm~!fy$Cv;spN&8$ehAt79qQrL32m#FUNX zG!mkSmP5=SVNzCy?X1SzRyYnMc&Cf866#3>Xgo@(c@=BOOBDw>v{VtK^l-zRv!)Iv zL2OULn+kjKrWh9=R;_eHSv_YriG$j$JzQ!MJR4R>SOb|c@}s~>k8@;&UFL8KiA-d` zwvk%02KYU8#Wuo9g^onZD-JU%#cJyy1zrtfWV6MX9x4}I8HroIn=pizIC!#ZNTUM; z9>U7Py@;7%yGkJ-3OW1@tgG85IcjFq5;v9?8WXjN;$wJo6PzA_Uc(vFVPqKmgVepp z2u&fp!+12F^+yLVazP5LKLPI#UpUKaXr~626EF^h!{dffU&iCwR4!qn>s?ui+KW{? zOC|n%bZ}6Q@XlwjdU&;yGcs5cwFl4o$Ilc$kDG%VObifKe*5^fm%kzL466kp}e5K#yaA>q$mF@(;niHV1Y0~Wi~P4ren+&tA1rGtk=Ej0n9 zRAT+n70M7&rhu9fC_LeyOXOK8yrj_&i33S~#a1cZupAT)5~+j4YdLP|!?($x6&!f! zFrjlFOqwwi6AyhLF*Q@0DO{bwkE%z>FbS(*8Jm|&=l{)nM0f3fc#keA%<|gX)ciWn z=Gx2uc#rV=1pnVX%m4qg_vq?sh}{479#N6|m)@g8S@UfcXU*4oE&m_9N0z4YzxwO{b+(qFBnZ$MP|kq!I+g+K{cdJNKdB!b_j5a2f| z@V{+ddaT^=!*bYmcE8y*`3J( zxZnl!;R;K>P)?{&py7195xq|xn4mQv3cWL6?$!b2GO!Q zFvLLx=ef`VZ)8~cMr>JV_hTDiP%HJyFVe*jve~SKQp4!!z3mi$6nuf87Ax6p>6jdIq^-wYIAm-ksi>oTwiXUg0bx{@&j84W^!xi>`|H$3o;P)@eQ zBI>ZXMaYXM?P71T@m$Adb0Nni#Lg3Pa4B-kRdD$<}pyao&a^jkmJ;Y z=L^qpaS>^#cQewa$E)mK&vS%A|#lXoByns)N*e;Ya`@k|N5-dB9 z2g}en^r>~sZlQ`?8;j_)hyy~N_(mejpJt7lc~~eTF0&B59D58|6%!C$`u?Pl=Uxa- zq4)bDPYXG+EDGpH;pahdI0Dff5tk8N3qD-G@+)XuG^*`efdQTw*#wUMlzvmhZ4hNs zqMGQ3BA$sstTO5uczxdt5YP$z%|SJp_?jKL7L6`!qYPamRFLOs$RICK2S$VQjnZH~6_i9wNo*s$F=E@9 zA?V3x7TC)=z>1;*gfiwW=&weEpiOMVSk#lWw+L15h|s;w){up??WGa6$o*i^52d}@ zq4Xw8%+m){$hAjH1}TB+WDUY|%Knh8y@--x8>B~d^u$o)$dN8p0F>DQOmjxzMVPYf z2#sJ;(;K359<3;d$I~5hAL7D2a2kM-`_&cnI=w(|3)-0=u`nJiRB@;GCZj1fH6Sdh zT!psf`)&u~$RLm#`T?u3I*y8+Iw2Z;;|=6F%S3%gp&^DS!pOm=f0`2pq(o;ZIy@-| zg}J-oC(-Rbh0?pMD>q@$a-bfRUj!=RoBlmk#(n4qVdTaauLOK%b_g)P@ex?()9uvkX#j9~zb_K(HH!ytUj*J!$fSNfN6*>V1F>8db z+&lLI@Dth_O8J^AIoie%DRfT77TjEa-?D% zG6okf0I!-N)GL8PmZtSi|y8cp-W zLMCG;VT6uhgmhD4Ab;z2C!ihv$urTsr5Mj2QoyMY*|(GDVcpfGfRjf`8kn|AMd`#0 zqzjX?(YwVoDCuPvrVBASX^@p{o*d6L~H|q>@tAiTTHJ?pnkzN5fg$E z_o`@@4G$w)o9v3{J_r&iMB~YbRY3YK?1y1~oaqaW9oj)fk{>YnB(Xg}8C|Kmba|R% zd6Ii-fIgU0&Y`7QsG*vGSUiRbds=l9ZH>pIkQpb@^Cid_atbnIeHz3qhu#R)M2r?t zWiKuW7yrt>;&k1OP-9et8gOWY58&N!tQeA2jUtU>ccYtKBAS3!yS@ZEvk!_Ffp32= z@NMgjs)7c9@@DueFt?3B?r98YLg-tp)8!FRlVo-z@+XIXL7zt=Uu+bq^NQWcJOWv3 z1PAPKN+66df;RpI$mo$$ApJvu{19n@=tWE=mlz>ObQ~p}8DT>SX!P6bX^1ZW1hx;) z>HWqoAX#(@jIk)0&d@!J5l*1By&9pb#v$JHAn_83E_bkX{)EISB(C2>1(N&V2wh3W zJ^)7uAybP$cjC(rMRFo zbId&?NJA27RtG5d%buuSNLlA)iF7=}b70^~}Yu?wHHzw17WTMul(hUo%Lm+VkIh}~y>pj-_8?4+|8sB)@8cN+s_ngE!X0<1LyNRt6rnFB2EfC6;l zW_QG36r&EzCNYzv{#oA>%+El`J7aeCE<%B2bcL4v3Q+nAAQywfojas&EhFRzqDC*o zkiGfH;!bon#ya`!o(aUg7gj!3S7M61XPZ#$SUJ*Lj>@neg52=9hC8vw5@ZFr6y4Y! z<=lI#fF3fy18OM(Et!s16!@T7kRI)TZ5@cj6=1m`Xt@U>=KhG#neQ8b&Z~x_o*{t% z1$Z(?_EG(}cML!bOeb=$0$}$xfMMGKd>~HT5(7_Rjdl*#gNcb4{Qq1oWV;~-eFPmu z6^4$=G%+zpC2y%YFqfJGZdSl!6%cC`l#W-kk8lBZ`14?NYV%j{-XR=DNIDHggEsdX584&IQOw_c5~kq?%0bs1nnJsf z%{^$VJ!XN(k}^C;M`dOefan0gjANk2S8Aco+R%cL9_ipQ1eDw!&!vbKVPEAQy5}Hz zkrH?%<)Xh2yOm%P+buf>D3G4VRKA2zPw7(+ve3RD3wDRON$gQVeAi3y=3W!`Xg9T1 zOe0W)X$JAMi@#wLi;N%n7KX{0){WTsdLC}`4p+T&3zIqSMEOy@!0No98a3W?KrO`M zA`p$sFCuSX4x+QWz`*4c#aF=kZCN8IUSj^ad|4BC4_P?)J&p%)%p~L<{~gBzPJdA` zXkO4wv^fPea8Ye)fSAi;DAa8MjIeg_29YCcEFcAr?T|W%l#V4N&=b>8kHm=f5Fo0R zV&tGX-ybwI)lP$v?(T3xxR`N6Qsaosu1?@7!&eT-BPzA~(0p;+XT1;Dif$y+X8DY} zjrz@SP$BIS!N?wUKo4^R``pE0z!^hHgjBTZ!13G`R1lPNAA9?XF|RRR{$7s1v+-nA z00_!a@FIvr!y(W$SRLH?G7ks}TBg z-}WOrlO#R`vnDX|c#ZZKP%Y48v{tARZ&={+2|uWLJq4Iw5=LVNB-2o=Id;cqRxXYl z;P#c}xXj^nH;st~kJ+gB7w|EQLB%yiD`5S~K}-=< z_BjZ!6N%m-C|-R;G>4Hxo3i@|B2?@3J&Fi~keoV(2<@$LEh0Gn5jiXfxLsM!UqN&4f5xzcxsJ1Dh%1CvyIfO$00+7sCHDcD zBn)I%!U67Kct}YsKwo-5rR`vU1V!&Sq}CzzWfD^FsIl}btSinZCrp@42CP8ZVI|TU zqez^H0zr6i83^^VQ4G_}srE);kpDm|{Ox$mk1I~t%pj2M`-0)>wS6M2zD_&9K+iB9 z+Tl0km>YqCtPOBK9Z!O2NG8%%VsShL(;z&73csE~UdQ8$G1B@-bIVI}wiF&1Ge zLOKF=HRdrgdwfD>JQA0Y81Na*p%S+fHy_0B4|Gw(jI}NzW2kiy4XT?6k3b+5V6&$7 zHC{$@cuCEXAB8A850P|TjKXI`)^K3rEdmYg&wXHs5yqrD-Ot#b0KPu}jD}%LzG3BY z6XfHBHN@TyX+L{_fogz6mV(xb)T8cwBP1xkN=iRvZ4B1LVk zPOrc;ACVN8gk1IgWkMga{sGD#>$wv0)$aZ(2$L>FG~6oyV)$*`CU8OFkW^{`lVuSi zUToXA>5pVJF}((+eNf}P`&4Pg-`fS!&G$|MI^M^n3#YjK3%H5zfZMYBZ-u;S#ywqT zA|&_RuROAc;>&HA@s8jEE0A1NphXcO3M{rOMd`?2esl%$ADlxMSDputCgnDCxwkd6 zVuy9zST>eZ-J>hu@sUtKod=@=Er0;) zTWcgk2uas&5VzBa2<$N2Ru0H_c0@kz*0fgrC1m+HLG303@g50PZ^)hr1VX!rt^pHJ0ZODWYa;*Cff?L<1I z&34EFs^p}ZpCAj4AR>!CfCKcnc#2SmC>ChX1ua+`#6S-jBf*+3iC_^LQG-Ym)qmm8 zf7)xmX*^O}jFI|?yc83pezHJ{ZV)rt8S*GimBYY}=+s@9g(R3ZZk7!|7#%BfIu_8& z+TD62e&UX#%>jl^yU7YLd-gM$J)uNs4J9@TvY|Z%9RWSv(+o#r+>DS=AUGooKp2RS zim)1i9+%KtdQ_sSjL64`!n8z_|4Swvc!ROfFy6?3O`x5L_T#{wS}>ass1?J70nrYL z?g(#y&AGv*<2d zB$UJGYA;uBOvNxPNdUL zn*bM}JjmB)0OSq8U^>Z(px*)dZNJ05xS}iZ3L_1eW3qNQ2HESj31np=U6W5&${^NT zBj_>2eL4!xMIrqKs1Y3}8|kG8?PF0Y`bg4{Pg4uBkUEQvm2+IsSr|q%e`aVkRX{pw z3<+03!PP{7oO$-9Ky z9X+j3c~96JWDV9CDeVvB*XNy~Uyhr{fp${@uK7sKH1v#GMY@UTa48~%7<7`Bgdz{- z0UVqUFmC~jXaylJUVT+L$z6xObYJl-u0M1kbL~R8<)dyAEKsE$N3uY%$ zBle&)w6azj_U^+hL&0mUztp3cT+d%nH0Kt8?rnhkPXIihqU?`Y%SR0tSgW)PVy6;3 z$2uc(oCct$8^Eez04oCkB0~V$V+SK|u*OIwCWs732a^gk;L~~(?YIr7`E?i$w}_=M z%CRuX0a%}8!9ir~KLlFZ)d|!(q`Kf1tV41HGDi>S$p=*QeTS z902)7!8+Qlo>KsBQ3*Dj=_s7Z?OaH$sS``zpoYdw1CGt+ z<}K?1)WW4d0JhEi326^qO}=$|w=5<_c2^fwE7fEqj$L(8n$D@5!Cf`jM%q zK#g(ZI;Ge_EVtvf8LAdzNYWMpTaa3bHIz;VwlNz`TUZ662g#4YZT_qm;I<~EtpUzg z^HRD1RhLX#(PM4~;G>r~&2d3V+W;N@xC~*3y~5=P>5*`WF8FPy%J?{V%BaRwnE>TN zuP@4R1}7JA1S2AUZ7z)(ue&82PC44=V@K$$#V_pEG27rPp|23exh#L}e zVIa^q2LqT5LEZ-B(Zx4$1oCE6Gl0tn8ha<{yAModC(wzyZXVV?q)RTM>g`^bfWgNR zO(~8G%JGlH&wn!k$tc%ah|aGED;=!N7NBw$2nulmH-fRG~*ia__T6ogd>wEo{w zBILSALvQPdWfLkq-wdudsM;7x)rRse_+xyD%RSUjw^!9OP%*7Qn}ES|gq^^KJx3=j z-lD&LMp4$~_-i^S+!^~j38%sYA6mBJaTlb(P;wE+=Rio9i=b(s(>egtN~D+t?s0q)HmbVEbFox5QM-J$&ODF?wnOy# z+Z05V@4>Ve+D0XxTH;n8Fd{KXV7j#Dpie+f`5nisKSf-74NRwA{t-wgP*Ee2%H$y+ z9X_TBapy`q1|UT0crmQ6z+&vd__0HbW(-CYi)!K(7;Y$o+3*k0F4rn zTpi7ud6uwVU?t&i>cand@Eazn)dTHV}MAfhY$ujI&VvmDn zMn-3_tH21sk!V&olsS+M!Xy2EKtsJ9XHYep4%rb;WGvgjk@@4?_Mow*i>lxN2-g|= z(b~2T23AZp6E%gYG~jqn&Q@a~JOUZ%aS&Nlda!c1)R9BrmMtEK;s>8ap`OX%a845g z*k0oJB3+&>B3~*|=qOAU?o2m&LA`7bJw;{J(X-#jH@MIwgWrK-8Wr7I)Kq};xK@oL z@z-@e_foGEhsnz(APkiPytDvVU`3hgRA5d|3y=;r1Q1Yd^BFXpT4x7xy3?Et7?4&eQ1*J%bPPu`lMlq*PC#VFZp4J?LwHG9K0ql% zjC=(LZg$3I+J!ts?;qKLQdf?{tRWs)_~KpqB4c=*?n{vQi?|KiD4v6XJ7SC*RWYv? zn80T|g>GsO(Fe!A$d8%{zFaa;=Yi_A7^tZb7&3P`SlA>;F<-Z4Bl=!yj`ld1f)GX{ zcO%sXviAwwIO(tyIIMckVPG%}+Ud!$sPS_$-X6gCu#&Pn;p(LKN^Mi%trN#5t4(3u z$kT5Fbs+T>P$Qz1W~_{;u$bu%O*$t))W@!Of~Zq5Uzg^IP9De}i%vc)r4?=j2q{CO zCf$kw(S1sk^$=!2aLTu~Y`nN8#Y~2K&JQWD(ZyR_3e8zwyy;bJJd<1gCyVwAF05JuDiu7Rwstld(A%lwkY9#dU zL+^D!0$V&ajI=wo(l=uS@Q} z0%|VQH*xz7_~pPCzX5*!SM>j)Xsz`EGRX|N=dA%MMgXj$M)#~SAjU>cn6V)KvJ)u2 z3SdtSKp(70vi&f?FH{Lme*jj^R_kED7KW~ef@(s!2Vp4pE*#k%CxI1qIC5Yz=A{a^ zFs{_9;+jrShx6e3n=mi!=wWXHkQIk;F`^xm_b}N>dz_8vv6G0ft8!P$cS7;zRscu$ zQA1pnYlkdoJ0eNI?5aYUrd_CX6s#JpMW{zN(VaKY!f>4`VJtDAa<b7pgWbJB+Hg z;Gn}jJ$43&1#_J%j#=-KrL80uVO+a6davc7$>0z?;}0_f!+ceFPyZMnX+ zIJ0nGM+*TT^x20Bj>5tu`x4PA>%c4L8VcUJ4szX;i+HlJunpDSTQO)_X+0uAc01??5``fo*V_S1gA|ge z@c>n^0CN&RTL)GfmxA^+=82o#^^q{1EO0}V=-{OWSu6+P7pO!s%NAf)R}{H{BHiIf zBXj(L@_7p4$7XM!;j=9f^&b5W1Gh!zD_Y~13WCoVWHOTqK-DsoevZ-x4^VpLLr_cR zeiLdqf2Th{ZPXLlV{i}ER|oIA^T;r;Bd6K}%%$g*q|V6Rhtf9PQ2K{nKrQMGF#i(x zj1-tcHIAnZy@`#5#G#>NMK_e7dUTZ+O1z~eN>?hnQvBeR50Q32dT~bpnF^p^7Z68- zxCiw-r1YhDDqT)Gb|CI)f%FkNh|~ligR#7+Ele+gzpoYd-Y8Y_1M;Zx#|_izB@7{Y zbm+&^q8+h`5C;^c_i1DyC4NMS8)$nx=ewhd(qyEwvXMUZD_&P|VGMn)<@Lm}y`r4j z`9$yubtxch{Sm4WUIN+WGgKLDssHVV@BI~f{j_hWX(7b;zD|2g=m7y(&(Z_vOYJv* zkk3Teh4K+1sMjTk?AJ$|I0Iz+A;rUGt~I-lf$lK^m1Y-1)HM(Zs)j!TVQ7THMMzvi zpsKgL4JywzMn!E*z-c^C{iNU&gZ$k{Y%qm3rh57X5^{P(y+|i{4x~)GK>FgqsqM{a zif!K)j%sP-UV?~n!(|M4WEA)Tm)x`Wamc_KR3Pzl-{ zWGmXGX9M;yL8&lm@8iF9MX#-?u&qV55+Xrda8%F^6E)clyQA06ARdLFJ(;S(;Q+yZ0$abph@~C+HrU7_O`z&(@LntnKBqSpTp-=mU07I3K3~Ny<6R(h9 zx5HFk93EipLyy+lvs%b_RB!-!>R9k~n}A}aNR5Xw(k}mjQ=z27tb!r18?gw8hOYEb zCxB9{NUpLx1lrgWYk~aq24Dy@De3hNfQQw`&9oYgU1SE8?;&)qM)d%=NgFT48q(fi zGB~c&mmlOeZ#2M;YEZ#Iz8!GJd@PJjV+k7i1SOuUkkK1MqdlKmcSY)~Bg*fg1J%_Tc&a>>(mAB- zstT7J#E=d{MI@Mc=_Cpl5RR_H2C5G_K+krPyb7ahcd>cK2KdmeE*f+$V<$tb-2d5@%qt1?ynE04_+*ZTKbZ zjj#(iO*sxS;v-MJJKVcO);t5>Eof=SR`jGCk(JL;!TA@c)9zQ~K?!i;jsu|Ku269b zYMTb`ezg98ZP&nkX9hJyA5HV<2v>T$nZS>`^$7U<-YLPLYy{td5j)0YyHen`&NYJ`9YIlndz2Jeu`GDX@jf^ikvv+PyX)6j3LQ>){%Pg4Ce&H^wR(fy6{4vP~ef%cUsU))Xb_68&=n z(AvQ3WLTn99ax)#3sy5RYKO=pkkLwfEYa8oL?-<&#_l_=$M13Aex2UwjYLK&+Iygk zC@L+Xp(08`kwQjBG8<%zP%5i3G7^zdiKMK^9wC&yw~y_9UH!g)+<)AE+@Htid7W#X z>s;qL*E#1J?{I0zY-pexBYkPglren(*+i)!puOdL@$X6|`m(h@7#-U}PIB9lQh7Ti zsWj}R8!km7&sXkAMjxS3YOF1-ctzFN7Fp6T$tRz3p!Q0mKsCNAI?isJUuOToY#2Jx zpJg`m)Z|UIqj@!0&&;4fTZmj~Ff%XP7czRdKil4CJIet?ypn^E8oFtrWb&&2j?y`2 zvax&U&r288^j-aoT>NASyEz2)gKSyA%8QCfN#^H?^oi}dNFUzDj@`jWT5k$%lSt*mO8ky3!XH9_(Q~X`2$!i;q1Mn5tosjr0+TV|6HH#*jL$ zvEj~O8Tl-ma+g17CU#BaW0!W!BJ-sBSAN!N1vhYaSmBn`$2w%kQO>bgMGVcQs%|1x zojku|3bzzeviC^m=K*y~<7RGf`>4KWtXo=6Sa_>fY%4=ctOHkij|D7CV6_(%*|Hhex{%|sMZm|)VCYdmu;j~+qkesDko94#fMo{%*q?Wtf$e; zQk}%CZp`Y+%~s>fh9BoxcATn3W=)}yzucK>@~E-uXqCqr6m|J2gDf)avs6-ZEm+01 zV-c6>5ZS~Ii<~|-Wm6#GWUO+cuS=VtZyM>{Yt5@ONqLz&t7^NFzcw)oN7{W z%IZn6tfSNycy@?r4QE%=V0WLxsIwvUsH&S>(IhNe-ZX3$)igy_mdYD>nWuCZX$7nm zuw^*~tkuE4q#*4Cb8SzuHwwDUy_^+YPcb#0DPxXM)nk}~sbZWwrCgcFa>(q%kop%+ zrzkmF23zrf>ua;^Xk_j>?ZdO>wKQk-$zhohhkFS+s$v6=Z^xUyc3V!W^o{ zCA0yuNkt5LaxUZn#q*S>HhyI)RVB$KtkJHNZN{*T)SY;JoH@PAnB&7#3+aUEkXeln zFQju_eU;_X{feyU27@ykp{(*2D>U|=ZgY~PdX{I|trV;{LJ!$|<#Ji%0M-f^$Xcsd z=;%eHYlkq!^(r%+$&_&{Q(TYo<>wbD`zdT;eluEmc|UVn9;xUwe&{E+WZzA=;gYZF zo=j`2MIn&Q{myjL!#v8Br`InfcOTQ%$gdBOK84crBHGJDrt3|5zB2Mhy5fvEeKG?& zHBxs?zBFmMa{tCV-J8;_dQYi2Em_K-Dup%Pu+i@|%wL?!tcF|$suXhRT9tpONa6K! ziwn#fUd_Pa5qplPUr!@!pfBCY|NCA-f}Z_i*@oXNo4DZy=UYx`ZG0d^4JG@%mC94I z>Kz~PeY>U7DXf(|V>a)P<)h?=3}05+6Txn30r|697mn!UXW}aR!a(&au^g8Ui6@~l z^?$7Lz`&LJ^FcBNNZroi#>^5@&*;(+cMhHtD_`5- zLowaGKq{se(~lFytc|R2w$*zs`BqMvofTim)Ml_UCL!~VXKFfAj@vk2 zo7A-U4ykKrR+~t3A{%PMU;xc%W2ZQ0IIC|garOdv| z4WLYMCCj9`zU?ZOX;6|kF)=EYk{rB_3uroJ9XNN5P2+f$P)Zv=R_#{Doc3e(i+OXQoLnyDht^*%Ac%gI|Kh6463Xe| zmhvO3T2?fo=*T`yV0vhm`&|Ck(lzm1>hTy^H!$7BmTBqcPCljKIcKem#Ef}G?!_`v z|B=%d(dvbCV=V1(k~4#dbF6$Y@g?oxC2D_jr=4oH(l>2wCz)_+Pm_q~6VjD%;2P`c z+XOP|B^^U^uhCpz`({j_wCBYUk8M@9Tw6OEl}#$ zRWX6z@9|?GS^i#*QM%oIE}a`MFY38z{T8v{9<8LzIf|u&St`}>O43g}Go{*c5m#no zVJ^()P;`1GbHRDET}48rV&c}0nVJ=q95;==>c=eAZi)wFZqj-3$!D-{AIVZ((TYQz zM|CVSU&~-%O7j!&Gil#2RGa(gHGqjSA!5Y6QUQM6A@mWMa zGtP2H%7?;G*5P4`?1L34m02_A9#b09GZk~G{%T21ORwByE&8(U4@MNE#<20Bgb8)8 zytiUMnzqx#C0{I)n)tbWq+d+QP7mhTS`oYS^eg~HZ0lFa#j}DD^D1H+#E1zUYS}N& zq)hERgI(3^PHL*wUSfWsbmqP&4&hebTE@~hZ5jNPs&sV?mNYXAq=$88!dUk7mTlst zHzxU8|A78?sn_$(o6QGo=E)0Pa)9*BIw1* zi*EALQPp~zOAo&HUwnqMqk3HCZpRN1{nAge+`Zr~yP8OZ_4`S`^iah02ubjLR62nuy_KVB>`>b=;brBHurqW{Wdt87!A_LiG~c z>a(z44bi2L39@AAVzlF8lufzELTO9e`z-AIfGN87(pNZC5e?E>Qgz>*>RxvI1%pBK z17!O?F;M!&U^xx2tnD8L8~?H?#Y3(myJr&B_#M=?CPY$8*2>yYpk&oU80?TrZ%qM% zNBbG19$>ISs*9gU>3v61WmnjRyl(G(Wj?fVs3V7@`=-|PNE@S1YxaN=`l=0k`^(-= zZGFgfW+!#f2C`boc6OP(Tw&>{md1+QmU{ z%F@`se4$kbPTb!;#+;aWP0bljd1%2&zbth>di&5_lrNiD#0>RGL?L}#CL1`HMTXl? zQw+E(lN*1nXQf@OxrtZJI3??)m>|9MlAC>zzB-*{K#7ngnzPvhsk#g3Ed4&^E=_{$ zj#PC`k?MKr8L9k-${dQ}brDYz{O0izK{Ycsz@D#;^nX_{;alY9z#F)tn@+uX1D7S8 zlW8wGo@9>x1Qndz`l&0u`@UQvbJ%<7%1!LYeHD%R%!)RBxC^BwYs|R~nMRcC^{N18>d_mmTv^gfO?p4M)9)cmV$>G9vE;|Ahwda+?+wwMvU2W6FZ>sEgRH4k zNz1x((D~uP)O)YSNSDp~u^Sl9-@x13#>c>!BzOlda)fI#Xh@y1I9}=i*{#i-Z4;?o z&t*!+Eu5^cRGZSC%F0hL)s4F%S+LYJ(aOneq^7BWK8*bOfqcH3GAv{?>RlP9=XnYR zO|^f8hNt3pSy!%~#yhJ3?xo}dJMwOABI;M- zLO-!gjSQ3Lf8EIN=ar*uv*9WW-JGi^eSv)${qp7pJNyzNyzsy<^5KDEoWm@ zy_9QD8#4PVo>QH-*>V8+X?bKOagF*dWlh}vLbmF8h^>wmvz6_tD{Qqk=RRB2?t7+IQjI%DDq;~7vBujIsW!IYiM4D~H|9uPnZ?{|%UV(Tk=YU*ls3SqasYYRJ$9{`@}stJz}WK9*A)z!j9JIA$<~vOy5jQG-klApR+5 zO}1UC@N%W=>e)L6&pA6XwGRyXePp2hRmxZLXWsvnNV&iACUT8NsPS3&?&joekvc2d zlT)()v=}tgX0XnT?W97OEj^mC=)|lZT^V$x^f$KIsnQeYCOw(ewl}k+Zi)}DvKqgz z%ct8X8z42E+~3OdrKa-qa7|Fzp@6H2jb zF2~m2je}Km-9=CEALq?KmgmEb&+(=r-7;e@KTyQ(|v> zs#&1o8u{}LnylfNWy>05zkF^&1j$p3;%iaartz$I(QV97#a@jKwu(Dh=EA4n9pF%K9cr;_0KRu~s{_i1W5l6RceI#*(b>c%5d1%i>@;rt- zYHM_yv*ISzF7*6lO>9}yMz5wRy$!it3Ln3!-v6K&%8P+)illtGRWlB4NtYnv=xP_s za9MN~8PjGq+b<>MG)*$!bELfXEPoh8>V&B-FTFBSadhJ?xxIAtdz|Nhn@Tmh4f{A& zBi$DKzDeXuhrjsW*)R^_3D~&m)}s`IyoGz^q~Xa+J(o z_>v=`@mAed*)fbe-Z@o7b+f>mbe?aa-#yN=~^oB!x4$-mr9U^}(zQY~V8gLJld$RJPZ7MeF~p_jlzTGjLVTOxSzvC4qC zn@3+I`(}nwF2)&;9Y+xuFV)uE0@C}fm_~6sYbd3^i-FXnYDquSmvz6!!o)o39w=`Y z*&yAmwJRWJ=1LuvK8AO>ayvS(K>L><@8$iDb8u1A7toO0*ObUej`>wi@dU*v+3#E;+@+#8k@o+qoN-Ha{b;J| z5ku(Q%k%?@)?$wI#k8W-v3lb6iamT#e9Mk={_xmS)#h)WCmCl6H|pLb~Y8q>GMQ1$oq|sg#|O=ZK({ zm`?;J`5utpOQru|)4v*SQx$_``Se>k`%hUqtMQWvc{7sjl9m%gS?SDOqI5qmTPHnG z$lgihW4QX|ZO@m`EGcMrnZ{Mrx&yQGq(dt^YQqWj9LKXKdBWAF>_oex+H{_3?tQS2 zSwAhM&Sf8KR&RV>oH~eY3@VP$0FP)kaRz@p!|;&DOwLzQ5etHoq`$+Wwu(^)i-9Ds z)qtyP9x7F6?fpq(QJ6|B$9|3j9lfA6>w3kr?v&Xb8U1e#qb*mMOGlM7gnj>R;F46e z{Mu)hVt@J5$sxqlh7utCWL)-dK1E2L6wcLe)%FxC&sj%RAq!Yf9Q#s#Q=E3#7{@94 z$&_i%6tuXCIpea3ZrmfzX2g_cO?s5i=S0Rl#G1;Lt9oGepxHc?Ez#zts_|7(Q*&05 z9we$rFCQ9zgjK+WAb*78lJOn)6a>QAxy+ctg<&y7uKX&Zm3 zxStqKX|UtRj682oR6RImhA2j2FqxO8XrC_8Oxe!QOqi(*^v3f zuqWqU%at=4UwcWN=&kE^vfqasvh3zY2D)3>v_V=h=m357r&h(}$f*iulHbzn=Eir~ z#;19pV#mZgc5YEgy81A2f(f%KCJr#taOQsp-jw@R=}*8kegwg-iOlF68;@j0J`>!_ zYBs(Ef67FcOH6!bdxwx80E<7MU4Z?bi4A`fw@q1+euyg}Ky7JQFox*;TT5=W32 zMb1x@GTm(gYwEjkwDh0ni%C57s&<%E zbcV!{AyMS)TyQI+aD>dsT)?ePI9IK#!x9Fn)W))dnG8me2aUf2tIsAOVR_8@Ob11_ zp@{iX@ltroe7hIS3YIP%{kIJKKQLJKiT#J`l<JQ}kN_JC~daY4%IH1($ijva@rM|ojOp7XVsw+pgeK#&n@lWkGU-v1DGDkv`YO}= zs>~d%GAlu4cDBmga+UdyRN@uniTd(|edWoc)SMK zbkhw9O*6BbZY*!Q`BBrY3e9r#n{DgcY{%$kc?r#SXE)nh-fZ8aX8RSI7wR`Z)VFz& zU-KhVn;%=#yd=AMX?gRqN6k+vsPZ?-Pxn==8m)RZLG^sLYIV8lrAMll6Q?U8vsis=Dq+b$#{LhPJJZ{ac&Hv^Gy|ZIRR3>Qw7a^{u-owy`y6)6KR` zcmFnaF>QLKws9zI)8}d%r;ly;JG!p6ZQcFb_6uw4nb>wfPFt^2ZGGz74pMCAZ_sYA zOS{0ZcEb|e1?RL2In^$-zTK#g?MAC>gc)dr+iHyU*BBS05tXVjp-?0Gs>bAx8dKHV zPqS@5!@qrOO#4}h?Puq-pL?qP{QC9_KDJL#*G#h2T;#90I7V}6s^;=S&6QU*Q$A|0 zQP)~)tF_)=D?LUlBULM_P;1jwtt}t5vemV7ZMC=iYwrlt&P&wZouj>{P&@yWcK%iE zf{)q<)H@U!bU0|+p~%0(k(dt0QacpqbSOR5p{%~cNyU!k>K!X=J68I4JQLILTx!P) zg&i+m?O5}%;}vzCYqmN!{B>@{=-f%vsm;;3cS`5}Rh_zeo%n}}y7dOSk6m=1hUq?! z(S4Ds`>Ig)%~joZA9X*d>wU7-`{J+nHB9fnM7OcT!Ri3209G}dcx4a(9p=$&?MZ@EXmL! z*U+lMuv3F!7hz;;Xw=Qss7JU_&m<%JT%+CBuCZ5z zu}_2XAYtNfXfoKDYvw7R z%-6KGSliWNeSk$;xW$Gfi_Bb$jTIJ~8!Wa8%N#??ZLXF(!Y%WXEO+Nx?ya!g*I>C{ zSQQ#t9dflg9By@Vn$@wjRwW0mj$gAn@yV*ZwRJ^T>&gJ@s&MPGN!I6ctuGw3zIe^L z=9Bf6)}5|(?Q|oc)2;AMx05>6=61SQ(W$PX(?ij@-mvpy*UnGFJ3mkA{4%%m>x#~A z8#=!iT|OFi`Rv-|Yj~IclDhoJ?eeRl%b$iW|AdXAk&Tj@jq(^9m4!As)~);CZg$hV^;*}>;ZV2Ur@J{m?&hr2-NmT8n_G8}G2K0>6O&2*P_9_ z7EkZBbX~9IhkC8N-YeyEuQhG#*LJgCKiEEfx_!nv`>aFu8xPrUx^BPavwe0Ohum%s z+Xp-BobIq|ox`3(4*Ay|3O+mRSL$78)ccTI@55tyA6?kHcw6t%)4j_c_dcoA=af;O z_|tBEs>bv=yRgssZGEaw_qp`A&t)aYt45C3-5hU@alF0Iv38r|z0;0$*Bu{xc6`*v zsiB+Glfh2UraQe@=k)53)0^u~?;bn7S91Q)#`#k>=P!euzfE`kzRvmQA?M%Mo&SDz zR%qK-bnh!0(znTszVh{bn-%qKUfH+BlfErwF6zcEZQNbjjdf|i$VF?rONUAqohL4O zGFJm*S0i^PqiRPrNV7e6AY%TzB`m zIo9X)BA?prKKCkp>Yn&Kl=;>h`#yH}eLB|n`6Az!+kIbG`o4YQ`(8Haqw%26?t{J# z8T4(&pzrGk{VW>vyK>NDBm7j7{hIFZYhLBo;;COtWq)-O|27`} z?IQf!C;Myd@b6INuk+MjPdUKAB*4fcz$7BTEIGhpM}SpTK&PhxU6cpgnhfsdF}O#> z;GW5Y?RO0BT{YP8>0oE&Auc9E+&qSOL=5pv9x`CZ5U;8sK2L`XQV#TM7Z_j?IM^d_ zNMK-KMBuRGz~CK$Ayt8)PXk9O4-GRJI>uvYM8wd@Cu0 zpw-I3sqKQ-_6S}d7@R&cI3q1M>u_-VrklZAz6NKv8=l)^`1ZizJ7*5xl{S3O;o4S8FJ!lNO`*v6+K2&297v0bHv%?5$AV| zsID4u>FJ2e%Ar?HLa+A-y%89CYi8)3w9vbUL+{@VeegB(QM-{1Jw`qW9QkbK$QNlN zUmYI#=H|$EUq^muH|kT5QC|W_eVaMzd)laYtt{R-MFy!v0<9&VcJK+I-UvBc^0PEB;3F>+^Ao;$+&Q{ z#o-n^!>!JQcX}4yrO6mujWJ#A#&jP##x8bDuk+5c^}aRM_uE)Mjfeodh#^BGhQ>w&rAG`u5;5Xd#K>ZjImaZG5)Igj~A`+lNlr z89QND`h-14Cgk6mQ1ETS0gZ_V?Isotop>a6;<5CJB}XP6zcum1w~6H%(G_;ll|!S? z#73V>kG^muy82A?rDxHXn@qZDI_Y}9NjJw$y1jT(?aoQ}&P=L%HtFHFNslxpH`q;n zGIa8@ag(1fp8Rs>AFpPky``P7ywF|FFiwC)+xc36zYtQgG=G1^CC zI-ZTuxgDeXUyPp0Gy}6~MxN75BBz-xnP!nU&Fbv5PS2-xQJHRQHocqY^d6DZdoG!7 zpEtet+3AkYr#q|6a50*4R*c!K>7KJ@M$VeGWY+Aw zS#!_Mil6^{R=i4FqFLNR&$#5sxFt*CmgU8*I2*U>dE9E1*{No;*LluPi=4e-$?VL$ z*&EN!-u!&_R+Tw9W^=ZA&e;(;CvVA|-Fb8No}IJr`JDYKa|_Mp9`c-fICAdMC3B1O z=9ZqFTlReJNtJo0%;uf;oL3b&@9dI!=kw-OpPhH<`Mk?2^RJrCzwSBzX5{?aOXk<+ z&A)ece%xDaiA#`YBsATX(EMCNix&wk<%#O%iEa8Pwu?$^ zAD5_^k*IwvvE!XY-S3I|nn{Mel8l3rOyiQwGmhvP1i+rK2`ND4f7xsu+ z*mLPZ`&|orpIhkoVxhBqk&EUc*ItWgL_DGvc`jWvVAmqAbBlalEE@EEk)J%-Uo$zN zSMrdc;DEgl!QI4WcDgky`N?<}7DeeqPy zCDVE>nGv)kHf~8=#*#V5mdv}eWWjgtO_nB_FJ0JwX>!!kB}O3$q*d$HoAeB~+4l@+~KRtBv+6SwkQ#>xxFR$jcb zvgZ5BE1IjW^;&f!Xw|K_Rd+I0-95JI{+(41zOQhBq=e;!-?`_AgW-&ZSWtr7NXWWj5i%w8kUT+^(0jcV;0 zwI6F*X{ENdPi-5VsxdoNGc#4YIJIMKs_u_eeXX^I_G^uU*P6~=Yo58*vUsg^?b^;i z*4k*T>uSHQd+<8D+3R{PTW7y}UGMYj9AB<;Zo1yZV!hjd^&aEbd(K|pKXd)S;`QFO z>wSN$AJjC>-y&`BfV9BzX~UMK1@BH9UYs_fHf`jOwD{3l>EZV2V}sMj%}$TXOrKDk z9$lM0`A7Oxtqs%cH_Qm$5IcKAT;_&3#T(|;ZdmYRLwwVWM2n1t12U4wXDnHkv21t7 zit`z(US_OrnwhGVnQD=_Za`+*_{nNJTiOZvCNGn#T$>;Zand0W4YF*3j0l!!JDea zZ#uhd)A`+-s*5*WJiqBut@MA*%T1SmY`UVg`I`Oa8v{1q9KZSYvdy)-H{UJZe2@Rn zZ?1c}`C-#7^%h$i?6*7#-tuhrmKT{@UKMY7Q@iEek1ZdxwtlkM%6Dd8g13H~z4d$M z)}O^&f7fpP`(vwucD8WHmJQEtGACP}mEEi)TlH?X+RyA(+BvNqa@r2h(U_B?nU$km zlGE{Sj_%JKeeGOBhg{>~xu$b+&9ibXOX$4kc6ycDrP(%H%Wd5TZtF2&ThHa&?DuTz zePNs9t8LEBw!2tvcN@6fW5RaN<=Y4B+3s~=yU(lbgPQH|x7;y!;EuovJBBUa5xi$d z$b}uDuXc=TwlmCf=a_*zBPQ&OT)uPso}Cjf?40y!=ago7G1__49P(xi&x@UqH*0y` z>^*sNFXYXCl^5S^SEA*vg#&jbPuR6&`L1PqcCEOuYvtWtDL;3u(cZn*VfXssyVK|F z&dA!GRkC~2-Q8P$?#|ZUlk2c&`|v$G=j_>)wP#Psp8UIe3V!Z6puP8?!``CddymZ7 zdn{{jNy*;hclVz7xwl+9zrrEEa(MokIr-$L4Z|)Yn`&sZod;ceg{a=Rf|2Aj; z_pJRtOZNZ1yZ`Uc{R$lph~5WeAqSewJs{tBpjqhw)q4ljejRAlp|ExD!nPrW8gmOZ zHx_D_7IwT>sQbE5ulYd(tAj>f2Tdj(G+S}dV(&q#>Vuu`9qjz;piPHEU3(wu9&*TT z?xCJ54%zQL6z{P2Q18-1eeNA{`gN#pha%VBMeZR*9utc^R}>A{TjW(;>EV#-!=bMak7|A-%<9M(uOkr?kBpmpBx>W438hD( z?;V-^>&Vm&N2m2ZIwRz0?A)Vq8;{N@Jv#5+(FMPbCUiKK)ce??kYkJI9$UKc*z(e2 zEAJg!_4?TA=EbR2#p}F^(BVME$?u=qWAI2kmFSokDpy}{QTbI z7fO#`ym!3j*YPVI%C7Y;yAe`$Yi`+{jb(RB%kJMRd+@95QHK)^y-z#|Iq_`ni5DAB zyxe=@b@hq2uTQ*he)6N$$)BN)07t33`DQ~HIO5OTY8}C!?qEEG7c}gq)RELYFbl#lOQ>`$tt}ybhFo~`( zTUlX|Utx8zqSKp-E~=+(txtFJKHVewbkCKi?ekCfzIfX4&1q-VN*C)&H}6W1=t|F( zl>_oCy)IVzyr~?dTIFwDHQ2iL{2sh$b5J~PJqOhojV$dzZt z=bxE)@l5pnGn0RxnWB0&#`^4Z@3S+b&(2zTc6R>RxfjpQe{(io^<1L$xrN^6lB3Tp zS$S?*{<#$w&#iiMZnf(9RO|EWyw9gapWm?Zd}jXnjTg^vesg}S>V@o%7jpYt*goRI z&UqJhZMv}M_=Wt77xM34DENKhK*#EXeX0+6S09e9KDx5HIKR5|V)gO+)hB*emv_8a z(dT02h>K_DT|Bqx;)UZEFW$dc^ZVkJj+d_Wxpdw8(#_~gw^v@O&A)W-;-$JbmmaFt z)LYj)_O5vvUGscp&CC3n*B5KvzNvYydikUE<59s#D^2%ZX@2QSi?>%=wz#U^>1rFFtL-LTZNKWO*1oIp9WGtfd3#l_ z#Wj7MYle>3j6<)P&c9~9`I=?fHS4--o&Q|3(YfB$@p|{r>vr?6_u72jq3n7e?rPs& zcW!aRrPB>JpBo;NZg{S`F<{>fuS++4-rgA0;--J6n}dCB22Q#;Y}L)+eK$ic-5gPO zbL5|!qjheDJKh=_dTZSLTTz>DO(?q+U3Y8ppIcMu_jS5G-RJhqNw;Usza6*v_MEcY z^XhId_;Wi!=T4I2okgK{7SF%4bn~6%Wp`HA-AVa#XN^wnTF2V;p|$DrYcn?2X6>up zc&T>t+uE%y?&fs5yUpkBj!Ad(R^8pb@9y49clW)$yT8S~!cO-N`P@4kdhf{md&f54 zD=E8oyzbtKKljRY?pHY8KkakBYSR6)^Y5SAeE&k({fl+?YyRB7qEmOxvF=7_-L3g` zcQ)7EEvvgxzmHn!*0*-5Z#%NS z-Q@cADfL7wMsWljMZZPs~Fqzz7meOER&|p>5(CJ-67q!Q>oga7eecWU6 zCgpFgSI>!e&Xqf2Tw=-eL7nAS-8`)u_K?2TktGu%d-h5o<%=+Hu>+fsk+anIX#~- z@_Fon=W$z}&pGjY-h<~0{ytC8eUaq!VxjMgyv74PIZ2B+V@S>KMLNf|EhWa=iU2%Y9AE4d{7$nL3zptmDL}b z?*GvI@`o1hKdAlt(5mG}b-jG*Z%(Ot0KZR5X2;8_z#s`TzT+!~f^M+H&WgV!MWair&&c{KG$irtl{I z%WG%tK&7Y$D|gISE9 zCA@%YT*M{Rpzk3a;et|>;Uq4g8V~RgukadQ@D&}3$Qz79Bqm@YqLF|^TtW?+9o7-e z;fJ0_bVSpm?B}?SSaU*0G+fXTQ*UtGcR0Sgq#aFh@fLrg3e6aAj?jBLVkDw49!(z- zFPbAA{*TCC3`Pv5;T29d=!gR^bVT-h9g%|rpLB%VHyz;)4`|%sH^qegF#sbm3ZtRT zvNFOGD90&O;4~^xg)_L0o3Qv#M_9ocozNLMcqVkkb5u6d6|xq(LK#h<*-}?%p}D56 z2)5Q018sDL7hHPjiZE|o(PNyhFo;H?uIQVtE6!}z6;*kxzWm%=@D&OoX1uRMHb`O!UNhq#^97rb4E&wM=*`$AK}mv9jL z;Ew=o!ah8M!U8>EzEn>nEGJGRVIi)ec!i!Q!9CnZ9h6q;3BE@cJ5vbP=m}-qPuCM_ zIeOwcZXhsMPYlK0qk1Cl4af1DX!0s2B0$%FO9V%!hZ7al?S zVpo{H*n_>u#{s;Wp)Wo_=d-55-v#<&WTL(pg>XdU$wI=#?1$w(>H5MK{s_Qe3_&2u z^7O@Vl;br1e$iA|vrAuWMK*Gf3*`cRp@Jrm zBxM6p_Cr(Q1WsbKs(}bYIpYb0iAX{~3j;A2^DrNEcz}*-20{m0uods|0d*}|4j**` z;fp=ki`X^>A`WwO48&Y)`lYF`87=*ZgHWBY6}B?ohOjkwc_`XK1MQ&+EodW~^*RuC zgbs3;)+Nj(+y*_y^|76C142XWVBCn%1jaCh8Fn(?oX`T6u!1%6n7<3181IbTjPF4g z#`hB15ZWT2@vem3u#fTXgaw2>2=^1(5%xqc*uwz_nBSYQ4;0L6@#U@+sQghL3A6AmRD zhF}C?ILeqGLO24U7>N_i8$~!8VF-Ares-j|rHFXiUOnOuZ&m=XAmu zn2A`-LL6pe4(4JW=3@ckk%%O$z)GybU;IO}00Yq+sz^pE)?x>CA`jJ=9%vvQp&lQQ z6vQ^8S^u}D!dk3D3#Qf360M+))@XyaXa^0nhbFY3jSlDt6PUsb=CD9I@os=6;~9ij zgq_d@Hn2rkWU?#^8?gzSVGw2@ZsHbhV;}S05x&O;5BL7H%Mx zV+Ee$1zzG6UgHf^#u^B|3>VFigoRL#;MkEkj(J#!w|Iy5_<)c21j|USE3C#Ew2EST zEXN8ApI{)4O*9Z^XL2mKjkkD*Yq196I`%H-JnbT1%g9r-IYSw_LK&+y5LJ&2#2ExC zX(?3y<2*uL7z#5usu&7qh^B_Z7dLPd@0%G4uMUPn!N^cVSs04C-iD$cuU!pATMt7q zMMX=&h%kn58m40=VqwkvS%h(zjk%ZyZ{~Z!2fncPG!zc#14p=_Kc*sWf}zMxGZfd= zv=pMx6CdI*8?7%IiuUk=H*TT2#!y^1uI|z3o4~2e4LNmrlXrpO6ErmX@Mxrjyh`y1Ln53npFkaDE zOu$4$qm7cWXbTOrM~kM$!m*99a6*i+vAEwsOW}%%vABwBxQ+(q?X)x&yO4+7Xx>pv zp<5SYp-QNR7HEm?%7`c*6(2 z@WT*Y_ivy8b*sj_=d1I;akE!gpP26Gy1{> zu5g0~+~J9S7=Zp52rqcU2fi2tKlmd6gE0hw7>Z#C!aMf+9wCf>APglOiI0qrA{>oR zjE50^Cj5dijE_YG#vu|>7>}W#A;aQg(q3XL%F2wz37o{<6UJg63a}sM6~^LcCD;0S z(uf5$lv}9ZFcyDraeTF`14~$;GdkWg7Qt`Xuc4O0g?Gl{B7T1~7Jo3HrHRnE#aU8J^<>Ug8yA;|<>89k#LVcDQda z5jzO;unW5}jd{}%hkYnOA)GdmX1MG$5v~{nKa}AFPNE#AP=V8^L>12B49?*^E}$A0 z@r`}{hf9pt;4-e@JM*p*Uc+_Vz)jpjE$-qT?xPOBkX&RUQZ2L;)?zW^>j;++t|v@G zI+ilM3>l1XKqj)V5u31_Wh<}}DOiQoScA1l#X4+e-SvcNNXG_bA_H02gpJsO&De@; z1HPSIGPD3_&I1P_!9Y&Fd0)Y z71J;SGZBk8%*I^g;-r(AD90)2_?U?<=!TQ{i>AJ2Vqz%q;ruu=QPxLGVed3EQHWp3 zX5zp$#&HmbP=vz>bJ9{cLUfQP;5f=~0_8Y~Q>egcRH6#u#4!eE7(a`1IFAdc z#zkC04KCvfuHqW5;|6Zx7H;D%?&B?F+s%X(EMbjK(s&W`(9fAZ9AOvOz!qK64c%df zp73P7{;+4<0lm=&j&Q;NmN^slg)3a(26uR1Aj`erjebn`hbIO|%islX_`nx~;0J#M zU{nMh|ybiFyPUn~6v)Kmt;cgMBDK zA&TH#!g+x&2Eh;hILJN&2nS;b0x=ZB5QJb1M+inB6eBSTqY;L1j70>-AreuTfQg94 zBuv3%OvN-z#|+FwEaET+b1@(BNWcng#}4FS7k1-4KHwuh;WNIXu$1G%Ar#>i{=*Oa z#9z1_r%a?Lov)Qm?8|dZHKX(Fc9u3OBgJ3*ML=Z!Yr6%*8H5 z57bhqJ!3BJV(fWy5rGyD%!U6mbJ5L9OQ9<=7|%o&x-;DacIb&-u*XK0ZNg@3Me1{N zu@>vF9%*>-!d$$_9W@8TKVjkvW0pcNEn+uKiHWgM|q0XdyhUErdTZkc9*2X=A~A8w*j;!$Mr~unMN5&QYboRgmO?3zXCr2o!VW$0$;wiE#Aj%AwiKQ$FYak6 zN>GXfXG;-}L>L8Giv1~;;!>)mXuri$ygX|uj)c&M#o(`&;uv8uhA=)9ff$A$1fzsy zr8tfdriWt$LNOAfF$!S_#~6%71jZo}Q5cU2n20jsI)P}$Ct)(CU@BrT4bw3LGZBkf zh{J5m!F()0JQ9$IWGuuIEXFdd#44;oD%N5hPO|@_gvU^Za-70>TtEfO=YF#k^DrL^ z5RU{b#Br424C}T2&r-BOTRgx+JVHGh@EA?MTk>wtQZ$DuT0jjgp^nyQgSOCuHafx- z4%mU682*dn|6?iUVjkj=fSXwV*HWZo12$tjZsQIf;t^ipC$#@@Zcz$PMJo}A5eUUd zjKXMyVT6qN7=_V@kz0vQ>Q*96%Sxm}$;3*STUd!Uovnm>Z!6)6!5D%-48<@6AsE9E zf)QAU^+-cHHXs9=kb_+8L@g9Nti)8T!CIta1G2CY`}lO`rlf`l0JAE8z$yxWE-|aK}KzW9}R)F%R>x0P#pb zB9f4f4hdGGJM7?uzHosLd@&khkcrxbR^mSD@Bk0-2=!>dS6C!l2`gBm7P>hc`%x>g z7%MO}oI03rB~q{&Yp@onScmmU!v zJ=lwU>_Y+e;{c|!O(Eey96}LhFz+zo5gf%a6k{gyO9)GG9A!9xlPJe2RNyo!QH3)& ziwmg6MO;D+F5?QW;u@~w25v&}807#;kYRZ-ag`D$&f)8EEAb7F%dEr?{KPN(Mjq+> zL--f}pm>671WFK)K^aY;0y&zZIhsKgEue;$Xa#k&MjNz6J7}OiG@%7;bU;VwKo@$@ zhXD*>1Y?-M7lYsje*|DKh9D3_F$_VtjvKg%+qi{0_=qp~hX3##KhW|d>4rMmq62iG z3q9z=0ERGvF-%|zGnm5ymau{~I-xVVzy`MHif-tR9_R@>^nyJc&>MZ=2q!qBFI?aX zH@Krej4Mb7OkoCv)0{sDC_x5gG=VDA&!vZ8H=$5DOinkY(NGwk%f)egw60cPkn-%i(ISFzRY>REHt@Fp4}kN z5dVa0Ny%EAz)6(j6e@5Um8imb)Zl3|Yw-+6^{j=diM9CM&073{ueY_Z^RpH`(Hni> z2p1d-uoj09J=9uE##GF}JS-e$EfygeOYs6P;TvQv2Eh-55gKAGMq)GyaS(^FW|Xx^ z#aDd8f4DuyTHHY`?jdWewTPK!Ev928tY=t@POyRLe2y>CS{Nr;3lo^a3>L726|AAK z!dmoLWi9M8IED#Y3g>gI#RXL3A}*l@mvIG=xz-{I<1qmf5sj}qNGHDI2Y%u=e&G-P z;veGKM*&_H`=LJJ+x z5xroKiagFgobs)OE8H+OT1%mC73UPLaDzKk{vY=41g@!T-xv7aivR%w?qI-xhzSA) z3K-_WfJsC}M2sOSDpf$7hlW8#MF|iThbk%xDk>@nw$_766)h?tD$Y1hQ~~E16~zJZ z{X%U|&$;cn=ML|_d*A2Uep%V`w6gYIYcKvB!}h@ljG+T?zyzM0a3)Wo{IoMU2*)ou zlN$Jdb){ax{zE!wAp<^Q{Mf6`Bo@ZOc$fea;bRNV6?}p=_zc#qNI5QKqKOOH$8#ZN zaC|!6VRdyO0T2kiK?*XEL#4olR6`XUfunE?u6|=l1^8nb=nYbk!Hz%|@&o(@c0vgZ zS6fmW2e^<;Pz1+9Tu2R^gy?}TWIU$FARVXs3=?!PZW7YTFj4mzCc_jAPlahP9ln7n z7(W%Jff~bc5Dy8E2-7kC8<>I5Ghr5}F`R^SHq3#!kPP!+KBRyKQXvh}VF73%0~SIi zEP}{whL*2ckmuQz(;5W+ts)>YjB<+4&osJhOfo8VFZi>&0d^i zNP~3H!bT{7ZBPU!L0joU^p3g^79`i;I>GcaIKSsy$V0e$6W8ay3-Ng9LOj6>d?Bz2 z`4cvDb|v|+8NTl7N*>I#q`LS?Rq9Hn!QD}=Y)J|;SM~6NAMV)z*Be*O~5(sN=(2MoRZOR04W1pFoRCO19Px| z&cFvtu!1gN4PC(ox`6H5d=R;0OND z3#1@}S+EhdLIHdS-@^{r34ejzum^sEaO0@TuHz4xCRgmS1-DfYj7QIz)d)D*_HIXitBI#=NZP-xsoXO={BwvRKrm?2FIZW z&cGG82G`*R+=M##8E(OCxCi$ksos^$fsW0%CScu)b4>RnB5(vJ_zIlC1$u%Tcz`E( zfj9VoFNDG%XoZ(XJ;^JWVvK2=p2QFQ;gCg7QUOV_p5$%6o}>jz=l3K$O;57^+n!_# zoP<+w2F}72xC%F+4jzMHVNb#aW3YhEUyaRe;5EEFc3mv z5QM>CP{0sSLO7^k7z~Fo5Cv0UDolgv@D0R4JS0FOXdo3vp1^s5=O=rT7tjRF@CI67 z%xRn-SOF_x6`X=|a2{gM;GDsDXoJr{oy9c;0(xK!M$iFFfe)7O^b+Q|iFx2LK6&LY)NhK5R;9t3G!hx z`~ZJ}5-5jbFwR>{#=``d1czi|au~Kph{;8`1$V)5q?kB^3%Eil{0QnPVv?a2lLgRk zDXt~-hea4(0>Stk0s|lq!3gn2EfFNi4%MUE)W2VU<}sx zO`@1YBUK}fgLp`QM3@0HAqnQf97u-wkOCS=g$1w>GGQ?+f%ULzmY8J2YWOh;=NH>i z%@dPw7z)GSbBdU>0i_X>K6zpi1jDz8$)jR1$vZ42v9&nAH^n5fMNDS%-N>3%=raMi zu;12=T!uSv7q(&CYoyzeeg~iMxeX8ni0FYSbOZ*>pgROXFvLS5q{4S_9xlT}cnr_r z4ZMTTa0cr)vBUbn0vx~zdP5)R4+CHzjDkpr0u3wyLwh$8071|f`aypf2!mi0tOElF z%nJ-~;T$x=9oW&sjr;&RVKkq61M_8=ic zJ;*?aTIoTeAqK|7we=pPZj%SeE%G2YiaiKZWJz(s3~pk$4t|DPa2x8O0Xkv&9i%*@ z=Fk}|fDe}N2-6 z*gVUVY=Ny%06ud(iF~yu=?f~DxXqLNwB3`G!?K;89ECG*7S2H}T!cEf1@+JXjc^z4!F_lHk74V5tQ(48CmewN za1ahb+ygJded0wrf$*spv4!z40VcvEm<+R@d67BLp~;JI;PzWDGUx-24HVGT$eRej z4qU;lvo~=EPjHcV6EVo44+KG9=m-5F7z$uvfH%1t=uMiX-sBC0%e;vSCdj=>?>^q- zOh0dO78VWkCLR;9ZeS9<2^Ttn8SuayETA*+!3r#4>|smF7HM~|12K4jC!B$6N#5i- zOq=aZro%U&N%tnHuo0S}MC*-yjNarnJihErp29PD4#oA}-+DT4#>6O@Dar5|wzH~7lbpEyH7 zXMa)%u0H-mFTkI$VC_JEw7>Y1m@t1bGTfi+8}3iM#`qHngpcqW-i$Gk6)_mI}Yd2oEti?BHo z^3+E{KEOqP3HcceB@$8)Eg?nlJ-nDDAx*Gjj)eRG{pLzYe>j{XAQlDI%ZPHdKt zjJK9lQ=x>sgx9cmn*@DWCFG}r67-kGxE9p0>Ler*zJ}2-2BIJuVn7Y?;H4ixdh-K_ z*$4C=M%owpL4OE_F_r-&2FAmBQ2@z<4Y1EAfb54dH~>FEIo!Z{4kA4ahoAx~p$d+` zQ8)(2;S`*PGjJAaVX|)knF3Qm?<4y8`Uj9TkOR38&?|sQU1qoq@A(4r<{%T!4#k2`xDSuuF+72%@C=^A3uuCu@Csf-Gqk`vu#^Ol2`~{R!4WtR z7(kqR2M`x;|IE#)?WsngI(T8}`F>=#6Po2!;U=0`s~Bl7UD=;R-$v6$Fy;FaailSNA~T4-)7F z0Wi`dko@cuNN&Mx_=7;k-js-?o5W}7erX^7{%-^;ZX)9Ult{%*`caHQjMyMnVK828 zFfq$ua*4szT7&6L25QDIUTm1CFr2A2Ov*BxQ(~B0YdF8jP{Xj(#B4+rTcBoVX0aES zu$R`dmo>4o7^9V9qiluI8nsbwmeINrqr6(9jZH@RjPVw+ae=~ko7%WI%lNwz;~lle zJDZG4m=3$ezY9}b(xJSz!=a`Q6%40J%&Aszj;T2{S)7w4oYS?OvrU{@#^i$7-cDcJV^b^TJ;86kKzq zo4IO;`LHLNB($;CWt@8z2*H^Y;v+nNh-944vz2m$4uI%o=ySwB<_rO=(rDk?= zce@~^UB7s{;FWeEyX`_R*oD2aQ<&K+-R)IM`(g3+5i9LS?zWG-U_bhmeUzC)jJrdu z(qVkO!^D*ilXp8zz2GqYm4n)>N4$HFL}ibe@ja4O_L#G~NAiUp^Z$tu#B1Umb5}a9 z+wGWl!Exg&$9yxVE$&VQN~dk{PQ@#ozT54zeYn!~@ov|r7hIpea&0o}`O3X# zv$E&g_@3`p_WZEBXX}NYZLfM#oy5dL%nBD9B#7Co#KwEXoQq=9*J7rVo0*3jFWk){ z!HvJl&1#RE^+h+E*KUGN?zSH8cH!<03GSj*?oNLmd02AMBk;9{w3DaY!!sz{vtN7I z;gCJLn8SY`aCnv1$UR<>7rjQm_KND{9pm908}2C|htN3YWGUV9UIm96S^U{9~|i@grL?p4uAQsp734woEDkkqV_oZKTh zeNl4uwWPLFzy*(hOW^@m5(2KR3b?T+pza@p8Gap@^3Mku=3J7RHcJ_v%*<29Q^_n6 zW&CWJRjJJSlFX)ACg90!J>_;PxkI8{lr48El{;UOyZ#bkxIMhEcjDi~7M3EguxC(^ zDySc#3TFp}lm>-f`cDffJiocWh8LXX8LU+WFGvi|%nn}skK+k956I_*Z1D^!P=#zu z3@OeI`K~l%$EA>+%^@YcfxA5imZ}EsO&nO3J@7#3!17B24>b?0;DuIshE}UWk0plI zWQU$C4LyA+^lWozEpN~T&q0?|gRUeFx|TiYM(LosOM`AT531*dHF}2KRfXM8411Uz z_P8|c>7}se&0$Tv!LK|AH~&7E@TI|R&4Vd(1@Tg_hAIqZDA=nN#y=`JmldXO6pZ;0 zGp`}Mp+hWY4B@XHV)f$?>&ruI-V70#D{Z}$c0-j8GnAs$N~a%{&X<+0Z|q9x;T64xc$= zc+%?ObABA2e0li%H^VjN5oun36F^uOJ@`#T*1rfH{0|}r?|d_|#C+6luTiB#N9~<4 zs_aj~2G^{PJo#hf>C2I4-$d4$e|_O!iy55qPX!IOh$7xmtYJ|GGo#pRqKx-Oajry} zzKvomqRqUcdBdVDev24-Sd7EW7}1&-r@b-GS7Kb>#)$tPg$veL z{8z^cK7D2C*|$?`Ev8-Yo_1;2v@0{GU0XBl#@=alSEk+ieW2i((;u#x{&?^7r&p#w ze>+_lCD{Ad5W#D{`LOqQ@qvBRtl?^dS!#BU+IXLub5(8HqW**6!0nNN1)bwW?FYcV+?vvm-Ji&Wbf^SZO|Gost)r7#71Zn3)xldxy@Wg(z z5`%LRL-r+xUQG;ZNmO*6q4b%d8a`v#tQiqGGe+*45qWjS=$08#ooB}Q%#0mAbNsBC z6LV%x-ZyjV)tS>0jFKOx3q-8BhS)FIE^qHMKeD<1IvvYH1uiG~}@9ON0Ewl4G&)MQLr(pP;ZL{VS z=gj$T-<%y+=j?2mQ_^|vZlAfO!{_duHMcBh?ty)C%dgHo)H1iCb8?kWa`o`!W3!TL za*|K(OFn%y`D{yaZRdFxeCAymKJUt`dDn90-Pkv;?&`c-E%Q?9JI`Zyogkbe>EWRwKUgvX<~l5yKlN@M7noUx^Hf}e_6WZT6*BSbm=eAfFrd1 zlC;6O+K@7B=rwH^LIM9i4sdS9$g+&cYZ;^8W&E3i0P{1`d^5EXnG2FKGjlT+mt`)! zmbvU*W)^?ZO5a7<5q}y1cw5rq;@rjGl`Y{P(YtTwfLVewEZRTke+~G$Om--^Tt`Sguz3tyYa#J#6;s zh_!zW{QGZ>`>V0cP4mmuj>ugwJ2!J}?&AHqORwiHd!L(Sxpt-B+UyZ)*UVmHn@x8wS{o$uF`SpF;Fevi%0t67_O60v>} z<@dJ`zautWnY|(9+FD(J-@5A?ZoS`7Z~5n;eg7}T^}Vsh^}`mi)mC@^t)3&dde7PF zyKbxhfvu7oTLV9Am0A_Z{R@Id7WA7_5WKD+3H)V~Gwol}&wu4vAIqU0Mz z^FI`6tcuh8i?t(*7tATnTvxpK!2j0~eLwCJblGj&Yq#B~-41hii`MUU`f0cGzX<2+ zyMB-Vzlh`8Z*FPu`qGe}N<(j!hJ7qm{B00lMDUHg`Qzx1KSp)g8`EoV?5O`x_}*n7 z%d)!cU)gJa_Ne`9=I+m3zkl6N`}1z@-}rHVewPDVdL1Yjbzs}v1I6nPeD~9V9XAi` z{CJ?G%TK%iVeH;xbIWVi|21%L%0C&o_wi4Mp58q4{NtgfE{9+BI@~v@= zwzuu$VamFKNGe#76$Z%_?7Rx&@(NB}g=uRAV_j(`spLggS|nHU^D3>%E3NA)ZCWb@ z)>XEWD!a%khvX_zUX@dMm2+K{YipI*`iQ&ah-c&x@8l!Cc}M)qk4Wl{1hyWLT35>@ z)j^Tf{gSJL^QuG2t3&Im!&<8q)<>0+qpHZG!;+6i*+$9PyOC$_C!c+oclL4l*{5}9pSPZE zvOf1pa;`b@+}q@H@AA%lC_mR)cdo7V9M!d!1k|#=t~Hof%id6He6W`DbFJy8TBhrH zvw-uwug_b|JI~*6-s<3a>z~itd^-Pc3ey|->7umjC3(Q5psz3Wn|CRA!=;demqLHO z6!z(oqU&X4z-86fmxs-}9I@f@$bvQ^sT37+n(0i zJi8G1?9%9GSLQ#vw(;4GL(l4NJ-gNRtls8%W8m|4^paOg$rtru-=FQ{%!=wQnl(`1m+#NO0oe7K2oyFCQ&KOcYh z_AA%Vuf*M6yZ3(WIp(!@%4^?Eul*0dmfU_F`1!T8TeG}(bI_RPekskto0>xoH;3MC z4*T4!==Mh0`;BVMn_($$A~wAldH7A_?Kh)8zlrMhHm3L6*fDR%r@WoG>FwmhZ>QdV zJN@%pb+?xI-Ytn^T4tuSByDP$bGRk>cFX+FEt+oc(t5wsj(N8r#>y9noX@I54WDa z-Fo(OYi+kr7kYoXH0INlluy?-eY$b@Q{C-Pw?2QW@7C7XyY23nw)-h<4>z?vKHT>7 zcH8sMZB5-izv}(DdCcdxDWBhM`uyRyV8vI~Wgu8F`4X(yAYokG^cmC4iN)LA77y!* zMQc5AXxy)*9d7JLP~*RQsqlxvh_i_KbA%$cZI48piN6VqcMoAB!uuM~Y-d?`kZg$V z4<_~rkDv`%@)1UctdS9P4|d<6{`ez}gZd7_AEA^f`^acMM>eQ$1no*GjJ&7;MyAeT z;Su^+u-||%rHr;=4_5V|T_UlVj9{!h!WcgX4Uk2QQS_B5L-9W3X%F+aJ%0Cx9kJ_r zitq?NOSfLk{I*l`8>26OyHUT3fbC{rl**#mY<>0+V`f_xQzc_@DY+Yq9mm*4aon_w zt)oz@kh5iJ41Q?wLs%W6&F-#cgaSL3+=BSY*lyMQ#=P#VPW`*4)mok^PG^LLjB)|P zV$&35K#7Ao^=AvU(uE?~Qns8OBV{TXp6c{e%(BWG*1j15WA zhz&F&?z5vz<+0+z5J!EDQtDXAYm8ujry;?Pg}kAQ$x{|X=n%IpX+%or=NhMKHyco_ z9evqRf?@`H%C=<>(Xiy9aY3v?mQje`X138%>u_tg1!ZyrO3IQ`a#q=jnqCh?IJ~G0 z2{mk?Q9e)Zsx_n-Lggz%9PQs|l9rYmvKd>^TbwaLP#zP4^-^-Ja-;5ef{5;f_&&%} zOWBVXy$rD|%#G8Cwr%KxWe;X6m4@8JEQS@JuISFj>iawtCo;1Qg|E)~^$jSz)T_Uj zn(xo*stxYY;FidaF+^BUn#zlzgaWN*G!yJNu0|P~CBP-5D3+0sDpdJO*;);@%gxkn zJe!f~u7+GPMCoQ+W_ZEvL0)%`+kGweeIBJrpp1n$jY3ELEP3#PJhk?#)J!undmFzh z+m=NMa@lm2e5uiCBTB>2J67c4O&*nPH2@cZ!JnVR11nF7nrp7&3`OEkn zh4kW(0bLieV>(zaZjsr;uW#sY+v! zK2Nb!&_7EaVx%p|&kkWbZbLlTVp{4JTfn+Bus+q0ijZnLDU*nX4&eqBkYHg3PsvvH z*xq8Q9BWNg%OkWsvR+r3up*+caSiV=HH$zvlN~hhBwIicp(vmej!J2FZG|RdRvj_MCpU zLO#yxz;ESR-+GD)iIdx}Hto=7dmpx(Q?8;cE4orzsz*Q}#qKCFubR7=PY177UUT& z_N~^$)C@7s6lN0w!Km=N>ytaX{I5IX*|ftlX_}f+J97aX#eygpb+H-E!0qr8BL?v z_|ARFjU6eryhe{=uecR%EP0S=B+S}s$`DP9hOW2kw3w1oip3O}4pF6Tc-{am(B&-5>Q8YO|IpLqwZ#Mgp{&U?V|wt?;2Xy;gYq zU_#y4hfLx%KlX|}QuYacLKvIoCZ5IM5zLa8&K}vpZVQ{DvVOcYvO{V@mGP9WQ5^9$ zmP#u$%nVmJF5r))jP3>?Bb^lK9iw{i!b9B_SZmpW>M}j4$bb^)t_f}J+elY2GD@Ro zDi6v_F!im^j9E(P%!HB5Kx;vT-vjHdON}1oC7h*^1<%sb%w`(})tq=!+srHm zx5aW4#q|L+q3Q)exCzWU8c?ZY{gG_ysw#4J;M$74=o zeeHvW6!oAq2ItO@H;&THvzR=fC~cDN`GGCmkRYdw`PwU1>DD-#FH-etWh_c9XJb23 z-@izNnOv4ud@DSJMOD*+GMsOzwLs%5wWcZgT`fg1jRA#cxgm}-G8HEs3pGmFiY1v*nVahn<%;n97bJ z43!smOZb4L#f?3KNt?UK_f4eZGT!R5$glP&<*d~IKNv`Rm4S#X)INvr?7^EVGS$a}mxrQ=kH&Ag>#!}8_sQA(T$IEi8g%9%dO)c%T{n^|Qc41v`2rZ2?Gqx6}!ZVEQP34Bd z!fb+~gzx`Qq`fA4DBvv}60$2@;mhW7Yi087=~T9%F>KLNgqE(Q*tvoKikhQd2q|PRCT3tgECMHs-C%jWF+?tinbTSlb zF9~B^!&FzPR~*D9Cps)!iUpB-na>}P~$6^BE|bkDq; zU!QrWMaNGWm-x#wZ=;jNNhgQ2jTy8<^Lotnw~vi~_`vIHE(tYfwLk7!Pt`q#j*Uwg zqnRFiNbO*8)S%lo8EgS&Q)z+qbu+^iz#7K2tpyiB$6`tLIpXWF=gglOI06Pm3a z_e-hOW@z!&(BbKMhscts)Je3q%dm~C#NuTOgH&-Q{%cF-h4NysS zlLO|cr0dq=-)xmMShpjyRMIo2@elLsCL3eDo4)_y+8E3i^x@f)3ha-gZiPuIsc!d& zAk{CMinUhna-F^vdl$cW+1*!I>Iimfd74V2O`Hz{E)RvEpcvP_y-%T$g6)>4%tUy!Ae7KTYXMK3o|3)ZNl?Wb?G z$wVs4MNO@MRVGoSqbF)5?6UD#BvNj+x1Ms39K2JGEttZ1`K1hvJa zg4${lEGSS(vGnG?Td?7+E+Q)5g)hqga*gs-QWOtYI+#~=_}$%y&u+hfSdh>D^%PrF z_>2BvG_L0p{ie`exPAVo-HK?xrau1}w<0>F_TO_SqUh4tKfe{xr&z33I#lr|H=>|F zxDgFy+V4a9>B0YHcOmQF?m}M8fq%UX8II6s1Pq@sPSsRKF>FFor2=08!)7&R1>2b| zsur&B6Ed~<{dlNPK9_BC77PcBgVin=VzzxXIgvPfEV$1RznDjw)L@sJp@ zoOtrhieNjdj9~i}kF@S<6(ZhZt-MfGaC!34$@?RB4*UM{oXXL=t_*y;?nJE1+E9nB zmnZJMX_9mCS(5*Tun8C6%Qry6amDfOft$`f=%g#K9G*Ydm9xfQo(ns)-#N#q<0p;%Qp|p^u-L|P_^uJf zY|RbK`fvIE{9y@vjb&cX3N>HzO9{M#H1&zaexT||*#as96bE)e4d8@Q27vD%g?bq^ z2UuVO{-Ar>^aWbjbjJc|GfW251gQ^T4!U}q?uh|^C5}SO8>$D$v7y;W2ZOGwxI08( zobIIPovxaO{|!qmJWBui z@T@Dfq8I^>v#^q)gY9@mR1{5#t(KuRXho>Vyn}%DjkM9@2=$Wea3f5+=w~g->6n>y zBGyJzyVbTP)*YqCktA!L_V^t-i3-w_#sn#|`G%cHR*fC$p_-f@vcr`QLguh@UmRpB zt71{xD+DW&>>@0cjW7A75q5p6Rtr}!J^2EMxv}CsL(`-bTpT0U7diAqT}tXCrLvDW{X(kvnbpFx<=2cxo1F)?wC&7C2Oi*xYgK6 z_4EYDkLmJbjF{q^86Bg=d{2XFR%TW$iyV~-WL6wAH%?SmFlAOynVub7GWpn@D2W>$nz6 zxejOFpv*emq4NPN+G4s|2M4OIHHvzG|qt*8fe_6)S2lE_79j|E>=P zO%eaDCj(u*`p+K?eyOGYPkJ^G|MnfD_v)klW3evF{M!D}@s}(!_M7RklM*L=GyRt; zMXjMlm7zmH`U2XaH4U@t@{R6t|6zWq?O=WqHHibi+{Yf`21C=ec>e#jKG6P{nfTZG z#IMiyzt%U}>jyUN&;O2ge|>zX`=d3{ge{VW|5ihwhcmj0gYMCNPq;Lg9zo4AVWftwo^XFhXUNPmMECur=XQ`ZKG4w26YHP&qQx1e0KDz6t*NEAGi9tQ3_| zF_nsk{Bb54N^7FVq=!@*9{7{-Im0A^T8J9NLKihFUFBGUI!Y6@7*G0B@ua^PpO>J9 z@U2NMwG1;a!%8w?k%^RA>JmjQcgbfhPz_*Ed&$KJv`k|QxtMVsstzkn@=;Hzr?Lm} zQD?#LiQ&-JVP&ha_0^bXjY|!cZocm{#g{|i=fA-~TOLj8suS}~TX8I{7~S^TitfNag*ZGp)|ePj*o)f8=kb~2-nvbGe%6pcnI z=Ho@N91YbJLcmKIip`)QhnyE)IY`&?cQm|Qp*0IzmF~e~5Dy30;wgHhx33N%7 zRLpQ#dX-T!0Yes5N(EAZfnJ8($ef1>EQ2)Kuo!ic)Tk`6wJEzgxuz&mJP0q5^~vJ1X}4w}XGIj7pHBA?@Wlq0M;ygwrPJLRHk-AiFZPkoGSxrQ zl!S-Z$|!*Vo2FyLQihdQnNKU6S!q%Y7t4_aM}!KI0d^TX$;U4LQt6=xmt&xJq(Xoy zklv9fv4+E6mVSg9A~GN~QUTwSWqLeHfaTKKm-{WDyGKc-EK0AsAu5QYdq&Tq_y)S| z^QjI7R8%hCB1=Dgmzd8tSkQnCa!|jLxhdG@r}W33;5%D<8`b?9-)a5nV`AAyQ% z$!tD{y)1gqY>8ljQDd}N!mKhZRBl(>tUn{$e9hX%;>fYuM~NJpg?BE}soraj-)T_u zIl7vSo3F%T$p^sCa9na)l_zPmW$hhd@> z^61AU{*EgKY?dwxb?79wb6-1>9bUBRaW2nT_2e~ohX3|un6^69Ue7q=U2{5s%%z6M-TmFOzipncAe=Ytq0KhOApz zsnKLIR$4l>2`7fN<@Nhah3WC4#&^k{2E4{TQfY``_VGmDf}oxkzZw@j)X8SBec!lp zkzr<$a=W#Jzx1BMkMFelNM*_SBvTfugF{573)3Ier&>!E9;r<5m73ZdtxRZ=nr=*g zp!QADCG~!SkTuKM^mS37VaI#Y)Tz8~>6HnhA*d`ilw7wq8c7mHl|v6WHP7M8o%RKBj(HAG*rof0u;h#(aYk9>}TP(iup z3-PF1*fsGXSJ@!Pi;^3*jE z1WeQ=A8P|kJ3j04CD}S}Vz(O$uY6FjJ+02jx1QI{vOpSP({ZG9b4~3;&ko-X`~Lhy ziw}n3=Y8EH|8lka)5Km~OpE4Tn@U+t zE4le>=1)IQfAIQ!vftW{x+HMZ*#~xc%!AkWlIbN!AI#+^cf5V%!RzzM zf!&@L&3%`ANOSJNl6UhB&i*vHG0z3X1Ie6O!Q+>NQ}(3Z9^ zs%e6ll^#(4!iO*78Q@g6e&Q^Ms*S=(gFZOpuhPt9dJFP}H6E-~Gg*p3`ev_dCTY5B zq7o(PlDqo9Tp`n$dKqb*tvNG|^?TydjFOEMV}%R*Y{?SxY)s|R6Dk-RgAH9*@MD>| zM>i;PI~b(PH@hW0eHz}0%39)k*4qxwW_{C<_|D&L0q{nZU?2G+&zk8D(z-80)4>6&xO-gMz8*SMhc z=>3saoCTHflOkMm3^3&BoHH=JvLw=K&8b2huWio2JMsIGolX_Hb&hpSKQ&J68|j*G zy551$GfuC(|6<=%@6~6+4BA87+OVY8ExthR)u3;CR|O&1<1M9Kb} z&YgKZBH1;kvQYdk$kyEWNWyzwV8*$k+9o7&-Ojt3EVy?eD&k9GwdTu^r2$ssDcMw2 z|JqTvr6ci^#LhXD^~hHk+KeM(rJpNO?~gRwbgoF@Yj2ib*<(e>*A_Xa9=x&)4a}%| zuqrq?aO1hV&F|~jn+7+$-Y;Zt;tfuzdVpOvMkegno=h0np4@wdY`*!_!-N4F!g9_` zO1NI&u*&#o{QF3U&8NnBuJsHy8Mf{Ew=yKuVSDwX-F{!|7I%N2Jo1O|^vbNzkw09#pSVBN!F=(!_ybeJa!wS)Y)=X- zI#sV6x!QG8Ws&EIsq!VtblHJPVHpeVE1e>zZ(INE=1F3H!I|-^f_Wi1l~GNMBzWtP zNAdgprkbulvUzs6Xw8LtucQYD6rMUGLqlDm(sqStb;$Os>$8%+c3pq!$(zOHw|I;1 zd7-It2R%hv=4YLDrSS6NBR}qt7n@Hp*OjCeQ+%yU!nd3mzvp@hS|ZV&SQ4>W9by-g z9^uSgax^Ss;g%9btnJ!hJ6+M6A-Bay<6S{p7z}kV&QOS|V|E!lWxlF5&Iqzyk$KcO zql>K@zuIh3mB|%N0LLVBDm}N_YT4-{_cd6&>W*&ljN@ZBV`$g4GYyK0zmDvVEm#=; za7LJU)S_g4K1vtew75>y7RC|I!$cCBU4vClGftox^~$~GideHnM{QU7Z3xrNb287# z*cgSJ3#TaKk=nnj#j^EV6(R{cBmDkLKIg`^fuo}LTbpJa-ZW#sGrk+6q9=WIZCiK- zIgy9P(CuetN(|CNx4EDQ=pY{@&cxUJVwBp7H$APaKGEOArsyIH_YKB-PM=Bj3kWNc z-NzTiCcU@AE1wNv%fmC|xoG(e|6!WbRNcy^`Tsa*?Jo_iDK*_*`C^(}80T+p<9&L& zoGbAzGB~0V)mftrwxM%u8_&3*^m?9q#~<2TWEVv^&(t-{9)FZL`3P2ayW^rD+x4d( zC5Cv3ajjM#xX#~(OE%C&xjeif(McsLVIsULyAR9su#Q_%qu{$m6@}v#>2<9nbVvM8 zN2f2p`9t)SYo=@aY?BLdo#apVW*oKRL_9^|eq*mepHl+QscT2woVowU>w9nhpq|1v zU1*$9TTwTCUzN+OQ%7%^_oXF{?D(u}C24ToV#FOjg zocb{?$6N2}v7e`BoO$LhL z9=_qMFg_Nrp)&(n=}Hq7);?>BgEGPyqf zo1?>06=APLm zkFK{WJ2-0Bi4E~{kH)N;B5s~@c+B|LZmSqOdzdhw8;?!+w@VRGWVsP`> zwx+mwkvl7AIU6jA?0ZW6Bo6Nh8jj&G1NE*RfA%Iha{9Tb`gqKeI?#*3Ubl?T#|0QY zI9kjv(=|BbpTmN)x;Bnq+wzTKCdK}~CI3P{i|^zPHLK9NQMUv$^8cWY`e#+wKWqU} zn1_X$^V(a?M0R*uYj3<~p^HGIt_5A!O0KIC>uLsQ9BFU8M+Lf6Xs+%0y9%^7D$6Y0 zynJ1qNB8dr^S_nk|E`tf{NL)xEIUeHcg-n=rfn$`u7F|C0EYi*2LC7l-Ng>?GjyF} z7=}wbaa<{V^clhH9Iga81dXglJt7IF;|-6V?!|{4WkMsBvS=QQR`7N2Z}hlanmy5# zGQ#+wiYQ%9>Wr=FUKDj=u_&6MyRoI%U?*&KCCzXsIw*)X^~B@9EjGbmby9q=;&Jg7 z#+*xki=EQP?v5AG3>T|Jvot-67Ad%P*p`e&Lth6y2Cq=WczHw{In!}$U5}q?%Eri$ zZ9xfXQ?vvMX{ip~%wRXrwK9!^{o&CYF{_jwMOz}n6ZFbqgl^W@2+fG7Ax8W4Z9Hj) zhb1XH9=0zeG%r^`3((&Q{`U&FHeB{s!iz^K0 z@mK|l2?S|GgUYKWm8OI&^x$V$JZx7+ z)xf=(gHe@PibgoR%c30Q);}V>x>_2 zWSjO6RDAm4s3prvVvIdS&LFuuiv0}vmS<#>zTzV@a*^qy!}Mvck#Hf4=8l_(4q^h~Z4SO)y@ewujfmz6U$c<;;&4H? z@Z^jXdqr=-F45eeaXu6+yu#tK7(JRTlhGsf<yg&Xh+mLZ*FGAepJeH34^ zcM231xM&Pxr_0BBT#_kbD353>4tl9_qaW*6)@vn9xCwLCUuMNQ(;OZqlP_s-bE!n-IN z0YUjdWE3TgejuU*_}bDeF?#koo?#=u(R*n}`UnSKcdD-=R!L!Y4V$6Fsq)Wx%wiabJ4kv5z98iBw_m<=D?!bR|ujiTbf9;27Y==Xnl)4`R+|Vo7=DF;@yP?xc z6i}F+&-&k6`G4K5T=7Tk-86lOuh;g+P2P&%?&fs0t^;bCu3zT@T3?clW?ecA_ipwr zwCJLHDbIJM?n5wqVB<$FZA)FEy{UZKSactqPif73j8)RMv^ZisKIyvY#)2=b)2(_m zsGFxcEuyThQF&w-_io=ryc7^W7>5t9OIV{mqyc1i1*HamIsg|xXnus@G zV%BDr(L>$yGrf>E9{uc=SJ8`1M0P8`bVOXOGRmUzaEBXe5UHYnmx@9Xx4JSw+Gx# z-XrF=3TgNV%oX1MP2m#Hl}o%3%`T{SI4tnn<#m+btJ#2WmO5{J$^gEGodb{=Y8h z{wtc!zVp8qbX<=8&%dB6J#L(bzlX*S?+s7cYEp zk~MZh$heTo9IjdAK|IAqK4kglk1Dzp_6*XMP^?UmenM?%r8vK9rgKt#ga~~#&VzR+ zo+&-Oom8s{hEPApnUliLheWDYP{TqM*2y7yz0vYAMsH;Y(x0zht+*u9tA}ztS9?@+ z4nOR^Ob-z_i!^m+1|=Y~oyS>fzaiv6UA!KYD)C;NY^U59Q(4gy!Q8 zK5kWUaVqCr<7(%eo(!`1?5KwqK8*@neEMMnwISzR#RjG6Yft8LvB#C=sTU+cQ=3oT zcy_SAYr5b0$Ac?+o?E)*(cmER=C~Samx+=jwcQ+6UdS8lncjSbOr^_mGaQ}6EO|e; zcssWIkxE?|^ThJfk!OsxS2SPM;7R33-wXMPwJSGSoch?mLhHyC;oVxf4XHg(tcme# z?YN8wk0zgp72&;FndU`IwR@bLw;;*v=p zZ0~IIw##b2xrbBIauTCl?YehspV+3-oC?bZ___N=9Z5eHACT$f+mM@$3NvgPcqFP5 zE{+cwkf{3RkB<-75VfXQUZID+Wrb!bAxZjpsrAEy-Ah;{M>jdF7N6`jXW-l}sdJ9J zhGas-lThZs1XnrJ%Gxm*XLpO(e6p)Njjj+DG2f(Di(efdLJg_9yuu`Xz$RU?zuU0@ zuXY@Zu8EuYdIyxW?KJ-tgCBBoCAa%o!2 z&9>2l^a1*oBCLB*&-gA|CO=Cz1i7V5^6)CCo47g3YhL48Ef-wv`q%KfjAUP^*>x9ZL`*-|dmFN>K38@d^-ZILyvr)sFiM~6Pp|xyzAoWxl1z#Z+5FP@BHK5=v2m?$HR9`=+~y{ zrQUnzo~e2FRQJH&`gL#eY1Ed?V?kM`TCSOUEb+~$?o)49OMjy@xOnsiY-!l z94M7E|Lix%`epoIU+@{Tt5lLrU2SOIrHZSW+v^rqRpTDF*%#g{_w%R0wL7j|`Q>r% z-0LQ^(KEZ?=iwbTOj=jidz<6=He(GP!)xyO=9ymj{I90#J?j@fv6+2ulH;-7KBc~M zDG@$!dZcIi^(tL<*p@hO`(;L;;$U;ES|jfVV|=rZ#izMW$?b4DF~D|e$c1PttD7O& z*Pq*bZal93_`;!cZiF2E&9U;ZGk3*ipGmXO&%n9A#=y+JY2b@j0~A{uu8FA&Y?Anw5{si>z9AC zYVP##X|`UgRWvfYOwYDK<0fPmz3`~}Y?|S&xow?@p*Gg9UBf&F<;R^~5LkA??(wg< zp-nq-sA`6;;;>rQ4m*R+FF4h0OXGW0W}S$Sel+TZZc=t(uW>dGlRA3O9{wVwMX#Qv z?p0r$b93&lIxkZB@8I&AiV>$ghUCh7-pGK9QIk9ks~y+Q2+uwqDgO1xtwqGYQl5rx zSX6nxbY5Rn%CkuW?=Jq~x@O}OYTkdoQeK;zgR*QAx4D^Jhh8XC&Cwh_^zx%^wYe#1 zU0L0ol`^#Mx^1;ahy8YGOZ1%%Jy?GXLX8eyok9*0~ z&xzHCRdN||;z5AxOz$OmcMZQLhHW}}Xuz7dF2f4?zWKE1)=ETQ*{Wjl&-y7Y)6RP4 zrd{g2&3A5YheJs*zK&{qa;H2;T3)eTbsndMk zg~wZu-2dFWdd$qD9jflmjXYKI^(9@cE9ZjVHya&OE_Y}v-`*Rq&CIHoAK$KgF0b&t zo0gqbK0R`rnD_+8R65SaVyWl2=CeJ!yRWX**trZcg6Hb zUL2g)PYvJPWV}+r$<8ON@a?A{x4pMk{8?~!gjc8jr=OmBusJ>Ma?hy5K_T%jXv%Vr zvL)VqX*{;>trD%~44=7^XO%OOK?_|KG&c39m@>&J=VQXNLH2E1<6$pl-r1oZ{B#b z`230YH`{vXY))7u)L(Ew7~_ZT=-4KtZsOFHxBHc!lE}jqoACyX)>bc{No-#5ti6|wH-ziLlRd1Q!bQrI5WR9ebD&BindRP4sbWFh63M(m%-BL0w+T@ql|$ zqh$>}Lp;ufSlcbG74MmGv!Q1g3&!9VJ#Di672#%#i1&6p_MmwA`0Wp@yKmtd$Nr?1 zL!I`QGcZ?^CMfE=PoBpjYh+FTI6r4m-krf^H&0rWm-aqkSnTcB3#xl>zS_6O>}~;> z2dA8`IivFK<~?dYJXw8KwDkpfTh+|h$?jBW20Y8V`wl!67nt5PAhXtkSBDbIPKa80 zX(1%V3LmFUzc+VhTxzcZ+le-Ps%A93w{U9rs(+qrH>g&>sz>iV7+90a>xJ9oO* zJbID>4FWRwJ?vTsD|CrW8g$Bg&arznjMPBh0(l2@qRfJmai=roUD*v&M%Mr@8{fBw zPTe0k)}~*^3Fm>vJ$+NJc6+evwEG&d*D#x6s{L-9$*ytvaNg58TxZqR&zai~}NtKfl-4+f?q%ZnGDkhmAO zA7tA-CDZ<4{;{O0<>|>)&ue6y_$7DeBL7A=`VHE7qlQE_?@^`vdPAQR-Ig+)dvA-| z(B{OYe%s=nw^?_;_|acdGvdppZfl>K>u`C&)b>^KqWlNtHQ2cQR7&qX9X8nYzC3VS z`weXZFNbVf=)Ahs<=)#GyjXdoV*i?Fo3-0r%i;LlH#VM*(vz8cB6-rmZZ4vB@|5PD zk-bhD=H+HNO2;HIKwa)VeqblPdaqvGv0Bl^2Wt(3w4rC6oxI0~OupgFfuzQF^Pb87 zc`KJNsLy}i%581raz4y=TFPWj<34fb;i~@Ejj82U?!Th{`*+E)&TXQcT`ae9m4l=J z@}SO|V8y9&oVQkXPX8{QG~onUi_{zL{HY_}}Ba|EasT#ew@114q${qrpQ~YmnD= zPG*DIR~cK~+0cUPC_`Y48i*3u)j!baVhT_)b;#tX^m;YVG0@n+Xbv!&O@U5S?d1s` zVXdm`TNo1z{?yXOY%~P;S9dgewTS7a(bq5-s#gzm4h#%5o2`M>iSWnT1z<2ZmZvq2 zo`wN|p$K3ERT6Ge+(0@)g2AAT4ZMKnK(isigBtV@c$tkc=IRDx4TG^}-~c5`6R6zQ zxG0}pDVAWcF_=ntm<@`*5}-E+C}&&|pO%`mt{S7sT8D&slU0Z#qIi|@4>TYVReJ;~ z(WZ97Y*cK#5zXX9PnrX@&fGniw)!N7AgICMs`vmEe}iIHOudwD+R5DM*|?LX_ZgoY zXj;ZiUyE3?xs)N@Y{)SO7z{MbXl+;83t zjw?L4GV?c8;u@sAAGIio$pF%r;G!{@iku8aDNtycXoE4@Q7Pe!O6t)7M>>h|G__HN z`GlxEgIAuadL=>mtZl1xH7(#S%g)qD`Po!LS)?7orKVC&IC&(bt43pB!b0wG6pdM_ z0zRR?D_tg(nnBw29tLA6)WsN4Qz1Zomu%FC&c1j>4Ot05D?(3+|UG?@^xl|$yII6p_G00X16aI{jIhQ8;&V=)9en!1`AG#PJO zgY{;{-!Zfdbx-!0%@1V+@{1yRosrQ_PEhu{TL+pw`42se31u{adA)R~R-kEsY7R9d zlo1AVf`ejH*?{)bmu3ll|GFVPgnwVxNe|)wT-QzVc6n0P_xnB^gc*Pz*LDA~EdRGx zBma9{_doyhl>gz^b=eQ4QvUn8PAcVpdtG<@7MeL9@c(yR=kDzB&lPgln+_vzy7R@wP-xU=5b#@WVQ{sa-(t`b0E?R+{%e^s;p|49bdB8iaGiaCGh zd(i(B)BZmSxc)0)A&1vt>sP=0Be*L@>L)@Cf*V24d#7=Yj#+!~lR}0NKb}jhSP~;2 zL86Vw2tTN5+8WTE|yW*5>xmglI~HX7us*ws!Q@6ieo|783b$N^2EO zu|~yIe$*%-p5$zK5iEaPXoiej=7Kl9m0qK5#KSJFav(~hoO;#E-lU!#X}K~sv;ixI zLK&nf*1CE;LdhRWQc7FFqH!3hIDW;6UnfFjTV0APZrJ-j#x!GzvJ6N-o%|9Z&g{-bfYf0w?A)$ao#!n8a=y2H9RdbI> zxg)lE(f6pWDsqnjw8w2#zMWJx$H>b*Ve2Yt4H!XYJ=55dHuDj=#)5ytcv@|FDvs?t}A=C0q1Sh~M2L+*0=Jw0&5LDelZ{X7kCW}+u# zD}4dl3h>Tc8a`d|5c)r0^$=DaH`1>0c}=wZ7ik6X>RKU&7TsSpA6^^tw*w;&7SN)L zirWk&TI{!iW>Wt}(CoEcLvHsge*jCjgGN1B>lLEDu1z)lDhJL~LNo&$RJCN~AgDi? zYC#d*xixTb6-G(T`Wy=!zV9I1?TQ>zl@OucMa*LN5bYKP%Pn=$dZ<22(FIY0F4BAk zc|E7L)>Rx?3gPT^$Wnf3OVTIS5`Cz|YxTN^!^A8erfr)Bh4pmM)w}B1vqVyMVRjgj zoLSOSXKQ5&g_zBPjqdmqxjXZOa=R;FlP zL#5pZ)tvp5SxKs}fRqa)YWeb=w827)nriCP_?}}1qk$iW$u^W3L8aAxg=kOES!9l& zC;5U|6@AuIy4nVE@9rSC?b6Fye^FJc=lYx*$lpAth~l+m@9QOY@`^_=t+w<9SQ|M+ z2&ArB_70x2my@56^BJX%f?lUvt(jxeYd%0wEyo6|8(6j7QXSE7$?HkBBQ*?N=1Sk( zv@Zv|e6=ck4YK=YF7>BF-~SahkCD-%`5G_g??gmS`_BAbD#vQ1Rb!+LEkQI;84G#9 zzP1@kr0&tCF$)w`cyB@KLst-``_rU_imRIaGL=fkP2WN#t7 z^s@BRBUwtYu6lmWMR2%x5Td<1eu3y*-dLvDM;dcG|1~YRMV8L*dctC+^RxGmown_Q z1hz#nOQRC)BQ@no3?Y06 zG2l;I4>_Sk@>PaBfIjBYiz@D1CzX4H)CmaR5G0M;O^92O7)heiEvzaUrD;i!O6z@> z++Y%UB(~mTI3lG!XOJ56$X0Jr%cIYTT!-cbSm)<(Kv^|cNz{2k0sD6p2mnLF#H){F z)3w6-6NT)-mXn1(gOyE8l!d;KI7IJj9fvjyvDotYpjVWESF3XHiX(NdJZ%{t#HhR1 z1e^#46x0TEs{?r21n{jX;87F>zEYq>GwMUHM5$Jk+>=h;>Dd5f0R+RqJ7q%PMx$Pu6Tqzgohzf;Q5kym6%vbXFzErE&VQ zwGusfxWs!QC&d64*lAp;XGkdk z*YFsIGJpiR#s)L|n$D0l^`memkwi1}UV-E#)c}lQG{nx)P%*j0(lKN{9uMl_L_pje zK+W}lKl1?zA0T7Gs-Cv~K7RaE1pbI41&YavZDqyBS<*#2GvHM?AhQb~auDF}$$*qh zfD?)&4y*+9Sq*r?!Y{&d0ZID-E=K{$ct2w38$iNaK)^@(@6dR&c86F{L^|UI>8-T_ zSVgfpVpAj{9Q_kqNNS+gL)rxO-N5P$`3YI`#YuoHS$)kZfICuK?Y98@w*u-Oq1?rc zTC8TQ?gZ`~eF4V5 zjDfy+iRXwip(>OjvKj`bg|+=M9Lc+OWInr5nkEdm3-i4!NTTFZqpgL zqbDMy%Dimkj!NFN@@HHz6XLoxc5n8c93<|wm$F`m0Yx~-B9`6}z3D*9(2SBCoUmh} z$tOSxrb@Gy@@Kw+uO?kL``D%0Br#7;u`YV0ne*rVw#s9VLrz1~@Q_)Q6{+Fhql{L;nT(#y4onCnD!=QlaZND25N; zs14>PEiHp# zex2tWs27a@Op5q*$E ziLJvJ)=lM>u2QmvPgM}#=t{Lgn`I2m$XKFHjRnxoM*v>SvOIdwBlt;b+K1yTV%P~a zH|rk0SkDPO>(_li-EJ67^-%I7m>f)j|G*2R9ecc{TinOmtf7gc zl2s%fJgEjSU}}A!A*BWIX37f6pPveSAMcY?@#a9tTC^{V1yS=DS!!NJ7R-dYq)imE z906r)%K#7kV<;Uk+MboU42LZljPU>HrlCiL&}#8LqlD z?n&9jx0V5eqV|xnT6O({ZDzgcr%7BosccmA>a|X7$gkc!jv6YCqBrXQK{c;nY28uU zhB^$C#i4ALhQ3Wohhnz0yVDjS>b-+K*m3-Fi0FwlwumA`<~z-%i1bhK*M9h-bFhWA z>5D0I=^oindoy5m9xqFblS`KLaQIOq+V~rIBXdLiE?2Q=Rz`jm^@l}mz5!pa!voL} z4?*KCsAFF)))t7j(tIr$oI?lPEnH1&3f69=HG+AhDM{g<*hgv_~w#&v% zrZh2AE@x)>Pj__ z+`_0G>pY5ry{*;*S6({^T-@~#eS6VYyOqO>hfC%EeN(OV75LiV2B_0d-lV#In%kg% z$;0O9_Ix#LQ-amH-l5QB?x30pI}v*sV(&xj=zQAT2~(uGNXk>5YLq<&QPcBq>y#R) zjTLJblZyqi>`ixsAgp@#9l$Zoi}Y`)%COk+sFh7{fB9&f7@0s`f*@=>y*wW*86oRUXS@oZDi! z~C$~Mhxh(MNl~xI8v~O#&51Gf)j)j3lBjZ0ZlgCbt$TbZYL(xg8x1x3?1?=iX z)176IuMb`76`zjed3$yt`LG*%f&IehDp9r*m3Bo^-Ng%Kc3^SNUj$)$1>UnYH7H`& zt{!CW?ZJLp0oJ-}0-9x&NYuX|3HLgX8gPk4!(;S-9Q#`Yxz+1)96#s<_D>u-3z&$d z62;-JsZQqIOSE_;YOJ|Ks_IozQMVz8ehg|Ds5)}h>6t%NyIR~jLq?KR`Q^pOQTe-G zq)W?Hy36Suq-)Fk4zQQUXS!ko!*`iN)W1^57&_TIt4TdeUnh5Je7(W!-Cc zj)}Fpd~fywKUIOGx#wO;vb^Yx%s+?irZ=|Vg7r%&*tI(VCq~C&3_i=FkLpZiargx& zteV&;_e9sK4M(jtsH?_EB~QV(s7t>LC$WXZ*#^vs-MA>09fW^FxEve@mn^hP^jrq$ z{Re{ld1-`pxH_(IJi_}?i@N;71ZG5xaUzJB(kYqFvL`}3V6+t0m^@L^L_N}61p8p9 zMM7g1G2|!0&wQK3kpK$BMMpRyI z$;A{EW`7cSSDeUuB31Q7T2X0F!Yo1>L5@4lpXMe6HU_r%RY-*X3*lN86Bz}$%>Y$msHFZ6Lea7nJ+r?Z7-=oeb%3%jj-tgnoP$QHn13f2wI#u-Ax*BFs1_$X zt+TDaC>0M+*91VtE&%7Q0PifoA9;X<`G8Wd0XP1p8!Bqn`DBZ2%ptL=Eug$ehXD0XDxNRBwb)R?-h$>9!cn28=`}6i9eh1ymUfMKvgT z41r=hB1#|ar=hg3Vj9Cp{TAiGA;D3^Ps_(A=K2qMAZvYqG z0@{A~7g_E{INyfe<`eYav038scZ40F`Ca=+oHCGhGXm^#K#thG3Cpo$4`nw~Wbs~# zyeJ@dkDQkCZvZad1gu2Y)yHw0wUMHV?1NI>C?Pc~b2ud?NVA#pE5L6NbPict=oSy6 zmSZ02rhkC6>|*Bpewt)Cu3fvDf}@!rV(>f4l(Ej$#;WIRStvruL zZasVz^g-7fzz=PutFsbi5Ci3$$d_|}d%dUdw&KuM7hm3bPtGfGx)$i>qsua9_saQG z7vDxyoCnBe5^0Vi&oB;*h0`YJT3o4tz%B0)^*WtMg>6U4+2t?;N@)-7Gm_snO~f&& zq@7B}CT4)BM<;4pkkh!vEOP!Tag&^~dU#Cnd`Yz%ydYZOO?u@xjuF-FojxMoilTCR zS$@*)k~xwJu}Ou zvfuC1@9JXrj&wkk1UhKMvn9ZHUiW~DMm|EQ<}4-Z=fZES25$???^6z_=DQ8aExE1- za9OEd(4@LS)AL0dG`E)?+pqYG*RprFEeBnu?FC%O%S*KfaI${Nw4u=0S&qQnp4l|% zNrFut8H=*Yf$C{80j=h*vz1`&xcG491?n zn6JdDcVFW;2wRj{mvhGPpyDdp5?39H{OC=c?68-QNH@O4xw&e%it$z>?oxy5STq;- zIG?Gk!QAUg#dbafzwtAsz+AUI@i5nQQUS;1(vf|3z^KhmNJoPjDvz9v`$6?)^$|%Z zyyjO%obxJqf1-H+jl)gOEh~|TBy=N;Ajn$afCUPu)Tw!lDq$6h14Cwsv%y=#&vlBXi5vkM7hqbha!n}PV9OHs+U|zq`*zM zFj1SA?&dSxVtF6CL-8pFiVv*-_P7A@h*jiu0>8d9C{qu>B5aF@LS;0Q$*eho%=b$` zHN8ZKmMh;~J5H`SYKzt^gO;`Eq%1#68qJAvQA8N^RbCQ1Am>j5>Fd59) zVtD?N-eqrT=0-^~=TVbF^o0033UPC+W@7{^ffK(41Du$fmIY);ZE9&9MoKnq1cw81 z7q2GIpQnvhGpx{Kv8o&e+)RKTo`7@g$%IZ;R-yrwci58+=Q;a`zO-74LOnmU0r7tN zj{Bm9^_2GD#*30i5~9_0eYmd>Ut~Yb91FSrC8q$YS1}Y=43RTgF2_+${4y)+rpT8S zZjcq$p+|);KCY%2nbnJIPH;jvzgNj*BEv)a6d5>Na ze|@6PC;m&dJJe<+@Tx^^S}iKeMPr5>py@pUZ3a+I_C$wy^jdQ04vbR8Mr5?^3)sfu zB98V0^r01&v*`MgP7`)W8={^$P(YK#Eli}&uBZJ`(>QTW7WkS(#bpDMX>7|^(x}yY z4PslybOx>E0{EcJ!m{_@&SVqar7d3ZW~SWhj@k^zk-pL*yPUcNT;AnRxG(On6ev?f z7K}wi3gkW4dcyta&bpAyEXhrTc0O{%0!FS$Q@i4b?cV%GJ4N@grg}^%5arN!%NaZKOoU6{ptl?) z*S4UgE72{MZh5<)E{j!fwkxpT5mxe@_b|sotAWf3*=7+TDLdu5Y$`rL{ z&UF%>-FBbg7c)MVhGz$Nc*+C$ab+0F3SN}6O*~ESoaupbi4-|(0n+8HBfG1h-u`QQmEEu z@{fk8f0ur(Ju9l3scL%wpnD)01&|#Lf^1w}QoBf<_=!s2Fp;9i{iThz+@i?Nyh@f+ zvaN^uI{7OrNZwpHtdc6g)YsKNqB&{G79UQ7yoIugpV;w8-qI!Ec0$qmTl|)*xM^zH zBPt_a_}HEM0Z6!Tpr5-gdYc$>aI4+ z8I8C#>{={`tQc(FVx{9LynMrr*|0x=`=YCQp~-3fgVu~)_Gj4LF1;DdFn2iG`z&|w zhOidrz?qz9!hywJ+?B`T*BEy(tPG%`T)~U)ol7l$6wRkxze11mN~ExVi5T>pnEi_0 z-buur!#Gh)N>`OsHm9c+ckUF&qiUz$8sqzyS_!Y|5lvWRd(8K{pd{%&9O!nD+FN*b zr#}MU&H?ranTOcVW>fctzo9p2=Cc$12+Q_7;1#p9WaQfcg6*kl1Nag;s+!19$!qX9 z2XslUOo7rV$M+HeC5W6R!YzSbZmHB#p&6XfZi$9_!HMx$|n6F!tQ?Mz2iG)tuDr*38OdxAU31^?*B%D_%Ds~~z|Ius3%-_k2# za}c^<9`q>%A#`G}RXCJCKj+xV;vuIO%6n-~{a~=>d=_%4!m6&@MzF8%aipd@^11Y7 zWrynTX@+E&QXTj;QEoiH;DoAc1|Nm;sqsCCK~7ZqI|4_Zt@I~j6nGATo85XqFsr{@ zP1Z0$)s+>mqfoy2=taR=$1D8G!5JExjdyJsue$4r$PL8dA9KdC37xvZ)=zxiNW-MV zD34q%3s}%*I=eZEeuOn};eNowVwyO&v)pektpvdBn*lx9rwQRy` zf(R$HB`{MguEHt)vfHuT2&vonFMg_#Xh3K~V2{DCURxF!&0DH<+m4gb8U$;u<~ZRN z$e3LGmQ)h*6l3NNB$f7t=`5^vl5ipM8wsafB;?&RzsQS~6jWz`rQarh|4c;ayiN_(Hg5wJxStDhv21WgLqGpC@L z$H<5;_W(`qLwy1Kxd&7~?;!(J$HmAG2FQK0;t$SfhT=P^S^==hA~G-#beY|tWp7E7 zw_^`X9~_3?EcmH<-g+ikE{i%Hn{6_GnIeZqgZi6bg z12|d;{ss67Mc|v?1=S2$H8G&hJp{E{I-)guIcryInt-d@8^N$V^88xes&~b@9x+hp zmLtH|NXa2kRgz1{eAG_NJqQ?i5gvQuOR-9wP9S|iE_#f*y7;UcoKjR^&0oTegKPtB<&-^gq|B65?rhJ&g+A6&k_x1H@k27j7{@4O&vJJ-4 zFOJxO5N?cu+Uss8ZkXK8h?#W1m@Zw}t_-E+l+^A8bcce-M43t5N} z79qH4Vt+?SGU~-~bFdE~*RU3f-Sx>RkaWYPCS0RCpOg1<3`$d}6X;18HTAD~9kJ22 zvfpGM7h6Edsw>E-WtYhPM(lX$+H53SG|PZyFnv4z;XK5AQF}Ex_3Du0#2PFnQk}HL z>h))N6s)xH2-(hAa5`T1F|CNHNB(xIs3;fe5lAC-ExJAv37aFzhk#uSR_grVP_UPY zGz|l5vGxs!NNX?0ATAu|=*y#L90{&x*C0%Et2#dBIqRsIHdz{?CZZW7m%l)%DpDkB z8|`|7=BufnUV_Vk)i!MA1qb8R;!}7=lHCCd+rW7fLKky^ENR$ggx)hA_OTP-Ded#G zAzK(Umr$~A&)rJ)EU9zOvbQ{c-NSn2LrwEXQq!JMWb8Nq2+bpRX39REw#C~GpI2@;XmXtE}qJUhU7WIRDv;Jg9 zH7-RzuPo$LPxMBSEJyBccWHLf@b(N)%z3%Io~t>O{D@$f?v+~GykkBUIx_z1z*>h8 zrEzVdT}p7dBnIo@BD+1xKyPEfg_q;My4|wK8W|O-ULHtjo2;j0<7e z>!~G)^rmss|GcMNpw2PgM}j-7QsRU&bsg&R011*)h>9<~FVao+x4LecBAS}PtkT9J zh^-CL*GP6qC>zy|>qgT*cRGv%zQ|8TM$I89(_5ZM==x2nbd%$N zzBr+3jE@;xG5j(9ld4n0Sey^%DYq0q-JugRZu73_;K?7|rXF>hL(8o2qAWK%(z{0< z%SG}~{z{!^%tiCEtXrK)$hGzo0V8cud!KSxx+oIygnL*QQwHP?=YrP%pm$LgA z&7tRylUMr!c?K3H(cw=BKH8pOmXBgCh*Se0wlwTd!?c<_c355pROPau>e8$b0;4GE zl%p%8lTnfDOYz+k8j}~GsqA1mikajk`Oqori0mm3(XzgvS&A$@JyXtVpMj)slftbR90<$eVzrFgSxhG}23&>t<087?!_Iyr72~sed!JOklCvmP%bpQ1TbnD)CY3TLAiIk;snCs(4lr z)ZdmPBla)yA+LDW7yOD;@FrFtv4?G$?x0b*8ycy3LUsvfrp%wstr)A;F`RRXkXF>M z2{$520SchD8{vc~Z9Y^&lHRBPhWi7-y+HMFk_g%&2 zQg@__<73573`9oDl$6Uzy&g5$X`fE%b7t~OV~@e zPml{>L_ZYiERlfz=wUmsa|qJiV%sCalTaJxqUC00C4b+6EQTeN`(=K(Otrk!yiHfx z40QDIlP{JmR%IlAXF*zN5JuB_Ffx|}Bh9EXW<&7wS)D3(W|f0DR7jPA|DT3Gf_ zQqQ=fa20((^#MQR6;0~5&dfj~cb zIX0#vMQ%yQwn}HfwFH2J>^M0{9#4OWDbBFaSQMxY4!Sxc#YubT-#f&ZRCFpX&>8dZ z5rp=P8sdufEJ-MfIt}RpJ9}x9-_XJ8)|%y^S}1K~=H$w#{^*%xrLUr@j$9Ip<+Dj% z#cfsx_K+v{PZJxV)m5FE0q4ZEgyH1@P@I(v+est8>NB)$Vq3LZJ4*>Y;!|@MO9z}g?E^0!;SLWfXMo_p$rItLF`N(>HT<*oJ!qqsW)sEBjingh`kG- zxRx2lL$OU|8&Jf@14M~hW9Y={wTbRKx{lK45m=5m4RA|X zBI0|FNHd71knlMV@gyQ2E0=C2+f#A zb-a6b2>ZqNB(ug%-!GWVxLe>m6>);6hRu~No`=VvOAU+t%Ci`^A~qQ6hTW@xd4=-y zyuq$*z*y5&I6Pc(?{CH5@7k3znm&9Vv^7K4yu1B*m^|4`m7ZJaD>>Gm7=mB6qq|*L zH!b`ADe`FK##gHT_{RmVMpoxuq}13Qlxnaan&y;}%Sy9Ay6CMyggN!^#Non(?%UkXc3MQ8z z92?7G8*vy5Ds8^>I&F_2|GA?GXoEvwG2={K%l1PYgU*#iS?K)uW=SfqWueiWBvGD( zY)=<*Yn_5Lp2Rm2F_&R^P5Ki{t^l4&ZT7zd^Kb~pLQuMp;&g*BJVK$piJZmM3G3Go zj`$AYObD?O>P@>#4I?eiW_Noiw?gT|8L0@shW-2gZa3I6b*Vr02YXAime&Izmy0cO zk1ic7_=(bCR6Lag2Kt|K8{yl4)oRnv)LqyHFbyYDl*cYxJdFI#RQDhr)cAgYb}UPx zX%XW)yWonC<&0JoJBCp%$iIWBJH@U_cwM3_Lnh8#p=>Ozx9l)aUW2TO^CPZ+9_2kl zqu&032O(yn7P>Ld9hBr?aZPqWH#$K5rWA+W1}nsQ31+3~YzO_2aKNC3ApJ6H;W&TS{zMIax0?fp6lcP!rBg=0eAIYI zZoV8%ZE44|n1xsOIan)c>Y@d1+r`i^Aa}_nbF^GCZ_0(PK#onW zbkR4Q7|U9b_KB$b+SQo2*6R@Z?=NeGGF)fa@s;1()eSx9ZG*8{gHJ5VU<~EMeo;+2 z#(r`T+usJPLTT08wmuw%g<-_i>MN?k%*=@WqigcK#8s^_tOkAG6Y;J_EP&#^?3WQ+ z*^`UtZPa|{YpG(&YBf5=9jW8k0*G>mDatGdEISSD#hWHZHHbRZR=G{-g4#=^Ilqov zB3e#MCZ1@Ho(Z4hpj@TH-sKg*7mZOz7H#6uoTixksK`+T@wiYPlI*w(!L56Mg--yP zPXQmE1Cn3BQLg%hFX3n-S!_=@!~N4u){3o8y;X+xi!X%7s{kI#<-RSv)LC)kk#Y-D zLu7OX=n~wssHVsfFlY|5Y-7uRI1Cw#6*zz$F|NVE{(3VpC2HwUq}gTr{$JlE(rH5O8A3l zOBh3tOZ85|DT2kvUPOs%BpMLpI+0G0R;<)wc7;_)j3(e>|NB7_r{n(ewA*sISn@Xv zuX8pHsSBnn<-r z+NteGZ6r0gJ*jz}Aoxu#;zQ9L(XS6+KI39Jud`VWMGrX@XPyFe9kz=XWoy&ao3+<4 zT&-oBx@{(B1UZGH{=nW;MPt;pALM;hex_X1XN^7x>4XW8o}LV7I|U%71DdA-CQC(_ zJrD3~Athg3EJt=e!Ohj|?~*RI9v&J z`=nPKPfNA5odxnb(jo9ani>L-1LbY_~H{UF&xLOYF zk8Pp5&LP2vt#ofiY`Nud@Ha4i_QgL_7dds zYrh8{QraW>$YOsQ?nR=$x05QF zjY?YlFaKJUnF;45o8R%Z-Yt z91f}y8YqrL0G_r6>}d}ulJhw|9WV+jBAzl|MAQPn1-u6_1-%lZexnWBVsHCIsviUI zpe1c;g0+Lc>UNJ~LU|>trhj;WM#|VVBcQx^7Iw2iZIzSyO=UeFk39|` zcN?uy|7=?d2Q8%+(Zh-x=IkI}y{Td!C3$l7Gl*Z-MT2nt1B7Wbn>fou=sV+JSat@u zb>;fwY6YxILa9Bf4uDjtL9KOYoYUzvt`mxl2r9*gKJ`aZ3>~YcgzaX-+h&4~MW_o;;Hc&!^yPImcb(LfnEjIm?B(+EY@k zscW?M0^V+~-LK%+a}RV&_QOcitGK(8q(%(lWq^8X(QGoeEv5waOxHIqqzp^6e?zju zH|7G1oszr8=&T#St8Pyrep(tPQNLX*%XcyFH4Df{T*KSsO>1exrj--@al$oO9myJwn=_0Hq+$H=4a@Qu1?}@C~Op%>n z&83xZ2|!w6E}gSRq+ky8k+0xdHAN)B>~7Td{@dqQaHE=EZun11b~pnnQ4PnsSA4 zoUv3+z4f>lq9_+2_s436p^(Yc-;rpOZLFxOwm!QgQLGz?ARZyh>8Tdp(d+3P^(Av{v66bv3?lIYez{%tclU!l!r4<_x$!;lT$D!+=$&fLy|)xx@!iDl1k z*$N6xxJM6*4)>uc--s_2#qYAwo!gLJ)AtJk?*0W<)9hkBd3JNip1g4e@>VS6yKc3@ zuX5v;oCRWZayDk=pH*PvB)JMqrRCE2WlTeHJ`pLLiP&SLFXe{qn`I^t4ZEU#5pg` z%u~5kT|S9mt#UV`CzGX?%dcxK$KrD3l6}l1vYdZ}?PnTG*)6^^TQtiRp%jmaF6!r@ z=5k_hBXLcB-&FS?jH1B|1JgyGms}yz>@dPrMUE6G+_qK1RD5kdo%3DkuJyuYkmW6zFsA zJYyF;0@ExKivSZJfSgzKkrJ39v5Q-G@sWjH_hp8iS3;7w4W-)co+mmJE!AhxWFMjt z(}`{`UmtCIx?CQ8j$H>LaWTJ0ArCdxvDfF4HEZ8eu6Y=H^}$&A_09NkROpcmbN_Z# zy_G3yJN{!T%Z;S^WH`4jYTbS;3c7t^fj*qog-FRxg+@~_J&ZD=Lr42^Iv73Qp*$CA zW$vU?EoVl2?ZlT)%#*!DZWql{bL6Uwyx@=a+tYj)7Oc3*5NF`N9YxHHm~8k<;| zH|e4DS7laLwSs;#W3b){X5EUuBg;ZW7b{>c*A`vTNHFVn1b|;sJG)hI8<@WMcmgT@RJ(cZ)|v)mgdJHQ+z zTqnFFNBBZsi9~%u0RfN5vX8PZ&xDk2v$G#smsAl+kK^uGE>`$_M=!1-pJ*nXvAkgL zaz6dl7j2m0P}CWQS8 zFy+AfLP!U@YbDbkKTo)>qQ(ATS1li+d5$K(<;#t6flj%bmog z_0(vT3(yeqYRZKlSHR+a)YwRm(t+Zg+%!~@3*2z5m8JO)j)Pi@E&FjYdy-=Fpq$0c zfOxi-bKZMwmqCt;P=Ea^ZQMCtw(7dJR)$)zyx!}bp(l=rN z*{wiU+1(SO7`!+6xrJ!4ms%#}k$WK@pgRN@ElszZG~KPB=y6Q8ehtXI4!@4240R>X+;|5am%z=}c4=RUmxaCoCP3VhnlkMX;NW9G9-S?0 zrMiTqROE_kNj*%L2y6g$p45jk(r-G2-z3sTLX?qGgR8QTTQwnij6qT-*#~1j4(K4* z=Kwf>MyOX?&8GMMY)!PsPjfK#sdTA#?b}G=b5I&kbBP9Lvu~So~}^td5xJL8-Z9WlBZ1_k+;{Ir%NFG&@C+HUres znV{sdy_I`8;WHbQH%3nMz6NMbSBdgW7xjkWruLOkMw5Q9nsoM%TPX8%W`V`m4H`w^ zmMNs9_snOgdUgKz+q}HlOLD{sL}U7@W_x^4^?|8~t~uukmoBD9bj=4}xQ5C8oD~lE zwIN_re;NA~hH?uqd?f5UOZ^!oWh*)iNMxF;lky+)q|4p?32Ms@T=o9tDSdg7BBOBW z)pZ@7A!Udhf8%M$=a~2aO}wm6x?%|GS3O&;9U7um8C(j9zRJP!tyG#JNbXS?CElu+ z?^CSs-z=ZERn8cMUJbi70_0Teh`R6NY@`qTM6`t+ug66vdl}9ldReGRyZ`>e(7ckS#!3imCIC!7U5N&T>hzFoW_9v*4YE z`!C#cA*mI0OB<|P>ujuJ@>2&TjYFoFvVZE}M2bJAK{3>REgCiwwX;~`;V6ozix8r^ z2~;Fo|rp9G07;j0Jz%2#J5vy=5`Dx$ZtnwhF+IDd-# zKi|CZ*Pc(i$;;OM*TH@cKUtz$!zu@Yb28MP&*|CgbB<>%iapa1Cp?{|K7TqrS+$fxn$e~M|mCLnJhWtzs< z%;=>_&yuJb2)8r^J1Mg?**b}?f%Levjwht7)8xKM3=Cv)()dYIk2Hm$Qln zh=??$SyC1T9nPk8wv$du>sTi>)IjceTGtcObJ99}ld_fYN{XR{OpcwJg>aQ9m`L^5 zDN|;rks^F0icV6!Eo8UYY3&rD5~T#GzB^^>>~!9XSc!6`)BuYPXYD#$NRO3t+#oe* zXNPt>{S@iR5|z_Z!z|>U+jZS3JyX)@vDC<&a^LK_KbKx`zmp`x&Op9{l%c6i)_qlj z45vzYZ7Jh%GDY{*>@wU86naYa*dnv?zIv4KN>i{*@7XR>QK>O2gJw9So9fDLsMIXW z2uxE9NH+_U-Cn6xmqDB2)6L6d_f~4RXGEqcrKb05mpxLc^DTo0@}17Q9Zpy3>Mk0c zro1fOGOWY-%FaU<(b{$CR%IQoRO*E-nvm9UclzM=4!0}yXDy=XC)0=O$~~^^QnqMn zn#!eg>oB?3m0jx=u^tcYM-Vp{ZM&1DMJ&P#XD8Ch2uDRl?R4)nK=e(J%#fYdF=;AqcRIB)dmaK=knbi7^h_x z8>qVocTfw@AX;c zpZ6MR4lK(t-BeR{eTrtWLx7*6XL@fd+0E}Y`9xT;Vqm&iknFY;t(^`bt%{N9eWuCo zc&~NBAxu?ibh`O+**z)Rtqu`>N)yui?v*|8Ui-d7RI$?3^nO=l52xt7cZg|Knw#GL zv+VKrIugfGs>+MfEi^lvO3_tyjQ3N{Paj~Bb>_ZqH|c~NRrM1VC6{&F zTQ#^n>sFe6>Y_>Q9gkEE`IdE0(j{Y2imuA(s-e2skJ7pM9IhQ`nO77YFE|rec zid8PBTL;O#O6j`OaYn1kt@Pn3*>CT6JtjR%zSF{oaU+VeKaT2lQF@MBrUR z-|epSyqr!S<7`^9fB1EOeQN%RPT%cqU(5Yz?f&c3LW!!BgPpRx@HbFC%}??vIM|!Z zcW5){a(Ypks+xm?m%O6Cq50{>CsaE-I3~-hv>A>#oh?x_bZ{z=SNAva%FG?AW@_ZT zB}c2?C?Yd|mYRi;%h{aH_Qpw>g>`B}ja;ASbgeg@ley%Znw^oGe6FFrNnYmCq3Ui% z?xwjt>P^ZsOJ=G28F{$n_O|a)ow=e;JZ6Rja&iaQ_iWBwHB@7q zk@p6LLA!dkJFz!R7CxsHh7IWT+-c2DjhVvtvBJn*y}miIYb+FgGK%&COy!-cEHtx) zzloyrE>m4+_LC(dz)8`4KyOpojaxKVJ`5O@=T+5vaKoninrlu5&QtVLGjnR#VxYD0 zWKe}-&{DIYhU!sTH7A3QD2A!^Noe4R+jlagT`_V>pA_fXR;|Msp`R6FKlDj+W=}tr z5vHj$dWw0m^KL)wa~a`Q`H2tA*UIkA(Y_QP;g_Ej*SA)7|5@#u@sTO{lOFUvCR-=3 z^DsWDI6u|D-}#1vb~-N{qPHu}XzO>o;c%MH2ZxyRN^|`Czg~Q_PUo9Z?DPEj_5FV= zKK@NdVmwN|Al=?VF{^$ke?S{&TClj@LNDvoEL~OOcsJ$T0Rv238g}aHNRLiXF5ER> zs7vE}-R{x}naWECSh~41nRf2^D50`od6i{IhjZgP_dPYHuAo$EV0_m3vd&iP6Wa?` zrw>fcy4c=%#QL${3jUF@%E)Tf)pJ@uPPcG&Fi*WQSgtb)`%%Xnj&x z;kNWayR)vf>qV`fFspEf)Zmj@?YjC2>yyh0_oNTLlyx&qKY9Jcy21k=2H$tNoui*R zdQxlQp|~M$JKR01KgV(M>yF3ThWt9iexL4`qO5Y#f2eZf1G_G{j#JE4&a@5f(#S!u z)G^gdrP+U&dE=84T~;|xO;)+sHf%)WGfCGf$7uyBm;9~08eiCT-Rd}fyUMjT>!`+8 zXq!<<>*7B))HHtFkIa5qK z%h`71>Fi&s2I@|kXH^Z2oKNR;Q8QWQwEV1koRRnG+!1P~>dqU^YE~HqZO98-+9$(# zcaHWAqo@t}vzGQ3E+=zzI~yl#C@5PxFvI0ij$XKN@`l2?r9*`4!<;UwjHhN6T~QmJ z>iXWTTbuFR%q5@IY(Kh6+zr%B7G)M|syn8-sk$5ao8)IMwNiKe=w{$ihS^9Zd%tOxw3Kj}IPbQS9mc++<6)9M{_iRpBWl@s&&}*%NRc_Bq z3FUqzN%7vb3WJ+^-b`4XQZnhG_c4W`3cVgCR1}w_#`|1Uuy*hDGGWcmk{J(u?kbGP z?e#HXZEMM#c;B}QHch>LB>eNfWWhtgzV|Ta&961rUJ9Jo>?>tv@4Q7>Yh6H4 zL9@TTnY;5=bFEF6f-0JMGV^z?_R^{j2;SZtTyGZcye(O)_EPYX=1{3Van9Qdv~~xC zoNo@d?=#-HX1mt@OCh(LdCr^STzg*YP(bMG=IHu9vz>Rm);e}6^haLo_daPocFM1; zcL-C=j~iy5(_@$E%7&9+dikSgn-}-k?Y6SXA>1r~%t7Ixs*RAy#MQsN9#)e6|oQV=dJ4hdE>G6 z(w`ZzAM+R7=>Kc+@y|McKgLSD($p-ZvrcI0N>3f7>XqSdp_El`r7QPolz~^~QVWf& zlR>&lQ{ybWvf3>4vrbLZ?erVE<9(F#+y|H?Hta0ZeiR?0T#!4! zGVx4nncnEplaz~^23RLHzAx+cX!HW*Vg*Zw#Ivfaj7BFEDKB%k^hj*7SjD+@t@83* z%YejYzg1?V$J8pXY_g0I&NyRDo?zeQttC;yiLa~3WKU#t`z76I3{^`Zyhjb^V2Jp z<(#QQI&O;_RCDHPU3sYVgi#&0=MLKCa_xM1_&_d7YPJsA|Mc4Za?at?E^vi%^vv~N zdZVP1GrV`I4L+6CuBktIYI3pnZvVmOvTj)EkNuQf>AiR9;LBMzgY+j%ow(C`f7{?& zS+}O?PkuD6APff8dYWzObpx5J^ilqZn-F%t`4D0pmab?Bw^{F8$=f)4~_w31u ziZbJ=i7FSi4jcUJ>HUg|^;2i4w7eK*^X!>qO{MX)ES0w2)-KPU8?4!|e%eZv%j2zm zp1rVJv&DG&CY7sOtwWx@9JOZq`sw>suD`I3dG;!8%}(PPXH;(V9-jE@b>*79>t|e3 zxix#C9a8o%D|cDrQmw3fFIMjmba_gA<38uKz+ z-W%F9G=A&S{c*{>uT-j=TqZ0N z6#c8r5*F90YBxKCyD3dH?004Hkyahy80V&3XlNOfb-tBr*zs;1YYYcp$-3RDpW!&g zP33~2bx`)})~+WVXS;QJV`y_F`$uv2AC76JsT&D5HslUg>-ExUE&Xidr^ENE`s%HDr_0QSvzm3y0 zi)Aj^8M(T@w+;SO%#ZeX1T^ z?K$H`zXuzSY$*Mz9{bXB!HfPo35TN zk0_Zi$9re*!A+ZPIIT_!oVeTjfWnX`J@4#YJu7hXNAKebLuGqCu&T%joMP$Is4%QY zuP1ve$^xgR`m`umJNJ5FwPs7;^lG1LFRbG>zuvHB@8uZ}eeN_3pWo}f=GxPjXBqiE zX&SMn*XQMHuUwuJ@B5}{lf4R zPu4HcELIF~NODu_Z4|tr;L1{m0Cx}fzP)>1-B59*BqPAz!^69`S@6c{Gtx~a{^P$ ze44g$5%fH0O?BXmmp+_A7jOQ!eC?&cxp}@{w%R<|{717=F(@O+PtD9Bc#HDYD(j#e zGymSz&JA1iR;*7ATA~=>Z00^_tEtw;>Y$Y`1LCT^Hf|lPwfSYx+LwXp)qWhLSGO1i zZ)^@)V-~ccI_PS3d~i*3@WJX(*=_MFwl53bofOhm9WiFxB(2)!;6q8FAF5-XY+InU zQ!(UZQkdGd_~7jYS9dvtGzxh(BAvj~@0;|yyyY}~t2`*Ko*&#Unx`)<+Nw{GQ? z@`#{qN#Qly7Vq1=^4iUasBIHw_ucVj--(cR^~li^l9$%(9<%?V_Dzq-qzMyu)$D(= z|E~7!yvWoEliF$yjXCgE`)+gOoC%XZ)Es+q;E(ow#i)!4DQdMR$JD9lJoJdl^_fWzu&_UDCC%dCcm8OpAcVXYb!8*@gMy-7{HNCdA#~~M;7iQ5L z&rMrXdwKDpfYO%{(cAK;@2ll{G3xrOjObmSGcNSI8GJbT`kRx{2R&!L>38?);d$5J ziI{rNSsnX73_em&`oTKpY@gY^cRXo0QeOILPE2c`Iqo}NI3KMp{c4A;C!sP^t*NJ^FH&p?)cJh>}KiDIkE5iENI&C-TC;-(%*+-zx7%8 zV#n`><3CFO{)m;Brzz}|ayg+`CN+GNym@->ow8?6=#@#&9i?iX;l5MBrQWPe_V6fO z^F`x#c05x*G>l8p?#h{qE!29RbkSAl6W8N><{Asll_!JR6-USQJ-_&j0CLr@kB;{~pF47ZsdmGL zcFpzi!RPbBclF6?*j=W5D?Vy_{_I`-JDfRLrmHb}bV0$20Rv~8Y1P&99G#S0xNFzo zr)Tc!cF7-|nq1Vj%Q~d-b$i!Sqi2~fd9iEcnZ|En-9L?9s9Y>-X+QX^e7NDjgly%d zJuIE~pVbXFo|>>kd6_f!Urjx4n5<7&aei5}rPsc6sqvVN z=a;Xs3|iS7bi-`)n40q|4qAqOZ5|bFUNUA+!OE+aku%Pvg!j8V=5Rsj7t7eM=hDJ0 zG!jo0lxYndJ>z^)XG_n-=Hyia_9Q+%UvXpLqQo|@a_@mjAs4pa7<4M}hS%ze11DX( zaOB1i>9G&IDhdathFm;RE7hY>>2F|#4@%D}3i^hKNT6L|=_HEyEb-!NdtE8AW+tT;#?0M}BcjFVrT-d&5 z(B75TuH1|mJz?U7nxlj2w68zD8MS`G^b56j_8rc;{;@pf)`WRAJHGEb-l1Kr9;Kh0 zS+leAey-uwSI0*t7u4(;w!g8%4WrcwtCN@4><-`01-<3!#9PS~HG5|7Z|QK;Vf8ru zi5qJ6uG-JleZcCZz=_op_wCw$?di=J{p9?KyC?2%+kZ3h)+GH&=O!MSc;LhSyH9T| z&`(jCbaG;y+JT3Ow~O>sJts9!JUHOM)2FxB>QBp`bZO!t{{t@*@6_tgI5+9$#KTh# zynTA-nEouK$qy$US$g1O;@$JNW;;%P(f8=qAz$|2ZI7Hgb@IE8#~KcN>v`{~{`_r| zzrH#4;J~j<_defR@M!Yyn&aOO{OxdGRHW&r$kd+bTqm7tl(HXKR%G2u=~8=Yc3mgAhYl4v`cq75Pp_)e$bJ}5krz3|ytd(BosQh2 z_=<#w+^Eyfew12KbZg3p+Q#p7-Q^xbV2-0Xw><({6bDAS*sPHMXg%ajx(RDMz0H` z=h_Awa=HGz%bN9br!7&r|D&4xIVYFSaF%eK{(6d&%@YMc1xcN6vhe5?ybl!;Mw8Et-iOCDzAlU2Gj&Fb%1!6P<9fZ>)OFvs8Q0$4 zj60mX`OW^W2VTvHzWer8`r(vbZ_ji+*k|U`x3{YfPuu+VTGzu9X1;xUr~dG)UhkfE zJ-Th?*SB}?44+r}?(>~vk7oY9c<=Y{G@bV{cTX74l5M%)bwpJVmTMH+Qy0|EEnyR=IvkT6^K}?4kXiy*R?1)SR^~KWE$Zf39$}%Jp;J+P2|y z-1@)heU#g(^0k-e&hhL2(*5XG*Duv;uO6Ng*8kP`qugU1T6_KHoKgK>=N{cPI(PEBH}yyBdj0#f+nsH5XS{p+;OLRke?Q;7`)Ka`mUq8L zp3wOwbMLE;plmVEgweOoYnbma_{l_c`I5zM%$b({cd&d>7#jT zT0YIUY1R4Rbnm(G{EaQ2*VtSs{SkEU<>>h}Enkk>wCns#xc7Sf{CzE7@7UZf{WuUSB^w;u#K1MD$zvKI`IGMJ{0Kk( zD*N~Df4<&YaC^tk*~j0>{WxR3x7=h`{DSXXMg{6mnf%+6p8u?F;3FSKzOAorbt9o#%(v$2|+NDO`=Vs8*LMHEonw#*fR2Y$NY9U*FLaR=Mk5ZbJ z-e0xD8N1FyrN@uzxHo->s@zjM-v0Zae?3=j&*!HTWjYU1m2~K=t2AcT=y7qA#*T@d z_P;+Iclw)*_U;4HJ&O+KN%egEnV*jPe|V4a|N296?aPGL<7NDi96wF?-w(-c&z1VT z&1uMN8zDE}Mzp8dh*#-0Vp_J17*l2=-1pcBvllj^Y^;1wF|; zay}MdA<~eJ3@kz>79$I}$U{DspcqS0f?r5e;c?j1$yS_IwH2Wnwj$izR#;owis2Z6 zE07Ga6;co=sI(P@c(u`1yhhF*rfIMhQ&jaN+pgP+?WjR5c3>xN;_)3@k^0qEOa-4t z6-F?GF-$Nw(oU420&B4?-cHmYC&5nSPPG%87TO87Ov+2_L>`82w-fVr*$K&BJ8|TY zo$x$kC%p0Njh*SB9w4SylYvKLa2!LS^A;Q$YmmfMRp=v`qi+&9{bqzm>!&PY!p4+SV< zN{hWn#Z)L!pGGQ^GurIMOynEuNiu)fi^V9QT!$l(Tcmc2Q7(%NR@IB(=Y?; zRUE`7ykYvy$~kbN>;hM~!5tp(gcrQw17G;TAHfJgD8dkq2*hCw5-}FzP{1-K zkP{|2iDXQ~G(5v|v`lmoRZMf2yoZN)g0J`&->_m5?V3!x;DxLdCy|4E6kz5QCov1z z$U#yn%R)PD;QCZ2(Zn*^$s4$ZbJWk{6=lU~Y)^EY&bCA+s6q|*@B{~EJBdTcp3nX> zOi!{9X$V+MA0P^u)E6TQ*~mdI#xgFSoJ`KeT5QEp97jD)p#e|v3EElgYuU67N>GLh zI$|61>S(PeQHL5n6wz1sif{M>O@?bgaS8hy4A6yoU$Q@j!vj%>L=q;T3{UYKFYp>~ z(Ye@3^yTq(wnMhBp0{-Z=!b!}*a>i{ZYp@3&&}*fWFvZC-_5plmnzU6; zA`kB{soY7VpaMCSPNH(So@CNG&M&CKPwGGL8+MzVgfmm@D$H5sL4r`x3TYB;#juT zlVn_FyOC?jby$xL*o4if#x~So2X^BC4&gXX;3Q6A1oLSlFGKAb=PY!G33{L>dchQC zFh_q_AQ59R9?6)9NtlfLuxNJ@17L}Pu)D=M23H<9iEFqHaRF2PoyBx?cG8pRVsW6e$U-)9kc&LzqX0{=E7)1=#vWWqbQTwJ1y|9I8@P#E zxQ#owi+i|_2Y8I9c!uYA0r|1coYS2-uR3#Hb>@8PEF7jfiE}{-cn_a|jU!I3vx`<><#3ZD= zauHLIim7P)N!iobJRg%lZ>;VMM7tN3S$tMDs#6=OHJil{Bj^MI>Ze92WDYIhaypSuc!&#uA= z@feLUVS1ANKU~ED)Zriw;Rr&0x{6TjVw&A>l5-QzaDgk_;0_OX!VBKm!?eESUeb>Y zKp=t;j1Yt(4AF=|EJk56Qm_vu@@`^_qMO)?YHY)H)SwnyRoz6mo|}loI(;{>$iPjg z8M_GsOvCtIZXyX2Fzvr`l$-ci=qBc^a1%c3X-|!t=)B8K=+?Ulql<2$LE2q3VuGT( za8h>{UwXNVgM;0LZ>+nRj1-I?&+`nKim8}}>6n4pP>6FEico?w)W*0Ab@ZFzE`snH z`)9g~RkPg1HdM`a7whl>2j@`FaTniUUqTzO3v-k7B)wO-i)OSTllo$0VcAM|QG(@I zft4sl8CGFC{7T(L97f@LnY;LbpZJB}_=CTYta2Ap5RirpWYGa~kcR>kp$UC-L05Ex z;Xm%e2xi;dMIWSJq<@n2B$|)ixg~ZN+R%JL8QQq<&RxVyd5D!AJj8Gn4>1xpNbKh! z#$p_<*m{T~9v-3r8$3NkV7P|}f=;TQiKmh7v5t5gdizP)|`cmuo8@Pw@m#aV6MO+(7F* zJ;@!ijl6`*xJ&&S`I7vE>y+DZ17E4%B)^fj$y@kCSrXzYq#+=W2Mm7*7s{Ua2@&ck zJ~B+2?0`>{RY)~_ru+q3lr^CZ9q2+2`uNKDE@W5yOSwB~h#u$-GnnHf)WVn#qA&@Q z@dQ@kp28Lma6$n_M$oQE+Jmn!ov$b96y?b|$5TvzQ8ayy0{AoT85uynBwyh*0;zu_ zKOup+5$~3WMN^-}np37*8Pu0cmuE3OXSLIru)-Q^Zg36z^tuiudrD z=_!WJ_7r0?JjK3TPjL_z@Esis*guPCPYL@KHtz5go3IT#un+rj0Co6O$GW1ZiEWy$ zC#k-~wAc=f%bwy84&w-p;_Ver;r`rHc;K=0gJn5>uBrs5^GqXxCuj~29| z4Z}Kl32O|89qbW`gIZqV5Dw!A(saFqc4sf4+RICH9pEKKBLR9=UgE7abz3hH?c^oO zT)f08Eb#W?`P_@=axWq9aDkV|Tg-fskmV%`*LaD))n4M&elL-Dz)L8c^b#l9yhJ@3 zZ+i)!?_Od&=Kl5)aeusoZjiUgjqw(Fh)D7lJMdt#w|IodI6Bo^)Xel2LuPr49t*vN zSmrIXuztI@xWCg|q~G@z8CZl&EJhZxk%L_1!K+wL;t6l$Q!hXve5eScuDC zy~Wg@Y&U5iu}0ZPti?Y#sOBTmm+DF4y7-8Aj79>+AQ9s*9!Z#hWK6^)OvX&i!fd1? z114R4ggN?QAr!m&2nBRP7j#88bcX@@pbeLB8CP%>?YM!PC^hgAKXK2*M`-o-5!%qf zMJpfi13xi!kdK&#>6nXon2$pkHjHH;c{pt!=_3s7e1s8(--DuJF6@ORrT5)$@`rB7@geo+k z32o>=7oDL8eRM%rbVGL-zzBvgh6#G0CwjpGBQO#P7>jWjk0eY$GA3dYCS!yF)58X~ zu!B7g;2;j+F#g~#qzrw9fHY(viw=;3laa3oLpUN3i6}%P2C*20IK*QtCL^r3uNZ@| zh#BlFW*~Sd!x6BC*Wz4##RatD60X3{%~xo7(|6E;E-WYdiuE&m#RlwL<||B3_=?`= zeTDaDU+ydX#9wzmA@T4NQV@`a4C+1oM4^|Tm^<4~%*O&OL>khOfkjx1Y?PoZ%}=aC zC9*gAiTqoB!uPhH(0%SF9=`GuYOno-E$ra{N4UTp9`J-0eBcW|_#*&;2tqJI5Q@)G zf8!^M?W#-v!57>VX%hT4?ocd=ID!Azx_mww7)P{^cTl*0`)kFQ#cJx z6@M{8(_cus_zQt9#{S}hnZNj9;V*u|!pfg*;V%l^{Y8e4zv$rWFCO{%izi4LMLp49 z$Uz=Il6VZvQ~bH#Wq$J*p5-qzaONL>(RG)eq>(&}CUm3TOrAq`%IC=z7*K8{4arNS z5qX)siYvH|cHG1b+{P^!Go1-~;12b>xQCw9ACUL)5RdQ}y%_$4e2Qmyf#-OMH+Y5D zc#C&1W%~Ez2YkdQe8FdY#lQHDANYyi_=Vm)&J2=De<1|{X~;kp9UupJC_oWPP=+eh zp#xoXhCaHWE4sk|Mlgm6dY~7~&>MZw7yZy517L}P7zA_LW<%P-4)$qn2Z!m!7R*$18tj6F2F^! zz>#5XF0h}rf7T)3=-s43Z`OsfDzh|A0 zx<^kENk$oi?chU3A72q+;YJe_;b##DDV_33%|6eFu-w=Qs1h zB4lDQvQUm?(gC6b%drAZvH@a7hXC;sukZurngPNBTXX`%Htg#jAok+`>TnQ;a2Q8$ z6vuEJCs2=*IEB+_z!@|`!!SVTK_6XUZ4@9zz`aj^NHPx)6W|mYAY5?g0IzMu28e9r zqX2~{!zz?xH7c+M)i{l!_yDm4#aN1@(E(yuHtSLlAY2OrL@2`Gu_8bOtPK!>==)EA z7`u@+Z4D67X9GlzeO$#r5lPM?qsaMWG`WC`As3RdWEwe&j6*u*3@kz> z79$JUh-dt0V+Z!Y{g|HQIC%o~ zIEhm@jRtry-VvNKsq6q@?tUz*^oB~6jP9j>qfMv zN1*6#9w-c83p>~&yg&OiBJp4lZNN8Z4-OPMIO7l~x;h66RUi7>kNtaApy;wVQ0V87 zs9wgtg6*h5Ef&=4Np_Gsu?xGg2Yay(`*8pVQHMh~j3YRTV>pfzsK+UsMH8BF4i|79 z7tw-N+{A6%!X4bheLTQJJi-gS#4EhU8@$Ck{K9YiK~4$BA#!1{f;!q*uU>1|ZW|eQ zlGhc<&(PTvD25;waTtpvl%fh-u^q9SIR?>$bN`jMa9pAa@mmAMXq?!=I^!8WLTP89 z5c?Q+mi@PxeH5Kf^Er!4fnp%6Fc^ang&4#l4)NIggk$;z`-4W1;14%=%_&G24Gj`I z!-B-$CO#vR8zc_lFpl6TKIR9BPxy>4xVI`u{3;I;zhSaANO-Id5=)x(B*j>Y!`pc* zj^Y@O;{@t)63dw86nPpAIDxZD(D1Ns6ic?(0~@Sp#xoXMnCk21qQ$p17U?h7>pqp ziea$EaE!o6*uWNcu!jR2;RI)Rz#X3Of;W5+gIJ709O5w=C0K@gcz}m^gnqTO4G-`X z$~)+9{DovE`@rrXv3@V>xsQDlmvIZXF?)ZINXKQ!AEUpZh-6HJ<~iPnY@oexfh*kL z4i9*u$C)6}3q4_q-Y`QSNL^;W=mb@$K^yj{zst79zv%RYdA?vd?}9`C3_das^uaM4 z#|hNqB+j4-XYm{#@Gp9Pr!7eSN&8`PfzNu7laYccNX1l4!*tBROw7V;%)wmD!vZYC zB4lDQvXG4&GmOAwsWnh!|%UBIL|N#B0kC@zy0o{EZJ0 zt+PXf>;pb0rx7Y+i z6S4<-q8ChIhTiCdzA#5W^oIooz!C#tg+UmMAsC8bu*PtVfDJ~%7DHI39cd2-45jW! zI>8yksJoIbaDzKM;0Z7Iz#G2sgFgZgh#&+b1fd8+I3f^f@$euzGe z4-?}s9+NQz)Bf|8dlzycO7I8&j%J&UW!dAx#CP-_&;BzxOeoI@6EE=!ud!+YeU?v~ z(S|E{3+)2>4d+q6G)$aCpIsam583{oc%SzP^TiMR#4r5DA4orCSx8{IiO<;Xn2Z$2 zJZC&)vE(Dm#_7*tq5-Xc!$ge;7tiq$ukaNsWx~ZOs40hw;kx0%1+H*|2Rz}6AOs^) zFI?=$0gU{rC(-E|E;>UG`sjkL=mrCHhav1t!i6pDQEL}2LhQrE3dlQ#3rDALksB8- z@^Jcxo}_>*L=lRy1WVDtxMgGsmSY80q8w{ci7KqadThW(Y{C|7MKx-%13R%BbvTGa zIE*7W3d4kO-pdUa78nRCY{52cNB1$|LSuZm(1aG=&^~SQE%^@bp+j94o$-b)O(V>V2a)_Lm!xp$J1b zA`pcb#9|cUkchDuha^luGA3dYCL;w?kcz3uL>97f2;~#l4$1Tj`s41za6Zc%F8<*4 zB(~i=wi`a+BR=5^{>3jy=Ci)ptS{W*0WSm~1Yrn=Qc<{2L3cPIpqTYW5Q0&2*?7kS7>0SZwB zuXAhz_`nao@W-i}9Ajv}b$r3rTjAmXY94TGAh}0`aJPsMo~R!gAyVxlgs#w+IJiWJ z&ZHh3DLX-*@+sLSD*BQ_EJ5+=NU@Y$ z_8-bCuo9(MgSG$F8COMCVm&sZlwq66EvUv;lrgN9+<~3gh27YTRgBw5)}fs8LGmyT zVKwz5kwOQZaRT*ty_R*yTfD=2G;E3#XV8eV zXvR65#|2zO3tG{J%eaDTxQ>T-gvWS-zmROEAF&VhWI#Tq88T!DeF;e*7ixj~KK`3TDq5uCx3L7}}h!W0dnG_{jk(U}J^05TPSc+v> zjulu5>se7^I7Z^w+$eD>BTD?rj}pJ}2j**{L@cIM^Ek|}jS@<`qC`jZejO#ukkw6J z(uXuhU-X9svKc;r%ponwfw00L48dRw#V}Z71cqZIY>><2@?cBZ4)$<>Bb?w27r4R= z?(l#oyx@&|9`8dIkiMiJ{1Jda1R)q92t^pe5s3&yAsR7=MIrMUMaCf>qmh6yNW@r- z!+1+4-(PdM#=!!$t(c&;1Ac7E#5QHKO;fO#aq7Z{oh(kOQ zkchDuhw(_l1SDf3CSw+|k%L_1As+=OL=l!?8A`Am=}*}gun2`H!V(mt%WJk7x}iJF zV2-(%hxy1vr?>Pw)SwOB5A-E`{-qDF5Bu@w2lZd{2P&mwL=`S6#E5347;z5w)nddW zJjRe7F=Ci?jIhQ*hZrFp5F_Nr#fX8pJ~>9TBYs_s=)6BhoIf2SE}-A}7%>1VF2#r~ z=&lnhmYT+jWmt}s0kOhtaIENqr*?3P70NELq9!<2Obw3}brG?mM{=zAm=Y@n{}U^Q zU^p~4#|mv++8HY@;|i`qvo2O>A+<4<&+EmC1H<$s{S-%u{;s#;o-uHg*x7N*CzwER3+4K3I z|NifNlCyzkpaKg25o$^jej+X7W^YzOWEaI1lXz!$&?;5a~qbM}A( zFcUZpVP}A5;4E+sXaOAITr0S3z~e13^&Em*7gl4F=A^aSPB2v;!A_^S~wGB47YIfKH$b z=mENcK41VC1TF(tfh)i@UwyixMt~P9!%5c5Z~;Id5CmLHmf@}g z)3(F8RH$c~40k94^2&qz;xzOh&;XnQ?gK4Y4DNt$3~uLw7+eaF3Zwz)z%F1npayb* zJfHy30{ehspaKY59D|br9|B=OI1m9O0b78>z*_QTa{&0j2Y>*$@lgzJcvTGUjdUjR4mg$*gZn$U zNHEN4;J6vU!7%~A11;buf@=V`6EJ~eQ=kuyF>p!1133N;7zQ2!W-6#7U;$VH6u=tr z1m*(Xz%sy69fM;7UjyF&w}EegN5ErX99W$jgZmUv0(2Kow98Gyz?}4d5p5Q*{jPci=T}T6Db*s!Ic90 zfij>Rr~oQ~vw-9|v;kO;i^XjKG*~R|giS2&BybkEY#)m&_Kn4j`NiS}w#MR?Dr0fW zfXvcZ+@_jX+-6|G$yl6ITP%(VSo8u>Fdl$w3GOs-r-4fW*9x!(rUN#BEno-C0H}aH z-~h}79D&)u9DoMU0S4dKi@+to?`bTKJpyF|9Ka2jHx`RK z^(+>58u%Lc29Q0E#eFpqi(5^M!-Y?a!$klysBySFW*odD;y-P?FdtzNjHjpso3CZ1-YSIje*+?CHv!6(ry`xOGA>6m<03qFN5ZA7uqU^*$EGme+j zC|HI_h>;a^K0!sZVr2vvFb9QG3PDS=mSzMCNgRc93!$7gJw9Wpf#j=j9Vh5$HmZzp zAvr|B<`eZaTYbg~136mZmO?y9v+K`TDKtw|c(f2(XfsAKRvXOH6m!Rk1{xL1j1ijW zDtLU8ewsZibFIO=QsJFqGDLHbW+n(NniTU|Om5R=#%FFcShOiV7&jTFIjS;~g_eT~ z0pE0lHcOwm-C%iF;g@3if;PK9GgUZkOtG-VbeuM4By+c6+N5IfxGB6Z98)TUf9$7# z(djH@mVx4s;h%z0=nSb+Ewtif1himOx>LL|-(cmN5j2j`=uDNeNN63B5zHsC=+1g& zslhrrV@V2$OLys4RtTpjW-M(X@#(H3${NG;w2TkONkTdc%c>LF>~vvD&yor`662&v@E7{0lR&SPcw7^n`J zaVh3nx|cM2Kxog&T-#z^PWO(_zG|@d&HQ}aTu1j&W#14wgk&c0E%fwx`s`Z}p zDHbQ`^ZT>!2xlf{Zfvn=p?@%vecv!MEpzj@g@Mk;avlmDb2F3qmi=@AE9Z&9u`+XO zisca9SDG^_oYj=My~Xl2-7h}pnPFC2=FV};Vfq49&P(C!!OT?tv=RD3eaBsJdzu4BrYmgy;s4@cAsJ6tD~@KPu_BMi$GiCE;UI=+pH5zfjD=wLZy)u-5K z84=RlU=f>>)zD&7&R8CwyR?Jtn{{N|M#qR$<%WwmAz6)lTRmfiK6gb2CpznBitS0p zNBz0*Lh{6{V=cBVjE_fh;T_~@StrJA4U8x(FGl2^n{|qB*UwnV%3IsvUYT_!#cqfZ zEzL_1c{F7;x7gigd=j6xvBRS+>)g2AFk_V}FInU{nAOUkF~V4_&)eSNc{i&)WyTA} zr~P@UqPb&P7g}bFGd>&1+ubpDGV9X+(?$szwTQ>b?rEV?o#Ns(`5ipp?7nd-&1sEF zQzY^V$sXX_!}|;Mn$ixh=*Mx9r+6&C zPUMrDeS_~HbxL66H+1+^X8$e4AeMH%FHCEP5t?;q7TNh@3+iMaoRkRf1%@p$?ON?Gv!W6 zSV4z~PtJMBcT_ngvkH1T_zpRbQyjHUTcia8A^|7oNsD8-)7JQcs~rO0oTuZCI;U-_ zf*T^=kepHeEWOiqeZj2`-{_nlQ)ZoX+Ri7i_^}Lg8Lo+X>SDn?0%R5C)XDVgR-WN`L`U@vSi^p=_w#*rK+B;JCwqx;R&fmx9AWS)?Ma4q03Kh^W zrh=s(0&RI0RAVsVIyB%rgHS+E^K zr#M<=mP+R`v-`F1dgnxyMJt`p%o)+b8=ccs( z>8Xq`X0Eh|Ee>c>*|svGn0fI<9-RSgsu>du8B?Pw;)w$XRrUg>cxJx7XkKUFUDeD~ zrzB=Ue~~~OG^U!>>XgDP94T7Z88oSyGvOp>YO#GHv6QT)3z#Zq5o=#Sr_@31l*-gH z_eu8!i-S38=T>Gpvp9a=(#~LCwd({^$1G9p3m1oksM!K%J+oB5Z$)QFwAwAz`6P3H z|Gt&tC5dW}R_7LG*~q@tolDZxb0?e)%yO(aMjV=}<_TQ-nH8+!wVk1rYVTB+A!ent zI6=I$Nj2~oyF8T>=G-JLZ4|F) z%3aaQiE=&~UwW)_MO*I26C9cI7pl@z;*SP%R|?$Xosa2Dn>#TbPC9?tzyFT-lf=A@tsX7T=SKG5 z@BAb!Z}WtQ!MO!1dnjI&o0lx`?00Tul|AWPRhhRn)pN+XO%>%3g{;9n4D=%-uWUd_iCKs`JykdFiQhUpQavFPjj5HkP-$b?&(H zrIE6?ou5tS?VXs5xEQc<)L%x{CDK6JXDi(Hq zKB=jl@Rqv_VU;5PcyfN7z(?hBgH;*O74MK=pX#G^xhbs-_D|sCH?;bcyZkM_a%oqB zZ~l=9ADzors>*Qx#E|?(!92aoEq&#RuEgm4qp9;wx_sSVxzc}qV*at#c`YvAj8v}f zTA!AGVq%`bPI{ zj8yIJ+B}(mY2pLKbr`Ew_$QGIIs|;o^&zV|t1HQ&pevP6aeXAMR{JM&3VK@kRM*Gx z)%jijxSN6hmXLx0fq>=uL|YA>tX$99O z1VYyltftO?TW-M(fv?nclvUHvwXL$?Z>hdvu4B@gM*r?g;$iU7zV|n!9$~Ex4EJm*o1qzoylH=UBo0R=*V2pGRsgbnToh zcrf87cYT2!=c9>Ew2;D4!9u<3EB%36U1`yUKc+4`>H2&Bfjj=`iG|Nv7q+;*9yxHo zD?P37=ZS>|*Kw@&q5rPj!k2sx*8tFAqF3#U>Szi@rmUpwKycdYPj>*8_O zzmL?u?bGzZMSX=B3uWtw-Es#lK21nr;evG}iGrggwh5^$e1guR zTj8rUofOho1hvjek`baM`HEO9Vx!KsJ0n_amL}q|Oa^rJlFUS{MVpAvG9A^;>ds8l zPMZ`7Ss1B~E>Y%ct$f8&7KvTw+^wwCPEQkuvB<%7Y)Mv=*0xO?#WG8%^XSfM)6SR_ z%UI^>I-VqZP;2k&AJ4LAtee-JeOEg(%|D4{IZ!8%8N|8=z;|HXdBjof}=`mKJc5WjAner6e!0$fGTwg*9XJ;Og$Yw4%9_0R|S8R39VJ zUR&U8*35+ZjotZeMITHC4znE9^~sWg z!6Jcg&9(LfACYA%#Rs z-Y4>vVr)A5P*%6rVV{4Rl)`2NA5u$-IQs(Hq*S(3!lC@`BHw*MlTsR+sXkOB*%z`e z*f*HPc5XaW+PyD&-;%UoF56|`P=%y8ao^ImU_RS*^iWNAaoWBQCxeA-7OA05Qj)tb z-1k51Ou*&@H#AC0oA#|}3yEU8B{UrCE^XWQ@nndM?XGS(CD}i?Z>8^&c(zAlLv#23 zyZb&#Tav`~9B62jl#T6M-L@o!J$JO>LU-BZzRxC?$k|-d;SNbTxj4o*RK@185BGGJ zI~2#Ig=*Pe!G{MV6`bO=ZK36C?}WowyDNN)Kc5WMv3=BsZ%8UbiW7X7>e=%e58vwk z4|@?5Z){uI!v0|N@cr(pwBpT^OATy3>BvJ#b#8I8@3MZjfPLgicXeg)*0g0qY~SD` zqmr7Y;_YqA_TFauB^-IyUDH;)b8^`*dx84MOUZ%3;#A)cN7xG+kG$$WaJM);?ZX%B zMFU4BB(-D3yW2h-XD=Q-^0vEnvUu<0hX_YV(xU-7a*4t>4C9E{dSZ{xp(HacjKUEI z>q!B1oRX}zFe=ACL2uDh=Ub998Ajts)OxFcgCQkq-*6Tupiytzb1=FjFD;zQ2^`Sd z2h=B)C54mWLXMQwNDnxaTT{LoWs?yyPN=$(7jSs6q{4T3JZEWR#@1DZlg8+}*kIV&2QR`fJQmmW=9agy`VK-0>A zqlu-*+E%o1J|1ma-E%ap^u*)}11E}fG$!DS+|pCNAN6xqvX8Fq`J%G)Oxj07oao@A z2?58NN}Jn0y3P3{;poPmV{N79CO;bHtWqCM4mdto+Uoo92xoQU(d|9Q@0PZwef)y+ z>A=y{fD>b-7ur4^=X^GLba&5*$^+JCDpI?8Q*!m(pL&29U?nT(dXZBQRO74YTY{yV;(#Jg>5Jl5Rv z<=y@F(mqLY+ca>jHQ?OX{`+m8q_}M!J$9kz+~ocTlb^`ll1Rrp0$Rvr4}Dju+>+VH zdwNGpRTXflvX{%4V?HD+IC!jsC>{;9D7Pp)4fA(X~8E(11>a`y=wdPwp)6_iDx|*+R9!}emd;7OMT*Hz{SC` z3E$5~+;%sfc-3?9ZrN1YXD{6L44jwo*{7U)yDQ75r^kl5=LVl*2lh0T+qTC>x#uOE^62epE1xkH zD|6SVPw@hK2g~jK;^N)&8&A#a?Y&z*Gd(WJyqr4sa@*+t-x47bl!v+B@J|;X1WO=U$>d z9UeFsQo;6Ht9LJLJiVfKFuKAmeeFs2{R5|023}6A@MvG#;$Aj-dUfyRw2HY?YYpz@ zq%$#rS8^+Oe(U<(E7)h&_Fk#1@J?Sh`)l|8nJu%ATXhQR`-dk;z zA5SI9Jibsjp9=hXuyUo}`go6Hjm^!yU*E0#Bz=97$MJ#Y*1&JZDp$9!Pw_Z0+I*q+ zo5{-0rq;_nPLjUt2)s?Mit*c^@;Jr*vZwd9LseY*2Cc{G;4cRP?{KQtwr?o+IFs<@ z)!sY4Ri967(0QCye|aPDZb(&v-$uPhbK{q{dhbS8txw;0(&Nj4FYg52ORU=1zOlvQ z+~}A0d+()HZJyd_@Ms~Odl>j_ZdJ11roH_ht?YA8dcUo#+M2#;$fGUz+-TtarmF4j zn{IowC!BlMd%vw}=hUWQkMrttF9W|DtV;FUJmPVo@!YH4@9tKmr*D4YadF_>MBsz5 zs@?6I$2~5Mo_pK-V6tlO)MmufKx#pQhRM|mza-4FgWW>x8+NG9Oi!YCb_TbQf*x|J zv)YrWo?Qto7JUzWt8=E3XrA5b7OS8~A=PTXWR_=7V~cIyqv-0q^klAQ??8)v(Bs7F z{PtwNXWwYctiH!-)rC{ZLeG9uD?RA@-0C8~EmF?`cB^yW_m$Pf>081)2ZLMLK~I{h zOWU_Zd0tLv_2_%jR$Vr=Mdo=$-O3C4VX(TwZ)?2g)yCF&eLvi-u1epUISo!>T<=M8pSK;MW%O?~<{t>?|)w&0*q zPEAAmwsOzECA2N=8}+R@GPOtd^n!e#!!HG!`;dJ;tM7%wfv)rv z%G^i6=hZ$Q>b$vC!Ejkd+B?iZ|cAJ75_oM4<5MVmmWX&S>uJ~zTfX2xR;)u zH23+yh1Q_gV+ZcHr>D&QdGx}CzSoln9!#an=e{6a>U(#$b}D`Ei@EOxE=>geeXRCv``+=ne;>W{w(swgwSS-5i*OOL0hJ0El>wxYm?g^d zt*`SqpVh*hG1j@d-z>du?wc$FmrCw}jpg#{cnh-ox%Ql{wf*K*b>6$Ohqw+QU9ijC z(Ykr(vu|@}CU$M?w`i~X;7#^0*D<#Xww$|MCs>d(!kyLBwY}f+UY*~roEO~LgI%!y z+>dn&&*zMD=Ztmj?w>YQxA;vC!lRM96;g`XLD2#g#-nq(v-&AB5Bl#?QFx4yZnf0P z?O?!p6_w|d*qz^R<##aXjf%!&=5`lJt(P1OUZ7_2oSP)Z=YPhJ%h0I;*jxek=jI9~ zTh}IU+P={iXZN+lt~(QNcQ+I7BJapYU4B0Xn=t%`M~MG&?*uK*_7^nNmhpb$1Qp6B zKq;aKe-R#B|K(ArDfVHcKWb;bY~}I*bQ%^M5Cz`>k|5YeerdqcD0Buv98H{tiIxVC zriHDHGJ#~vM9U?ja3UN91}}>cp=@Ls=_ALmCbV4ET65IDD8`&*&`<)Y>y%bv8eJO`h&uhYo-C#*G+Br+M!B^g z397cG#NliF1F_660;IwS$2#y zN{SeZA&3I)bHCsPMuo5&zG6Ss9K41i)e%=jAqx0MLl^r9E)1^;yGH+B@B!6Hd5F?4 zcRttJc>h{MjjS& z?Cdvpsd1XRl8BH-uWP~hI<6%(PMJC65NAHWH>|0&zS=D?%g#jB(dv_7kC=1GbOM1J zNc&zZf}EhD7-Dm#H)WlM*Nfqmzx85qa$c{@wZ=P;YAAwtL}dGH|B8y&@Itv9{g!>< zA?$4kL85al)M-Z&*!lH8HSz4nQjBC`|0YA7M3OqN%F*jjoL zg^v{_^ySVIdsB>cG~DBHfv*0z(%$mf|*X>zUi)nvREiDUD`VlFCA zx)yeP-mOaCBU!&*VPDr+T|ssEl)E6rVijYJ`K=Ovx_jrch!Eb9o8eC>VlmZ2BcwRc zDPFx;dW?pG!<3q$K<|y3h{5HX}MP#hFlEjUy$C2SD8p0~sPs_K6oh)Yg zMEQPS>J^Bsp~Mx~$7zPG=~({!Y&u@U+@2PC5K7OLi8up{;S>riCb08C&C& z55J{8r1=D5*YbJBujJX(UKo^Qocx>LB02#N!OP@6B@Wk!#37L>jc_ygxWzXJ=7e}> zr*SCtB@Csx-4KdrmuCAW61JyVc z0@*qc%v2?Xf`i`RRXbf5Py*3I8msCE3OPKq(FeyBvA;R6<*NQ=Nks1XBtuF_zLf?s z$nl8Ke<5^NA}uSB-B991rN-IJD~X6>t-TT++;}iB{@A>R67q5LC(&D$p$)m@T$`sV zUOwxVcB`pQBn@k#f{Gcfo+qZ_S{)YpHc(Y#fdQ7i`6w}Sw>^C zhAUAaR4#%_w!-T{p5ipTKr{}zJR+`|(kWzVbR~=!xQ@mXong*!6via)l_|N|IJtJ7 ziEq4yg3eE>W16RTQG$8%u+X%K*Bj+z{`+jc^Q?J+R20#H z6wB*wqNa6UeZAbRb>1qs6BppW$1=^!0?rW@myF8HDcLyPy!urMV`q7hn3MToB;+YV z5F?}cB6IXfi%l<4On=;SYz_NP`ha<##iKn_u1)57Q8XNNU3!3^PNwSqew0 zxAv~YEVw4vL&uLSsn?a$$wGZeQm$8%etCqwd3Ak3#P^f{EZNdrJZLXhgqsMMY+Etb zSN#i_y6$AxP5aN42)wGiw0@8s#0DLkb!y)=bXX2bcCSq9K$ zmbj-jateRg`~xx7vc#*K+Ep^$1gme|dQE_6cvr#_Sx?2ZHRc(XRHVZDD-@}_iBLn> z{(<~;B@t2<1EDO7!iY*q6h_|`h&|RIo%TVjp7M~VtcGhd<6szrdr%g--cw%}iaI0p zm8p%h;zGioQkUmOj8)$(Vdv`oZITgPrX{_QvfmnWMYiUq&9#+X3D25!I8^ZD&=%^< z5O&#D)ZX&Ly)-;+TlSLc4JCKO5|AHmhRMvi2i9-YKCO&MtloN!omGv$EzRX`HBTkT zUz;Najg{DeSdF(N;gw#Um|5Lcimwi#^_=G*mCJw9 z+_;9ZHA&VWOt+{`%%WtmvEAR<*O%}*`E^C%2={5VtkpeF48L(^dK+J zW^3g7<25gPQ(}}}n(&`%KKOJivBWA=Fl-Yym>N_gk_vy)#eEvwKwW-C5ND2%sV3Fl zmej-Lak=3kHTVo(gJx|Y&p?$A3F5fgf^ZI8vxqniS}=37GVs9ikY>{m(&q-mlsa6F z8mOQduY^595JNZtCj+%FkGO^!D9g=xQp8dWsRMiJM7~uB)}f|!uD`}=fOjgg4~JlN2n8+_ z3@Av9IYmjL6A&SrCNPz!!tjlVU0a(e{CM*uiuGX|uLvB%YA8v3wy+_@?TBVe2>VG5 zk`ZCh@=Rjbvy{u6km~R&eoqfYgxKRtmiNkrHL}pKhU#r{hbGECYbu=pquE0nHRs+SWKYol4wZLnvdQ(OmZ79BU4H}8|vE~QTWGonIh|Qjke8u&*OYv?@Xlz?WH1e==Qjx%prRxLW5CRQf&M_r}^}fV~h$n4&nBKbp`R* zR@|?)E?tPMn(5SV$OdPsz*EwULw(q0?)F|7%#?^Yq>&rQ4d$-Xpm6hNhhJ^B>Wz;v zN20`tf(L&VRq`Cf2F-9ZSV(e&++{(~x;e3ISA~UuZHKtD3>$5B^S^BNjWJu+uif^C z&HkN--Jpkyytx2%d43vV{j<^gFAa9S?SfES!h3Vwaf8u{j!WM9SyIA=4KZ8Z`?me% z1DiV6)b^dpKikygxPOa4CI5R;$fpPYV&k6%HcCbC=ptY~0B`l9v&!Uxea$(GW+7OcRx@=Q8ouE{LgufpQ^A2+)Kg zra`9I0t6o;1MfZ*zPtdqI(YPJ6d@KxQV?Pg1-?*2D1u&nkzhv0hSJdth+y>*gc5c@ zOdrH_#XK2Q3WZjoGjWhbuo!%+5Io5TTpNUh%JBqa&E+`w3IjfZm;?eJG2Lw~=OaWb z2*9ICW4SD2w$NX6FP35%Vm8Z+fs+lR#7}*QOK}`z+6NRd%?gJuAbN1|=wWDp2sO+` z@I-pDN(;&77z^};%(=^P6Y37xgX zII~OMzIoePBWTYf&S;Ls11K$Rke{qkSPQK|DIX=&m|R+!ij04=T1Pn7_Svv4)|MU zXZ;vn9x42$K4~Et!q*u;`y++yRgeRO|IUJg(nul1YMEVXkz`&HDfC9s_~J<6AxQBJ zxW*dTK$0#fv2b6c5UmXhDvA_F7_$Z!BGo75M+%cs5~8)x5eg$oI-xdFILDY}L8MUM zyW|6PzC{?JAd;k#!3kr@EKQ{FCnzy1FH(5*3Vcc|l2}lNARuVV1T+i%oVyT;lI{3# zwjJb=PNyT47JQ`2B3^jl&q~)s3Ofet0=}%ZIEmD~M~~<%!jOX&awc@t`vxD0Wb+Zd zg$~hs&}a>j!paKxG+J4&t>hkk}BOX%Zkc9y`2yVTF5HjRrP=NCY za>OHvRv#(kRdA@K<&X;$|1$2%>LrIPhL8pi19BL0*LkpzY7043QO|-@?@NKc2bJ)X znQgDxKa4(9?0t6~ixg6^zuIr7TFm+v!EjxFC%#CyfFj6$)lkIxhoOk%K=?m04vqZ3 z7>3aE`v0X-i1`nrki!B`-7c?3d?Q83C{i(hNm;<4PP4V@S4Ny#r@L!;V(Lv)^bApzA&b>TEaLemxxK(@uwhD8#7*h zV0~ReJRh7JR1azOjaUe`$?J*jU{!=V7dJ+7YQnNgLK-3(KjM6J`Bg@Tj7v2)$?W;` zwEeMl?GM+quJSCq{BVn4wdc7{o@^0(dg7b(3!k0<=f_vFg|*j@{Q6|eQDc<1DbB^0 zpKL)h3A)7QXvnrSq&fQ7M@MhvmR_FN!LJB)>9)x7Vw{ND200!$8nw-S_3?VE0Vx-@ zsD1x&`AJm>mLd7Ct#aRwm!BnVNr?MXq3Z1HCa1d`dIJjem)}64{;%9Y|5DfX6Kro1 z{!P~&!u{ri;-D&P0OMdT0^pWIjApAnfB~L>3-A#51Rw!UfB-NBqJe1u74QPW02{!# z66^!m0xVz#V4PO40YAXF&}=kVEdfVhIxr7#1{i=DKnH>Vqge`nKto_gf-DEV1-yZJ zAQWf^LIC6cWFi1#2aiCa0&r&oG++r}4lDq~KsLYu?gGY%#&6K@o#1{5$bl3f4u}Lm znf+#F`{*B)*?dr31Y;Rg0wgqkCjVPu{gd7Nf49Q=Z87>!DvL(JaJV`dpJu5<5ZwFQPbhbAsfcP4s-u0ffbL z*oYmH_ux)$2tohmx{X7?6-kj>QX&D#XrfUTEKsZ%>6(O1CxhfK6Ra>Ll4(UNVJaef zUS$wt6*z=eMZ_r#-zV`MEpzZ2&j=hXiz+RW%&;UkxlpbrV1gP?E{}+FSuPjS2%(1& z*_nF_5jloZq`XBM99<}V=`{~mH-k2gL&b8Tp0Y@J{pqjta`%ir`it~0_zI*BHK*Xv z4A(8=yB6gj^jCBWrWA%zsn&?RtIH_`<5o<&E}@xVR>EPKfZ#5O;Q5aeM7>Vv>trI% z0=JK%kJOafbQraU%UG3Bef{+?iu{NA+M1B|r}olY+3BoYE*2J|E`zxc^FQ7)|58I? zw&tO>@P9bCg$c&n)p+m0H1_v@n#RJt_g>!^=dAy2B>cl%_5ap;PV&!lRrDGWarxnp zQF{Jb7Mb>k^u%r6v?gI&!sbnX%qT`o?QWa8d|3k`Q~+P^MaX#N|E1^*k!)K`LTxMm zm`IEe5U}OtnUTW3rW46V^HaY3k7CAthE)WY@FztA6 z`@>uS4WgvM%wrWiDAic7u-y>y4Z*t@rW4vo(hx1>Por9Gk%DAc=&>9){~2Lnu7wQA zvd|(sET~AD1qI0lJ3Pli4`IVdriF^2jAUa-vPFyeW|;7x6-bJO>bU&p?Y`2I)Apdtr=N=T@KiGdDWK4ex46N4&9QSO1U>R@tZ{J?|)?&+b0wkH?< z)8yc4o#*f_Y8j*$l=U_NBGW*Z_9>u>~1k48<019gHZg0!{HxE2f)907IVWOtd zF<&qt@b%ASdw<}d#CLC*DE^uqO5_jGP~wYLN7U-w<73#HcW-AgP}6sp-q}DdNhtnJ zAh;ZGA^JO%z_J6!5h6@{77vqyfd_42V>A9Z=iqn!mvfbFkB{5N5Lp$1tLSZ6iZVvO5qbsPzgez5=;~#nGi%E z1FSY00aL(ej1X{2kV0FjLP+Cc2+S>HFa~3&SVo~*@LnvS;Iat_4@PBFOruZ;IFl@f z9L_5YG$ECWWg-kNmq1YnVW@-L5jqz$$7LEIr}qxgQV7{mDG)D)Mm5I>_zWQ|Np`^R z1b#LrC?OTi-7XWUr^pBn`NBFOMx|;3>qIQR-L#&#p)hxcJP|};%`VImi)fA}crhXl z6Oze2PO;ih7eddPgDeU;H$xegmppu#VuKZStcBg^_=Q1h zg~RbajuuCB599gx&dmB7Nj{a9imsOwqYdFb9M3PC_PK!<3csen2$gF*?X46;?@bsw z!6DS8_{JJBL*;VNk|m+2B?tEfM|xF=XLYXestA=Za3ZF)Rp?%qqF|*^PEnD-I0$u_ zj;kYBt1Pbrm;=#WZ1QNWhIS7GahgHx$nj>WcffMX;n8L0UJm&4HYu($OL%H4w;9P%tqu z66RKnu!e|)NC)Q4;5EKQAUhbrnj$1bWF=YGV3@U7LQ_cyRtgCMHr3x6#W*Vp77-xf ztXfoiJ;x|=b8L_&FH;uOSQ`ArDU{j|T?1ZF7U1QaudpmJE*vJRyTylkIW`O*aUF#$ zCufPV?wlU-JaGf1?l+PJ1g@u$t)h>AbtE$F0!F8rzZ6k0F4f&8cK0G@oFXivG=wYu z>D7ecQ~3F=mEB!t-I77<+|kG{rx#9 zSkoq$5*nzIyl*EseTI%La-mm+zl(RMd4auRSa-v7wjy)B?uO&pJjg6%f#o#JBqNoA z-IZ}^6h{^%|A!=gJT1rndx`*~74?fCPfN;`$B0^j&%s=_@FiSRa@H2;PjJL^AzqpJ zLy}JDqlIF1VegCOhk+(|0yQf@E#@W3g|jIe#Qrpsf;*b|(}xn96>SmG)tXf zAw)cX?Hm(D#dIu@zTfinGSc!5&|`5>Bu2a+$N&$^VLsyDQKM z*CCUtBgg@vA{xa8lOz%tav`thb02GsdG3v;-c?uE<=%09f;5=ti#YS7b)YKN;=Ov- z;Y~N4xzinC@hz>?y232Xa|8Kv3M? z5Mg1n_6!87@^c;=&s}wDen@qxw#Zb(a_5_TZPIgl14iy$*BKKI+LY5=0mEGhJz$(G ztXt&??Q^l|a`&0}J$0toy;s&vH7|6vu3IIn>UXw@m3Zo<1X}^irm)+;E`_y?84!Cq zio#Of4nz;aLK+o!s=RxCUQf1llagaI$Q02xuuheOx=FD!C&t9+BgG{lEW712O2!@I zVi+fnoON1Avmuk$ib$|lCua)}bDlxuyRnFYW!9bFhwsNz8G`Pc%wDM47dt zJ<9#dJkk!wYtK(+3U^>inW=(C$+^GjoZnA7%I?SPqzGvjzV1Btel2p{xpwLv!)mVd z4!SpQwRsBexm+~dYq}Cq%ogqi@xchsci7Ohrr)DG{Vs2E{mnw*J3P(U3wD>^cZN`A zHiCz)p=qg3le-qvv|_@$J;Gs2+?%&M7SnK|H{mKfa*O?PrwYvLC^5=INBINfs9f_P>*zWYR^bTC~$1nNR7c@IeDOEV`+<)H^SLayh*S+@K zy=MjOvy>P_!3fFV+hO*bK z2=oi#T|uZ>aq71#8KEv`p2i(6H)l+T@vh(Q2% z_W3g&I|rO!waf3b!JWUo@>~?#6jbutF7~6BDUlBL3mK1LJk1J<`Gu<9uy24_|4p8R zS~WYfMG&2~-;DS8*9p(=HM!-Xle@Yj2ZI*OhUM;}@-~Wk;lXK&7xugQE(d(|i4vB; z3vW9?pVwbMStk_NJr*(BB)huJUXJq@%%)(E6fmMy1wBeyFx&qy*c*JuLdhemN9~Sk zm*p8F?W?RwjI7gwXiK&7=M=%DO7fs*Cv(!Wuj`@oTgL*^zZIq2W~#uH3Oc{4p!@Ml zESzs$5cf#BfKyVpmy2qvWSoAu~FPQs|$CG*}l!WrVRZJCYwKF&*)QM@SvtaXl&M`UiYa zSx!YGnJ-eR2!$y$XyDa{UNo2aK@7NCW;398lC~NPo_#fQ=`SWcXFPa5^6Tnd7xQ3o4ScmmXV=Bc zJw~VO&%g)0;G5rmf71uIAv0{vnR}}})!iV=AiYasxAK325&Y*z?~|X{Ir1v)!lxe! z!Z@Yoz2WttpbfhNnFoa>Mzd_#(bDmTNGYV23|R zV;l({mYE$xyB1lS;l+w5Pdl^k4@T};R1NmuIIDDqsOw}DD~M>al^1GG zH790{(+W=YKS@$8@XYZiz*;uqqW#ghDD&472~uprntyyA(Ji&p1h{&nE5va>+(&8yIS9{eHEN3G3)2bc3-L#>GYB;{oF%3FBh~<6{Qnq5v!#eCuTU z`<#DRHkc1f1O($bsxg7_^W5scmj?cKEDb#QuU#4-7@rHFG_E%nk9gB?-cEP~XOsml z7k>5wzb8h#xm<*c<8rw)gp0=;PeHIZoX6v*QZAg}BL1+*JPlm~(t|^ng7}~o1SrjcXeOZ`c=*8&JWe8c;~^qm=!3cm zAuY}Zq5D995OSlz&mkcV4t~hQKo|@h4&kDnm=_*Tlv5Bc!wF9ckHj16=M6b|%A_0ht^GDyc|BD3(29l@JR!yzXA+=6GlZx*vmnA}%zCHv7Q; zFrWjSd&wxW6G84qZs#;^~<}F%9vD%jiW!@JuJX5RcHe%4rgu7oNs&f(~$k z186OD8<)mlcq0ra1d2mRC?OOs1=j};4-bj{6e0r8MTCgAXbpx(XegOL!x8tng5U~NG zqGH4vTNDr#yVy~q*fnazSYnG(?3x67jfyo!W9%)Oyq`Hq?#=z)@B4e7=Y8+<{&jtx zz1*{A&6=4tYpq#R_93EyKD4vf16TA(Kz&`&d%Z@M4SX7?leGqVU#|_Sv#Y*A0~fum z-wxHslQBZy0vo8EJQ^qs8lW3$SrA$qs1IWBEF6+Lt8GU4p!uEj`d9;=58oFX5z%R6 zUxPv4cccb|W0FY;R43Al1DaWJQqx=;G@#9f(9Qm^H@m|ZVH-YElh8pQ0~$FeklnKe zBGBxkV?320Zn>Ev^k(T=!xg~bC?pV z#2OetPsLB@4T0)$u3L(llZ07#T{W7`DA;XRA5>T8>vdZ(CFR8EeSCBgoxCwaXI&V> z>Vq|eo^=gu;G)>+lagK8A&bCxnmd`Cl=i}Fa_GC^I>tip;w|*y<_Jbw>D&?D-$03I z&>+Ys+^in(#To+}1T|<-U-2^f)^DJMHwX$chga}&GAia+o6KOv+<$TRAX;e(^7i%- zY+Whd;oBn22PN}pJ-SN=Lhpzw;`q<2cH zRU6$>Fh&I6Yf9Q~GAUj!pPf+Z<8-!Ux%d450 zqZsCf-sp`9j*2<-%f+#JeUQFmL&apFn>Ye*6Xm?AL+9=7g8G=a8^`6g3p3 zFzZ9_VEtW$&J^0rXs)UlR{oBGg&tSjGH;vkVc4J&Xi^_iPot!;$i*5R=-*iJU1>|N z{1w|irj4hB-cyZP*$8#7q}Vnvps9u?MsLN?K;7&~!c%e3o4v4v$lhxYA&~}7ieCdg zpLF;%Fq~Dt%5Q+%Q5bfMMbU+m5oBQ?%t7Hoi8U!fK4!fs9KG@3FVEYm_0?-eDo(Gt zL^CM{zYI))BYTWcuM5+e4P7-SrM#Esm&AxUC5=;*(33@@u(Y|FgHG`q%XDFaH)Z6% z>N94|`Nx5zCZm`CJdT{^=f29T`l(+CUVeTg&yxNnbojRcimRK)-xnY4eO2n{Ac2WU zB+Jh2mi8}^h}ywJHIZK={x=5suQABYjQ@w@3C;=p<)O!4CprH-r3m@gNJQSD{pS&d z*N~t7Kb%n5iDQ2qPpEoJEK$N3p7A{LLf9M|z&M_TDj*xRR9|58>O0k5$x#iX7EAQc z%vtI>0hlK9vbce)a&thh9#P9Hk5q$`KkwH#O5Cf~h?N(hoAd0JypBjPO;xIA(k611LN8an@ZM{>hocQDsJx}$bmi|R2y9^`u3ZvDhVtpsOM6tM&-3D#D^{kY`Nt3dO zSXciC0v)+8RZ0F?jrdU+P z)m4c!rE5yLX*$JC6`rnk4&#i|C+axX)G1Nns@bEQJ!K(K^cH8{R3%ikRozUy`YR68 zl$&_7EIV6Y+el(Ec5)3C)lFW$aWV)o)}R`hca;qN6D7pcDMPPgDpoAFp60N4DNGDw zKAa>}7gz5Eni3nW+P~Oq1jYRUxlin5B8H&d0Ui$Q+!{^uIik&sbz(y*J;P+urW=M8 z#G7nSC8m3&xaLhftWzxrLJZ5WC~`|xO)Tfc=Hh85QY#gUXOPpeD1BydkU{RcS~Mp| zv8u6Ra^jw+m#e8#Wae3UFf}vkcC7Wk?JC>B@(!q_oR}HavDA}3kV*|&qO+{_Qalah z6)8b;qYs5XVSn#X3-X$L(bmi5v}Y5Sm^;tk`iIDkozto1rds!R*#BA|s{}XC5A#3! zU|is=AMT%hJFdx!@7J^G)7(3^#f!J2LRJ~0d|lik@^UuJah|F;J6G2o%Zcz-s(Z^z zs$2v+G0MGauD`A@$J>XU;EItU-ldUTU=1Ws6z6yo4`&?D@s`Y{QjRUIfW!#r3@wAw z&Y_{c0Xcyl&KBX6)V$*9`}5wq+DnpR1xhW$d1|9d-VS8LFM7?SXs8r#7Yk(~9L}@= zDat`iJDj`?)n;fWm(LXsE26olce|xC!+l+#rx{A3yV8E+otjj6Iyu*wbD0kP)%4(2 z9_@XzoTmn6blBL*e?hC|{O6JvvUcu;WTBz2=3Q* za#;|&c7`@7U_oGB2fNMzeb|FLUgp2DRUUZ(D`Pp+>Ud&)qopw$JIUg6fAR@P4fE#e zmu02>D%XH#Dyu|`zs{oCJGW#LAWv=o57m|yTCdDpu8o&unu&4X9o$={$<=r!0 zdV~a4bg%ARvmkzDtmc+bJIvj6;I^CT!Ok&2#pwe`AhjVI$+=!|!r%IfM)K-=m^3R`1kKLl1(m*i= z=M^W;Ii3QtHGxvz2YqN3ZQE1H)dKH=2OE_vJJTDiwlDosD!ciX5m_sSk= z^D>G$Ed8$4uC9X`RbJeptoFDjoqEXen zRkxn{yDrVLRIe4(eP)z?`%_VA>32saK23|;v|u6!qn_nvmy|sT_VoY3>yFm7s#NEM zY|jbiDSl$QXJSF@A?Go}%a!wZbTd7o_R3hHpSaT7X1d?=f_Tqac_q2q(!HG>W+rd* zFwV+BA}O`Yojj3T5Z~8*)}-F%h}sJ)8KX9~4|H$+U5BsF4RCc>7gaQW!2C*U7j8Un zz3$z?qm^Iek;F@dfjbj?JDixG*07szhm+^hcE)@cck|M{Q6_uaVK(xks0*SuEg{fQfAdOJ^F z+jo1{sKp6e+=rwG?n>B_!bos&K6&GL&xYN$a1aw*yWI4WTLu65--eX!3tBC;nC8`7 zVBXn&W6E}{?u^#afg|J@TU?9E#O@r&DJA{!l#vrlW(MXS(;ZcAHd-B1y?0Tc;6`nZ ztSgH2-gPW&wbf%qtvu$Rw%A#jb3DR(X>`=QbaRtR@UqjE?4$5t%#+xE;i zB%`&rqJyZsWt(oZ>jYC_fS#mjnWJds8#wJ`z-h%dZI?~CekGT?J384Il#BqA=U0=S zT}sP~S+`TyJvMl$WA?^NRln|hYggbg^OjaNiRP5RTH&p2HY8Tf$zGWoZF2Fbn15yf z%ikG^eyumVIL2(!DaQi1L#RY6E?-B4O)hEZv}tnf%wxMN?e6T*v1?hSO;PK%4G4FixPCM02YT&?FPcHma-o@td5xT(u3w^YBlsi$g9Ox>Bj(e-Zb=*>=>;=Jc+A-bFv zw{r7+6YL^3O^R=pu<7W}rw2%fH_%PH<21WP>(?jzTo$+T-gdGyaB9S@ohN?^-0EJ( zt@YZr>pL06nWUceOK<10KeBhx;|0;9M=g*0a?!~mQQoueJpI%3W}bByGUb_Kw@xYP zlb$c?=C*q3l4mTpsOZg@Ej?=Ap6F-U*V8y}`pYo|Oa{k8M&+LhfBBo=cwNtWxyn=X zwg{uo&7|bT9{#yEn=u7sEgJPkm^+v(SKf}}U~~AGknsnP)~hgmZPk2R|L4Co8(%TD z_OL65EqSMYwbf+?=G`bSC)|Va_pGUN<#W`-oVwSXXZL7iVT(u3rw)yukbk!SYlpsp z^-JqHIb;MLyi{-3Z-+gKy>b>dIi0$LTSKc)cgS-!PxflwAtIr=&%C!)x1HDf`+ik* z+pXK}$F0d}e#K{=_nFk7D--41eX!S&)Y|sdY9&JxzQi!X?D;4(yC^Zlqx`P1Z_V{DMShm~bdD+c#Alg3k{9zehu4`oi%$M} zRNrG<-pR;C&)yDizM}5x)^CS@wBpw@rk<7qCnt^E^V7JQ(;wd&*u8t*Rnw1*3hEww z@$TGnZ*e{~Z|~`z*yIb?^q3p8sP&XXed;z}T#Ji6!$0ZX;oeX_1L-n(;qhbqSAAYl zSzyJF%H8i~2t3%j`oxkg^ILhgy;ppE{0XC71&19W!7+ueOGb~+n{ssC=%h-z*{5#j zjyA7R>Spx!3#l_Zruy?^AzLC&`aBtxp4YMZ*;bW5D^ zRTM;`pK-h#^?#{0VD?nua;zh-ITPVO|-aix6y$&TD5p)h*rOt9Wmfgbi~}Pr%kz)96PKIP1ur5WL7qGOI(C! zRFr*?&NTN_ix+*y1+HlQ&Cor8c?;U!Pso+u-$cHvGqvKPlatgv5U~;`^QH&L3qKChrVOsk9|-VD8EkcPEDlTVggX=o*xeccH8xI4E)AVQ#3z+DtD> zuAk~@o^qkg?BO{)FMQMa0gi!rp?%GzJI!^f7oX(Md}YnHHseQJ=&*@fp!+ zI}>t>Fg0U#aYn4a>n)p!+~Kk}+HA})RUn$(*lD0yAqs#NS2 zHBq%4P*Il`liw$3os$|{=w$92sAV7Jl16%7Ox8CI#)RZm(Um>*2&vlcxl7^AkTs|3 z%#(M}+HJplDkY@i4|X5zwrjRrsb0`@blb%lrSg7Sn|7w#MWaJF>Amf|yL;H8lki(~xD2?obn@4i(}Y)1l*y@FvA!%S^>T>%Ls|M0E@&w>-*oiPwqM$2 z$dlb^Vp8$F%LXmj7|Q)P_hN2>J@I#)qR+HE@o<@L@w?qM|I@q~w)dVsN|@ei?X9PY zA^R5AxY<7`(WPaK_tWSQbIZfwbE0!1%G=y4Ds-Pd36JMgE31`XxbGNF$Fn^y@AP%9 zKUMhUMu$`_R&Ng(ZKLnmyPy2T%l*g#i=)lHxPfMmM3+THi68IwHSfEZ$bIRNdLl^X zgz9r|$c>o)H~>u=Ht?SZprz67FCwaEtz;8s`4R{{wEw;sw;q80cl5u%vmWf$D8{Xv z^#IgU$OF(AxtS?fdh$F|?$XNxP_`QnRB_u}jiXQ_X`W3a7$G##_El z+<4ZbBpE8}@#2gmU3INkiLIS!EM!YOL(z1qi`=G1k938Ku_WISP2v{~JhN4gBeSx0 zIVuHqPDU1wUM#1Iq|AFt@32Yl3jx;D`~|AgA{*xROxag(~{|^ z#JXe^go3dsRKdZMP<$;OHcI7!Bz=ZjQos*ZB+|}8+Gd{Xq&WCSXV4fnF4*FL-U+#x z6M-UwADyrhYK+ykjEYTSCAjfPVItC%-9=2*HKhPfiG`CL6^l->(fQaUD(w>$CCyv; zq+~C@iNR1L>OnFz=9tB1nN`R~N;Yhj0F_Ic62W#c)n}-BhM+#in`Z4jCmd;1MseD% z%uy956IDY~Xf~E+7@U+(kO!0{lPSnQqleje$f6XQW#_{2OLQ><@hBeaWHWAx#7i_n zaBRg6BVR_wlpvC?9k19VUW%||6Q#0~yv?CHDP3(+`9o_?igRUtnZm}1B1rMjVEUMm zq-aK8#rdkWM#Y?_%TI!vQoSXgMDoJ17SctiuqrG_yHubm7P6fmy zQon-atW>Z`)0>S-BNdP25$vQ_(-hQ!5{gTq<^)5yNoiQSQ`u~!JM>qH%JA6b7~aTO zVw2{iwA3R68?;76FQDL;)DoJ6>Y1%tX`}d96s3M5N`ZKN z3hFE7#KrcCVhSs?K?~&9q9taG09Hv1U5S$%bg~f(d=yJE?etKwD&^8-)rTf2Mhas42dgAdGS_D$T@?&HyTnEh zjIX+~*v3(roXNKvsUPQC9F!|Ut!=Vo_!}3S5PYs7%2nG_?(4UC&dhNIqORV2j% zH_1>AnbZv}Lq%O_A^PRYnW;PsG!&z>((FFdo6}^Iq%O&}!3T&jVFZdzj3Po68|{*k zY|`@KrgKgtE3rMClL`wQ&+f1@?FiNHa4wvjgl@>;RtD*D8;uf`?4+!W&`OZ4Q=J+a zoov!BnbY(cjx3bOKF7&6Zz9Z9DoyA z?TAB?NhMLk6}o(tF_Lx>6z5SI z8ep3yY!f|kPMpl__(KK+e}Ckt$$;QL&lIm!bw5_M`hhayfk!u?vz{r+G5l**{PM!_7X6dWzqO_iH_P9Rsyf_(N&*9e>YD2|HrK6h{?BA^ zW0ExTosHY~d`kS!W;*<%fX816D>=Nr)$?0ae>MEiUxJc{!a<7jpwFxQ)T^e4%_@2R zE01^iNvNg~*|=K5E`AN`jKmXosBt_ND^}k}rmH@Qax-}=cFxHbwFzl1@054qm#oxy z9;R07&u!uS)uGmRgKBnC%m3c(vA=DE({fdD=&G5_g)Zdn;GEiI@Ou&^e^1M9W^jg{ zpS#Lbqxem!+KQovjfdipnW=hMXi|k3=;e8ICwb+HDrq9y#ROdc05rJcI91)(dvWG*#zquroWVg}Ze^ zUkL_BhC3!1+mh|oR7v%!!a=x!-~Zazy)GO*_-QTVQ-h0idi5qe42_hU{fwNrb8|+k zM0#oj4RNKQvLS@|5PaZ{^`Av zQc{NU14VpBj8^Tk{xH$0S!mUhnSW~4|NW@{TP?epCERu&(Z#rJ=HpckC<-!lW~Gwl zb>u+vN6OgI_ED;)6<&sKKON>LDTHj`36nlOJ=1jPX;^_Vnno>Au`(7X~tUp&8<+y9gdpTpmiWT&tx@$ zXV^-(WV9Xw;`ASj;SyrZgUd8(sqRQCZ?|1<@K9cnv~R~sA(Gq#-=rDw&rWn zLsYZHduoj8q&@EaoS%c<%bzGF@xyZ)n8l<_rVh_{)Y0MNeCkMRxCX?jM)dKUj{y~f z0U=GvfAIT8`I)coOY#fs!6rJ;+u_vbSi8Vr5}9?trPYPcXDcZPk3` zCjEUt6QAw1^AP3tQLZAXst0IrO|lyBwBV6qN#j8@)(*xM+F6tVB&bD9F0%9B{9jYv zlW0{;qA!UdJY#rf@vP?A%!3G(gFM%GeuLPn#4cV5Akm#?G>GU!b|$48iBEZ^^KcJB z*~n8$=AgrN!6NJ^ZJZ2d37Bigsq6_a%}+3_$|?GJ{5D|L9f@f9I_sKPE5cL8`cdkWxPw+hCe!Vg2I!$12fqg&Ne-izdu;{xo zlL4C&ve*!N z^?>4wki`!|UA0x&-;>pM64MQ>|1(_*o|!@l;=P4y$URuraZ zeBu<`UNsqyO!>8=su@=;4eX?9;%CV{cmqPnqrbaO2Epds$hH{HPo!&XTY##22b_n~ zje1b(0|AX2!t;af7P>smNO~EDu9VQ(;pG_K#{Qs&)dcjE0`DUwc3Dd7kQ@p>@5hFC zV@~CUoz+6^i=tFaOYglG_HI6q%&iYN0DIxz9G)QvaT|Oqd}8OJh58g4EL!qU^q%ke zHRU4D?_}CMwKHj5^L{Y+8#@|<a3e$1DrHBTZ>7SCT6qnzU%mt?iR z@++~vDkK{5Nb|f8hrz4t{Q1QOF6^$x2S~e=rYQN}mdG{aN=GoCXFZ5LYwb*8_d1$z zgH-Jej7R=P#Cb;=#CvR2G<-_tmyo}O!1xU9jKrl|8LR{|_XU{BFTw04wfrqoIBTEe zmm?Z5($=~cQ1&%>pVg(R&7Z*XOA8~heh4F;bOA)rwGp)0?D!dNzDc1!OvoinHyQ(u#c^r+`kC~hObXfa z0F1-wmqVpo48&ZS8$xv+D_p__n>l&I*278&i50z>KF;DDN&ES@I9}DPMFkIa#F#8!%zxIexw$2DhXq-y;QbA-y7%HlJmt zP|ew3w`>NSl@7YxSJZNVUhMS>LMAdO(3{4r-?W37(qjrEkjf*9NblKcUmz-#?}<^>!uOlG^UZ`wF(IRS8}7AP_7?% zDRbuKB`+G8C!?hR>D+K~b}S zcl$Buq7=>4d_(sAIk3i<=eJYUVB<}bDqabOO^#uKF)aaw(OffHhoeloR6C`AJ+IIwv4?Ei$`f-_v-)M;6IFjwX*aTXd zo<`YWO|Q{XyMQ~GAMvfW_W_n>Y`@lXlo~9ap}FFtZ8SFk_tS@GJFN}*8W;5Q4zl8i z*R&7j8ir`$Tpudw^m}1WEjoFPMX#G%Ye|Hr?uzMzqV>s6<&pk-4vcE&j%6f% zCgJ=wY_Jn$7|&NA&TXe@_j$?QK?@vrQm;TPqJngbHNby32tI;PKqQm`Zb;Ruaf!@7 z$aJ~P$iLUmh$Ms);NE~+XG4Jj7?P-}0R$6;RT#`x+)UM!lhAB;Ky*iWQJFZbVZ%sr zWLIRki%u!W$ZYFK!1}%%OkPVzmFf;yiT@Lx#GO_reWU?({YdB`jxw=Xox(2`P=i{4 zS{DuI+zQYMA0bA_32$i+z`34`=}Tp%U$CgFBV8gJ`-ZtptSJxpx(?Y5rp_{$Mc_2J z?xQK27SPfDRb~@yl|;-z*)Gt=D`YrFBGB`^_(V~!u_fiJw+HxjpyYe-bBgKVGhW7) z@UjaX-9I3gxpno}6i+8(=pg z49Z$^NE@G{sj>^+7a*)LVUxjZ4b%=#C+XYmDCK%c*-hz3;akf0q}QsXYct}aoekiJ zU&z_o&Vr9 zPsxBTrE(5=>%Bp2Ub2(C`pbD8T4fh`1s#ZlwF3`}fmd!x7vB9t+~-Uj<<@Fs7=WHp}i_LT4>?z~2y37+|rc@zSs!(qPfNhAj z9f|YKcLH772e7X%rB0@-HpGi>WZF0uS`)N?uQ=serrJ0=e`96mei`a+ZD4800 zMoxK1C7Z+cQ%S8s%pjuKrxf&>MZsUcy+`lrO?nUZ&J3`#<^#UUV_JG~N}Hx8iq2O- zEcyWLzT(=@DpMlhcoB|Sr`EtR{?t9HyYsf*bk!pEy@M`^&xxG0L%#;X^8*u{nEVu3 zN+(?THI_EO+z}1+Y64~BPrE=_*zHp&za9!=@?*48`{BKeR6^Eo0S;eZ3Y>c#*=UP) z+=4Jm3NLW(PYgm=ry;Y{4Wf%Pm=p8fGHl0azR_W*!)(=Ljfo;MsL^3lIL1Kxox937d*0yL*_bh?%*j{MfY+FfhK0C>h+31&9V;;hA_2V@!n0isnj$AeQ7Z_Kkkk;O8q)WTaNlJnR!$@&Wl)j15=Wf{wr)X>vEjoC+p)&=P519qhYhK&UDo(XVW3K+8% z>LOc#k7VYS3xCnVj;dcV)e6lK#+5on*OTbG*3P8y+H)PRwfoxfIx35@DEa!GGeom z$V>lH)_x3v`>ExyI`>B9AxQUkf}j%7gfPA*@3glwN;c>f&5~>24Lovwz4$wu0{v}V z5I{R=y@!7Q6qEvfJ`L#pn4(oGRv>$kw8l~?3kM8U4|| zOVm#3DzaiYiGLC~vS1}ic|l(ZWU$npMh4Pvbk%566`AfWMOaoF5XU&+#{96!hU93U z>1rW`snXMR(AVd+y~K7^Kjw9yj#rmILeT17N;CRv(co~@PBN74{o#u+D*V7)r!CnS zL4iCet6B#ogQ#fg9*@c^-yw&%XcRpIV=#%iXQ25U!;(>xwgX>i5-U(#t@+^IuyB`( zF?2ZK;0Tbz?hH1Vtg8I##VE3?QkfWtn6fzRPw8B?ei3^e8|Bd5W++9EDuC#!&@H4?Cm-lm`GT5|4Djp?_yW;e1Fao9 z$!v8C)R(sbPh?D@{~4YyDlazVXq|%bS2Et$&J=%zK))>^fBqrzKj~P2+mwN-3{v}a z!txqRZ?zc3)nZC^poKl56UR@IZo9XT@s)j*dr881Kb2OIfyS)8M^LTNWN1z$r_h|& zqtBo@x1?Wq1?AqcXu|d4OIXsLoFB=VGZoUN#Xr-Q3bHNFU@t-ti58Ienz3hXAwrhm zyoO(PUr?5>e4y#0LMuw0ocRfsIHMZQRlBxMF6Xwdr`U|FBr+4cWGvFJ<7Qx=O>$~! z-{l34`K*_+kX^|K>9W+$4l$HEEwyds6hN~A%EuMAp>a>0+d|;CtOsQlBsni)@~<+! z0*Qs!hb(5fx#ZS%^5100DQXdBt35u=5>2a;@c^2B#ff>US+u7=+DE!n@FziL8tZ3GoT(=a! z9z254Kds{uHDBzNUal;S?Le*Prj4}g<7C;on=hCn#C2((+Vgh^_>VWd3a8(g?cyHS zS*pewZ=b5Sj3{IM#*WY%8gznwkMyl9I>qyr)SVkgT_?~3t%XBpW}mwo2?3RNvdOIN zv~+zER#@Y=p%mY)BUI#%1>J>U1O$_3GHMf4w=dK=n*HQ#^15~+Jkct;66S~ljJsH~ z5KyobFnk%c=AeAVlx+P4%2%F%RBV*aHOTG|U9VG)^e89N14o`lQ$L3|muEK-SckP0 zR;c;fkUr6H?m<{57875H1JYq;$>EK{nOJQ!VkDgk|BO@|6^$#S5AMt24Ds5vx9n`l zg@y8%4C8NX^(O1PqksjVPJ-f3RC)*}tWmlKu}E$WR*_4A%*ue;L4aBf0X>@lG~AZ~ zSY~LR-Q_~+02U)?F>lpI2kBS@9j87yXs|U^Z+y&I^Cz9pQGuS(?LH)iA&<~$#{Lo|3I`xV1 z&0b|a2Bdz>I>dM*j9+SX5;lm(x_csHt4=(~*}7-ViqLvib^@M#BkZD;fgslL?BqGl z!zxkqiDaV5=Ovbbla zfL}uXbDC_?l|nHj*l-b56G#jqkwIc_Hxkl)Jn4zhYB69+2eJl$shLRKKg%gAMh;=2 zbYou9Sc~MOe`+LPD5D|XWB?p+2;$pf!2A+;yrO*MAvl^3lakz&lcXM;f}qj&poW0@ zp%heAoUGzal|6XUc*s%ak!yG6uNj=u%6Oi&JU{TmLl%+0kd&d)jo5qR^%lR;A`-k%R?UI47tSz@QE*= zC5~Ho4We`w6jz2rF_is6>sbr|j5RJ;Hku(jOZLJsWN(lfdrb=EoU~l2kq0M(UnJ+O z1pI(FIUV4EQHj?J09B=1y|W77w;8Zpy8Uupb$+-ZB&|h$t;Z5C$FZ$0bxO--3z}Jm zX0kRl^4fqoNDKX{JPTITS8MObbSsMeVfATkKx7?SA!jF*rH9;2(|Wh}G6V}x87Z`? zNffEQc@&KGSDPFHEXZULB#Naw8M7E*_4maLuNEKDj$q;QW+0EsO@iPE!00xB*tTT1 z+t!U4=To&8OdbZZYSbQGm;$TVi)41~y`6fZ+Z+T@O;Y>xc?crp$uooIf8cb}jdEXQ zN|p=}DovDItv_Zmmuta$<^o-^Ia`BSMc6Vt+U@s)5k%AZ>7buo%%UgKcp@$_ zR(q-BJ0ARuv{CEvf%!6K5l2E!rqU$XX0lE-(kxi~1}ptRoZxu|qRv$`QwE8$3&_Mn zxL&7IGQ7>XK@D4ZX>ya7znp{hK)$dH|1nV-()fE2%ZjK()V+_ZfVmKy$6I$foBhm- zba~ss`agvGpr@3we+C%*fug6uxA-0B927H=2-&V38GX=J(Yq61BVm)c-V?A>P86H_ z0IEs7OJfET2lD`_E0EBSHra+mEo9wPWHS<(uaBFBf=}vAeqU+xnK*dii!&XD|JW>+102IXl_Ou6h$hxOVny3!z4Z&KIJNps+p#(WIx=mwd6ot|>d&s#o zOSU9x4qahV5U1v%UOKFmKf3vZmHI+(2@MIGHoeqsO)|+_5#n365JkdkN^Bq%`G(GC z>*SJpy@7HQzRu7pCK{d{Qir7V?VlW$D zHf{Db8#9%qCukA|=!GY+dcqGqXtFQ;5v}$C&K#%A(ALel4$;bkFhjg*1U4)f>XS6` zNn@yg`UKQH=}$bRKWR0bBFQ~NDYDN?#`}F`){#&^J6Y9;a6o)ZfVmZ5rF1`~gDJDF zXB)~Sd&N`6#T)GC#(=Mz0Fs*m_;X3j7LrA{WU(U(@R_w>TvtOZL0k$_Da*rZMa`w; zwZr9E_eP$tp_Q}Nk&;FPH6TP?KCksun(G4WcDjFdkmk zeY;nwa9<9kKdLMb8N}3lI4oTWYSU^!xx3)sf{(rjevVxGWq9^5#9L41_ zsghLfeyFx6!+xy3W!yvX?S7>Nb&>}dj)|?r#bWnZa%CJ_X&fq*3NPtc`h6~a36q>Q zyJ-enK&!w@@ze^TCC7Ma-wa;zK8A>q186S|u?}q(W8|DRG&++W%DH&3JU-22VI`_! zq00B5W^+^^?#LBE=ZSz>*?@<5ZY?Nl3R}(o(@19^QmxLTMF{3F4Ho0xlC~>dLwFi! zInEfGrN-=;3QJ-JQ7og6akrp3W_JQ(JZev_QA%`#l)ymgv6{d_oOut;qWUM99~&g_ z+A91!ZsarRJZ7Z;bOb=wgEMAg(Y7B#SSLs0bJ?@sWSfpz;8602*KV1&;jkt27dYgj?%J&R z&uGq788xKfhh&^7YAlix`~uWmsb{OS ztqU%1kw$H46))(n@ot7`M`paR&!Xh2fzzpcu>D#p-{u11%-cT{ zahANQBC7b>nZ>q`AsL1xTc@g_P3YJl+6r-DOV}h77e;2O7 z^VK)N(YA7tKAEm)J9ivFK51ogc^q*I_PYCEcU3%uPX1bn^p(nhsdPYlVsjYNvn9ES z49&mV36?4k@x9ABU!(<*vgN_k0h7=hkuIm7ZW{pY89{O4C`BfCUNu<6Lu4hE`TznO z0!}fVX^USyWzHM;j@Q94+$%Cp&ob={EfrtLZc&H0K7;iLvVe-L?Wc z5GZNAV*GG{hKbebN_{A`L0L>BB9SJ^HH3&;0gE4{VjYu;)nX$g&8yc#HmybxIf`bP zAU=!iPMM(Ayq;Y$5z(~nQ9xd+o|I_56YP%hdoa)CmkW{Olu`mequ)lyZ@d((<--}E zM!J-OdYlumhx3%wLw7KZ)G{6!x;Vc=tSp|ycIH`84H<|D zzX8G-Xi-;2AjB)0v%b_Y%6i`X0Mb0UF)*S$1XXY`QfK9wPS_5EcNi~Y3Gp>whttfg zH$1OUyIkvUsKG4XFlOF9^^Mplr=0o3u;O?Jz)|T%Qiv(6JBVB3C|$H<0T(-dKD~|Z zFF8ZcCsh9$dFYeB2eGgp^cQ4g-DnBGdnsW3Zb0I9Q0C450oPMk=5#-SLAvRt?N`^n7r31v+%8eKbOf%PgDY+6&qWgZZqUt}H{o-aU_`sKn)`*q z=_}Hco{*ltXA@&Fd(}<^pSAJ`u-CPdFdO_3W`u3pUV31vQw7~XG3GAtU3g@rY(3ri?+^4l4m@6>22@eo<#j@fafHD zqf|%3!$U+Sn%?7p7q=-Zc|)7wMP|G-nc+354TZ68HpB!V8J}992Q>wxg^)+2EyCrD zrRz@WFzqsV`Eow4%KnU?KcWso&;ov{@#h|=cA!=_2H?=i;Yj0I`3=-2uXM@&X|dm- z=3?9T&@W&06C?I;HS&A>s{29p$)9_A0abf|ZfSX&>ED0G(`X?+GGJ@t3-*jZU=Upu9?`Vr6Z$)oht-2P z*p`rfQ9H86p{ZieAi!)X%2p$w90ElhZVHOlQj$0GuNrcMEru>naFotu&e?~^8W%x- zhd}< z^b&%t-GbL9VQ=uDwL)!+yn;phFo*-0cCB~n?)T(8NjN5wWTw)pC6%N66=}>XX97N+ zNAc%oC&nro`OwkN##sQ z)+0`rRt$9_iQzmsvP|ngow=Ngg%SgU*74Nj(-rVqj%c9_Z^8+qMdT)c?+(EH-GH(^ zfId$FZqKN)8A5dnb&PZp2cnSSp^qX)&;>Y{-z3o?{l0lOPgHPlLE}g@Vhj;YUKpb~wWE&CT+;7`t|xm_TlC7=K@O{U|#r zI?Ee^gsq>~I>xC^!Uq|QW&KVIrqbkPG&%7PCJ-BS58E{3NPc0UQ5S6>_&}=2_Agzn zsPQ7b-d#KL_cO#I(gC7dpeD$pg4)Cf^1}xa!2L}z(ggxPS5UpEMrzXQkC4nO3`VSz z!#|)BdF)Y&Yo3sgsRGKqDkzj+9I6J&McT|zY(^OI)Y7m-U4Pmj_i3J!s9h5kSxw?N z&yOHP0LCB<_Hb>ACoy0mDhBYSRQis40e6ZKVWPU%Z;tTM@!H5v-UpST>w%9baz{3< zGn0XJ%cf!g3cVm{KW=-*a5Bbb0BUBDi?RHD8BnyF$oY5P*M|sR7F(~>gldqDr~!Ao zLmc9WgWr*!dm0cY!3x# z?Q#~Oo5?*b)gam(+>J#mu)G5%IX_(~+X##;LSeOPO9}I}?3LGPLb+8CW{3}TumG(w z&b{S)!V5)S7Y<3Md-1^eI|ie-#1UfPPKH?qnQ{Z~1qf{yBP%D?IG&|Ecu{#O_8;d( zq2F+aG8Ke$o9Keoby{|_m`5S4dx1}2^UGfR_N9Gbj?qQwRQ~{5L4Hf~w~Og@0wvX* zqq`3-0tzodDCgvcGEO|m%iJG%S<6e_Gy3oS93r_+l9g)P>ZjMlJBuQDDgIzPvPR#h z+gFAS#c$P-j%k4fQlCS2#_TKDddu_iF(nkQmV5}>`MuwfzjiV@DmqVr)k%oH;=EK8;_r;l zI+#wWyTudP+U1oeQ(qJ2N$ZNGdKDzhARAMgC4R?6>%8&v;Pg0&4I*(l8Tzoe&Ge2K zOSuluPF;|Oxz91IDy%P@YZdhSUX9h-ewhk=8zwz@-fGkh82tnH)WQcM*3M}lQ5Law z6g9~`;IPrWNQV7+=72ap2CM$9WftMj5t_)aW&u7zIMxfvE0M?zHJ4q0HgbILs517* z(2@I!oTTez46XYGHXy8@lF4O;|3y0wWB9R=rRpVd?>fBpzmt+${27#?wqMb|8PRNc z7`raEJ*}Fp8%qQ87^l&rJ)x+)Y!DPZbm_pk4g|Z}u5!}A+6+1XOgtfFeDEY(&qybb z_qWb^OPyt2GER@s&hSm&gs#*%rX}NS4R7$2+PLogro6cTwnMIbh>2Z1BA?v}sVfM;010_M+qZ(lNARmtkg!=|FO*Up>Z zxP*fJHKl}){ZvX0mHj&T!?K_ADc*Nb&@m47-kTyN93GH zMZ9i7`XzC$s3Aj~JU_XHYj=<9(AG1S5%*zV{%4qGBh1a$xeToxJ0@q94wdCeSWW3C zWQ6<~kF+hT8S;%pG}rC$dF}RymUywik)%)Kmb7c*+tZQlOGqDH z3Q(3&u7Gm=)>3ZNdQff~Ks5t(Wd|sQ&S-(NIogyES(~+(62wo=yDM{Zf zr}r_`JF6KQj0=`m3PeD26fPzV&L4*ZqM`tETEgtY=VJ}&)-NaCkEflJCXlgZ?#k+05P}{N5yBz=KCbwbUBUWSR z&N*eQQ*PB{EhA2MgXLZNU(^?2y+s>E))#cc@ob+TEtb^wQMe=5u8<`;3navPv6(7i%kReC4qmk{=G6c2`Z; z&ins}tOD2EfNzU^{qAvqKUhVhwB=)Po0@@7eGw53&iG`Na-Un-*Zb@Tv#KwQ+WyF@ zLTmNn6~@UFBo@(A&HdZgD96YW`hg3z-l{A)M7xNIh2LRX+Ze3eXdn&~XO5zKLC2uI zLn`Vx+33DJ&KC)%DJ16_S+G4pj%bP-vCcd$H>k%@{*7U`^~?GWEDEh(x@WsHl6!*a zXC*9F38nB;l5rHIP^d9rWyv0?J)UwRK2*~wY=8=l``)Apsn*?PBInWVU|*Duj7UBtHa?P(&lNPArf!U;T;( zzf>_?ZoB=x(nyS>ACsx42+FervDS5lu`5!`w6!Pr@lRpWu}+EoJ_Q+6%pQ#R@hBCf zNO{QaGgXL^OQ?`;~4ds&CkqJr+lvFtwLuPL==;8^19v1!oC zDayiE!`2H`J19tyYQ0=I{Zp1uXNOVb%i(|pBLI<1Tbx0UVu?m2XxHd0w+Bg2v}p_KVnbalGT!xT0j-`9)>hO> zM1t>Qj)Jvq12!y?_FbUtfpkW-bO@+Gf>*KVbMRk+Cw5@+e` zH@g*}>@F;S{``tzl<~`56@O;qb^kYDnlB8d!v(7;#3q_m&l__(VBIcc6(p~JTYvPz zexn^9d=<4d#9A-rWPv_SC4-7t^*E&U#%JFmsUL#cEe)EC+VvoAvhE24$rFpXD4K9B z8t@>7T7y?EH3a{C#a!!gP;TX)K&y<-b7LNQDaQK0)p^9ZRR7s#`W~sN>8V?8E@X4_P7F4XVkuyq8}looE9;t#gVW$MO}H4##I=@JIx5;XmfEx@LH8x7MHF!et`C*=hF2Z^t~ zrxiypLK`9%Qukl9!=03WNjtruNaw+(tY5D$(!~-~Ju8kwVwx(rxq&iN8+YszP?2>A9SdRYI2Y^3dtJMmNXX~{iJ6u|uZxfgjhlO^7NUW2H_OatZq(Ycu9QxZQzj`g!Hfp2EzlHCy z-a?zlIm4gl_RL3OwcHu4;7lWmTmU_#6Yb^(DRqhYaw`1pBTzFd15#=Nlrc1LyvrCv zjy7rZW~zTOPHyXTpl>qh6}oAZX*V6B#wn#3D%vB}LOa&}GQzGtoj}33YlixSZy7JR zCsQ&@I{h6c3LIGWkoc!(ZJVd6zYsy7_qY7ND7){toZdh1<9AnkkxHacNo7|uGAj)w zq_TH=Nqg@Ug=9n&k`+P-AtWnAWM+o!RrdCK^?7{%`2F$w=lAIGyx-S3_jRsw&ULPH zo$K5yW)FR(!jgRrRH=a@l*S7+tq;*3L}EWK!L4mn#gvB8x`y6nB<_TQ2i(zmA9Vj5rk*8nofZaqcfXS_ zY4RxEZA%|#N=5JL(~dq?i~dL9mVWD}Mw5-nv~>Uv6aF&uH|LWDQ@L%6vkj)M#+$+x$~Df%WK{;t4!IKm1aCd4`TIQx(?>U zO7tEw_PtnV#bzryOpDfA|K+XSWZr#ic2LTfx8@WUo|wR+NsmyPk{Jt(YAtS(D{Bd0 zYCN|U3Ja@O^B%|IX*OS^YGmF{rxeB3v$J4@Eu|bAvW9&;%8~x>^VK%3w|*5YdCrP* zjM#OV%uzf{qS?y$j^Uv_*Ps7h8Yv>z;;|fg%bCn+9bW%tj@m35h9MJcvHO2B>*g|) zu=8jwr?vn4mBB*=CzMQOYp>za+W-Hdd>CPw_YpF8A$v$+zmX;0zE33 z>QA;~$El_>B3n!Or9KejUrlmoz|Vl=0=X9JA37GWjotEp^imbkJ)}U zZ0J=s!?blXV(&5g1GBvDQj7rIxHh+JT2%oG>!8*kFZo#}06#Tl1Ld_7o!{Y2rbSC1u`!;AvC6 z*000Ot{~Z@bdUqKp_&6G<4{9}bFIst3~l{>+;}=DnQ7HLY(LFIQtJ?^=@n0}nQ)5b z(`U1RTufxT&S3-dNW{0^ex{we#FHaA!4xv2YAM{T4?UGaVc*slPL|7mVMU9TX$T`7 z^uN!wlIh|?shv3#|9zb|nVSAgN#|R?vY|GOx!stWK7$=!$eRDV$u9WH)L{Bxg+-kW zWJ465kF%sqmDX}D(;X>jp7G!!zK`XMxZ71|GG~yhp!BJ=t}eBn%9Zi0-~5S@J<5!0 ztvmLV9aA{-iMwW%S6|!xkb0@GT|u2EE!CL6Mz!1KqsxxIiKHUP+9d@_9vfyFAU^8FB$D{*6^scGfGHi>DE)zDuw#Wp_pPO zd{Sxk+zH9pXR%vB*{pKU*@)IT>lW*1=RusJP1|7$aB8qiv z?TvO(+6kUi-A!q$nb-lwkTQJVS?|9~`6^0!=lYI!SiPs=Nzma@=IS&<{tH+ zko$ZEyPA8hhzm>6hN-`KPi6l`=C>RV<4pM3T1y-L){y1W-5X>v)K&Py_YW>V@#4Tr zd>CZO?B}#laLZan_SMVTHnWvX->Yf;l9ZYXgXjN#sf*fDXtsRMK26}-ucB2f^OxHC z?4d?6U*_^UzCy{=Rg~{MSLGis@K+S`+LuD*w{evJ{yT-f{XwDTlJOj5h9fdAa@3o# zJo}VI!IfenE&pj-MTNmjR}7O1;@#8m8}nNuKl8aNxc!FU|LbQy`F()@JxS_+KijA! zG5LSY<;Om=2Yt@c`QNX8swPYHB>d>Il|(6p-}qF}lQKz>t&(&~k&fX}UUPUVTsr^&r>a9At zNC(VOElVBPDECmJyGS~0PMfQ#L*~f8mvpI-j+@iAIdw#%{9lP4W6U{fYH2oe6xv7` zjF!os)6O)_zEPozl;I+o6?4>G(wycf_LVZKky$rKBP4BHqvEhr?AazO%_YLsMalV8 z4=dRndhJWnCRQm;OYIdSTc)RVMocwT_OtF&CVN=Yp-IdvQ;xCj*DQNd(osg5vCJZC zGgG;55fb&^iD(*CBZCDsGV(On=NYTBm8 z+R{}1g{1Qq>Byosx1@&p$-i&xT$LW9*Y>&8@Dlm2ja?ek6N=h?l^Xd{{%>Pf$$}I; zH928zq9FHJPjA73Ts6&V_E2?`{@?`*CG8B-Z1WW~QVrZ?7KgMmwXwgdpp)7yLS|WW zyAfivL{Z<`aEZ(c7j@@qM=Ql1*4;~GR$o=0Q9UL_(ahSYR%V@*MnKxw3dOn=n{uw0;khjICvw`8IALO5>$^+RE-QZC_$Du}o>QR4*^t z-DT}7Y$i7=&3fD`C4>4pC#ISx`#kPllu=%!bxTb5Qx1OIrz)dDufuaOvqU-Sao@&_ z%AyWm#q5{LNss$UGOP4D%1O^PQOSI4s+W1DsH2v&r=LpBV>7GF8a-_zY3~x1)sM~R zWY!jG50v(Osj}g5|CG$DdY$a0{Y_N2J+>&yyiwF?l60V->Ym2~sxt5B>G(*`D^Wf4 zcwl4Z{UV(x>G>~JkDnU!Tdq+}cfsipy*6i04c3)!%GF(UnpSlAl;tq_muj82oQ^1J zbMw@Y8S-y)JC~o1YHXAI;MC9r`DV2)XHIiqo}C(2DE~RP%Z<}tVdFpl^6) zL6KVfYMU7fD$DgP&SW;Kb*>(jprF>y!1l}{y>{KJZ3`7Nmm9dBq3!mmwyRX=*shz` znI%Q-22|VMSI}MFE#gd0W4qzS4&N7bl`>3QklRt+ws>^MtZu1>D;BJnqV8PmI5^8# zsynCMB6U{}Czq_AbGz@6SzV((&0|cEV&7}sE6=QvXw0>7UZQ9|!Kn7kS}P5Io3W*e z15X*<*S`-4@fml;`d1Q( z{vq~Pu4)fW_x-H0v88{C{Z+M2qtg8}RX2xP~CG|G(9~qLUnJ;fGYbtYC4|jL90{`gbuuFf7eAPFnwOR>fx4wjrR9*b;8qw zZ>Sy%9n@_9@TyLH`uxwTCtC(d9BB9H=^>hJPKORwb9mySyEr}6qRsi1!Fmpy1S`|S z+;c9@wd^bZyheAOXLv}?mB*GN)McM4sSv_ z@0E@^)26{}sGq~Tvd)L4qZ@PXZyFkPo<1!3_~z)(Z637@O>t;e=u+(&qtf=tgkek0 zw^((l=@x6UjI(In!cQ?>u6BzXz3i3s@SO|4RCKx1E#7b0yM4ne9KJPoc_f{%plyrU zh#L7HCS9LDO(s<2f`PS>0^Pb+Q`($dm166G3i7Y?x-_My z=fYPpGl-8O~xI}9c@WzSjOyV&kfjn-p> z8Cw>|wCnfLu4**rvFCy%tCpL4*w@x*TX*xfU0T|1V7>kQ8oKLH+nm~VLtGu+)^tAA zE!K9~+jb-B9e&qvhNs%*s;Jwzj#jxqN1kQ7ytlf2{b=0_obJnQS2(LXxjOc~(EXI* zTHBRD>f`Dihh5;@-(^Yv85LV zoHA;#EeO&Gt{+=@VUU#ZW81D#@dX3 zoi+T3@i&=`O&W>&$A5Pm*<$=prbtFJ)!aqF$=b|BUUrkAX2yP(j!vUOP1?#9+iGT+ zPcU(^+h@{2cC(k}()|+#J2|wNbd}w*fIiXHUdgdX52I&WidL;EcAca&#=l43XIraQ zY^@ zMf>d^-HV*um-lq_-szIJ(_`WurAZfhPV?RslDFr{#1l$W6nlAim*(c}_n34=X_{lN z0Po#pd54}%dZ;ubt5=x!o~wCBJtn_bnsuR9ocG@5yc16*|5ci!*gMU;OiioWb&ATR zxz@e2?DqB6I#)kM_mXF7@8x#;owY8q_Ql>udgpgP;J5m6_0%DYeSh>$F6v&sX!SLl zX-;G4%(lxO+@f`J|FkJ%0z>=kl|6JutHFG_pYpsNeGWZ4)VTV7@$@L=`LFt%cy?H_ zrqN(VrgCVHzUQ7*=&flgp0Qdv+`sRYvqy$?c&R>f%cY35eXnO6bzk$^W9A;^s0)4X zdLIi}LkD<5IYzPH6Yt90H6J}@T~Us6?DyL4cxi{P^|S6@icjs=()~nDhacv%Uye!K z*YBI`$)*l}_Rs!4Cb@;rw^Ye=lVwhnlu|e#$Ceq1#*| zm5d#xTFA1FAy4Q*&E2f$|Z@Vyf zt+BLs@sd?*&E2(X+QGQBGH&8n5*eos4 z9}KHxy+ts9-DhX;m?-E(cgvCP}e>t;O- zOj0dV99ZphN3FoaGbl%OzvIA*J`JV?K2L)-sFr69toOO=QV`%dZ;$Gs3j^=^+zTlP zemd`jYK7vUCqDOcb;3pkUtD^`a?neo2PFlOr-SbKgTjWt?x;?Q&*ijAq@F zxx<#8ZyDC5MkCg+u4&V-yz?Jtbg5k#Yf<<7@v!yhKP7azsu5>f_fmiO*7KhWyWChA z=U(@E?(p5`zf^X)qY>{__jc3pgXh2A?{a@-d_>**$HR}G|MqUE%^3=gD z@*l0b^2H#%b{1))T@=l&jTL0K>a{OVo8qDzX5CLg?yR1+MY^Ah>VE4%3i8kNI+v%% zxU~IfJ(9N}_4O?lEOKdQZsVY!*j>MS`GR~GjWC<>3QBhRJuEUxT-xuqnXI7ft>33S zT&(SnhTq*a=nbKY4bpT z*s6t}uXcGdDzT>RAAKv$MKag)3~V!M)H)m3u3Dsd&A`KUX-&J~2BS5z46hj$+ve4% z&oppem1S|w=!xz68jVB)7tL(jYg|^h)@ZIba9@?}eyyj6-R_e1M~WxQEcRI5d!1c* ziB?1L^aYE8QP#3+{^L=F{wJOSCOF`^qdWUT(3% z{`#U$6T1aQEG?VB)%V^aoyczUKQFDCFj(E;=_1{{Zs8F*S0@Z{b$GL=^TBS>pK}@~ z3|rywX;GJ(-QpvbHBT5(@9=9;*Du|YKQEKGTB(ng&eCgdm>!YaM%~6~w35HR*_OIiv%AoqO)sB{q#_tR_+*p}VKVCRlJDR9=FLqzGq<(^plcQsg zKHaz7SXEf>mgY3UvFF(CJKgh2>nBz_P3_Ukf9vk@yb9OJADw0^_Ri?O&ti3r>s0eG z$-atxwst>QzWSE>^nfw*F7!Fxy+U(MQ~iwMF_9PgHf%k%V9m!Bv+j)_egPAhH;Cq$?>|JgC}$99GwVpU2f7kG#8bf)|MrVNA%nm ztZ>oDwwY+69J$HVJ+t0LCwo++N!u@*tlSr=Pw19yn`fdCSv*cN+i5}%C%fGyS~rVl z+*q78!OY42vPq|jn}cpFt)5`%9{;P%^_ZWI=~MeH2cuw!WC_Gb4D^=|P2<6Cx&xVfF3{ppsX z@~Sh{oKk)4xecX{NMuA5lCb5g_3=^4Ai z^7h=FSh;iZkDaq+OVjeoI!!vWbBgvZ&y3O)dHbhLs@*x&a+j~{?&7@ib(3!FoHlh= zV8-tKd57*!x|cmYve*1CyQ?Nvs84>HJ!APUhE!MDA9b4iI$-9$T~W{WJZxW?J^5q6 ztjoRPChmQE^LX{-A5OF1^-8+A_h;V8pOYoM<|vk?+LbA-K4m^dUU_c!-WdnWI!-zr zHl?kyhh6Wi0sBlQo!vjBgR-Z0>5{Gc25OyuJ*CSV&(zY~?)z+(YlJlj zPlMgDfHm)q&aCJa{c6vHv&Z7reE2!Ds#lES-p6*8*=t&c%&O@X+hgyuvz6=Cd_5Abf9z!>De@&cb7V?Ww9BM3 ztJL$w^f@Y}sg`97D-H5x3g&2*rcEtV&OBwFFL!T_ZfSaKSzEbkt0@W^bGt2BkY}pl zQ$4Pu(x|yToHKTtYTc@yaYs3Qu99w?W`Ec{mS{Do(3^#k_)vlOn?!5Yf zMT5~(ul2Pn6U|+;F4{Jjbo1U?yK23;`|6AC4Lv=*ch}|}-#=OI(wwQiS94Oc~3d#tM>PpRJ)|XY?RLh=e2$M``oH6Y+(F*&3WC} z{sEINmo^MI?bF~~Ft2~`t;h)cddgEqDtNP1rPSKCV2Ps}S)wMd(ERcW2| zP(Ot}n|hRIoV(g--KcoKWVJrUj^$bQ*LtqAJLcE1&u0JfrRS~amtNjCd|-`MgW)}|f}mBq_fI`^ zDYL=6z~^32;qC*mhw9|++7$Td%-gcNeC?s@nRi_a0;bO^-F@)Lq1$rzJPLve=9TY0 z)Ntrt=Do0j;Cu5bcOU+7=#kw0w1NQeCCdB;KzzkmMVgieg|{3r7&mmmJ<^YDsJ+?e^V=N;d7`0MkB z4|Nh2&;K~@#I?geeIC8nNxC%u=e(0GhyOl*^j9ZEIYb0kwHquP8kGuDhlVKhJ=LQ^ z_IzWf!u0qMwZ7Gk6^agzdlqIK3+dSRw10)_`Nu;F7yb&->w6}vqMgGNr^2kEp(cIL z?x<*g{>ik$#qpu0ea~H}&~|tlP`LD1=-|HRUsZHI|1@sKvQMES+tf%M(KBw!)TMK> ze_x}2#Bh7lYTcEAVdLIkus&j9{A`&I%Az%cwo-ugO-E7J#U;*xH3FE=gLjXq0ujzXKc709-mY9#d74g z7k_n&lq1rDueLjCP$j@rL?+2z6J^oZr->cNC}TqnPFe!weRJI3AIJ!I1D*E1gMN{`$(?$+fY(`rc7IHrL^8_WFH8qiXs#^f=~y{_VQ;2jZiywz+FJ)PK;s zl9>k&MBQ$4&wJ>+!|y6)9)1(`@clh%yuC%;!B18;-><{P1!8iF?t%_dNP>ELpx;Y^c(Sk=fg*U75bHS$#wG^ca=B zk1Z<~%C{J7I8zX#x%bJ`%Eb#?%r~677o)rPX>8>(`HwamYII@^_cpDqT)FV0>xPTd zV|(v?cBFER{3nkMwFR*jd!IK{7A*V}w&BXX*kOBLd>OX!+ozOSS2f~BEqmF1_~z)( zxwGos;>Ij{HDLJmZ=Z{1-B=mtw(Rx9;k%;0l+C(zJ#PB4H<82le*022>yAde=d!nX z!w*D%y*lfzTYTWMcL#?b{`R$T*8P?7;mh9N9DXeNTl1`k*W=@tefTo`C2kik2oFuU2XOgw}i#ZS_X_b|Lwcp?533oE0=woIO0iOT>oqRI+_mW2~qmnL+|IugU^V`1*AAL$sx;Fmj*paU%|0&)0XtFM<3eqKTb8S}lwMgG6<@-Q!vixpUfHW6N;E!5BsHPK$8PqLTji@!x) zqGYF+__Eha4Bh7?QY*cL#tAR+>9UtNQRgL2q6()_jS<(pgbilH6JE$`^b%`PfPPJ0 z!X0fRbR?_XyjdUrZy0%`)rlHdOxVMZ`>-DeP>zE*gu|%75gf%OT*fuj<2r8P4jOP55AYC=@EA|<6is-B=Xima zkankDp^1SQgu$@H5Ddo%jD$6;a1-4odJ98zhcSAgH~OF-OkoWx*kBZFVTZ$0yhSBW zAZDt!h=s{CZ{aZ0Ta3oTAa8L#%Ujgo0%~y`cX1E*@c^>ftOIS9dy8x=#;Z-r8PE$pBgL5n3a>Ggd2fhs@d3k_ZXq_~BZf2miTD{`@f{7;_Z4jy_=-+w$nX`;nZ9Byyt8~oIC*@DKJbG-0uYEG z%tJ8dBLtxcLpUN3j|3zm1!+jf0%RZ)3$X}U$i`CSU?n!9bGEPO0irZ@I6yamiQQgxn2Z@u-uZXhiqXe!>XGuy*tl`FJ|cPyC+hC;lSd+fOuv z`H9_;eqv^xpO}T&Sb~f~Kaq*WSc0X|cfwDY!VGCA{lxe>KapGSC)(fg z6IxjQ!cVmO=_l0jrk%ftTtho=?=O_J{KX>7?(8o-;0Z4T;+cWJc!PIP@8&NwVc*?f zIADK1=R`k$@d~f;7Vq#LE!br4FGe`}i}Er4Lf6?}48w3JMf;0s_>9dl{$dNh#QTdM z&?;ozXZ=MG0v`B_#1H-=34>ewg(Z@H`HK`7Ob8Itz5${+IzZ$u3J@N70U{i2R|kj@ zn*xM2Y~X;=u-qIVVzvf|j3WU;>tukKbSpp%d=wz|eqnykKyhimj^vcfu@Auq14Z_QK=J!spcwi-P)M2sg%kvopaKod{~ah6z(EougjA47Pzw_0+Xso{ zE*{kmv!k z>LBs!EZ5q;^F$Fgp%|O71zWKVt4!yKJgmkV9KSP9)ZjWa|IQQdYjq^;i64ks#13dC z-GYv!wb2Q>&_QQ(L09PEBl+}+2IvMubcYd)VS*m$iC*Z9J}^aJ^us60Fe82@niIbe z`x7lN00S`ymKcm7_)4Ck#9}7(514d(=aX!F)&!X#DSr#b6BwWgigbQ#Zaub4i+1*3B}ln z@=?L!U@N;|5kD?i_)H8I3X_9{B6?0^J1`2~(}RT{l0AY&3YOw5Ny|{JIk?e&Y}R;vXap z!9ofG(vZP&gZaW|zVyJD1$g>L(w;V%6s$YnxN(vE~>CD4wtV2L-i9|4cnnn6C(cI3=y55>PVz-g$P+_KM4`N;EthhL&Tis5HT0qv_eH~pHT6@ zB2+v?Gn58|3Kg_LTd2XwGE|Jgd^n5<6^>XkGE|Hj6)LVdgo=Q1p(1}msMvv>2yzV- z&PPJUSbV+`D!#zwMyQy8bj*1WD(1oq)8B`R8OZ(=Di%ZiUnu8ys0dRI6NB`^#B}2@ zajZv}IEqRfM-AR#SKlx((K1X#JBEoEoP*x@Fj40dCe{RoiIQd=$z|eI)ZrTHaTB#c zVd63hgTusnY``WIp%|O71;+Ek#L&bru{xD9q1M8EBC!C4*nsueh$CriCyt>K1#8)+ z&pMK`_(u9W9+G~HMm#|ip5hsPAZJ~e_(zm{(UC}@8_Zyi{&2!Pye|wBdBn8{Euv0{ z?TV={bo;6!>4Clo-4Z73h)x)T{@cPtI7+aPX_t~Pv54qSoP}kibC8QgJHkW`R-wK28~4oz$d9*&!^8om!;XZByGO%BStZ8@Lr#PV)$ckI>r>QCHT43$ z(_tbIsb|AP|8rr&x+YBYx)8?e7t|THTnZCgu?^d?4@c3imi(8)ge7cI0QD_*f3prSUr1~(ChO;=2cWB1PRyV`M7yQQ1TVY~2mfj5$xv+m2CLGYxNIsZ6 z4--A$^d?MfwB>PW8qqb;Kg>8`j+T90^*SdZ(t0e$A(B;LYp+(85G;vOF0 zJ`BkBi1-kVc#J1_if%k-h$hm{@EkAj60h(YZ_u4QMtDp59p2*un$d!f_=L|eCf^t0 zSA540{KPN(#$Wt{L=w*RHk|8ixR8bn6rltasG<$pLJjRvfg@1je^x6)1@>^jXgIj?1%Z}AR4WOXI)i6796-!Qd_5DB9qL?V)qj2#mq#4cFQj1b0i zB81-D2(blQ(RDTRV1&^au{%OMd=w!{UPcHF6@vBXw_@Nys4(mjUd{te^C1Ncu;|l6<71wYF4Y-SY2R`jOU)?D(P#e$93GoZQMZv?&3aPkoP4TNk7ID zJVg_p;~8GyRV(tnCceX4yvGN$;3IxQ|9Q0Njy~uM3k-%OhF~a$VK_#?3CreYdqU?yh41D^1P4}1}TFoYulk%&SxVi1csBp?yVNXG&! zMGltX0LpO?hj0YPP>GkA?7()w8zFeiKKFNw6=twNTT5L@^q5$&7)y~0bGKMwiGqo- zq7dt`5k=U9dOVmED?Ut*6(eWFisK%!;sj2j3a3zw(>Q~Dfw5vg4xliE@~{CLQG`t> z#%AmeV_U4_=@lNiC>WJ8z_hZcs9h!Z0)66Yvm=A<|=3$rl?bK!yc2tg>qupBF} z5=GdAVr<41Y{fQgM-Anc5O-iFcA*rzu?KrmhJDzNava1dRO2+xVC0lIVFhd0U=(a& z2YWbR(Tq5eg={Rw5-ddymLV5;SdR@T$3YyzVN~D*jTiC6dwsm-hhHjvp8C*&Xx zRkVQ`)KP>@D8^=N!B%X;c3eW)R_c6PoLGZ=tVQqbv~84PH};|o`)~s1a2^++QxYe1 z(S&Dsju&`|S9py#(BDBlp&NRl7uxQkei2p5_8nvUuoWfPi9L?9ATh(-)z5r<4H#3E!N8;h|D zBU9pq6|7-{OKamrEiU5<>Tng;P!H*K@xpjxyf8r@)NhU#xA79M@EY&%9X+rj9~tj7lI$3Yy!g%k0@p*mh9osAba&#_+If#ZdE zFIe>q+}MfP~^S?c1&W^BP$Y{PbxU?%V1~6% zVw z#2!>BB#2$g38EB>+b4+1jtSyQ=LB&Xb+`)ut_dOlmkbg_E%N&%h>fr?O%Mr?HcJo+ zxIK*WtrLVi6p&#@nK(5eL7c`#{Bh?#-ZeqI#4EhUTfD=2d_Xf=@DYFU57KT4LI&N? z2SX<(h+!Cx5f}+8*dljYg2=-fq!aX(6R(^5>+^b)9}nq5DQU(hj@e!=(j3Cn86%2NJ90R1aS^I z8xusE;snta?V*JWTc|HwLM`fW4~^)vH9@%SNf0wI3$rl?bMX`x%2*cD_jBAT5=0>4 zsuM&!-ZUhLuFu$x*9l_s491_tDMSn601U!NSYayjor&Wx0j_X^J0@c)reP-e0*OHg zM>4#bmrl&ULM%cya*&5@*oXZ%fWx?gI$Xsy)Z+&3;{kjq(-#j(KY}0W$HYcF!BaHh z8D8KyUcsMy?};Cv_=Y-RT8XFvRj5M)!!ZIQk%b(@y`|oe0P&7?h#x3#P7p_M6vt2r z_b&;;3)8+Qi2hQEVmKyZhiszQiCu7?sVn&+mngpC8@}TQe&QE?;}8DgAMVK~iU+ul zhj@g~%89~CBT-l*qC=vHL=@iZCW<(NL=le!!$c9@FHt0#C5j}tn8x0fii&;cEx4IOla9`s=VLv)7`jA4SL$VA~& zm?(S^vLR6{KAI?&pz3s@P&%6^+Mq4|yh{`{pAv;-n!^X4Rp zV1ywY2}nc|-r^r5bCZM|mXC_x!2`0c^=LCP~pNTa8BlIV>-=!=oQNx}~Hi11_E z@!Owe@DGvzmW3P?(GEeF7f3lll!Lyw7|MFXlY~fOJ&;55f+W#`uM3mJcYMPS{KRkk z!C(A?WD)CufHY(v3ppqw1#3~zYB6miCrQL1Z)K8LjjMS{VsmklDBGMQ+_sWGfY-2y z6EO)>F&R@Z4bw3LGm%Q(G^Aq|)3Y%LbKwC`c)=S!@P!}z5r_Z;AsF*8A0Y@sI3f^< zC`2O$v4}%F5|D@_BqIf>$iM<*A{&db1WS>FWmt~WxPu1VMew#HF}#HK2rF2_2BTmL zJJ_ScjwI0$+USI>rPTTEB=LMtk|-}r5(g1>fPFv&A`yjX#6TQO5``5>Vm+LWCW#67 zbTUbN!B>2PWnB`VDNYjauW_6jlf=3w96uCdJ^DXQ;&nUDE8N5_+{PUgz9tU_v?Pgv z7=*#F#F4JaqJvSguG+F%Fn=Jm~AEe4y=e}eyr#e}@L{@FGxB<($WDylb8-188 z79azeScpYf^CVg16rnmlMc7BCh$D+r#GTbC;x3+|yofS!5Qk7&nj$0zQ^eZ@T}g*y zYy&z$8#>TMXLNxjj-UcZaSUw|btP?Kb}~hn!xU;vw}U$RGp#|iAZii^5C>uq2E!6V zFcibko;<^eBVdJ*u!a`%JHUqYDA>Xd_81KZIKl~IFc!`j2N#UT1h~Qt?&wH)6N!^B z8B;J7(=Z(~piQ1mm`QpTW_ z#}Jew<21`+Ar>JE*;s<5$iXt?VmVe|C01b#?9b3%YEr~WRN)ln)uxDGgdzHJig<}v z$iA8)7GnuIUSogJawA0;-ld-JQ_nC+(v@s~nj+5Q9BOa@wdnFJMd+a`^kIO~=P9BL zD_Yn-6ygkJY$i6q^drX?<3CYfu=<=LY;Xow@EJY7kPjhH_{uSZkt9`^KwBzRc*>>< zANax#0SLrA1YrEHI=;su+eI zX}XekLsNPGFjag&GkP-b6Y&ea;v0V95B@=FSgL5TN);dR37_#5`&q8P9pz&H24XM< z!4gB@>X0hjpyQk>x}YmGVpD}a4A2c`u)qNLFHRN8OH#$zoK#`7H&rabQsiJ6a|l>)3w0%S-$=s&mETjvF&xLxpQ+-)pfs^6OIKoOpC-D)2qt(rI!(O7Yh;a0 z6Efq{MD_SIv2n`(Ved|$n!5Uh;U|ZHCm8f30u6{Z6Cnv838Ds2n}h&KL<)))abi*f zY6}i6gH|8|31b*#R6tQcQBY9^t;_@*idtKwwH2)us6N%UwhY>Se{B2R=Xr16cip@0 z{l0H~YhBj*vCpt`&OYbtv-dxTu#*D+E0Kbz(^3!(2rq{`M^cctkES3rzynbr8eGkS zzdb>E1Kb3kfx| z1_sdcRSMDz`hXF92EGJ4VLQ*ppj=Za@VS)~_)JO)eD)*-VF7!<1`dD&xPS*7ffG0e zKDSiCXSh|!{DmqcWSI(i;Hg4H%T)*}S0O~;vrdI1=co`BI8mTNYC#=11%5behpUJ5 zG&l>+fCkVA&Vlpb0%!sk!6ncPT0k3U1((4U@CkSh+q(*BJ9q)p9guc{F3WlOh#j3e3PPfC3^g2eSbQSb#ZTE{G~tA-{p&foX{f@dV3(7mxyP-~-Y@ z2FL^_%Ttl8o`Po}^SBDh1_qD= z4xECz1ggOaa1zvjT2Kd0fzzM?9KQtlzzI+TPJ&uc2Tp<0pdOq7XF&sK1h+wWn+n+u ze&~gI1mXr%NIXaY*9YOcfSce`u=_6L9fq=h4P^&Efy5^&L{VyoyE3IhRzdn1q|d=m z;AeoJRv`%34pg8HGy#`aD#RZ|y;i|zan$fR95vzwmVl*T84v<@-~l|rahk+~X)16QC7Us|Yo!4S9yz64)^``~Nv z0DK2V!5H`+JO$6d58xLdOI0Jsz$tJV)Pn}l2zo&uFoG(*8aWQC!3l5@*rchEq%2@i zBgr5IsDK*iK^kZQmw{ED8nFhJWiSn_L1Z~>zebHrUQ{EmK-e*evp_hA0IrwdoPca# z0J)$56oMj90!l#{xD1TItXYlB0w^E?OYjz;0vq58+`tY{3G7?c$W{;v!hr7;I43|3 z6d(i$Z$MoEo?yeE8o}LBBY1#-v}!wC(POw)pahhHGEfdGKqa^|3g-c|f@aVLE`uxJ z6VM5|p27Kk0oN6XPT1idLi!j?0sN0}T>o1j5lNg8B6S%cVt1t9M&4U!L@xM>hu1uO$^0~(+M24Dgfum^0w0bIZX zj=%}HfTbW1ga>I5Qk@-+77W_}48Q~|pxmHAqCphc2^>N+@H|77 zqS7D~g9cd$*r4K+9qxLr2Dt%lf=ZbG6w*O(3)}{Gz+Lb;sDky!zz|G-0q%iea2%F> z3F%khKKL4Z10H}8Pz~!JLi#Ou1RjI$z!Pu+){R0s2FAhn;3;?pegMzG3-BZO3H%IR zf?vS|m;|^y4MG5Oz+7MjEWumA6D$Ys01@y8KEM~qfE*}*KR5wSf@|O=xG^)GuR*SZ z8{j@zau}`&SO)r`?6e{c5&|{?vl0z53!Ev_APs<0u0iI3=f~g}z|IpIq`p>zw1777 zC3x$M20kaPL2Q66xB}KRXpo3=8sw9!8f3>0aDBnUGjP9BwMZn`2DXD8KnZq&Xs`=3 zfs5c0Fg>wYw1e!q$2xDmxs$7fA2e-h*Ra)eBq!zgY zJ_C2b=im!41nz-h@Fn;P)HmAU?nC-D_y#-%-+?D!6pVxK!Bg-I`~&;|egZ!O<~A*| z47>wkKrGk;_JVz2Kga-?VC8nm2Sh*&BtQztJK#8!S|l7qfRnr6824(C#0z$~z8o!L z1P5XI8l<0sk6}9fuolSz*&wJ$i%gYhk*ZQHavYolb>IRpZL-6efi{>n0*517Bngaw z$G`^G*@F3C6z0DN*N$qD6%|^z=RcVnL5PwXII99_sJ_*u`)<5pWvPBI`gf_#Auz4*#e{EGM;yHMn%i4o8L52FwSx zzz!?`3&Gof2IznRn1BWB0UI~~4&VVtAO{NI2UdgaKnbEiG}sB^Kq7cKrA2-L(bKRT z>;k*N```nx2kZs=!AIZ_NCGK94KyGfWB>!m1;wBQl!7u)4vv6YPzO$d)1V%l0cSx2 z=m4Fd3-o~7!0<|ooCD`U3+My=;CrBWtwq*@S-)u!6cB+em=6|!w*ehE050GGN8kjU z!4jawry{992g-rKBo+A;OaL!pD&h@%fG>~%IZ%LAzz?hj?}7jj2-bkLKx2`LsDT!w zg0omE(f}HP)+QCnXQUzp;9G}O~J@nbjVHc zDL@4}WELQTaG?$yD8?Hn4 zfqRuY#I0Hf|8wx~#{FN4;}B+6Q3*4HcEHbn>)8H6JcnjllVsJ7ge1+5!}Doo#v~h& zNpzA$GhRrWHJY@b-6SDt?gU;$gD>5riwJ2+FqV*vM&u;hw-ZW|=D}wvY38zIuE?}5 z$*LI%rp;C+JGYy{aQ+iWD2=2`7KqFSl5B(~N}5GY@{)G5;iUO-CNZ=*#$gsMpHJo&rV2o znJ{gqQBhTxh?JHr5SkfjHXPNqc2Y_5;yANGnypL~C9jg7)^P>R|iaspNMi zX5r{GRGlos%u~cd6s6NSYE3(4o8lFRVswT~trwAbDc;Q}mCjVEGuz1?DZUdZjn2}k zb3_zHid;zK(Cur~1?`lOlvQy=KAmk;7mMcohX(C$w=79n8)qIs=gBm+BCEQTbP`Z;&(hdCm~sQ+~@R_#?(pC+vBQ(&2tY;(BBzN zeck@{lhNNkxjI>hp$sucXWButRjcAKj3JTfh+;ZVt!c)n3@@e5qJ!?C zPMyGL45>~>7BduTy^zddc-QExIv649^f)q~;bYXm7Yi&<&v93e%{UK}NWp^)jhV*5IEK{F+ou}Z0T?yzrFAD*CuGW>LU zftWp@E*8#HGFI2$u- zqE={C!&q07wyuL0qB#|3)xcP9OxqxKjMmgQTQxJ@8%^8X;h3N~J7Lw%2u9Px#7=3N zMxiwfs>n&-*5Op5IUi>|$Ow_8M~R*5G)>LcLyV2e^xYlKt(r>{)(;q)bm_6;MFW}^ z;aj7O%{A%!I~EOV+Tz}N&e&p1j~Ba)YpyiEHNn_AntrguWlD2(;w>CA6wOE$^Ubv# zLMqA(<78+$__o@vI4Z^rmu2Y10-m;~nM!3wC^IrU1RmPH2`Y^lsmsU_yDGH(LK_Zq zTTMnmhii!TTAU4^x!ssiEM6R~z0qtVWbPQvDDPODp#5~hM#NO2nU!L>+t&ADP z$*k^hE79JGvkhQI%Q9=lOX{?Dn{9)cJC&LB9ZOoZUrgABGI!}R8^ucpwD*MbmCW5W znN1x_hqYhE&5vQeZ_I2FFB{k1Z=N5={9rWmO2@J(?KczilbA7RR)<(+tf#Kb_V7iSysQ;otOH!*{+DWSDAIA!`&nG$%I`6bDu8jme@m)Iwo9D!`xq! zb+^MKB=!5a1r5v(jam1^p3$k#nin)PKN`)t-{F~%`oqM6c4i!!JtAJ7mij`t(8!GE zWIyg$UXuD#+`>U-f-HMXyrM4kW%I%zW}-6tS;vaj)L$nSK42cuWxo)w97vrMzCFr3 zSd;y-W94w_bllsAo-;o-W>1RW8BcxP{PqO%&}jDSj(4U~f1h|8$4WvCc!|hdhj*u; ztYnVCv{PiOGl{2RtQ47nC=v5?rY$roOQkeebc#K6vnFXYmRe^ZOC$;%(Vfm=X=)5s zostmU?07n#r8OFCBwo=vix#?&l{#uz(CL++n>$Gtv2k=+?GmY$Pi-zhE8 z&5LIQu+n5XT#0v`&Z>nG%t}}0ICpxt>fV}Ugt9VpIRc5#fX>F9sbpo=R?MY@5%Yf#JB{Fk8-JPXl<#2MnJ7upB%7ddGPC23Dal zcY|bAwBEVJzL|A+G1C z-6{P$lMXoh3N$}i5@4PtcITk>m7IJ{XMk;*S3C!^uaf2KC4sy&?-mZ#{+Kd9vop{m z&3BSRvp=rO&ylQAq{-d69Q*2;{DRIkA!)1PxqSN*#{6Q*+UT^^EnK1f$_BCihr6edVEzq4Ov#;e8RCflIq^*tT1=!cg3Th?m>eAM=@Ph46DGTa5*R`g- zH^~dNKdmchl&l{}+u-h~w6CuzXzE-)oVGFEF~W(E{q!hz=NkK_ zn!>xCAtC8IK2r z(5U^Dn!_(Uw+yEz#=AVX|HOEBQnGbC{a}mBg#Fdg!>>EHPNg53biuLP(IUK8sCh=R zJ0E3taEeU3LTxit@qCQkDJvp+h4C^pEqp4wOIc*m73PtVI?1Q8yLClmuW&_%-d(_9 z_tX?wb%lpyq{j>R>|SG$jaNi;MrMmZ$nG00TF@1dkdZwp5V4JDG2JUNEhER>RmSe; z6x(-2mSp6`y9TfaWW`*sZFLz1Ev~`rYszBhu5GOuhbLV_+1GW&0Mc0n0jH8o_lh}i3iP%eNo>}ScreojY zlz4Y3Z8ML>yBXNGWhHX2C|+iDi(3)=B}LynP(@Lw6llM(lD=GX_<}gOO5P% zoYHMwyGk<8$1fdZ56eoUymr@RHnl7rVt=VD-QBgjHS^Ns(g*CXbfvLg?+;|QxGx)J z->)g%-}U}*W?THS=j^YIrSV=LjAveHSvJA`X0-HR*9TLXS114996ui|uk_lNmUYYB zL*_8XDX;F@SCVxn-Xp+aTvlG|wZAUwZi`2-!}rSa`mX)0Szk%-x!FXKI99R6V}Z}IwQJnMdoXPmJw!z<1_ zd&GUY&fx{;NKaRsZT6%1qG;(6JRTb36&{G>c`qbuGc`^n_;3WuL{M{ao~ zD6+@gSJXJXtT}SGD|ZBW9(v&L zN_X^y*T)0dlkV?~I=rqq`m*ce;q2-7cb+@^W;{CSb!a^Mb;~;w4!@5cecg3vD*N}z zcW@jWu>vnmGB@BoL@?$ex5BhL$<|Tl!(S5 z=qt$56orB4A?9#QYb&g}Q$h^06U2OuS$~C%R26NoXcY@Nv&JeGbgL2!bEm{24oa+~ zOVw!xvWG;*A#yA2yVWIzc?psLj=8*&E7jB)tXd_(oY_&8&fS_;!&_65P!367DUfOh z3^pEKN{&Ts<&tjguwj0JR}5!Pf2F%Lb=TbP9j>D9+f@7(#3Y4ZPa=0Ge zH5{wjs&(CIAvul--VGe<{;Cbq^ynPtR_|ucTVqw5yVDbLT&BF+IaK1YFlk0wj=;mm z$g$xb+t!^?lCwC$XOLqnKNcm;tjk%_>NCWdA9ZYZcV=tOvMHYj96SB7SZUTkj=P8N zC}%at_zN z_IN>eZbVtq_1w277pZY@K~kfI@eY= zbr%ljZcJDe!(G&0-6B0ap1ZkqRUFr4tolm#;i=rMQ>&7=eBy}?X_0wen1`Q^E8w2! z=`OO(i%9S@a9!mm`lZFZylt(1Mcl+<%u`X3tNdPbdm)?MD3_tBL91MYJD$rsWi19|Zt?~Za; z)Si6VePlQc@rryQzM8q1rcZGR=vPS^Q z6LV`!dn#=6RS5wYPa>}&dROxDHLU?uo>x?jMNg$ie(F>JjVIODkiDxE`Ff8)4$r%` z#;T_(BtJbNkk9k!ud(qy7M-8j8YtxXj@2yaIhK&0JryY8$%wUd@8fCtIUZ|dJUO@4 zzUO#JeqO?w|KwQs!}+BNYh!r+{k87iC&%;4Ti3?%-W{u5(Q|St|LD}(BwhfqPV8M{ zUQp=~q~is0>%4nvYzvMh1Q~d1l%3L`|CD%pNcN1Z(Y~SdvC06bI+-Sg0oZW+IhjmQ(@kx z(+V0r)*E>nxTm)DoGvLipRj(A7a~6umTqo=}*Oa zpBX4<@px~Px4HJz{+=_#1#JoMJ?CxdKNatNcD&$9>w6Qttz)MS_MDw6xH|REBi?%i zqmE(R)0&I{$Sdq=4W>FqM4is;Y4j-Un+m2mM(R)Jc%M@g_IqsL zIBu&wUC?tbr0`n82EOC={?oo@YAUcB1a{$zS8?bTH!5^ z5Se2Xx4ydPLP_DBgpdHoXnB3DcT-*A-PVv`$DL92^*v3kg@qp=6$8^@XOXMLyn12XP@<4X+8Yw)RqU12lQuOcz-f*c+z9*sN=!fvoCu-89qFn zu=Tm)$Ngs~y|0cRe%-ot!tv19+1EW+rw;!ahx=@4OYFKAw{zj z!}w0x{stSLuIM6*wlJYn>R7{q-mZkAxzk}HCmpeo?$e!CMD`4qIqA8L_PyODMe`EF z1Dw+2ja;9ex+1H#@L;F(s7B}Bp1+2&*Eb4$dIySZJR_7&nYE2edV7b9<|jtPIA!%W zy8HBv7cFRuh;zywYh2OWH&yiZbVQPqfp|{rV>BBXfTEb|xvj>jp!VFl-hq%}$HZ+7PKEvF zHuzkNE_QC)*6eh6?A+$wYYD|J)7#peiiqdKe6FVz3p}?Qor<~VxAk5xDPEkoebA{y zem=_QMqTlew(UbsrBUa1_ulwx^m_gISf85%#qORvMxDxQ&+qTOIb7_SxZ}Cgk^b}X zKA(;kuV~vb;dFHD{K4K&r;6X1-hp$jAYMrJ8T|JH)??09@(X&OTmQxJ>U-}bl)O6~ zC33DIHdXq3mR1tz87*_JvOlRWL;Zyu=A;?ruyEyttIbG zM~6C});Bf!d_GXJ!E>k5xxTiksrU2Yl8uQwW1P?QH?{bDF~Wi1is%5lofmKRW7jD{tEyx9HQcmKA+Zrpk^^?@d}XNNg4Rj+&QOdhXLLy2Wkv z?i;l&KbE-9u;{kDRqi{+E3a^c`1}*Lv=+S#-Cy zbzR?hNcpM6{SAvg?{D4U`+anIecS%#MPH2lM~0cBiACRx zwH@sHVXFM<^hY?C2gJ+CzR%5%ba=+0E+gE_n!e|@N4gT@Fqen&%X;4zydynraa5N_ zQI|9OUU(eon~tNoe5=2lFfTm-qdYaOBhJc#+E}@k*ud&uK?)c_zqQ#<*9i`+hDtawjn%z-3&1rPlXl-I2R( z3BfMkM_sA!d)a#Ai|K?=m#6wGjlRDO9J%M2sC0Q&d!?!Gm*FE{CML$X{GR@ z|0nl+r=yQPYdhHN^6S_q_xq+3j{Y!xa83L|><#p}Vmwmqt zADvD-^xWmQ{;QL|zmFe%-F9fg<@d3xuls(VI{N$cAsioP-j0{yW>?^sC!u^iuiey$ zn_ppaAPM6miguz5?^t1aIf=?QiEg(r;yo*7y-K3-32E(Q8M3N^xICG|H?3>8G9nu* zW*T*gje|B_-v(dDz;;mOHp?p$Whd^d_t-@xxO3AmV>sVqm`?6yG z0aXlt&OnE|Y}WS`3ofhT_;bfQRv2eZSG@g7mBh!)JH;||b|rneTE{2zI=zkP{7U8l zwSiAjbjoE!$4dLl>LUKU=+4zfqGzSUD|H3mGOaUEX1=PDyIfPlx2o%0XEfhf>3BfX zz_%Xg+#s91v(ovprkVfNc;{y0?8Hi!SDJP{)x0ZAMoO;~EY}+OHoUHFMp9|z;se@2 zzOAAwN@j7Ya>-@w5PyDj7ktTCTjjD>+6R2Qw60j$oNJZt%Tw9UuOEi7{gFS%_BY$R zE%yEWyR8uGK`-mh*NFAdYlNNlbs?Tz^eKD?!2j~syuTf~-+)+6;Qg!^Z^rG{;e{sf zN56N2yapkOgYgdhfVIJ9ki%cRRw@p}lL=mH;Y-UU zA*QpOqYh)|Ws+;#u-$D-MH-SA-31ZkQ7we>}u2Yf9 zHJEEe1%n{z)G-H82RHScXriskI@+Q<={jAB3H%jasDQ+fqNqj~PS;|RQ8XtPdsG&d zA=fw3da0|L8EWCf{Gg)HGtDWAo!2eGB=a4ppM@w#F62>!^b1RSv4?hofSt;+d7V@* zP6UOnB)It~)W`F4g`H)rT&%wM^AM9meRiLf$hVj6y)N`EVyQ7yXMI~1o{=Y|JUqTn zDw3n*ZcLfs)=eg6&@CAzG7nMiG@ESc8t0 zS^J|#wZI^Rf1&BL@me52873C=ETPJf+)IrpV@grxapujaClz~dQlF3v-R^zI)z@vt zbbGJcJVKdLML&y~i*Yk^DmKg}(nI8@aZ7#XN00=!sa-_^sUnKhoQq{-(%VH34dOW- zPs-fTI^}op{ZLwC}IZK za_!aEWscSP=mt*pzMJk2<$G_tbIbE6o=@z)9dO5H9il_=Vjq@0x+#9b*QPi|kmza} zN{4zX)9sZ;bf7~4qTYOpA2+<#4HcVI=YJ;3q^s3JI(2atQLK|1-FGbbP~pxfV`d4T zlttXMc~lnEOZC@{^vYxvvA4+&bMo> zIouk)G*+$+fy~O4t1!1=8CSno4O!@F{t2`)pAu89G6YL>0wWEzaz$+=nT%eXgp^{W z5i*&#VZV3$g-_7fETMy*r513BbhSW5OIhREkELj_&f}33EKJS!f!mLasKeFCDI$hl zvj5_G&Gueu$MHg~97$uP$(!I?jktB(kM8-O%mUHcw&RoUC6&xRGn3{XD<QOCI#-a;BqR(u#03)>{^#7LKCy4CTWLrdptZbBsmUjbsJ; zP-DB@{b752KWLoh^$Eke$xqNnx#Wj%_2#AM8!46%MS&UE6T6R4MvTRUL5a5=T^+We z3MF?6CHZvJ#)Pi!MN_OC)Yuc0)lL1QrPR@aV}0!&b(8&@Bh=)Fc2CqcRT+WpN;8XO ziSU?dvbCdxx(sEYrm?n?_pvqI?h^D-H+2WYaaWlXim3wuc2=&w^jqlVJcpjn~kV@OW24wIoQf~}fH zmr`nVaI5*2F||V9GFFDWmu_!21@-}_tw=zkJhWpHykJw^;m&I7P7Fg1ooHrsZ}2_6 zC)D*jQVy1acXmS=5$H%6u|xO3%)_}xl|rm4Vs?|mX;K~Wx)t3|*qfR1S&W7xHDKGZ z6l`92OQ!^5s>w)nyN>?=o!!J%WBxRCw{qukmUc2ArjmYBkH#0IKZWOs`v1fLV zLMTs!ht-QEMd2yjt_z=sZuBCtQv};9*7WkKP(c>=J0U?YA~=5rSLj0TdaScZ3ui^w zQ|K6eVS9#PPX6tou)c!t6rNqwh}%(J)I9&iy{s^3o{B!t;`dVbN6B#ZERE zB(A+h$(9TyMy4=S6>bDuBvYVPqWbo3{zIxV!(E~alX!KNX(}*3>f-gb{qBOYRlV)` zw<$Z0M`V#B8Xu%>2!|Vy3Z+zIS%UX^`B_$$1_8r{X=X#n6dOemBD$X-qUc?I#lPY z<(*5IQde$yM6O&y^~UIg`eP^~6ZJPW4K&jGYX3&3D}tiX99(zRL6E!Gu8aQ zmPcz8X2iWACY3~028NiivV;%S^9%wu^+}KdGD}26BE6f&(a}1It21yps72;tsppCO z&_Y2^#hy+s+aArs4n`+Z{LrwQvFz%6DaD`a?IR&rg!R%0m8=Yg zt|-ThZaf}2vz14=n8eN@3xhrt+8~tvV9%SW&c<`WWRWN$!zO~x8l>rtv%1|ERkPHx z_bbHcs)&ro((q`Sn}G=pAZE)X(lZ3x%V;+@^sB!!->GQ{hl)Y{$Y0T~%P+*{Ky?tS zu*fy8+|2wkh7v_gQ@W^Y&3R`~hb&K@vNei5-&qE(apgN-xVhnx=n^J5X6FE{5!!*0 zQ}kMc?fcGsik-ccp&=313by;~T%98xWJoR~>m&-Uv=^&NhGsWhw>O0PkSe&D6sV&K zZc5;sP~5iDW6i`7C^UAKxTvFR!z zOE3%5h>TdHeF{aPw2AOjqQoaCMN;-?4Fyh-T5uCievChafb$-rBWhuv`-mEwSr+`J z2YGx0g;ulRg-*q)u+_Wco`H#l2YIkQg; z9>%k?ZbGhqjc)%oI2T$i_p>6r3C-W z!Lxb9|DAzq{%2*_raN%I^j$= zn^AGIlaujO5fo{jALL&Mf^eoLqJ=n9a%UXQg#7JH8D};H|B{XW5g$k$#Y55YDfrp= zOpyYXUBu&Nnc~&(O=|N*1iTeh2kU21f5!9dgx|qy>99E?iboixM7{;Sy%`~x*un;S z_gRFI{9!fS6Q`svz+iuP5x#BHBG@7oXF`PY4pIWnWG+4#g+n*h zOQGOoK7oJ)Ko&M`7QSLOK4#xaoQWJ>XfjI(OW|L%P4Vc=0lC2hje=v}R2ThIb($5k z=Yz=o5r5XFGpMx|Wmmk0`ec0dM}7L=UGTrHTr4ZjUzLj@oq5j|yZ?iI+h!*A?FozA z_V-oF^vyb|>6^U&Q<46EQ;{BI{&Phl?cTO`Z`jON0>;N|`=fQzqSjx}V)tr`@az=$ zHPbr%>$U#dI%FVP4bJkj%EiBFql|>-gR+qX%QJ_AM4PfZUvkQVMBHmwUn``sE6l$* zRuLpx)4WjgePxhnCYvv$i)Vh01&O*|ayDgG1&JV6%!-mA5!5aC9>E}y=KH|rh1Ef# z?K3|of<(VSwoaIT3X3(T{>)z&B%0Z5U`>$d&Hj%DiDq^eQ~qbZ(jZYH>}~|M`{wnq z9Oe=%j|7QIif4YHoHGS3`g1#w&J@EOj$=DMj!+mxtRR#G5tDEQ7Df(aetV`=@M8QA zmYZN-$7Aqg%Y#HGM`eqCheN8L+O)gph(!_#hlBqn(j0Q!xDG$ZgG51V94hp1Hgc~W zl3lH|7$O`CA{t0g>=JwhuGT_CgCnVVZ@{&?+F}A%Wg(+M-ZMj6KYaIug^qSINW@-i zdMx*dGln~9!NHxjXuzFuHsBg84E(b}A{)Z`InL*1j*Q-67bF@yzu{%e84H>nyo`i6 zrdpGABv@;S=h99GiP{|I&Tcwmp(I=i;tt}@JEOSs7984zAkop~PM?+3Sp>|ikl~t~ zhtP{bBD$~B!lplu0`fr9@SVBUIrD$&ZK&v*6Mrd4L`C16;({QNwc#Dk-z)Bk9U z$KQ23{D0|o=v%1&iC#zcH@yyi>}T9x-Oj9KNlBq1j-|OuQW-!anVCw$_%KI6Dn&FZ zDYdI2Nd~hTSjn*_`EtACV} zD6=}B#0Zn-kbGorbwRx2vJ*cG;+CJ7$170O2Ax{xRS+(qlFT0QsicrIZ=JSM^+#f^ zujcCLWeClrWg{nr`tJgYd z)r+`px}M!oHPcR-|MlIeZ+g3}F?*sS{%j^otgKJWW4|hdW-=`svi@i4{ol5r6DC$k z1V5{f-!!1F@Xx0$;QpBDgckx7%mjLTF!Q|C5(I!*Kmgzt!bO6aXQDHYR<{Bom<#;C z9e%1DFFg0V)(?1*8(72Ss4! z=`jP4z&n5igun^dfH6P;)?f)B18)Eg%L#L<=fr<%Se8N~LYUbHl`wO(@RRpHjmQ5X zV*UT9@p%5=&ydwjE~1g70%{kZ)FmL9xRPL^zmhcjyi?|4(wsZ+h<14ZYqdp>@OlSY zaNdb@KKNRv6Dfk!70U15#tQC??BY9_9NXR1L6$5dWq#piA#l3Sx4RBoYdAo&OzT$i z&AeE6_k5Dv^}EolHGO!Fu!ZSb#|8#@+W0Bb9 zaFAd#UbV>Lj$7S%r;jPF?9p>GZNp!Gdoll`ZP*)?@MqJYn`=Ehhkd37nue_7GZEDP zQgi?I_D8Kqepc|GSa3HGX6}}mTL9k4_5RB{IoQyf+i&J=+yC^N|Jyff|Dme?UwXTy z{+o{TpWd$F+X*=Kg|?Ym^4Al9xqsA>h!5V6jM^9V!TWy%+e5RgYi6-8T!5;Q0l(f< zm6nT7D7|}^Yp?$L)`Q~L)xrAqf*Ue}=lkKf}6=e+zm43gM4K1pEc$f3rWspCNVTzlF+gLh7_Xw-4*P@G)5K zOhk>xJN;WApWTVKvCt7J))H%QA6q2hiYz#Uv>;-@OhBzs=xD1PCoZeXo zVKa>cN4FS{$EhH|)dX=lC}hLGLkQGaJir~Ys30T;acMZ6MLSMoff0m3+yD*EQ_2xNk9V*leO z$-14u`J2VC_SM2aTY+gyyvc9Yzzt8%O?mKE#ph2ZtKfmoZyX@R&+2YV3_S548q;?@uvS+5r;SV$G3RXT&l@$cCg})$2Cyk-~N1D1IsSr@w29W zOXT2ZJ-s^p=5Y<)bm9p-uA%mQ2S@dF5YdSK_WI4!n%~~>eLelk7GCcTPituYuy*GB zym?#$3$Nna_={kkiZ}TM&dSj1H*qr*bSZx3X${`wIX)Rbf=8uL=6`%TJ^eZWGPCis ze%typK1Q|@Z*mb{^=d*0OJO)a)8BrDsJY+54W_=K;Ab6zsF_c*B$-KQ7)n6!9btTw zU~Xo@Aj4d;D4ZrFk>EetVI9s~O2dLV*6HoxjloV-b2C*(D4&W-k0qhPg_w+n&^c6$ z2rr^7G_*(FRxY=lBYhB)6u`l}9~vqG=BSZvqDmGg-;0&Rp_-#6DG-YD(#%z1EkUrA z>NR0bi~>^x(K|6nKI#-P-?f-z`hY5g{hU}Mrt$QO>T`j+*rhj4B;DJEnRMeDxm1#F zb-1dAMbS$cIk;U*OK)sHFNYZ4qAFmBCbX0KGDda@b(v3&;mFIaZk|XMa^_U7?tMfS zOX)e`%pWtX8K0(n`(oG9g%sl>vJL5oU+<$AaWqD%47Rv@VRqkIUc8Tec3;GgyX-Po z_dj|eQjjGO;uk2?tE&Q|ArnDGHs9udWKAtavYK!2!*MguR8;nFH{oRZRAP!_@@jT% zzLsn*aLfBlR((skrusHJuV3jy%G`clKwVn8npeQ9URt}FXJuY3(Mu7&yqc<)M(5f3 zt;x%0FOkE#e1{%f9OamV(G*fmsU@T@k+XtNlbIdijXnoYTctOSvk0e&j(7|WCh3xp zN)`oCC3l3gf}QZPc40D=Hppr!-o>^uW71_o6oaLGAbB0FN^ycA%;XRUlv0~mVV>09 zWWqU4OmlmIJit6((iuu^Vh{&( zVPqC3Cy?GD3Jf4CAScQpD_CMZ3zEoOSwH}%!P868S1XbG47idsX87@ zmd{)=fo-M_QO^>SSu9Jvnx2#4Lozqb5#cb(fRa@~vUBdMSW**2C>UrDrP9of-BPl! zAReV(#;6H`+F%-38?@!D;<2d=vMB76YtSu3arK&7bc>6)8tG?I3Z$A`V71JN$K|Ivs+2yXx@?Bm@Q5;rgeRM~bSR6-e*lDLzI3v5I4$6tw=NvT?lvB3&SAV6tbFSz8pFG z-h(%HomxVP8$ubKBQ%K}itk@l&A5!}r4PLuxTv}>oQ}yc-4_Tw$H|(g5;tHnYjeD~ zDHuL*qzP@roCve^p|Nd5YvSE!A0BhIUS@0F6z8fo=Tdn;3ze z_83&~QW*mhMzV-Sv6m8aX6`dMX`CuX4O*p^v0SMO7r{b>PRdAs`P{E*~! z_2LIHOHFP{=@ghNly#591mWkv`)Qrf#CsLtAu4phWH?Hb)UJFEVL(T-nmjdL6J4a3ByC4nS?GqxtEq@CsiAjZnn;57dn&|&dN7df6Myk}BG#u5ab3$@VRUd}HITJ++!Uz4+ zy(7swCuMri!^#9cHQigyccoHE+R<)fq|zoO&0w?5t~hWzPVZ!9BhfB3lFU-Ca~S=R zBy}=ALTDm&Y1lzSV zb?wRX_t2%K7U6qE(v_v3?$|p751YDuSyKlnbn)nUqDGQ;BW%ymGeX;D@tOpsWD9R_u>Y-!m%v>P8Qw8M5p@M9p+nRvjl%z4uhAGeUxXgQ+e z1IZW0UJ|Zssn{WTnP4|Zcjx<;OG_8_MeZp|VBh&R^he?Hg>zK`gOeGhf2KEsxmA09 zY_KE2txri#IBjuluTaf0VBh{Ud0^$+*4G?fyrxn@We(w^5>o#U-`;ucxAN#*s<{nm zCDCiokCO*-H7;boT}$;+cR_Z=De{aIIYLJ%me3C}odi}}#T}xNy>AS)LI19HbamggJBZO`AkVp>w-IR{! zESlju8=o9(mVN{I@t%k_UlxA;o|x9QAoBc|yOtg_Gl|H15xKOqd;9rXKT3I_jF7xb zQaQMtY;JE|8K9!W^E^Yx_DegTNisKwU;1*1t$s7}!j~HB%3viOefHtW%{~qD_#X6U9qvwS=p4TFG6h95Q;B)NO(6bLO`5bH9skhDSJ;Q+3E7Qk|HM|ZwvSYXEp7Jt% zPOZ7u#Lko-Ydm#8`e9q9>@&DOQ$OEsGW3j>w>~0w=sro?bq0Eg zIrY6G+u?cSAIXPp_hDg7h@F4Nyj4&Q=J>K6#t6$`E6oe$Sooak7>S{3OGnz>zk1*9 z)=2w^op9Nlld;i|a~`I2Elz4*-?z^dKUYL+=)&)TwqA4oQIXItbhZ%>y$#K2)Kq6(uu zh5BC^Ftj*U*f|o@oJ7({MrMwPe&S7%nhuE{Db*`WAHn}wI2#gIYQl-Ixdx1_)#qcm zGSP<%^I{zAsH7EnQBUp_t0g1tp`*l9FC*p(k8iu^1XnfmAz>c~Hx>EeDx-u?U#_Xm zZ#YajVO^_@|E*KY~osMc?Lu8icjVr|u7D#>|No1tJXMU4Z9^Q_T0D~>qNvvJmF zoToS>n(tR_a_{rr_j$jyzW0xB-9K)YXLD-Tu3fu!?b@|#=##YUk9og1JepW9EXlur2#)I2;nEP_^s;r)!cXQN z=(j#$iahDShfQv~Z#97idh`B z$JJ5!Q`K&LMDJJ8{hQAwG>wtQ?20{FLG~WKD{I=FSG_`tyZIPQRkhU?)W6lr9Myb* z?@y1$hwS=g$**->zB@4GWMbE%+yjy8JHCGJGPlR+Utj5*79`{?tXMG2dph1NFC?19E4au4+^Px5zbaZsMv;z(NP?6~k^AFoF17np9ncpRHk(>FPIPPTX6Ax28HyXV9f zrm9m~sxVvQl&HJ5ea#*sW|-}sG1(*KcmM3l)aU?5YdBpF9F^lwD+l?Br#KXThanuNJ4}U4bG^Tlq|FYQE8O;}- z{5j}JVaw@fN`s{_VIKZP{%;Dm&s+a<*Lp3E&I?@MHLyk9rru7yI{HgN)gDhr9GzEi_}Y`A<8jl)@+Yj<#>>~F7Iup^KGZgzx4pV!kxD%;Y+(=` z;k2il?~u?M^K9=0ywf&!m~iKGRkiJcG9sv9P3GC|d+~rj#S34#ia&X0`}(`jYg5Z3 zV(5e)Pekl}IIM=#p_@UeAW!54-PJapb|UN5gIKfPdrijdf7$-G>(MvmUpN1MjCQ#e z>HTgW+csx*ux<1IW!e63w|0`8Q(~Q6t-F8zJH)zQY;mi3#D>1uL>C7EVn;x1>WhN_ zu~Bc!BMt!AN7&-!{GsN*>?4G*XCMn2S5a8}cTN94dj0Xm(| zWYp_*I=!7T*r=0qM!iw**}`B942sZ84tkxFlg^|IFzR(u-Hx6*oo9f>q&FF)?>$Yr zK)p$4!pG25>gnAMo-Ofl};*AnWV)MN70=>znhngg;PDI`g1%|0WF2xuW4 zrX@OQvAdIAa@Ok{4VrI!oB~jkQRhSvqfwF-8wl?D1kzHSL9)|3>WnKSNwdtr9+8vD zXmrr(*c6khDQsgImpG1)zR;VD-*})*z0Tli3YHGYx)H_!iY{9x&2qG4=!OoFe#0Eu%tE$soDV2fdU58YmbEI#n%!$_Am0C)@Wuwk(hBVp3OlRoy zW}}Dk*U3|Pww>`!fxKLNTLTWb7WO4FB43n2R!r3dvDO_(d zn@!c6jGjT^*oAJ9v&Eva7^4k(gOig;#=9SZ6VQ zQiVs>RP;_p>6NQd?;)|!GtT!g2DnMpC}HwCqS9GbNhV;UM4?q3c4NfqOa|;sZ=zXJ zT{+xhY~|`12|}tSJ(ikFzd6!Bra(z&*)fcca&j{0EG_f_*rLTrXR#oUk$OB`q^44V zyH0QDfr`w=i3af4wj`q>eR!m0gsgM2R8#3KC0f*>i!4=)y5Ir*WrI$tYRPld!N(wd ztNFq}F(;!O8LWjqNctlA2M?XuNEb%yjnZmOD@%sZIm2qRw8_$1^Gx#c+M_Z`9#Vii z){a@y{9jbqpsaV&m{i8dyV4dZQR=D<&62#}oGiiU*Z7{{8xdeO)5`RtjsXGaJ2_7J z-Y%Rtl)kX&4SmavZnECw^;F^~CFwe?(bLH!{V5xr(#?#mP?e`8#yHii(+3)zj0|MG zB}ipV50Jxyk}G&v=nI3HUhpch^pOLCdZoH!1_41jeXwP}RD zV*`~_uQu*_qXj2Jp@&*0db~J7HX4h+F$CB+>GYQ3YL2ugAW;(}(}ZV!xLaccf`yw* zrsAq9&-8mLQ`rRc8DOc8vFc40{A^i+RLtEofRP+PTTl;TO8NlG1sJ<0k1Crb>kR3; zw59+{aB*kF(<8`oQfo2=>5YA*43*CK(T^_i)CK7*^r}8N(QJ;_IS1&9odZnCx3vs- zr{qprhI3-s5$UiaUQ?ea^>Wvl@YKctou!ijgVzNoMmF$4T#K%SJ~$}Rl3vNKpO%hj zVM$Rrl^rd9XWSLxi9Qe-^P^mVPRH$vV~a&N^UsN;BEp&fJg@xZ?XutKojrcGO`{O*`~2S?S#Sm* zjy^sgTJZ9KX+@k`SnvCg{y&~txc_}>A#usndTi0i;a`s}B<}nC|8Z~;_IEIXs2r1g zddI~2XRytuHEI$AdCW|UO@lw}x8c1jPiG!!l%YuG?+I@2j^OW2sgmJ04KI@=C)E+D zvg)Q3q}t5ibCSJki)8QEkGyiJy5AMSWrHXhDtW(KE3ERL6H*6ZmH*{@^opI!WIOML zOEJyIOa3;^&$je$jq;`)SD#~@fBbwx`dq$by`ML={pU-FJO1c9u3GLQswBw=l=U2R8z+q4 zId!zu8ld+G(&#exM z>M+EI9Nx*A_IB%Zj;aG>9p&`Z*gjfCB3t)Pb;Mzz&xyZJpHEva5x#EbyzYNJ2K}dl z{>w4w|Bj}eAO7bU^uN(D=%N2O2K}C6&@dlGvDtTv{r^gerAJtthn!^zPjBRIP_lY> zL^&!p7WI%u9xB%HIyOzRRXxPM|0In%J;`6riD;3hxXZTtI6w81)i%i%fA<=4`JSXk zer^*Z*tT#~CPkWxqx|#}B73lj;gXYOGqj^?jU36m9wd3m{_JFxo7P8}-E;OE^9S=M z%Ts4PB2oFLMb#)M^URNma>?buB+9JKiM*}rXs$HP5?+uN#W8}(Xy>EP%`z3M{KT2c z`nXu1!tji6qnSHLjt}eTSg&ZZ*tb+FWQHgE$0;)P#`^@$o{}6md$86%vV!SFl)2{NS{{de z3?7qh7rjdfbIM6sR1_7WZ|(Dw-2k&a$31!dLrkYr0+CmLIKork*IUo_x+$M__8511 zT!@=Kx_FP)V4t&}mVt~YiHkDW=PPI{*fc)E$gzjsE?P8KG)amwR4a%GH-XgGtp*~$ zzuBCV6;Y5!w(LJU%50M-!^pcR-Krv|=)-bBlG%IO{*=Ixu?5pg%>mK2`kejdfMmz{ zv8B?`uJx)+ymNa?*HHU1u3r{=(62+n{bOzSA5aA}cAQ+uwv$hsLsrh)oME?E{uXFoo84rX>KEqJx2NBR~&$81fH;?o{3ZsXMJJ9==(wMTJ02C36BlczQ>_HuW`XDY>qT zvPZq%)lZ=~X6=%dzA~NGR6L;Kq|bIz6_9%z{V(;1=TCN7_@1Oaj_bp4pAcT03SJ1$A?>fa$KHRF+b*Xzlj$6AzbXuwYhDg zVZm;z&zGF86g=lR_L*SgHs!*?ovGv1Q+i{qhTyRiZKf=7bgxuFGi5|T*LwEy@nQKs zY$mcnAI+cE7N~;V92PD-ooYWP+?pDnAq-uRv@1%rnSMB8kwdJvbw@unczU!&q@2Y5 zeyoktlki2S2L>OW!WR9&pQ|~1x9?WJ0J{?@dz}Ywk?JJ}p4V?lx0|@njxBCgL;szF z*NT)cd-w9h?KiD|rt>F9y>!p5)Zp=^8AneKwmEyWH13pPO4-pYZuvTlFDX{tbZ_`& z?O7W-bz467N$(AdV}0kEdldxNt8#Vm`gJ=ERmS%y%k?p3%-WmsXh)O!={*9{)|>ny ze?334Znw1)pPbp+waMBxzdGj(t1;`M=g^VEyKSCz^UZIU`ma6H>Ur&*-SW#H4IeqY zfBuM-^3C8&PuF->aM5S)*_tOEP_j2rrWLFe%1eKO6txUp+Rb z1+JyYc6O90Ik37syWG0BEc}XViYX^HV@5f9%PndZH1g8Vi#RkmCZ*ES2vf#Gt-g9> zt^BBAO>!c{AK|s8Z{c7!!gTtfYo(kBYDo#qDL3jG4>!$=G8II*oAM)mb$sY<%84=! z$u>8c7*(qv+pNz&VL6=>p+~x**%6iUk3}0PS2fd5&tc}o@;&SFy4IdB=~fSa;maaM zWV@Plb5~QDc1C%Ryu+?3ft|89<)W9*zHviT)&U$eTMjGBpSI&|LxNfF8yTbP6yZXP z7AHg;%pKivRpoKR^XqoZKTvz`wpFG{lZ^WO869m79lF&q(o|*Qs{J3d&4sdZ%472F zPa1?Kvc+Tq7myJqT_;}(q>NLhZRBJg%Dwpx&8N2YeVG%H5_r9jM?0&iIw)6ilz-E& z`J}&vYkxP$=gEjAh?E~G%IR}PvtKrLgLLRcPK0}Ywx@o`s^&IS5+7k$U@bE&KfB(B z+y@9?GEILNk)PeBNkUT{RE>nmIbKigH(- z{yN6%gzC;dAA_TE@VhdfQMn2A9o-6Jj9NM4p+k6)-d5d74tK5Tc*f?#sOH@ts`tDe z-K@*WrFOni>0#rltkYx~9r8-0+J-r;Zf3lQHZM+kFm!9zdYg}GG6M?L=?QkF~L){Y})xAO2f+Ep!O4jjDvL0}c_{fv0#A#Msty*W-;9O#oS4&{Cda@97ERi&uZ zktQ44p8k&wMPcRR?2^U>W?Hx9_!@&*_BMk z$|g$l+SEVxFs@1UZlRg`p4*SBmua)C^pT%|bC@>?KK|8j+)Cmi*V+RsgVTol&l`8= zjaprW%OH}E@3^9iw-?8$f&yr8J6tBU;iST6+}O1jOEz}& zV>EulEs}1z7n6+NHo_-7ZUE`5qZ|Qo>Ugo$iv$%+d%PI6Q z>0;Zk&VH44jlWx~v0qBXDG{dkbw`&T7*sqY#b19oJmd0XL$yeb_k^Im2MS zy4U+^N{8l=OA7M`glss#(Q+ZzoMOG(t%@l-*1KoLnSoWiAMAa{r~l^TNsA5_#@HP9 z-gr5y|K`a9Hs0zLR%5)W{ElYi-Wd~r-N^4T7KSIcyOa@neQ@fX7a2A9Kg3#3AW!z* zc-de-e{b(QFAO19_c*=2)Za4gWRUBX(9n5pk~h9Ec;p@LedlsU$hTS;_)NX%tW}C$v^FdO@^`~LmN~bQP!K-J+TsG-_YoE^#H`ng6!Xb3dht6&K!C;@Tw8cxG z!V7Je_NI!wV`DG)ct$Jr^ zYM`;roa2W}M~2#FuegKpnA=B}KA8BYaqMm^s4HT%PwoNx!!?|-y3U94|`U{v9Vv_uIC-P;}EgJds>B? z4MuVobIaWq-k~)H?QwF55H~aLyqFLwNcjDe7prQ1@U!K2u|MfCyF`F}_M`}^(IAGueu9)h~}7XpX_P_e7{uZN(2A9zlv z;e4w4f7xV|Mc#iLcxG1qpRLAhwj!;Ep8rj@8egB%{Jq&|qsun<&e7K{G_=l-h@52h zDp;b8pXqh9Fv?NwlwTS(_owKB($)nfkqPGL0#o#)l8Ay05xhI9Yvz|mTycraDe#K+ zh|DjD%7wrXouH4*FNrEJ^Di=A-*hVqFvS1@;A3GG9gSv zfx$P1>PnQD5?FgBltdjdN9O1wqYI;=g=`vX)SJBw>Y)76)C_NpuGA-CVKnvTm$uF^ z`^HdFK}594H%8wpx-_c5#Vgu?B);+BO_2p=yIjH3T%eDnMM7-qGDn%zRpJYys6E#l z8DomhH2CINn~YitO7JnaPxvXD zmrkYT_);GT=$gU;F04iu6#5jHy>iX*og;>nn&J)W3i;;vGd`^gmPEO#ZQ_N7$3_cD z%+u*jv+o&mBx*0%ACpk(+m4rly77hq?!E(Z%u!e9N<(C#DLSDrTJIa3YiOCjH_GG{ zpI`1@;1f96oZG`S+ORJsrcQKjLV-ER_cSuNddKXKibWv>Q8@)Z^lywQUcBpT#}xSF z7_g0q+yZlcshJm=h6!K~1?HFnb9{73Ry3+8jP9)WjX?3-s)ZgBbfUN)$8k` zd)ks#qSC_Odn2C`^b}{sw zv?L-tf(X1o z0L-o@1SYDurNGgGf_jy^Lpb#8FqbX*Tg z)<^5r&CswIekDFMFo8<-v=xCIUOA-$a)JfJ>8P54js8nhv5Ew5-y>$PgrAg742eP?>>wwwb+lRKoi%ID zzn(3MDB(ZP7E5coysY8<+dG1U0>J09#ebQn|Lrl(|C}xU^K&T9$^PeT@&D>q!pclP+_z`Ihgh)8=5OBR2I4;##(w~MozpYyS1 z&W_Hu&bH#4DK=!g3IIt(A3=DmnIs)+rt-eK=wBt%r2PHqm1E6Zrb;e+u{D=`?-M0L zqJL%cZ%0a~bJ#yZ)fgu3`vnFD*YmH}*uP$Y|9>VMH71FW7b-aCR`_4VtoSblT>nm3 z$>H_LCg~r!2divs!g!MS2!;55lZ&04XXM33d|k=J_%9r@aYjag-P>%5cFL{;I7<6{F=8jaG=VX`3#YFCxC zEVlAjrg7?Ewd@UPN05B|$wpe^6RxtU9>QxIP!nXy!_J*!+qu@SEG0F9NaK{$4XH}S zx0j@QX*PTSB-0>UzkPLq?^2~NJCb9enKsQSpP({o6ir#Wr1L=oRSWt2j})dz?WAuF zylW&`vX=u@c0(e3RGQ2rp`_+~X(}UVf~1nB1UpD_ShR$8Drl0!`e}+3QDC`&v@}kU z!W;1ksCJ^AE-Jh9?BgSR^`T@;Kvi~O5wMD*6H>c!K}%wH%0pJvZYn#;kFWLdg{~w| zX@eh?a;pzym|Eq%clE!Tn|U7zke0gj}z<^Ibw>PqRg|Org2ratX-^8mj1Uw9)?AZ#-G+|2qqUDsWx)_8U zyISL@r1sygQB=RcXRqY|>8M3VAvjqIfq(c#5RZPj0bWNPsk1gz6>A|E$iTP=rzbKF5cIPrjj7cY6rQk*i~Yn4XP?z<7B zksOt`=Vw4Va_KCfTG~qiNo}WS-(NM@NqSi+D$msI~u7^=tcTqv zZt{iOh}Cz^M@4$4HJ2pWNs1gIG;?Srxs&QmLa=AWGim7C;0(E+AYQZo_><;#QhlTaw{;Y^<3=k#kyflX18yM(KSA=4{hY~q89TKfc7~d^g8sZY+Iu7kr(a+ zuigdSz2Y^f&Hw8?i2L#78FcdQ6?vWBlDAksi{Wp9?=%fvzozsOq~||ehRBR@sv5$R z(V%!Ne5LVFWjH8WmBqnPx}=PsHxPEQC8L0o6NIgrA5IbPyXIg^8(|o|n~~I$^1>dR zgw-_VW$YmbN0qP(J$4~qz6|iWB9d2;W`Jp~CROf;C8!FZ-O~<{)O#1AhEjPc%rDNm zhhVl#$xno_FrIN=Jwz}RqkUY~MkhU&KcQUt{64sCzOjXmfKR>2>!5Mih7$hdF}i$f z+r>$0r1e=EYLFDgK0~zNX{Yaj-LeW`e=83L`T3MupScii+`If9>~vwWU){${X7Y80BY`t%qo(rvekYPs4e>6V5g}E zyY5Y3SG#>BSBB}INg%o;enaI`e@DWn9lF7}+nljbe|n70m7-#L zO;tkPqVP|Q;*u7W{>s>~CW9(mG-}T6TpAU5`yj+m?$9*t_lbTU=2)p3&5+K}|{>W(e=)P>I80yq{9_?cSdLA?RJ`vYEmM@54plTb)< zThh~hg5e@GvD8@1KzofG@(pZLsk5^L`Ioh3C@G20MR#6#iM-LJwT;qEWz^XE-~&$! zM?4efD{HUANAzKS*#S)XLU$$u?bb7)CZ_84>TJj?HgMIe3u&TD4OFZ>EQNcbZ&mGR zx_=(VU8#Z5jJBtQ!LJt$P}bh^W8m%;{dBtvuBFMY@4fJZhi{YDbYC8v-sTh{#|dv? z^SeJ8yx~`vnMQ5N1u$tm9l}IaAx!$f#OWsUfs$NbIJ2LVU1}kkac>X_+nHbb(6vRu zUw@WF-r0a*eH!z*z;lo19Xf6qV&@~`zHG+GacKLxmiq%$%B`?q&A zj;gmvW;cvxoJ@z_c_PAw(sJSnxlJ}5mYoAS6(=nFp9U;@3z+_n;!>sGG^xrjTmC@b z&!Mip)y`2ne_H!G(lX^6n{!CG*PhB--TuxGmz$q}cb6-7reIKV2E(N8Gv~UJhrHu8 zm_S<>BpCn``wHmjby^klwuMMQ3CgQa4%Vgod9)4VLUpbOiaRx5#C_Wrz}}L!sh*Uo z9c8eu&m1yyabGgVZlDn&#@Gf&Wuu*|(!yzs#z)l;)E0*DIWCX+29yX}7$ymgs@oXlJ-?-X-I zpqML!Io7!c@l#%$@6&0fN8~NU=g9^XDSN*`XZ8;Fy&B2&14uU$gYo$CM0`6Rx)DomB4Su7{y<*Xm(rdC!A==`8-fp+ zq_;96G9g!t?J#B@_cnck z8LET6hD8P~5@z&xER7yI;|GS-cu*oTO}n<4I*T5_vjHwz6%Bqcq`yC62vmOzMwrC0 zCz!liM|r7Ang}6;a!=xl)MaC~WBCbx1Woh6sNMZ`R=s zpzjOt!m?+-r*C>mbu{Jj+B)F&fg4;8+8zFoRcj5%jR)9uA`|~255Yj?$*4tk6#xKYfisZCzfiVe%ej+l|?v3~%NEUya#G#@-61 zMg+y-J^Vo3OBNyXyaA1nFq<(b#}1<1-%{h^9*yBRZEytn+mA43D4{7L)OeH3;82?G z`~kQ=5m%(0vC^}#G+3E&Ow2;ngv0Pfwu|SH%|%Sqi|8A^2gbtb7F}PLmtJ?s>q(%m z)TrA}(@_-$&UqS{KJ@x`JWoJ0&9n1RzP|R2rjdL^gc*NU?vL&=FI#R;YY!U8ydELM!Mn`!-Jqmhboja!B_Eo@CVUa^`*v?7D{Khv;#Ug|^I#PN8 zrbD*>iqL0%m=#7G{m_DcQPgFah8R)Gti2DBWM+#B77^ixJCGR?>ICd<*N`gX?NMPB z*yH4tAlF7hH*NZp9b*#9$Aw1pp)~UQNCkCGmBXbZxkD7KxoN6P#luXE$*th`1&r#} z1^uv2D|+e=KVBR8eb4JqfvxSA&TE`_{n2kauY1L7#b@()t@xbIN&N!>1|i5!9!8-E zEXATwuKOpkI_9Ek$-}7TdvuCwD$hwC5h&PvvL%19FPY__B z)_4!MWi^>z{1tY`4hUbgQ}~66HD;1s{^HjiZX|?ZhVyjb>A@q`5V$qrK*hwaXjJpP zQboKkw&COiHBL*+tM6?v^sEXr=02%Bg_OpQ_I*(aRBurGbXff-UY454`^t;F4nY82 z69`s=-_sO)0lq`N*&1-Y4WP?cfN7biOqW&6kEUWPiQxXJ>IzyPq9w1A1HhLcSkCDP zl~adXgmyYgX*0(-w6HvoPs{<`I>((Zi%6_avHAo`vRel#{h3M|;o-&lN*>ReM)fQ0 z%aFX;c}#JQ$A~78K5%~yWk<)(rQ%`s^Wjid5e~DcSZQjr0-GH4=@8ff!a247?Ghc^ z>;nW69jo*je~l?~!*pKn)VKkkD;-(HYO6c>M@SE3cLVsm+Rec)GsVGwj%hNvmJ7N_ z*4y4!$^E@ULvnT7!jR?x()iBqN80}r+Y;K;>a|-+jg-sAaEit?>xvL<@DbYlrER}t zI$yLADW($jDQ6CMMT(A>X>(`#jnO0`ZtD!U^a^2G;x%&Gql6|mQG#$RJxQo;F|Dr_ z(?PHp=m+Y3O|1Bvm?=xe7H_8~AdPjqF=~!GvPzRJn7;fiU8=lV+L!vyuOWSIEvyk- zuDp)9cY4Hd;t3HYxJQyK+BOpPEu8o>rG3Rjs>-4C&043yUXNpeEPo-UA^;g#kA#??*-zysP86umgstc0yg$Y| zD-iy>Nqrea>UJ<3n}h<^%!Td2kV={&`A`<=Ng`4>AUykD=F0f#`6y91e&xheN1AGR zN@%C7I`52Fv&1x_V|KTWwSomMkau7}%S)?ZSe9kv)>#h7SPz&}4$DbBT~S9ynmeU; zQIT*W%7>!r^ng9#gQ@|dN5Y`<9NHq5F>gr(&Ltt%^!f9Ud_^hDW@0JwsI8RP1m0yk z_zH(WC0_)56oLF(%tY=l!ViD3DyjDvTH!{vKwtL>s!w_edi5)WDoLrKxg;-UhL!go zAl>H6n#j{MnRJuUq+d&(q!VS*f8jIan74q`kL38O%=Fysmb^Z#PH?Db*m`HM--qA} z#+4HIz9BmMx^q*obG`+;Y8YVgNQ8MM2FSKh$|rA!a{meo<;TS&pxuC>%p1xdsbb=8 z(3b*B`a=<(CUy@_q%gzFEqa0NoeDT72FNI~tEEkV*W5|GzUd^QlCcLF|4Q1f^U$ls znr{J3m)IjvW*Q!8JhYo9*Nsjtm#0jjgnd6IANjrz&TbXJ!bp&R*hbD6yWe0tL-?VU zLL)_(qH?vpV36qJ6vI5ThiO~U=&Hh?j#%fci+5wmbdk!ux>MPKHMJ3YGJz}`aze*v z1_QK15J;?EYYe6D))Ol#^@w~LsiuJ|?+I?lH;~y4fb2D>8v{X&5+%1W$d#F%bv02+ z(>!K;F`(pD1=RF?GRl^R_91^kwK&9?=g#Y>YfN9t*VXrgK>QMH57ZEg;XPu#J%@$HDn~&c_K;17=NZ1N4 zt>NOuhKh*LtRa5C1T4(09r;dpdO1Z1-c~eO4Afep$-S{G>%`Z5j00oEL6aw6VSTL3 zucD{Q3E63=>+m!XQJsrv%I)5ei-|O%57lhtMK_2hUo6>*tolV{Ho=r8pU)yg=Y!RHjrCbF{`fd(ielWmJ?;FUSy>DL9&_WFo>y_SirhmhP5Y{ zW<1?^Fhpx8>3oH#0q<=+9-7DB^`A4W8xpmy+BH%|f#|}M46ohS;nn2^O|iKRknh2@ z3VF=es%PY<+(+PpynJ|sek+IpZzSs-7%cw$$gFZgkN$L%Pqdc z%P!&vUZ7Fd+I$(Ql67L(Kh(iYEZw4=$m|M}GbGZT>Ex?I!QWK_Z1Di}_5@6+1#qej zuxE0&ZW!8iq+kSv!r9}nhAa6brgetH+ihZ4|J)L^Hw3LVo?J1$eougwHP+`D^taBWk}*By z;Q3aXIN3?~%K*Hj@v)miHxvV`0Nlt(g`GxZ`uQnR)l82 z%nn%J*;%CT6_E~6ccM)FnJHM`kdY+{<9jmtrzW`Oh+>)G+*pD>G2&!2_gcp)mR6F_2^3GYMO++*ybdy0>@(y4@~jzt@`p zy|zNIvHTblZ<(NVNH~t~#T8gb4x`4p)g@$!PTMu}5Nl(s(U}c%>IDTbU??M zfEJj>f6TfG^In3r= z6|f{Eo@M9#POU&)U$uqOy_sN@XN}7+R4>d|3>o1tl*)V0vHbNU)sWQ15p1<-U9Pw% zNJ&cnP9nm07lO5*+Nna__)S?!rT*HzYH+Unv9g&Oip~uN&P-?VsIcRldlyfI4sZ1~@#J&3NTp`fA|*^rH}MzNSs&0Lb||nreI#@B!zkT)w5Jd3Fy; z*Z+<5?V%#%xt>9K!+6rqM}36?58mAahYq*B+aqp`vJk}Ga|sFW^gtnfjBGf_|sDychr#xnUqy6WaNujlPn$D2XqpqNHD2E`r4{yrt6O<2HDH7qT7n z^46DuKmH}c_U86tQT^N0Mx9vgmk}+?Lw-cgB6u2!9E8CtT~nHIte8dh!mXtB$CveS zXoreNR)Rjq5%x!=t@M47x>;v7Ws2V&gLB{@5sqm8(s$uZqD^uVu9nto|5ST+1@Wj} zM`%W;c0<6Hgovs_*l&k@^&Jc(f+A(0`WrOVZ_*G7{y-*Eit?iaXLm|&1D2lJ2U69q zsjU8WR$IzYwK$i$s%HN}+1c@us90^&jlG6r$EaB8sumkkfi~~S_)gurJ6lmde8S|l zSA*LpNgO1qTO|`FG-7U$>)Eq#t6-Dd11U3F^rT#CX)N&Iw=}q zYg5%4I_T?)>&Y9W1u!5_ zMqiQmhUX@T(wl@lY|kn3gI@uI#dtmb7|`oE^CoKmxXa``x zPJqK9d00fLPSBs@CEW7!L)Q6*-i&<;*x`pz)(a?J7MZ_ng%w< zO0RdnMV>~?DSN5b8e6FeXvGa8IX9QlAGQ9rrjfFcO)PnbN9y$E2>-}wKZb6~c9v4|e!5!OHKaYR^gT?1 z>ai`Zyxlth>g#0-`mk$RcQ3P$Ryp=%Dn+Xo5gp2&#pq`heW49fqIf7_Lq3 zeVi;2Vc4@u&@MPuwJ+0VrjgaZGOn1}PdOal7FelCF>q*It~e+k>)_Ls$sgEI(T=?@ z^`p+Mu6FRU2opQ!fq;h|vtOW$+Y>bFlU(&V-!eFc?(0Rm$@4@Uu%~My0`Sak-um8+r4rq1?9q;fV+*rKc|!f z8-MblZz-@asVM~xB$KZIS-0OaHQ6!$mHnewas&g<%=Fs!g~W*l9z zZW;s~=s@}VB0#ZN>e()a$dZzP7wm$Lt=@6F*yG3@PVSr0&vuFoWxdoBdtN6d&_e%Sysl{sr6Z2SI@i1uS1EdrAoqKzFU&>>!f|SG1p9+1 zJ|<%*L;VIV{W%lLOQ{)_S8+oRI3^fYCtV`cbu(+R!M>x#FH4xjMm zbQQ497s0A7fsGnojoMZT*Vy8(mUPIYw?tFQF5)J(V~=Dpg!famLEL?K#LGVJKFE*n zLO!edcmg`scC4v(Q&3&$G@NHJZ7Px4i^%=SiPvJMUw93w#_NY~xW6Lb>(8s-%U5W% z4X^-74Aq=An9k#Z*% z0BUUnB%K6s2Zi&+E712u#$WaQytsuV47z|ZD_?5^Xe{Qs4g@K(vKKmc=+^cAhO2qp z=Telup+o4av8%a*q4fPlm`>+k4x{MC_sQ(q_8Q$X3w1EJ$pho?C>gx?urs5y+xvmx z(R?30^sH0cG)t81+L=1)Fy|`Y?MSbODd2K*>(o1 z(fdST9owjoi{vk365yLL|boEp|5A+}@)x~<#J`T{nE*tq!TuAp^d`UpG6>Fj}YBV#p* z5N*aZXi}L|`VYc5cr04{wU}n9X zF{Bt_izwo{4EwIV92%b$fO~5Jo!0@X3M2Qr1!#4f!bl>2eFxeM;c=91iC*ulJA-4u z2lPX_ePYi2K(EM^uuS=7Ib>g~>coj(+9RRNzz)O$FUM z$ATzgM9G<50c%Rg`l%lSK|U$0WY>%_P<8d63)QIrs#)^;sC21Fe)X2jTm2`HIiqJD zGTwN70Kff`n(p6z%9_#BCN6{Ur{y5wWUCBD3cREBD$7&;ipk$)2wO=Fm`Y=u!R@5Y_cAU*8IMjT62f?V1bi^1C#G(0vxhH_73ii$c-9M{#%NC z9YT?^a5AdD7|pbNn01)-meq|`SvZKFV@V3nsA-LxncZhRs%)kaZIo~wN~ZotMoDKU zUeAQAqcP9IOQlmQx&7;MuYM_DaOTYF0K-WAKe1{?!tbE zjGG2eApoDe14fBwN75rn_jqmZm(FWnoT7600&6Yh=uyG`!trgu-pBS(^3i3>RMtC5 z`#vGaP==Gd@i=zvx3_Q_o>xpZ%Bm0SU})V=^v~n^Nzr*QQ;+@tGq*nXU@&>by?iKt z{DV^6_L`^TqfMuvao#J=8eQKtg>Tw>_XUpV{Ag{$`Gb)aRW_$uf;gKI6#q) z?~vOc2E4oj$b1YNy^Xjt^?(>kQMqugFHbR2jzOb*^ohrI&A!xIZ(nJlv_<>d$7VC= zHDw=zSUaPz`Ye7@$5blsoIwrW3V-+|<5^C_>wW&^&ldct*vcEd2Lg%4E3@otu>S3W zY>IZVC(T~#~(Zg~2UhyxfhD7y*8GID3bPujn`Ac?Z{46W! zivu`?oHvWPyQ1t$%SEUv|yGKBQ_b zcIH{L$g{;}?+0TOOk9Qf|KkS`=8V47^eTNZoIj<1Igbs{ptSCE?H_3{<7-1t zf!m6-;+Nq?mW`n5IF0j5xqcq6H;i0lRn|^ooX9a#Ksel)L*5%~S)ThG;P?VL_TVSZ zA@2Q`Og~+T=jE)wpf#%IYND(MarekSw6LJ?Hz7?ANahlF}f-Pz{SF&4RNt1Pz$-0Wz4=7>8pIUDx{IiwSvI-4k zQ^0!tVi|vT@Z9BL-DnMKY+MOV#Ojx|sJKNUM3-AsB{UHTp}Bf)=uuS@DJC%RE;Gaojqr?Q|P<(p`Idd+RK>DD8 zh$veGT%|ytAkg0@6OJn%*d%0k5$~jpU#VVbNs{-kBJ<{qh8XA3nY@NpZN}?7H(oEL zccVx*%}9z|#%;+xnEb@yvAmXZStUFpgK)S!o#m5UI|KUb1?(Lw7jDg@=IZ@~V?OI9 zR#M&_8W&4N{CJ23pxoy-7)^Nu`BeRQWWM+nua9Sm$l}N3(^((~vvKUXje-P~`*jRH3n`3)w?0|b04=}4gj9SW~?_1W( zpF>8m`y8S$L+@$$0~&oQ0%}z_9Gx(Yb@+n9Ig^($b-cfN2OjT!hsR%0q08ZMtxFeV z8J_37g2~`QKB5-C#1mKz{|sEOh`rzeA_mDN74QZ(*B_qSNVvVF!p_y)*vHsdp9-%w z1^5!6$@*^rujvEDSE|M+SmH?fVG^LhV8HN^0R1R<3QuOnz41|IOtlpA+jWe*>JI^n z-%{STp9jCxtNabL40zetA6nru28sjF8UrD-#YI_{gOv%x#LkescRnvic$u((m)S`B zj~7xFGsfJ=IlP48Etmo1k1GKygk#$AiTXy`7&UtoQ-TRKIS9MF(^fdYG5vYn*U+2S zKZSAc>Klk5tZd%^vYj0$96~2sx7_Vn{aH1{2$jF;#2~kh%+6r*kc22xPV55qFfVn) z30*NSHk=_^*C|&>G)zY0H-^D<^Kh#2EorA&rzCyR32WQ?s|ZOe3+-%uLWMUWuke(n zh#Ti}buny`?+oQcOq_3+IFd(eU>(?$Yy5>p^y@bzLT^-=+MW$HN;GN_ZLgIl;h4QFA zFj-(6PoL>Md3C5U1(o%LSaF;;lMx{Mj9M)%QzjNnqy}*_P~Kk%CqfN*eKFM}i%H{0 zA^l5I0m1}Q)AP8RlUMYN5{+k*ElypB%q4qN38|^17XCk!-Bnl>U-;}=D zQNqFi5m78a1-rYuy8{(L>F#c@I}pSGQAz<7#XtpA?Dl8t`EJkuJkPl}H|OFr@0v9; zYi8EuUNd{|MEX8+oGWz?ll71+6K*y6r{jN5_5a;B7@y?PN5pjoXE8YF22%{WNnK2d z(>=`BIVZ<6B$XjeRlCN&_Y8{ID>QHZHsHg@_!kSP{%NBxu?(9gk$vYPPnR_NYjGee z-Bc&hZ=K1toc4qReyr-8cai-{>&cusN7USuR~9qOgtw~sM1FZ41>NYuBPXpQ@6N|~ zT&6kSmMn+d*>tsx9VxIW7Y#)RmL;qs%c9w~Gbo{4~ z8996(Te!9*jl^21O^zN{+o{~b2Gi2XdvJnnd-WD0wpZPb^r1#n4LGVDE&6sX1I>B0 z%h_L4H|L)lF|$C`)x(iUV4pD8BdP$Tt*nf15S^r*zd7Qvmwb+jLG583+v|4MSk~6nqaZ<^1uT2s=7bWXa;LELyN^;LV*Kx$KU6Omw` z`n1-~j{M20jWd9`Xf9rqNZY6C%(;Fii~iG>0iW2@lov}FYTqS^lcV(*N}&x+*>fzT z7>#2=)Jk=vap|z6CvKS0`pV$(dpIocNred1*ImqFHm?X|nm{dCh+>8RybG zrg@_MSsojVK4N;7^L^5LnmCoZ98_8^tLLP1FYLUAGR`E^%ByuZe#<$o01r2*S|Yb_ z7%H<=_hxGpG(0JPYtetcKAm%_llQ7obb9S#@PM>Z4v5BCTz55p7V@b_S&zEVYX0`% zd7e|4((c47rpYa}>fTxNeirS=n!`J7 zW`s$ZI~l>@JR>yAawU7{M{ly*-BCST|Ml`a18==zQM$iookvsk->Eol)!SI3=;3bR z-`nJ`$=vXk&**30Gj8Q)I{IIjpdG_icTg$do<+I-KklCXPfO2BGG=|{0^!O~&Jq7E zDu&-UMp~);;Jv954NCaNNLt)z{yW!q=DT9b+`5!Lh?AYtPIarK+n>&1b}r>Ddphe` z0L}dIKRF@BtF99Rspfz0Z`tuVM|0ZJ-|RE?TwIhvs>|Vjl&-J39I9^ZbR8LE*yFz( zo@?oy!Ivtu?9^*sv}vd8tfbV%Gr6T#Myjr=yJyqcxRlHpYFeBKI!ZGtN=Ka{RR{Vf z)eY2HX~~^IB6*s-tXnbVWUkbzn@pvCALgh!yo`A|@$dFibxmq8h)U%%-@n^;vmqSm z=Z7(u{=@0~VwY9^980H!eNP$8nrTj)X;8yYbJ(EP%=xwIRx#idxmLd11yWdBZdAB{ zbAwG2gr?V=j%>~&*DT0Y)%0!V_v<>d_{w3S40}N5+GD29%o~xa;UDRoRb8=+Ggfhkl;F{0DxvgH-8^Ot()Br; zzO&`b!S5kOor*7@;o3Wa{&|V?@2I%OQu66Q`Kgi-J5y!<3a_cUGw8u>s^+GV*SVP3 zWRtwBS;S7!wie@XGScK%$~g;k;{2_wk<8#}33uqXP!-!NLG_TLp(;5)LG_TLw<@Wd z$Q!?^mMP7;3GYbp(M8iJ;S1*(yLh!ryRB5;x>wdvjcIA9Nktqi@68$?OR%H+u0GYP z`Mt+9x}K>EId}Bg>z&2hF|SWD(Hzx|wCPMB>k>zynGNk?cT$NIv!@l3USAvd3sqQD-gPlaTgjVQRTCX_c5bKYua4qrPUMP z+gzbJLieLFi(vGHE3(q%KSu48YR4KR8%@pSN*&#m1=PA>(=m^0YNjoNw

~K46r` zW=o=;Cm6HYc90lvLKQoyZVFXL7+MxQYEymWKqIw-cYb* z1D$*|H_gM}JFpDyH8@0+4r~t1E7yiIPTzmuG}~FGYTpW~V%O5lwNbSh^sC8h#LCFp zR8+kYbUa4V**l$1AUn5a%AkMmv8m3ulP8x_MJG8;DATD1D za>@IArl6P$)wmp{Q!*3H$~^9>v?A`Cmhg#a@GC|;z~TObdL6`xUo-ch87I-EhZyrJ zO`#1mSGAfz*}r=+{VhI{Xa;_q$F9RUU$bM*zxT?zs1Cju2K{?m+qyR!XL`vo&YRIr zJRw&$_hg8w&EY}-L#p5NjXUK5j&RN1DZE{znX-=x*Rq;ub3?66tCsrUn`)Y>wqDK3 z6F=BhHYU`$MfH{0v|ueGoO{uYU-Xg*ip!sssu^e{x75AO$Gxju;Q;LWFmf9I|mySP3X(K8c2St6w~=ePRwOvikzhN?9r47@U-Y~%7j-+Jm! zh6DDagaHP-X(s7LkE;FTGK#@<^51ok#%rs8H!`Y^b~{nY6x&_7fDUFKQN4zy%<*D} z?$7;;s#Qj5=ttje)s}tEmQ`*_qx0?po!EGMELvs+mos=5))j++0z;d^PdJv)0fcRR{@Ybi+8N~7TE^}id>tm+~SS#pg%2$PK#jX$L7_F&+Sk)+d*me zh<-jQYMwDvwMu*C@^uf*hBPnJ2JXRuqWsOysJs_JCq`A>OEc;CpJ$>%Cc53Fx`vh3 z;um%9((KuZ6S{KeXJ$HJEKjL5>!*8BN(bS3qqXh2@*Zu@{fZcR|7LCW)3W(Z@3Ws~ zY@~@@8RATVnG|?=-E|I?GM4XWR=o$biPtuvjZ$gNs%tGBJoO<{xJ0T>f>SR|dqkVW z*>{Ny)==H#1f8bkSu^C^Ych_ql>eGVoAVuJrMqhT&njS?A6Xx%jqcf$Hs*~yrVX@x zjTMwuZrZ1U(%RKmoywvc)Y7zi(eOrv>i4u>yEm>R@?&mg)3ht6{a-)k#y=(f&p=ZD z^YdFeigy2xT!lh$u=D4RP1Q;8qjXw36b%&8RiPZEsJTj^>8GfDRH0R@sP|scpn+PW zu4+w2sp+gzYv!k>dsI!YSgqxIHG>AyrmGl@5|dTJ@5^>LDyGHK`Mq>)pl;q(-C~rw z+pyRB^4C_qF00)K2JHJE?!|-ch=q)SNp)l+6RkkAAVo^ zXoEV(yVf~5s?OG14itC(zT<3y9>ym+1c7Ls$Sz1?jYURag6&%$n zEY`aIUaLq^x5S|C&Hi;uXVty4v+muvy7w>EEq`41k)mFOLA|Q}^`6eESG}{|i@18P zF4lYVxZXQO{SOB9KXtADc~t%XR@MLJSO3S+`oD_n|9M|u(XfG-HBgRjP;+$y4W|aR z;v3Yt)SzxfgZli9B7ZGu(@=YM!=|enHuG<&d#s^eNyC;O8XEA&f@Y13MmI89-Kd>^ zqYlR!nU*x_{Gn0ThK`33 zS+H7Xk-yH8V>(MO=`63%S*g}^b?c^UZJMs1-E^Z<)6MZsw_a-MSkZKcS~I8C&75tT z?VjDt)v1|#d^68W&Acm``F?2T*RZ*N>*j$r&4Xt*4|QrD9^X9jQuFAF=CNwJ@vU_e z&2*DS>!z&M-RrNr|CsK<65YcebdNS{aonuM$m-*}8JEs4jM1RM_5BiTA zwyZR3`DAp~Sj$(JTE3}h`A)6Xht{n=*|hpHyVX~xR^RNW0(6){7oHlKDwP~Nwro-_zrZ?Jj{@A9A81kRC=H`YLV+<|V7+M7w z_Bw9Z`-WlPkB0pl8QGc}4X`yDILF9-jS)ZbJLI_0up34rJ{pZ`WIV>)c-$D{32Tfe z1sG2`ZanRV@r;kgvl^MqF*liKYchY1$--SGixW(iW|=InG+8NaSGQ@q*0$~XIc+!Y zYP&h1?bfWej+Je9NV}bl+U+uL=Q5_q$778;9yjlJa!kk5 zYdWR{bWA_q@!XA$=RbD5*vK@?+%((PG-r(M zbh^E!Q&~W#d&fIH$m&#H+3AsVu4vP_%C_^ud+J7&g%T8vhzFX z@}W(aPqtmY%<1xVSC{VzU4CYD`CZxNuXI&2?5b|pwZ`188qQs7C3dZo-L-C2*ZS&a z{AqGyJ2UOMW=);VnkSmI$TrijGHa!7-rCUI(9Ya=u6bK$^Y)469kb0lRhf5D?`CG$ zt(#r9?sL2KaPHPKv72>vw?0+f`l(yk7+TobSqz+OVef1)IMHHgw#D!&i;?QxM;mq@ zYuA1J-0l;dyH8H+J~g}h^s4SN)h%ZmTF$kzoIlrcp|j=UM9ZbwmdmRwSE~0|ZP;V2 zU61v1du(*>u{p8F*6bdRRXuj7TR9n8Ionz7o@?dmY~`M4<(X~eU1jB~-qYW(XP{lr z;JH0RoqL8S_KeK#8C}&gR=roeVXs8HUdeNNr8@W8m)L9niCzb@dmXCkb@&sfU+bfd zt&bU6AMa*;a;){~wbnb*0V|F^9Br>KX1@NmqCk@1}(ijX!)%{E1nEmskC2hWWRQR z{rY+K8(r)-C)sbkZ14EQeuvV*$;iQZfWz*24z4Z^{B*SEWe4vk4!+94e*E0B#o)jJ zgM;S{4s{tEo-{b}^5EzvgJYFL;*EwR4j7U=Z%C@kkbOx*4qP5`=*f^H%Av=MhMpKO z^whkeXIzG!O&XeUd1&U7p%;|HE*TBW9xyCt-mt4K!}5}b62vk5Y~wRdf9)&EQeBPmR(l9aZnks0P}j z8+9MuWc+BI^`n~wkJddkTCa3;%P*r1w8ymRK89a&H(5WXUGSI=r^cA(jOp}rOqUvC z&5Xx(8#uQ6{INZDkL{T});ecwpQmH{)fi`EJkD<5xPkM>+3y}VICdSjvHBH z{AlCxV+W2OKY#qh-Qy=GkDr<|e)`k#GiywkZ9HL4_X+dHPgt;i!lK{_OHNH#RytwD zmkF!1C$8x}aozZd8`e+U6g+XusfpW4CvN{Tai{jAUEL?SjGwe;{Uo>DliZUhd7hfI z!>e?X&zDJl+LHsiPYxPCIb{9hu;9rNrzS_0PLBC9IZk^@yz!L8fm4#_Pf6W9Wnc1? z136O;J)Lr-#?)iRQ%?+>dTRdEGrOmrO`e*OGd1(+)bn4aUeunJ)qUFK@zbuXpOzau zE&tTC!klT>o=z*OF}=ii`ptpUOXp9&vwQm8UggYq^K{0$8Z$o_&-^rS=9l?1zwVy-J$dHOQ!{^+&iwObrb1_ySn?ASvubXb zr5Q4-cFL?eS7z0HHmiQk*$qu*b0KdsVYbeO+08;`>zAAgc&+YqcZoiuIY)t0a z4VpJ_!94pt^9HBP8+v8l@MrTz)|@}uWd7Je^T#ikKXK3e$tm-vUYS4r+5DL`7tA(U zFn7>``3n{-+_PYD%7Ud=7A$|ZVC8=cR_QEUW4UnMgoPV6EZh{baLeh1+iox1{@=o# zI*WE$E^?W$XwQa4ZXt_2PA~Ghy~yXkMSePq11uK@O;{YVVR2Z<;)v6Wqi!#b`H#E2 zCGjRp5(h0wUa%x}&ysy9OAcIFa_HHTBQ=*EGg*4va_PwlOHXfDnijG&{q)jvx0jy( zZ|OyyWm%TXE>Bo?Wy7-EkY#x(%L=Y6yY_5ZQO)HgCd+RQT3)(f`JFw>@1`uje`R_3 zv*nL!uBb3sQ8j4A(*-N4_pEr4vf|a16>pxccvo}f2a}bb2Ce+EVCC06E5E0#{CQ>N z?`JFj{Q=9+ud%ve+trPFtZp)Kwa&)X%|cf< zPhH*O>T3P!)vYwvv~IelO^-E(_G^q6u4(JKrhV#~j#t-os$SDYW35@+wcYI3T1;GP zxpA#k=-OUq*7m-$w(r-q{hO|{?Xhma#C3xx^Z*r#;sR3I#zGop|Q!S?IvgYO}iIva&_J0p1R5N>L%~%O}-kN{o8I1 zwBH=OaC4~Z=J3?bkykfIS8tBh*b?7%OQQXj)^_U& z`>m%IZaw3=^=#_ajH_EStG8az*mkMywru-tISaR4b={Vix~<^qwrkbfiZmQc+B)8} zcPw4#c*oW8ZmQ$`tB&Q>j*m39SG3(;WxxID!tK?r+h3$^e|2^Ho9gZFG>OMME^@JVa~ZtXWoWL;@aHZgHFuA0 zw|lI^?(vIuPjuTodGGG2xx1%7-#t@v&+K-4<~r<|zi7`ww>^vZ?pd0YqL zw{uqa-%&3j$9=DIpQcio}s=G4y3*}-l1A~#nzH}}16p1E$`&)s}A-Tm9S z2RgV1FLDoca}VF^9+~SN{oFlP(<8o}N1}sA@*V|N&o^I1b3f_nr=09pbF-gjxL@tFep+|^>V5ZX(A>XK zPyZ&9{dG3`Hw*XIJ?pP`*T3a=e}m=${CR}o-~i*r0d3s_+V2bKm>1CLML?HYfoAOk zyA2NPzBsUldtlFff!5rL+zsseBCzlG!2ZpHYe`{aghDyAP4@HVqehEyr5xs zgGRgv8d)oNbo=13gM)XB8yq}-aqvX<;K}=fr{)Due-S+6d+^L!A+y_u%vHxtu!MhQMzegNx9(lZH=gab=5s8Z(meFUevW0QAM?)OWH@@92{M`IQouz^xb{Y_w%C5UqnBu6;shZrm|n_(ko6m zC9dX{IL(N-+Uapx_u}gPh-;u5->6r7lPU2!TjHBV#OtQV>)nfQ`6J#yH=#|h1fwYl zCR-BPMI>}cPcXfg(D_F~SKUPOUWpb{5-qnRT16!GN>A*4FR|~B#QwTTw!M-DOi3EF zg+Ibb8j_wg>|WA{A4#KhlgIQ*9ycXF=XUV~<9tZRGA1uf}c-B(BJ*E!a(<1ud6n@l~Xv-McB z$YZ)0$Mo(WYx(n-L5t&UtdAQ_J#Mn~c)Q5s9Wsua-ap>?=kcyBPMBMt*kLjCgyq%~ zR*@%qWt^}sIML_TiGFoX+H^c=H}vGdr6=t@PYynCa%jQH;jd1P_<3?vi&JB)PmP;; zYQol2lOj(|$v8Fb{;3&1Pt9s^dXDw!c~eg>*m`ddk-Hy^-nrGtv*ccM9Sawmv>td~gwDMfkdT_Dc zuZs=zE;Z_XsmZiUI@>Nai@MbO;H4IYm-Js>YWa)ipRCrVS%$;1jF)A#^~!30Fsox> zR;SllU9_^zOtZTU%kDlc+j3jBRaADbbJ@KgWcU4*-CysrEuXfAT^_jXvc1>k!3Qr7 zExbJZ_2rRTIipQ;#tzFFzbt2>SI(rUoGIsWraj1+@hfMR-jzAMugsfvWx=*9i=wV9 zId^5*gDWe3U0J1fbxrT9>!w}Zu z|GE5w5AqMc&Of46aLlye#IS->%L>kT6`Va-kbbV<+=GJizX~pD6<#te%pO*Fd0OF> zZH2jBg?R@H3knOby)G=$x>jO(?dGs+rOU3}@w#^R;I;dO*UDdCd!%)}!t{F8uESB2N#yuSWUtLTGi(WhZWUzQbp^(y*)u;^!D(eKwqf3=F$Iu)xAFRn4Y zxTa&VW^{4w%wnzb;<|5&>(?!5*r{Z3o2 z42Rz^UVfvk_lk`Zvw{+_ac})6(&#RrJkXnKyfv-|YMQW`F%# zwta35n0{-J<1L5iTSE@r8hY*4@He+c)-4^~sdVh{((%hnCwiAoK2$pOTIuvRr8Dc^ zp55v8+~K$9FTcIe`}X2Px0hbKz5LDXmA`MV(!aB&&z-fy@2p>bXM^LNP0@F@WZv0U zerNmdJ3IBucJ(Q9nO?TXvCJ*H%p zUS|1&^74nj%OC4MtnBmf$@GWM93NJDKYVfM;j3#8-@JMFuI{4`ogRG}{^-l{M_;`k zeLwW*=e0+_${+pt{YcUBvGjec9PzluipMp#Ki2ShtQqsT*5Su>u0Pg#__*HN$Mx$~ zH0)f_ctnNvj0&CY6-|9AnjfxcalJzSZAGhkm90Bh8jh$mUQyZBr?UOw%8u77JH4&! z@~5(E%PRA}RTeXh=U)B3zRo_2V{aZepX>4fc1C&fISa{lSGhfinxc{;1*vpIdA&71LT!HQ=KeV#2o{A}s< zXUpF{TlweNs+QGj`c|)-QN3Y%^`@BWE$6GZJ*?jTr+R10=eznocbV~g&-UkTG0#2D zKlgh0+~?18zj`nHJHH4V@gjJ|3);y|iqoU{KN)57?hO<*n>VX7f79GWvsWyfKXcvx z{Ex=lqe`7(Pj!>G52Dmeb{^rMH2yytFdQSW1e>rKRY=HllO#O4=_cJu-NXVNZ@Wn- z!m#zOo0wI)i8;1DbCYdw#Nv-`l8QoHLyVfcm}$C8StEC8X6!DtP2EKolRLZ16hu3? zODrNrxr^T{ck#!fMecHXrMp~44s!8*ox6;=rl%MSvn}poj&86(cUYnatk4s^V2$4B zgTCm8shEN3n1-2{h1r;+>UX3cN8H@ya5CeaqaU*|4|Q_fB`x1w&K0v<58WjXJHENg zPK>^xrx=6ZY98_jf1wZ$QGf!J2Ji5wzn45l1?ms<5}n~*@?|UO9bQu4 z#I)XCvJ&U_Q^p}L$;k7PbLdmzCF!s96f;Y`WEQU8_L5wjDf5!FdtTDuv6nPN&`U1~ zM%E86$%fZoFVWZVmclyTlKfUr5#P{T5|D@_BqN3VMoqk>F`D3^wzs^1o{qP)L@Nx( zSWJZrb|V}SNNedWXOWIFoNVna3)*_iBJAtxE$U|8GUUCU;<&B1oWM!!8|W=h?7ihF z29EQVkF&hx6TIem%i|^9@_VbdSnTi?OWgUar>LFoEp;&ClDEvn@|)hW=MT&G*ISxv z`pCjMK2oc#kBl?*5y$WRdsHtUQNp6Pk90?lgOB7vd!&zKjPa2jn|#D?n~wzk)Kdh& zpL76%NC&DSf)RpHRhs-TVz?@gbf~Hyk@QEXkPcV%BZ~fL#2^l_h({E|5{QXNLNZd2 zioMu}{WySwIE2GEf}=Qw<2ZqnIEB+VgEX8)Ix=t$nK+LNxQI*0!X4a38Ghk6ytn&^ z4}9T=E2z1HVXi*X%FRb^vh2Nx*658)+#~Ni@d7U560(qu9NcGkKCuATP=sRKz)jr3 zTfD=2e8fZ5G`_6AKkJWAfj;sXE5ey3l4($i+t7^i5wjQ{F-JF8Kqt{hnxa7pX*9+2 zTpxLf8l^s>fxo}?6!q`>h}s`Lg#wL8H%1d^LkCUK47z9oW0=4dmK8qI16G*!+DE#6 zXE~6i;VW(G`bs;@uJ0?poqT1in!X~O7=b7hp%~fSeC0B7ptSN8v!1@v&%sx0U^~-S z7OwG?Bz1koHWy!U#CGg}%^qK|gDnQ2cC4>BDfJa`iM|q#1SBFGmyv@jNIB#ym5i54 ztRn6wJ|P|;9>ifhqrVzY@fyrusg-Xo3v zbeu&7&LI=$aRC=`30cTS4ld&Z(-sqNqYU?O7x(c1<#>oksK8$+4*QBg9W}A%jIVe? zE6rExq8?VA^_A7IPxqDHMZU867TZiiUttI%IMTltso0N$Fed*nF|Cw!z;krF!*)Pn znXgL>nls%`N}(dz(@4?;w$~pp{AcqYv3pIYw0T%U=Ha;#D%JU zRT|5%94p|6op8fGWTOCuxQ6Q}LJ97nxS^jY8u^JSW;F4W*_eYp+J54SOy(2a%ug=j z5^`u0)YSEpTBwcJsNcd*nxZ+X^!(%rGTZt|K7N||iGL?Q2}BS|diu#tR9Q0~o}n6D z=lMzQQa{PVP27k7T0aTEs||j#Aj(ggG-N-yzRq zgvY2rC92?B!ur7trj^vu8|ql$FVpc-*I(x9`^%`7{!&L_BWv)Imqm&4?Dn?yyE*v>3tzM}`zYqjwg zL-ZzXjXu~#e;AT+5Piu@C-x(rBWB_PF5(ihk%i01!4+IZF7i=`Yq*Xg6r%*CxQnNF zh1ZBM^p{vX!ef{j`O9eLJB~OJtFafiJNio*zI5@IpZJB}_=CStbfqrP2KpBMVgMu5 zv80~R9f7@B?mqtFZNvO*{iOn%?fhj6w!#rR;ez?;V)Hqf~T;JWZj}zFGME!ORE%r z(Kx_1JmfEG1^%L=r>{8pjCl}sah|j>@d@!OzCo&4F0_B{FFhI7{e{0+!V1>tjf_`p z2Q+xizVgOjHX#>z$j36KTmQ~qHee^(e)E@hxTYQ;L2Uvg7$FG7h|U4hr%Ql5undsa zJp$y{5c)?4$guGN(sx3D%$gb?K64p&d4LSw5FlrE1xOk$AlRF7up}ZtCZ7qADfo?? zi~z|xM|s%+5||SpLvjOT6nuUJh%fYvI9}8PMFW z4Kd^u6N?Z_x`cQGaikLvkDK(D;uaF=zfHV@yC_2vdH0C-@c`v`goj9GSPCAKu0Sg3 zy{IHzg(rB5mw1J@_<)c2gwObb|L_$WN0{eP=8u@;)CFZdKN%=5@EULM>{Oss<0)Rj zH#1No(fned7{df@(GKm=0UcqAEVyN}tuV7NQ0CwRKH?L$T?-VKqCnY$)gAN|tI(t* zP_*HEgM92p>zjcxwlq)*A5ho0hAtI>(AK71P2FVl**9sEj`axoXQtcpV*EC4- z%z~tL_aJ%GBS_vNvlG{-DM4};3uXpMB`iDZD|)~RJyA_wZ{jy%A7ZmvLDHAl4>lMC z2l7g12gx$hcJqQ{(flCUxFATTEeVnkq`;?(zT)oMAbEgtJjCBEL890iBx>lnEl5m# zgQOF>pfeJJf@EPzkgUMPy+NXVAV}^X43e%#f+YA%kTlE)l19)$KkUd1lJ-}aCqgmm z73=#pNGv`Ci93RD#a3T&6?w=-J_<1HOOQ-OA?aDfB4RP-l3suk(l>Auw@`}PxPvm> zL! z<0MWY4e7XyE9lxFSh{09YBywgF&JO8gXKGF=md*SvtThqTU6noez4@X3zhAr ztTv+zY``XL#%goYDDKHPLxV*zFIcqa2TN1v!gX1&e0BQD4U!it_VNl1{^TF~TR@@1em8jh>M7H5f{}9Q;AN;lnk&Ct=avNe7BHOWlG2>wI zvJhE@?#n~O5WOi9=O5-Uhsww{1JdaY%vIxtys}IREoNVN-+*mo|h%#!v}r{ zL`v^a*^5-{!_LW}@@0Cce8o3>#}ABK6e=55g~~R^P|2RAuegjHvt^62kP?2CS|fi3JX00S`y`Hbg4EFcac4#hAG#|VtXD2zrSz(E|sVI09x9K#7*MG4AqACK@D6{tiNo;e>cp4GWWskzw+AS(v1UhRK{TVnmp1i4PO^gfO|aKwt5F zUznsE2@{j!VN(B8nAAf9G(r=sI2|S{u?nNp!ek7_;$B9Ww7VQ8U5dlR>}Hr)z_~0; zbndatnEH@qUc|CLCRVWQ5U4{53v`DiuD%MBT;w4i1t`R+4`DLvcbLq^2}QV^L8rRm zGIfc*VnoAmIgK+&!&#&ws!_N^BL=aE!@}0#GQ%ufd@RDn7tMNx%Wkai8!p9z!{r_Z zEvGHYD_oA?D30McCisNQL`=eDOod-SxcDOg%VJ0$4HxVDaB(XPmpx_SQvEhuo}-mk z1nmqF@?f>T!lh}1?8Y9r!VTrkBjmJhgq*?|q#;``Li`OPzaa{pojz5VheE5;HbJwoHnUqWKXrV{3$X`A3Kkd~xqsggi&)N%A-9D}HB0$RDJf zi;z^*IUgZfIDaWZF2E=&LX6>tZ#fY%@>+z9g7saNAA2ey#Q8Z9^M6E0RgFl=*h0Gr zo{_FbCg~T%mw1bJc#jYGh)+1rI2Z7l^ndt*ulR{y_=CTwQ8Q9BQ3qP6iw0 zG=nZ$Kp(Bp8f{<*W0;^VI=~d2(FI*$26J?S1-io$J_FdFr{GjEKxiIg$8J}Odfj-iYVkuq!x>$W{o3b4|f?E||2rU|Cb zbJ+h1*hgruBI+uv7JA&jWHRku=xl3_%G)1mvyWWC6DVy$(qJdqN@`n zn~b7lM&~F#_eM!R3h}}^N?!Mm605;c5;8qXLg79mO0Lh0k`r^HqeZ`v9 zQL+}dPzJ@;C_bx3iJ+64z9PmYN@5X*c)UdYJyFsC4bceY0a4OtUzBKHiW1MfDCwRb zB}Ii%V*Msc{NCbglq~udC5}I%q^DZ6Oi_=P*&5OEzFD+9_S9EYz^Qq(IKu_IF@9jQ zymE+^>cP?S8Dob>OEHQjM@#IwXi30c9K~@&tdEupc(6TMp5ra*?TD7fXpRD0LlMjZ z^cChPC4C!rP=Z6GMWcWhjOr4B_||PMP?DU#b|v z_K%O26F7xF3DMFQ_He*p48c&iq8R;;MDr|(ybQ+AWH^Rm7?xnidFl?kZ~+(Lk{d12 zg>3U;w$B~52Xc{zd=%m??%`BevDioej;WQ^%ulQ z4$*{og=j@wjjN<{k%u+(uR}iR^~4R>i9-6X!Gp9Xyb*_Zl#riItiXFTU&!)eEM{XK zPNIVGDp5`PE#BcHzThv0En*$97Kf3Kay*3T;ux_;f7rkQ!{G;ie8q26E{TyUgsq8@ z^sSV=Ek^1%vc66+l8Zd#qX0#C?aH#CZv^!d79%Z@hlWux(g=-F6dNPOC_(GQ81Y5; z-WZ9%p#3po4+jiJ{HYi*ITIsoQ8y<>np}+$ZM4s48Lu<_jTmVILm0srIuEFy$1xII z5hEdpu8NUvFJq*`w;1X1D@OV#VtKzKR^lv&D2u3Kv@q_&D z_=yPmBN2sY#2^;G7#2rNKs*wWgk+>36~7th5B8GYhy6H!gE)l4ID(@%hT}MalQ@Ob z_{;QXh-pwH)8>VID;D8wFZ4i9SRo9PB4c@BJeBiCLJ9Ihc!in2!Zmh(%ZeFL=WTzL-imXNl>^z&Q*wjgvvJM}DU`DL^5v z;SC0KWjK^(aZ&^B=5aE8F7reE`Ek+!4bcdV;de-1(Tdm-t7bsHaHE z%!m`cia7aL6(_Hs#!1EdIH|~(E2gHlb z;CL|`5-;YMG%Q|DjEWbB(eaWtHeUWLiI<-1<0TS@H^qzV=6D&fC0>l(*PxOK{dczI_Fc5>_fFYQONtg^9`vkE?%#H+E=bs>h zqZ7m+M_-|xpCCGDie_jIU9^B6^wAQAFoGYhJxh=xlpy<8f?QUqktkPi6}h+$_4= zQ4Et4AeSZHcNtzRli6&@^ zc4&_d&}CRhVkelQ3pzued_A;4SNhG+igZhulkNr!bcZE+peL-*3)bihI}E^ZjKFA& zK{%q2kLxJGJ(S}Sp5rCd>LrOl9qk(?$@fM{@&i8+*d$2;5QN9DZk8my(FY%_!= z+gm2_EG0?W7_jVyEIVdn4(1}FJKNZfu%2U*WY-kxZZ>rXEA&J!RBvM0 zeb@$mNn)6kBuDP)D<)|ocHG%B+Fjx!+sQ@7$vxY zn|NiCEZ$v`Mfr$#jCv%?9^#HU$+8n_mHLX-i;~3S?VuI zmT{PbCQFm08bg;QOE0DgUCuHf1Ltr7myiwX70EIiE3pc@Vb3(JRwauA(SX<*ZD0r^ zj3mE1(Gpg$$0+hfV<_oi7>*GbgRvN=N;7;saTaD{F0LXMtyfb%dZHJ!)+LL{`ebQ~ zc4&_d=m=AELT7YASD3*Z-CzMr^gvJaLLb<}0Zu!T<@e5H*|aNJHe(C6VjCQ>9onwR zqJySrhA6*exr{4_2v3%D>@H@T;m-q>;Tg|g9wo~+e8(UBh1z4bAC#zpn$Un2>Y^UZ zV2+Epge<(NAg^wUY_8Tl5L|KYNA_~!nK`c_S7YA_|M{w+MiX6uY_%dzyvlIzK1pMfaBt{_` zF^EMR;*o$uIE*83t4^tflBny zO%*Hbfh*z=j{^L`Unp9nN+H%6rOKzasqz{1Oj9Kf?wwLayK}1SGfR~;E9Qy6Q1na{ zHS|DV^n(rJKJmT~R*g!P1RN$kW^}5YAhsEkDuytE3C@jAl?-HJ*8EhNyf9U!VEnpN zsk@1B9aF^^CTI)ooh*|d%X)$JMR`uDJj5TBkEvq&gXgZO`k5+E@B{KIRsI)yZvxju zvM%s;cRG+@&<+F)7_k!qCWuZzQ1l>%2ndME5)cp+lqicLyQnAu0|KIhY%+)pGJ=4> zK}1A!6hQ&OK?DWIZIBUhVI0JL8RvaroO9-!Gw0qrclq6S-#gI%?&|8QuIk#l>Z{7v zD!^ml_)kZ$VZ01GM=QhB)ylAmfdAq$%*wG0EAlAA)K->ZJ*&#Fw9qn42hl@(H<99IFM!h*%dj(m z7{dM$j-$XcAgi?u(`+xpv;b{jGH?$V1U>>k1IQ(aJKz9*F*rKHaS1>HGY(Jz6~GDX zRN<%wOaK^wI$6|zzyKb zt8m^6Wen&Ah5*Gi2m{y!qyVWv8nCpl49mY&h8+YB0i#1@*v-d~CQl*H0P|;MSil&h z$ty?`U=6Sq*Z|n6>LJ%&mtoI=SHNpP<4qYx0@DC}0DE7CQ2-oB_!*ubNCI{NDL^Wa z3G@S_fGSds!N+kiI)4`vb9#Xfg96Ju200w}2z#!o3T8^y( z_5gc<=PS#xcfhcJIaU%-j+FvO0JqR`ObWOIt-!|Z<=D&X<=BL8%Q49v2p@0<>}OYC zu7Iy&1-2US@vXqpH&tL^+ImPn91DR;z%Z-=696-SfQSlA2E;^FV6lKrTm?2Oz5?65 zvjS^Ns=(TTOThNj3M?|G0&6%_fo&+Qz;f#=Fv`ga3@IffnF4T)P7-eqMoTOw~i)zOBICzk}xi zb0K~d`x3YV+y<_z9L2s;JBl3!6oR8z5|9i8l6uI+*eYN( z;0F`{2Y`dXD(O*}$I?R{c^t(a11G$WV)ejG-=i3ltB3T#u^+er`~YaJI*MrnlL0oM z1JnS=fT^pGVp3pU&{51|$5G4^=*>8aZOl4~wPzp2SbRMs?C4R15|(j zrU4{i1k3gyI0mn2d zu>nnZb|7|2CAI~K1GWP@fF$4!7t#;-4j2IL0ke(tkgsM`VuyheKww;nO$Sz(LAorc z#4f<~AK>^D_z@Tdo&jg9E3wl+&mzb};2NNB2hZwJiJkNVf1#DwMPSn=NRP})Ed4V* zPP%W-D-wPaE;1_Zk5ncl`ujzLZlkytQr%h8k?cIwMI3r zTXn~%s+^!Dif9UFT5{MQ(Y5NVRvM;OmZ4Tr^ACzv!zMIjOlYhb*Rnb~p^0ELix?N2 z8LeTA_6$bHuPRo%3H2V4`ZZ_uzA*J08S0=D0dySP9aSG7CJu@w-gln(AZ#N1djE0F z#F6faPe&(?5|hUMJ^iY*eT3REf^n6Y_>3T!Obrtz%Y~`^N0qCFYg+8noPSKy@~WoQ zGfgp*B{5;yxUlTPS@!!_j>lL|S6R!Rv7DJ&t|nSimw&2R^)G5wH(b>YeWo48oE%{? zIm%^nO!(y3eUrByn;ds_@{VVd30vX9P7Y_M>|>`LW2awb?;Y2vnrWhw?V^(# zu9LSu*HfA#Ih0C;8;nU*Z z{DUf0+?7;`AZeMTS}n=wAt#QJga%i`l*@AEYDaL{nOxmkE~kg9H^wDFiK;1&@5(cb z;0ZGSszFusZ>vvr<=aL45#6b-hU+2>12YYSYYjK_7>14+hG|TXFr6OdIz1+0dTi$O zt+mtRdZzCfoBnSrPR(`|=0*tfGKKlI!h#-Q;h3;U!>HKQ=&-9%X@pT(rcp($QDu)& z^_Wr3zo|9#g6n@kWopmNyJIs4G-eH&&bse9>p{e`mrp?`l8$(DT{HvCj}q5jI9yj6in&>?@AvH~TvrYMKriPKGf-F;`TYf z^N-KA?451(e70EAOfuWd#?8zw(#$@~%<;IHQ?J>w=Vs2D=B~5NrEcaPk>;LR=HAE6 zeR|DTJvaB$oU>;3oB+2u>mug_X3YscK4(MkoY3cU!Zhbb%$^(NHa8}6Zfw@vt;gra z_0HY#e6C#6B5}5b!p$N%(jq0xBJH?EdauRa=N1{7^D<}8%XXWW8#ym8YhM2Gc?G@m z3ZKs_(wtvBd;Vd!`K6Kb%d*BbrS^`iN!9$Tg4Ey9j|$3BK{M*OTGY#+6BQJqYA)!R zJ+2Kka>0%NfGX7ZQO$*8vlqT_Tlgw+;hU_5?~X70(7W*C^MwdYjG2jXsaPdSOv@Il z)rlF`e$|3%X3diR2UMV5lgxS{5wRATnk_PuE}9dy$Rc~u{JKS!*A`j5SR`iINX%?( zq&9X@Hul;7A^oRH<)>1+by0SK*>=Hob{jzR=?l9s*5U}W#Zl75F;RzhYm`%awo^x))8%VUT`!!vSxbBVkj_)q@-egJFQm&~MJ<2x->dNSy6dc$t|Bcr zQ*$>ncegpwZWcLi^G~>0UU##4=_b~aO3bA;?ozvGseO*r@r2arx^&q~sk4^5tGT<> z-Q6SF-80AC`-HpCb@x>--Tk!w3))UW5y%P8e^=9KvAOqQckj|@@3I{4iWA zde>;JtTkU*=f1K&dSye-%El8b&s_f}YEG~FbiMTH*7EH!_r2!s+ZXM7BggmV3E$h- zeeb^X9ne}eXuj&c`>F@gtKuK#ta^N6)yVZ#PhYMY)mlAfzWRmx>Q~XL-{h=*_s^7^ z*84H~{3gEgBeeZB{*Z=K?cY;x`f9CMJ3ulgz{VrME+)V}H^8wzz^N}_*{c9&ZJFyF znbbq(5hL@=m3i08eEMXoUdjBl*R7eeF2G~mx|nr=x$A=K*KO!q7y93*+f+Nacuw$P zkKodn;IiD{iu&NnzToOt!8O|dA;qR0^&2ktZRmQnp<8=n&zz0dJT~^lY`l@X@n-$T z+kG4FzS=mT9XdEC^u9;vgP72VxuK8iLr3~TpS}tm)!sBVXVVLhO|N1$y~*A5u71;p zzD*y0qt#Spa~N%ZnA*uOMt|7E*I~ruzfozbdorBUAFlU0oE+C@%3l#-xH&?wKf>r_ zgmHhwtk)5u$&sdW|9g~~u9+JXup(yN=9s|!F~KKeHuT4YzK#i-yg6d-=BO2$V>WM& z-M@M3$<1;7n|Hk4ET0^kI5$?YA~tz*Y|8%Fw3D&v{jqyr$7W34k~w!v_KGdJo44fc z-;#fFOF{pZ!q;1hCT}gCyY=vjt)-i{mhInKadK;A|JLf)TWcn7tDU>8ZpF6x&D$FG zZ)-fc?M(l+v#+-`O^$1x8+TzvTz=&5XYTfEE4KG--hN~M z_M0cS-|pXj_x1Mpfyp}t=kB<_V#kBcJ09-e@%ZG9k^UV|U+);596vTU{>6&;SDWMC z?2mtUGX6t<{Kwbv2wRT*Gfk%dU204%6gHj;yI6&Np2D#~;dDc>?2W>io#bkfB=t=4 zh)wd$OY&|=^0|?;>P?a#J9&*oa)4*@y4d8vyyW194$~WH)o;>j*t=^jcGr3Cu8-Z_khiAXz7SEk{XOu-kKMyE23zs;QWHdCaNWjZg*%qwfomMn`evgV%}S6%vBy`|$yOC3** zYb<@6?W~jIIxk1+mE*A`$McIE?^8KG-zrs=`sw7Z`A0paZ*#+R_D9UyALX?_X3PHA zFZORewLk9L{X5?7m+Ryu{*P%WJ^S`RlTJbNyn+i}1+7~O+P^61I8|`@+k&pQ1>HIa zd*&Uy=5?@d%fa{?UmU!7>fr5f58i!ya6sq(H~L9$9@+8kh+MZUaekS?yDWKYSxSCc zT4P!I&9c4k$})7zGv}9Qdza^KEzip@&u=U*xLIEKuDnRMqIiDAVeg94trcbY6%~yY zl{YJ@-&NG;9<80PR8CsI^=QNIX(nymTG^go+0j^e`DSI;yUK3es-F2(*SxFxwpQK9 zue#Y-b^B)3-FH<3y48d8tM7YPKiFFRFu(e7WA(^QrCQQaP)Pb0ZKU~se%JWt4>$k( z@!g-1DK*%iYVegcD%)yk2Wr$#*D!9?OnhHMOgX0U$5oO3Ruie^t>aejkBg_&N&Zx4 zv$D=^Tb=!ZI>*y>PPhKB8q(1B^6-J@>#hr~ zOMS0U-i>GdLQqRdV@TxLH?>i!ySWy!a<|6gT{9UXZ=s85Z&=0 zu<$|f*#{f$J_!9!=nvIH0X*t@lHZ)eqNJ{m{4LhZ}`I+&ufk?YlqR{rJOx-qS&=r}tMqeX!%{ z!@{SJ&psWw`}FC@r=xm5j#>TqV%3kYcKrCJ@W*#&fBbOw$B!R>M5c{m3rF$Qqbl*E zv@b{1&W$p@8=d&`C^7As#=>W;)z7r!pRvDurhD!g=euWmKR+XEch9YUelDK&LbC9M&FU9+@h|MZ zeBpTRh0}L0mi_#~dD=_Yg)gP6UwXv9^!)Or_qmro-@RP*^Gm;JuhuMl6|nl1Qt4`A zMiD4o#Xc!rRf&{^hemJI!n7|h(0eWO=s2ZUDnomwYmBKpq*Aj@k*~R>^F|zOjPHP%R0K8$g(9 z>%bT2#KMy7V$a|d7PVDifDb3nW9);+Dms))6DDO?)57=eli2G#o;QGy3iWPhA$&K_ zenwZx5$E74)YWlaKHUS`9CKy+No2vMK z!-N7!Gf`=ZXsRM4f)Et353eR%)ie+9Gihb7m18RY0g9`wq&$6?6S~sXQ-eM_AsHsE-Wz6mv_k$b zrg`mVW01`(^Sc(oj8Pi5liNmcpAeFEwt|Rg_2eu37+eJb#SosAHoqugjAoLo$hOzj zHyCbbrxJrOi)MW7#{2<-f{=`vRC|gFQrzypHr=Gld8hn>SjerLlV)F0Bhl91!<2Bs zyY|_t2oo}DJm>+0#wt>g&g6uV>DJoxE>zO)y>Z`(K2in`$mZjw_DDj8iG&PDCkj_K zB@J;rlUUBJ0yuWb|dVS`c-0W_Rhn2tLIw6Z5hf{Fk9Iz^UPag*D_36Efnd;+PxL&E(ju*+p!lF z!J>KnfOJ{5qfSe$LrJ|z@Yy&d5vmvjgMvWMj>o92o7xIyXGjzWs%Njarurm9$H z4a4*%du@$LJGs*Wsy~aemY*S}anolec6lGo$-HT_v+leKHzFz#AJgGL5Gi^4@JP3?CMXPDS|masJJCirK#O5nq+gqXCcBplfR3NbBwLPJdf z1yzGb8*Q9lCL0I$-{)H9nGJa+2Lv=EOHboGHUIFB?-=QBLnK}c3u;Dq{&5I zdEy$6Pnfpq?o7YOq5Ctin`~~nizBpn@>awz-9)PU6)IBE(WS(1NI$HiyB3@x0d=M- zUVK7>Mu$S%?7cSKr7tpc=I~wJot3tZA%_`~KJQXNalwp|bY?R>Kx^28#?7vg3iQot zXEh;Gmo2sBopVRSQJ50yc7q+97Qu4zHep+Ah>RzdY!o)O;Lk1 zM@%F!F%mMScNIKxJ3ZI@@r%zC?!d&baKCuMjzA~a}G zxe4jDb~B2yT|I=0_LX=Gs}wa|;d{qx6w9=kz1DabTEO%Z&cPLgsk5uP{Hkssf`|$! z6B)zepsI{MZ?uTGS`fJxujz}NU;(8F+FEaYMRuS4j0qWow`m6H&d>^~*UEJQ_t~~b zhAJQVd--#WD`#Z36Nc3Al#Gma5|_(c*?S0yqfS5u)n0hdVzD;aGEeNthICWW=PJ%4 zQxpo~hO!-3nE(Y-fx@jw0#xQO5 zKFr5F=qL`Ud|{r1hF;(}X9PbiBii*tG7>mpLntO;ca_wuc`D=@EXm@M?k1xUrmIB2 zWk6fLv(lthkS<4C*&K@7Oa|5-8+vMpw-KAhk%RGslt(!`&6~kRR6ZqsJ8Ym4%ljc^$PcSLh<|5e?-F&3PlX~ zArz`ELP53Zcm@{?6b(4x0R%#^o`q0Rxi&HZ%_E1!-UyXTvJk2su5O6xax}nN8Yc}$ z7dO7Vi*hL}KnFI?A5F%=WXq{iExnYO>ULp@g zcX-Z%K=}y$-gQmdd_;BOH}R$jl}Za%PiNVKYc_(GIZax@lVfHWZU|02!4gz&Q_yOG z1ffiw1IA@wm5ie#or9q00UE>v^|Js+R87BC5UazOGD<_S6eEPblZB%KxZZ~%6B5cg zQmx@JDOOiGneDS#Bv$wm=XR7^wN&`1`TsVI)B;id>Z+(v*S2^-q5KXss~>ilQvA6k~{C-G$#4Hr23* zO##tsKb5vLx^^?ou;UDrwz1Q{l(v6X9DcXx@wEiM7Cj9cc$@MW}Uf=gW1TjK~@x`aqGmv`qF#NcbE=EqMHyGDS#qyKl#6fs6 zS1uhnrfxwy22OuYXMvAzF2f((e5Z@UK{=e?&DxdZeNHOwke)9)49^C&^Wg-oy%T=( zI$+br6B2@F(|%29?GJMWzhey*_XXAYd+0I#>)r65dp8i{?*^`z`~$BBhVs?GrLg~# zZw5qw_K&)MdNZhKK^~$c1GgqFiV3?gQYDe3!ImlLNaU-n#pbo&tn-FE5Spe`ePUI-y1JD3ffCMZ7<@~>=UV{8k0I@(1ZK05&gL-9c-{VDxu?kf`5vt{p!0JST9A+Uq&kscetrG1# z@0XI}D9_yBp@y58Eo#b8W6+Wjzcn-(l8lgkWjc&%Iq5f>%|LeEQ#n6`Bs6{hEY^oH zsSR1v{V*cErwfE}vBaV#gpOlqKDT=U9FQhLgzIU3d7h9vxt)k4*AiTIF=xm(;T#>rHQMlB8g$brK?B+n_)z08&G5GeXdH?p@n5C z3@0u4#R4=-xUv~g*28}oUqU?`Z@*wL`LFu-U#NfoD+5W(-wq_vZYpBf z^^>w3{VKVb^h-HfyD2mvC@N@E=r0}pI-2%~s$tqIC`^MdKNTir8Tj2ol%cA17+!@D ze)A*tM(>S35dg&<2FNIFHa(r=2N|!6U==NkDbnun& zu2dHzb5MdjMI%#;xCv>3FaJ8efhsv* zqLpUyH+}q1gBs}LD{C$M$+f$w0@^=X!=+L7FVz;Md=LYEDCM0lO8Kai|J((`2Msvk z0R&1pI}4?LjMqjdd|dw>`3T*Zujiwx zO{Q{Fl=|*9$bSfXaLuM3cA8XnC;q3-pWYjSOHZ)lz)>mwK`DcCz_<)mihn*T#XldF z;vYEmGX_W0kuO(4q(J{jksO}rgAq!Pd-DFH0PfI-q7!zkFpoJ4Px-+LEL>rFmTI*_ zDg8ky53*5I_2aqMpQJzVK<|~(ACw+W2vBO=ms~N&3RQi#{cK)BB6xa0?Ae*A9v z`|sYr{z(_5f9$;zN1&<~zPydrD*N{@K38Lb?YW;8s>CA%jbcquO&GS&Pa+818d>De zNmBOjNSX$+I1~E&cv>PKKV*Edn800(DJB9N!kF=uu_IwTAraiuc0z~Tm2|WwS_p#H zBCsb2iZ!mr#BnCvfv_Pscp|yf%`C-@)oRUJbdRNI5$z3UQMe>6!4fyIbhaUE_wija zMNZFHlUnQ@5=r|^GBuk;ZGjDU;2F5`NC#I|>ID&W%I!08hfJ2+J`sGo&>;emCi3Bu zgNrCVjPDBL0iFSzYPBE^YaDW^p16d+KgxGUv5PjfW2|HMW{ zq;G$;Z;yd5xap4$Pe{<}4-a?kr}o!xnWn{pOTL@^r>v9vZBFpr>^r#y@4p`H%M3iE z9pl@bbfYhNMUC&&J`<-R;g$N0Uj5Ox+^!)80358mXu`hy>btG#!PWo zx4cg#3(99?Ma#T)+%c2&vTzTt{4Xwv9J`~vj=*E*4z0*7$%(RdUUtcBsrB3J(l7E1 zFNqvntxCrN)x0uG#sX!JEcUv=SZDq+A^f&sJ_F42m~LLYftoJHS z3eGV&w0*T#>7y-%ArZAtwiq3WC>-8WxIVma_)_igsp^r&+!-yE!#8VfPv<^qtlzjT zbb6=Z&=>q=x9Y9Kj&0nw-J`R9s9_wKF*L)AL3=`D2|Zv}5WdP29e-g`!%cM&yTJ1}8< zGwuvtf1nN=4_~a`aG-AHff|R9gTrlx4w(fGZ@nBcJkLjOU!BRH4E9Z;TdxiJT1C<| zE(V>ovP{0gfw&c4@O3+Pe0fpi;T0?^&yjtREz1myhcsR7sFiue_%3!?F}l5PMeuiXmwJ?M&($(@%B*1>sF4;sJ7g;8 z7rh7B{v9`M|d>FHf;!hq4`0o-^>%?6Dc z<4%>zCyhx{VC8?b?Qm^qn<({GwDIX28)r^ zM5h_TDeUQad{NE@_`TRl#OjiYD;);q_VV0f&*`y=7q=8nrzH+-Y4y02<0H8^egB5_ zlJF^Q2Zt$%(=Q(!ertJqOY`uSR?a>?39F#MaZ4cN^sumC*r9z)Ki-tqQES}kz?*)c zfG@3eiAW3;yeq0To?gcAX2~uI(mFiyt%UjBLA)c@9U|R4jrIAZHW`&e?R zv(+=KJ!Os6);H>xhg9X-x+r8!kFCiYw^aviD`fQ6$3rRT>kIX-*}fv=*z#N6N1{zH zzpfi=tPcrUuJj7U?5OsnSSm6P6EUI|`s|7?g}RPGPQ$k3 znDK#{)Jx;$12te)T6Wettbx5L>te0(6k|9&-=czo{gfJq%o^+F+R7T-_`T9)Wv+ql zX^~U2Av&c-xpk^2Et`~gxYK)V>aB!HH=+lyL`N^HlGa+kOVyJ)8)rSqgaq_oS$!lV zI=641Y;$!-@zV7x>o=Z;x>c`Cdh1*D)ll!cl~r})t;UtN8bjffsGIqu5nRA_rB}JH zx?ca~^;OH?*TL;V-`Bu?$&|{F?i1F~+7nG-x2!f!SlNfqb^-^T^~-N1M}@#g!mUu9 zx{V=TaC7S=sxzn7Y+TYi)3LX1$w@{_edWcT4d3)k+ZZ2p=BhC*aT+bLXvwXO@2^HJ zQB{hQ+7n-_xH^sIaeciz`V%#Zh_iO5S`*#Tx9a^SsQ6v?7`dLP)n5BUTO#>Bfuz;@ zy{A^)x~N54b)Ks7Ov`h>mV8?5>HVFJ#!qS|(-Kj*;S?$3s|`iPct_1fZTkc=@R)H~ z%`KJCDS-Zv3xbjlk!gnQ+9!hoISvm!a!a%>_Z;%bjqy_P0g)A5Mdqa2vZA`6F~7p$&~OS>aVhoz%$ z3$D;}hrs^$!Qn`&Pa9ai%|5Rk*Ks6!>08U(6Xqpj8*@;P~9hclO4#TA{UwdBH6D^I}1o2Cnu72hp2)i&6l$ zyw~}=IYRP{^LKT^j^i6UN(E_SAhWY%d__!o2ru`>w86ysViAL8^vz+A_n9%i5?CXN zzPqAQZV9T3gO(RrCcr%h7leLgx#w2HtQJcsq*snOp`At}E!0$*rnW!xuiG&%kDBmF zZ9>PQ1y{Zg=d@kycpkQ=85N86==Te5w}{kKE%`e9RGERbpO@np-+@pzrw)?Lh)W9gHTu%{cs1pt^(6c?+w6uSAR@;dz=szbJ#>4ROo(W>NAXS7DBZ6~9Gbt%<~^ z4ey(TWZYk5b-zUfdwzMWa{p}xbRZ_Z4)ZCIsv_}*y| zt$2MS{B@vrn`rs4vN>Kpc_&GmH5O!WD4`ZjXbc@Ft$or_I}$v6Ni;xDYJ%QqdGX zy#bei{YgRD>W&60%+! zQOA`NhawU(6hkl)k?;{5RVQHO0gphnSR`)_vW-XON)ZtSM=&TfVh}K6AW%ekmRKL+8_YB#F)hv9%u|;`89*r zdfP%mis5+>2?l!+kEA}}KvX0!ro+)kW$;-J_`0c^m3h(*Bn1&so%yo~9BswL7^=z^^C48s26eK>Sj&)^7zV5L#1u$-oaFHg zBrFmvAo#r?HN<%IFA)sg>4S-RrW8mMKIqd#5I+Oc+4^D|93$!a=xc*`goox@KsMnB z4qk0=7Gi)?^>Ga`mrFzVBhz+ zBW-xxholCw2ou|wxsqrFA>oNkspw{g0YV|UJPws~g=FASsMQa75Gfvy%Y$sucQxdZ z)8&wBeE6WQn?6Jwqe>8rLglUTgU>Ykff?kt7)G%r^-XaP?omq-VyX`l95#MDWQD#R zPPNW4=;Ff+2d;}uP!Srn91g0 zm>7|WxhfnR8yf@>OK|Sf08^?dZX+@v5nLqJ1X~_O3^M~Tw?(yn$VVYk5FH-L<8BgQ z2ukvB31WtsbDK~f0^?~QBzl{hOcNkK|+b}*O1}krVq-sKl`(WwJBLwv1vk)%C zP>l975DCdwH-w7LP1p(rlAFaqpq3=9PJrA+%~2{M@oz!Mgd+?NBE*ukP$2;|7ssrH zD4$A2umoG$?Aaum&POmrUo2)JBtHSdw0?v-`#(WZG?hBUfR+hE4{~oKm(MWNO)l-<4T{9|HXYQcLnzsU?d3 z4~s3GK~DPdvWXsp2A?6 z(60-Y(8V@-7yr8U=a=O?>VAT*PqQ9WcQG{<=~NdGmzA>%sE+v4N~K=+Ul&8sWAP3t zouIYDv`0Md(xC3lfE7%xD3d(@X};yWdINGvoo#&e(|inEhA9+U3wT(83(lwqGQh}! zKG22nnGvK_-4Ab<(g&11YHEu3#;0h1^J#rno4NqG0rMCA>T(*~5}i(4&>ufP7g!6XNr(+^*rPnQ-HV&Nu)Utf}gB2kvBMO=^& zooj+}Wxg6QzI`(POx!PD0g~sSyy&3aGKV|`Z1eVm6iF*E*T6qF$pOSkNiv;2(tgo8 zhi^r+<{GagcSdd`2jtnVAU*(>GOwNbG7m$q+_maEqdjf?z4}C+UTeIP0>DyhCpc(w z@G|r|?>8@(lzkzkf%_JjR=>z+QNRD*%z(*r7>oLKk`?N0_X1X!`1G6ey^4KS^!xUQ z`BDt$Et<=?BX?M2#@IQV;m{YoGfRC@k8Nh6%rTSS(j9$C#BLF>TAW!9rtBV3)Fr>D zy|bfwoEgiSbbJlfeO;n@24uEZwOp^NCtXwDDP{PYM8#`_Z(luTaH-$m!1Ilc_gJo* zefwRa?!@@6w6<8;AMH3C)$h(a_`+m`OO!7+nia@)adDnTbxgY|TMa+!T@j$RPw40= zv{P{`w%@K&7B{WLC7b8iWny%A4ZFm{XxG~mI|q=n32r{fmg(4+&~l5TI9-N0gZ|mY zb;9Y6MjhcPiw)C?U5!eLg$C9I0-A;A$> z-86x4Z*hUFoZSW@t3zM-=j%<^EqxTu`8=@nkhay}zT}%*^oHP4ID4^|UROD1SJ3>f zeU7_Ib(8%AgST?hN*(fx19f%bR?3_jIyALpP*CDhC@beU3WN3;!0qW8-h zrmyoEn!!#cX|f>u{4yPxupp7^m`~E47*#emRwv4Huu}hIk}VOk^SR)Lok(UUbF1KWg^5j8z&TDRN3%q$W=;$T2qxHc@{4GaPztCOBPClla&kz2LTTvoNGf37+7F2te zeW4ptQQBH)7!rJ>)PLwJ-O{aoC2uU-T&fofEAmpO?h5DFZOPsx&>3>k8)9ZJE>B(K zRXy04I_R>djjMO0z4k$P{ta$nv1cwjd1~?@z04SPvA^FgL2fZuZzv%(SrBy7&?7I! zKR+hrfM@PtjNbI!QwJ0L^A1m~;I2tFOe=}?8}ZK{4lfuvRI#Rf&91W0OvAKzp5dCC z!u;!zR)!Y}O9YM;^+7$6Da8WGk5h6=JadQKa*JuHi?^h-p^!9Cpt4J74&cUBC?Rkv zA=TR?EZ-Fqcp!?S%9j}Mb2Fj>A0Kj9e9C>$Ew}Bc*m;ZnV<=hm4sB@8d6%>{FFlz~ z&Mr^AU5BUcGMow}7mAp&{D#}y^7ptEbLsOq;ZV4*{T81qhd8Pc%B|<<)E~orVYy~o z38y20RE;20%6*QIKFtZ7;cy>XB4<~!eBeZCQA1Fh%NO~4T1q+jc`|tHJVME64X5oYU67yb z-T>^~(t>*Xs;Q+BaHt;9LF z)HyicS*_$1t)wUj9F;h$NxZ;b&0t3M=Z1NSBxe^%yUC>$lbmKQXIB^}*^dTWqH}JE zz`mHYFLs8c)H9g9Liej25ceksiJt2RW&2yIl;|uEDqvg`WtW;zy7v0PLoW&Ko5N|@&J#j1T z$d;Odur-p@W2&uF+fL95ob?h(X+ce1@}=<4*4wN3XG4Tb<#e`Tf#eihrEMm}P=lMd zn}m8pYxSSi$t*ljc2JgDFl_zVl2dG6ZT4ri#s)Ao8)x(37ilfOU* z_Auq)AH?3BkTPKn+x2adOc2z$hTXY_Oi3W^z0y>8y6uUiSuMLbFy(SBd-t*Nyz$Q- z&m`mI1~}~=;>{}YXLr_S+nqAr?V0G>V7$A*cu9Fuk7WN4PdE9Paq%(Zp;}`ZL;&_d z;Ee4`PbPgG4|&W|HjK5#yZza+Q`zhZlUm1T&N|oFuYR*7amCi;d-PSX_l+$aPSQ)4 zy@ql#6U^Hsf-QueiQNh#J?Qj{>GEN;(AGe%H7i#YUV&flda*2bC;{5q3!a^Z}ZQu+mZ+E2xdS_ z6sD>=q-I)AeXuq^yU6+ecKgBY)8Ng%D~#40&cU`%%M_%(+MacPW@@H@&J)5bE44Ua z2~^?NGaX-VpKy}QP@Q?>BuP^hW?;5=Ro|*5Hif@7&+;*jD2SzVru+2E>vE!Oz-hJX)J7(^)*p};hU4@j*3VS0lOb|5jy;A$zR{AMIHE`U*9 z`^ywZ`@g0*{*R4z27aC5SWcJ{OpqlqQ37Ft=4p_*5-14eQ22*aCLEXqTm=JRLZIN* zhy|xg1IhpgFnA9DfAF(#&|9qmU-x11z=$&8Zc0B~1{ewI3Pcn20OzO%Y7bZEfPcgU zNSR6tV(z5rdzfpF9jrY3=IrtYCJnOrJ`#bgq&NDoaEaUl#?jYn{qOkEUp9S@j^ z#=uPswF)4piwy8R0o6pc(V3_HTpoIs#w18|K7+vEFv8)%qiLX!7ONO~D3?UPW^qx_%gvlQn*q}IsM(-pLKo+)1UB%Uw1HFm@cRm!K4y|vkS_#H-p@OykLq6YirnZ zFY>iqn9fueu+l>Pa3+SH;f|qs=sp1pouQ@W4mWVo;;Q3JrnMRyZbM~iX>sW|)dhrS zvsqd$&M4Cz1?4x~Z>coM0g`2nPQVGSbs`at>>@L{8)$SQfqTY`U_$PN`N4zhqWTCo z9w#t=(mIV0K~|7l1T9Ah^><6Na7a1$RS9)1j%ZqM_Mheqsi_Tx1r;%kPzsN$0ne|h zL4^rj4eJp^m%9S4D~px3wS>V&=^E5W*4Jc~oC8|4$FzPBYOiCU~IEkkW2$N)qA z;`nUJ$k15JFHf>D<~qTd$zX~U17C1agFJnd27VV*9aVO$GDBDDu2{M-6ccnt?~U34 z?Ny$)LG5f&4L?OpcPAtPi3a|54Gd#K(PuvwL)1pW{}NaB@Ec`OS80vb7}{nkYiDI< zwU$f_?Wfo_kU_C6Q0k)*OC(!`pQpk&=BA<2#6?+8<*aI^Y-Neyp`)c3c)^BCP^BT{ zZETO)E;Dp9vSCP;tZRj>^0;!CZK0w;T2%P0P&QcJxwg1fRk$eaC;KZcZC$f=^6Iqv~8d@d9xZ7uu;-$R+X~4Gpo_i1SN+FY?RW*&>crSy<9lr|9u4^9P$6Tf;ig1 ztlZM!z@imOhf52=`+s`C|LSZ1?-j&#j?(Idd zHgs`!6@>b>2^Y^KtzMCmY{dWUmWf+K#K?0=K_uglEa)b~G3 zS&b6k|GBxjH*;y`)vB4R%l|31a-j&5({)XA`4x-*$)^6l8fY*q{$0?thSpK->Oucu z+~9Y{(}UJac#-}YZ}#8eb^G6LO6?*7srcRPzT01^<`as$oyiPU>ZYy!nM}@3d`|FNnOU91jbAn& z8)*M_#`hRL`StVe)eqvEjs85)nB<_;-E#M>F&}Gu@l@&`*#CWL6rGsVeUKTYM4%dq3f2!|0#m zUwTvZ#b4=g;Ozg@wHkwx2gJn3r}7&Ed~$@s9hc3jWQQxqP`K|$|D|yM?+Wo>6|nk_ zw&{{RTAfYbw9#y=%HJxT6#lWH4~Dv!D`H0I$%;h=vJ}6cd$EFwp1f!3cc`yQR_C16 zeyDVjiqC6t+@G{Y_yrg$kZ$0BZfwvr!mR=*ZKL=Q};}nuw}U zveJT_8z4`f^Wdtr^=u>^6&y6tp=ul;M?HGFNoHCNJwvI#8ZZPTTLVf~wZJS}K(cTl zN$MiFPE4++DwXQa1Mehws#3;&>Og(Qeq^PUN|raf+K|@%=1h9GW@84WG@;pHDNJQ3 zos-EO$@0AbC)y09he4ZDpnAc+ z1geG3)J1xdgtZ3YS&Ky=yuY=Is_;8No$tb7TR2qSVV*R2izCt+nTw zQdYm(TA|>(k8tUELQ-FL}uqrb=Dt{`{rV zP_AbXu&RnXy z-pP6o;fBj{X#K=4m-J)9rBuw;UH?I8Bik$4Ovu}`?h4$Zx*wSotyVy`tn39OQA_nn zX(7K+HU~6E8xhEsT7tzbS_7!AIoS0x#AkHrLf4^P>3W~`Y`QKRwOA$d!x^%oisv@Z zOP*>bcKK*M7sXpL+G$L+8PIedR-u@p()FAaZ778B^yNWIG&CPm zX*^cXOW^=uJZhT|ZSN2bAeh;Jm5GXa%clYx{&Gm+*rK!36TIp+WxTtG5vE=i{% zDi=`sw3ted5kxZ%}IdYWoTGd)&zf7aDzf8cJHJnzA zSNh8Eq_ydw{aHg&xfWcfPs>yxvPI}hWYdIs?OV2q72z$Gz}ThR$^O_zc4|pE%+pBj zHRlSz3B|PEC0O#oFvxA{Usc&i#iuD3ouNE5;004Q=K8-!S9L?rP(9{{@{8X(sVhrS zUK*%^$aRdORMnocDn$onD*dF(Bz8?{NV;bz^DTWFMdh3Hi6Hx7KIG`RSIN%*P1fH0 z8td~gqdIQGvK%=0Iz5^UrABRhkJ+BG=03WxqJ#HWV2?L||2?si@{68@{NPVd&IF=c_J-c1y{d1@Ou2oIH9Yoe;s}qf z!j4=U)fUFyPlUj%NhIg00Xm=XQQ0WM5NdR2BrLT~%%Iiy`ip2)-UuxJmh_cG_h(1#e7%yK3$lA(yoAw7 zX+P=hRGctAz5@YQhyTWQ_42IyQ@KnUEczIk0QlpsL}=XIO?fg#L;m$LoATyYWZTxr zh0yU}VII`B`$@Mp5*DUy5|#dSPSasCcd|EMya2C9Eo_E7?M*dl8@qQ1y75u@m$Fo` z4B_2IDQ5(|bk3=u2KD0i$f~-@MB(i6_Ad9JWq~93UTDSECrMfjppLo=w$Eau! zE3N8*c%0G8ReFwO8#<6ZTh6p7UeHJ0UetRo%J7Rq>qB^4KQi_eK)uKuIZft{fSh5X zqR%p5t`t~&_eqi-l5~eNDOVWc1FfU*#OK;jcJ(G%-HD`bXOal`l)8kG*K z6UMgwbuU$@e9IjLlg~dIF8I$)WjQVd;YU6lzAc2V9I*+9*KOQB#uPaVz@oti8OQ+P zZ2ms^j;zLT-#icfzCunm6{r0e+Cyd#zc_cQ367sJbm-x2W!SgZ=?jE8m=>-B*KWqO~$s1hXgTcEk26e*p zm9PYLK7cVg3Jjyo?pjJ0jU7xlhp@KHYL6le5`n{WI;-I3XMnl3@|tw7C*bKG^=Vm&5tEnxY>!ZS*AHfNd%g`Jz29mmJ(M7D zY1UMdg;k(kJCe+dyrNDC_3}W8o#0B{buG3qx>(UoMUAvS{^RpF_ za-DqZr4EE=XNjQ8ZYf4mdbA(J>O3M3>DNKBoic*O%#9BLBc?H-lO|e?8HOB|#*Bfo z=X9|4Ow9soM5k=9me>mhsvKb8O9XXN-2%{TQC`{jK8`@XpuZV}b7}yM{wZg*gd(3d zr=U2m$zD)Pg!S9y{}xokQn7FcwB^;=uX2>X0py=HHKO&bR*;VnLEGuZCP=^EdhDhM z&gAtW{_ws>ERNe^am056XjPzq+PUwo|gz0^kX%N zt?*-8OMj?r)Y&Uq>L|h2_)`CsDKIeL3k;mHx(U;XP7Zp?GAUfR zgi2&3ueNrBugSt6OsglRboS9i#?#FpiS*^!VbHmZ&d9%345xDI{gvdu8P?$R*6usm z;Ufa+{$VoYIv^xjIX;&D6>BSK_t!oE(M7`kMhda@Ov2zxE0&;U_8EEl%4oS+!zuI< zJD?Qo)QRK@g zo&c6w2y<45*c7W^VAeW#idqlI3)yI5y^d@QCF@hPdN@B&6{YwBTCWAbCM^V(wk8A% z-nOJ8-OpbD;6?o|bYDLz3c!U}0M;wA>C0gwtN&8~3% zEvOpY0g-K<;t|My-6e#%!+c0%vHnt{$0Yr+Xp%d6C|SQE`TCVU+(OD#$d1 zg;w-_X@#)0J_;z6`vW?&0leYex2b#;S*S-59h3#`f>n<$2O%?I zDAiAk4UJ9Ip?;mEW9%0ty%H4d^3`B9EFk+S+nDS}(Kc|31=;5L zWUpsik$owk{@He9{X}+Ot~1$kp?R~W5`9QBr;#;GX$^bxVrRkL88%aLrl*JjOxn`( zk{guozrVwX{jwf`5Fxt%lr3V-#Ch*gCcDXBN#A7==t|mjB55%qb@*5<6P#D&LN#Ff zMS$&uB(D`3Gv+Nqi@Luvg~q~c2U_=T(+o}9mcy}Oh_sdV(li`_G-VEWX=jcxYd0@l z1@D{IHPTrp2(zR|2uA(MF~F22ZN-pnjk>|LwKdtGPnaEfR?`@;TU;dql+=#^=MUe7 zd0qO4>(CJSqp23~L1w_i*e((C;80{1E-mLD=l4jYDx@y8Vi%)cny+t86UTlvF&g3| zaBrgjVaDfX9qYqo0CtkR3H1 zvTivO%-I7)uEobnftX{d=V8#|>WUp$>w1(+gcCbn^DgBI5i|D9`3QTppM9oFv4I}S zVV5)n51@WnR@XwAEDgr8%M-J`=;tCfR#qi+BpXa`a$aC8+4%#=cJ;+tOADAS#bq?w zc=2gESStFSTXByLF64Y;KF(z8*HNa+^;R`y7#X&B2D#O)SXjM!3rw%gV#i=9^yASQ z;XQjb#!1MgxtU-DuLA7ftO)R3JN7KntSkW6-`oY@ry^*TU)~ZMl}1{xNEPcp(w+Gf zow~|W`DREbfj=DlqaCs8Qk+E}Qp!%T>$U}9RgVM={mdbp|0D~sgrccFU z5OZ%Dp$k288ZppNduKRPpk#8}BTeBfm*C@H&WFX>S)wV<)r9O@ls=o=bMFK zO>uwnr?3tT;OOJgkZ=;Nc(p9lshJ+xA>+nVSMx*`omBm`R19%|L4D;pDLa}DmL-7l zvZ5J79L*$4H<&c3o6yLV&F;9XYkSBp-+&d6bSlZ-=s1CFTqm%;jUmZS2k6seCaCVh zvz;fxlyBP>LGM5S*=fHvg0J{ZcBu6x#PSJt&#h7)G3)eTVD*uJ1oVYSNjvisA6((@!({dUF&zVC9 zvFiZNoGJD^l8B~9rmP~{A15e}nsk-P`3%Jc8T2cT!uRn!10ZK|wiZA}H;bBPKnp4O zW)le=oXn2er?%OXG1fla+@s&$G!z?|Cw`!1qBb;s*W zc@30FZdZDP{v)V1n@{!H<&ax}r4UDn(vDAn+I*pe>sQ1wo>r=x2;d5JtFf$#y?~03 zyg@zQj>6?gJD@HNSO(p*+kT;n+t@Iui>#Fm{G#QiwXs&xPE#r?yGJno)kz@V9YnHw zFu)t@xC7~IZ` z(X!)g<~YC&NesVl8fe4eq2=+90Y7~NP=@oT!r^eZB>u@o#(vh-#)rK zX+hE?(l+ih;JqxZyFF(CYtcGswwSIDkenVD#gL<7`$IAkl5(58cxD4Zs}D_da^MZ5 zAF$B?#gK+#tQDDX%j_yjc-Hc)<3U@bplpt1Ka$~G!cgiCONyRUp)QJ*KrJS7PqKlf ztoRD#JCMLahO5i6k^F@IG=?FmO$o`uc_c~10@9+#@Z&kQk?NUz^?PH+;i7E9oG|0k z7d5?DvF;VF3}T9f8{^*8w==Z|sEv$9G5#ahx-pzFRv$}Kxv2N+dpg=iOJ~&3eX%%- zx!Mt3%D}zFWOR;)8s{^LOHl9}NTYStuV{2|QcWjis8o^x&hSiH_UmHJGJm}U$x2J$ zQx2(f09~3*2(Ips*U_0Xde?njshn6rlOk55;x9->amFQADHAZ`xDWNc=35(sIW!EV zRd-BW*M$9$JDMmzaN5ZPyn&I=wDNAmSs4f9QgdgbxA#z%=wt6;dC$f-O>LdjW&UZ+l$z6d>1sEKKF%O0 zl;)x;l82b?5QvK1Q~MbZJs$J`qC46{Du2LJM!ASux|Oy>!}WY-L4NH2jHx_S`#IBn z#P1DKEb|9^=tMOl-hg>in`DPKXEz@sRR8S^k{uXe`4CqVHe%}JNPpsP_1F5vE=n)S z+m1TZ=o0$6M&15~Y^us9b^6nHaSj5B~30dqETUK>or3$f%W;I|@`++n3R)BzikRPpiKM0UWN~o<3GacOhM;Ef(cm5Tr>esBG1W zN3?m`g5dB1#_@0j*F` zif+_6N}$A$F;moR{2UsupzD(rIvU|X{YGD#bs(o9!o~C`X8s3 z6)ZJ4LG2=;w)I3fv-U8HUz$`&t*{+~s+fbCl@rhxjFCM0us9QlrdD22kxzC8!H|5W zVkX(WB0EA)IMCrW$euq;>?-XSxIZd|FMR?-a(KluvS9*u@8LCM_Xvci6++j}3+T+l z=g3YM&{LI6m-LTl);NeIAV1aOC|36EB?8bo6&4GMP>IBnJc#;a|dm~NeIV8*2peRpm$4Ca~%#T*1umFsYLdoWPM?FX&{6na#jgBfuJFSUI`<``(5 zN=GunXXN@=kAnBPO(Es3KN{g0PY_1>(nLDQn+h~|Xa?dV5|M`uXQC7jpfh9y$}&0! z@?2ENHWm+9s{dIQ!&|T{M3n8N#w@8%A2x4on<)oL7XNZX^5k?sILp-A_=l4u@qt&Xh-#p{59zFO5wuyAZqc}K)WK$>%NL|Z0EdT&!E#4FE6Z^ z`s?Y4HnylQi&VZF+#R|}gd_5Tk0Z!F{X{k{M?|2NxMKPH#2m6Ylj!M`*lPO~0^B(8 z4y;dVb{D2k)7R|gjOU5*L#FD03HAOf9-)ctmKl2!wCn^S6GhVm`x_H22=a(wG&^Bg zyFoP4Cnl9zyMck>m)$;%;efWAWv(|TqmRBJ9gT5$@SQG(oSDV+ulT(R;Bgwo{{c9= zVl}`{O)oOW{y`8HzHws*rtm$nq^5BVaj2PyV21ICc!fz2cTducE?h&SG2$v(W2Uvb z65i@k2;&i&i`FT2@CdOA#D93M^W33z?QVKeiaHdWcraI*I}nRdR9txCmf^4vg&UPD zBjG~?bpBEBA*imRP~D3KIj%SLVSPx_hyleuP(!43KeX&WE`3UW3c_O_6hX{YP@YgQ zAc#?HATunMeuB_4#ZXFx11UKR`A()MqL1%U_dpd z(aM#K<#g(96Rnhvm7%XKuI=d1X32ORDRS~T%VO?J*IA9Hh{@m+%RdPwhO|$rf0uI zGhvd`iL7_D8Vr(T1gp{-5h=K5b8RPIbrEM7m%J5ivs|QuZGrF1(gvAP64i;uce3@M zb4z5;TGymXPb@&$mU2TgjHbpKH85ugHWpMwdXU^8+>*YwBeC)&xhkUn)+{&4h$u_; z&Z@_IF|9=pSyxDcu9DPxL(PzJ4jcm?&LaEux;UD7aFc8o-9BX1`ec>boYKf%G1D1z zNgU03#{<>wCN`XHx}0Jt$BmRewl4CYx(V2lOJgR3crYErdLo>jvl5YuFx^(X*U)s!P5*5MaxWJ7A{hD;=n_IU19!gbc1^{RIc{*%c6u10Z$H z1^UV9FhZJ*@zyM;I|Qk+LGHzWyVmfm>sSVtMoM3tM@V)(n*J>|pScF?lhoh)Ec54G zl%R29%f4F7^5Y0`)|?y-d;Mzm3gEtCLoL7`Rnq|L=+bE{!qA-O2Jc$MEPlAXSy;|K zm854fIscQpEfhQU+`KM<$|%Vb6R9rR8y-w-eS@8qJWGciyZYVcst{b)-E~$Scr#~S ztccZT45E5Nb|qt+cI_q>X9;S*jOQp~S#%vcAy%sh3;zHxz5&35%#+(FK2d~Ja>u}s zKh~(U#_nWM;5XeY#4^=vMb-d_ywBo$|A295{N+19%O24HqbV$$=FJoH$B=fWz={?( z)EWlLo?vFSQ6L~wyyg)mNAVP(BbhGGpg~lm*{ZdCj-j!7O&_PxEa{)OBw_E7+CVUm z5PNA(2oHQk57gfPThk7ADmEJ$lQ8zTo|D{0am)UihybO0B`w6zg%RQWL1UT@Yq9OH z)26i-kj^}pK`l`+y52fK&r$N{H8|-QJ1wa!fTVdUNjYm(BlEqaSEFz2ZTIiI@))|pNS)O>0KBk>x6@E5djS!KwWM`b?~3RzPJV`t!p^5K_FSG)o=tS_5yhN@P!P7%syFj5 zR^4qb4sd~gR7ui0*1Wn{snapbOS+~9nI>2j*+ijc=u2vGsvx~I1Zr#qafB*+`B>sY z^}B&ozSbFW9PAtfhiS_~NC($SqQMDp<%842@mE?k9#nGt4WQ)kVgTQDwgG5l3Ca1b zA^8lFhgAnypJK*RIiu7lO_rD}B^}E%_8n5W$or29{~E-SQQW7+-Blbt%LAfc0bCyk z*7ZYQRfbMYFWYny?=56&*)l`IpZtXZ)@)lp63WXxzceP6!kNeep08yEKV&Z=1)-Yi zl5ZV5E2A_oU=-o%6%T3q=?6_5wK~y+ZK3oO3es1!Nph7*-U>}xQj^34lOT0Zgo}sy z15_cJ?uHc5Hh%@Lf69YO#g)L*YSyqc{tKv`*=k4|g-GgY-$Ir%Rg`@MVWqGr1yI*` z;qNVkD!Bu?`47kid){*M+2y1-%c-662cz<9Y3(ky7V_K&c935E5S&5to8WnH@AC4J z=Hyl~@vLKM={zQ$_(6)}FibL;56xc-NcL}{FA;OD!3=9Oquv%6FVJ#PCCMWzU6CTo zSh^%OD68&pM=j2jr8zJDg{%p_n!jXucSn%&JEbQ+^R1R+*c-~O#T^-sDF9)8CC5dO{*82gHt2#m zxC8^T&8*%C!TQcX#-%RJ27U29z+u1U0=A-a?_o?Bk^mQU{^lu_!meRaFDV^d($c#P9zGN#b#&bWC{{G<>gBQp6{atZ#{pamH=3!Fvz0KuQ=x!w{_)e%|aU;!ECG4&S{to}~4W!-`m? zjijiA38nq`I8IW8McF_^A!Rp)5tR!=YQn^caDEq6UEGL_*1GlbpP>6?6w=i;vw+)fM7VbbG zYMN@^UGL}5wVHTORRrF0lN+}v-J)F@p@h2Er7;6JY+U{ry7y{NMa0M5PbMJZPV9VloqjH$?L4BYK8e2SEW(cEpsI@FPf$IY&LyK~Nw>s& znltf<^`RlSYGMtvnha+ihSDqc5|U#BI4A(*(2zRRSsGwRCY7eRXw5CmWP7Y!i37;C zjwHoSB&V@w!Z&Nqg(V%aKqd(7U(}ICVk$o8k}P0JOXtxB$?_-oXpGe37b>yE@L3R- zzCr`qdXW4hl({cTr$}bNB${q;%1S2UnDVRs&7D$?VZcE?rT6_Plsb{CsOQ+@Z?ER%UPe}&|0-J z$vjnACiQ+qD=V>v%<9wuO<9*#I1t$?qA|q>eP~5tMhHdsp8;NWB zs5KE#{k-{r{+V!i6g!32iS0)7QGjc;_Za>}70e!s0X;R94tFg(Fdmy{(?#q?+p{3l z+X5O-z?j-I5Vd(lt4ZkW_{(>*XyC)1PhHepFPjEw2 zCs^HDhwxnb+86{+PssIy+*S1sLaU1pPJ`r*#>&sj?xB+U>mNa}`!)hk$$1A2;y*Kt z?={-bRxXpC2`|)Fxc_IW-gGh9dkzLKA3;wR-cHyjJd5P+gTe76W8Ds@El?2}x%vXd zCP@K)Ku?ir8c0SMM?apiJQDz#1!2S5QkcfGlm|;7K1tDpK8xFP&nNc9B7{O8H9OM( znAKObZXK-WD>jQ#=MYSs9VTDO?Z}9~0m|+KXb^iD$u|^`FwZ8uQ=u_`#vE)ZT0w9N z&@sdC<_AZDm^BJST$#9RlzNW_e4W)RDaHe4s^odjbM4Pb=Y(J*go=c%Bl$z%DJT0B z5u1%bR#FxgNqP}NGDWzc$NP(^dH6TwAffk5(0g%tCdML1Rm?}J1|J6RLDf(y=#MPNzAIej#Mnr|D?xEvMe<4*Y_EMV)ft*K_d{x&We{uYcq0gu zIz*u*gPe2GkKJP2*P4;cCg6~3$dfS(rx`_m{XAIjV*qehD-Jaz^VX1S47nEm%s~zO zygWGZ24d)zL`57G=Y}x>;_e`8I2iXw0RBj6QZir>PcIz>*jtRF2IJ7Y%rE9-(nRLu zLLT(?)Vit4m%UA&z}O=@0QYpn1;@k`u@|V5O2ZEmNpvUCTIkt|3IX7Qd`jO1T9LcF8N*IY}GT9r4 zxQ~&)x4Z`N=`056Svtwsg(TT4Fbo|${fe=zYd2SvV2+-?zyqsqyKJ3@lewYZ17Dx~ zX{v7e(sVun*R7p(0sY*>wTSFzB*MT%qIu2dO6qdcxeIuG_>LI>$%4egivCMR7ZDh) zOQXDI4E-HAv2zg@tGCXQ*FV>FG5+mf_l-;Ni-~pJmlD5Wj5I4-s>fOo8u!}@*==)1 z0<15opfM%mCX~8RN)U3Lk1>!k#Ny~v%Y%1kx1l^z#A?bD3_S7Ii8U!#m{2x#Yf1Sc z7F251hU5=olh3vDVnzM1Y{i%s=LwJ6xISkj(m>dk4vPR{K?Az0K>ST^hp^9GQziDr zZ7dQoox+AJ7>aS0Q+>w+J$?NEO_PK^9!DP~@0av&FpUW7>g4waW9#8VM8Gi>aP7kow!Z~aQqf*nk)n*TPo0uR@;=7^Q} z7O56}>|_rjmu$g_OYvJNZ#_uT=@2}R6g*!agD(92gQ8fS`afAUPIb!Jk40*kts<;0 z*QQEXa|;z}DI{(#&UI#xiIWlUI>g71ywZ;er5(b9n~Igeq8F!}(n~W|#ttE;0rr3o z(Pz6^=6sX7o+(iLWh5)sKu!?2u?Ygvg2g9Tc+Ja3k55s0c$(xSMpu$9kSx1M5`Be1 z4cuX&N|*K@qnscRH&HDwGOB|{wHi-(Gn1})+1xabQN1&@WOh7_2@`S7O%dsFHTajW zxgxxa*U9ENxieE4r~Ih9!yL)?f}^RM-ifT|zP$uJ{l@F9wk?+j&vpkzXAbar>6|^O zcR6fbjn_{0-nSO9>-#Vhp=R#uStTv6bJ(Zxcf!wI;t{^?DbH&m!5jEHal$8D>p>pE zNb&xP=J>$mCY8dw#`0wI5XXyK8qKb>lE06kV0=q2L~}+O4S)&N>=mZ)x0r{xLlfpd zL_h!V5CKY`@3BLedmo9uc|amR zM6CY6cRXRjmOKU2@fje&9&Aa*oDnP9Q^3Dy*b%HOq@vRc2d_$&r>nnwWW!X#dB`96&1I5R8nhQ0~btF?IafLUT zFXEK&HyJO9%uh12Yrv-ilPz^+L5OcXr982LW6f%IKzB`0(Jw-CRdq<(;0L8cO-T9( z-}j&u$!exlT7|dQeAsZTF9fglqjViw5<#eRb{Ys7(@7#103QOp*+SsmS+*KJ**- z{p+;*D-lWq8cL`BBAKusVCD?IrV(+Cdh`L^4OyAen-f3}JL#j%5w2ugp5@9(>`3Le z&ZYow2v2`oxb2?F;*HHM*kU<(yc<;=S&;Jkingqb>7zLJla}D5<@Bl^0EV@R1hCH= zz?s7%0sK6~4nX$lA#R*ssMp{bjIWq(RF=yFDs2H-9%k9e^~SMTkaSp~vTm2nteyjl zR&K&*1q}As%8bhCIkENZ^tPe2^w5XZn|G)SntSW(zP zsao7V4jGFKGlA^nIR>;XU+_vs6KUNmiz)=EQA(kQU#PB;joxICS;f{zQcVU=BHoa+ zr;x0J0jd5hl7D8?_6}&~bD0reGM&Wf>(&F*Z97Ra>>SA|=0&3+D=wj^g(*??zS6nPNazaSVyOEfIq^=6;wq5C@ zK32nI2Vsj#q;S|Tg-$QbY>%48*DOU3_a0hGs{57=kre-)R?EG$cCs0sE<&b$P~~MW z6|C8JAe-;U>9$<9FCXMOl?+VGfv`=Ds+h#a6i`VqA)0M;*b%O<#)_4y6dLjG=wEty zi3MzY83Dyz(^5_Y~hwdv-mC93EPlx!XHnghAl9HY>Ce8gxO%4_1c9ErovQ@o- z3FEe*50bx(`1Dei;MTmgh%{yW#s47K2SEKto`KyOE7)htCT+#>!8{SnEKDF~mM$kU z*nNLr*kzk8O+tYcvtj!lt?pFa!13RGOMI6twg$=k4mTi#jlrJ}{q2sZW!AGHWyBp<5N2dV50BNCMg!+~1F0F`^MfJ4hEZhMvK(#(9oKQY#l z=R)W`fvuX0U`t2ehX8)_=gx>y2zX#j0^^>{sTBy3f=Vnww>o8Rt$l};8oV4AyS5gLeLmEjbpK5*h@|_-Qf?ZH z;g>JU{(uMhaFwF54mx79FBWVg`>sU9t@ede{q1fKTFqkh$bG+Z`z;@x9}D>4*d)@R zzh_g~3Ev}m!nbUkJfG>R#1v~@uwKAhA(C}ArS?#LXwIkek}oz)e$BpzLoD6%&`?*g zi4_K0-Qc_R0b5^5Eiit*#ve)cSUvfKmJi717P6V+Q_Y#A6-UKY(LSTQi2K*Teh0bI zXp~AX_-Z*iA&DKBFp|Vba#ZVN_DjLiSJ=EkbSlr+`vzSbeK?M)OR}UXD@A{3&qKVG zU8J}S*~7!xhpA7?r4^jq^;JYvVqdCjBLunm2jL8LK0@Z=9uC4Kt8zMJF-o=Z=290T zZngIU-q8(_j#hx=da?$~A`n*Y!^+|&`p8DrpNi(md}k#$|I`zrwJsn?@!p-hT7M8U zw;I7!8Qxkx;=_rhyzr_XT@1?JP4Q+9bn(afQb)*g+qVNv8ej3 zdI3vBZt6WvijH&T6Oq;a1X9RIWQC~zWOY5srtNu_%|F1h?_YsZ-Iq`jF0;ui*4O=^ zS5cqO8SERSUYQJMMlQ^lJMiVWIQCVWXIq}afLu(nhLw+5gzM4=lteDf#*E0_bIR#O zC)@e7(nUVw$$m`Z!*pB1qc|r)p3*8W%xrC8#WMowc@3a|Af~{R$VDD z?X2<{xA#KtOy901eIpb6NI9lZ$ufh=9gm$%Z*WV>Za7#us5Za9qVfGhp*76^FKVNR zztmmLLeD9aiXVW%H01O57;2R$*BvTI-*+2(L0w{0i1qm8if z$Nj{+cQ*uryZ>B7R#eH!2BKY= z|17c#1wRK;k}u*Dvspm$*P$sW%u=njmpFo!Ya)nKFn?t^)V zLfB%f2PAhMlUO_feI`=;`4TBMd2qS zOzCZkLKuY9a@izCtTFwy>u`D|AHB1VuAbbb^+0A!F7o4*B`HIU;gBC!i1HY}zd#rb zK#CUQedkvJg0{$h_B+NPKZ+`2;o;?0+?>fL{I1jBxPUGdSj_|Y4o(gWc!$!@_Zo2?lug;}g_8rp9Y-9E0eE%ISAZFt zDeG)SXL?>pxo*n~Y3EWI`UJ1Khge)~@ zovSvITQ0b}26s}{E|Tkez(t7C=_+tH9jCMdww?k}oJ5e06x#5Jy`teMqFpaC7f*wG z;U>*b35{#>h{Wv$iFlh$GWmlj{x}%g7fLaXloLCWH0}iDo3xD)_qMLW!8;|Ad>lbC zD49wTJ$vyUr4%7Hk9SH*PRu8XTSU?VhBZ5awSQv;{9TPWH)-Tfg?&#FL!uFd9qpai zkIJ{s8WXqpFE?ea^kn+XVy#dQX9giQ5Xw0A1JYC*CTcsAAibs|Sv80+_~mzJolrxc z^8jA%XvCsv=hO*+StF>^Tq?tMY6NkmJ#5w=37)-WDC=_l28@(k@=63*#jgQJs58*w zWd_inhIx>Fq9<~zBYA8_f5Fj*zEhwcwHE zzBKNtp6oT?y5CK4Cibf=LHy-M12g-ZPVN6cel(CDM)-FnxqtuCo0-Dtzd-p3!OZ3# zGtB?}8-nT#@$;qp=f+TlPO?HSRMaR_sLv^CR`cV@N@EwLiLcT$T4|oCtg}p6?~tRK&@8mhuwRNX^WJ)>2TnW|pPRDBMq`rT6v zQ0TW+@q9h0n^UZ^{+On1UL-AUED znTB$ss5i$&uP9WnI9hLBrr!KzdJ7NfExyOUYOJx` zrp8L&8mpsgtj(;kep!u;hiYuTS7WO}f18c|b{GAfq5Aw^=RL{#<%RnD%Jlc2(?3wH zf5=c>VZ*=OQCIn@k3_4FWvWjsQ~z^FefpmItis^Djlo4Rin?)itXPYrS`=^)a;8=j2*n3u}EZtM&6- zt>4wP_^n$lSKg;F)JYL<64cmcs6KC4^Bq68Y-H?eR6ESbEXBxTwo%<}MwS&u_0Jnw z-8Zs&#}7RlH*ht!3o~|zF?P%{ZnWIEafNZy`^L?cCa$(7E&NP8VoX|QnY3DN(t4YT z_jwbacP73@wf$Xd^RxTyQ)-9IuHA84?at?GcYRko)W|g4)wFwh7EImTj2mPNsGi)j@WGwxf=Qr4MmTW79co#L1}^Rnv9UtVWnMV-a> z>nv5)U2a=zHfO}S^vCk{fn;kFNM{=l2ZTL?D{vh)xUMV z{+)OA?-^NTJaDyo6lV1##p>B?s~6j>UY)mk^UkW;$ojpj^~W&l&neblXIp>YX8rTL z_3wAq3jPD7n~fB1qmybQ&#|eo-9~-ErsjJaLt|TGH{06bwq~ie7ISRtZnw3(U~Bc> z*2cI&1Gfft;SC&88#v8r;Jm#-lM4-+y>H-R+|bRfp?i2k&(wxqa~ihZ-q8C(L!b8z zeU0t>-RuIx?b@f>h0L+*xZSSv1-q{A?Lv+1!`i;qM)ijU7|n97lyarlmTj&v6{L-EqPN$4T!UGmV|H z-JEj6o$^wh^5-~B-R?B~g44|RPKCye=D0N~3U5@B+Ng9+qXpX=ExOQX$@@mjjGb4w zIj;(LUX$v)ZjSSY?arGnIB$9HTxQ&OyIbR(;f;5vHr|unc<+kFe;sc8_k+dZ72%5Jzvr6<>6+pA2fTb)%;z9<{$i_85Ty0F; z8nke;i*R!o>E<-o&3UC;<4U)t58aw;w{UIPqD4RpkGK{sb6T`o*`iHli?$D2w9|I? zYv>*j;2sp`9-QOeVWoSgO8&8ydpB*5u!bHH0UkZ#JR);Edad;6Q|Zy~p~nDi&zOdu zaRHtK<2(~{JO{1x98&2y?4jof?UpGGTaFB9IXbT8n4FelSGF8q*>d8;mKoY!Sq;5% z0=y>2c}>ajDp=_?tveNmZ`jd#)5X?XKC~_~X|ugWo1GDDc8_dRKDW)j9c}hsY;)j4n?okv zhg*18MR*?_>3w{z_sJdJr!IP*`QUxdr0s)sypf${AVbK4JE)qY4-`(cmTkC1|sYX_&g2aoC=oHi;ry(oCx&fp1`f+u|p&a54h zZ5NUg7&19NWJ+#G!K#pHRUtDjh0OdIQdqmg9QO`I-8+QGwLVZqK0i!OCo^0C9R z+8tN8cU;xIHj`VPMzC@m-(hc7498>&vRHuOD@ND|LHk*X=`Kw@>lizT|fMwyN8Y zs&2m?b^G(No5D0y%Oh0k5vnsfR4xv!u`5*nNT|W%&|3TyzI~WUP?%|tFtgEN=99zf ztPZPpB&`19Fl$w~t$ldIpm6(v;f|BT8?6p+d?dW-Gz#iS(kM15)+`Z$j?wv1p@A|2GsA-RIj~?B7^yoRdM^tf- z-n)AAz1*Y!rykL!J!3t3#`ox%FuG?_anHfKdJetZbNHv8$)=I19+9JZM5c|7OfQZc zw<~hO<;Y2&A~Q{+vOS`5gQ6x6jG8hzD!(|YV0F~ABT>^YN6q{cRcP94j(xAWLA{Cx z_L?`j*ZkGJ79Q!f_;IhLs@}`(d#?=Yy}C#5HKTj4EAG8vSMN=idvE#FyUeuDc8@+g zd-TcJJ-SbMai4v=`s}~l=fI~vhfMn(_UK#HgMa7L_jqyNle_w!y4?57r@rS*`(5zp zcd19eE2H~eEADq=SHD}A``!6OB+~z$Y5xZv{U7z{|73LkXT|+rtnUBvNdMQ5`@dBU zcxON0gU5i6JqCOpJ>YBcfbXjZ{5Ue;*X04fKMhcrMQeFROFg4?(xT;(=o-7D)mNfx zevURYi!t_$sogWiEG@>OB&P0~n0iNJ>OYCG)`_)sh;7(D);=NDF)y~!n%KriW1BvS zZLSmN>JZnWeVj)^T+6(;R%_zg9F1#xCC=w_Tsxh3KZp2$_VGap@xgiV9oEEmIvU^Q zNqje*fng2!yA#JBO`P~7F+(RQ(<~{=At|SQ(&U7sDS1ix zB}r4)BuzV-G~-Fq%+E=MW`pK<4l3$7s3dJrY001ky9X`0GHA)?LCeerukakas^{P} zX@l354BoJN@TMz+w|pL4W;SHI=a8K}hwM%pQeHA--|iv%uM9cxdB`ENp@%((R`ncu zG;QedlA$Me4?T5d=$X$$&zTLo;5qD4&tX^6hFvQec4POjTStf8elqN?&hYyV!ymRE z{y1Uy)4bu&*9?DobolEh!{6$Rc;_(UL;De*5=MN<8}V(;h#yBs{CYCtk501EFq7_*iPwr>V_#N4h$WY!N)tBXMNQDI;609ogpC$hJ>Mw$mNu=Qt`LcvMj0 zsNgB1I;DZ_)Pe*mrW&Mo~Z#lYqqa&yAZ?s1D-ow9I9NquR=xFn_*p_MW zk!cBI(vs$-4c?PB^jO-kt7*fYrj5`Ylj1mLWbl~LiDSl088de6nDNKPOnf>fLpMFk zF+C?ZeR5*@lqu;2YtyG4OP_Hyedd?+Li4e6T8_;qiX2-qW^C!au?zN$U37Kqk}qSI znU7o1a@?xOacdICt(`J%{k(A-_Ke$fb=;OOVz|2CY&>$c%kLQOOX?=jG1_C-ozVw zCf>R_@y?ft_sl0fXgTRoBXK&udYsd^JP-CdB*#e86P7vK99-xIxpk< zo{XPYGk!nK_@kStbjs8Y$y6m}>gH$at;^Iuo@wwbvzDA?Xpv>?l~p?`%Pc+1qBN^+ zd6wn1EUT|s{Kv2cUfFh0*$(O1PNmt-<=IWHWjFhp?P8JR=9S|flH-w-(=tD&)w-NE z$8*{~%W3yDr=6VZ=ad@|k{gti8=RlpVO?&gw%#!aB z@+lLpO_}s{N~T4AwpV^`RDNE1etv2G)bjl4$Ma`g%b)o*ztEy!j#oiZR6$94L1}5h zf^`K8j~6U{RW1>Eo32gW@^xyN#kB2S({@Hp+nqkG zymZ>W@@f08O*`;)+98YShrOm(MNL1NKK*#<^poY&PhFdS=Iiux7Ben*&A1db<4XFB zYo#-8l+U7d!> z1+y*Y&#t?7w&nHNR^MjZ)S1(u)f~HCa~#IbahgBJdGDMi*XK0*HpivTT(?$p-FwaT z96Q%*{@m7k=XzhC>+@}HJG~;mIz|4iiUNBT@m~#t#}af13(}|)k&x^Y0 z6^GU-4sTQ(-m18Jui_qqiX#h(d#x|-bE3H4^Wp(|B{6kMVq2An{~I=_B(b1m(E5@g zCrXArFUc68H!r2pypbK|jUF^_Ou@Xd>*tL>F>m7Yc^P`8|BJml0qc3~`@R3~i&hdV z36(UMmLU|GNrhIDBo;|BlqqCfhJ+9cD^^I7%2dYYc`lV^WJ;6FbD8IHzg_#@`?>bs z*S`0@?mgVk@f?ric=?>?Z#aMFZ}@BU{cr3RxOsHomMMW-*92}m5V#{X(6=gZ*T+CV z?E`xT9q^xWAYjdbzyk+@QV#@I9SHq+;IMYk(Lq6BQ-Y4K2|9To=yYmOL{(7K$DkPP zgK>inCQLb)wB}&SfrDwO2Q#V;W_>)EqaB<(C^&ygaKW13q65JtsljDc!52OTmurVy z>K1ZkbjY>&A=kHuRGkW`DGs^uIOJy2(A(WY?~V?=KR@*0_RzuYnYpZl=^JB|%HeQZd{v0+b+nLRl+Ts6$1d)Nq@u#rw-R_olC-}Jqe$#dVFU0zj}VdiE*mg--OV!O?G=qThK&e+Y{HlotJ^I{MqEXhpji84}ZYYK-#Q7?p!D zs_8LmH8Cwd$Edf9)ff`1IW@M;g4niy#%iC5ZC4uG{%Nd!vpB;ZaUI9RbzTtHWo=y7 zgK^!{%OMFPrzTjfO|U+g zV3VF;Ta#e-Il;bNqQj8HiBl6dOj?`hcrei^J<+))(dBdEjCM)0h9tR7O`5YdY2Lvk z_w=NNHAxvy_cJQZ|oC*|H$zk3Unk zpGo<%G-c=0l)sv#?(UJgcTDQO1*!Z0Og(TW^cIk0L(i5hpC#_9SIhdZ7o}N*Yp7}IAyIIEB9vOLKGR`f? zDEu>{_+Um!dPZ4I#)Z!rh#Ron#{V-nYY?y-5HW~Z)(3p`0*IBLLvpN}Pb!*S+eL2hfWp)^v+sQe% zi&t*f;N0#Rxkk0Qy}slc>*nd1hbo%ysiEhvtuT&bRW) zw+_y?$;h`Y%O6{xZ}%mC+?RZN-E-rO&P}jA_uImAlYP%k2|nkPan8B+oXeMUGjt1P z8WqeMTHt0|FneLaT;GEE5d{m%3KrEDELJOAYE-zKuMwOJe_vR*%C~S$aG_^Jp=U;+ zS8d_?FNNN^MH`KZHVrNEaW2~GRrH5%(e{XG=$6I}ElqGPP4X&D2`){G zC`~Ub&8#oYRx3MeRF-F3c5Y!=p>J7nL|JKB+4=gii)!aD8J)jkd;Y5P`3kS|mBHt$ zGtSr6p0E3I{+8~A+eR1e+FrQ7@WKPH3y*>?Jju9FUwh%%mkTd+FTNUj@s0DvcU~7i z1Yi7=aq&y-#cy9OD%zLJu=2*!%9YoZ|FpBbX=Hh`^X1JO%71Qtsb$Ygt;Swzz35Wg zotLyDFSR>=seQvG{pObqdtUB1_HyS%mw(xLxm(EP?wOa3Zd_i~>+5CX_E-80yJ9-+ zO8<3N28LW2oOxyFjVoqfub8*LYB}uc$Z1!t)?Kv@xoVSn)%M0!yRTR6+h21Sc5ULc zYm?Spa}2rWlzGkh#x<9(*JiY@m^G}zZCb^gbrthMD%>+G7T&1v_*$`~{q<$TuCJJO zedW6At3$4PW?uI?e|=rU^$pD{H}QM9QBR#8+jjaw}RDEJ+^{L3}Gv})#8>*w5*TnX$i62{& zxTq$1XH9BkP5Swo%!Zom=Cx;g*5-|^J-4W~aA$2%NNq`GZP|_53twx?+uyi6?8en; zH!9ZMs0_JLoq417#*MnKH*U4ByEClr-n6<0>*^kb)IE-@dwRaEp`q@1^P4Z)-+VRf z=9_6Zc_-=*AvZr|-u!ao=C`jm6?(VC>{er!TgvNisf6BA&AO#lcdNy>Tk3kZHOy{n zy4-HF{m^(7L;3-|m|0-Lo{iH`3*v)%ttZq4#XE?%CGev-@_>Uhlqx+5HK2 z_kZ)aKY7>vDN*;QUbsK)+5PD)9?b0Zz}4=-Y>x+XcRiRN^}s#r!NR%+9^W1;(R;Ye z?BNQRhbz}Vd$%97w)Prj;b%cP=Eee{lyjymwGi^ zv1_>I(QtiNLse8m&4q>=&l+yFcy>$g*&VZI_gtPmSpV!%=(8tT&+6-*J^S|Th2Ha5 zX3yWaJb%}M@4o-bpDra2!+9^H#@hYV70TLL&1SE5oj22C?(FF+{^f080c@NHp=J}R}o{87pF$9oMaG!%+^UN6ED2IwiWe~XkHSWb!* zFULsPJvCC4S4YZEP{Cl&Nb%VeDSb9aN4xt)%%T?Z+TC7z0}tj)g7lv9QB9jE6lO zFacBGgsE`GGAxBBj-maTNYRG@ZbU^&9Zp9_${9o;3eku`CbE!?9Gu0V*hrb56e;dl zgvAI!|70!-24Wxc48VR2q8~^OCWl}shQSQO@oP$?SYtG9r$x#g+{HcI#{)cs{@FpEW6#ncyM~g^+3_{NGZYkFOkywEBj&e z_ej|_I7(EkqGasoD6vEMm?(LRNfV-EGLA2el77pg-l5TQ)j3*> zrbo+Z+?yRO%eF+z{T5dSq=L=hv6&{;V~2H>@NjA*os5lyr~Tl9i4R$>jl z>%>S?KEqbS&p0zlPjMO%h(rvc5rtT2uuUrrz(5SbV03FABVDmaKSp}=h!Lk=G4dAg zplTc=&7g*#p@CKy)F(!^n8wKGelhX|zfaavd?ml(J4^<|NFVgY+_5oIZWklJj*Af& zOouCqoMWUI+S6i05Be~`ZI>8%fQ>U^WIJ}iU}lVT!XX^SuXAH$*mAbPCcMKQ&lvGX z2%c|^krz<>5hDdVVx$lkFfAxX>XChv;~tBV7AH6t#OB9H9Uhd#Nbgdv!*zaQixDWS zjgc!*eZo8t_AEw@!%`6|?oDH56^8#DD}@-;CRPSxfp)C?4vTKFa@md707lWiMqb5X z+7HRo42_q#1Nxc8HV9lftA>RV4TJoOzRaZ?Tlk(G9rdD zw_&le8f)N*sxh$=KQ>m4$Cv}L()eJksH1yWtn@;QGqLhB(ouz4Ts{{oAD}3V6(t<0 zi~P?jpwn_8l4gMB36=+@;X*BaAl#MV&Dg^HwIw{h9T=?EU&*< z$weXde~OhukVSfm>EB{y24=z)ZkUVb-?{!!l{jgJQLw_5#&L2J2`%DegL<4?Y8fZD z5v&m>j&0*)3fAMBew=)V!XQqp4dY}i&UA{CETr~|lU2Rrq~17AicR9A0&Duji6>t4 zixbuUaWct_v9JqC=8S0(Crzx_er%kS+Q-Rx^cWu}Mw8;i9QS6%NdulC)iqAikd8d$ zW9fo8xevodak6$voP;maQ)H6ac!JK$;-m}Kui(7kjg1IFC>pPdlhfXD5`joG-N-!A z3~Er{5hsaz;>0JA`2=&l(Dg{1*hX{i7>}UTI5~*G%sA;=5+}nEf>2yXB?hn5Q|Olfo;^cSa;4E@+4nJSyxM&FtthvnGuEj|dQczUE^@45<*X|Z`#e&;$vJihk z=TV%DdCnZM_jR1?!*T5T5GMgm;$=VfDwFu$G+r91#>@d&Ntue(};43kS#Zu{vIsVArsC`3t*|hkTrMh?jFPaE=#4bVWCq zq96KW00zPlv!}(&9Nd{6FP^jFrPwuIN-%Uzyo_8JFQ?b(DHf7j(0ft5Ebxk#-?7gx zUIOtdFkW8c4c_7%-a~nVp5g=f37_!|?GMC@9`s>=W^6Nxw8B!j9b`UeI2120@DgwE z_-MTB3}fB#cv;3i%Q1)c3UV$v5A)%U1z3nh@W5g$!7?m|H#Q;$v4}?k5}|w|UVg#_ zTtqEy;3jV2HtxXuWV~3w5}s$`#S1|QLp54Qa2#wzZ6y2QF787uDqg0b9u1fp9WT!4 z8q0mcTlmJs%h>pMd6W<@Poba0y~I>_VlBMz4)5_1U(h5uUXoJcB@OziTw`=VQC7TE z7Q{;xs!@Yl+&~@5z4a6~$=kStM|g}UP%7j&MchNE6~{|U=$6DwJJ`bk8{mzN*o4jS z!5;`g87|-=x|YU^by>WO#&~#Lh?n(PhYdJ}FpRs%eZ_pZqXxA&R30yfk$^-bAsH#y zbD7r}-r+qiU5S^=74By56XbU|ElH56a7L?T3DObo@L@%QD6LEol~tVM>I50@ znIH~0x;8-~(9SDC4Dl58c(acAtxu3bnEE70KcsBo8X#+Hf@tnYkj1+bWD5@AFpl5@ zKEZqs*B6_x84XyzFG05A;=u&D0@aWNxpas;!}UXZRPEGLd`srs(KLm1P(w2`$Ej52 zkj;HSHga$ld01J=wZ&#^K{PhrPLM%&5@ZY}`ROT!+)EH|Y{D@txSt>kaTI?%OOS2Y zj(2#Az3&oa9}eRvMtw+-N;rN^keN*rd4JGEam63a6JScb`i*&drmCW;SU;SHACCCUxlnVTq{ z3lgP;Pon&c-?k)5%fAw(H5!E^iV6b46D19GxQPoVIYv?!8 zgZ4GDj=YImxQ#owhX;6wM|gsIG~gLt;5lAFSu06Y&=hUZ8zV6ao>&VntiyV2z#i;{ zKlUL2`w@r;MB=nglJscD`Jg8nAJ$X6(M^&r?UST4K6Fo#k7(5+iLWV=L>oHjfI5>T zY2PPF^l)uplKeS@WBtbXxVbn<4tXWXVQ6eflF@sU#OAo3BA5(8C=Ou^%MO!Aa1_T7 zh7&OHPm(_9i+(Uge+&#{t_PB&7ka}OCK!TSAxYBq7}xYT^EknIpU_j}<|IiT@-gac zlGx@Zi5*gq3cr(jirv_QNw>IWq(Amy3jF}miQG?4B?C!kGKf5Y8MJ32nD#-0AQXo% zhvjoI4~yY}-?0&!;fq}e#%Y{E1fmgxSe#&-=68~$1%8G)T0#TS50Yf1a7p7argtp5X;vLM4^)*uPtPvh;uvW@IGGOtjBQmh0D&WzfrH8I0lBm&j}X zd$P>IT+G9Kq#zYN6e(hap6G?%7>IHJi~h|noS|}1t}7_EJdOajdL4PqyV`A zDe@^(Pw^RfwDU1%e~Qe-JlqWAIKe4$1V>RBk|H-SKa_QcQsfqHL-k0CG=tOe6qyS5 zs1#|Ot*7V_&AFo|dZ9OraT;e3fp4iP@*P2Je*gy&3`JUsC?SM)C=P*7P^A$XLm4Xg z2~E)q>S&3>Y^y=GLK|p77ww@3eHg$H9ncZOFdSwuhXpJ#0wXaB(=h`xF$->(jm_}E z7Hq{I*oGLyA`bC*i+4DX&iNu1aq!4Yk;Smd=APpY?xKAz^To@&6nO>vb15<&n+rJ} zQj1a~4L=ogEikSmMLL&qUlDqbIetr#wb+}hr!f90RZO6vk}9q61_$$ah7b%NwN%-H z5ZYVGKd=qP&DjrqF#hLMvBv~V#3mflPnE+MpRcF5*Ev=0V_^4G8Gxbzsp38;RZ@nh zO2fib`4*fiR%LpMUttX!cppxcv7{|F(Y7Ol$=}E+D5Y(CELC=%NRP!^d_|)>snQs>_fp08Wvav>9)Er0nE3fisnljnl*pvoQzrFcG0NJf$$3 z4)I7pB9f4d6r>^z>BvAPvXG4&oJB73;J}#q&y zMEcd_Z)6>L6O(A)A}5o#$ve1qBxG%7VGp1qC?>qw-4htMv$@RmNRqT&h zt9dQJ4N-{3ZQQ|K%vr-cFdyz%1P?665-i0sEQik8G||N%48{=L_F^nN-^_W#ek=DJ zpW(YBO?F}zzVGB7|CJ`&_VOAz!n1pn=NPAO8fOrR801~&{$gGh^MX5;U=4QRFF0j$ zuWhZ;szqlG9HX_xz`10a;uPeLi<9R^#96h;v3fs>5bAQ1M}6=#U179>2e8| zaRmd~ri-n1x}0p6E-8BH(z9c_BpIbkGG5-~-*|7f*_h7vchbc#FP-mYrb|8ytJ9^| z({vehpPxgbH*I4Kp>0AABZs0dZ8Or8?1%msPJaM75Q8unLty@6AJT#xMw(&xkL5p> z!4e}d5~E;+(Xhs^u)!GEVk|5f$BrBadyJqzo^-$jjHEvb6KPMvWH@39oG=y6m0XT?YtZZN$c)S3Qi~BkkxesXu4;yCRnL$=kkm3mrnJfsC*-uwkfu5rG7F(NjCSoaL>KL$ z2Yncz6l(ez(i|<2j&p_?Qh=8oGi2CfJ;nB}8L|U^!WTQyxO;{)fie!8X2=m7g}Y^j zEPbMv7|cL5_YsT$Z@15*$Vcw$HM^=FcH6D5}Ys< z&X|swn1^DN;5;ti4h*klh!J|j7=2&@Q%u5SlvQNN?&}$nQpLGoQ8mvGp4Vi^3%JxW z<}=2`MU>+bX1w6onDQ1Mm=_Lx<#~nj_YC<7mP(nj3`;SqO{TcQ4YQ%Hl__dNGi9B5 zrfk6RnVHh;i=HAeDpNW=&XmsR0V62hXUc1{EO~=_)>*P>UzRKn%aUJWvZPIFmb8Uw zc9sl+e?gY)!^`to@(MSfW=X^AEGbpWmNFFd&lcYn`igS$5~@&*8r0$*?&AUK@fu(8 z4T=HTvXgyxp%LvS(1JGFqci$p2{z#rnhngB=7>Wg3ej&+whV(A@`h(iKF*;4g>bgX zmZ+cg6*&{KB^PJ0!zo+-gfH}`W%Kj?Y%#>4IoWa)zs$=P%T3v$yd_(Hg0qIcLWP`0 zx?nnHU?!TfTouirhPKdxE9-iYM(BxNaKe1m{TgzlKL%hR24OG` zKFg6{gdpm9j%4CI%-`mS1uU`TeU2=}+%H@^)HXfK`?#N#yHKc}6+vTkRXZy^F!L9E z#SYW6;tvnIv+@SM_GiV*?X2i|pOs1ZXC?64SqXyOz+5pKo-4!QI3`!7V7hm%oGH$g z2zZy}O3{T}>2WDnhE(TDT3xQB-p=KE4GeTd{urg1c;o9mvsldIxc{1?`Vh_f|<%um0rRT|E9Klf>!^F}0imaME$%g8UJZXkSEc3XP$MyxguX);qJP8X}dmOw4u2nU*>Pkm&)<_3U_h=7Ge=R zP{s1aWHniXCA61f8J44#Wxta*$dzOrxr$tkHSmNN)?qz1z#E%zi*0XWGwnO%ZTQf> zOY;9;75A`({&wtuFLq)V{(>L&U@!a;f>0d7Q5?n*9K#7bWV}=4Nu0(RJYrci8G~4y zMK1D?k3w8Q1+JqOH}Duw@D%lEz%!WrkuSsX2c9wZHu6vS;yL}DLAM9xy^r^g?eK!vv=2hyECVff#`^jB|ucMaMJD55J%*x}iI6 z;WqBTFd|L04oi%{NQ{CNM&l^%Pvlu5 z@8ED|zPzP>gglC42*U}S#3`IcB=WNIWm0y&IN}Pf;+M0`8{KgOHM#jRrkd9#R$(pv zf^i+!1>+v%%RyW~1ymk#FYx3Q_r29Q8Q=DtH0^dyPIW&ghE9C$H1?b{8h1`~9M8$4 z8Rz6N8c^cQ$JfOyhb|1!0Uc3s;GA5??6`B1U3E_U9-Na;v-B0a$vxN$f9%741RxLx z5QKyH%=TY!nD!AI!wH-o-H;~M-da?n}@d7XL3UAQ;55|BoOwb=)wlRP7KtGsl zFOWWv9R;G|TOg{4Mhu?d(9Qxmid(x2?z9&ojP@e(1i6}A zi*;C!jo5^ftowu9f$jJcyKsu-e&lZK!Cv^|G|L0X1Bgd9auLPy3*<$VqX982drp2u zt0~-1^nejeV2S}4h(Q>PAsB|CFoPAw!Vcpw9uAm*iTDkZF%t{nfl5@N8Z|iV#5KZu z=JNq}TngnbrcEyt7Z}edls@PSS68kre6S4*+zRCxo+E8`p`6EMv~cJA@$;fWDMaXM zo*BA`yjX#2^l_NI*Oik&GmyAPpJFLJrO% z7xyvZYN52`SQ==Bk@T&gN!yzI6>Vs@MqAj>AA_;5g&piM4i3;_n+c>gIg$JglQ0>M zm;xtEg)^pMI$WT`KCYx2W@8TKVjdP?Ar`>{i?Ial*k>uZ3~S(tweZ3^tj7j;ViX;S~ID{iOjC`FU zxz(pgCigEAM@+-I!A0^B-A5Nm4~!dEB;(9Ufj-9T~_(luMCBBL=a!$?@Vy#q=WSg`=~JTv3wa`EMM^r z-=Q!s79|K)p```wk;S66hwqh;26+CXKdM-oqXmA(C#zyP^lPy^!Ba$77t1;98eJ>{ z+4ncwVwr@&V~b@dhQSQxu)sysp_g5;oUt#KhYrQ^2s)FCr5y%OWgnO!!KGN`<*@5QnS%@ieKhWRiJ z)>p(Rl}HARr9>Rtl!#{A5*eyhB2)BBq@YKMob6j8xmYuxL?+pkNRVxb1lX1EyK{Uk zI<7=kV+}lU5_g?S4Flzj^ShYwG!Ee?RbEhmwS3q#zaiR9N`QhGc^8 zs!BF;a2EMEhYPrfa(u#Pe8E?I$2W}ssZ<=WY-p(jn3sxX6dxzxWnC(3u@1xN2aYL~ z19&&KRNmtQtnEr=G+xEcy zjMuon;^H;VpS(`?BAZu~i7BZ?YNJ2xuH*o+&yPL^VkrG#FvD2-zhMfd!WDBd4+}Az zZPt-nu>(P{WLXHAi4nAO$&utavH*oB!g*XkIV$iRitAh}jK^whMjrCfqOwd{p$&A< z31cu3lQ0?cum*p^7rSr}hj1FdF^@^OL;C?9Vk-UfRos(m?hUkWGCt0-T#tNwuS}le zHPjw(59-TgI-(Kqk$ZqZ9K|t&BMfoq@tJj4Bj?2vKG=e-_!Em7ofi-6z+<)ZqT1rT zG=mzNLkn};oR@iMrE^}A+n<+zP_VYYqHE(HtElG9|3K%@KU0)e(_X2~5+yA^C7l>0 z-Et+p7fJ@|(!uy2SETJ@+{o0vQU4|XWI5U;O&tB2IK?z^E^p%UqR9+(t+gXo%lDGju9sSV zEw%TUX#0=X4p^!kxLZ3YRy+8TcIZp(!!31=n&^a$*Ezma=j3jk)3G`cmvo|D>cq5c z7iZEgVSKx!rR`F7w@Zs{mvO0G*2{J|Ep>BEbo0mS7A(~*+O1m>t6O$S_rgov@|Nu{ z|95K6{trveZutLe&i?B)XH)j{NQ>){ak)p^Oi=hJAHcHbLjP8S+7TXdOeBjRe!nH zvsb-dX!L&7r}rC&-tU(6{;;R_r?}o*_c3nOw@=@`rW5-1U*31%-oAt5`wqR**X(s) z^H!#oeN9JBFtu84YQ5LgCf?NcimBaeQ~Or^9QyW~IHBL9<^3G@_H&Bw=X|A~%jNgo z)$zk>uMDeuJ?vI1vpapw?oBXzu-xp?Ub83hX7yLhp1n5P@S@eooM_Qzg@u;Cg-(Kn z?o|uDHx>q(mK{tjJ599gvcj^fKmR+;(&(yXuQ!&)nj`v{jxe1#qW_8!1N}z~P8c!t z>IkzpBg{2NTAGd=Iq{#YGdt00&I+q}{#Nb@Rtv9MdAzau*NV)VTKi12-nznio4@sr z1Z&@aTZLK8F>$725+;sGS}`WYe@t4!n2f7qvfhlz(X`DqwauSsTkwy{%U+%MU(}YZ zdo%WyrrjM=yL%Ju9;~o?V%Zt|2U;TXz*~IBz{;2}8Ni%gTX6n70 zY0zd?hyJrVO`6qZ<*cp&v$`kEGOC!>>)kBlHm-g8yP8gN?Z47>V1Vo3B-f#Ts~&6K z#?7+7+sH|7Rx91C1Kezq+-xh{?B2QAx0&tGfA+*lvnQ>b?HDlIDQUKI#cY>%vuCuK zGpqj`w@Gv6tei71V2*pzoP`ztb?MmE0dqZ*=6Y4kUH@(_MPoPhpXW1a-qw|Ws~DR! z&$r^=sa$NuqPlmBZng2a)8FIXB##FxJst&kJW2AXukd*G&f`U!#jpA=eluzDyOoPS z1T6lPwD?QK;&1O3E7~rR0ZSTBUZT8eiOT*Zs>w^#t}kiveu;Y9r5Xd4YEE9-X4O)y z{Y!O{m+D?$s`q}WLEB{=1}y6|d0CfL%ewAg);)Qd(e-7$-Y+w5yS&eU<))LD_g}Sq z;Qr-aD9*Z`;3mNAhal zzbg^@uT_X$<$3adqC9L)Td&*!UitsiwP82>bH!obZ%}A?%RukOj^4_vy;TCeRa3mx zD!p5L@cuVehIQQBfA!{pftv@XY#v&<+3dq+b1fgsfj%Q0eXLgdSO@ypr1;oY`q+K& zvDez-FmTI6$1RgqZ*dIV;*_$*xpIrkhb=R-w$2*3)y;A1oYh%Xj(@EDAJ&BZhXr9%wg*>k5B;$Hu-1;F19yZu?l`{s->n+#hrqu+rTq1! z@~>|n{!(cBQT3$p6hGxPekuq2R8#%bs{C4f^i$W~tubh~=9JxS*8DG(f;HFvf9k+e z0oEqf-?qx%?xX*Y@~?lZ{W@vQKF0(5oKp8WSM788xNpXPD*QTUO~AYZ0q&^*3#$S= zJ_an&-oI?n{uNX9uUxZ#^@07Ksr$XE_OJiA-&;Fy)1W|~e~+TCmj{Jhof1;9CS*h9 zfspFdklLz{x{o2Zv_tO<3cWWa^ue0YM+ZWmq=wd4g+BWj`r@A~`KqpSL}Ty~O{XJm zJdbDv9nncUqFa4L@6!i9^Z2}=Yx*zX(zm@PptoR!dvI$roktDoK9}_Jh?6C|BD4)*T!lcjMYhx z)vbxu`y6Y~F0R9nxK305w^eujUo7tG-7aO*kQAS(DO=a3Y&)2;BR$2pCS}*>6u)+< zdxoU?PfZP2n;Lj9H7GqbxF$99bL!!CX-9{og-uO6zBcXT!5@`fBWlv3KBvX}qq449 zpVM>NW#kUY$e)@~ur{OU?`pbk_@6K8T3?g(>~q$OcG<6nWWSl3{cdgchlAOl(zCzR zWPkgdtyIj~ z&INP43g!hDxMvhBtS#{PQm{m~aM{owC0ti}6|N5cFN(MR=gO_yf{S)!6#3Q`?fO#W zr(3*dXtBR@ae!BG;J=}4Yes2CZE4n*(j48g+@WRp&SeE&WktbdB^hO9wPhE+l$GnA zzdZCG*KEBr^uj&oe@nqum5_4P%yP9Gc>{r4%>dgJ=~uh+fXS8f_s=`*cz>$=KqA(cBaD}8TN z?)qBk*S>1cuqyv)RRQa&0z;~TGOL1bRE2)6I^4ec=&7@` zweJ6&by_F7+@G}mzGLWpr>y(Vb@yGq-JhZNV3yefHE)5my8!AH^sl*65HT;h&w61^tA@upDtmj|qo`3uHT%rF$hQDY${e|*| z7b=HdsAj)VyZNHU_ZRB=FExh0)SUjZ&4!m+hhFMrztp|?Qt$gq1N~PWhQI1G{Z*F@ zueu(3)jj)_(al%AzP~cof8A&JYt!kk`)_zX@X+hQ*{_G*d~NpqwYmNq%i(WEPJd&y z;f?j7H#XUCY;V4?`~JpW|E9b1zIFNjc831DS;ODCO@B9M z!@GHh-nnPLTX^%G$M<(j^xrQV{(i;u_bWHNUw!DkXZCxqoA1|ufA6jTVbkyrKGQ#J z-SA=Cp$|K`98!%D)t)RbD=O zsmJe&T1%Tavo=YU(x#+Rx*WZGLaEE?#}u#qufB)>4{Of)N$Y1+obB+pg0tI{w0^wp z+~`q`NUIx3TP786EVj91c7vzszu6mOY2cGTFG9TXNyCMtFl*u1gj zlqPM|EF4ByD0OLUF}CTr#+FV^^vvxiI0`#kIyT{NtI>82O@^3{n>a$1-q_I=j??W& zSvc78Ia{uZglv zr{*oUS-S2vEij(fvya05S3^t7u0D;!9Y$}}o})9yRb4|YW@$~bHt2N3O=tWL^&=BI zjPcbh(F%*Wldp5OTezm-2o;SwjhQ|s6^;x1M_G2W>lSX~`Dahd67~5$>gJAyKI)Tf z48r|YyLIv()!1;pNx|hA$9zrlyLaqtQJ}~PR|nWgS(Q?B&Si`l=sZ& zwo|D<%hh-MW&SeB*K!RTNmp%88*TMQzZj_RF}HU0RZ=l*^kXPjHH`w5nZAl#Emt3n zA3N=~GVt+Zg`1youFBmeCp7Kdy%o7C9ewEt&YAe z3shW{wAI=)>bJN_{EOL)Y~ZTC$I`&nub*-^Z~s08THAd$+dGWu9A2?(hTm^nEjG8d zR2l8VT6_0M3+oPDeAF%NPsgg;s<^2c8yIf(v#?g-0$UpI^tDhibW`i2+;llpO#iyN@uiGRCdm^GY3H4J@x6a5GOa(K6mrJ_@Kut}uJBnPL0n2O3qokAw;=$g}f-$Vy@-_4f|4R@Lp zm~ZnnX{;FC&C*(Z^r_qaV$)n*mRNKiWod1*McqZETk2td2g9D0<|+!VVbhI%EiF~N zeQ6r{DDhWBYiAXU&inj3aD}aTMiRH#jNE<8l6PxVw^SH(<5nts)J0+FqcB%7Rcbrm zzua$v&8g%HRkxnumMS(|%q^GLbn9%XzP)R>watRaep?OPjKVb>7ety|US{rhMAMKP zwN*>KQKN1gmxpx2=u^wft!={p_Iw%|I_ffqPCRHfhEZ<_whB+m><49M#FHV zx(e^)uUM=V&R^I+SQABEmG?jav4xQW!MOu{Y-JN)-m3A`)fv zDV11W9nf6PBxJyZ47_$Gtp2Pb83{olUpat z+$qf(CH7Y~?AUsQPDdX=$K-8Y~>%fR2HLd!75#J-!W z@5_R&cUv3!s2tH$SgLI1p9}jjJuNh}JBBHhD-66fi*!bGGjFV2+%55{!?pzf3(dBN znUqH+4%(-fY#S8!UeRoki*U01wryjw^Bh5QygatUb58|KeMm{Fh3O8SiYWp!eZ~HsBwmjkQ z@RzrVin(FqK2|)Zqx>CIjo8jG(7eE@i~1`|$GpatYE8%dsctETox(?Ij1KotEU^yT zHZx!Am~U*se_-!TprSgq{ozyHb>K8Kdb(+#QPgg1urZ*`prS$3&7gpwfT&LMI7Op4$C#M^uR7-5-1p^vYrX%s)?06V z>uOhZRqb87cJ11=Ygg6jgF12H7ETDw3Yu^DoJ(&Z%M-o)g`Av-L4p*@#jVQ8>L&<< za>gTK@frQtd8V-7*@cUbs4@O(K1ow5zQF60OMSI5}H~-#l2|0B_ zN_^pWVX+Io;A3O)cbUhmuQ{h|s)QX#uH1Znh`(DOL6(@0zsQ@&d4&p6k2QHI$u@>C z>k;`&jE-!QJCDfQejfARloPyKwFQ$~QPkk<@4h8eF{jYeiL=c26l1J6@@3wA34f4V zsfw_QO_uo&ksVx^#-)?;ph|Ip=p5BCWu8k!jECH)bR1DgaztHCP3gqr`9(@DGcP_Y z*iAlQh~Tzx$(bQ;E?#GX+rvn*S!y%sT$J?}swlPIZAaMXB(b-UQ17D^v{{A5V6PJfXaCR+YqByEauvle_?2Q~ ztlPq-?9lEV_tZqI+pD4uS@)?i1`Bg@jEx$B_qUf}DHgOEwj5e4JaQyG)MHK9h`u$j zQrY>Qq3IzIxh6^Lb&&g{u#^*w`NlwJXU;!QQ&1S$Th@M!=%3ngeQD{mRXK*}sDPZJ zQY$^&7fp7~;o_(qx|J$)ROufzzo_|OM93boI7*v|;ZTx~r0iQu{qqKdx|syRZSQTb zao&h`Zj&5lopQuOR*0@?xEk6oE!1tXT${xuo>u43;0^^8{o#dHyV*b1yRJOEj<^=r z1M6QMxxaNmeeL|h?5NoRbsEP!!x3o{ho`2D8<&u7&Q1C=2R||2(k{pHU;RX@5&u9? zTm7>t=p2zYr@VI>i4DnA8*d~;a8d_7C)N6LQsTu)$ErBlvXFzkJ+d*JNIKzrCdYqF z^9jcjaSF12oK#UtI7?#8RC7}63WBmcDI&bSozx)b*^!fafG$zQYddl>@_54=XXNxq zMzoOIgO1=t_~wvSoK)(B!e*z;Tu`r*DLEUmzYDb@y?~y?+59vt=5j023MiUTqCy#% zkV*rQuMLXOaMlWG{D;JqlZKGcR1LsgL0S`r#5m4!-VkXjQabRNNl0dMFOlPCXc&=> zoZ_b^juNRxFIOTv3;{H(1LS%v@gsMAoe0z-#$1jc}?kE9!Qt&cxzBEVW2aw>;xt@FZS531_P5&fm z@`(4Uc2Kb2FSs`U1XU|X#8mCy>B8UYoz71Gr`{<8+5RML@+Xs$Sz=OpLQ?YIYa4Hd zsd+Q*e{}5s4?0%)`(JfTK0bNk#02(tUNTaX|FCK0f_w0~8Dzg4eH z3%h50m|eWtuw_b}tS+oh?YTGFNGuIjyP}P|+s=6yxjWi8l8_e~vNPKFN*}lA(2i*1 zrw3NF>bxh~xaiPJeH;)gmf|1sk0McJU$pTv)uui-LFeW)Bv6i8FZKJQjVDjwAHM5R zZ4uzWnx;AsZQMIn_GTsMxE&Xd?nQ$=86#gs8^4LJTUP^kJH`jT8QE#5Z*KSNXd{%H z;|@j}Yf@1I`JAeg8&07;3#IC#Bfw+T+JcT7V*qrcSdRcVm!luSj6t+(rn~(;pZGS@ z+ZJu~X8-X0rsacOUjom}2jp;uyM~wiz3<1?Xk+KAI*Y6wHa~MaZGIL&moHxjDGX=b z0oU8)X&V=ycERMd);8eEOcn4+JM7*b2OP6_yS3d$>%XiS_%l84yQUV*!a^hVn|<|v zQw+Qx_~z3Gu{_P3^iz@tQ1$n=MX!}v3=2wXCmyZ{n$-i4bEV>&t@_GXn zDY4$5Viy5v<_`#QnuA0aQmn(HX~z%wd@Jn|qU&B_8+5z@4|;{GN}|8VpG_Xf{-iNJ zih2X6oH!hRqR3D(AT~1DGj`xW`$YVWOY>*oz*Z8SP{;*KI1mh$=%6=8(=?41shoqn zUX+^*VuNAQ@(YuPUYLCS0{#l?{5IbysHc2(A6lmsvNb~BR436=c8&4I9mQ!|=D1^1 zon%KnobF{*-OCW$wyrZNuR2V337L7d!~E)$3a}HE`0jvUi_i{%9f`W*`vbtr@ST9L6yZAZqwvi`7|>5HAE7fs z8bT;SAVMeLbjNo)(CCP7*teZ>yN&X{*tf2*Wt7oSQy{Z{%Pjt5!~VBLBv=bmFDlU|DHJWZV9i-a(pywE8GTE`=1OnGgUVXo zN`lEXclKQ)=Oiv5H~fR){U5q%1dK{^R!TNcFSr;anQoaU%x#a_wS}A$+qBJD>gGN`tvZPwnk!B?-mulF)!D_8} zryz%a?2Y|B+~z;G3+8=^{nUxji-pZk|53;o{|W7huU@P1;xn|1H&c2yBX zdS6eZ*L;ceofnaos)#IU=SRQTBO60REXvY%GYMbwHlGmw-IFQE`Vr~lQbH`>eLYJ} zq%SH6%JQU$+9K2-_iaZaJp;O|;95r_d#4y^;f$;v$uFNiw+EruI5&Ftx)qWB>V(3U zw`b*oe6LqZV)HKGo3}6D^#Xblu~DvCtGn~A6;Q74c&9=cxRCxDhh9)`p?VmW2auc=7s!04=Z=5r&$XJ2PTIueCay!+d)*DpL*B0CHL zG-kRI%T3pxC<%YG8o>262>by_p)pWROcE<=OF0^;+Rz zrVx@Y%r6%DZqbAkbAiP=`B^bMz$d5LccDhtve{42>05Ym{a5?Vf95CF`z(^kP6h;i z7oev#ftl5Y6U~0&3%_CsIrZG{o7MW3tK^(1%(h-!SmXQ6bFpP$&z8;N!RH+4415{g zAQewZd-i|ndnCZG@TL0v#Gwt(<1bF^Nlh_glFj*m=nIR74nAyPn}P4+E(Ap%^p*## z;f-SOMO(i`68?KzZt*jEYMZ-B^pijLBOGlh;fcN9#cl5L+4_UFes$0N>TLZ^;bTsi zno&X#JGb;a^PCkfQtE?N(f*>}snz;#whAO$$nrPj+tNZ?`bVu(0TgJbo*NOP6~z5T zJ;<^`DYi-!>HZ?^-)t^ z9V#B=2Q#jCWPxT9<8l@B@FY7FvRZa5RyxqSE)2@L$kccFsXXh5D*j?anp3x-4)s7t zC2J=y$(}l3ZpPBW?(Kfg&Ra>&-#Faf|Qxp z*Lo~)omU+@lG7+}t&XiYX1H16cPXjYDTjHB^vbf@dDZ&)uJt(;y$vSkbJ*_y&o*vVFLDx|XSiy89v;rCw8xR(K( zsAL z{o1`ze7Du_)K+AZ8n(G#@afrrzi{4wE{C7->4Ny|cJXYNa3ltJhoABeZ)p;n(!`Z<^a8)hdAF)+0LEKad~TBv&?TJMRYilGa(#|4V+KLmuCBIGBU z-CIWbH9RYsPc zdzQAY&!6w*+#=h_GDvfI=Xjw$ztN->Zsl8AaJBO^p`pG)yt7d4T#{IDdGv(5*_!!F zy~27pvlEPLQAkv_h*vg_F1YVojclN5Jq;K=QnjYYcInChu`Id!jjjcn-9hoRR?}!` zzb<=2K<2EX5t|;i`hq4Do@MXDpaIt%`O!?(A`Vt9#O2 zy6$yGL3BJ{?;ccVXyNZ$b|*GxRBYKjRg9T)vC!+LMv%CjoXZlA+jGvjs9 z_>JN1U{RM)F{@u-;O$1k?R5pWcgu2;V^=3H9H7g;?JFx`vIs9r4jPjrV+p$%{z0Q*NAZm8hx-LUDBr8y|I%i17F7}sqCaAj%DO*!;eza$P@mIdUW zzaQ4%JK=WYxQ5Z}$m#sf@ox3`qf-i22jti89e+7@!pExh_t$MGy1lZ!VqwmPf^QoQ zkpbfE;QZAa3U2GyC_H;dmiGR}w|d>2Y)8||_Vat#=S{HxX2OpnV>Tpb#jBPo?gw6W z9d~<*&v7Ofm}R`)^e%=v3vJMMHX-q2~m2Ir7$oh;WbWNU24y*NaRKmKfS->u0( z*+X(FbRm#?Ma<}@b|FzQbF%HnM8%9=r5mu-J*0TunBu~DTXemvQ*iDU;^l%vH+jjp zkGIAkJFr?A5~V$55YKpZJ!z1CKd8&XuH*Xl-8$N~YeKtPo0FM#Hudq}7+TbKmDj)0 zx1ZkvM|!5oL&F19_(hA;+O#VJM1D~Gg~PlsGHckKp>4YsoWE~3!m~hrJ8Rykp=BL1 z+V+?>EM`tcOi`6(?s11)?fLO-a$N#Gt-9=*sLt&Y>o00w&1hSnbn)J*5w|jGGv-&_ zi>-Y#!?k|usE4w(eR6V|me2Pe*Kle1z4ev%dRLD6Z0Gogu~j=V=IU8dtSC;tw`cjaOI73d?f9wIaN@#*sM@5!7liejZnb{q%jW zQ_saseGe{scxaRLzAc|k+xl?t%7=S5JV9+{9KFgvAx@$r=YULl%&6&fo|+(|3eqohMm~8?cM^!|IQZTZ+CX_cEiH# z)aHGiA&{Ly;1JJz@=5W{4BcGXUQ+0b;w=7lo2 zn$>3iUbOtrrsaS0rlonqzuU7=6=w~7aAq2z6-6Qv1s;ZCC)JJ;g(vRDH5^X8u+wCP z3c?YIf&)?na;%83<7Qrh5`qKi57@!Fs8vXa(txU*0%uVofvQPFPUGy1m{TAp@)}N* zXyBQP3ay!pi1tNR7XS?zgHu01IVEs$R?!-#^kk)YWsv7RoP?@*|qZiz2$= z30637Z>h6_XuJ)g0v%Nn1qgb3+IlNVOU~BQ+goAliT)E#RCs!W4Gvub9TB@kQK?X1 z7mOlN5drye8p7J^ga;KM7?Ft$te`3$@8LbA_Ks0DK=|Q_4 zT9arnGFm(s#*rX3x~YYf`4M;kQzX-91m`lvq*Hlu8Y!p2gLZg8#KwmzM4QfH64zP6 znVeLR1BN`Pt3Wg&c^4DFi6&0r^KzC^Q&i-b$qeOd*uPz{_C$VFUnhA43)q~T0moWLm4F4v%+ zz;^(r&~`VlGfn82$hj#o01licS*pE2-U`qMZC78=&_LSW2BuWbkuI;H3=&W1rb5FR zaK$2eq8W{Lt9Br+Oj>c43K9pP2rbQ!^AeGmhKp!`CuHVyT&%Q(IzvOD;dm0pTB~p- zM5}=)!IDJDuhHPi79JARaV|%|s=^Krt|`cU(@U!4OdIgDltNoXIikJ5>65LsUYvxt zP(i_XE0Xu$QsBkHtI?Z?zI zakXa8$rPMK;S7=(_HZXV(a6b^N|MzDlTPcYmVi-C!Dn63rfZEFgF<047&*>Io2G^= zG@`A_WI>7xMTLq}^o2SYrJ{d;l(RN!IZa<{g~}SX4S98nmj(52mvEJx`p<`ktaOk!7sVnC*5`vVy8_vNS)WOinrHyO zLI1Nd|KF|5cnt0T!+9bOI{)p7qR0+r-?7KgaBgTmRLuK#%QLx;C-De-s&7Bmm;HoS zZIvKXMXXet*rR|4i52&X@LU6N6sI9y3ap)^4Y!r3xlLHN4iP>YPwZ)@1BO+QTm(5S zdkD^GL%A~Isl(kBxy9P3#NyEmYUc>gr}_n>WDr6*1EERRXGKh(|GaCkVfy@EZW=aO z*=Jh0*5+*>uFvKInJM)jTKKo>oM~xSgXT4H#QpX zw9Qm{g55TJZ)3hF;O}!gY*Gb0;6$F#kvnnya2nST?D|1#bCKKaF_oUeg$Qmye7W;B zg8uG%_O6&OA+~zj)p-8JCa8r!W%2$Jg5xsHmk|HYVpxq!>i<}c|2wPkKXeJP>EB;M z%ratVHF7L^#srS0X}D}1N!3_oYUWEc)+2mch?LK!3$j#=(x50^2u{#~Mc?pjo$Vdk zLel+sj^@~QO{1q%u(6Hc-{xGZrp_zeI$JYv4m}^q2WM(5)18FL8WQAjc6&tLWQ}EZ zMDQGqNer%IhwM&*bN5o)2+IhPIWerj-_34KoZU)&vCc!i(g{Zr1B1vp)6lGRa&V4E zpDFr2WnAna&Sb%19W!w;Ar^R70hd|EFR4wwyi=W7JAT2I@i+vJiz2ZqCyQNvr?OM( ziqbCCwQ*e1=1NhT<)^8aiqc|JQ|szdD)5rMPub^aU7IU)Z7L_5vujD8QBGVRzUt_V z>${9AXivOR=d^Q3z@4m>_d+Ug-vyU~<3>~`AXqZx7PsJ8dg+V+{#?oL`h1Mw=m zP}6;sM3#0)W1xF!syk`qPFhwVgYtJ+$*@Lts>h&*BTM_F%~B;tS_hgI^odXgMYJ`4 zRCg4Cw-j+k)X8w#k#|cumE7Z19ax>xqmgaRw&~aIdtC33c5blRq9KF#TzKkvTIKesZI7in9dk@FS|r`@ zDDGpSa4gK#rk$>LYpWJ} z6}4?LA5p%vQ`tqp68`aNdnc0EMf~%VlA7C46m1L{1cby?i$1*4PIo;~*ZYr;KYli; z&N1PbPmGR_ASTB^BVTG#W|`E~POVY^gil1jwx)=>XNA#5#kFmPx(8{MDwT4_^h(Q+>&h-ye9|h9rnPfT8*tVq?yQe(y&(#Z z$DQ+;b+W+_Rv#v>pFW^ScMRv@fi5@^3-QM}X7M>ZRTmq>cZnoV{jP7H&Yn^`mUjJ6 zxNS&3P`}xY&+6uJy-%@tEKLkdhk(xRntod4-&CJw8&aGUS!f$#K^_$8nijjnmJU!9 zHU$=2XV{UDVNK4w&XFcXs**Ox5yv=d$3mT3YE$}*;avFer>0_E+yaL=#U1RjjlGxX zvT9w@4!yPuiVZ&NTz)9A-&xBUS%z@&T-X|&Y{8qb0TEVry*z4@I+dsjqc#h711+-m z6p|D^OG6i=bgD6?#~F*2U>i+@6q}62gdO-5*WfmhB`lhp%YlEzR;PR8erQB55@Kovj3rR7CpmJ2OP)~r)#MvrzBltT(r@Tp5r$+C|bl8SgVS(OsIk{_~?FS0jg z+fSYEyg06mkJKe7j`S4oB{(-0{37X;BrU=|$z)&C7qJB&HDtpW*R&s(gec2G z&Myu*{S3d|NnTpy5&J6nQb}^V)gv8osgSO9P~vI*?3o|~evUEOTqtZEKNsZrkr6IxDdcj#M)En}3ZNkc8 zL(95Yduh)y;)ZtFnx#1!SY2d;b2qQZ)?Avb6^=W{cHL6ku=UZ=k+C~9rMc4`-|qOL zw}<1QnU9XneAHo!dohtO-XbhmCsc2BUzY6tHl(6q>&9hTNB`nVXhdLWqkp(G--odtIx{F>R zCAyHan3}8g!^@J3tM#E}$(MXQVvmFsmxh=4gmebJPx9GrGlrLaHRRQd6}90%EK51> zlQ3)P;so`a85g^Kb-iZB`E17{*&as{N0+<^|Djt*MQzFtyW(?4x|ipsEJ$!Kt4*o! zNm;=P)kn6fk2o{T6+34v%S}F8n{s|_!Vj-fF4v@#*8(TG_5M1gYf7_g)(>=yrv+Z2 z+V}DgPZjg^M$VYr>(SLJo zouSO_8;YS0S9c9Bq4mw_@y)dV*)8$v{o^nD)EVUc4g15J>4=LkQI_=vKa`kbmgsNz zbq2DwD3O1BX_=e3IrgY+ne{Bk%7o)-oa3V-*XQopw9jc1J7%6a;qG4LqrHy(iI|o_W&uK$6qfp8my~&js6dC@RlUTy!p)Rjlgk znSJ=;zVI#D@B!`9as%TpzfHXSYGiC>0^X8o(bchJd-7$U@XNZdp6qjs?cE8TjCBn; z^QzT=#H6L3E>Dxx2ESNCege8W;E3UBQLwFJ|ISB(i?ckAro~Szj=$BqZ){mejMKC) z9lySIXI{0#XU86f*DAj5ojx?TC{O1XzRh=m!=~8@wUeAKdWIjBE`RbNj6W=A z?rGAdsc-I0hujXjiQnE$FVxA3I%I^N5BYYR zF8HNe;mfe|*@4AL;h@t?m+|d3qg9i0#jSCj%8bQ_oGaqOtDYxXwRJ0$#Fo?;6K^G= zq2eyFB_`JJ{&H8hm<6MY4~^?oqbZaCiG761z~VpBx;^E0qcE$A8#0PXpYu95EJ%ft z5UUy^`>?cIjFVf)bN3ld=^huR7u6U;o9?7l8oy2YDy}K)eA5mf;}xK2IK4~A`AQAm zj960RTe9O|8v2&zRN@?4(#7#;8g8C+g3fJzb?|&$NNBY_?z#K?TVWLog7Ko_%ddOp zRR>?&<(v2P?fmTK`PDxk_)?$O)VJ96#W!pN`Hu^ago$JSdf{=dyM0qP*Kf{^*Omp=cv@W4Sb2fvt;;=*GbyWC)x9_(4~mI&4zQ(&K;ilUlp~-JuF)hx0fMmud)}j5mr#c0)jn%*mxt z4|e}yhhJ0UU=;_pgd>i`Ud@>ZPKZTCOHCxsRuwp2X-WF)05#$?k)Xhu;?!7$CoAz& z&X)Me4CyUkkI#R`JE1vZD}>^N+0qcf6bRaf^H-2DQ9PmJ;Ka^jP){#&A-+z*oTUlp z)J8H##fu{QMP#@t3>OBRkza=@-Z32Ngp)9xpbmN zs%`I1!773XDT(AuJUCXt4 z&V+GfZ^ureK}6xK-L407xErxWRjG@XQ)pB1FcXog{kRVm5k?DE2`B#?(MSpI0K7Q^ zntud|nq2n!n3%*l3^y@>nzjRH(w@QrbSo-Kh$T6v1JHn$Ib(a$Ud7=+Otd8*gB(uv zO|8{v#?FuHM!w>&5pbvgVJ+?$a4CXwWJ83JC^dmb0|_)x(Tm@M6MPkaI$A0!@TPOl zhYPgzH^BND!oi}t`RhGZ(Johjum9<3vv@sl(nEUXe2iYDz-TxsEfA2A^nLz*R3 zc|b~(<7?*O(OS-s5Q52~6HI|Lp;@jFjXvbKnxh79PpN@?UT6%Y#y}%yq~21*fwy4tIwFu#qnM8`GgQaAk>996R zT#46pY$kOO?nMFD01GQsM1h+i;^Inijb*D5aTf)~BgtHJx97jnoIYXvv ze-65x&=acfQe+@jcu)}cVh{^Qvr5P_N`m;F`qqS7xoRZ@FRmn_9G7&atbCcCSr0q{ zy$PLlCdueHuYIC>6eP$Yj19x@@(Ia5ZjoI4Az*%D>$82t^ea1z|ocPUZ z@gNk9YejIPVzVbB?3k%9w;I0;n85A&CPKc439{Nywk*ZWiW0KKG zxrD5zK3Zu!W*^}+@d*i>()5}W6LyjBDb%1h?$2OvV2*Hk36m#rZG6C1>OX&+Ah4DC zpH~-0wD!L^xt>0Wh5FkQ?@xK@UzWwcU5x*8b@4y@B=mo-E->@|admP31=Ht$TwO4I z{@bgI%7;+u`#-){>i?Ik3nx3xpEq$WeAsQ@h)D_TYw{%AX4u)=wQ{#BjkOc)EbJ`U zFSexst62mTYdu3OjU}WsmUFFq{a4QTr1wAkE{(M>AogX%_0TJDl#A$J$^7jc^RqMl z$yD1(?BN_QFP~0^PTdWiJPrTM`gTVaOrE#0yJ7XOY+C=Ng8F^Nj1F3#^Dj0?{^Z;h z240ZdFM)&M&^?pjNskbF&Q9v0vci=*5&d!7{Z^^x_^j<-oQ+#!bxmlYJIA*?EppZ^ zC-%}?DmhD2CT+{LAsRo)yn~UVZ1*Z^%EZeo9CYwmd;|@a8L6DT<$}2?aWMWyFBofu zvpr{PBX|*`l}uuU$%YFg#hc)y-8HyfW^Z|5ub6OV=y-ojmQ-+M%Z3xYG5uy8NmMe7 z9QJmN#dsb8H^6YJG@ww#o*4N7_gq;<6yxm@QYrjDUXbi2%@#M*MjgPexDzL8QDT&? z%wrUeG_aV0WilUc=oq?lJ=l?ja#9msG~s3dEHjj|Oi(CsN579NFdAJ!)Z7!I#}gICFS6r8nUt7&IR+t^`-gkRr} zv*oBN+jLIqsN-xbjhUUjZ$^UvyX5S|xt`AX{4aCfDkXJzLQ+yX-midPMPYgs%k5Xn zUGv{ydTsvskLmUA)&Fm&TXQXMSN?@G%nq`6zHNAuv<+fYwZ(~${vk?S`#UZ>%->yV+8&?70>%v&P z&g(27-m2GtWZI!{lGV?sb@{cri9`q)`*Xm+4!5<4bB~`#+$_1k3sfxyX5{aefLR-* zy+eG3PBR-(z8@YN&72S5>&PE?HMNQYPu+?j0Xj0Js8~Z3sINzCpMKz>ZI42JQ+H z>Zvl3>xe{$X-I4a_#ras)4|^t$ecsCkAjvnz`{l(yfT4Sip1}8WpIj_J3A4%R)xq7 zFN3H8)$^GLtsKW zfbbmx6IOAx%$G*ig6yrZF?=~4)DLt<0We`EYC+h-rblCWy%1y^htVwAmWKGTWdRgy zeaD&iO^vki1GGNa9kr5BYg%Wp`h@{Sd;Ag22tZ^FlGF)5d!t_t0@_v3oVGZVj~5an zL5G`yyvzOXcu_EE&m#W=$aHKEGTS-<`!S&Jd@h57yen7GHeYccvGt9Ih}V=bMHljv z56EfyWC5n9a zeg_Std-GA;+=rs>qYJG-{^KLgps?>Kigodwz~4fe4*r%wa`m9-Xd_e8a0TeU1CJIV zlX-(+6j`GqFmS@eIg3zb==pLKbh!mAjDRpHe;vplvHlu`Upk`EiLHRz_`xB}tIL49 z9s3>XZg&9??joXJ?jt((1lq5)ZTV2k2{UeFg0so|8Za^oOcYr_P%Q+IS2MB6kUIsI zXoW3`)}Uyp9folUSU!eu1_fRYP_`k+j7Na3;rEFGesU<#q&6}G610CiLYcxlY{6zFL*b0>o zwfP)9GC@D6y9XjC%y2FM&FvZhUbhcKZa;FW!N^UHL2d?ebB2KFh2U!y0wxBx9fe&! zmg%`o$T0O~J<-IWCqBrqx&AQ%o9EDR>OBAte#1Ce4PRu65Gqh0j0X)7iK(fe!A6AH6gDFB(_|Xj zBON@}O+w`H8KULTQuDxaOChmgl_)y74bkR0L@Qn(>i-fL^H6>1&!ESwb=ZWBLA&W7 z&=BU^)k7=at~`!GTXhNr9d}P}S~@&zRNv7~qN(A*nF;rLQStJ&)C z%@Q||`R0BA2xPoQsoT#e_3`SDNw)Y#9Ek5aw+t}H??K!htiZ5w1qy-6D`8NaZ=3{7 za}z9AuCaKcRSPflvEcbvwAR`UT!bQF(Gkrq>x}v}$fX!ioDL#yks0cVVJ=7JbA+!E z?jrmGK(H6|kM$fDk^2N;HUcxe%;2iLG5+n4@kU^4SF8`l{}vLmE+9S=iKpF=03qSD zeyEFCbA;pSy%>toI`lEh1(6U6j;@2kGvF`k3*kQOkK$z!h&E!1(G~+Bi8Gt(`54x! z;IZv4=B&@K_RZQtI$|Uc`i%kwX1SEL`7nz6K8F~d)W$!9%Z*YhT?-yyd}-1;2q1AM zYPjwK$wwX|cweESWfDZEZ~}?$AW=YP@M`&~eKVhvc)I-{(3DdafL~=c5HP61N0miD z$a>-W0_54Ow0;TlF@P*8F{1T?Qq)0>au3?QZz$~cTTX8510H*Gl- zW5-MoD`aLU6@jV1C&QouWymxkBOi`x%%GGY)S|#D9@U;B;hu<&Ix++F8FIq0O*ua@9iODGk!Ewpl=EYG3{j2Cu=MkWeV3c4s8TZ z0B8+zgFZnUu(8bi&>!G7(FGGBr4BF-^!#K*6;lv}ehT!V!1z52l^xiG>wOzV%-@2A zqHUT$9O7oZ;Z7njyE*fYOiw%A1@4Lmka-7bm!LfELjVTK(dG*zY?ybT?+%Q;ixBlK zNAx*}3&M?Vd@x%Ugk!IJKye(6CCBj!H_vG``xi}Ae&SJIlzf}=@@#8gDnrz47<0dPdXQ{F&ZWuZH)!4ZMafn5(j zLB27+>(2@0)Deoud4gSsQt<7B-Z5FZEk>7nBEtq~0Rol`%9mjb`Xe(1;dd0|Ey2jv zBl8n!||ueIJT; zdYlJn(goxbm-WSrSg8dt@((Hb?1F+h?Y@;)1ABvaV zzKDvivL2wI4NM>{VT=SUN5}S{|Bf3mBs*cNX4Ut3lf_Y|2Gp58C>V8Kt65{S^iXJF zx3LoOgu)t7G42P-q+J`pTFX74b^HJ#$_#DqtCx9=e8twTzmU;F>yv({^6v`YV31P~g%R zl|FJsOBrBuQ_BPlm`+5t8wB}`_x}w>zl|kmnruK&PCwre-hSoij`&{$uLy8KM7+wa(C zck}AZ3+^+*CUX<8Azv~BZ>GRVXM!eoz7{lHm{6a$-HIi*N3}`Z3msk+Ik%4(- zo+S<13`}Mcpx)fKC=j-wAOM*g2+vV)d@H8W)oo}dYCD>NXH6}3gO~1rv!w+~AWh!` zoX2~CqlF%G!6;@{W&D09&!q7G?5KUxO|0T2ahGs;qo`Sm$v$gzY^X33s60^s( zEC(aTmgs|XKxYxmtRODb@kZ**sJX*VQCBRtbX5mfF<4w7d2we$>I~Z=CaIZ}n zV@3-CU|86bv<3`(e10Er$E?_oc-7paK-j?qBwdVZx5G{YL%HuelzxmcHhWIcdD;$s zhOl_|O^nz)SUb8E^VYoZcEAEAUm@u)T~$sSq#zNrISMVS_T_`T^&y-q2%pT`4f}E8 zXG)1dSVgT+biPW3Sk*y`_}A=k#4a(%fW-I5^_77}Vcdm>lL*B3CG>+%hnW-lcb9-S z`mqami->gM=&A_uW zb4Cepfo*=PM%^Wrt5J6t+xss?Ew)F>5K)UNkZXfdHATFlu)27=A{VIZyi6)ph1|2vX0+6T$&JZdkp zKy<7VBI1e2n_-?@9RiE(J~b3NYXm{O_#NoFK!DcYAB$uL^Y(f^L4Mi&Pcc3}dNUtT z=>tY!(1d_nGhwnVnYoyLG!yuKSbu2WlgNMQvK9E=gU&&o-I7{cYmM?TEo0>{;0e$$ zvnl)%f}2=pqvp^rVP;GsOu~U!z`jFy7LF!p0Q7)9N2q0YZ2II0;k6x})mLf6q>i;tmiJD4yCM$Ie&Za8csGur67*{r#;y-or$ zKcd3$1ej2^LxHxM_t45jK-(juL+FMOiZB#mI0ECh4&fYXwisa(f#jdoifJzE`vHKw zldz-4#Gx;-f1(20*Mhk}mL_bAm@`QSq<}5v$U2Ncq8GqX2+XQ~Iu>i5Y8?8|9b;*B zle=M$L=U8*=uH|T>vWWR0lgGq9|Ei*l}&`(JqwBBlYrbf1$a@|G*K=C$ZUzuh9|++ zYpQ2n?e}?rwP!}xX91!oMTp+5LKLtQ(MdKppI=4v^co^}_)X?*X#uK_ya5xqDSTMa zO8P}V02dBIv>*o2htNrChzIUq;9VJkW>&#I(yZ^HYA3vhLkq{=!3N2E=oS0~?7)=} zc+Vtkg%8~xjnVvc&SWUa)kVM`_TV#UX_p_7TtO?5JPX4@KM)a__MnM@Xkq|16+)PP z2j;^kIQ12z-Eo*i=O0Hhs_|QB_8dEC4;^ZcO8q>b%>H4bT|y&$!MN=NZUyJ7t)OH* z-I&!3eTErAH#|r2^xY|_eSSQSSB2N@Cn2tFn+c$$20-z7K7cV_E<|yHKl6zffB6l7 zUw|#)>H5O0;*(om19JsrEEli#bbwB`dDIIf{a`uF zyB2I1MQ$#}6e>-1Pod7AgCPzUfGvn-ot3|UiYk#=jPMF*2VbIq`B;Nrp)%$U%b3;8 z=`|owH}k&5@n`5>J7l5}*um3h2+ZqKzk&1i5fW2>L48>g$kjmLdTiF#C-!PyqGR&_yt=Y@0#V767yM zq?Sl@1m0t0S_>dJ0tpU>#%4dVKOpl^@YWi&+siR#-`ODQtU&Z3#?-vQRXPAW9+*3T zIkqF%JnId(M-N2HQ!pZzBt{oC9n401V_UXu&zj4Q=YmU6aG*^P6y|{*w0=cb+Ry3@ zZP+%qKj7(@7G{eOl8*-Zfq2|PD71G0=nh>9*w-kMu0oONEQ)xnKx~_8KDl8+jlTq@ z5B~6;NZMr(VKqFjAu$ekY^xl29qdlO0i5H=Io<@w1kg9gsBZz(751LmJOCW?R?t*O zxCpdyEz0uOf%)Uy5UeRLt&zV0;h3H3XEtEyXQuDo4k%)S(N+zzthH56m@PiHKjveo zR}<>bx-kr$A9NFVGXS1`3t;~r6R=e_J_Pu~JJ@;A9?WxX=)|VOzMJDAm%X=ue-ik& zAET+U@E?T@ws@ZLlk+8xP|o9xi;W3(s)S+hx(d#ea-q%pMXX)p?c;noVw zssoChuqtdd?2JPKp~t8u2qqp@M4okAqF%L5sJkymy*ZGiP+(<)dR2p>P*6V@1+9C; zpn!YI+@I+~7a+b?h^b2Li&207+wqPhMo1Y}jFNX)_k`~w)`F*{kx0(0zlsgjSD*K4 zMGW#zD_gh5N;mCFvJ<{u4PVj*D|l-x$-?IjJ5ghBYsh4?7EzavKzR1Ny0&lw`(0^3 zg9|${_cNm#fNpTTXh-H(960aT-J~_#^QSiS7Y_w}j(Ow=i=ZHvqJcd*0>q=yWW^Tmev`ydkw&>Gv@j zV_8LOIg{4RyR+7|CYh;(0v@biM zdW%Gh#c1^k+h_E9j8-P2$QKr$u6=>J$=XqDEzpleuj}C=Q3blimS7>U^BlCt=Ep4L zJ~AwUvo{jCRj41%eEvR@z5$o}CZp2w>4;vhMs$_UiP@MFbjW%{RvQo%?*>V>J@Maz zX7zVa)blCY_O3a}kEW3fW#T48TY)Fc{`xeit~iMNHRwLo{)9*l#)ZLs&Ox^=Dqlds z;P}b6KUZtCQ-z&FaIir;!qX5AyWpUN-g8CLCNK=Dv?g#EjDH7lycX{1CKxOE@^Axw z+Amz^y#N@+CAP<<_k*c!I%vocbV9iCMMo&oG|fvFf}e;%N91c>+PR{}GpRkASq$3d zseHNvy7E~^@b=^bL@l5Z=7V0QBwzVC!o?jpoNXlr+rn9s7i`q%$!Ph);RSAJ=;~P1 zFcqN&Q=W7p6*`{+T$P=VS7q7AAw)=nopmFmYKg{oM&f&se4bkj%2DZ zO^5yoE7oI^C7<%!*bi{2Kk1E^JN&y`{~lm>XaI;f> zjKZL=k&kAhKCKRg{;k`iaHkXUJJ^1u84cfeM#E{S$1YZAst&nq7vydux6c*1!5xr; zo|)I%=j>K#ksAmcW25ANQ8N2>5GGCN1elCz`iZ$RGm#U zhX+`bxDPFqerFOTy*=nV3|GaI-cGm@)EVqXKIy@%1_ znc;o-gH#?5Ty#iyo)!Y_dIl>+k1@;E`Z91fS%qPoH(Db(AuSwgx|hM>KtqE`6!4cw zkS~)WZ?ha{70gYgt+3P!jgnyi9|eRSrU1(%NyrPxvuz7qY7NfP)d0LDV{k<926O~b zg6)>@r6d)ad5G1FLJx)2z0=@oq;Ac^RO(_?3RqqUif06sgP(QS_tOh8$bXPsg9aYX zLDIYPd4QI{?9(-vEyCMF*FYh+;b+thMc3)WQ;2^25un!vzu^qlvAC<}Dp4qW5DDA0 z5yDuf$B_hW1NAgt{$zsVqCm*~^HHeS8%jz2#~~Vb90<&VWNNnH1T>-;30%&K^v-vH z_c?>84Q8Er`APX6!?+s=gP3{m07VcqmIL_Bv{(j=6CiT~z-0DpLNFW4(<($?bwb2V zMU;?%sLu>UTVb1M_)0_%_adse2ukOyqxq>qs{@131Lj<<1}SAdNF8KjJ*9atZ1di4 zaF8Sn4juwIn9|TDkjy&_BObe<+u5kQ{DR^yQMd0~3}Sy_I3G;6!ORM=T_#{`24{W( z`cN(5+@u0XsPiaP-SXVD^9KA;Uxz3x{X$dT`;4-Wm)FlL_O?b3q zB3SsS8YGWxLe%&rBK;OXGD#JXvM@D-6Cvw>G$<6HoqZ7|SNSnw=t2$(7TRcJ`i{2lis!&yGKCxnxloT! z5RFL3$W&pVy0*4K`|GQ5>nq%uF$T?Hd;5pU>OBjCbNO=+UwCsI^yB)H2|$vJVq+YE z!Q)0l;QxoMHxI}9iT=jjx4|1X$x^oLTcPY*DQn42NVFgdDN8Etb`nAeZAd5*QD{Yy zN<~R~p|omI+PB|p?$7u8T)*eKoWuzGFCsMl~GZO6x#$TQ-}Ko*8V6`rC=H`caCI3Env$)mL5fI#;A>ZI+=)*6jIy_ zj5WhKIF{CWhV3EIq~@?Xn1rzzE(dj8P-h~C=oI=5BWONUhoBh^ZW+e0I@3~k9mS-J zkdU~0grenxf?Ykp*a?8lCgk)Cy+QAc2j63~#`&a7mAjd_#l|S-bO%5$gFqbpsM!8zQ0bPATe996(UItp&j+JqS8F|HRnPeM=zj z!*E!SXPU8!NZBi7*@ePk#&}^dhZsd{#o!JDO>PYaAP@3GX0-#bIsitxfPP~%5!^Uw z3_;eE9Vl60whLvb0Xx|oQbu=+t@!B9f^C6CM*tBid~*1v;?u)73!g1MZ+zea?g>OQ z3h>Mn4B(MPf+jvH{`E*yv&4$I3;{)Jv}wo&E#cn&l7(^2J*vP~^uOaj2ULeul$79K zV3I@^WS>RMz!lA+eG+s-fVPWqM?fB>X5jO|N7bIL2dbeBq79kk1(mauGXB;j03MY4 zor3`es6~Ay34o*l^koCg!v=+XJ_yi+DIwX|N)Z!kwY&tSTi?)@Bi}JxE7WuB#h8D2 zbO%Du7XR1H#OQ9`P0f|7z;O$v%f_Zvpb!Fyp~f4jUW?ZDBJeU6r1oz{UM!4F?%e^p z4QowATQ{IJOs`@wOyVIWWMX+I5xNkNSJ5T#%DiJ?dqwWCQT8G)FrTof=RDN~$6+Mj z75F#NaWt1}5{SBC9Cw3&JjRKLy$1fqUDYs3uXJ^RNwjPzQwO^tc`WY^I3Uk@7ztZ3 zG0ZvZS7?7V4Yu37N3g~g!Sm+hK#R2e2@dy9;Wtmg?=3*fGmplp!@9S6J&J!l0>p6X zbd;GRi(rQ{y`MS53rcNOm#PKJfy6vRJL!!+^e6M4DR)d8?WFRXvO#t0jL=6qGtaE1qJEkLkP%h_+gk-D^I>C zF4c(L$n>lL{*1E#AF$GpLaK)RZN2!+=#eUOtzi#_%A~jmGUbT)QTt*DF4&m6D>j^q zMLYqKLVRcbrxn}gfpK92aPLppIRIlDB6P9-=L*iyz(i4AA*z(?+mY3e?>7?ec%Zj* zKt(-~vlywA>ixD$JzBxW> zo2KKth}^s&*p|=?JVS^u%TX{p1gPbRDQ-qVAF2Zt&jD<{0Z>W@u@RXWZ&4w94MYvD z2_7c|7Qi7!-oq|s0wqguM|CE6oQb#%&jDBS2=)VS9C-2XFyKPI2eC0hlri2*{fOHS zfOx`DlrW!o2Ye6+#FJJ-PkB-TB>2l0J9hasnekdC#FH2Efgk-DWJ}Pg<3Yw?@jrvs zke%Z)Bj95v$Q$_$F(%2>en_PDLogNFI>{GUT|@W1gW#!t6rIhAVt8}JP~yRUPdqG7g5jAZ`O34BuBE@a+<`qKx_94Wk^0&HviC0=bBU0VNCa%H3RF zf)j+hAA4JfTFv*M!cac_ zj5XwyIU_O#Nq`P#1UG(${+N0hb{M_nMR>YV_}n z0Lc9Skv(V8ml_0wVOUc|#q5}%jr&*F)-nb5dLVPziMAdL%Dk(!#l*KfqvpO9wW-<; z`J)rizD(#BdJ;GD2NE~Uqo)T;p>H_4pZRU@#Ip;^m3ehe4Bd3HmjTzj)~Av%r4;8! z@bC13$F%kd+QMa{gt2B?Yp7?q>o|gn_o<&WA96$n>Gmdcyb>OYtZ!a+EGtHTKXqbh zLOCFr{S@Oh2pfxY`T3lAEG15WK&k`WGRGK{ z)NrpwvntGgpb;0Q%%0CuV#A&X_YRRIL8Nmj(k%N+mznKK^n5x|_YT!y%0vxFU5bOqG)6XA#Rvfd?DGPAz zGZT9SCQ58E=FU&aNLICbqxbV4?e!4fm1@>mvvZ)(kDatc! zHr&hfDmN1f=YbemfJCxo0&>2wsPo_%8^Jqp7l@HI1i9XtIO<2ZBML+l@FMXq2g-TY z2^i34fkLV%i4kODRn}eV?O})hu zN3f5QSjLt{(2=I5@GBq~MRhVCuNrFmh!H5`ho6&V(!^Re73)0|)q>XqL3P^l&FonS zuAr%HYzG7{&|m}G6~TQpIGcDOD8+~1DqcSX7t&&m>;MGkQ-qHqD-irZ2PcCcju*0n z)^?N?gJ1!7M0n#6jH1Q#g|;CmFO1+*b~1ubCQ?}OnGwf-jeP_)^!*(d) zRE&9c142vJIp)AR52SuJ@kHvE{nX&j?ZPIQ^QmglVpb@bd<2wEz&PSsJ?m>AAp0N< zhv6t8(@r5zsu|$t835z6z(OPb&vkUAb6DS$&LhKeBdFy$YnT!IpQW&5e;5aAr4VP;6=K9qcL2)WTG0T$E# zDWQMd%qm=3Bc*o{D+B7uXNXBZ2Mr!(F<%rBr&E>(aVNQANbKWZf_dbCC%IF8>BBNC zW);do;D&M)P6l!F5x3yQR*aRF5S-HJd7|J5jwNVx_+436>vo(`WPtqIfzB$Eh? z&4|{1c?;1k_y!91lY~Mbj0PE?FpH8&JOrd2=!A0x$gDvi;xK0NB&3yVt{tTADsQ@A zO4K3=TPp7JA-baxKL%M+N5G*TMeic#J0gOOkR?4tRw2SPK~ze?_+(ZKdUsA!nC~80 zq=mSk9e~z0ki?qEeJ*ece<6(AlkMZ%@q!@`tW4^%fYJ_9#o4H&7J-|&XpvAJ^5kHg z@=Qk&ZzQJ|qpH{vB#Z4u@=_#iIs`6jR_E|_bKN$N=;_Pc8Ne_z17L*%%v^2w>+z22r5{dW^f?yL?_+(7oSGptlWF<#u zeftzl-N$)~NLHJVhB%=i^Wi7rNk(8fPNk?LitM1<(&Jk|%mY)w_$zDRe6zVz3#DS9 zaOl>V`TKb~rj6clr+0d|?^(rJGf?TJ7;p{Qw-%*UU|Dgkdxs6644p}~Z9;%<3a-&A znrO>JwmE^i(MDIajCBvrecW#h1r!p45Rh=%#qIN;449=nn~?nM9X(@s`=<$t9kx4h z`iEgm;ej2*n0}!Cj{Df65v_ly8FnA{niSZ9BmR!HBCwao0!c3TV)0>5%1IuQTnT|& z^PC!YGx>4JlWQ#=TOk_?%kafYRNmp+3BgDYof(D`xnGvEQN?4h2Xnla-UCg0i`x1} z0WLt@GSlUJFfS|R5%-*e_@b9|U*{sY7~~_q2Li@LI_ z1I}(o90D-!KFmb@=|xy;i9|7qQ<1Ex2Hnv~CooiZp-PDfRiYLerzk_w;7f@&7+N7I zWZ6;+c^$?RLo|w0pxQ#OHLPmxma}Is6x$%5R3r@D{h6jwFm&IDQr>HyfHJR50E(gV zi7}O@wJ9JG@!B7b0RE{F7|2~{Vtyuk2 zG=K^NfL(?lVm+@97qs{sfDQFUn&yL%f4L$T4nw#RA)VaAZGCu?NZdli=o-G<7b63H zWXv86z&UA(g^a5NKGOAd3fPDHSc`A%U67s}3p(s|sLFo$5n7^0x#H+BHFpzfu*dN) z75F>AGNiN8_ykO{{)|G5vKe!+2|~&Q3@pOLNZ1`%3zjW4qWbaXI}8l>+IRbH(BN_Y z3d;8uAxE5gu`prWC(F>0TvLNXAA`mqQi6{vxLQ00C0~~TpU`ctK9c9)TY;|?UpKyo z_^{3qn?Se}JP?7E!hOWF6@M%7wIGu}2uHclh?I?}6M`&Q4n4gRk!(a}2SYt?M5GiS zrS~3*8$!^Q`v~x?z+lE#ByRD@T924! z0%D@cz^;aH5IJffSwjSvV?p242TAR61f?y%V_>JvM%Ju307~`%RRzeAy#E8fG_WqS zGw2U+_5+=o3LM=a-dO8UqD_lWl*E35>!M#Og~~dytz*WQ3xdf%2P62eN)#oJS0fmV z{|Y5WHUxqEI+P^35g>;ZhS|7W4l=VoNEefpPIJ;|N!@e$m@IOhV%`N@zEt8zLnrxtP`K# zAc9FEpZL_j@>1|Ov5K$CI`QH=sOn<9`-wfN}-&^VlkydET( zP-SF#y$m{Cub%*u1os?B?xB=p>%8zR+%&zxVDDCkPc$9Sx0yl%| zv&gSVNkT09Hys<@xwv~T8NV7OURlx$N=0;{MKRHgqRBmMKhDr)V*Q*rRyz4kyL5zh ziTcW(frf7q*y;u%Ul9464Dtdg$i9Xl%G3428!gJX%i*&_jMmxiUJnu&#{Hj8nEyc{6TxEf|8^gR{4P#x~r?Nw-_So%P@KJ-U(E zWRDhKmfvD0UdyMkK@VnD1AjFcMIMFRo9KN0Ss7BOs2dWz))I&oX zMTJ*_o<5}`Lfh=@iSla^h@p1LF$6@t5P*s%ecosq#|HzWN7)1IBfG#JL^>2n)54JS z07<=IQ^LSRBLgr~n0K=_sDC8)ED^s*r~Uy9Akt4~dOl`>D76CTergHHM`RZg9u44hnd=gW{Bamr8!G9E(s`0QA8FAftX5 zxdG^tDRr2pfF8BMm`WkE_R9(2itQ2sJLN!GkyC}h#VHS?nj!&pup|{}4zx#eFci$| zD0+V=Z#{xpH5zEFt%Ej1LRdowi=qNb9SvN9)B*Ycg-nVBDLQL(#x6zS!rCK~D749g zuB)qDXdCU z{p#!qk;E(C@Y zQqA=ZtVw2JxnjK2*J03_=}Mxnvk`-Y^^0kdO2>>P!3HFni~pw^j=0@V7Sjb5dW);| zl<1BAd1&(LAY?U!fyB?zEIj@brb^oYSoZi9f!@5fY*eDy4kh}2`+`K}B60$YANP$o zQA+|cmU6;L1(mHkkbZI}z#nigDTjh0`DFmt!QCVcjL6)H^x_L;?9O{*svgw%qhFGg zk*R(dp!}(h4K#PDGd+&jGjJf;)&wxA1?gD($vQfUGWQVsj+p%ejCTYYIUu%#FP<4s zSO&T)^l*8exhYGpsrSgEa5F4sqN@OKTm=Q+pkO||BG`QuNm@Ms4?vJCrqg;YG#sN( z!ht6ZygS(Bat)U&>!B8R%#FZ|3OfTi0!&J_QuAy#wR0$U4*G*;10Cd1w8-beYMgAY zguIc=MM0FVI>A&w#!4Cn4y+xexrZpneosnGHxdQ}>ZB2FDQ` zD*26|>^@9m%n&Qdx|=Cyvmu(zXc5(_vJ;@m^|45haOfuHOb)#{@`j^^YI!+yD3^;M z+37jG?VR%l!S?N2P+6lZT-MC5bK8M&9nwm#0x;?aQ_=E$Raj=2uK0_988%oMndT3# zf%b^5sMOE-chLtdUc&d-$*f+cnSTNDRtlg2a390QxZ8r!*hOdh5SrJ09~1EifytP1 zA~FcHs$l>-Ebe3l3@&o{CBV^l0E>SDM8X2WrS?`hmzMG)*b_)C1@#T*QA+@dGoNl^ z|3x-o(z)+CVQG>^Tr}sNNVyN9lH}*eqI>Vj^rUeb#%VSp8{T1@?$a9FXbtTNm-yni zFI_~EQ05x+A@}xO^e)5}IFw?(%#X_0>qi>M3?;AU>zuKdUcVs(>Ly~5U?u&A?gf#QwBow zD}bH|4NqT7av!4-p(Zd|{?dNFP_E0Q4I2S5#y=6fAcu?VbUuSXRj} zd=E^K`5?6852`jt<^l9NfF#y-a)xeHu%<)KU|cZ|JYT>b@)~-DarVRvF((&{qK;QX zI5T4$LUBIEJwPl7Lrv6axPnSp76Q-Lp_V0q-*MHY%y^Fu` zzj*jp@uUQ%Gc77C(CPsyAI^K2=b|Zo!#GS} zmgqEM1OjhTZFQ#?8pkl0vjZndVS)LmroYHK1DCcRy{8vbMj=GR;x}pt1UYJ+ssDj$ z9Dk4rO*`Wu(!-Fx1uYF1$)~=7&HGWRlgiZId{oHUf>TC%63t1~6WGa0X9_l*MBVGS zKX3F+8fd}P#LWPKR}o!Mlx7=|_Hq*%by;33ljXn!dK!a|&>7;5O9Dgz+{V45FBk(h zp?dua7?Vsy337Wg@_vHU{ah=U=2XqU^}xeX=G^a6RBFjv2DoZBe4k9mAOwJk`Abjl z_LTBuu}T?TADBMe>lvQ7B}TWljFag>^y!)yEp4&tA8_51gtFoIOoA;(#Hsq={+voJ zy2`yv(n5bdXaO_`Bd{|9limyc$rm$5#PWD0v$#6 zJKSBMwj1a54E|hJ6iJ1kFyoQ}7=}d>;y`Ut0l~{yBpLA~ieTS;SWFpjF(n|(n?D^x zfUO?)#?^hC!xA13U`y2kW&4*xEIAs67!1;4hg4a6)M<@6x8pag{`>Kwe7t05{_5iH zJM&3y5UtZ#{{_8@95%~i^-^`jTs!WFPW8g-Vz#e!f}p~6z$hG{&wJM$1HB!nd@nwT zAIUxq9|2Y(;&}$cy8j%q>k(TD>yjz?g%?2FQ{vwsux=B62!}+%wqe$7DFdxQXa>rz zRD_8+t~4-LMyj#D#Zt<3KWM)|(n{+laNIxig<-#<&gl(b5De9Ng&^AuELd-)wp&?? zk-A!sKD%SBBP<9sk0fe*D|QlPW0X)pWqAh*q(&kKyA`gF$}Ry~0EPr}Z|E8ZO;7|r zJkGq26Z0|AjTh+=)g4V|z!KkB8eT^|oZorQ0%_To3djWG=4}r>F<*`CMno#Hv--G?~A{8g(XokQ5 zCo(HR%J3Pz$+H1w0?GJ-mt5spcNb?BhqHC_7viK$a4a(=F#z1g219aY8YY zsfA&;AF+x9C~NfrVDd+#XMzY_m)gf6C1)svH{vn))UOnX~;Cw&w7sF1sc4>qVCas zG|0!Mh9fU8cr;o=tSC^AIA|hyLY*ok3cE|D*~J_3T==XL%O6CiNPWVhG4>M>p{k#p#f@vZZ;NwKY_UG9{6+U zgBE6M2;I)+AT;T~OE@!jKGxdHDk9d@YV*bb(K&3!haPk1Qu-2{Ytg}N{fj0h^GwC%{74q!^)MgFLjlzI z0fcKA&@&cA>^$nkVn8^soVhF^QS9F$$E*DII6nq6{e}!?97hYgDZ*=62WgViE-U4#)wV8Ykp}mMumayrug{L z%Gu~B2?5(vUnyaWAV~uDIJqnXa8MQ?LIuE971hYUb%N7bU=;Bc7(<>-pt(kjz8bL; zN16+Ya5qoOWdCO}CdeTG?Mp=HZ7mMn8!D6oNLK>9uL4Py$^zJ?bY+c|!QE?uqS%6- z!akV(j1ttzHGn-gk*Ke-3>C@KcAV-91lP=$4Fcm$E<(b>$@F}9!WQ6;Lg_OPw$VEz zrz#M?SPifgqP9^x#Pu@o#p z9hZcFAc!t<_i`caST6SPl|p&efJVctqd*iiZv`U#^-Sja09?DA%1;ve5<N@~FvmL%fMC?jcOV!r3ngQ~Ah7ni znz6_dDwC>^L2HcY)VZK;;Q$<}&t6f>T+9(Ms7zw)gxERCWs6*pl#ir&FzC3~`snO) zR)V=aDKpU@VWVy{=D_tazHX9!18Gz518g5cbDo_11Pej$Hr8FD)DKKUG=-|evil%_ z?HSCgrvSGG0eB&9OvSFRxTV$tv>u=x$^n}EK4i*LT~Iyt12+A)`l!ZM5;7+~fbz>j<8=p${5p&lT)2GDjeKwqc$UgRRy?z;&=X0`Yl=ZwCK8266jY^yqh={wt+75xP+N$hF$MK# z9MO7E*S;a27w;hyT8S5^80b6jB4>W>%K}Ld`t8595sRmmaPb5M=|D zdPEH%KBuphxaFrlGP4Rx2XDm^G>&FvO@ENQtg17wmp$%3NdToTU>O-?kh3Na(We9Nwu$+()a02&93HKyo z+4hU5V1wsXFxdf=pvzK@FTF}P#}9a-$I;BcTMUk{>WR}f&>l*|T_w)zh3um^EA?6g zl=VvhIuD}Nf6;1ls5b7d{2gw*FtfHy#;+N{vZk*vnDR2JPnzhcNG9iF&i5X;iRN2B z!#05oeE^xmA3+8?aHelMt|pP^Gk}_>{ued2T$^0Os^^{z94f=p5K>F$d+m36)nooo z1UK_!gQ4jN>%jr~=D2_cH042dF~) zUJ*9ERK>XBOULjH*_VQAHcz~OLfwiel&lVd@#YjeoU<&KdR$OREctZVAtN?uOrkAfg`js6-#4TlbHNb4p&T_4oX0&Lg)YLcr-mVEW(+_KRhzY= z$fO!w6l(@ofpH*U+>27L&0XI({(b1W063BRwrM)Dn~^GXTSSScALL&=04RimfINEya2h;H&(X-j zCnzRI7of*<0m>W(aHqTr86-t7ky!x4k2~I9-ykOV76>O{y)Y6Q3)FeI7q5tr4ia;w z1MnmQSrbKj)BYP{(wqTqz1r^uS6&HigBe}oG#z}CU*ZL1qLepvYXmg{s9|`C9e9+ewFAuSL>=mQMH|I^ z1%f>j)AS$=lNcOI-|OuJO%`>iQN!ON9OTqjq14StfX!QUx%x0j+}NP0Vpck-JeIK;+aJ2*LJ6B{+QD^_eosVJWJ&ss~;I zj*7emor|I%9Jm+T2Ifc}+z!Nmc4kUGxMEj3UsGTJ+0+3H`wr|Nm}6%((d;fzzK0Dx zA?O|n!zx1FUjd*$`b2#VL=$dt?lH`TSHK#BP zTUQI(TQfZKJkF!vCRAKNn)dSJU??o{#55SABM#1|YMQK!0|%8MuyG@#c5QG2eAq=H zXnnb63kHDI4j&SC<2dUE79BBP{WkoZ!cQP@kKxJYRMg&`t=fT0Tba9%`VuP_`BMwf zj(r7*qf3cV^rtjocsvZr(#~F9HlbRVfFDk&RT5HX&&$6e+#3Uc+Zpd?W2=?Xn!DrXps2GAjLNB=CD@7Ob=o=g0t*rm_=v#l zWCWl~i5t#?hy;XydwBxRezIsMDyT+aN+t&O)EXfynS@H)9Hd3zkd%aPLhD{7paY$1 z94G-deZ>2X08Yc=CmCM>G@;_i&>!G@&zgX%1{(?xJh+qIUYdesgGqfUiy-*`x|^u_ z*eQqkGod+75mPWI{*j6pluwCtcbBNV05HY-B!*nn2PH<(lA%RtiOgcuau!;aWGq8^ zC(_>pAYDBWNm>y|a>M9g2gJl5Y5ng+jP5(QFTE<~TnDQz6Q!1%m9E|;T*jI_9mPJG z0|=rlNBl-`7nG9U`=^)jOGdf3imt%SbH0z9KO42K!pH8v6R0z>-f|~o8)}~fKQY(& zvKX~c(OS0~ER~lJJdYv(cQ6}~VZ4`GQ`t$E%fa{@jhxYg2TO(y zuqT*1%c}v;8+9S0q-N+khR9mM@&$3^qf5{d9IP=*CzS`b4i6mg--!w1TP zc%%TAnr{z)>y`?>j*k$=yoKGz>=Vz2DEG8V7h0mZ8o}*TED_9AKtXRhsPxyNxVQGY zQQ8s~3D-oVDotG-)h^Wlj=vUw{xpdWZ`b?XQuU_Y0ZVp=S!FKSCeBP!1ob;&=qSOcWq5++R!j0@}8gn3J5f zNL~#!$9xNa%;#=)cEjUa^k=$z|ND#oPgk4Ek~)>k`v3ihI%zQTf7uBA&p+Cl#H029 zNaf+-8E_c)R^vqg|7lNTJC70%ek#U_M<^ceR~RLhRyW_}FYuj5Nr@!$Ow?mLSPAN}GM;wAIh3!;SF zt*5#3)x=9Ru@`<63b5A6;cE<-)SKuRAiRb(y^XKcYVs(1X|M2>_8H&#+LWZ3B(ydo zLEntOTS-QXH>93Z#4-%!zcGlWhl+_*Bp4O(_m|50CasDWIgwy;kN;7rTwGElk7!4N znY6(3Qu(5!weh025-fZLMoXu(ByHfCFq|-}NMO8F;a<|__z9m9X5SO|U8?v!2|eUt zTVXS0t;8XT?qZw{Yb!?3T3IJKrAdVByZTvDBqkBu;andpZa?22RyWsT`C%!1hWP@(jdy9d(zg?eK0B zT4b#qk#ej}>U845??M5}I_W75R+FwKE;17iRo1OeIoUSpL85;gSZR7^N{iCu7wjda z!m(x3`}xj9P5x+uVcncCW1R28$H_dl0r1%`Zt{r#QI1^f#WjN>=h{tw^AC#2Ctn1AZNJoRCZF~!UOq1= z##gkb%-k&PCC`-dq}U?S+X)t40&kRam<8a>sgrspWGFC zE^f4%@Y>q4R^V%|!lR3rzcK6CJpwg zW0#;OCcw5)tTWtQ9IGTiJ3g65@45^Gtn?{Za*Uztr}&Q-8*q+;=FN2aZEMd zE=Nq1^rbFC9xC32(`r zF@EX%;N(B$%f9d2j{o(Y+9r(~y8+UfU{3MW4qs``p+K|DG?uIeViG zqvqb|m6@5ea-4H;$Z#8fFC%Mh8_E$W_b|%r>a&uaZyTl~QrTy;uj_WatcPv5v&c~o zB!;r|Z6zT$t_ZT9H$ICXc!v#>++9uIUju zZEgA{^>K?_oNDANk#h~E-{(I4D7T$Aiio!Bn=$i-^yJfcqt!$&HJC}ydln_Xi#Ntj z^oqW@*1VB=`8~XAgGH}5n48Ue`BDDhv$gTry~-AI1zu}S*AqMW2q%jGy;WOyOcuoaBUO(X(>qL2#}m zZPeTKVrbR@f$?;OTgx}O?|KtxStszdTj9a-%~8AF4_ThQ{EbI($aYKSgiju`doKU5 zQhaT@wRXbSzS)m1|B6=}v)$G+;it!((aV3@6o1%mAD!^GZ_e+_|9F&GcJUlBwx<=R zn`NyeXqTWP#(&F7KAm@?l9*beLyn+=785yPZRQ{_rld%clf@<)*f=nP zQlGhKe7TAp2gw|Jn6OXrSR6Sc(cE4n4iRJ=Pg7Ejd4vv2C2G^DbmVrN|L4EEfP zbZHsY;EK$mT!`+iblF9!5fxc2xu)!S!|C$Hs<9Q>_i|^(%=_q|(5JdNc-P0=*~9bx zGD?DKiNQHyc{V`~oGZ%KYCG(5b;Rf1a*)5GnxK|zm*<=}-)8=_`RWa7d&ur!ahDVG z&E{*4sU3K}J3h}{#nJAHmW+BuMSf8pWZ!egv_2R0Lu-&L7re5xl54AEa z!@gBZTK!OHepZKvMaH}zT1x5_>G{wxjv4dKr)sGmuE@_X_w*5R3Y$88t)ozlxR3`S205f-1Im zd4^Z@}1jZuX+h8N8i#$sI_(v(wpE6C4PIDEeD(U6n% zg#*KWONAqL=$;5U)m!)=$bXISnk%}eLz+JpLd|UxjuM>S5z->IXEh2sQg42PV}+4CuA=?&pc^JlyX zIafRR>#e1au3>c^vp?T6`KRZy(Q8}VXZ*0gFgp3~t!2NjK{c}GwsEA{3j#Pj3D$ao zbK7;K`ELivXC`gb6VvE$C=#>{oR*o=swb^+F|xjO|i>|>U`9sUsb9TMfpZ(%8PqC6}a4IKv zs=?eB-Acu3hk^?^yTc4zx~|NU(UJ(M=*cfL@UFb-C8NDIq~5u(&%l4hwW#9h&qB^~ zidcq$b9*vn3>K{D=_$5044ZqsR>tJ^ibp+r6AYs@Zgdx$+pc_*xxdvgPUGgQV#|vw ze`X&1VHmH`%alML3uTp=8>MUXX_eSt43*6~w9zO>vYp_-_Zxx-^EF zxm1lA9pvv%mh~_UcW^nPZ(PNHr$W}dG2GXsCfc~}#htbiU)5DBM2;OXKJ{W?uw?P> zRk7FWhK$d3-Tf^aAQ6##yGw56pobNOJmHKpCZu{p*9@n;sX1r;GlJZGwqr_Z0Jj{gChK%IXK1V6JcIKK% zqz!w??_3+L=i1e0CYSasR({v#Xe-z5iRP+l&-3MXuZ?kcz3O4Eoi@@WzvpvIfNM{# zxxva8z57a*uU#X0qutzm-pf(>1Gm?1xp9+aVLk5^XMdS(Y({pUxrKw~YpwlcYjy}XpW}uZ!lcn?RsB-=Ju5W?<%I8 zY+QfO^zhv!qyHE#IsiuGl?z(=CcZWzb74e8HH6Btud zxYQUY?S{o>m%t}Gg)4>|wcJJ;X6+I99ISA?aif{r%duGp1;&#V`V2QYxV_f5tP=QA zp>U^hlds#`Xv^aQUpp1<8*UDDd)HvuEb#5Q!sEuxac&>REH4OrCyGOcThiUe^k-id z_@Sma(zvC_?Q`_(n*u-W6ki)|t# zIWGnNR49IF+;-3HSM;0@0)IOde;96m<@Tpx&NqR7&lUePZvXD~Z*0y#0UkajmQg(8 z&N8qPU|8x(d`h?Kf|WZX zZmpV53X~*b6W!f~Jgv3U`A#ZL9#0H#=j2%%rt{xXl8a4>au>a2Ju_Y4i<07aQoOsE zxQ%T(Bc-ewo1Eh=?rGx~DyUzk@jUr}m}G{HXQ+@z*|gx4IJ=2u{5rChKS$9xBpMX7W7siI{wbZA_?WUztU4+IumDtG1g$CybS0z5FYt zBxIN9ASR+>t5sb4cg*V3 z{ApLJCAp*A!!E-ycdCVbwU*S8axANUx!RlT>$N6bDPLr_Gdx%Kw0(=#J_`GN7@&&4EW3zibEIjAEO`md7b;I-QCw;K5#~c){t8Vhz zHQHy{G4F?i;_E|OJ9queo%3@Z&r~JBiUhkHp*(8~2WG1B)QXhO9N9eEO%AXatSi#h zay7*5k2*|xtrA#~S&?fZKJSf#!fVxpiX6#2yIT$_^VP4Y?N`gs&-1$F7&Kq^iuyUV zBFWvsx17u88(-0QthO(IckC_K;rW(VH2WtTi&vowknYZkxQ~Xpt!OC{KBB9-Z z2F_`M`sx~&nu??sERS*CC1_x;amA=uYeC2f=RJakAsW}4ip>_R{N#L4&?rTt&< zL70I{m7wuqjXO;xz6(~xxEvQWxukL5Xm99()hFDY5RT>%eP^z^c)T{ih^0ym5WzJcL2mD^Sf11TE_v|z?JebQU#`r4tob|t;HTZ&9=QEJ zGUrFtKT^uhkC%1l&9IWv;$2rN=9TE_E}UU)tR?WJR6#O1!(BYw#-my&q-?rm>Q#5y zaNFo=(UG#*lIcPoD&cmy)#4%Lu96v!9@E0@8>%OblrNRc%pW9b08**rkWcF1L zi|~14)e0krwn^p)dD?_KOgy3zQjsm0=jiDeK41Tc#z@5h$@~mY&u~YNBhx|-*GU#$ z_4EsOias)ZbzNR~J*2oHB@IAStV`9yMG#)6n|*S;ebAyw}s z4_sZaDco)B$n24-zmlawUWwuE6OY=2REtWLJ9=e=d*~mXJ5sG6RgvM97w+kC)G_49 zbg9a#UM1lRqK~?c9GNXuE#zGu?v;Dg)BC9Nog+5hM;yJ6P4l^PbnzV+K@E;R*Qfb; z*F@Yo*5TdixbWSyrFUw!+<`^Z?&vF06KG$XQP5!Fb6IGSdJW8@LaD}ZpBrx$S!;(% z9jmz0RPJ-ACFq$WV7HYufEys*R1ms1mH zUYo|(pi}|NDop=a29M&u`VL=wo#wC(lp1C*&XG6p^KS!uwRu zzqJOmZnk%}k#@LiTKI*^z+M36@XA7Fg2VlBwy%zaLw@s>r z+#OU|oN_H{%Jj~H7RkHAL5s7lMO#kqmTFbI`@DQ{!L^tr(|Zb9jqbh}UcCR>+O5-j zrB2)3eWkMG@U__e)B6ifd)<8#w4}CXU2EN4zB564MoX47)vUi?_b}y5%%1lTmYl8G z@T2Z2-`T`HA7z(z*2Kvif0lAKZ_g*cr9Cwp&5yt2J6FDEyku#A&89`i-=>^9vFGc9 zrH^VhZ#@2i?|jFe@8Zjz?c7o@&Y44zMLOi_&aae=bhW`%=oqL z!pDVwZ!P<^bNiPW|Gr${@#T>S;LVJe(&Js<*B~O!U?h zUf-_g%YQpSE+Q$qUi3x#Y-vU~P$eQcw_bcjhpV)ZQ{c3Sl!p3AFFKY=b9M$AM5OlB z%dWV%Mq2b*phZO5SiQoFi`%5dgoA7%(kC{ktmw>^7IzAAjM$;ypz)&ffV5;%kcV@I zi~cm9OSMH(XM%inc7`?Rc3nDMH2GIhfKFzLzP?FUx37%h@|B#d!}`Xjy6*YPtz90; z$-bm-ZgTmR@07;n8#ud0^esB6dfgaD8!QRtMaOkjP|OKy3vC66TTYPR>l>wc^HTv2|mp#`Q`W8;?UgS^_mwrqR^AwpnYn7^`_cXFjWBs%$XM2_{5pDIsBuf>&95?Z-md&RdXUuw z`XY^q1v(PEm4{bHd znRLtUfrDz8`t^$aM)`%eydF3%57WJV_?%JEq}#y{oDYQ=U$1;@v@i8`l%MPEu$fs^ zUyKfZy&do8E)i~rB!##LYYOZ~idhcC!F>TO)J{!YE0&+Tyk zteSPk$G^f0;w!u=XmxF_@d=-So(FJ=gt;6$W_)U3;L!vBq*c)_b=Qqg``jIUu;k*Z zjV{OE8J{1x`}@H%-iQR3dJ&TjpL?8#@SN<>ZJ2rrR=4cl<(ngNbWhAbb!G1TX?sJ? zL=@>ZhM8PHb>GZ?<>!clB26hKeI^ea{KE`aSBacFY;xz+17H7DYgZo^Id#e8zRAN- z|J9AFn?;&OOdg+lxLz*u?dtRInpw?*8jn)tAoE@CT9ligS3W9~Tf1t_jd!iq&95{b zSIVucSu^nNbYSyn<>OOw8{V#Y^6pGR^GA&*U2+?>BVW8bTh=^Y`D8$D^Qy@A@6NS1 zf7f{WLT+nKALOQKIkL zty>t)Ar1MIny5+dI|5rcRYNB7X>X$x-d{{;5z`!=E5AcKTH}3ZS&L-Vu#f!CRngPm z!y76s@GNLw=H6(dx~}^z*aXDv%YGO=tM2lT7Da*QiTiToV(jX=Wm>T#$lI6aALCqi z#k^HhV5EFs{@xg`x~q#?u{}7kukc~a;<{@aTW1Ko=-5{zw>G%0r?}NH?ZvHqCH`w8 z>aMr8nhLxe-nVb>+Ss}q_giOfc=>Vhf!k|0XW#r{KIhv@o+YIcu}RszQWmx=Ux_}( zE@FpkpRUE+yRYOQA4-bNb-m?c;kfX%&i=z^VvDBV4zX})er>j->T~Qt(f$+*57Rde zOO6<>s}j9)*uuN{jqj40wd;ocL6?<%L9tXV(s{z2fGh^lv|rZm4@|M()-%|j4DUa?cf)wyllx~jr+xUe|NO%Z z+ke+R{c#4n750HPxj1rsQ08n>+DGvN9sY5`yM~NsrG5LTw6t?~oW!nS?^!!HjOi`y zx*aFC>)E}pX_I5*YyESTFUJok^e1idaC_ro zd352I+9!7|Zd&B_cC}^Q-7jaK+~wUI=r)>Vd1B$$o+tNhH;1{sJ7#(6?$<{L9&X+o zHRJu2bFK5fy*cps%;vZmA6}n3clq1T15bZ#j-T;S@O=Bc??MNMW^PHJG4}tlcAr5} zWnrAg+gDUTkf4~6ZUhNtOqc*OAm)fU3uaMA$1xC%C}tG1fFfoQ#f&;C26E0BnqWr7 zywB9ue%sxut=g&j&Hp{8&rRpvZn~SswtDdNd2P#+#s6&B)A{2NyBEE_SvTL-@1AG#E_D|<=tkA)J*z>DrwM=G=3rnePP4SSds*n$ zI~(03OeQHCF7fL0zTV^L=9YCADs?Y<^?YByD7tl6-L*=+YA?I@4djv1Qe&&qpo90& z_nM}U%)&GRm5rBpk9@B+@R6luy>m*Vi{2C8YtMPqA*|jlWwUDU8Sfi<4H?55RO)$X-#(2B+$k4I)` z^)Iy0@H;)a$?kz;Cu@%=v>NPpezei$ffF;drx&*O^}9B@Y23gmlN&BA?D((W?a|HN z4xEwEaARR-jV+HxH*Yj(&Safkr@Qvs64RxH^^^HAI!8};+pr~}OH1b`?vA<_PWQOC zC8JBLMNgK-=-!^)t7gmQeErs22CcMh^kR1J?pw?Cjn58R6V)hlcHh-o-|4q`Ht26# zy^7iWZ*2XlZ&Egh-?@C7ZC|tXkG``0U@u#JtvLg_Z);$nYB$&~O25UNL94gv8JG?o zyxrEoYR-@w+nO1)T`+i8ltJ$~!)msf7?^Dtyw}#yX^vC(?G^^+X9pjQGIX6YeD(HD zA9y`_xOwBbb6gH=@3yyH+Tf$>8m}}R`FeYwl@=fWJ)v&A*>tqdj)5yJ4WFKIYZ72O z)@6tDN-LYEXVpzknvVD1F?OZ(*r(xcMpsNHzTPoqWqXgOm(@lOO(*N@oU^jSzNgpS zn#P+>aoM?OWykAJZ>pQ-n@;oJ>9Mj?+>kqyo4qOG1?}HsI=>zAAfwr@qL~{0USqm6 z8v1y0bKTy|L@ohn_{xa>81z zZM(dB*Mkqe#c29cG&&wgRSZFlCai+ZC5SX9rvNP-7s}?sZLGoGh}zeC!nqp`1bo_S zmp1IjI+G7`H>CvpSZV+9`ERw!pSGSld&I-RFh<;!dS>1(d$c_U*u>PYP@0(e`tLFH z7&taY+g;V!Y|HCCEjx;R{oDvjDSwF8iE4oH{wR%QQP) zvfp>#n6b_yovdBWFO?kpcc1In;me#lWt!hF;Vt#qV_kMT*|@fQ6>_BCem9#D;V*jJ zZkIXl=<5B;4M#q6>fPI-V&3r^`_~$dDs$?8+v3~2lQsJ{8;-8;JfOFw*8J1m4{S9Y z)6RLwZOaz(gI6C2FdRG7*{Qdc)%?&K2LcVpEpQ%j+p72cb2SG}8jj!MJf^qxF!S>r z4xUXKf5LhE5$g%1;qwk&N}BM%d2);P^Gh#=9E?nwnCCq0Nc%OVmrD-bPnz`Gc~*-K zex+B<4sCv#G`Y?2c}F@NEaeq`Y?7D7AI&TmS2BjDTE znQw=0%Iy5B^r6P#I^$+Fa`ATUqFeT8@L{cSvpczL$?RfW#xD%?yUlTaz1^v6`?F8) z9d6!j?xNSbUUcny_W94l%5L*^yx!|%GyLp}en%|3%@2Ql@P&=*f>*1LbZ)%hnah#h z-4?csy?CVOrv-T~$F_G{bv7>LNS{>;KfXRu(QR|P1f8IPtK1A@&$xFFXqV&?4Z}Z|4Y`1NEu4vjZ|A>`+;+|{uZhNle*s%i<9#i68 zZSURfTv^VsV-c&?$Hi;)8FWtF`glmh>Y%vf?S00btDJWHV#Jy|ap_up=bo!RcKk-f zUs-Y4+xxCs_%`ME-TiC7#N}1=-E8q*=fvYx>kQ+I-1`Msd~`YSa@F59@nseLPFmFX zpGaD@er$ZDd;cpIUtXWcTJ?{|sJEH@@1FZwb0YuKKbuCq*Rp$ZuD12bl86mK@t?Na zC7=5~{$$nujW;=jAwzq9!L`sDXjn?J_?uCV`Op{RGN&Unv8 zqh*SNuDZ^UQ(EJ_I*qQMvzgvy!Xh_+EWH}RM&fYszrC7MG3ku2lNeV@avSa zyYG$!1LuLm!?gOJw(RZ~o?!HH;8eGUYfg7=vgO(67JUY}yXi)r?$u;#*=XZCgVtK= zy*X`HvsDt6)`PcN8gx7}v}T)WqFMOhK+DEU&y1|uJ}}YJddN9Tqf2Kd*6f&**dct# zEz4$a&djLU>7Cfcdgyb@79E2Z)cBuC>>fTe-Llov;AJ(t9w*vb4=cAcz7+gd&F-Sa ze&NHuTAI8G-c%DHNejM)a>_88g1=7%*v``=-HYBXOhN;J5RD|zclo6&B4b>ldXp@wCZ>% z^k&VWqNHiz!`E7MeiQnjCQy=RS-Wht>e}(_vzo)E$@9Wp0?b5dA1x~uhPLhtX}G=HsgJRMjKTQoFAr}dfIOM zww7beD+h;$8Ks_?G2TCDOpnT;rD4XY!JEeKX*qUCrK9P z&bGd#aQ?cvZnKvyoIi8*DkEun zz_sbK+x45)KGbOUI^uGjcDuJ+sMGrY;iF3}Iv8p2@h4I6KYc=>pq)@6MmZ(oVsyaO zNMWc@gk7&wW8r$IeXJbp5G&(4#mab0z@r|qGHYb4tVjH;Sebq#Ru-ViqgZL06f4c} zqBK@sLiK~xi<4I{Og2%hZxJW|K#dBxTg1sCcwoI%oP6vVC$Ia)Np-4;V(!p5nUBv- zaq$Hs|Ox`|@(%s5$s3v=S+BFt9Di8*Yx$4Ph0JQ63yStg2EqzTyuN~mB8Z~Fbv zmU-~)#3`J{ z83f}T&LR{Y8P^G6)aMb7i@1c#xPoiAitD(ETZqDK+`(Pk!+ku!Lqy{#p5ZxS@Dgv) zCWtwL5~k?N_S=ybu%Wgjt^z-(-U*4J>duC(Ku1z z{;@dOmcu!TW&TN1vMqK{o0IL)0iCduzRsjSIhxE$h?8vSrpL+fToc70(uJHxPRF9m zI9ZHM^cm#D$w0*8F702raq=6H1y4(|~}U)}ogq5&f` z#Z&sWX~oNS?80tb*N&G6co@aY@m8!G{}h@iHsBofCUPS-!wa5>rr(=RNIfIUW=6jrK(HJ2{D* zj33lfNLT!%o=W~Ar;)$O>Es`B200UoVji!^*_ea5n1?#_&nFi^s27s%ScJt`f~8o7 zf3N{l2mkkhta!;l0n|e{R@A^|DD#T$=z*T-g+8piFZ!WB6pm~g+GvC`PVo|qa|pvg z#+)Yyk%RG=+Q~UyS`Ckv)-XmJm_Uhfj2Vv!n1~+q^@KazFq6I+sO!Rg0rROBU?IXs z#mjlvmzXFVFaQHF2$AEtf4Ig=0N%{tTrc4ID>YFJCC$iTq$A9!+ra`(w4E^=E*OE4 z7=_UogRvNg2^f!wuw>aug2Pg-X)IpJcGs|7OuGiazLze(0IScJQ{0`ws1QO;y>cGgpivPoI&VxA znPe7psV`Q>%O$8AB*?#SO%&DS8~j9GqXfyvnzpe6l=WMede5t^bE{msbcXn~gh^`SM|!1%wKF(#xEDwv`z z%wPdaSiu_Y(E**&5uMNlU15W6=#CyRX1#6xTQ_>rR+4t)6imf*tb_+n;uKEf44j)L zNL;f7DQ=Mp~Jy%S^+-0`7rf_%g$oVQDm`t}L3X+VN(Mj&ns zVy+N{+qi>=h=%TPwu>f+aN$}Uksu>SC&&ocOyJtX$QfKm7>@~fwt)R#$GrZXATz%5 zoXwNR0o39*{y^!){R5*h2K{{!#13lA*~wgg<1ylBg8ag7{DI;a>xDoA^}x51NgJ48 z=;;I*21mH#JsO5`Ue0n}@DBJu+v|{my3jy9bfdpM*#OedudLnxHSWl2pMI zZDEFf^qZ6IU;#^5!5Zz+0Ugl^ozVsTS*9y#gKp@K9_Wc)=!>baW85@yI%Z%F=3*Y^ z!y7*Eg&(%SAG@#{0oa4R7{s~`kOy%HgJ}nnhj9c!IErI9juSYEQ#g$?2u29b;yl6- zjsg^71nVp!i!qXV6h@hBPL$PHgKLPu4V-6P z;fSQZiCc)mZ9GLKs__PI@eVq}IVYHi;Vy|XYeb^V#x4{fAB8Bws*#EE*Qi8Ui?^d$ z9`E5ZhOulnWNe~@;w;X=b3&qc!5hBtfgi5TNR;bnJTp=5&QBCIJ}%^3EaAA9CW<@m z;P49W2Y<6Khd5sdK`74RE`oz-qxLAvvzYzGJ6u&W7dlCz(>h7q^|`i5Q>f4u9ncAV(GUG$hv9I+2y8Y^ z5>I%+8=E`OhBtiRix7n3jG_ z^xq_z^MY}&nUBOIxs%Lx;8Tz!b<|1Ht(8)tsHTRXF8aLX9H1ZiqrvwiUJEBnZQo>R zWujEH#x=WSxr*zEKpvJ3OqL~BhUI8HGg&SyOqRr@$+FQSSy~>!DaM^k78^6AVj0O}&TQhd$K%$%E)i z9Y`L+5gbMkj^a2@qaS0=kr#0V{b}1FlKM96sqc~Z;XwV6jK(88#=m%pcqAeT$w)ye z(vXe}WFiasC_oX4QHnBD;{!fo0NeUP{zAR5WYIuP(gFYIg37VoAnxh3; zq7{Z<7{+2Q=3zb-K=(ZJjRE1wG6-i7jCvQ@E*d}|1~5cpBwyk<(ehfdjDp^cWEpsm z7S(;0J-yoAfRf~&ZO>xjS&MB*lH zAquxKgJtiKGs(N;J>16wJVZ1e;W3`zU(90cQ}P*}BL*+<60Z=8IK(3XiAch1mP;X1 zk%l?6v&jO?r8a4uB1&{XM|6TcjEqyHDVn1hT3{*TTavBN8pbez3Z`faGnm5y?O+Kj zSff2Upd&h=GrFKFY|ste(E~lv3$_@6kr;)9HXIAmk%3HPAsac!MIQ2@VUi*ZV2*aM zfF)*Q4x;f0*=VBVn9&r?&>TK^h1ZBh9O99Hx+>O>dZ>?9XpMXnpaPZnV45Ny@dH~b{qpT;nJCF7~b7jR}bbE?eMEliYWS~h(SM&(}8PZ5ZjoTB8BKTm+O5V z*Eh1~bNujkNI2^6IsYc4swx)@A!eA_=Vr-!m9Q za(wV^l`4I#Ql&5Sd-GhkZ>p4yO_dk(Q{^Spn7=1g7U1Q+RC$HS{z}D7+(HuVDDpOv zsZ)>&*`F%Q52VTpEW?kZ%=@iWS(1|~OR;KzQn4Cq@E4ZRx0YOo71YbI5}k5Wr8Bxf zvxIYv0Q~bImG^R$3e{kxqVW)ZR<=sxF)K~}Zk;A=Xq&*B+6TU9Z=A;cK8^c*8t=cP z@%~Gi_~UQJ^|eite&`Q7*kb^CE=iMKu*Ky-mOqgu6(`fA8gHO|Dow^=5`JLEnKbF^ zqEzf6cjF9#5rR;h#W{o_9OrQX7jX%faRpa#4c8HY8;HbB+(Hx*F_vu&4^0ynEDTE% zH;g);CZplTZ3{7mdMw6aJSJcwCSfvMF$M07U4*IB(=Z(~FcY&d2aCCVKDhvHScE0? zEg_d;IaXjLJg^F@u?Bx(E!JTv%lu6)BbVbJ>J8WkANax#Td)n=;g4Nd!T8X@I`f6)@W3t%h~%8!Op_-VbSq7UU>IKCPLo(P ziB1zEG{u_dX<`$bCf(2neUXeISzJ50?AJ)8VnaULg)g>XZUM(sm?j1=!8gWyN3-HI zX^s~7LHidhsIAcjUC|vq(F@0L8CCd*PpE;Sg!Q5gOfemw@EO0*s+9W=7GM_|ma#AD zG%-YDG=UNRg<(aSJi_S8G#P_&7>~)ghUl5aDQ65B;xt>beTLSU0g8*$!_VAv@%^XFlwSw(Ro$6 zbU|0xpd0*gaznb9dZo)-jGxS7-xk{Fj&HbnJY6Di2Y2Cnnr()pOa7&F@w%NZPhYZa z#NY*9;uR7R8=o$D$VUMRQG}qRbh$TOsrX%(E{dXbX@J|6^x-!YRp}xa`Hp$R6sX>( zOYSG;qK4zg}iSeEUIhq`U zv1m(s9BEFPVFLAdw4J++K{&BgT5GwahQO~n8BE7 z|0QNoFC*7sHv+H+2XGKa5QL*RjuQyMIfUaPZX+Ifc!LjcT#+Gjund0Kf~&amUwb9n zz+N1{VW{D_Dns1hjzw4k`@b^e0<0G*6y`u^Vbsq6&33 zvMrQ&(T@sT@MT*pf5tCEg0YA?gj`I9l4o%j|3Y&&=Q|)n2B6>o$8{t_mZRTM`Z4qv z*9M9);{@}3ird38#Oo$=d@Dm{W5?|b*@g2784`vI&|0ii1mrOv*o%G8$^n+PhH`vds#J_K&6M$&fQgud$#BIKOvNg*Oj(UJ_zP>X4u9hxY`{irf)9M* zhb=gWLkPuLoI^M+;yGe4#wJs);3_H+yG*I5BH!WzJ|d33czmW#B)^ii_>Lsnb-HCr zbF@TjB-3Y3wnGZF6={R6=!Tx?1zYri9UL$S!!ZJ*Fb1hCpN0w4>0|~nF_Cr_nT<)* zlQ9D`F$=RX7xS76P6 z;wg6Z%aq;NgT2^?{Wy$C{W-S?!+C78%arXnjVNd?S1KByKD3|%Q~OLYLpxZ&5>~Lr zKn%iQIAS*DU@lA?I3EKur3G5SZE&WzV-;58g;S>dan6*NGcx5hViAXURN_6h&ty*L zWXfiE;|cyntHqpW7~|&JOnI92Ka-MsKcDY=$QuZD$$( zObI%~ah>G2V2#1S>~{!r63Mwi`OQq}zgnqKla;8(8@$Ck*m2u?@&i8N6Ke1oUr>v$ z_=fMWXZ#P+f&57hAb*j+@dt`qnNkOWy3jy9)JFqoLJQhx2p#C65%i!B0}NulhGb(j zff1TwF#XNQ=4gSIXoc1=MjM!*JvyKZx}qDp!w#?T8YxJ{H<(3biaFZB8tpNeZMnkq zcBTxw!*v2DOu}Szy~}+QJgRvNg*_ea5n1{=_f+Cbd4VO5!k;FDI z93$X@ZYk_HlG1o=fMp5i8ix>%{S}#Vql$U^%6XD3`H5A!S+W{&`dN~Q6g)M`l5lgf zW0oW#zf+bJ!?{P6#LUf-q6M_kW>J=mUYjL74`oR&xjT@+=I`PY|(qhSZKv&i#8f!eoD5S%*mF@ z+-z~H%$Dk^Y#FzQ_puw~h&>!I2!k;c!{7)fIAadxVjeEz3O;D&$VW8S&5;&p1^Y&< z3&99MD8g_ac_>1^<~h;mZ)>hpDjneq80~CbL1e}w#|`t7>@}!gHV*941b_7%Mn*hMHee|Uq_2bGGqu^ef3oMXJi+(qR`;8CSw zUrLVb#{nFseULna!2jAff*>5lvH#kP3BqyelQ@A>I7;6!oTfg5V1ytPXK@Z;IFE2# zz(pKqnM>qlT)|ab!*xX91|o42w-ANfxP!a6k9&B4hd9CdPT~|E(SD34_!m#{49^jR z7l^}Y#>SHgXpx#DEzugrXbUr#V;nrN3Wsq7M{xot(Ik!Yjj`#Bg*SW)9R$wJGvsnh$a2@q?SQq-CKOAy% zqy}FxI*;oT75O}Fl<>HL*c#>q1OG73+PRX`FjtDY=1MV2P>KdNxw5ret~m6}mBNAC zj)If?PR1oy0&y5euyS0ktbhkDP0kgcg}JiVBUkp}`fB>t=SusHxiV!_uC(#Y6;mAb z%9Rb?xiZr)SNzVh9gKa-GNHVVBPU=YCgB4<;uCaU6JM#dNiF=M)*%~07j^S;MFaJqiAMBkkvh=*uMK?|zz{vr3wD@< z$#6vzmYYHvkyFX(n1Pv?h1r;cxtNFfSb&9agF6;sF*ac{JmCc&Y{6EXz)76KX4*zT$Ne>o4Ja;Xx_uDPw)?UjupBGkfxMA{gXrLZ0&d8HX`20_v=MBqz@!@bKIcjBmprNZktcQZ^F>e>8mNy3(1aGW(GWVAj-BwwF8CRc zhWS$4G+(~qJAU9NTD8cRbvR(0F9&f5hcVVPU&diP-ry}vZtd!+U%{4UE&-XB@`~ zoWz!leAx=O%zSajA}mG@aOJRoIi3X)fW6p<{WySwID|kP#u2RB&OTy2 zhU_koe!&GY0E3`+u|O*B7f2;y9u>$7G>I#aF_{H27J69)^70k$3*<2lm8e2B-oT=u zKrCSe{SO85`cHv$>0T%aqY5P`QK?8IkCRp8N%9olQJ*0{kildP8AARbWptrvK?iyW zrC*;kz**{Z2tyOv=gAhNF~X_akQc}fWJg@2?n>IACwjmR127OnFc_B^H-j0Bw_a4l$un8bSxHu^A7L0mq4j;)GEcgR%HAsZf4m z>BBgfPcM{qSO#zSz!!elfyVwT|#2^GOi+TIlm;LMu-s2-ap$7X86v_b{#32OYFpeMyM{x|tk$H$?gWeI28y8V8h;t1~ zd^yJXgw=881BdZfDzC4|6F7-eIE^z1MhHT24v9!YGE$I=5|rX|IP1q()Z!by;|G31 zae?~~>Ouqc&;S95zzsy=CT<}Lw{Zt|aS!+L0MpWVy+uC86T~13xtMUVP$pv%CSnSv zVj8An2I^m8+t7mn%&%}Q!vfZ5kB%6MVQ_>KTrdJ7F$!bw5YZ@r@l~!Jm_P|z^hO_S zy+_4CYu^l_$d6WBn6vu-A?7^oyToYJvm)qe5Z}{RNqS5qz zq3nIkxqZ$##drL~u@{_A=)dHcVTi`KjpnZkB`l6*;^|8%lqN93Cwxxh{+Y%76S+vr z;BzTzmPcb4p$VFz8H~{eCQ!l@ZD9s;Sill{p#O$3P$B&t=MdS*K`!!;j{+2;2*oHt zDaud|H9p`YKA{Gm@fEfB2F3eAnf;+q=3p-7VLldMA>81OMOci>xPq&=hG%$=7`(&_ zyuxe5A`bCLKq8Wm4Ee}4kEJ+*ARNUp9LEWq#3@|BMO?yVT)|abLj-Q%Gsb`7y1-Rz zs^Reg2XP32IE*6*!fBjAFhXzvS8xMA(cv@KDEgo;Tr-u5!Q>DO#V|O+3C)rhsxif4J-%2j&M%f+Xf7z0255>on2V>##^Gw-H(6dRTF^$~iegDZGAz~> zOK0@M>~-uX7Ng>v94XMM5sML+b19R}g67yI*u=Wo8navtHB>Q^jNFb&gDhQ&LI z`PpAAt_QgdhYl4>ATHt(((v@HQt=$m5Q7(ZiA-+GLN;=ci#G@kES3<2V!}y|857^} zobz>pCX=jTeej(E>xW{FH%ULw;_{)6Z3s6ZuV(l-mU zQAK+WITzK`^T-95kA-l9I~HLvmS8E~Fm@UFmRwG*z)E;v72eUmnp}hT)PIp5$hG80 zaviw|o8bvBc*6%Z-0n+$CjH1Q*n!E}kIZu}j^hM^R+h+79K#6&BLvS813QlrvB&dOwDDp!+eh-6 z5=lju^(E33!JaIKAHF5>bbpEbi_VcHQgN?D`n=$Lq96Lh4)$=s2#kbwY>Ak}m535U zQ5Da5MCSyyfgVY0w~k7&sF?GIC3r=9Ir*B5Lo8O%_CN}CGE%XF_HkUnYh>5Y^OQ$glW@TKvEp`hSss@RnLiOQas^qd%r&E>^+=>#+s9P=~8t zP!|tr8>&lqUR1(!BK`>(G{a%UAstzGgQgWFvJc_70M9pEb2x)wgy0G`vtCbJrS>Ad zagF*qB5(tdxQSbc!qT@TvJ5NXf%Vvcjo5_E@Ps$K;Dep;$1dzf0M@-Pk-u^I1N(xv zc#j%pM$bg?EV)lz`;5YN2D3ztyiu-r~t%jwd16@4OEtQ9e z#$CNqxd$;Q6-!v5JvyKZvXG6gjZ67k@>2eOyi`V6mdcI#D#f9Wr4oq4c-OU5-s1zh z^k6(vdY1Ax!liP_p;ST}s1(PCmC6a6#3`J{83ZE)p*V|k2*Y`V;{q_<3x0T*!zmvIGGaShk;6wmM+F?fkrD22s{QgOgg zxT4*tQYoHVDrE?qUMfcrj@>g@2c9DaJ7$*3>e*~>K`Ec(D&=z<%)7QqF>_<7EWuJN z!*Y0F71nMpm0JO&;=jLC1|BGt34x{Z*U?h>bg@)w@CB+yDuv05Qc*&MwlIS^+Q9-} z;!5Q!YSA{UR1)%7XDR#pne&Gu2*Obu!*QIzNu0uIgu?Gfscge;T)|ab!*#rnG8tB{ zOdR0^7d+D}laZ~;_&j(Sp9e1!2YA_+N#{Xj61Bfff}5)p5eLiU1|renP?>Z@C)fs+ zNpBP!FB8KHWnvLiCYG?m+OjhF`KC-pykq?sh0z#`1xTwclT3|rx!Isx)@hc@r8X+X z1Ko12`*N=Pa(RTu2-Pc>vp9$PCMt!hMY))wEduZk@9_Z-TC+T!pq**CSRli`Tzph2 zh2DsAu8DG4jWzfSYvBdUQRQLPK?yn1h|S%4HXdP>d3kq73EO6jd&92xcE9-7S~NaK#i%#ot(uf3N|{d*w3e0ps9` z^RRzdF78h_x0-6v(NgnwT$}?8rk+L4#!zZUjK<9W`Y{^|u>ybLIW{tO6Ep z4eAGEeexk0jYoKl2J~s-Idu$P;x%Fshj=6)87W9b8q$%0EMy}Wr6@x=)TqWAyu~|w zK%YRh^hG}mJFFHbIAbJ6!4pT1s^u7t;{;CP6iy=qVQ^rdFOyer71wYbHxPlFxCN7A zYT1nd?7?2_!+spZAp{}_$MGOoEuBMHKL%nD24e_@Vi+9Z1nW@h^J>Ywpq9p$SOx>I z8TT%$KM`P~W$w^J!DV0m7JtiVcmU=>zl4FYi(*~r3>k_s7$VOVdkQkcK3 z5LdY4<*-V5h1x+phbQ9@j|3zl3CT#oH^!urY4}c^PG%qrnaDvla*>Zb{9@cs{6GQi zLj0yKB8yRiQk0<_YE+;SRrtf$YVr-~xKalmwkK`vbpspdc zNKJgE)+WD@Ur~!6_>OP*iH6+%i~Nm0P&iidZ;7!@=y01Z>QUE+CQQ*5W-v!nZjUFE z5b0FOXLp&yv6ZrSLZy7h7kou6zTrE5;3t0JH~yf`#7dck5|pA0J0?}iPUK|Lp@3kcb@mbCHKal%gDJtewg+&a4!-MU~=?MOcg_Sc+v>jukkA z?MpbeWn2^Ju#){;!#Uf*d<0a=!(l4LBOIeXg)<1oGy0w*ggO*2sAKRNuMmrP#378^ z&*Lht;W}>MCZcc~cX0>z@d!`w9Ix;i@km4}(vXf^M;B9BNdc5{|)@;sj?5 z$HH@!{QCu!vhr%BcwiM);8d=+(bUd|@?OrRa$E*iHK+x>NVSY3gwF zqVA2p=!fU@WndunAPhzwyDF&%N9xXGF9hKX!f?vIN}4)UNqcn19=yV)0afA&FAR68 zk~x@*c~~=|O76N<$$dP)n?+T!Yh9IiY^{>k{_N|nD!F&6N(Mcw64RGeqMuqNKWeIE zWZh~Rrd2Il8&}IN>_!0gU@!JzKMvp^0&xn#O{%4B(`tE!=V;!dn!kgnmWJJ`MHhO| zhXD-H7(48$U(#vQz1%uBq&Ys4Xo z{&=#CEJp%$BI*pPmikDdPKGJvu^YiCu_z~p@dj`44)5^+ zAMpt__>3?3iduZb5Bz~Fs}^(kd%%lpz>jMZ4%@3`0}kN9ER|wPNVQDE49vtV%*R6f zgN@jP&G1AB(vXfk z`~UKd@!LEN9K_jE%eIcCvR;^{QM63)Y`CsY7iIkzPreC#k*97{9b07swI|<~z1~x= zwvK~RQ`1Z3pz%MyBwndy>17_R5#-r;os3p$J9=3;)C>1Cijv97hHhRRqwC%AY*s6? zlsenIx;oT<>Dj`zuA5T#|Mb1^NAbZ7-Fr#x|varT36% z&6Zwm*JP0D=slBI1H(c-4`G50O@ILdSb%MOQt*f7)Z0YFZ z?w}j))g!8YhO(8L&(dh!J6^qN>lY|nZ}VB{(CDRCZ`%fHrE!?gn&?JZUVYa!c&}^| zO_j>h*DqSXrT3t9n))hJ zN8jxZ2G-s~qBNVU+PeAfiZ`m8D;3bdw?mPWS@@>#X6QG%+E ztGDvC)8BnI*Xk)u9W=Mq8KBhm^|IC1G#y~Mt==PLOJARL`ue5=9k*!>P+9x>Md>#; z9ptu6_mQfP@77v zE8wx!)7mnK=i9OE_n-Is&-cFXzy9^F^?z%ftaUy6+0VZB-uJV|dtV9M1NK~DPO2Hz zK3pTR33lKM^E%D^>?3AGYX=^MNE5EiGJ$JoP6 zF)7$lEiCCYkF(#9D5eZJb_+}A&ExGOMPiR&C!Mg|Z*`)5lu``bG*1hGXM<$>XpNW_ z>^vu|?p&Q}zi~z!G~m1_teIaeu#Z7Wf`f^e>^eUSk$o&t5;j1z&2C7wklM4E5>_yY zoUQ1zD6`*`D2W;%`DZuITh!WfM3UHGGMugQvsBt|R!TMxkh$4Ssg|wwTmB1A?vm^_ zKP!#>He$~H0oRJ`%c)iq_FQHTFPNgv?&!37V81;v=g{sTN40v)+-$=EgIvmjC0yo0^(>FV9{2V@=$*msnN8VcJIESRfyp{ok3{!clvSXnpB6+XY%?6)-Rs^W`2#pAsLlF5bTFJ z^VH8y=jacw(Ah6=h&#CJs9MYKuv%x{?GjBTAJ#ffKEONNWpf`nN}VuYzhmR% zE5u%DP7{O*W-*Tur55{iIXxg$CKewWjOrGzUvPRts1g+)W<=}6G@A1ap;}pdY%qFS z9FXSxig51#L$B;gS;-)qE7_1nqB@?}$jTX;_>!nDlAmMajI4TaQ@Uj10*UUZM3vMr zI1))Ljm&gZ5la+zrg~mrJVS#hH@6!L5su-3u-P$5zzY0Y)4q zCxPZFa%>}>9T|+X&DopgDs{ZXJga4FBj@bzaxHVboOt%;;5Pr9PZwNk9ot1`Z!@@X z4v$7rI(8_}P7QLoIZ0`hR>v!vvkw{D`8kKWD4mX1XU;wz+@7BE`2wZiu@hDLl(9pS zbC~9)aqJ?NJ{#Ook&}|3u&HACoQquacEyfZgHAtjFXO8P72wt&MoQk zjB~n?SWX$*-oaUA2bW>UGGjw1&w=&Hu*=hX$o$vKBr#p!i zQA0fcyv7CZTBp0Bir5f7oTsAsD4nL16`P0n+`OhVpH`=Pnu={9N&LK)E}u@P`!f|g zhmz9sE-v`=J3T;ECWIW6)e-4yp4xx_lovJxZ)R zG<2vt@9Ki@6Q{q3Di4QzrpxQ1t)FpvtgJjX^x1S?PuludPJh)@riFYym)F;|e%|Su znaUGGpD*TJUsw-0KS5Pxg(PG02WWmM=cmM~?4e}a{Gl{Itn;_bDsjkRa{fq{pSAOM ziB)++hyC-%7W{C|-;1gWLyo}tS{jY${7hL@GIWHSej<9F^qXruj3SXNlE{p`#V~chdZ0oadO;s*q#q{HZSgIOi9M z)h$EEy7TWZ{NZjr34s(Bsqm9Aic_Q{O%va;sQ&{xw1OKG3Ha(<_&SqRCTD_HLO zWZwB7Gd0UYnTrMgSoj1YLg-pVr~o5H_|s8D1gX|&SYRtPOs8Xs2CQ0iXck#&)J+He z`gpab!&&}P<3&1-h!oeFg`R|^X#Zf~l}}Y`Ied~UHAxQ!j`*~-Hle5ZQq%5WKcew$ z?b_i}>Cz7tgXu&Rx(**Ilt|6|8B8LYROdJ>tdOowXT%UOtU6L?wpwc0&4?qK@aibT z+1=6)7a8%ymEt;&(9=4pjekfY(NtCEGkkhlx+Xm&nYc<@M+-eOCtceel1ltwwk~M+ z%%b$8#Sj4zi>?n26=4eT{-GkG8L2*OSY%u1kRB=}nzQOzp<;5OV|Qp7aW$_#YFO-F z=)4$OOSBNz$Ak_I&?`NBf@sTX;DzR? z3w^r79}w5@8V(KTbr-H*3R77IUF{7;;!V@dWbxj3|tTol&Lv?e+5#jv#2B|t-L(6m|qmt&GI8T&&sQZi_?oXF0$w(B3e-wDw7n& z`ol~TiKI{r%PNXCrNc2KGE1QfEm0S3?uO$?E<8oca7lO3)5)>BC+oa6w47YLzdN#wN&QA?jaEp`DqgqM6+Vc-XEBVETx}!Qt>u1kD9V0N=cbD@(l@>$eXMO<2(6hbKGD5#p7hCVyGs!N6aeMjx?mp3KnDOWF}gL z50gt|h5l^dxQnE69FbSZiqqLKWSFHQg(=jsl5RHe&c#zvMikw$(nWSWIZ~|h2s^Kn zmHTf3KDbmWpON#^vdZ*L$>eCQiWb&5C#&w>luF(>s|p%vT$I%;ZW54V(CXkYC8ng# zpCcm2lGI@%O52i#bdHqFW~o_WDsqXUn^Q*K#8XF&sQgPB7df?Lj#wQVriM#Y{+pHL z%_{Zg5jD4@DSdM*d5c!PEv$)O($c-Tle~3Sy>p}~z2xHJ=6-S}Z8|Y>ak1q3;#SBd9^IT3)`~eh;2(!_ zNgy?6kF?sJ9ZHYGy6j;!i^JN;XGgl@tX=l)BbT^mZ=`Rdx*X6pmxo>EpS{_=&Cli2+2-nz%jsutEpDT`B%)jD!rCQgZ~JqZ zE<93;Vx+y|?45LOj0>ODq6+I!pPlOF#DShg{ zL*#OpbYWzq)3)@x^c_-{BdiPBur6}xv+f;bE-AbVH%GesOP??9sC79izHmFN8!nyo z->Gytrn)dS(#k-oFlVH)D|5mnopo_GtWRC~rhC@|mki#;mm_`MrN1ridg5|ieDQVI zHC^d~|Lz%=6RL}EMy^elE~W2&w1#aiVhzjml^fMTf3g(wVI9&_?H`HYTEm#xXzo9vpeZKH*c&6Ta~*_-NGFxwV1I<{E$(b8Uls}y}HI9!7%#|P{a zxfYTxg^g-#%N;WINnMLrmssIia=ByAzB1Qh-leEft$(@m(m(q|57@7CEm2+CJgVcC zyJYNdbv>)Sv@QGwzns#uztgpJ_R`ML8|mflOZ)p>%g~n-!oQZ3djuTNxR#SH?;riT zqTDOvz=UfB>oPC=rn=mx=fDHkO5Wu|qc^+D*DoD-;#wuXd^mhuS56D~bjGz>b@|xn z_;h(d#;31b&uK5Gh2NSh59;}J-nC}-@`=$~i{+mzeF{-((d}8`6PSwNfJ79fj?|t# zI$>K8l97m|)U(>f;kU^ZVLge~lm=dV-so-riio8|97Qf}FASfAD_8+MB1NHUFBzTW zR&2=NQ7PxO?d9Qj_!Ut-JU>d~YZNu3}q2QX-{A)zL9}Z@OZ8 zMp826g0`b8{Qg|U&Yq-H%Ej4^zR~-O6}y*`1e8|vm4WaFn978JgCa^B>B`9H|HcdY zm7Ajv{VP9RI#^3-7hky@J`Gp$0uCuD9jYr+qto2Vq>MwYlq=dR55phvD-ZP?>ZDwq zz4CbUQF`U)ONaU?o#?Ai!~Y_wJRI;@riRi*y83MNFBO$38J|s1x>;9e!yl_FkM(@^ zfYQUe`f~JfcjcE$pFN@Uim$#7|EsPtE#UJRN}uZLo6)~cS7u~<{)%!S4s%9xg;5LEo4vv_?RM!O@6}jCeb%%}3*j6`W9F@9FvbtFjv*c<; z&(SisJG}0wv04A>#-*dRZg<7qu@Q4{wJPA4(rrrBy?JboTiuj#tkvzFwtHK|3x0J= z&#_Lo`?K9U$6lmYUtBuY@Ad%QlMwMYNp)Mm7aF&Rq@Mj_f2*jzobknk+cc|(7x7YE z-O=;K1Gh)Ko9PK^DuSbcrzOUV5Rx;HE073SPPKq|`pDXBMm?3L}gp^Q|l`?suK zal~u#xsje!YxnPXy?JA={m+dprQ+Pb7xxxM{2e~04M-!pKU4LVjQyQ^?nXu$)%^!; zZ+XPe{Bt*Z()`??&-PZ2{hWU8)>0bXeFoiE7x9ba-0gsLru!_ZPcimO#ko5f=`rqe ztUgu58}+%Vp7c2P7reffu{Yi4?k}asyZ=qx*B0@s?%czGj70aBs=kh~U#HJK%E(A| z|54l574h5LxyL;jsqR0`_VtbZws`KFr3``lEA+L2h=1+toOMkbu|TeQ)^ohf{b$~_ zn_~<9HP4rh*Sh~AzIHoe5w4jHIH7cZqq;UVw#cn{k#VBc{a5X^hY?HsnwLE%I^BPp zz4mx)DZS>Wr4yO`?(^vDPb1z+YF-C?rEy;%U4J(AwxZ_ejISo#7g^V5BbL=QZ+gCZ z;J(DW{&H-&yXLp0ub#NS6<>cH@lIE>5Rf_JzO1_bX6)T`%~D3@EBANW>kAS8n5$Xt z$((op$L#gxv41Sq{9`E-qC%K{1QRl;MFa{^R0O%-NCT~@H9RiBQVrmKG!x-cYt$>y z`#0}5)gS_Djo%7zRHUTejA^i;79E&Hq#CLFEj0$)YfX-4QK>6*{WeU)q*~M7EI+F8 zT>n~)VMgr-Z?ouB6lMU=MCR0*1)gM5(c}R~4YIO!_3@K2R17>oVj4BoTK1ldqnhvs zC>o=l+7I8JjHj-Y40teC+^Dq)Je5c_RS)=RRy?X*bNo~?b(L;_#x#CWySDdKD)oc8 zfgp|XQtd}?PYI}4%wRARWm1O^6pE;3f#tbDeSLW1t1fJGVZOB9asY~;oy7g~QKcU)5h7L1V-Ke7lo|&PpRSzB0 zta?-zaQw_G>N?#}8uNn}bwRyn=BXde4V}<@uvGWS+cS^{4l|s^#G2Fx2a3Rt^T@;5 z8tj_-kmDk(hdn$jW}3OwhxLlAJskMMc^b2T`iQq8oCiTNT*x%vP|pe!6FnT&!zCK? z?e!aui>V$?y5Vx>>ZJOpUa_Bt^W1Q?W_3pW#>NodF;yhgVBP|-sp8BnCCGj4vl94v1)s6aXfjNmD6!l1l#_Cc1 z_TxFp9&WmkF6M_X>UZ|$qw=#f%OxtxXyd0&_(k9^}#g#EH4a zCZOTdx4E?*-jdPVOxq0&yuduAhmU%6N@KgdA?bKttB0>{^dWOiQp2I%yv$CI^>d?- zHES{&K7X6n@8O3Td&;!SX*e91ukoOf$DV2IDjQOc=TCU}!(+3|wM`Ajdh;K61n|dR zYS#8NeEBy2iASJh>@{=UjfS+qf*Fq>_1GKDx8}G0}F|sVQP(~)_%J@=XfF2GhC;!VL2qp^Lh*YJR{~bYqbs; z@`AU8bWbKmi)RsX!>AE%8QQ|#dyN7mc(*wl9%)r#d&VvYbjdC9(n28 zqIl0piPnSVbVFVqSp2)=a;?)NdFAoqWY1`wmd0{^A+PQ&PW9Y4rw!6NFUf1(77ILM zFuGtC(L_-fC=+?cl67HP;u=N6ahcSU4eMAel8ZvoD=YKd#Meb>Ndbz+x3XGKjzkyB zB5zQr0!x&ho7K9_TJm;9)A5p4&n-ILHkM10qNTT_({t;bZl~5ILvitKNxx?t=0*a` zHAm4Fcvj=NjeKLj*0oY``S{rhPcD3e$D%YTI(pAO@Z8S7aY#$)QCxj{_KD{X$&JG- zw;PJCz|tAdo$4FMv~G_SJ;zI5dG6BPNMpIbQ1tbd&U@~jyKzG6zNEPRwiNP;$9$c| zqMDo^2rNT+C6K?))>7A;A39!!_1Xh}{oi+0zWw}-s@*N&hd&EFFt%z#qw!7Kh;|i=at02 z*`oF7Ie-6cMZDKR$;~#F?~U^h11l4~4ykW;Xnh}@e{{Su+3Pdi%`VpZ7v~@MR;GG= zK6kTEyMF2XH*YHiUdfp80hXUh;BhUB=F<4Ax2nu5 zg+G2%OABax{!@V>Hp_oQ<7{BH((9Odd`j!Tz468I>daQJFLdJ%Spi9nFMF#y zy}q0qf2)M2QHhMR{kDCyaFTHA=%1wOH?y@B|tTc2OGj z)mnR>;{O+Z$CV~0>ZsnQbrUvlNRraDug=f=%-qCUT}X!VgXKE9w+M3^4~OO`&4TKg z-eU4?M_p*8a`lP&7;g!Dn*@h7DJ}czX7s3$k; z;O#1x6N*;vv%1M`@P;H6rBBi6T{<_pQ@0^Q<-V-w_b$WSNq{4BR31U+HQwdqJNtE! zl`5|j=O?@?;5$4xs!8S3cm9EQCI8MLT~v>1{qp%I-c^!2hvDcODq2wEjCZyA&M{r| zBUQkO##i3wba&F=jW1L|eU0boU6_ICA# z6Dq3DdEMP|cvF%(s!!$T(>QmxTDK`fy>VGZ_fcY|>R?WeIyOkn^ih$g6gp0&deaGY zjE@?gQo);>)SLU%aXwA_sTSSl9`)8`b-YirWU39`azni>s43B>MLpG_+ww@g{X|o; z&jsC77rga_dS_o#s?Wu_sXpD-CH3y*CV@{Y=H37tXVR1q)GYF8Bi|d*#jR=Dd!kwD za|ynug}1ph?eA+Y^SR8wcT=}5py|`)=31Y2$-Ub!cS92|sKrI;)1kgMrQ>dIN;=Wf z>T^YR?;*TBsp(K(OQ+A(xqFXw+cTOzUvBC5>BQWB3h&5iIvjLCIq5nzQ>_t$m014@_?)1T>#nZpHZyOCFeQ*t4No9Mnei9Z^59ys>9{ zbIyr2s_&@ofz5`!NzHkEZGOIEa}U5CEt6DUy-u^@Mp&k{q8LO$6pPtgufX&%`{&^IQ=ko5n=bg{4(iZI)s;k ze|nCHw6uTk4PJzZUjZtL4&D?R9E${Z!R(M=LlAD{gdjdx85SB5hxh=+3IkVa^tMf5 zMj$Di70nJtn5_(s0*_IssIVv_B9g&1`hWn$gY~L2;t)Yd#x^5kWXL8XV<-k~Opo3a zhXE;3tl+raF=4^%Nbt+5{PmV)>s|GZ>9Lk8ELZ5YH;ma7XN}aC41~wl{4#2)S4OrC zVU&pjH_)Dy<_w}EGG&~WD6<#Pp4;sir#Tm5*<`%1n9U-|g_CqXp~RdCAt5@1me+-| z(RDHg3sWeMu}?C)H102?hBEg$BBemsQDAPsTgiY)q^u*+E{+QGewqfiX`ECmNE!EV zv;cMdP?a2wCt1qaEJw>D<1_{C{j-#>X@V*THp^Lvg_yHdveS$ZJOcwgv$McJM);I* zqJyK%oI!H*W(1?_PMX7ek?kmYJcSVslgttP>98|*kr)q zWg*8&`%RH{L>%wql)MUhV#h$?9fVRg+7aYMp-6HDLn)V= zGYnTAT?OG3KsCJVqyh9iR~SRYE7%jX<|+q_XXf*&5ELGgvBznRWk6=2YV|m}PG}Aj zQFU42ERlyL#~|MI3I=jOGBnovu%Ro^Bmo48}=%ZbaE}0aSy}n@Qx$h zl-+~*l=gemG=A4_+`Tx*2^il}a*w+=x8oi+DOa^Zz>Usk>zI%L3zNcRh<>KVRMF3h zP}Rf!5!o?x1SBtxCOwze%M8>qxr{bjbrj<_E8vWKIIm5aMB^Pj!cjv&7ouV%XOd(j zSPe082>B=%rwB!%h1Ppec!lg!9d~6}M;zlP4G$|oavnX3d_;7u1SRzO5u*Cd3RYY&0_jOA04iJ@)JxFo#k{%EFiUUD=h zm{cNTlWPs&Sh7MkXR?{s3G zJY|}kSax)rrnSCo0EPquM&~NNh{imZ%Xh@OAfN)t)p(79!18qRnZV4x4cQTlGA(o*Dz!xT?LWEfMJ^MNvECdGX-E!@qxm!rj224GVH2J_Ry`gSl-Z(61?w6al*k zuvn&vqtG{oLG*#{k%jV54Ecwu6i|Y!ki!y{u-R!i0tCS=0xMlye2J9FA`!5fy`Z7N zTy7i21P5m$;Usm(1ef0el-zU^ylP@?#|o~)g-#>8ST|L+t#w@%dyrOFXaSR+BK*uD z0piRwzX|`nEgoqBX=vz9;6V!VgXpn)v%MjdgOSqG0H^FS$3iVxY^?QOZv(mmPk3Lr=;BdVtoxM^B2+z1z-%|hf0`P|%BuA+^LX3s_xkAXCO?nQQLy4wlbT-ixj4NVG z))$kk{bg(>PawO8O01Pma+BnE4u-6(3uowSu;qaf7moiyUOdhsAq`*;XL&ISUn@(F zh8^3(Q3yx?+R+hohlv6LGrJaIpon{g!W9gj6fw?H2_Xplx3O0!yx52{nrNEG!Qirm zcc*DB3PwMwjuL}`6eWNE>-O_L$&>gNgp`}~oY)HU)D{M|I90X)10jVTU$dM|!Jsy< zfFPm(62{n?STpDdqu#9sM6j0j6I}@2=3p6R#Q71{GEk|xS%q}GA~%+cp0(qEhG^LW zyM%SIVcaS?8zL%j3UKkmS}>bxH>1Em9*LuyKu81~`iRt$J;*l|n2)CJC7d2-5%Yql zX(S7QfHo^X8ogSD7s5#<;I#}C@)$M-A+Vnmv{*tYJZLu16F#8XjNZ4Mei~FEOf$2f zlxSL`?2G7tM(}7gSh^>9cmXt?)(o}?7QQ4F3-gq}Ku~H+xetTuK=Gk-U4)>;d4Kf16*tLLU^a*X8w#A%QeGVG@2syoN;tta zL6qsE2*}z+40fiMmMhrzV7U*(G*no|aA8vLgGpMv(K89J zrCkbdpa?A(s6#E2VBf(u7TNA~t$$$sgYYOD^;~|pf{(UBFiAi^Faldpr;prp~nP+ZwCVE1hZaVa+kph7RJCXh5`!%qDYzolb*>r<`AxVii>zI1TEKM zDxlj!MnYyPLcoUNGqWwBaJXTm*Mv1ud2$8j6wZ5;WlW$>a)nYEOm|RuU||KQJqcfZ zhn*A!CSnqpjr63J>{hNd56p}WEh;SRqG-N1&DFMSrlU-+m~mYYkW>JXv$6O1Dg*>k zoe{|pQzc9>w^l$5Fer1`4``(7Vrw%;M_?!5tsy$17VN`dUpEgYs;UGNeh9qOzy^u` zmVv01TR~5}AjL+>&?2DmJRH z4iw$dW)+*~J$0I|ww&Vd_EPG(Jh^EtCkd$nI6KepaOtK6w9E)ZMgeSrN zrDU9Bhmsf^yjoc=u~#eE91rK}lp=PRs-A;@h?_Yy=jvh^TW@9RKfr7t9gAk`s-)3= z&TXKV2CI(B;OB(*{iOiwghHv|?a z6yt!Wa-fSATVxnm)u>}V5y)3;YB2-zQz5%|(w^RSbo_|9Q6@$SETWd`P4BIKIH_5` zZtL{(w55YS8FK^CDl=3eYUu_EW+{`jS)pgR$;Ip#14Ci^y-`+U$E!H9qAIz}Hrp5SoV~ z9A%+jvz}Mog+t_pQWTz2g0uCc2vY`WPE|vLw7qigle}kD7O) z<>A00l1g=nPpjk@oKTMCqN2PfD`-!pR^#aNSt;W-PI(5v7&5yP?S~Ym=%MOM(xvwlEyUG;);D@xt~MC}4eJ4NyrE1pk?ejhNg>=K@K_;BDM|{XfJ2{y5Gy0^r$GV-fpijB41OuApy!a3 z4*qlxYmS$9FwBXDP>c?>7e$8?92MRZEa&V=Hn{~Hco4N!;0))qxS&7BuKe>Hdnn;Z z(*C2rpJRpoR(}U0ZSXR}Nz@Od{^L3DUmjxpECVAgk?)VNb_exSe(&Lw-O0Rz2NRCG zj~)Fp29BBcFqTg+|M8fK+50~QtTF#rR!~lp(JyM?&GY@46X67zAW#M~#pV$DLz)0V zg$!9%kfouc1;)$(fxt4bkkOJk5i-hZ_k>JIntC}bb%l&Rn7~3NAEbOtG&YBfg9XTS z&{_m8&_LWDi$!W3HC~v*_S4${du1g)EIw2AzmkE1p=>2_V@CGRs0(V-4trwIY29 zqr>_u5~z%jARyzNS5!b9Gz=LA+d+mTDo7K+ka2>Uxiw_egV+KYe#8c%AT0}2W(i6R z_5{_W133iH9bo~Wf8Fna7Qs%Cv01z|Py+!n8{^J_XYYkTvkiTrSI;OEmZ|Gt5C@JLyZ$C^!=M4h>$6Q-g0sWtZyZ206u^~FAa1=T+rV<(O>k1=rz$dh_x+< zfHyMJYoZ6x>Jz;hf0h9n>@^c~x)C_yo(1oJn+CF>fhBHe3|?XlDBcKx4Fnnd0Q4E7 z0SKVu3o|Qs|MphJQ6gYUQZ?%TSz2MpdSI&R*(%CD9Hxc zKH!?%5O{nw{KyS7!rMRk+4$3QtB*j2HU!Ajm@aVldddL>gaWuhPU|6k5Bf+g|KS!NyzRsrbnvCoh0*WVmHlCTdP zAO6+cGJ2n8ZS-Cii2i#<{d-1%De-^Kj2f5y^Nd<~Xn#sd!l(OpA5BZ%|NDORt%=nu z6Q_D{JHqL&A|MN<)qfoH|8g=3FqQ&Tq-8DQ{jMc2qBq-d_XS+%&Eh6vKHj>$ z4WyIw89M){=c;Mk%Bvju0R#edg+oV_V%K$W=oSbSuAM{wl?Ws|Oj}(+iX7B^|1yW} zg+LHNzy?*lXagBGq9WQj=y<5rR0*}4&O@yf6m-#)30;4b);vUD*tj#Ct%^ zrW42}4tfHq{_{dtap?XIaKQ;eKKkf7F2pz-$XzoeUY4_ zgwlT~DSk*fhcpj0Qow=3v=(YGeHE_&8CqkKDnQ0NAYUBNsqN`($t5Mw?2fY3zx6c|7tsFYzMp**_I?;L+BJ2GFe*-UY1Ck7WkE4*_1_bqgt7c z?rBT(wgtiML|5BYV?&@Qob-2LbIm@6Du!r*$!4_05KC?tg=qJAmglax{B(!^x{c2b z4YEV)V)e|B8ozlNUtdJ|K*xG1lVGloiGVb`Zr!ar zuL%KmtHwgeg|H(?2X%lJ5`q^~-JR=#2jW$n$g{9^pOd6bzdK_7ILDd1H}-ZXAK~r& zV{;U(wJKWURB;Y$j`Jlz^#94K_OJKN6_%0}k(T{`?3;$aWFo!=+{!+%1_F)_as?MX zaY6)P1_*rsl;GYSgv9`QYxM&FV*nNql!7z@xR(QX0_cy6Tmbzl>kIG+z*>NH08{|| zS#U1^6Xg2^AP$tV3B)%5@Bz>OegG>0JOK1GKqP=YzD^5gU@uwJ9V0-BpR98Wcn!zy+Ro4jN>^`@?W3BqB@TV2wo@+}pQ_7R7ZA*6(`A#xStWfqiZ( z*z|f~gEAaeL^1IEu+AKZL>5mTWT3!?r2DqRI?7-*;*fdjA^w${6oJVz+paMB)+2`u zdLaCmNfmN63x~xb!%xO;#3fggT?PlQ+Iv{7Vc?P%DNg-@{;5|BC$9>UnZ1uuUFF7$n2bUhoXn25FSGlyN)S4s3H#h2s}}zliOTACq8_988GKl0hFWQKYZO%P z19MIaAdByhblnV(DM$M1_pK}=WU(OU>5(X_JbG=cr7LnxmH=mM=8HHPzeyEIwzZmN zfp`_Sr^#PW-jRud;iNq_b>M>l;z4uOYqNwz#@*+ zZw&es^Z!Oef`Z=f4!<)b|2F0S%{TrZz&9cmQIJ#7L;aNblgt0$_bIdY^UwD2j`BYL z?01r+-Nb6p#A*H}m^TyP@_ydvhx@;rFHg{x!@$O?`rXERg?#Xy;{d$FpWMZte8Qhx z#h)C;pPYu5{&RT~n&?{h2Rp!iPg)?aaL`1k$<%r!IGqE6ML^#~*V8I`7UMl%0_yma zy7-+hF+~x;G5rq?1)T^O0F)~@(f?0U2EhXV^qPnQ>;a-?WjbI%5D{2Ci$X$Zp-xjQ zq&DpbphFi-r7LSU(GMfiaDWKVQ#KK8rhb4_h(`g^0B=l)1>D6S3NNO4b)tIJ2rGR;<05}JDkrE0MS;axWg33UGFQ8^hK_~&* zuLnAu19caGl5>F`WgspCC6<^@=nc#Fyi3u0_Tt-(mVev$PX?eh>QBDv4~rjs8D_PJ zBL0iVprF^zzr0%ge#t|ay!&nW!-M7s^vr1HJ1WBPiWS20?evQ`KQ6y}_fsDhVT621 zL>PUw!4qM6Z~e~g9iO`*j9$585hky42ky_CBaE+QF0DhXefQ1K@)w_6`03Y(T}#(8 z$q2(40s{T+#&6%4BaGhuybfXb&J|z>!sy*~PlWNiA0JyGR{#CCoA0i_`E~g^79?3A z41cso7>?o)hHu?Kd>4SdUfd30{MFmvoWSiZJH!Wz_cuOzSN&2iAw+;@d@FES25K&T z14{S|;N3I=VfYpJ^~t;6%t2XLgyrII%MtHBSorAqbpl9sLYRH^?)gv4*Wd0I=}Vvu z>#s{d<#!UW!Pc|30Cj$a5r)_75Qg`tAT1L{7@zy`r8UAR^xYPO;S)9x1!-S_%H}|c z*E~Tr=|Jus(EXhSfc|yA2U=WnLKy#8^Bk!04rHGH<{WtTUI;W>pjfZoVkpSB>eJRCjIPawB8+F&KLmxof0K7agz3BM>=O+g`c|?A-~$Ns(m=;|7xXtz^cQ_0dQJ2V z^4u0gz#ILj*F+DXRli=1Kg$3Ow(2VA^u;Zi%V)v+yN!ZuzrF;?^WY`E2gMt``za7) z@B`3ij0PZpj$6*u7pIrA1#Pac5cHh!_nB|WAaQR{9EeAP=F{uR1Y>Jy>6=Xb2zYLV zumJ<5VEHEoWqJASGRXJD4K%`^{cK#8@V6rf!ygC;)A{R}6JM|6fC54RfRNreq*`M! zr-u+`diz1Rr!AgpdxqYRjdQhLIlN~e9>QXbao`APAz1NbA{BfQN2i;i=+K!l#4E_# zRti12O^4Q*LiXl`IQtxpK~}OiRwQTjc{ z)>-|aXrx8f$jN~?SA&(|bbGWnLC-G)#}HSZ4xOxxv0jPE9;l5mK&=?U6(R{1lJ8$7 z+O0E1GVBYP*14w{_Gl!*91{j&@AIiR?}I`RuoULOfVf~HK^hL5Gi)EsBv^_G_F^WX zND|5p=3s$Hy^={N6vzkTIXIM<05}{%QFw?!0+oP8g-%F8K4!x}ytnPj(WghcKujz@ zQpj9a6cJ{xq7yz4haU0c=nXOZ1S$xO3<^19#ES25JPuhZk+W6aw)vP-2?CC->534X zfJ3!jVR&+YYHhjl{9ybJTq&dOwx9Q^qSN0U)}oNcSs}@E9M%|Ff0(|*vy=hYqj)Mx z2szMESdmFRmSzVym#0#mw`X2tofI%XrI9Q8;}ch93QnHKCE6iB%%b(3v4a91~@ZLNahqyw{-z5Z@(-+7*gIu;PQDiG%|) zIPAHN1jcNre>EDSmP1Z}9k~<-4guPI1_s9%)%l`3W_f_UtpmJFxc=Zwg;rIM*WH=1 zg9Q0X2Xh1@Sak|-`~Gk|F)ISc>QlEt#9n}cW}wiuv1e{4K>qd7#{Ka}L)V6%@vuGS ze|jjv8^}4FaX6fK)L$Ih;5B#D-}H9Y5MX}RzNg5*34-;?EZMJi)tR4Xy#vi+8vw-< zns*9NJUApOTIqnzwa(3d@)~t)T~&l}mfz8Eks0uf6SztgfsUOEWE8~AyhfE{3(km! zo(4u%-gMx6dyr8%(l8vK>|Yi09Lr4(t%{m?O+2=$Ad1y)wiaF)-pKhj(68F~`2Y#O zu4?SNfvz1Ejdz~Lq}#!cA+xVjL78KQBdnlxc#G_zhqbYV3|x#V25+(F z*t9Sj6-;(~pp2)r>iN%HTy=*@Pq38kf_4pisnf%4v^0Z?iH%qS;rd{r+Sp&f>C-!Bj=Eg|eJv z5E2$?7k5KQ?Kogpmh0ekGtxC! zoJu_R%A=cCf^k@%%9brRo+bOOFa(A9HpYH-m(h+DSs5e%D$-{~gHOdnY>U~R(`A6B zv@zIv*~402d)d?hFS}LuD>_#CRvyx`j+ZK2IeV#C5q8k2Ny}e3deGNPn7q|@Thx}l z)ac97n+GmGJg6{ZY{fhc_eIuohNie3w$W!2@mR_Z)zO{~tZ0kLZASY(&lL?*>W-t7 z?M>N!?cA0fs(X9duoOF!tzNUAwO2OXW5}G^ZSB7V3jhka^7Q4XFPxAkKYbB(+{teI z^w8_%&NZtd8g3_gPZwO#2kUOWkJ%pjG~_SG>BFX}J&&>8HTe-gKRc=7NKd~qN*Sme zzyIRRXF<4=Ukp7?W{gtK&~fQqyEba#d(*G(yfyLSO?v;Xi&vLsg#)D*?wkCW-j_gq zuo{!n|9E3!!*XK#+NcL$l7E8F3txThX!|!C8<%eH>i%Zqe1lflUbB)&c=y#{O|Fw{ z`77_*SHG=U#T(pp(fNgz-%$}c8sGan)BA8=`!_#1zg_+c`-zWl<;`${`JtftmirkG z_l4Gf?F5<)JOr(`(sz0z6U466e64Rw-$yU@XFNRU4${`FI^?^i^+53y-+PTTeCf8( zx1#mekmW@0HM#>{bzga}{%$0-T;xKx&&sx$ZFCx|h1k{ock-ukz$;wilc%E=9WDD;P-agRk!vkC0zFt+r|!-qRN6 zXqEHEr47b9Zlivy++t^y)3Hmn9#-%>V$r_35}`xQA%Ve%7~^Z3H-NHXbby#T+qZ7o$pWl$brVh&m_xpmdHAEvQy_>T{spxj z>ti$sxco;4g_R&!eUE=0;gjk2)T#6l)w+`F2Kqox@^A5al3W=j+5#qe-HuiJrw=CM zZGW`JWGA+Nmvz?U;XzNEmrs6PIF7HmW$n2xpOMhwWO16|XO|BuOd71XxFuwYe~p_r zpq%3S*hR-Rp|d+GT7PI-5O&zY#O5iXK11{SZxg_m3cEmkKm5FQ_r>pAEdIW5ye@{q zd1Uit_==$q)!F-orTfkN-aeu%(rI;l%*Qp@XCZYlA8(2};Js+QvbbR26?(VVQ~8$4$Xy?Ax+x4@ zlm@Q5MQARM1d`e2e?9*3H?LEh(I>8a_d49`uesjqPCfk@q!s@U_TD=xs%7aL-g{;b z?12G>=!l2|4hV<}hzN=}1a&|}L_}0nl8TrR6XqeKA`Sx>5d}mzOr(we{lZ-#p>F^%S&7}k}E^{ZaUWy z>k_G~Ib*Tl*j?LA{U663o6>jFi;uC#Li|cfJF*)e*%Oqekp0DhC=1!Y@mxoA?Ai45`OA()2CT8^ z@Tm+Ky_xVC)^}5co$^@jyX$D-#H7BdXIqw?2G3DT`=poyhExlT1AIUA7acIlz%8-7WJUejZZuI|{y~k7|c`q!VvCr;EyD1)KK26g6j1k%XQG4^Jl;#D@#9OZs)1Zr6 zvMsZoO$+#xy?dfoOMm{TAz39s`Zmig$6k!w+v2-BuPO5R7T?;urfdDT_?8;)ouPLy zdWinM)2LpWS8^kwWtmkrpDeLMg5Ko#e)4VFbznDCYxnKTk$R@dMN03@zO_ClrjR#v z8I=>ggKhZBSDy$#6Q6u)BMt=aY?G3^kx&ZIs z7daV46A$3sn@`d0iXi=sc!O8I*zsd%k#Q$)Mb4DUh*ux?wk-3=FAucgcTJBBh_Y+; zDSD)}c^^G%ghhLU)fO_#EN%HY2ODUD+38Tp$fE35 zITIa3-cwOMM^G|;@51HwqNYjDg8s;vTQuo6XOB>3F{?I!ym0iX-DS7b_2k@BMgCE) zCb6l;?fwZh(W!MWsKoAgtJ!DC$Rh2db+2O1b=R)lCR9tT8D4HYo4>Cnsqpe1r$N^; z1gG~n?Oh*Tcze&05be~$wwOrgjN#=IM>!kcBP)cX@x72j|7PBweO5L!$F|=#@hHCg z7+h;_%e6mhe}Ch#JG(8_hyP;;!J@Rhz@U6SiFq%h8>_sLwc2~}- zoY+UZ*r_chI@h?^Nk5kSIbAY8r1sAIApIl3wTW|-i~#$z#?QPaws})X<+J{YcS%B?rkhW@i61gB?iTj ztufBl8R-${ZixgF*4A0a2IOX^*R}16G|7k`>pS1ssatAI>&@Bv+KKr|O_4r+YaiQMv(f{y?*>Iy5GnwD7fZBKXuF8|!T0=iRjCN4P*Ob?n(KSPT>k zgd3f^_`MGJBRk3>V;|FQC$i){i9Hr0a4}0AtG6+vXyohKv;2O!xoc|9bi^L>gc;zk z>ym#i=g*M7xlQ#AVRo^bG*Ka>Z&p)B?3&3FVZh7{rZ{hsy^dYu)i-mbq?XU`DLCC> zux6q8F$~DeMXGns#};8Dg_y?S^%k`1vHA+|=dawb~j z>}@$)GSOz7t<&1c;m2-TbXivtZj&FqaE$f1o@(t614@!S+NMRGZ;G_>^xJc`#=|b( z=XB_qSQ}A#jsI56WgpR+M@cThtYu)6NPs^Pc99MfShXyBky@DF#<3+%cAIS*L(fFk z=Gt`0tO?CbvnxJ7EOywmU7}3u2%E9CyKT>A4;z13lHM?U|4oa<^ZCi8=I4rRk4!w4 zd(Lo8Riq%LX^GWiZO5XfF7vU64ym8E)?`p^Vo9*$*A=>BCE*VIQHRr-4AyLpb;YOU^tu#MU7DP8a_%-V^nQ! z)1&wN-X%v>Qwt)^%M^1C0eP8x;+0Usl-mn5Eq28*G)|S+AGpXM4@t z%5{6}G5vKG29G_mxcg3rBWR#&R&un%n8Vz)H46TyJ&7^oaO3Qh12?)EkJ*z4XZx=t zX})kHV;pZN-FXo)6kP>kf3=rQ%Z7-{dtsy5B=NQ4)@H*mR~!2&+lTAV-pNsxWFL&< z-j{nD_sJ=^90m8xlEE!oHEf9St|jSU<-Y_E@WW?D{%h#K4d{e zhLnp)1@d2s{)avM{~TKW=g{(h!J#GB5YZz(R48tybWLWY90a;>DGo!)DR)(Dhmab8 z2OL@QIIS|dk(oR}CZ{7ff{}8jS{OcJm83d@k;xhALt7Ju)sA4 zS}3L!B!GpLi>0_!DWvc}B64s8z?&mRs0%Jw(3Go`BPM}0ataSX@RnmHGSUPcMh%nM znwC?I>H?~hM98?rgiclH++5OUwOpO$m~bf*&jj&cIZe5tr>?9T!coLNa*EZH&pN@)TtN`e0worIoj0#~XBY7D8XO@wAjX_6QwN`hJxBuR4k zuu;|Vuw<@MIfJVcx|CR|Czc8kNz_9n!=9{Aj4opo_=Fne&JnMXKl+uRq9d1Z^e{S) zfM7uBfLlBnkLqx^h4y303}uB&~k>(AYF#6yg9B2 zHWyHc9fSn(+d7^L&KeV3aZo!byMbxT`T6pP$6YTF(yKnY(DiX|bJUXdv`L8zBh_-M3Hg28UWB+u-Lii6z5BM^ax zISI=0HW~UW{N9u)s7|ryakA}ZBBg^N=REqN4_nidQgT6%l0yiFe4$thf?y^_f@wg- zh-Dm1LK9@xsR?j_t`2%^M{$pYaaVnZ=qXACGk%#;hIfRFyEbMNv=hquJK#@3!>mBp zh0ulpq~m|jUeXHY54amD6tg6D&V=gZQbJ<;pbanw7zJms82GSCGB+T!R?j!7e%tO& z)Ov+Zj6{zN2Z>{Yc)>FAjVL}Kn!YJz&~Z#NFEpUwVoqXYY7iJF<)9RH!8(A)lN$}2 z)6EWbYed78L_Op4!<*P9=-Q z(n&BQ6eHE6Fy(O9&{P8Z=uBzTon|+Us6;j#N`u$wu*)2UgjT||F~NEQX}KAOn@F1! z!S0i}DwaQTju8{9!2u66MBzXUqcJWNwqMyK}q$|ZSu%0iyjO139Rfj8S z%ln6b_usv}f9p>g7>fRJA4)?)S_vRP4Ck>_%kYTSe{&w`01+)l`E|7q*7$su>9 zmev0qvC}Z_cYx5{zXN+N|9#=fY#eST(Pklk#}+mI9b0sD-0wS9OM_tkqS>c+OD&z@xbT7uVLCcxFfyePIE_m^G zN^rZ2T(}x@CQz$NODt$s#Xbp6BYuhwqs}7;=(tuI^G6`RT!4FCxS~bX;qKJMKq;?P zeB~dpO^Eub(o#@=Xr)m{wQf>X2o*YtsH7uES87#Jm3@NAov+^kSBN*N>0?FJAX@1> zo)2lsS;SYFZDJ6#WJOgVXo%eRI-rI1m!d3i+e@n)zbFJ^1>=@iCjz9NqS6^^Zy)Tc z``9sMfz~qBeX9z@8kK52W%dN}#mwe$*40+Hb)^c>s#8TBz4kZ!;{Pvp_V1fgknn$t zI#Sj7zp}x}QL7J52LG|d|FOjX3oJ3Bj-avlFM-ZS^pUoDB8^BLgTw|8N-EJy=}*C|>B!Pyzl{Uy8!r^3(7ptqf9l@_YmsYe@4iTNxOq4KMJIvC>FRjR>=I z#3hiO`IMEhyk@6AE~l&?#(5h|8EJWWv$k-OKEzU$4l{*0$AE$D@xwLDYp{q zrPYV$qjgqQCP-#Hs9+J5#)MY;3q)%@!ouuq_1I+c$3Vj?yj+A07*A@C6qOu~_Nc^o z_sdoygOuTgb~uknt}Q?w`CQFn%NRz4#^+&}NdZ)iFC6IgAi>!ov__7E{;wOex_>16G zhf|1{LJRP*^|YlNGYTkP#Q!9J88dG_k2fFJ?}!gN!oXlX5^J9N5`#rwWk|mqa~dd zdn?ePOMT>mCIm}0sbyXcp*9U~zan*X+H_9sa#dvCO(P%2&WvX?!Y22qFxM)^<`6q1*P+go?PwnJ3(@D+VR&?BYIM;jv@% z)AwH^ZFg+Q$2)A}jkW4MXNui+)~Zt;D+cj=i{D#|1l@#>-*1VuQA?6UoC&mKJuNeF zhpsE`(8^5I+ zl%H84*sUMYQarw!6dZkQHFGXU$QfBV`EX#U}-=2%93mtO(iuHGU zndb-Wxb|ehpzVQ1Nso3ZMS+{HMpuRxId$78Ijk%V+IOV{v0QGwjeKJ5Dud0HFJc_T ze!Vh#XStLF?hUl*uqhATC~f6^2?;hA`3ghn&}5FLkrfqebUm$MZAGxXzim*OVR-50 zT0L$6|?)_dV#mMvrB`Ug2Hc2=QT7>|Fgm;K6qc`pZBbs z)~*jPG?;C$ZBAsjTf;XdoS)Y1*zBHn&Nv-hAAV$vySw?K@LopeS|-|t9%*wmAzuazlH0oip0j7E|S?K4V*RjJ-);Cl5tTPd(1S+VwHccloN9S-D!N zo||RJOigUla0;P^7n=D>jI{>VW(s@WWyj0chpy9|o0Qnb2%T4_8Sn>sC^UE>TB)+k z;gY1fHevd3XRo!6=4LV4su++M=a6Rd!mY|!JGEO|%(0NUc+nnvY)s!9&st-uLi(nj z>kwM%o8wj1bQfjcO+BB*{xP=iQJ@U&KCk}jvq8r*GLD{mwb(Upe=kxm*Kb5-N8}$N z86zHc#JQR_)E9QfWt(iME$p;%UAtyU0sfdKFL}50AF0={jlz8n)D1OjSYwt3FqxBJy)#U%UxFgZ1~ZHdiyzv9|X$M1%#$o${? zHhT5(+a-%_KB$b!pzRvPNsaBg@8Wn#dyhUh3$oczZ1?D`&~)6qaagxp^NLf0OXBjb z;y$kcA#WQR+uz2Snr>L~t~1Wmc#bm`7su2Q;}%?twi!`l?^Pezdvv_hOI+a1DIH(v zVp2S|x#IS#l;p`Zg@d+-r(N}~>@DbSdXYci?yHo#dozy?-ky?{vo~gtR_flX&yxH< zrtG=*`r4gW8g*C8cMe+WT6r%fShUzgJISXq_F7n>*i^;yN<>FEo6TI|)?`@`0DOw*i zxFM=C_+tIqhUo2_eVU6`TmL0=$^7UA3(Z? z^woQsLcm#T`yO|+kRL=Mfn-_%iPyxH#=F!=>-b}RP5gJ}|MX|||5W&2W&ZojTnFeB zsJX{Qn=)v^J(@%94C=i?j4!~XbnXhQ>>BflHlD80FnhXPbAVKQDSS)Gbg4K-jo}zd zv*=*1hkSroc!opGsx%g|G{;Pe2-c)HAtHRuGq1^op<2j{|NUo{lr2|dJfcy?Nutc8 zX$(F`CYPo*JS?XcSY}zPWASFi8fjA6jI7%6tnCn%xpHW;P--AdrndT8=Gds0cF3j4 z9+~@^BBf$J?M261W^sVKpbk|I2fNjoMLL@Vq~OKBv&kyA9XfNIzKi`m*EM(D+ zjd9xLpR{o_LU3|4l7EM0dMckqIEVzrZPC;0#1nh8l(Ynx`R%w86MWDtB_SVLo0C#> z1;jS`)CJ6p>UMkf5mmjBbyqO#Z=;I!72^%A#F1z`#OB!s#UH)%YFVGU3+>i#<1BPG zJkgDdcAc_j#?4o811|d6A2tg=*>&FZJFhTH){Ni!Hk{qCVf@3KI1BFRnnPXVG)%O| zSM%qub~Al7;@ew+Qvp4rnwRQz7goU#lUyfk9d9*k#KMP$tA>VS=4-9S%pYN^n^+WD zZt%T9STof+*jR03O{lfVmN&9yYPo?eFEKgvQe^+^grd+UBU`Qc67!1F-^zUIuS7R! z><@A`weIpgM{q3ET6X)@;+pNDz4|L<)h*GHzUYgoPFQ3`lVES|ffp&hQB>AD;Q7$zsgH+m>m4dt%?@U23|^KARvoAGuvTGzNX zGnf}{GcMFHttQYenBAS=$gDfl?GqII)eT}$1Is=UBR16ragD1^UF(+v`#7}gh8fcG zWwu5osb5Ih^Uotm8L^hDN*?t4`0Yo)si|vw!8$ukv1z~`Yts$AM!sF;dZ*vCkxi>k z^(e7CR20qX?Q7i2PaeX&cycgPh}g`aLx!`DY#n9Uvt#y=zzZ8Em1($4LI~KcSeGe% zD<^hD9t+9XG_ov+r?lPlz9V2?&c$9=Z^gRhmWdDDT5)9S#V7l|uHdP>lgK}|G{a?q z12gaO`<8|Mw;_P7B6zj!mJJOBUuD*0$!>`z!JzxFoNpQR_&tI_v*%PQ4Y9bNpFx_~ z(mefrQP94G(bKcYihq6v7XFf!z2ZYnsL=fQ(5-+YB!!^CZKDW^!?ueX52D_Lv#OBN zE$63u-VAr{)^{2f&3!qga{O@)5yaOSG6TA5r)HVSszeutuWb`n<3VIYowRZg6KB%;M$?!fFTIVtqAtzss7j+jXV(AfAd61R>?H; zEp9`M&ei`_zO0P%(9?XSa8q8*rS~8DZo1!4Z^JG(%BSDX{37V8gR%nTE`2m2=-Vvj1>lJ7|HpU zIfsRoUN5*&W_F@=Jg!hfw=Bn-jWV+}!lx#pwrKFAnqk)0f)RllBt1NlMeP#9Xrqh8 zfmQ~#n?2|;(P0`!>x%Gg>IrCv^R&vROnY6&!C+WK%`pcL`|LNWt<}>=w%t@65VJkl zBdG}A;v2d#;gZGS!$PC{VZjI4ksCc~0zxZnHzp(}MOB3KN$OY_JHfV8U-PmJzW#Bw zxOz~925(*JBGBxOf2MlS_7IGCa(TJNz~tJdaO+&ZqO8J8f1xD=7{@vrC3AK&2eYN( zAcPxdPl0vrK*|=YicN(^-=rCV9K#Vim5umxr)z1Kb9Kw&HIf5>88@Z3;LHGzs~ zFRuHBo$b03%1|npGCTqh3gHN zR#E%>@WEU^NmftQr$RRKD%MW$Qw*cWI$R5O1n1!u7_3<}gSHzTxPco?eS%9jbiB8& z2p%Ha(9-*Z-vN(j3-vyT%cg8-d6B(L;qh{u)o~34qMv(9g%%ezue|u6fe^||@5LG? zJPusyZMzITy%BaoR^cJgTW2OK(tMU+wLQFIeEhiZ3JsY3>(6t16jRd2d4w!03O;ay z1e$6VEPVcP*%0>wpPz>CMM|1j|oyE&27yA zn6A7@7o85>iafO0cd{Ufkh_;|vnFbiu;#rp3P;iK^aY z5uAUzS5De6ni5uMIn~m-Zg3~XW?Yu=C z1G?W$rDRlh3IFJ9?*NJSR!4U|iJwyJz8R2l7mE2nQzm(Dw2&r9h9yY6^_Wo!BAHO^ zkszk=zFR4lp=mt{ZX3&lb>2Wwh^1&wDMG%xLLyTz?yVFMWJ<=5R0o2@j#fu0A}hRe zC02CTIa!i3M`p?tBJYIA35v)GN{Kkr&fCM%Tb{)Cz8?i^}jCX?ATS4*D(TbQQtu*2#I6r|QV$ma<#K@(1=blDOqO!#9ifZbmWU*d- zTDvm~K82Dsa|UN@2XcKzmV{04@Qr>#K&=+c} zSDI3?CL5#XET;R!C?)gtdU+!bNl!dRDV~q81Vk_Bi6BgGWj&Qnx#QJ8M9m?6N4yc= z9AO6F1Tiqo5rh`#Nrbc+x(R%l0L|Ux>xmarz2sbUAG!eqXTF?)S6Z@4XPMH= zgw|pa<3W@VILsOqSreMCL53CTL4n8-n(o1ZB2NL+$=n&EOHHH?H*PU5-1 zG+F{ThzFvT5VR)b?p;7S07B@tknXNck%6&Siuopz`Gg_o_$ye^Tx zx6*1N`VaaEl(t^IOkoA3!L@C)E~L8vVyMs-SXYG*iS2U5?pYG=hn8x{*JE%&T?SDq z$#io2Ev9UV82k`Xl3rj&rTr~I2AQq|$C(PpY~g_!aElw>XvKu?=9deys^+fjVwDEo zN)bZX7DGjTcM8;i&ruiuZbgZ6dF!-9=16y_DPaIapa+m9G!lKmWJ4ehqK-%xC=w{= z&2jc_t(K-+*+cqdoXDW^$zXRwQVJ0jsH_7SOJ_8?CehhDydL6`<%+!@GLR}J2{6gT zR(fF|G14q*DeGA>1Bs$QP zHmRB!&e*p~+*=vG;--^cvG-jkZ-r>hdNFztgVjaV9ZVLfT&!Wz3yM9Qh*ShW(emso z;wJtx>WH|B{}pxg($eUIrRAQ+Bedo9)4==hzS_U_1pgCtL_SgRpD%3u|Mv?U|J$OD zlt0KE|7X+@ndAR9>L~9GCi?T&e^2!PA?nD)Q2dX;LTyK~uZvh1M*b{X7=s<2k)g4z zVSV7g{404V7I6W9qG?4C>jP=JK9I6J{p(*j<3;}c(dzm@qYT<;BW?NeCpanxFu_Nw z35DNe{;l0{HI)8?Dbt1UWGDv*$G&!bon89c+x=J4#STd@xmV3FUG4v7Q}cHP6^7{V zgsL`Z<#0RW|G_x-uU+`u%gdwqcFRd8nwuAc&#t7OM|Imj(*zs|+?XA}UPVbo8WC&FicPA;RlG#mPpd~$|-5@&TTP``WNcM}hPh4LqaaDNU* zp)-DdG;am@63)wsNPMS=`{8~+Pn%PFX`TX&;ZrvJ3#%^vbfdYeK5&rU$-!4`K3{~N z`8}NG{$W5PgewFIdV37I@$&(=@+U(8ICZop$tJG+AS2%+#>R$XI;w93Sz5(mYl-IpIEd0P%lD9!)(p#EDWQl#??Qwq7hhP7i}mM;Kb{{m3|#b;Sts+AYY=rc&S^8sY$0B`&Ojz$60 z0*890#j^HH8&AqiXOZV3Kwd2x8mG?}vOx_G@efuOoJPU=Mi2}+)6Nqy{hN@^K8tj& z>=93d;5C-2$L~6-xySLvZ1dm_o)v5VtCPpF&i&p3_89aI@H77}9?RU;;j@Au4(Fn?(>oHP9HRI)H9xP{GvX zD$ifAt8wymI*u865jiaj!_WcFz({c7pErxg3Knv6Z_rkP4pXa}`1e4M?fHSc(m(jO zaGPZEpCA-;9AMlDfPxwTeY8OJf@lZf=Z9Y!es%c01|zz4d}nF_5=1vr@!O9~Nj*N| zHWGwhWF8q$-cORKe=lE7kzx6Y*$#vk2xlQ=K3!HFd>D@z{L_Z*Y zW>!0htwQ1$#v*2=hD8xM6!uUIhsg$8T62yDc0l# z<4bKUv)l|CYU!>84IBlcx%d$ctnmOl-|;YVGT2F)f;xA)YeT=w%<=eMDa4R0Z52b0 zEFOVN2#~qf0ZKJ$<~%X8H5^EbzFGjO^12nq&CSmmaO)@+zynu@0!F1uQP~arQq$1p zQv43#cNc_CuY~}=W`Nhvq$loKz#*(^Zig5q0O=xrpOCRS4@_cE*dt9Iyfo(ZO<=p; z0=7T$PK6HxC+B7$uWBLk&Y{AMe6VP!7mVq+Mjr*HT>(~@pp*>WSzv^kVXz%7z<%G8 z;ppxmC=ly9buRSTFBHh(VMw>f$6y9rNWf$2?pSn^nz;rO8W7+8W`6Z&lj z4k0j>>!Fo!h^UQ2+c)q0)C&7@uvv_mF>EN*es;pn__kz>4Bz2Vq_7^+65Du$(A7N zBGEFw&EKf~dLzH753+{aBMTFq(RBcEV)F<;uzS=)WR~_v(^rvrf`qmsjEy-GeeoNB zjD!Jbeu*;(Aj+FrIv6z94gt-9Qh@QpkWWn0dz8%>fnJ*s3C<`7n2NQCSqiJj_+#P0 zHy<&V?x42CJ!A#9BWnq=Ry+bnnCs*#n2c=)8kZu`iG<%Xkn?>GplQy?MDoimxglKC1xublV+4UkujN%Ixy2W8fN2&n0tpw0VM^rM~yBTdfD?nCbDL`Hs8dBqI=9QCiXGeeEj_Duv1DT7b z@5IWt7VX?ZJNh%wm!H6J`C0($Bm4^(x0pg8_u_#vN*J*Q_#;EyDnJuaM0MBN0*u%L z@CNNL)&~H#9z@mWRlDGmv>e0ZilxwcA}^JN<1PDJW`W3v!f!c#mG}{15#u_2IbTY> zL4x#NYXz{akr;wsB!2VoJBi;b6dJ6=%D5Pb3M8miz|=>AaJqjr#^O^Vm??&#qdp>Q zRWg`Y;=v;Y%m)y&jz*Zebx4g|k5nw`Ld}?KTT$0;2kIWl2goOqTv`Z(bQGT211iux zrm76}s{TM(>otJ4&=1zU#~zEFR0yF#R5^7v3QUVo(*ao-SXtRy@&i0))-~!7j4_q~ zrq@i6yMc$U3s7O&LR46?2tW)&!qR;Y^Q4pkO0}dw`+^614-iqtAHx96B*uMC!YK?; zFgnWC^{c^SjU66!N}KTbi|Dg0%pJqU62Z8<1S9Y%Sckf^JUgpe~76BbtawAz4-ka3|}!(m?pIr6kSzEQ3ngqpE#R?bwPIxsBMMzGE)qY zBJ#Opgj5z;RrtiNz_1bFQ*9_x?~2**4|_r!uRvpDY`{Z4YzZ?Bs~wZq9hHco9S!?I zMWALm5>908yV)D%)qMe0I{l&;jEA)E}ZO1HTb_bxKxc4uJ&jEIaQOHn(EaAO<&?lmN+|_t! z#CrK_htYlQGknIW5O^cuce9xf#aN=m^Fv_2FYjCm`&|NUWM_I{Xa(0NYCoW*?1wh| zQ*{jgHR{;D1*n5fVa}@-^S^)}j$y-<4*2s19BttD5$#5$B4Tt$yza+t2q!OeC4C7!d&XG|fc zr3{jP3K7O6UICMrGf~k&iHby5^LGJ%2z3`r+srACK==MEG(63>6CB2o@nGOL2-?Do zAJWzA^!zYj&=97|G+3f8Q$`kkhYO1^Oxck5fYX`nw3Io~h+21KDICtle3J<9 zR9l1bVV>5a?Ak?ut3;+DmjN6hST>$s#Is|U(kBV~(rZB79)NB&|2gHcnTToyKWZl+ zUZQ&HSiYFzH>fFkkD5n2Kf)e0Ecrs}#6eux-!bDdFou6f`aYpUnaP6*e+G^i{CMFY zU@gYF%^p{iLhRWv1k?po7-WL68Du#E{N@W~KpLY9r0p}kQS0!TnaJpbN;BHe07|ex z;{F!aKA*W5%gYV(z;x0$viG~0u^y&t z>k30W6I}=lhg|+jM6C`i7z_{XQ6i{gACSyAXU~(eo(mnwNPL(KSmZMWoc0+B%-|S+ zWAg!wV*&as0GPfIAd=W|Cs-cl@(mD!^=7xFhGS1ti{?_(FdoFWdb(I5|8ZjR69tG4&Ev_0@nBh6Gjtl?O7FMFy60#tgHXu@|7y za$?kH^&h6Q6&W} zY{9P@ES>iiK!_<+u1 zG(B%5MB;KT2I}Ppby2Nhq3^+tkW5S+VoTrsg~}u8`Iw#axJZxJ2UtnkiZTQU?}}#)=DXn1ua=A=`Xs^{)E8Tuw_8lodmSw{aB zL~niU*{PZDu%!dMy{RjIljGR8ri$)Tp*ySHO zAj5a#pP@q+SAYV$3MCIvvOecIOwNI}7bxlf9)#2uzrm0PECJFGSTn|8DL~)*C>g5W z32SqbgX1YC2Sdr$(_etPje|#P?oT`#Lbs{Ge|P_w07K%Th1!uqfMhbDlQEzwCw?QD zOA~B=@qz`%ILu3T85{S7*b%s-Y~mtQ2zHH5542NDs*36XN?`$*O&D_~;TAv=^qrk& zX93ge*d51Ul#ED>I64>yjHV5SDTUbR`JF%rZq``hYCL$v+WEFJfqswZ*yvG{=CtNQ8 z4C_O7sm^`T$Ejcq8E*QVq;u>f6Ihvz^Mm&f5uYKRpVkQ=5lugj=2l;Cd3uml@a`=zz)pw52N`D(Cv{!hRh`$SQ zCf?om1Zn`$_ux=3xJ%8zees^Gkyb`ifSWXg9C}Xm1{s()W}+|j;UaK82CjqZ#tAT1 z%vz`<8y`g0i0b}WBLt84OFHRGs;!cs9Ma(R^M?X(VDwb(meoQKfmvq9)8jzIK^=Y( zJMz!jfa7}40l_m^J(-<-0D=Yr48d|wDyn=rOIOTxV$-o=sT>2#A=p~?#Q3TXZ}x%7 z5$-^lgJL-nI9U}Dj+ySB5Njm7U1l_T!?Xkg$S0%PA1Jg5$3Lo|M2_Q2fF4+~h*X*H zm{d$G>vjz8yEcpr%s!&CoJl}hyeA*IgP-hy zjJN2P0J=+$0Am@M5*J95!RY|Aih;Z4?oqg-!aaQv~dvmk{1}j zGCW)*7B_}S3+D*bSLC$6#^mD@3-M*(pS(Cm+Y1#R5bVwVupqELZV-X$h@*80_zQ|d{yF5ov{;LFZ&;$qGj;%h)gU!_U^>LNZXq5| z3|3+Qdd|h;ceQdn`ik*5_H!k8$Y3BYw(#YjV66(eAkYcw_70y=_#O_FK99nIy^{1r z4F->#DzF_7^FjAgJBp`1e1YPt=(KdeG7*Auw7fmhFchYS@mD{OoBcRZP( z<#f0AW)u_EjBM`~03V_;cdE?5+PqUptC5A^;Av#`F6)KN0-}iB_V+{C*P|cl8Nl-;{jxIK4tlry z;L*X797ar>h{yZR*cuBa7%cvXoy87E z%GKcM7 z5PxQ5DvHU3irxwd>4x9rRk05}i2tNL9)1M7%A1&f9niic)(h`%O-V8|&ZPE;^4RmWn)AO+c~V+Ay~(FBu!Jaz%=3Kl>QlT%UmtP3&n`$nPo1ST@$F%{r;1i_9(vC)AvkbdApycv&a zs19?)SYkV_dikLclz>iw77)i29)PCZPsCUWu4m|eb`NH%H-HNFAS&%dCC(vKf}Lk( z!`)^VCUt?W&?EDA$lRXbzzoXHYR$p=&)(wMp*30!wa4h8HWnXNoOl<*QG%zgmgMN< z^1P?Ou|d}a7hm-ILhoTZVIY{^KY&lnOP8N$4T5HIZyswKImoO-_G%gQ>?0n2O@i=x zg`mooSpcO=(R!SoJ$4&w2*VJP3P1c5@Ue+!H!%G{24`99sX(%OPad`a?VlWux^KwX zeI1jEWh!9~UJDU72ySvSa*K`EV%SERgJLKe^&f)&wF+Wm1YuW-#Lc#bziq15b7D3(^s=7CLb<<>c!b@5RC8vdt}ekF&f>uI}Rkc zK>*t(00<_7LfgI3&_WhEq&lN_Cp)X7s32`cg`SU*?S!swLC#8SM%al$f1W>karrSY zCnB?SG-fFiyass0rTaDf9>)6>RD`W|d#_PM21~Mb3?C#*8PV7Gq9pC=i%`&>SWZUWYhw|2boJ4IT8NFkRjMwkAVZgSWH17 z*`|%03K}}0NF^aPDGaHPL{DpG1LzXtrbPt*1B>SG*IXY~;VEb}z*e-U@&Xgd{`meg z@c%{N#@orrwNV=C2H;7S`+HZQ8V#FT; zlaC!Bc^kv@B?eppqPy^Whm7kR;b~snhKgRfKn@0bPIL5mQL0_2H@tS$F0^DGTG|uy zH?@O{ky^6^b>u4nzMcROoCF)z>IS?};!Tk~3!5f1gv+uGPmb%lKv{d!_ULL057+}S zgV|h72goaI;eK@IIj~-FVJDb+2IZrG#2QU6M|oSwbBM|C=SM(o;U`d-tBW4qm^u18 zbe^2yvk7ksK>jcXmWVC8co-=9A3?phzvvFuqfc% zFaxy7fbRq5L3;PcqMFCtGjLD7>T1#m6S)LK&V15Gc5br(174L90ldCs1u}b6 zi?%m{i_E%K)EKr<$f0HI9g|>SjOh-H5$`4w&C(;=x-pJuj2S-!*<)TuAbWLVDzcwt zX90dQDFRFgszbLvLz0@2utMkMkD(k+8^J*LEda``0EK%%U~|_e@Y-t}iKF83*DKH+ z37e$auxn5Fib}-I5{&cE#Yf3h`zECqM&>m!5{n@iCR7G&yyH^s=-^Z^J-so{bjSov z2?7uhGaN}4)NS(szLBZ*186$)p*Lhn!gh<%j)kz#l6lsZSgv&|0LsXW+k+L6S&cQ6 z>;|bvKwYJPxj&DGewiSUiQM~0j8!5RP8{P17mS>Js9x&9AgNw)4a>#*&aJyZs(T4Q zk4gZ;;{e;qgx-b;&6r&TXuSfEd>f$R5dhiZF}FJbe*S`_2GV*6JDEq)bFfcf)IR`> z`3x}kCqOH3NKgy=P*Weq@*!Ev)O3)Cr9d@PIzGX&tqwQ98VV!IWWt>$t`<93%@8JS zp*rw4eF6CN6@Vp!@E)a&FGZNSoL9&*cnz=s?X!M%UEv9k$&@*$J4{Ii;rY-VfQGv$ z9`eH!6bFMV_OVG1lz;O_5xz{uvIDl6V@xOIB0YRN00X1S3@HS#BJ$~344fAn93wH$ zxOiO8>5b|;$m4mAJs!{Dkw~09!5Iib7fH!adI(B>aPYX3Lrx?Pl3WiC&LJ3E@;HYc z1??uv_>lwU8H(vn+z95mJyH?}v|H>1z(AGQz4T}}Ajb)@xe*hf+Z1;alu{4IlKrD8 z1SD4xvo-M{Xuuz4irYcs4>DYRFQmrM@>moG5Ys0lE8TqqfC591 z%ibOV(TZUTRoe0ey)xf59fBPIBCdEDON~aB_|Y5)^E5_=&37gtF=d!{OfoS@A)}Dl zw-`Gjwk>BdxD410S~!Dd`=zgdbb3R~?2;8}sHB^Yk_Di9bv^h!nuUV&CtEP{N~0QK zm@2C@nEWV}8G6-5@FPh7h{;_<`uF%fTAKuYW4GPT0MEOLI`o4&ut&~iqoJ|qk+yn< zbmG2Jqzi~j_dbR6=+*UDnusg+k7L+Ha?H;|>qrIg#Ktc_i@~0aRe+d1cKPV1sNaR? zOEu;z`*zrCLZ3|c47eRkmoOmq2Dzm;wmN%q1kX&&NYRkWZ3(CbueZPkf*(opCT zH5Rh_=h}o_O+L<*Lc{tx70YGdIeLpyVGUT&9 zuN2VEjQkwfS>m&C3!pH>$u}$nrSqkz>RyJb#P<;GM^?ZAWJQ((<1B9AS4@F zRgBW@LI@!gipWSXnN^FaRtd1%q3294F%-_kQ1p0!!inh8-V^(I)l5Mh@)DYmmkc_@ z7uF_WelbXVMPk-DBLHMpB2!Hg`V|Q;(zerzu4UT>7xyfyKMwAL(%P9+|mAg z4}ch=Z}(*Y`(e|VE5sZ`#j~VPMVHw~kBdjT zDbO95yU04SBLcuW5}7B8JYXYYnlZ!JIU@3AzeIvEV=I&e=;B=`(+`7zf4dn2c@tb0 ze@n%By!tze?`V;w(svmie;+> zV*JaFAZyc6WD)IL^(V4UpF*wRo+oi$zy}v>)tUPku$`F?z}8dJW0P$__XN6oE~HzC zhZV#W|JsQ%88Nzkf1+A0UQqfo8tr&&nfwTvLH6#YQ&8g+9=>2|F>ld#X4-U6Bd3hy zAYbP7<(9F-!%Xts+zIL3 zF~H`B?J@l;lyW%A;U1~pooLL!6vyJutcn8Rl`}!Z0&6?vi*ih8vT%GDgHpEI|RYd=J2a7F0|@h0^PY=&0Jx=!QLI7^s2D_SEL7W4sE9dYp>!iIv=RI(W)V!KYI8907es}Xjr~Xs z!g{}o29=)K(GcV{G@X=#QZi(L+rZ#F@TS@s#O;N4oQ+O{^F3o0sMXH}m_HxjIMGOZ zXe6;70xOR}dWdy!1$lv&J80sBL3MUvIt+S$0oufPEeFuU2(VWAW5GaN52Ovn03&+> z%(VpA4FzTTS_9o@ARW* zq4RgSz&)!25GVxLLbP`M*BDG28wi6PK+T5~cM_M?+7JXIQQOZ5-LJ<(Z5MR^Drs>s z;VO(w8$mQ`jcCY*0=swO@$s*nkiCG;|E&{@264b|vA`Ul1)!x3utfw!+nEKJ>cQ_1 zVvqvJh;T6?s$<6^$=5JJOUke}6rAou>zj2l()}nScGR^3aWogi7e#a>4hqJL~da=0FU|tXxIT{!LKHco64oZYdR5`^X!Qy`aA`@ zbAAu-H3=Sr%2g*D!Tu=_;!5}45Y-0Anh}fynCXYElsFFr!#~#{wusHF8T^0PdhNk<40-Q+aC1N6={egy(-6T3ZlxV@0H2{&AGODu(;0@TMtZx7} z0>c2`keb~Mk$g;cRMAuNh}C`R28MPIo?kvC(eRJMcoz8sVDcB$h=YN^yC7Z7iD)>> z^z7tCi--7tacK?!uMB`68TjMjpd#xTfO|+`YOtJGezc}KDu2%{#J-nE!D=ICTFg*w zb20fM`W%}j#wT6c0nlY>C<;5!T=X(X<4+{wab!gChJgz*%W5RZm=3F-mJ}}FbolSW zkO)wTrgMj_O=o^Og+bTrxCug#7FP6b-&J&%yA#~C_Jhz1G;VKiAkOHf2i6elWi$vy zE22QPVP`42uRu1P5Hd4Q%gVsB+Ls9kso-^{1jB(i4z#+(S}gEPv?&>i#$h8-i^$Q| zx}Y0qyl>($5fS+(AT3R%PU}BmWk$>58w6q0T`YUrFcP78dbR*ARZzo*$OsH_Muy=@ z61*911q1bD+IC+D;3RWLJX?xfr}2SsuwwMSl8hpvdWklq z4Qkh+J9CeMeB-tCAm4U&J6OA(O>P)81|hCPT_GDrVF-ZQ2GAV@xg`iz(szvyz((Gr z8YabvWm)^Su$)BYZ~5h#?uSd zd;~#-FF-Q_8gb>4(Nlh(@8Bos14t|+<5Pw_)!Q?8`|7-cz-)O7N{8sO3SwmO)U3gv z6hq|cdp*HeZS#6j0Xrzm?-)aK{5P?)86d*}e?%!bKg2d>Hw3f`cyhqYIc#@PM3k;Q zxr?+M4@5mdtf~EPfcxaMn=;v(fN0Hgfu(+eS1@uFJ3@Td;{(=l3n%J0Uaz zC25!w{o~{^6c4O(y7mjP1vhVLM#gj+GW7dk0@7jP*pr#uQL1wg<}}LzEtv4M`+MR! znDd+mau->ZQrO9BH!tjY>t#tDFESvrg7I-=j?Wb^~H3%x(R8^e{K0U~|KGsmvX zx&a1(A<{vQ^&rGRU-W}hL}d}R7Yd7zj_G`gTARD@9DNPXM_;_e^KQrlEp_!3W?KS! zfKKaqi)1P$o;J`Ur|)+bBl)WXr@oXNK`>~YM$+y)l13Z9B3TdhK*eqYFvY0R4=o^D zROm81pYjFB@I&TK_a6|r?@pAoWgcbYf^3~708&fQ1!7nj5TypkgPQA$aeNSnvL8hC z@M8@UQ-LLG<&hYuAY$C!*#xi@15I0Sb8@D!ZhuP9sJTr71zylB^d2@|xE}eaT8qD7 zkk(<~boY;s7(`0+%~FVz)+QLgtb?PLEvQ>UCjJ0XaNF*JKx{fJ2)dTJ2ZOlR0l8Cd z07F$^XaZy0wkr0?+T^j%FNVfs`M0hZRJQjRD(>;W7^(y)CW?0&J^0iqE-SA_WpyEA`W7l~8YQoC zy%Z!Gf2j~M*tia$GsJH&;RX@1z+MyEYq=x}hrbmgR22Rl$vQ>}#wl{592IB2fm{N~ z)q7Qg)r~EV+V}xOF)a89)ZeE+2a-}I%7xS5D^p8Of|ZG~U2_@z)DyzW#hWM#*`J7U zi-yfYOT>T)$~hH9gBJjfVNFnWKq7aYSWakmLKJ1&RgqPuP-joB}`#0TUBmqDfhWEimaS$FJ3 z9)Vtakqo-Lq8gMwrmGX?pCB1kIR>J3zEJyw9qND|hKP4LxH+{QzznOAd3{nxo>4@p zkqsyaY-1OrvvHGFj$Tz9f_lWf_ak<^T+D8CTqX~7tOp(`Fx=^{yo2B2-m*F=9)Z{^Ry3u(qdO5w4bj=EHby6=qkrd^0gDhl#_rs`iRvJd= zqnu^vz{1{5plkCSICl%S0ex;Enx79O-XaV{v-8M}$fV{?|GX%6{sI}ubf;MCwu#O_y8;Fj_8f3kHcH|uR<7`FCEm+)?TKj*b zsk95oFS&>(NIc7hU3Ce2XbQ-&&mF|Or^31@Q$pD${H1~khkh_|O&+hk&`q(hs%WlW zaw^v@{srCSk-(5MPBWD)-RZb{B-o~NAN9#(YCpj2!I`EN`JfMpIZ4}_%0rqIV-!$- zXR<1=4;6qpry>CPVsvoB12v4g$0t7^w5~xUiN0WgC<>M9fkMgA)u_E^0SSDuzlEhn zyIkCh!gF^~{qAD`D<}`@9#I0(Py$rM48X1*sOmpn1e5r)!97&^+J&ne6pihPsqa9m z$iv-Q4U`8RJQ&+KuaKk#`Opx(hMWh6tQG{Ektg9jZx}Ev59zcky4H<8_6K1!qc)h~!i3M)X{#POwyvLrme?T2WPS9YPduA(1U|hQx{hnG&0nc>X1@ zN$D4|c+G_RBnJ!*?D|A?unr?qH{hioy03aZIyU)nF<2rF6D0}#$}&?nw?clv2&P6&(Hb8(As0)jI5YHEAjSo;Fc@F-vBXL=Dz^0k8jS=YI!=ka@A2FM zG#gLP*FYRk7%P+#>Og(t|B>_*cq(+9f&rGBwIb?NrF)=GeHrj>RRVat0Wyp@x|$2O zWU2R1TePpwM&Ml7iQ)&>Q9O>~XQl}lq%23|asNf$9ORYH)I+i#i5>?-s!Ndx6OY+r zs5Xx=NI7ALr=L85(xvyPYXgDuLIxe8gYJ^Lu3cNuM)niXV?0Au*anp9b4)d8PzV2D zEDoR(dvHoyvf03%;))?sX;Yhf@%K5 z6x{y#u^|-_Z=6w#5q<-ippqetRP-qzc(&n{2l{FxnH^)SH`bozv>uIBvVd5U@6IwrmiO8)Q#|-GubFSd}XjXidvW`ppoI1E}c7s0Gi6l7sKW`YMIrcOg6I(wMDZmxf;kVipNvV`0_A4X1`Cw^x&kJ?XW zo(E)!Nlwc+aKVgVO%;N5)Io1F$L}Nn(u@3n5)2gU1BjK=#UAQlYW&F(Fy#aFPo2XG zqJy>FFutFNj`;u`Lz~T60bHLhoSM+uN0NYBMNB2>Rfo`lOE5|(`*4sYKB!#;dT~j* z63p_+0*5oQ3ghJ=^b(aeNCJr*ny-@rQ|Ap(?+b}PyfU4^LePOqrRHLtGMzbub>%Hs zcWayM7NUu*m>#MV`!=1hU;{YeQhEk*UdqTdS3&N;{O91;0(yvb)iei9WbMKl|A@pC zIjM|453-IIZS@#wRQ)AXOSlZs&;#HMtCI*KZC-5$E;z7(l8u7p(U?txNH~a0kOi*) z3q^J)BA)mzL_Apm0t*tIl7458Q_rYKC$Qxf1GF_jgE>W4PWnQW#H`6azQzX4EwKeq z+5n~>e?z#G7V{$S8R(tGL67^701Ta<%aFWco%=H^sx$J{Tw)=ErUi6^LX(N+f% z*?oX-#%=?i{r&*Zp^O;0;x(X=*nmmc?#`oc8AXYDf&pT0s(7JACm$4R`U5c07+t%b z7v_Ynml$fUZ%1w$=AT{_L=JZvci_1e>{C%00ER@BC1isbG0<&uFhuK+n3IPDCXN*_ z)Y^v)z~lgMuAyjF5gJ@a`ec6;Swv>P)*-6`mIo@_(-Wt7fQvIr67Y`MA22;0my`wq_)DuqYvguV@a)p%|y+V0E`_1s4oRz z9!A5k%2`Kbd}WwsSv-)v4y=;UC5eOnf49JfD|jp@txD7jIRbDGK`+YjEGUqTEeUK% zzr@a!BPoL;^eslnZ$h5Z0kCRG7?K?WFF|5*P;F%>U?Y+HEs*n+JOv1I?II0K`G==^ zn79GhWt1GzCf19{#hN)p&uIfS=JjGcIuk;FzmvdY>vpUT+IlXDhjuHVmbfCyva4|6 zgdVF(Ls{YdY{2mkrGS6e)B_GwwgP^y$N3O-`#9>b0#{1!X5bSnKqIzqe}J;kv7f-Y zrtu3{B2d#z?tk3#183-?oWvKL7D2sx`_b0md!h?{xg@dQ#GzZdaEhs`vrxQ10O0s) z;N%+2!_-7CMDo4?IsQ3oU=89o+Z+KKZLsoLTa`;$0LoTqhoOP>I;b3n$z;9J-(m=& z&DMDOhbEfHr85EOWy*mF|6F1f*(1OC7e;gl31OHvw10RwqyllOLtxmu%w%P$ov8L*%j$0zhgM7D88zl$t!`M3{&)4Sjl(;K&tZBPRN zD{^#+xnRbJi5ghq^zb#Pg?P*b^+j0e^eJ01VtrPy@94&dL_CrP!Qf$ieuMDDb6 zKj0xtGGSv9e}Hijy9HCz%E*urMX&9%LH?y$TR?-~c7RDd_Q2^O)8j7=;35wYp$cG% zZ3|2-!E9-HA(_wy|CVC^iln9^v1uQy0G8DnfN$#nc18g(Hlui~$P<-%Ax^Z(;!R+! z>@uW}`hk^6)!jp`=>2qz&&swO>Vd1xOF1Y@7KM%O9sQ2+}v7k@RkBC{1b zir#F1s|WNpWjN~ekP2B|dF<(c78maTerO=`khJd&U^2_L>sM=806TqR5om=#w3v%* z@MTy@dOV8wP~Du77>vdCAe?a<@f@1Z`5?0MEUN&U4-@GT6Cu(gkc`cZ7UQ@E7~6jx zP^at$VD1Q{iX1A?fdYLPr)CT^>kjf7eR932_Aee`$k1Qdf8p`BGq@0N0T)JM>?R2A z*xz8H{il7%mbe8bRD|&jEP6^>h)fZdJQZsKkPPKR>HI=@yPPoeXf33LW$)1nSccrt zH$OyO_bm<XNNE&nF zJtRsh63G{_xcN`#EGAt%UqQM^TahuH_X+24Ov4f8Lfp*>n3RFGC zFk__A*5>TUifZ5nbk?Q;?^w+O42cK^EdCS=$RSKZHuaYaz{KqL7*pFNPKy}sQ~=Q; z`>;yr%4rx@w7d6f5FUy)H)lYT+;c&tNZBmF5v_c{OQpvF&&Qtt%xnG*RGzPLiy6lB zT?2V!UEH_<^zp_Kws5;Fu-_bkq|B+$U|&^i3g~kKHd^2_6Xc^@)GcC7R$?5f8)R{c z!ct<{3HIa)ffgKOY9Txu5-6fn$UP%nBGTs^qwwPWx`O{ig|frNHA)9NGri*ulpnQ# zELHvI0N$?v=0FutM&E!82cF_5zR3zCh!e+p$2;sd5|hYrAg`NLAQ41P@3G$z%a26( zh!aW-S-OjD6K5Uk){zMVC;`CjAm`C{DKs&k6F{09?brZ)7-gQkSiBB3AMiyb1H))xL!&+i!s~ zGdq;IlQMfK#p5!&l_BrSC>YTMn9cy;&IIGpOA^tMXOMaN@>dcTCV^&Z&kxil1DA~{ zp$}3AfiuAeoX#i&Noe5$h#VycbBNCR0IQ1)L&Y-ac!~S&hkA-+;-b+JID@R?Q%lZP zoZWs#5hoXTe1h4bE|Cf0nFsK=2n2}JaIF}H2}D?|M?hWMv<+AEPui37W5f!m6}}*Q z5hVNZ(C&x%+7D$-|5d$&aP_zvsyU%9%Lg7E0p}amTm|Q9kb5dF8erfc!0{?nQ8VsF zNfY`_agt@RZX3SQL7OaijE=e3A~zhmjv9tE(7yABp~5@KBH50mL3h}_0qW}QFm9N3 zZsH!6$qS`haB!6lnbtuy~4$^fjX2WXNn>*aE<;2MB2bka`HJ4F^JT4K4lw zG$iFqGA0$SLLi@zO@bZWWL;U&hucCAuIR!B{(dnTnsIMpxma%`-8%w{f>vDm8?{W* zQEN9WbY`1^g&kv;P5j3zZ_xABIru^hxy?Xv68)9F5Gahp_~Iy=8VPtLC^|C+>3yr*!RMJ*NU zFIg;?ha53Kdds^o@+fx9B&{rovkc-HQWoYdgozOIQA9>7%QZwHx5#NCsAzj8O0~g4q~0+tD#u3ym+gI z{Fw`EMW`=hVSXac$^sJF0zH6Wb#(ud8ZdTBCG3!zP z9@!?^5&;_Dpw3MN5}@9H6UnkZ*c0@5MFbP+$-sEv@}?kL){DfP^${ncw1P+tPMnWY zB(<51%N4hdiZP@(Uq|0)Zgz9fytIe7Q6*tEfQ?dw zT|2xl5Bnw%Q$dx(Afn3l0^o%hgisEmwu9^x+7dCu=t^95bYsNQx}wggQF+b{(83f_ zM@MYvzb5)7vJ0DivJoD{|AP?Mf zPQC913Sn8R$-1zpnVYgVd*MZ>oMIs3E)FZE5`u5z(E~gQ zSTMg(pM2kk`6t=b2kPk;=70)9pI8^^l=ni$&gLS^GY?rbO8qE~bm;*AllUEtm# z#SDX4TmYOEwE#zt0>ss!L~CdfMq%nTp7DAJLKj~F+`j=V#CoFdb}xo6bC3wE#ppqd zcED!R-_+P_icnSw$*7aFdXbJl!HLDq$Kl~25?123L|9&9r zlyiJ5Tz+a96de7rN)cQ}UIQ)SKA*nfc;$*|EVHhs5^5U$^K%jPv(YlH9y=i zB6i%YDuiO`)kmU1wyknE;4;YrAU=vQXKvr_wVYv0ZO7KadO;FLMAXnus8CbP5bJ9= z*ks7T0?V=Awi`EAptx9X2a+(p@gCgDSwI%*cE~a7-q?pU4EBl~Wa*KKw*}UHVsTu? z%3^(rAR1vgN~;YBlkEU5R{#WvGG@KiwW1#>U!o)~czY}VcO~|y4{E4Eoe_p84Pq@{ zhs8?^|6U4bMk?J8eOe?QhCVSQkj=S@#GXrnfqx(Uzd1-gQ2~9wJekN{>y=WV1$QBX zE0K6cDx1fVp%jBY$U#HWW4qsB+WPk(>AMV00rlJ$MQ32Xu`LG+h{e*Lnpp9y&rdvJ z{se{-^bB*AmcRS!R@pIFmVRRh8|#D zwGJkyC=2FgAy6BoiHhqZvQaTWE*}+H*PrT%EqTkEoZUp8xrLR*ao0S6QAh12!Oq`9 z0Bw*PmepPJ0<^l0dLY_{SdD%Rlify$ta`g7m{m-Pe3L-DCvr?1J0T@?7JX39 z77Ky`EhX~(fM~ZJ*!-wP#i$bRL_WJa&FUQ=OMdjoFt{W_ewgoLn}TsI;(fvyaJBBt z!cbU9+d+_0y$*)3zUZ5YmQXodq{{P6Ug%d@0_-hjRuW?^tfSV%K(Nq@QmUJbyF&b0 zjI%)klKZ!lD~PU%M=+`h$)Ha=Cz2S`N`kxs$jsnGjxcoNq4fe3{!7`5LLsv42&SQW zc4{_;>-QfRddyde&%)u`{!SPH$}ck2gNfrHek%DBGFnnlr!y6~>k?riF;^vuL_qKR zlXjL9ramS}fs}XRTp;ftxn3mqRe~~DeMzVpCC($#f+AWeSr3HwZ?gt&O)PLQFO2lSIWe^JKgo1FI4=AwaXvqX#r^@$|s zCX$y^WJq#4VLMu@#|b>5UC3a46_z$kAy;Q}6Aoio_9c<=^G`emj12^1J!#oF`6OCi ziH_6id)q;cIOt5qbvjZ$j6PJu3xjGL>u5nOd?q-K`2AhH9ZK&#Cy#;?SWYOUf19iV zjgW<)xrYe0B?O!0QVza?2tI}3uS=|6yPM%&`^=Uz0+i8#fBKqCklUq!&P~Qavm$5{rWp$pp|S0ERg* z%E^JjwrV^L;c0slo<3gy!p&|poq%9*8dZsvLeKt0!j2!Gm4l0*7ym=v3Kln&^$x|K zN*l0z1a}hS><&?wf@be9;AwqG{@{bHMKPJ6x=#QT;4K-n&N7UWltNjYH8H&R*`Vw+ zhL_x7WF5KgSp`MU`Zh>fFv?uXHe-l^Ay?9={ns({I1_O0=lHy2#w$bni^R0USy zMQpiL`zAaeP^yB6ge25}t8GNoCWUUHxgJ>{%*XylOKpXQ|KCqQGmm|iU(cANzD=Qc zn1sH*ti< z;JlfLA~zXoABd7#f;Ggee9#Btg;)k%xitf1k~2^Y(~Y{DiEeBHGBv9ZME62@>$0wW z-%CPwk8+YbN<$bk65?UrDK*;2sAJyKGK*p$uxyR&K+sS@?4EVhQ{aRIb`I8yDKKVO zTJyFuPT{27fJ+PoSMvFYv&7BF_D6fg8RmGUun@w*dUt|YR+6_CgW=2LFo0Ph3au?m zAp?Q_WUbL>-lBtI=TZF@=7n;+2{4N3rnbHWHP^-Ou`o_zKVYUCT1Uf=t`$o{1>#py z>XHDrF<}%fkLLQ&l)7*-PS-CnGC(@e3hq9&8GWKC7tnfcS&vz8av|>Kqbqfr5OqHv zf|X92S;91q4R)@EDr8x_lJtk6JRvg|hau#*Z(AnmS% zn&MXE=-@fOE?S($M|SuGUC0Q%8<7)SvLPDLlK&>Qk76z=7TbrJRW$NF-y^@zsu7K( zW}%UU#Mtc!UcMhir|Eh$PBXL1_v}CzJ68J%vJTe4LZxmW1HW!f05Q$z)gdstl>IYI z4-(dI9)f@rq)%a#?w^52L%TL&3tM7(nY_E;HPrBf$;C_q^e;_w5h^lGP*DzYL=58tG&VbyU80qOjhAW1 z&a)f+6Nr!lW+LIQ9$al`0axFe$j7pDk;gn8CAWuR%-r+HCk`U3KY}a(=8KJFa9DUHkML$?;zpmJ*^A zqyGRnLNn6)*k8a0ad*J8t0Tab%>WyTj#b+Y@aqnM0*qC7!r>$YNBa~imB8lt??*s%FC#TgB>o^Vpw7Pm$a@QbH>z-TOz;(! z(-1N9l*9p|iQua!0Q~a-II#@Ctq_QP6{E0-rYi9~Sp|@D1mHt8z*9^&eW>aU7E&=X z=}(m((A$qdqAxj2A=!_f(P2lvV3zbHzG3Y3$imWrj-cDB$@e7sYLFbP{0H2JsK(5$ z;!KhcpfU-yJym?j-bIoZjtC)n8*S2CDkYE%AZ^%(;S7OkQ)!)yuVW+}afFQAXx)ld z)W{Hsw1Zyba@!PqTe*!W5uR{>GSW&#)u~iOe!3xR)CbZBp8Z#$A1$uC?!^lIqe(XG z`UQ1PEw;IM(8nLHMA_IwkF@`lVo#5Le3U-?U8#7e;1L#XNS6Wc=(2gNb<`B5k&T~ zoYBDwWL@@v0Il#u)^aao5#v728#Hi!#5(O+i(l8Ep7^4?+z;U2a)2ddSnUD;c3?AO zb?NbD)FENuAmTe;(3f2YI2-)@nTolm&fzPGIZs*y5prRI3nWtTN%c3+;8J$u5j## z7nosK$U)`GTTmJCLdNV4ay)7Y%2%k8c#&9kcR5~%1a%9}`gpNcc3(NZumr6uoNe(E zGwcC!%(MhO3$EUHDPxW>IsUo?!z)~m@dgfD+0e67Wj$u4se-rMNpdECXW{FPza#1uLzm3I|%Y9btqL(2d@bGv%BQ+7<8x% zQ-`mJ>|%Eph_xmzW8+(spjpIux zb3KWl6T`7_lK+aOu^{(Iyz~K%%~7-ZElu6I-^a_G=GZnlYuFO~_!2LBpJQi~z@(*l zKliWrxzikRlLBm3=oCkSoG6tPB`9cRi8+CxtAm&|Sp@r32LdC}DRiCwHEC3CgFb!b#x(EA)MSg35jBP?SiNm1{rG!UWZ6 zs(ex;-UmaYLmV!|!N%V^4bz^AQq|^zDRbTb4;PtM=5FL*a zYqDC}&$}XFUJU2SVX;f>zBJ#OK+U|AmH_eF?8_baA_BFVQqB&GKWFz(Q#*7eY3o2;X6j3bQdgd&Xwn*AW>%%1aq0~= zDLL!lG-iIFzI$qKfRr{zXg9MYaDG(k?O`c%jxd_PD$pQ{YruWBOJ}$R|1nF$2Cj$w zv;8|G!_4X@rw$5AS6RoV2{06_lG7xGl&Wls(lE{LHW>~= z8f$Eu(!^gXtfI2gg!I?g^{34aR9x2u5jU~2pP*&$D(>paWfQiva$xc(cqnCc?XwVe zv~twsQMs#BlDt1w*j>TNAzj^4xjy+|tFW(vb6C1ohjLqT;f!#Af=gPuo~24}aU zn8K2}bi)pn$H|9cMWPg3yVFf9Ro^8aZWW1DaC@7+s6+K&6^k`8s#l%q4E6$;*I86K7zPASKX z#g4N3RP*{YYxvqioKCVY?c(**n-^x=*d%t2ec34Q%I0~owkIdWF0=bme1UqJX|~O} z;y2j+l=woLHH&OdMTy^GUv9$}uBTOJd!|YJ;j85Vnd>UG&T^i6EB^e|iulYZdF|_* z=hY;}UinvK##CxQ80W=0iEpn~4QKA!r!#%+(y+vztE+$W#S7_D z*RRM)a&`ttGn4n}ieA4OAjv!uXq>f|p{J1ARU|2T1j3mat*2{uypEuhpDZ-ooc(h@Y9RzwsQe+JO$;?tr1SPFIz<(;*sNe3f`<%$6^{4sIeKs1ldzv?AbJB+U{O!>TChea0&)J!@aXS0b z%LRYAhH2@L&$vvkx&_tVitXsMyXmeD~ z?UyFz+;3@_s-T#>oPj`7*Bj#&GRLg9G|YN<*VKRS`&gMK#jV{rLzW98_I_xUX;s`d zI_p{U!dUxHlQNe^wz1_7%bTTge^!%?yFL;tmpfW%R>1uwO!oH3c8lD%@{4M@zt+h< z9N7_&J5jmlEcf@fvd>3$#^+ATn_uVtp*DAHWLHHlR?$Q5Utx1Uj_kgYJ0rh%jQdaB z+;1a$hI6r`zH$G1JNNHM99tfnf&~}NP?zHxjhD;A`V#GC50vBYN-)C#uccBqRVgRd zmFO?X%(pb?=6We7)0MPKP*B*)rkj?RSMEwK6%@_4^6ushl-KG?xgaPhY#rRqtduwG zN_{FQoo^k}J?o|XqOQHa1?7ZolDh@v6>PiGq=l67Z4Pt`2P!P-N;eZy6Sl4H7Ohmk zFY5UVY3AFW?iPQku(B(2myoWoU01iHykbaKR;iFdzTN$9tk(5i*%yS2h3!YXr7IP; zbmctVw=mIuDou7saaVQjjL_mKdp1WoA*H0`JW*lmO%8&N3LZ)s!TWUc?Or*^@+hS% zbN2P^!T&A|rYr2|g>Jy8D5 zHU)jc%RQaf^qIsTEV7m&RNbJ;}i z%8q&4ZBI^!T^d;SyLUB1Gr_irSlERiH^Fw zp&hzKTq6Op%;a=k;n1t@MZ7_Q+RW4*-PxhIN-=9B(43jZqo)wsl~*Jjw8oX0;i#t( zdabEQd}PfkW>&hMe(3f7qS-+~8<;sgdZu>xgq!TWpj~~r41G(xZl;uiS8!I}K1+QE zl^(TX<<8)eto_^d-Bo%Wiq)+{>az}>*Y{Pq8CI;-8Pb+j_(MNHr7x{m&pNc%rC4wN zTCQ8wQii8O2VD+@&)+b1>q@b);@UCh;bZf+j@=%XGTXm)y07fz{N2~@{Fbs131iQ$ zkTOWF?UyWp))mgK@-xV)y=z=z&k?SWU0rN&pmxBw1S(iRyLQN+wDw+Xi5o|RRd$_} zVRh~Oq7vw1&+K|X!-m=ittHDiBG+U$6dRtdeK=SG<-95T#E{{I+D9`bt2x#sW;aP0 zb=3|^9)hOM&u;NE>Z=_xKD3r&eO319Vx#-Dk9`k8ZMS5f9Wr`a`y}?zMve_vvs!EEN8((E#C|)pK`+V>a6#7*5rJ)7CYhTP9isOi4&$%LH zOdTDTEKN#^63)5mXUshM(ztXlN3=ptSFy3^(O15unMu+5IoF4brH_upmgaJ7vdZa} zGEv++dO&*r=1uOdy&fj>_P#z{S}?L{)vUgB6aBSg{iWChHr&2--o#k-&D+w$ott;$ z-1%W*q576STxlJXk#pC?)IoJz?Ql(JOi9kY?WXRk?;H+ed#KNOaNg8c^?lgk#?CEm zIgfsr2B=P?9d5SX+M6?EvM@~bL!Hc-Q(FgJpF}U*$o;WP=KTC^Z(Xr*Y~%hkDs%DF zwi(wKpBKh)Pg1fS^J8hZmwIMl={6E=mdib5Bil27yPDhUX0t->FTt|6 zPHi`Kd-K^WuAF;1S@!Py9S&~edW(*7f322%aB7FI+xzH6C*OR%QZ}f#GgRQiu|;R! zd>bx%dSqwJosTaUUA*!Ax9qUUuH-wD^5$JfXC%u7#!K%YP-sP38U_TKxOyznSuX9P#XVY_lz>IyR{aj^udZJoe=l%sPfi1vizT z;7;XRh@`Rm$nnJ_=(=+aSjPZ_rO{~W zb3y40>rG*^?pB%x@BJt!+iSfeOyF~+`RLxif^xhziD805RaU`ivxF3!Y%;=x^s4Me z(`1E|GHmk0gqJ8dd8cdMRlaIdtR@m(h5gcWpXyJW3N_L6Di50sr+w8v7;`J3$S(6en7!ws_QJPcdx9?^3`6xW~5ZuM9&W%q)XPB{{#lyhCkM8kgUt#sv) znChH|LJK48KfaaeI1;OrJ0`TK-u~NL*_TJK`+gHz{LcRG+qsPDc%?ipVM`4Mu5mf} z>g4J?abfEShgswDmepxW`&5K&>m9_$6#}cXs`nWQ+rM*=9al`K&Qr>_6L!>aR2f&Q ztUgekzf{;c!clu%xud#BX@98jl6ps@ag~?VrPcel3cI~?G#^)G)Kn-PNEP9|Hmr&CrTF^BB(>WQ~(cTt>K_UhVbKB3Y5(443?HeUIAS;9_l&&|a64@cGfTw?~W3MZ$QO zRD~Pts~rd~aTSShT2dcw*j)Q?v}BdYy2K^Tjz+gs9(x~JcW?dGCFdO%d{B9Q|In^` z8`)hu9F6%^UwM~i-HWzz?Qt~GQXRWrT6%ADqU&9CQ^%t)8tV6NX>%P^Ul@M$L)YQ9 z{o8)J4y&7`t9~{p>-O5N@AiggQH|=?(`AERJ2$y~;xWIb`op07t=H}rx9>cQU#tE( zT|VO#H|6#Zfr>gdR^TF{lqoy{JoZo)M&9vrTfWM?qa=GE7h0> zD-G_azH*o8wcf2JC{|^2KTW|yx!2~fn&`nQ@B0~RJhXakFRDq39SOdlRpnvWYxhh| z`rwgG2XY>JnAO|=sGG}Kop>N`wx>K#pvsdCm-9XL?p>0R&S zcWhotP0N9T$DS)PoVTd!yso(P@-3~ zlj|*YQ=g*~_YYt7+V14`LEZep(ckyW*}W5--1#-EeCn7FDy_WJojkNO>>kufKRA-; zoyY5GtKn#POwFgJ#k-K#D@0?-nPbL2N2k2YdA(CKJPhj{e2(e+9Od+8tZ+|zFYdT-oNJfRxxB>>Rt9nBY?9Z@nF;JrCs&DQpbZ*8uAae zEMMAJ?`Lv+EoWoZ!P6y6@7FK)J06+R*mCgf>_Wz}sfbl~kM9UM`B4mw{ES%r`S_mKC;vWxL&r8RfTsag z6mLO?DIy1f%9<%>nk1KYZt@l2TVt!4ZrH50^jeFr1Yb~yX7-t8+pf zRw$oN6~C*o-0pptd}DRZ=>qY45zAfQhg)jZiJhr_c)x174~R4#PdU?4@bK|+|KpL{ z8&7hcy;?9hdqwc^b;XURww&!>iYbkY%v^s)>)h9~!%Ls$uh^Wq;el5BmUEL!pAW3q znYr<+*5$9~*p>|o`zK{aiD`FkX%$@d%F{nHGulx5#@AN4Wuy82`!hHBXy4p&UU%8p zfd8S)&0DnZd_`R3E$_;z$e4X62143gCEhu$tdHE%eB$Bjwp9`n87o^Nx86PRIHY}p z#K+#1ZIRnPpLqVdeUHQ>@2abju|g+bg;udW($-nm}qMl|>AWjjXUgYN-=iN^tKZJseeC4dYnL7u{@Ss6a^9Yv zlRs=PzbpLHwt8k>-1x~q*Dn7o{P%M;n`S&u6T|KbZxLf*0Iivz-o$zR%IqSJodJTH ziH=P?b{!f;ob3UUnn~eJ{MS27i@3i8%w;C0=n5HKb@1hxAE?SqsnHcXd)3#MZ*!my zGxeITlu>7xFaN2)1Pqt!n!m=LnJ(X~RNGY`DH6HH zbt2ueSxx0yt)$rTHA^Ql0-H5!ubq{Yc)w=VL}o&>uFCc6l2Y@6LMO5+n+%P*%^J5Bb z-ybp!TKl2z@WuJD;&(=e%#N)6+E?~${;q;M(?jMXYyb3>|2Y-M)z4l6E0<$^h0y85 z(EeGot(?PnmQ-pPq(1DIehiaWXi3$Q)9H41)n?ls4-@}T6?i(U_O8KWyOA)NTSs;q zlz!+pE<|%=v9gTp~#rV6GLa(Qy)#0 zta=i;tMTNIGncsr*$)NGS(lvMBxcmPbx?4{8n1Pk+0BMVH@*$Ztq9(~Zhv-*kI~Jo zL%J(M@2xwOeQJx*oo_=HE5bz9SFJz2@9aS6W7jzm&g<*fpJ_h(aP0A_IqNdlx2!*V z_w3`)CmZH$xVgS<{khL)pN~D+Gbf5~!`1bzLg!wEKFyi4$$3NX`ty3{#>SqO&56m} zFtEOD$%1!FpEW$$(z#(!t3CYOhwIPU4sH9jVOZ-z`nk_`&wCGTU%2tD*2QDzzFvR+ z_|VRs8z;3c^_=@*_u}25-R&E{f4KB=!SB=;Q-}6E+4#Hh@{e=>xQ5wF1W<9T)a*Y%|b+c)`Y-wZ#mb>nqgY2mL; z0or}(=k@Hzy8VmiZ(cj=R*i|_xv@e2Lz_2moOSz}iOGUDZ~YIS+PrPno!2Id&b^uO zFaNSR?&BRsn}zCIu4fgBF)6qEWlU`f--&F&%^aAwlti0aK3HX_jds+8}@HGmGkg{ z>56UdqgI}{x21iV`p~_BCrKa%5VZxAwXYX)O$WG*SBORMOUd z?Z=Mo;r1Wu51(n@I;j05ynX$R4{e9f{n|RL{WQHj+Wuqj;qwc(z14nptbNOkkB<+x z@7y-2{k*4ryZxtkhcC8oo6&wT-oE?Br=N!}|Juf;Gt6@#!G4mrtYcv;t@Bd-LduQF z*=3zOV+D0yIbKM&|Ey7Vtvyy!XC(YW_KnY`WjB7s%IS=zU)X0qth!V z-k4fmc5~-;U7fL>3&r+dBFb*HZ;vz9c{6_D@Qp7!%I^HyZlUv*=VGP(bVk|Tg*zN{ z#?>#@+?XyYySHntov}Kf#xKGV`dRkk*UosINuEm`_A|WY zFBk4g)A_7^>DrB%+2tcUcjf6!IbQ0q|DjR-x_wuX&X@2@w{H9}Er0WCSB1`W`lY+} zKb^|Q=kGo$@U_PL!MUHltKV%_r*Kk z|6F>({X4OIQe;nO?u^*tk+r{b<{}Q!yYYwK<*_%v%jQmJ?itwl)8q2HwSO93eC^yb z)6cQTLQs<3c9>R5Q0}M7EJTageTI316SVp{FIh-Ra)b;s zs}l_Sxt?1{7jbMEo;8}VsGs|z4QnWaGy*S(6_{z>ba_}e-b(A*;x z(xpi;O|u?%z{~qmA^S9GSChccj>Wi^Tq!4=oZKWh`zl;MkxB*sB(meG8_jE7sV1FL-6UFa70#enrG|e>Lz7tB)nznaP^D&R z%IPNY$5-JQZm!gRnsT8@;^)=XG&8AES30$;Npf~)P#SZ8rM`b^Uz3z+=USToNToq( z>iwqK%R3{}_)k?DJxzVuG-pTWMtW9frLpwh(I)AV&dq7F?o}dmJkcc6*13%q7^yTX z-8qDX+ra>?4G7cH!JSw%B6*^s~n`$m}stor%Q%Yw7UMa+P9*I zUwI{Gcz9^I+SdDuHNJv#>gO>JinUv;y=vL-i~v2&jBCvf;y0`O%re9DwCb;&=@x%d zwPIIhl%96)wet=V@2gf`$c)v~d3Wt%x5Tfi)xR_2^>lf!cQ{D$9SJncO4HNRxPGl$ za?X*UU0Hd0`cBt-9HizQ3AvC}q&Gj}`mJuMg-6!@&Z^Ke$hdyjVYc&;aI@^ALWZ?= z4_asYDMqf#K5^Hu&F<06?7*OP2eMD?GkSRaaoC)ViW~Z}+oBgtU4Q;|PMl&CGv{iw zvG9#oVbZyZn_O~wqfPX0j15X39@(6nbJxSv)&8BY%<&^zF60dAEsVJFpqxCH{x;cB~q=J)T@&tsf>>PMZyPUlGoTZcOEpO-fY z-aDN)SJ)=iLC8eGF*w~|pQ^BJt%KNk1;60Tn0-3JcGn%GOccX|vrq3^AZ$P8Aah=E zn^Nw?KJzJi_8vKPrM*h~H1qAI926Xtij@wI<_G658E`arRFhJ!8HMNV)8e$eXI{GU zsp^8K`Ti}=J9>0@R647Rr1uB6xRmtFPgl8DUE;rgUCWZTo&`LrBh{s)`(s*MANMRw zSN&35_H_TQ7Pp^0i+R*IYAU1;B)7QF?zK)=6RD~4KakbpVcKiYqpny}U3%a^i|6uQ z=XCY?HMLIFcq)JC zL#Gwn=S3@@=sVc9Y1vfos_}Vo%1z9ItDe5IoY(kiX1_+XzQ@x~+c|Vd^YCkU_;)>* zyE;esX*Iq+(^W9!xnhI!h9Rwsug_71FZKO1Zbmz5->f-rR`^zUWv%m;^V&lp?JQcnJIZ(eZbzFC@N>^5qkk=Ylmkd8Wt+8ueMN(ct8(eaR^cIcXz(ruM;2f8HKm8?R zI51H63OV6YFr>e-_U5i)9g)!MF2yGE*Q(rVC^q^M`l_$=*!*oOcP5I>zpQ1yRiSRM zSLLo|iQShlgpL-FjmNXO_o*8>MTC(z9 zQbyOt29*wl+*?<+E)Ut&hHyK3sh&rsVd$bW@MD@4OEm-@oOo>!Tm0fomrY96q&w>qFNk+ZRT! z{doUy`~Gd;TwnZHn6P$Iw5)S~thn3AcC&qJr@YHY&N525Cm3190R;dt^oZl}TVdbXAyQEUX zJLQDC_+zV;Z~0Ox^}JJWyU$*39a+bJvC`OJ@4J44Np{u={He4sNaMXL?_`r%C$y-_ zUNl|BL%GeSI80z(_7~R>Yh!X9Y|o6?2Z=rZ!<9r>X+v{(EN4%*#B?+;tyj zO|SLqZ=ZAa+2Px}N+Sl<6Fjz0P=PCflC&OHAue&IJ3f1v)C-@+dAzW)~UQCsj^)Zq{6F|qu&Xj1!I zysM)RsZe~tM|d|Y6w?veu27u7Dvv_37CD}UA{Sc23x$9LZRnr@8lf=^&;;gqJ+V-{ zfzhNw;f`FmtfZ}gLeXhkp@>=}mqjBzrBGxbhH@tH3a;WBuHyz`xt2xDMjYiFVlMJ< z6Sr_1`M86-xQ7S0kB4}K$9RILc#dayftPrN*LZ`s_#6M=9p2*uKH?Mp#b?FC!fp=)$?KxJLOpZXgSH za2NM*9}n;lU+^6T_yMD%g`yc+z!)uIcCt{&VGave!V1=~fdZdSkxw)_!ybYGoZ*6r zXmgJF(~i7I^y^$C`r~1rB9UfaB%Tc{5`W_z8VxEEXK)rb5#UrLLJ*2DC=rfy z=OQt~wMfjw4YwlU?NucH!c@^BmZxP!aMrJaYwM|g}Uc#1sEKO;WJ3%tZD+~oXg;v2lh-*|_A zaEogni0|kbq{1>0`1z+(E-%)@nNXyw1&_M&}q9F{>1WjRxeA+W2HbZl?fH7L4 z6->|?rf7>cXpeU205iy84hvYq3f8c}9oknAZDEIw=!DMbf*$CJUg(GZ7yu`@z!h$A zhX*`y4(I|5MmIE5|p7F z6{titG^jx>J~PKU;vdxG3&&p}TgB@P0^d0P4vDfh8bAjPp$k3e!vIat94*lbCTI;) zv_(5uz!FxlMi2BvAM}Mi9N>rn7>Ge|f-_t&6vHqaV=x{QFbUIF7m3+xio_hu#XQW1 z4;Ek{)?))UV+#VHh6a1r7KwfM8~@lq7aQ3#3ByyNI)Wzkc_R^h7|0; zPVB}W?8QFp$3Yy%5v1WLPT~|!;|$K?2I4l;7ZQ+&BqYOY3+sU;Sc+v>jur3=VxBmQ zb2yKEA@qY|IF1uIi8DBhp-S4ra74l~fjn(x&9H7a@3Z?D^FiL1@DT6t6PAZ~ouU&u zV*u_S<~;`m$UVoH(%Ji{%4E&);u`N~h{$HYxV!Cn*#BmH9049(F3#_(-i zEc}paQ7m>r(YaXk>_&7f7OvPduvmCF6^juViBTAhF&K++Xyj5X4A2B}?DZ-Z5AYC= z@EkAk27lun9ETQ*eh3&-EW)s7T(Q`Py*Pk_ID{iOHG%f06btid#ljjX3yZ~eq+%D$ z7ZnR9c;L;3V)lD6`<=ECg6Uz!Vm9(HS6M9Pp+z|JM=Q7>8HaHM$59nkEUKYG4Qf$` zoakcVA5$#C@idOP;RRlz2*v0gUo3iH@YZ561RfZN2^fzf+l$3XoI){5P>K$zjAK`^ z=)JF4^hF=oV=TsD0w!V-Ow*`?M`yVorBFc)4Qfz_Kd49ZbIcj#s6ZvEP>qC(#Uc@x zbI9W@#*JqA#bWW(Vlneou}J%e+y%>JM~TO994BxRr*Il)a2Drqle!Oxw}=mkw~3F4 zkMRUg@eI%L0x$6juaQsPJ9tC+E&j$oc!&4+fRFfu&nUnzl%WEZs6sVVP(y=1*z}IQ zfh`C?AcAlJ5C0n@9>d^0dl^&k7jAr|{GEB=4(>uzLEETB9sZymvdUti1px`#&_M$< zgf3dZ7@-J*5~&zp#TrAeb%_`PPk3Pj#$yV?5rJfE#Wtj1J5sR&$MLLfi8$S^L>y6+ zhy^`Lgb$X$5B~UwFOc;t5#y9{*;q8_OFzVhL|y1XAC1r$6S!tTY=WjRgb|vdIa9Uzws5{@ErvxLLK(@FA@9j6{RRc1u9X6YN#<{K#3TI(HMiVxPXhe zgkR7aSR%C101aUaJ9I=RbVe6+#V`zqkrVlGDG^q9=~^P9+)KncT);)R45mD`M3he| z5g)?kvZixb4;Y~tTEG~sVTv|r3p1F*0(R(xp6CT{Ou;PpU=>zl4c=n%TV;tSLU=6TaXZzFuJO zTw*_8DG_OSgva=TZ&-S@MC4rOb&=2O0^5*+?a0LBJB%ICklia0Jzg_!^g_G$yw)%j z!!R5^A4m9DgADRwC+&TEwV0xlEfFO>97Hh(>6P7|t0Gn;@2Q z9OBWGV?&st4ID54{V@=Ou@PCwMh*Gb zi7EICbFcs(@D1NlfMS%O6n{_;*^d&T12f2Bjy^xh@2?VZ1y|9$utfC30Nlha+(tg` zpleZy=!Wj-iN5HE{uqcsaDp>j;EEyeM3-XjM>lkbJ=`!DL*M~V?7+=Z_6NF`m58}` zRmmFSq9_%Ykb{;wrNSCEn1}iB!2&EqW|LBJ17BfoSSt26D;2lk-?CJsVP)G=u@<$E z+Lel>SccYirNR_@yO)Z6*pCB9?OiH%!pfO`(FvW=72VJsz0e!aT}s6ZyhLYD>LAIh zR3u|7wjl*Ou?q)r7?)u*hPj~)+M){8P(h6iV@t(Ggp4m0G1xSzRQx-+RQ$pc?^5yC zY;pk~EJVf}a)B$zhR57eF$}{o0#@_MAr$C=wu?)}v}L6t7$NAjrc{JMiEu>V!KPC2 z5RY*whJNBo#lWOe;e?kvO2u2e!-hSjA_)11O2sGKIZ2;crD6)&W^*o=xk2krsW^A9 zR9wJCT*76fBLkVZf~&ZO>$rg|WFrT;$iq$C!foW^4on`Dim@=*A(!=iTq^A0fVLbv z68oV)24Elt!3oZA!4r&mLK_&5$(V#SSc`S=#RiMJwD+xzThjq z;XBl5^tx2cMcCh^Lie9iVFo$OVF626p*vjQhT%AZ!#IPpxPSxiONHmhQsIT67>424 zgw2ThL{7gm-U7yp%ScBCGI0eEaH)XL?q!Dj^j0kbSo1cn2#~N=(9I{ z;zpk`kp-*1WuhZG!2!GN%Y<+&6B5P@EE8kl4bwqoqAe_)%0yFF#xbN!Y`{ir!e(qi zAc7E#5QL)nurkpC#%PIFFhOgWq7B-@981QQiKSSEl~{$~iL`~P$z?)~g)_^00bczN<< zk&j?M#)WwQGVyCunJ7dNN>GXlRH6zEHnSGttO*)KmWhUFgvMxw<}gNREPD}3gd-AB zh(-)z5r<@K#Wtj%M?#tCiC*Z9KIjX3IKUD8Fa#bLju9A%J=lvU=$u?8x}YmukhQZ+ zlC_yR8P=QJey;CNJVK_!$Bt}8!UYYQF$ZG`a zu>l*g37fG60SHD2lCkJ<8Slqs;`~$g_dmQAK9q@n{w)*Zips=zSX7pYwRL6UoK!BF zYnO|M`sLyg8XA@J`-gIV|4=UU(Fl#v7TPV#g-fe)@yw)LY%?tv>)VuzFVJsGJvhqC zg|VVsv_vb+Mp)-^p@eZC&LP6ST)6fx7cD%f2lHX&q8pNimy2Y~A6YIAjxHBfupC=1 z?oKEd_wWF(XP1kSbIQdeOhp*n=a!2j^U8(&eELT}7%V6kP0$qT<>lhaDY>l8igIz4 zcny6hJ753?!5J?`O&AC1r$1~`H=7;j-uASi%&A`~`31j928RNjUoq zhA_e=Y(^l0kPyXQjHZp4auJDWWFQmuapWzzT$G{=Bes=`v1p#kJkbg!XphG`%Z1ba zaxvxvbzphDTv)*xHn4>qZsH*xLFWefKts6au#eycle}`#8m4H2wrGd;n1bn;fdO~R z#Xz{=59%S^<9z@+Xn=X|$QyjH0GqJ|0qFjb*ChJE;PELc|~r@yN$r%+;yj`=|=B7)!7e zS`8|Mz*oZxk<_e0Bx5VKAqBgz2Yay(`*8pVaT3?jp?!rg!(;Ob5qpuZ8Hi<2qYgS2 z6`}zeLJ#_AgvMwB6PUvSmau{?=D`OGu?RMnoX1}J=t%5@&gg=!*vI*9#2)C0KCp)a z2BNQ3g>Xba^v3`g+g6Bfb`_#Kd^%PL|85mx15CPCh}JMg8?;3`bjGhi6`~MDsDK(1 zohn4m*a}fIsX}B=A!pMoL>i=76+##0VFX5E6qY~@6*Q

CHT3HlKlWNy!rlys`2%XUdH#%2}EIhR& zID$01K^e;Ny?do7K*t`8w{ImsL#-5+u!0kwyH<+VH{`N>w@P6$xKgx+DcYbd+Mzu< zzzpVS>s2X^kEj%95R)aBIgcd=aD^M(F&IPO0Z*)*P$|~I7k==^dThX)iPV`y-Y^22 zu>}Emh1Yn4cX$uQluBWTsR%{_lCc#p5kIw3oWprsz>sN-1D+TT#~GEPANs=uM`n>X z^zf+^y-~KPQdFQ4Rj7svYG_b{TGXNC;!4p9+t-pOKk|xh=#F0KjXrS07)(PB#`-g+ zT)E5>XE!req~j}!@Eav4MdvN7Kf0kSCSp2XV@^<|ScJt`iX~Wv+t?Svcp@r=ete~9 zgvMxsrf7!dXaPCQVF626p(EUJFORRW;JK|*cwrbu;4e(YH0Y<&J`6B-N2QpLZ+qB# za68Oi!3mtiDVV3x7c5}~YuG>mTi9X7*-9}JvoIUWupBF}5=ECQMKMZHigHw-65>&% zkP!2@Qp6z^@kl@tlCc#{pH>P(G(!s*qa|9w1gqaviZxh|4cLfH`1%j?!*>+m2Y%ug zexnf8AK5dgLxWG1LKh982Yoa`V;G<*4ABlvelmA-EaJTcozVqd(GA_v1HG^qpGqpl zzxa$VSW(U1&~Q&J`yE~wieVUzBl&XK5n>uKjd%>laRMiC8fS18=Wreua1obr3F*kd zxj&WSaebwDisyKS7kCLnQ6-Gf9LCttph|RZTqP=?K}eG-5sEPE!fuR{SBVGaRU)oa zm54_wDtlCkDqQSYB`)DDHuSC%0dVM3B^=QoPH@AGdve)k;ueIXb>Awn8ezDIVEZb5 zcT*+GP=RWwQI9qKs>FJ1K(%v~P(ck1YEX+h)I;V{B_wF00UAOVjS=QiB|dpoiGT4K zfn%z~?+I0+5I<&AiDonD2a9~F#9}PLQmn)(tU(}x{);Qg6@n28CC=b1;@4M+QJbp7 zXpF&FjKg?L!1<^uaREVFxgQ~@+0GpB2imDs!ul}lbc`_}=6sci#d%!7MTA_f;?MIc zaR7&K1jpf(QzeF@V}6zRc9(e}`ysh~#^)hl(+{-X&<7eqAAMmD2RNc1`eVvJRl*yS z@fW7z@w+PV6z=bn^X&vrqz6ZUoG}9o<_vRFhCPDg&~@wJNjV&CSnTy!Zb|9bj-jk z%tbI7HLDgz*o&Oz)gli!aSOMRkK7j3VzP0yuxeQ?tg!{rh(SG$n^ueV@@ioTE9_8I z3;iy%jm9X%?w4}e9+XfnMJ4pQRtsJ1ASY{x-7fkC%w z(HoBNL>x}g-btiV-hos9^+DW^b2yJnxD1Q#j2%{Zju&`|xA+_X;2qvW-lJL+qXebs z-LqQsL0`F$_wCBLY!~Mhs#Rhj=6)5lJ|RLpY2hNW)Rw7|8gLjU43Sh%-6&WZw>D&te%? zU=_kp57{vG)L7<(U+6HdT9_dcQHaLe3Dv>}3$X|pu$)HS=^SGjl(^?pE#Bb#V)Bg+ zD|k(;q-_jZRn2GUszrf+wfKQa8`%@MxrsfB+i(jdmq@{ObPc0kEPEE-n1a7B75+Gm z6F7-)s6q{lDtgfQgud z1SBFEr*Il)@D1Nl0FQ&TeTey>+i7x$4Y{<9O*opzz1Vk?b;Y^&{FzNm!%-Z=qFdw_ zOR*d)unPC`tHrV1JKU^gSVEIVSVI@{!HP*mj6z$P}FJSX7}JDyX4B4Qf#by@~WUNhO?c0T*!z>0jis*^^ab4(4JW-rz0%#y@z6_n1C~ zf5Q~XGgM+Lwjpe`N+=PIW?$v92x26n5RDkbA`bCLKq8WmjIA)8qY~}W0cMcH92V#g z53IxQxhheJY4cU07$qo0IVzxr2Js72;t(tss)QA+VS`@yx=ba0pxtu%K}Yn!2z*#c zomDD6+o}@VaTXVli`!`HOS>CYV)7Q1@J5e7>Z2d}V*>Uf4fBIk!Uqem5R0%FORyYz zVJe}I7BEIj*!@>lQpW6X)&z603ajx4_2?R*65TK>N+lwaSjSx|aT)2z#1&k{HC)FH zWFZ?l$i*AX+t2FiH7jXXI?n_oN;2^KkR3$#|CV~CTzx5Y{S$qEs9W#`PN)RIHC}Z7~~)q^KH~(UXffjAIm8F5LXbFV^!5Vu!C?iob59=X2r50Kckf203A`pqD6>?eBX>y3b zGmPOJV}Q?P_9d1e5>bf8m~{5Z4YfF&MJ^C@i~WdDgh7cux7ES{33%{8EuP>Rp5ry% z;MNPZxQ#0>)uQeVdk%B{VGm#{yuYZ$o(i?ti+wnblQ@ObID@mefQwjOqZTW$605Kp zYp@o7qXcD$sg}!PQBFCI7>`QIRj7svYG~k8s}_cJYGH(CXb$-w`oapV#42RfvsduU zL?c?a)(AN)VU3RHgwE)KG8I3&M>%?M+!fu?6TQ#}eK80g&`@s-aV*ARJSJcgyfFnn z*oGAB!Xc#L7>?rv9-3;zV?4qae8ulJ8qu~b{UHv9?KC0+kw`+Fxkmg!J!BRdp#=d6 z4WJ7vTa75TV_rDWNh9WIcn^X<{9xNfBkbXV!5D(^m;i4~!E|h-?rLHz0w~`i-o{1cji5q{0-5FAP@38Oe56LKySE4=%X2$qcu#Cj{^KaA?ilZ$4GJn12jQX7{Uns zM=@TwW65ZZNI)WzARkYiTDhzlHcwzJ5P)EWBLb0#Lp&0Zgwr^Kvp9!KxQqvQh(~yg zXQ+nOMD{HtXhR1L&=7X$h-Fxg>66$Sn2A{!>8%l?pf`ni!Wgsu(g>UB+>0PYA^|OC zYlJIG=WF;HRU_su(TKfxj}K7c@8yhh6R(Ldjo6CqxF4eti{e>(e8OjZ!B>1o5!6_i zpb;w(i4@$&BRs|vJjFA#OJuL%uVnVcHueSj!UIZ#BLdrS1Zg;iS9lF|DtiDmknJFU zFx|=f6zX=<{~q2GWahF3rx+6!VKJ6qDf|(JUZ*vpH~OG2-d$qP!0NI_^guxdYm&*D z<1AiWVXs{!Cpj8%5i@UT#8oW(%-UcRGRnwrIeTBN5fjl-t44UBg#_&y;Q*rsHKH9} z!A-A5Y}c<5sn~)2IDkqtYE&Z{qr#v@)I-*!hOhN%gsN$c_=9@L4QqrFk%&SxPUA72 z;3=NrIbPu{ezvR;1+8ktzScFo2iNe~*cxGJStG14&bCHu!Q8GjVj~Lr)(FYIhR^8N z2<`qgBG0`>Ode7rW+G`=jo69>6Kg~~hU=QkCQq#q-Y}cS*sy9|jaZGH3u?q36s@We z<#@TehRm6M2)C{@wpn2j=T#s zV$r137fG60WfZAF8fPWBc@>nT5{Y9K9uLfgz_Td5-i0Eti&3$=9(XI9X4Pi+HlSk zn<#HaTgm~%Km;KKp$LN#;fO{o;*pH)*oECVf;1e%Q5?rf)S?c})iwN#wuYb8*6?%M z8sP*NxMMcvzz1PaB2`0fuoFEP+b-g6?8P4J$3FDr+5zH09KvB7K^l(Y7>?rvPT~|! z;|$K?9M0ncF5(g{BORH@z!hA>b=*KM@(^D`9uQMUUJ!?PBp?wLzK)+TzX;dp#H?9?HuoickGF~Ibf?xQJ zLijbS73uiWyjGlRQ7g_v+qhO3!V^O=48t+2Rjp9s8fKf;iWvOD@;0@?plz*qj7N9^ zqjt1!%e{8B;(o7MakF==2R<- z>BztpWFrT8c#LP5zmYt_Gno4^6vHqYJ~)TNQM3`meb^sM+qj6dq*`$odi!dHKK%ZR z2N*YA9IWMcqm1Dg>wmRY%)PQs8e;CptB+6GJf!y-e%GMr_7&yueGm z!r$<*;eIT{I;^y%PsqB|2`xOrV?4n#{ODUJt~k~SyMA?I1Sa7HUg8yA;~Uz#*NJv$ zj}9<{FZ^*7E<+fjN1fQ=StoX47tY~4CJ$w<&>B`Jwh!kzcHuZq;OnS5;W@faq+$mO z@B=@wc5CmNt3?u69|Qzc`Hpq*&$iD!(-bzEFHrs zkHAQb!f1@aSd7DXBx5VKAq9tV1lN&;Y~K}iE!LL8!amgQ{TlYsK{`n&;wCcsgM)hJ6CSxws6t1MtBGpVK!aMy`qztk zVjccKi(`RK)R&0bXbK}VLkk$AC0fA*tznAx=m0awVF@c(!v-DE37yg9zepcliG6V% zkMRW4Tfs}1kEj=O@Ek9YJ(4_+ zqHWYos27c9GCwp!=B#>g1u2W`#V+i|UhGE|s0m~S2iC~1F4CR;|%UCcMN05fk z@wB^@b}=WTUgTy{{}%n-t{0B^)WK04#|fOoBRs|v+`LmSZs9A2-mMqIFcvBI>&28u zwCQXvTi-+`ZNO%1K>*qq%A~evhx^TC(nGw$h!!&Gc1M}i&`l;8VTGHy%x0lXQot5= zXts!JSOJGsGRYC~t7X#f-7;z1L7BAG(_Gf%g-mJ+L(IWkY zj1W|z8pA7P($^Z9WL77WGU{d06#HSo#1Y(Z(2}x{gF%j3(mTAz2lVWx zCH2BQ%tsiMXxd*(GQ?mEK>z|_;G`wJcF~gF;4O~1YDwj=bk~w*Visnj1f|d&q9y6! z?MQRk`&smfkN5?rvPT~~Kz>n)^ ziRa)?c|FckzJQCkgv+>!Ysf`D?&1zgF>5vB#vIJWJj{m=7Gnw4VjX;ujC)90Lm6AK z4Lh(CyRa9BZ~_nU5qH;WNw@s9q}#ZIeB8x7yk1YvH)}~o*dL}PJ;PJ9i)D>rhK#GM zIgZ{WA2^Qiapp25A`povoZ$RPoWe`QaxM<9C>!0gIv79 z-vL5;2Ti1q4o@|g*(D39GhE;bH~8QLj^oxgA>Bqk+NKCev4g)NcS%ShT}aC^=`%}6 zpSw!Z^6rwP-CL4$&;SjwaE`f5m#BwEXpClXL_dsmkfd?g?JP<2TqG%amL%!Uraq40 z8vdFqNf+izQq($0`U(%V=`s)S3j#F#lgAW51dk~HF~Bn`PQNy>jDY2RXV z*|C2m={Qc}6rzgh2L`2*bO&`ZZK=7?mIgM^mZqRfOKqtedLY1DTgvLFE#2#(Ee+|d zEsY(dEsaA1Pi?8^N^{wKFKx*O3vdn>a2~TqXiFv2w57SL%w-c6X-g9^3Hpn*rN(H4 z{XyE2^8syXCD!L?OD4YNvXqb7(sra`2X-PIHlMU51l6M|$Jffc< zb6MIG9qB0UQGS5?c!)=Mj3+q8^{2$+#An14#FIFM=N!MlY09sNFL8$QS-hrvj(7nV zaS0hn#~ZF?65kT95U=8I%GZe3aRXV%Mh5i}!8_W1k8hN}qX0ipf>M;B92Ka78cm7-{DV3n?&V%FRzaCnxU zV}OlHd#_mwW#DKheXjspdz~~hS^C}q9es7qn6=Cgob1$SVL<0}opiHS1%Xqu z8uqTIN83DkGykQ$i=LdZv&_JHd-$4k{`m;XMP4wO>g8Y z?^+OYEvuC#aQgr8R9+_&{h*oljg|85iqM-`CZ<8NeH+Kgdw7K2acXTBG&j9*ioB;! z=!2}*jzRNPjrYoXDMOz)nR*2+us2AP_f89ak!9)~w8+=sjJ!{N=o_at3xk%V8>Gwo z7KFabYU3ZYOl6QIx7Q8(}yur)Sy+qO&-e~J;HuCwL2EHCcVjP zc|V`9!mM_gLF-gaKFa$m!%Cgn-wpD!Z(1N9kQP>%)&6bJdf%oz$1Oih?bP9C(8lzp zn^p2b1z~kr9W+6kRZV5)PP$4hXEXia0DD7Sb7zH8C)><4ILOz~(A>pCspl-W3l2#) zG%1))@d~m+9wX?;-;OKNC zd-EX$%68cn{=u;-BPVkY-Eg_HWkhhieKY=raz(gRwq+?(Gq|z`66Wkc;UpUz<4#?9 zo9J51R78bjcQXym^))fHnB@@_?%drj^k%w=iN$Q6sOaqOj-j_zCT13Mlu_}{J-kBi z*tb?#%uS0*&hFtIde66YH;Z}sQQMt+E)0E;-rC+`enHf(?4JIik5sLlEPQmM_c`~9 z2z_F2%Cl}1(TB2orG`H9H63NK&?EY&bMIrJFVamXSuFC2KAGJ+GxU|pbcV%ZW%OC+ zK6gXk*thYqSdtcfF}u&((7%1#EVo#iAD!Xc_h;z4^fta0%L<~eW%t#Deo(d9VzFE| zCfnIwKkSozTcyPcMa<1?d(*JbzHQ?yR(izTadxl^`rvuy4o1PtsyuWIK9?{qk*h#$s)L%p2!^3&RT2+ofBqD~NfQ-OoR)Sk*4e z!dEx;lXL%wuu}W>`4)bP*st0BQ^U%A+dsDO_lW)BJm6SZWqSM97VCXt3$q7ghE=QD zf3(=3j4gE@csERK-=V-_V_Ixw_Q1DcHNG7-mso7dk5xMl`WaT2-a%!txgfSKdypoq zUe!TnxkWcl%f(4wspVj%YZ;)3)5&o%RZ4zlhL(XIae6M!c1oQLGag{!6K9a)?5J$0 zHZ!vfR>m2*xOge`9OMeikhD1C92ak8BR_dJ%h3F|)-J9Kl?EAdd&{tbxOO?N{>r9m zxs#<*H(u`I7NIn9F!!(wSHxT8xTPwa`tK~;nVgm|KF7;DyrZAh8OyEt36ouhE)4ISVU=#Vtsr4)&QSmG zu4=0+%M{(jnJ&X3!n-?I=UZ-9B+ktlmKxsE&-$@tsz>4im*L04duLd`w%p;9xFlzI zW_Vw<^+(H{%ET2eBkqPfIM@_e?n+BslQZINct1az63gBBiGD64e})gpuu)m=DM;Ly zGg1>iNNppt+N+xs;4(@-!r4KgYqd|26p}N_G{V(SVQ97ABPrZvv|WUIhQh?^fKO6% z&S=MoA!>!0)j?%ayvrD`2u}xFh1H?7q~x42-VsCnY`a+<&QIFzGIn9a@C;jft0M(T zyK=_*M~qb4I$5RZChv0@7ZEYq!Op|#s3Q4L&bZWwv3_==td4mkA9WdjEMk0y-6X5y zKFKF@#%D%MRNKw4I-yKH>oVbP#AJt#K2|5wk}u{=cpEXruj6v7Q~AjmE)#!7OwH)% zYjwIH`C86IO~iC{$1PT8bhl=^{6BcYuFuv7Ig=eD=c_yIwYs3(`ozWCD{_HD=QOK} zXQesJH!f2aMlQ+doNjfwVC%b_DgKem)Sa`e(sj3ea``JFa)m>e ze5(w_wy!yVrADsu>+;wt(_`BYm#N1h*JN~gZFR+GTVc-B%*b`>E+4I~Dz}xoOuHND z=g@U?fz`FNZIwCG-bSwX>sn%UJ%5|pW%|#^jTv24RyPW^)#XgrL~d4hm04%$rf9j& z(2ojm=%#C(tw_#eL&vm9jH`7v#b7y%+Mf!E`W_>F^rM2tqg;CKN-R-S!7o@byo$VhLtM2Y(ov*uH z?m8zTD&C=ohxHxBcB|YusZoi3Jw{pI_1JFfI`>#qaz>9y*7tn2cg~%g8MRIQe?3$; zil;aDSU*VH-aB{R+o&CWJ(pWQ%-`!O9xCo_7dTfZtuotnGIKl-$~cb4^Q z-5oPs7e_>&b?B3C{YJ54ZtmjL=<|Mk9$UZl*s;KM$+75*8GT+`|LwD5N$!%&=*#Lp zAFcmU?pWcv^lo&9L*D}HcWFD;k7M=n;G^dHlKZVM(3_@ zjJd70H?#So+!^n>(kteUgM-56Yue7_+?C!j_xv2X*?i02x!rZu!k7mc4)!+R3wG|x zUF9G1=zr03ba!3MUHdlXZ@+%aZHn@DWw@^U8S^fqpRY}E!LDn$>ohSR)cv;Dl<4lx zcJzM2Bg_krtN-_>+c==%WuFLo2vZXZ(P?ej4jLZ_}E6{vFC^D#$&OS83SM2)cEWv%-xt7Tdf|r z`J+v(a!;x2rn|9fhd~84b!mGlb2q(>t??UFV)G||kJ@$f&)B+*K`NX2f<1M)n>DfZ z>OnlbNpG)~+ZO#eEk`F^g_iAJoxClkagx82p+b1>)pHB5i_^(;GEqni_8Q~`IL0;9 zIGHK5!}l7w1$xElIXWv8I!E^!=LLGlHS%}vrf6_yZ)>-pg>eR%&i0CiKlZlE3-XU^ zs&RHw=<4m0y9GzY89BOmDD-UiS>*+%#x?hM8Kuzo+-K_+axBg`(`AyP(Sm)Q^FlJ? zT4`KnC>n?F>+TkMH?Fm#tB=Cq=)T^0p>N~b_`5DwG`X|S!7c1(T)RwHUq#a&`v&BN zY2rF)T(>9;_4YfvDfQ#!j&4eYk?nr>Jf&&8g}+;zqM7G@Pq%Qpc&kje6h-p|`-kWK zPdqv|FVa80tHwP`VWM|nrdw1*e0Rse`HI%I2j=EQrN;O4AN*Kh>Um&+TlBH`-kF16 zE7~kLup}=!Grq58@JB`4@B=H{V(!K}I1VXLv^#oWO13i_zzuf+vU!o z47Y7R6Q*Vk^|kH#Xo>_aX3%4IePd-UaEKEBLCrM zZ2R0f{Kjp^!o($+!_#g1{y6+DZ-;;4GR^QTTYJ4DpWJpvB(88Ak#Fl@d*o~0&eX(J z{v#gSI(i=Y;kN5o;+o76n_t`ZTX3W>Z`c3%wA>@5ZoBU$`Z|l_+d)5$)aC8bByQG>l-W7yrD?hE)lUj=9HndLY@4QYbFXPq zkpCz{I~UJ1J@|x8D(bY7M^D0zTYb;+;OzR&i!bb@y-3- zNs<1eyV(uClh)e(z`~^H%+dCCLw=;SyLrGrDONMu$<9OXsNDTvL{hxt7!Ny7+oM)D z52hw1`i~i9=jD0S*8R}2q~y#olkA2rINJHv4Eg}VhJw+9|ZvmqT3JyLKf;MUlPS3gB_xta;-+%AKCm@_tJKB1Pr#_s@}VGMV8lcYsW(6p$FCb&#uV#di{RKlEFhy zs^5LGLhlv9=#(qlnKey|qD#FZMV-pQ_SrQDw?)Gf!uN4`c+QE)R zHHV%=H+n^vbw(;XH8mZJV%oi8Iyz$pJL78(Z;k2lT50W!Qy$K&IeIpx$7|L7&V<3k zTWj8X64U1u%Q&o49x>G%U$k<>D^7Gcb?`_-&55lmuXx3)4`(QkcGjFayK>TNb;jY1 zgGbNSoO!bHj@O#9!`aGXwwm`Bt@_$4q2qAg;IUgZ=eDkT=(W~*xIp>dgPIS{uKL;Q zjr)hU55D)L=A$R85bs3Bks{@bZ{m9EL`RATyJpu8Y>maelhjAbl*hel2hYY5-pLt9 z$_I}xsvUk3%kxetJ5s6a*3?=T#R#>{4 zbD4X-J)6+u{nq_spAMeey63wm34PxAjQ8#;drf=3U$l0_yFm2bmxH|xdme3FyW@)Y zR`q-Ll^=BO`RVN1N$x$IyOF9b>Uh z=u_O$Rc8l zr~8z591j>8x>Yx)FgeSo!g_pR=sZw{VL#gPXAY4oWs>y8YyY3fCb zQ`>!NI=W+qZ1MHpg{fUWwbt&q(9z6#-*c%wK6~zWCk%~lt$*cdYM)OXUzm2qr$K!pBlKcteZaZ2NuRwLCpHdUJX`FtYoGlcC-R0a-Kt+)xbC6P0qco^(8~|%mz-Pov(LNtPi!B${G|R*PuC&7jf|5; zp;wp<^2O4fjWjFaU|GOL+smTJy{<*p4p&1ml5Q9=>Ez5L*rW;UVoY)^KECG zY6|_>)DXEiQ{~$sI@LP#aYMt3!psQYPW7pd(CeKIG3PR4eGg}xIy!XyY{ROjnTftf z%1#{*yj>oqO}nZJP8w zmvQFXp-<25ee3D}+mGG;?EBIEGrtV|<;mV1Pv1lq^f7wSush6sMT@hr1?NRQ^x-?R z_w6jq!WZuDBm!G*FO$FR>e`^pw)3l|J_^f(QF z9>4GH!ff$^A#0CI*i`1e@^jfi3x@Ca%pIQEy6>H*|6AX;we724oRhxbV#oUd!(ZLn zx3@4SYr!S!`wPSFJ=nMZTu%Oi%lF@3GJNmJzIUJI=oegJoRx=t&D`I#IJb1cRnb}H z@Yl2VA1utZEV!mVs}8&GwZHXTZry_MjI-L|`-}D;dYap~;N!Bhkzo%s`#TorwJ*5d zaW-c7LHz#1g?U{IZdlL8g?*E`|LD2Ao&^*4&n66ivvvP_PxJZ~Oft^t!v1R7e|+(l zkp(wJ=Te9N+OYpb;g%~4ZmG{@gniq&|J1oHlM8NVoZC43?b-cjp8i++q;1eE_wM)Z zAAYp;z~!g^fBB;A;_dCP{M7N`_2DP+?|xIbz3Y{Kv3__n>}lq^-=5pv^UCA9;e@0Fh!AKeXmW_tJg#eW`owZ^+U-`%VkA4~c$CG#e z@pK0wMwor53UO#egY}pg_3ERKAafe2+w{1YqUmF(Q13>1ubvQ7GyB*h=;B8EXL_EP zX6kcPQI<6_f{TP=dS{>02xWDn!?q%^*v{7HqN1*EWcL;YiR~Zs%^jf@HqLxjBokxI z^L!QU?MBC7gG$WsI`1(;+uJy6n;}BX)SMTo=!YAfdJVB+hs^W7BlL5PFFi9Pidm-f z5|!O(qf7A4bTPa0e87m^?Z!FVc4moZ*v>Ch*?-eGw|8g0c;=&y0;?uBks# z#az}T3NCIJJ9YNQj4)R>d2cK35fkb=#G~^<%?M|8^RjJaVu`2iLcMC%`eto!S&(G@gA4mdW)(KS{;W(U5i$px zRNS|lBZEyUiO6fPb%eXOdBrwUgv3iT*r9Sd+#J(uij{b04*p;LZf(0HpmXT@$n4bz z*KaHDk_6g@ZmQ<2Ke(~Cd`FMu)dxeLj?5`M_~x_nK1mRB_^!(J?St9D6(f>`Uc+CG zxb8iev#sKaWRYh0zH08_gL%Cb|KEJ8c23LAZ5G^5p&4PQ`Q9zXy%xf6Y32yq%3s{F z>zRe;7h)Q5RJ$!}DGRO?`YAg{oUCrETi)JQDfSDsjku`Y*SD1SRtEWnJs6p5buVmr z=UJu9PsOzI)q=NMDub(3erhkPhgGn*W%stK2tSR+DpGqKZmH?5iuGHTY4x>woNL+h ztSZqj++>xgJx5#WgLkL_1RE>zF|re%Nc?tH)3A6S=I=Ra+E z_t|c}Uj)-8R|_3ln}VxL{UW_=N~>^A>%nc+7QZNsO|2Grx3>0H*ZHl;v}vuP#jS^) zRX6%Yn{1J4uVt+r!8PrEF`c#;tJmt*!`o`Q{8rj*acb}Ntw(!ndi+*Buq9Z%3tQiN zR@3Jf%N*6Ieco<89$Y)(7w0vaYW3OMdSYAc6~B1RXolMNaO?Zhhrx>qpP_ zApVKWu_CqDq3wKd9pepbix zPce;Es{NL=S%d3^{;8c~HCDgXZKK=j#r|owv3j-t`nHR`^+EpY9*pg``WLocepWB@ zPiJ0iQcK@%yBge}^3U+P*lLyTZ5!X#5aFMxx!9o&INWxJ- zeCXb@eU1LPrb}1Vi-Ny|tBlm8n zf6z!scV=E;+vJPe?SEp|rU3ijz&Ty4_)WlUQ0}yW*l*y8eIu z-C32i%b(mLk2P80ri zhh$0Hh_v2oJk=Jyx5IyX+ZAbpW;{cqJ=_uSLEEHsZ|3+$oAzACtG~3}k?u2%XKR*^ zb}U?S=xgcz&hb3k^4lGYw;y^aJzyIz(7gUl$C3{Y{VaX=!T5ID>rXrW^vfY6ppp4; zktV{SQ@*4f3uyBCxY!mkr*rA{c08b2^KqFb(z{dnK|2v}F!SSbTjb)-uwUAF0WH7# zUhh}6AHnx}DKEa)YnPLny(w?w^H23Y_=M`L|CH+4ZvGJU{OW!9uJ3>FTlwS9@lu@r ziiSJ6J^vOj%ArWu_&bD==QG5SHzOYDjZ!AIl2hbDz)^v!q=mh(jhZcy*!FegvhnZSSH+R)$-Uk zB9!u|aPYag;OF$ApAY#-VBE>h$&SRF3Y`)(5*mP;U4Cq`BQd!Ujna1@G)V~$t)^%& zI>Awh=w{s|xs`|XA+&w&A20m8eJREl=n)J}<0?i(t$N8y4B4lMCE8F6ffx}Ag;-wf zNbwp*E+~XWHI51_v-*edf;PP|T)u%7MIq8?h#iXHNhJ#t_dV{9qPX^zM8Xg6BZ#}v z?~40Zj@>NAcXNkoDQ?x`Y9iUJ96^vSKN$)y{Vo>`E=ihKa2d&(K^)Xa^%1SyzCJ=< z)7?)TtSds0WG>!MNv!sx5Ih#rN1!yMkH=@pK8AtG6cK!yQ$OQ~Feh9z*c&EOwKhiB z5PR1xlalu89F3@{I*GaXjyQ)Rwu(=W=~%RZNv~b?k#>k#aOrkrdX2Z@xG!*%6J|)s^A-FD8fi&DVz%~*U>-5_;os2#LNnq_VImi&2$aHmripW#|hldng2v#s9A_^iamE8RR?IRE$@eq<|mJ9(h z6gwi(2nZyypi3%H3}{Z?`XCA@4e{$MyZ}&GBhlSQF!*8&N#-d& zrXWTTutaE5s9=agF$7Ww-$$XsI4IRf!V@UlXx4>e9!AIhg}^Z^A!3YD2%Dg9hK>Z_ zq7KRXW|36!3&)CR5sGxa9!G!{)0O8Utd|s7RODf4ft}^CnUP9%;7*PbsE82mMubTS z5l|h;G(;gx)T4H?rKqDqh{6ZH{X7v5$n#YwFdX%6tcC{ph>YSQpaK0swSX?5$z%wx z+JN(mj5tQ<^m3V~3BiYm#%hW}Bmf-wtq5f;RQthP;DO@`g39&7gh1^+VXGck_B&r( zLCzS=CxSRL<&c8DR+03?s4SRzJ`K@hO-4k8_>y!lqTwpSy&4!x=4il?yDpEId}Hsr zF$y2#lW0PmoC$Zt&rV)y5%_g!V0CHsE=#Kg1f7I+q2}(o| z3L@0G@(>x2ODuDz5(*qtogbn| z4PH7j2Z(_2`<5iF1(`tR6?lS(Omij7Fs{hD91OiR86O@%Nm;~rG`IyWks$}oAT~{0 zL?8Dz$45*IHOfBoZz55fo&gY73Mp+0EK#2W;%Y_}3;&!n?8{;Y<)Gz6N_6SI`iCHE1+`HLk%xonQ=dE>M0oM}g^#5r7q0;6WqN zKtP<5jT9z@Z|pJv9L%y2F+^agj*rHuUo1%)Lz>KV1u+XXIzq|5S-^T`)GTSzQ~LbB zs!1FxWP;ih_PHVk4pS(vBYUC{ZfHPqHBNCb7X$bC8gWu>lamshJkR}|;N&4}`j&_O z_#+*K3MF%_V{ z&gH~PthE{@jIO871pcM;1L}Q%qYR8;G|N8Jf*A(t*;TrU*XC6)4V8j#&F6$OXayY$)PR^n$h)$tBfoQTER~lO`CboIp}KDfc%uGb#neODhLgL$6iZg!d$T<6&JXcrO-P|k?{2wpvV zK;Ni{_!0yF+js<`^Oa%qqHy$2$&e!kt++CFJz2(wxdy?MjZx`(d<`(;tMhn?uJB|T zBK-T}hV@YzQKNu67B@_j!DO*n|nwv2vU!IRTw^~XQ3NQRg?vqa&ZQXgO!n|!oO!CQaS za>I?|upLGUGFU{%Ty~nfR>o5G0Sck6{M5?T>ru`p$UAlG5DE22Kwn{EC3+(kIYRG` zUw`D(4JMV09FCkCq9h(YZWjv;06B~hp^z8PGj{iZdT~-94#qASDPjE-2cRbC2V4Qb zxHGX4CZXV~%Ex0^QShUYLQk4v#Q!ov(WA-w$G|wG08_Ca#{9onLhS=VSlLe`ACU~j zzU_;U)`Z_ArZb~`b0E#)?erO#T*%+|*sLBw#LU%W`(`1;kR|dn_b;lGRBqHpT?rqa zA49Q8M?5f5G7@28v?6?kaFaP~%s;tKGEz7Mr9`qZXNdU941-B7r}xEDkD#-&27-SW zDNL@DBU#km$Da|2C()t8=%>fe# zdSX5NOa$tPBDNz|C`8{mQjz@EJN&?CtedjKzN>$1haHrPhlmt-*yO+2Yu``M%;<=V z{XW!=_;z1ZsN|m64_b5`7z#?|J|1!ZuITf51z#-5ip_baH)^D?tLHJ3VMbRV`ETm+ zwSXy(@y(Tf3LVWXA1m~szMN(D3w22~`(4Ul;K)efcl(pafKJWPi8^{tIL6cG;M5^J zYm8S?v!bqm-y5MpJK%w}c$hduM&0K1skzUW zWoHANEdidEhL2EBV;w;K`=!kvuE~O&g2SC?(ODpstqe|pglup2l%kr%NgC(ND@Gk_)^!osm3X2x%BqZpG zjJP#aG!zPhAS_4dJ5d6m&WeYdXJ{w}k5D=dcZNF~>KN1p-8*mrt(gnpl~gl2D^Ut1 z>F~O%7@^za5s3Xv8Ra#E&c{vYMq~_y_h?YM1^Pmr5r(8ljJ^hMsQyTS`$h_4k78mm zppk@7m0X0;#aaV5=>CUF7zWA%EE%X#&w{bPY!`$$pp-i=L3_`s1=Y}bP)@=1K!KS_ z2pj$4y-At|Fo%39bv9t_`Q!E9^NW4f<~LLFl3y^)XZFq`_MXPez%WNHJ!hDIHlu&Y zJpw1;Z`{KPAw@Mi@6Fs)GLgGkmy-I&JVT!@Bk0qx|II1?bvR|@v)^+HYh!9oj&5D* z8@UBpsn7MxX?y2Jd(Xd40K2q9MCzA+Kbe2XBoAp$eY9|=<^{*JIq!=d=IY*WP-Fod|5 z&Rj)~Ma#J8LZpS=BZJl#xoZk{!vFEl?l|o3U{8{H{I1REl`yiL3C<7l!|WgywE8li zp@NzuE^DCU=|LTjmVwAT!<;_Bey3&$?6*P!`)$n5KST)sARDCVL)ydbVAz7>|5wqu zDgRXGBJ}@aN;mPJ>RkGbEb<}qPc<%`c-Z}1;o7|_<6#3b?og3{DaS8#`;2z`um;58 z<3X3H@=9rdc3i3*SA{Q)h;ULz3gTA8#7q>+VfXQ!sUYou^rnL5a4>ZaiVgHR*}21f z(#~^y_J{fOi&;B)Eb4IfCtW_v`ov?30JhTi@*|&7`zY3+I9%p~;4?1tvruJZ^-{Up zj}eolI-*2J+>qiBm*^&xuU+o<`64gj`qC@_8|~Hgx4U*Af!Mb zx5>O9U=Wfagg}MW@Sim1R1jd_0+|iL8^S^eP7pW{$R}6{gf$RWL104=K!8mIBm@Gi zxR7}eW zcCKT4{_PP+ZpB0LVf`N%?+>LDb7F-%QKn^+MhXwf!C>}R{X4TC8bZho3DU;@?`O?_ z+Q3);(}&Ifl7Ww2qavOk7We&bQ8@ECFC@RYDJ3H}meGNKv&N>at-2TH(Y|O|8u=Ia!pegC->f(= zk+5nore4C=yc;b;P?Xjf%@AT{$a+H%nf}5|YhVjy@4PS#_kfW^+Jpz#IO|m~15s3X zV#RZ-tr2S3`(*8ZVbtw~+gGr8HPMV%q>i11%T|CP2R%f~=vQc!(F~Pf_Y1RfPc)+o zgBx@vPMVHPvFG!ZI~Ffk!Z#wa@02;Ip&KY$Jk#ZhhYg?!M;OlT*0u=YNADi&;mn#@dpxR zk~#hLPyf!sn;s~bg9Q1cBLDQx*Qi45f6RuWV%`g>d(wiU^uL~oMX4XcPYWH$f}eIN zK~cuockaK!w(5u9z2ylxJP1iB{nFp7;b)DWeGJjBeH4`b>nhlReddEwf8K=Be|*|U z=66AJTc>VN;U|zR5IrQTeg*|U_CWLlj?zDWR|Yjd`qx!8bPT_bgJLPa!cP$W8UkfI z_5tWK1%ydRJV``J3-4DrM))-d!g7?p?N-SYxqoKeQ5KVSa!HI({i`1U0A*G39z z|Lfz+V!$H_ewc@glFMqiLH8%ZFn;|DV9EHkaK|GU`<=fBp$@O_Ri1|-X5p?3mf*I6jvr#I1eH%=?^D;z=Oc5MPBjPG~Q$2#$Xoeyb(>cA$2g>ogu6&X#N}U%{oHVSfT< zpmbryp@IYV3iMzx-BB7m^XDMJ<(}z%^lb;`r&AET&t5+p(eoNf)5PEj%$U9xA@