From 8b07a7f1807ef771b7b163b7728db215f9c7552a Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 27 Jul 2021 15:35:20 +0100 Subject: [PATCH 01/40] Store exception generated when we tried to update the queue last time The use case is to alert when queue contains broken entries. Especially important when ClickHouse breaks backwards compatibility between versions and log entries written by newer versions aren't parseable by old versions. ``` Code: 27, e.displayText() = DB::Exception: Cannot parse input: expected 'quorum: ' before: 'merge_type: 2\n' ``` --- .../ReplicatedMergeTreeRestartingThread.cpp | 17 +++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 31 ++++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 2 ++ src/Storages/System/StorageSystemReplicas.cpp | 2 ++ 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 1c9921aad1d..eadd414f1d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -153,11 +153,20 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() storage.cloneReplicaIfNeeded(zookeeper); - storage.queue.load(zookeeper); + try + { + storage.queue.load(zookeeper); + + /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); + /// because cleanup_thread doesn't delete log_pointer of active replicas. + storage.queue.pullLogsToQueue(zookeeper); + } + catch (...) + { + storage.last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); + throw; + } - /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); - /// because cleanup_thread doesn't delete log_pointer of active replicas. - storage.queue.pullLogsToQueue(zookeeper); storage.queue.removeCurrentPartsFromMutations(); storage.last_queue_update_finish_time.store(time(nullptr)); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d44b86fe9bb..8966a34e825 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3087,21 +3087,24 @@ void StorageReplicatedMergeTree::queueUpdatingTask() last_queue_update_finish_time.store(time(nullptr)); queue_update_in_progress = false; } - catch (const Coordination::Exception & e) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - - if (e.code == Coordination::Error::ZSESSIONEXPIRED) - { - restarting_thread.wakeup(); - return; - } - - queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); - } catch (...) { + last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); tryLogCurrentException(log, __PRETTY_FUNCTION__); + + try + { + throw; + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::Error::ZSESSIONEXPIRED) + { + restarting_thread.wakeup(); + return; + } + } + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); } } @@ -5562,6 +5565,10 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.total_replicas = 0; res.active_replicas = 0; + MultiVersion::Version queue_exception = last_queue_update_exception.get(); + if (queue_exception) + res.last_queue_update_exception = *queue_exception; + if (with_zk_fields && !res.is_session_expired) { try diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 800f419cb76..9c3b9b12e37 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -174,6 +174,7 @@ public: UInt64 absolute_delay; UInt8 total_replicas; UInt8 active_replicas; + String last_queue_update_exception; /// If the error has happened fetching the info from ZooKeeper, this field will be set. String zookeeper_exception; }; @@ -329,6 +330,7 @@ private: ReplicatedMergeTreeQueue queue; std::atomic last_queue_update_start_time{0}; std::atomic last_queue_update_finish_time{0}; + MultiVersion last_queue_update_exception; DataPartsExchange::Fetcher fetcher; diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index fc33c6b421b..3af7352616f 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -50,6 +50,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_) { "absolute_delay", std::make_shared() }, { "total_replicas", std::make_shared() }, { "active_replicas", std::make_shared() }, + { "last_queue_update_exception", std::make_shared() }, { "zookeeper_exception", std::make_shared() }, })); setInMemoryMetadata(storage_metadata); @@ -183,6 +184,7 @@ Pipe StorageSystemReplicas::read( res_columns[col_num++]->insert(status.absolute_delay); res_columns[col_num++]->insert(status.total_replicas); res_columns[col_num++]->insert(status.active_replicas); + res_columns[col_num++]->insert(status.last_queue_update_exception); res_columns[col_num++]->insert(status.zookeeper_exception); } From 3f291b024a315e6afa3401ebbc5b52fb49e0e3be Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 9 Aug 2021 13:58:23 +0100 Subject: [PATCH 02/40] Use plain mutex instead of MultiVersion --- .../ReplicatedMergeTreeRestartingThread.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 39 +++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 5 ++- 3 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index eadd414f1d5..edd0876c6e9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -163,7 +163,8 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() } catch (...) { - storage.last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); + std::unique_lock lock(storage.last_queue_update_exception_lock); + storage.last_queue_update_exception = getCurrentExceptionMessage(false); throw; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8966a34e825..a8b6d4170d9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3073,6 +3073,12 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke zookeeper->set(fs::path(replica_path) / "is_lost", "0"); } +String StorageReplicatedMergeTree::getLastQueueUpdateException() const +{ + std::unique_lock lock(last_queue_update_exception_lock); + return last_queue_update_exception; +} + void StorageReplicatedMergeTree::queueUpdatingTask() { @@ -3087,24 +3093,28 @@ void StorageReplicatedMergeTree::queueUpdatingTask() last_queue_update_finish_time.store(time(nullptr)); queue_update_in_progress = false; } - catch (...) + catch (const Coordination::Exception & e) { - last_queue_update_exception.set(std::make_unique(getCurrentExceptionMessage(false))); tryLogCurrentException(log, __PRETTY_FUNCTION__); - try + std::unique_lock lock(last_queue_update_exception_lock); + last_queue_update_exception = getCurrentExceptionMessage(false); + + if (e.code == Coordination::Error::ZSESSIONEXPIRED) { - throw; - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::Error::ZSESSIONEXPIRED) - { - restarting_thread.wakeup(); - return; - } + restarting_thread.wakeup(); + return; } + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + std::unique_lock lock(last_queue_update_exception_lock); + last_queue_update_exception = getCurrentExceptionMessage(false); + queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS); } } @@ -5564,10 +5574,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.log_pointer = 0; res.total_replicas = 0; res.active_replicas = 0; - - MultiVersion::Version queue_exception = last_queue_update_exception.get(); - if (queue_exception) - res.last_queue_update_exception = *queue_exception; + res.last_queue_update_exception = getLastQueueUpdateException(); if (with_zk_fields && !res.is_session_expired) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9c3b9b12e37..4741d8b4605 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -330,7 +330,10 @@ private: ReplicatedMergeTreeQueue queue; std::atomic last_queue_update_start_time{0}; std::atomic last_queue_update_finish_time{0}; - MultiVersion last_queue_update_exception; + + mutable std::mutex last_queue_update_exception_lock; + String last_queue_update_exception; + String getLastQueueUpdateException() const; DataPartsExchange::Fetcher fetcher; From b9d8ee125b52bc9f41862e888072d874b5002bd5 Mon Sep 17 00:00:00 2001 From: jasine Date: Sun, 15 Aug 2021 22:26:40 +0800 Subject: [PATCH 03/40] feat: add conversion between snowflake id and datetime --- src/Functions/FunctionSnowflake.h | 208 +++++++++++++++++++ src/Functions/dateTime64ToSnowflake.cpp | 14 ++ src/Functions/dateTimeToSnowflake.cpp | 14 ++ src/Functions/registerFunctions.cpp | 2 + src/Functions/registerFunctionsSnowflake.cpp | 22 ++ src/Functions/snowflakeToDateTime.cpp | 14 ++ src/Functions/snowflakeToDateTime64.cpp | 14 ++ 7 files changed, 288 insertions(+) create mode 100644 src/Functions/FunctionSnowflake.h create mode 100644 src/Functions/dateTime64ToSnowflake.cpp create mode 100644 src/Functions/dateTimeToSnowflake.cpp create mode 100644 src/Functions/registerFunctionsSnowflake.cpp create mode 100644 src/Functions/snowflakeToDateTime.cpp create mode 100644 src/Functions/snowflakeToDateTime64.cpp diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h new file mode 100644 index 00000000000..cf3a91b8e69 --- /dev/null +++ b/src/Functions/FunctionSnowflake.h @@ -0,0 +1,208 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +class FunctionDateTimeToSnowflake : public IFunction +{ +private: + const char * name; +public: + FunctionDateTimeToSnowflake( const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isVariadic() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateTime(arguments[0].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime", name); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnInt64::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (int64_t(source_data[i])*1000-1288834974657)<<22; + } + + return res_column; + } +}; + + +class FunctionSnowflakeToDateTime : public IFunction +{ +private: + const char * name; +public: + FunctionSnowflakeToDateTime(const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + + if (arguments.size() < 1 || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + + if (!typeid_cast(arguments[0].type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); + + std::string timezone; + if (arguments.size() == 2) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnUInt32::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; + } + + return res_column; + } +}; + + +class FunctionDateTime64ToSnowflake : public IFunction +{ +private: + const char * name; +public: + FunctionDateTime64ToSnowflake( const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isVariadic() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateTime64(arguments[0].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The only argument for function {} must be DateTime64", name); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnInt64::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast &>(col).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (source_data[i]-1288834974657)<<22; + } + + return res_column; + } +}; + + +class FunctionSnowflakeToDateTime64 : public IFunction +{ +private: + const char * name; +public: + FunctionSnowflakeToDateTime64(const char * name_) + : name(name_) + { + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + + if (arguments.size() < 1 || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); + + if (!typeid_cast(arguments[0].type.get())) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The first argument for function {} must be Int64", name); + + std::string timezone; + if (arguments.size() == 2) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); + + return std::make_shared(3, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & src = arguments[0]; + const auto & col = *src.column; + + auto res_column = ColumnDecimal::create(input_rows_count, 3); + auto & result_data = res_column->getData(); + + const auto & source_data = typeid_cast(col).getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + { + result_data[i] = (source_data[i]>>22)+1288834974657; + } + + return res_column; + } +}; + +} diff --git a/src/Functions/dateTime64ToSnowflake.cpp b/src/Functions/dateTime64ToSnowflake.cpp new file mode 100644 index 00000000000..87e35c25371 --- /dev/null +++ b/src/Functions/dateTime64ToSnowflake.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerDateTime64ToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTime64ToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTime64ToSnowflake")); }); +} + +} diff --git a/src/Functions/dateTimeToSnowflake.cpp b/src/Functions/dateTimeToSnowflake.cpp new file mode 100644 index 00000000000..246f35cc1dc --- /dev/null +++ b/src/Functions/dateTimeToSnowflake.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerDateTimeToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTimeToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTimeToSnowflake")); }); +} + +} diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 7e8f35bc0c4..9b1a7faebbe 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -51,6 +51,7 @@ void registerFunctionBitHammingDistance(FunctionFactory & factory); void registerFunctionTupleHammingDistance(FunctionFactory & factory); void registerFunctionsStringHash(FunctionFactory & factory); void registerFunctionValidateNestedArraySizes(FunctionFactory & factory); +void registerFunctionsSnowflake(FunctionFactory & factory); #if !defined(ARCADIA_BUILD) void registerFunctionBayesAB(FunctionFactory &); #endif @@ -115,6 +116,7 @@ void registerFunctions() registerFunctionTupleHammingDistance(factory); registerFunctionsStringHash(factory); registerFunctionValidateNestedArraySizes(factory); + registerFunctionsSnowflake(factory); #if !defined(ARCADIA_BUILD) registerFunctionBayesAB(factory); diff --git a/src/Functions/registerFunctionsSnowflake.cpp b/src/Functions/registerFunctionsSnowflake.cpp new file mode 100644 index 00000000000..f0c2feddfb5 --- /dev/null +++ b/src/Functions/registerFunctionsSnowflake.cpp @@ -0,0 +1,22 @@ +namespace DB +{ + +class FunctionFactory; + +void registerDateTimeToSnowflake(FunctionFactory &); +void registerSnowflakeToDateTime(FunctionFactory &); + +void registerDateTime64ToSnowflake(FunctionFactory &); +void registerSnowflakeToDateTime64(FunctionFactory &); + + +void registerFunctionsSnowflake(FunctionFactory & factory) +{ + registerDateTimeToSnowflake(factory); + registerSnowflakeToDateTime(factory); + + registerDateTime64ToSnowflake(factory); + registerSnowflakeToDateTime64(factory); +} + +} diff --git a/src/Functions/snowflakeToDateTime.cpp b/src/Functions/snowflakeToDateTime.cpp new file mode 100644 index 00000000000..37f5e07512f --- /dev/null +++ b/src/Functions/snowflakeToDateTime.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerSnowflakeToDateTime(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime")); }); +} + +} diff --git a/src/Functions/snowflakeToDateTime64.cpp b/src/Functions/snowflakeToDateTime64.cpp new file mode 100644 index 00000000000..ef9502a224e --- /dev/null +++ b/src/Functions/snowflakeToDateTime64.cpp @@ -0,0 +1,14 @@ +#include +#include + +namespace DB +{ + +void registerSnowflakeToDateTime64(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime64", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime64")); }); +} + +} From aa49f76bf0184fb2e4abe9487df7f7c88f8344d4 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 00:49:33 +0800 Subject: [PATCH 04/40] fix: style --- src/Functions/FunctionSnowflake.h | 11 +++++------ src/Functions/registerFunctionsSnowflake.cpp | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index cf3a91b8e69..3dd378e4956 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -25,7 +25,7 @@ class FunctionDateTimeToSnowflake : public IFunction private: const char * name; public: - FunctionDateTimeToSnowflake( const char * name_) + FunctionDateTimeToSnowflake(const char * name_) : name(name_) { } @@ -79,7 +79,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -107,7 +106,7 @@ public: { result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; } - + return res_column; } }; @@ -118,7 +117,7 @@ class FunctionDateTime64ToSnowflake : public IFunction private: const char * name; public: - FunctionDateTime64ToSnowflake( const char * name_) + FunctionDateTime64ToSnowflake(const char * name_) : name(name_) { } @@ -172,7 +171,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - + if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -200,7 +199,7 @@ public: { result_data[i] = (source_data[i]>>22)+1288834974657; } - + return res_column; } }; diff --git a/src/Functions/registerFunctionsSnowflake.cpp b/src/Functions/registerFunctionsSnowflake.cpp index f0c2feddfb5..7a0569ee16a 100644 --- a/src/Functions/registerFunctionsSnowflake.cpp +++ b/src/Functions/registerFunctionsSnowflake.cpp @@ -14,7 +14,7 @@ void registerFunctionsSnowflake(FunctionFactory & factory) { registerDateTimeToSnowflake(factory); registerSnowflakeToDateTime(factory); - + registerDateTime64ToSnowflake(factory); registerSnowflakeToDateTime64(factory); } From c414a3aebf8b819b11dcbf0f541e0d35f9973753 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 17:24:51 +0800 Subject: [PATCH 05/40] feat: add docs and tests --- .../functions/type-conversion-functions.md | 144 ++++++++++++++++++ .../01942_dateTimeToSnowflake.reference | 6 + .../0_stateless/01942_dateTimeToSnowflake.sql | 23 +++ .../01942_snowflakeToDateTime.reference | 3 + .../0_stateless/01942_snowflakeToDateTime.sql | 32 ++++ 5 files changed, 208 insertions(+) create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflake.reference create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflake.sql create mode 100644 tests/queries/0_stateless/01942_snowflakeToDateTime.reference create mode 100644 tests/queries/0_stateless/01942_snowflakeToDateTime.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index efd28def688..5a733f6be23 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1339,3 +1339,147 @@ Result: │ 2,"good" │ └───────────────────────────────────────────┘ ``` + +## snowflakeToDateTime {#snowflakeToDateTime} + +extract time from snowflake id as DateTime format. + +**Syntax** + +``` sql +snowflakeToDateTime(value [, time_zone]) +``` + +**Parameters** + +- `value` — `snowflake id`, Int64 value. +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- value converted to the `DateTime` data type. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC'); +``` + +Result: + +``` text + +┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:57:56 │ +└──────────────────────────────────────────────────────────────────┘ +``` + +## snowflakeToDateTime64 {#snowflakeToDateTime64} + +extract time from snowflake id as DateTime64 format. + +**Syntax** + +``` sql +snowflakeToDateTime64(value [, time_zone]) +``` + +**Parameters** + +- `value` — `snowflake id`, Int64 value. +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- value converted to the `DateTime64` data type. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC'); +``` + +Result: + +``` text + +┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:58:19.841 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +## dateTimeToSnowflake {#dateTimeToSnowflake} + +convert DateTime to the first snowflake id at the giving time. + +**Syntax** + +``` sql +dateTimeToSnowflake(value) +``` + +**Parameters** + +- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md). + + +**Returned value** + +- `value` converted to the `Int64` data type as the first snowflake id at that time. + +**Example** + +Query: + +``` sql +SELECT dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime')); +``` + +Result: + +``` text + +┌─dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime'))─┐ +│ 1426860702823350272 │ +└──────────────────────────────────────────────────────────────┘ +``` + + +## dateTime64ToSnowflake {#dateTime64ToSnowflake} + +convert DateTime64 to the first snowflake id at the giving time. + +**Syntax** + +``` sql +dateTime64ToSnowflake(value) +``` + +**Parameters** + +- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). + + +**Returned value** + +- `value` converted to the `Int64` data type as the first snowflake id at that time. + +**Example** + +Query: + +``` sql +SELECT dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64')); +``` + +Result: + +``` text +┌─dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64'))─┐ +│ 1426860703129534464 │ +└──────────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference new file mode 100644 index 00000000000..dfca3a10eeb --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.reference @@ -0,0 +1,6 @@ +const column +2021-08-15 18:57:56 1426860702823350272 +2021-08-15 18:57:56.492 1426860704886947840 +non-const column +2021-08-15 18:57:56 1426860702823350272 +2021-08-15 18:57:56.492 1426860704886947840 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql new file mode 100644 index 00000000000..e5895db7004 --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -0,0 +1,23 @@ +-- Error cases +SELECT dateTimeToSnowflake(); -- {serverError 42} +SELECT dateTime64ToSnowflake(); -- {serverError 42} + +SELECT dateTimeToSnowflake('abc'); -- {serverError 43} +SELECT dateTime64ToSnowflake('abc'); -- {serverError 43} + +SELECT dateTimeToSnowflake('abc', 123); -- {serverError 42} +SELECT dateTime64ToSnowflake('abc', 123); -- {serverError 42} + +SELECT 'const column'; +WITH toDateTime('2021-08-15 18:57:56') AS dt +SELECT dt, dateTimeToSnowflake(dt); + +WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64 +SELECT dt64, dateTime64ToSnowflake(dt64); + +SELECT 'non-const column'; +WITH toDateTime('2021-08-15 18:57:56') AS x +SELECT materialize(x) as dt, dateTimeToSnowflake(dt);; + +WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS x +SELECT materialize(x) as dt64, dateTime64ToSnowflake(dt64); diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference new file mode 100644 index 00000000000..bed18023f6a --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.reference @@ -0,0 +1,3 @@ +const column +UTC 1426860704886947840 2021-08-15 10:57:56 DateTime(\'UTC\') 2021-08-15 10:57:56.492 DateTime64(3, \'UTC\') +Asia/Shanghai 1426860704886947840 2021-08-15 18:57:56 DateTime(\'Asia/Shanghai\') 2021-08-15 18:57:56.492 DateTime64(3, \'Asia/Shanghai\') diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql new file mode 100644 index 00000000000..f6f171afabf --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -0,0 +1,32 @@ +-- -- Error cases +SELECT snowflakeToDateTime(); -- {serverError 42} +SELECT snowflakeToDateTime64(); -- {serverError 42} + +SELECT snowflakeToDateTime('abc'); -- {serverError 43} +SELECT snowflakeToDateTime64('abc'); -- {serverError 43} + +SELECT snowflakeToDateTime('abc', 123); -- {serverError 43} +SELECT snowflakeToDateTime64('abc', 123); -- {serverError 43} + +SELECT 'const column'; +WITH + CAST(1426860704886947840 AS Int64) AS i64, + 'UTC' AS tz +SELECT + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); + +WITH + CAST(1426860704886947840 AS Int64) AS i64, + 'Asia/Shanghai' AS tz +SELECT + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); \ No newline at end of file From 1f21131db680c392e4daeacb47a3ec02b162ef86 Mon Sep 17 00:00:00 2001 From: jasine Date: Mon, 16 Aug 2021 18:52:10 +0800 Subject: [PATCH 06/40] fix: doc and test --- .../functions/type-conversion-functions.md | 18 ++++++++++-------- .../0_stateless/01942_dateTimeToSnowflake.sql | 8 ++++---- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 5a733f6be23..4f1a2d49d23 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1436,16 +1436,17 @@ dateTimeToSnowflake(value) Query: ``` sql -SELECT dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime')); +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt +SELECT dateTimeToSnowflake(dt); ``` Result: ``` text -┌─dateTimeToSnowflake(CAST('2021-08-15 18:57:56', 'DateTime'))─┐ -│ 1426860702823350272 │ -└──────────────────────────────────────────────────────────────┘ +┌─dateTimeToSnowflake(dt)─┐ +│ 1426860702823350272 │ +└─────────────────────────┘ ``` @@ -1473,13 +1474,14 @@ dateTime64ToSnowflake(value) Query: ``` sql -SELECT dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64')); +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 +SELECT dateTime64ToSnowflake(dt64); ``` Result: ``` text -┌─dateTime64ToSnowflake(CAST('2021-08-15 18:57:56.073', 'DateTime64'))─┐ -│ 1426860703129534464 │ -└──────────────────────────────────────────────────────────────────────┘ +┌─dateTime64ToSnowflake(dt64)─┐ +│ 1426860704886947840 │ +└─────────────────────────────┘ ``` \ No newline at end of file diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index e5895db7004..047d8be7be5 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -9,15 +9,15 @@ SELECT dateTimeToSnowflake('abc', 123); -- {serverError 42} SELECT dateTime64ToSnowflake('abc', 123); -- {serverError 42} SELECT 'const column'; -WITH toDateTime('2021-08-15 18:57:56') AS dt +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dt, dateTimeToSnowflake(dt); -WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64 +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dt64, dateTime64ToSnowflake(dt64); SELECT 'non-const column'; -WITH toDateTime('2021-08-15 18:57:56') AS x +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS x SELECT materialize(x) as dt, dateTimeToSnowflake(dt);; -WITH toDateTime64('2021-08-15 18:57:56.492', 3) AS x +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS x SELECT materialize(x) as dt64, dateTime64ToSnowflake(dt64); From d25d12d7d4ef4c9ff9ed74984d87cc054c836ed7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Aug 2021 12:30:02 +0000 Subject: [PATCH 07/40] better --- src/Compression/LZ4_decompress_faster.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 6972457f11b..72a611e0f43 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,11 +439,14 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while (unlikely(s == 255)); + } while (ip < input_end && unlikely(s == 255)); }; /// Get literal length. + if (unlikely(ip >= input_end)) + return false; + const unsigned token = *ip++; length = token >> 4; if (length == 0x0F) @@ -475,7 +478,7 @@ bool NO_INLINE decompressImpl( ip += length; op = copy_end; - if (unlikely(ip > input_end)) + if (unlikely(ip + 1 >= input_end)) return false; /// Get match offset. From 60dd53784f54af466596df7a1db5190cdef05f8d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Aug 2021 12:46:59 +0000 Subject: [PATCH 08/40] better --- src/Compression/LZ4_decompress_faster.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 72a611e0f43..21a2cc01a12 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -467,13 +467,13 @@ bool NO_INLINE decompressImpl( /// output: xyzHello, w /// ^-op (we will overwrite excessive bytes on next iteration) - { - auto * target = std::min(copy_end, output_end); - wildCopy(op, ip, target); /// Here we can write up to copy_amount - 1 bytes after buffer. + if (unlikely(copy_end > output_end)) + return false; - if (target == output_end) - return true; - } + wildCopy(op, ip, copy_end); /// Here we can write up to copy_amount - 1 bytes after buffer. + + if (copy_end == output_end) + return true; ip += length; op = copy_end; @@ -531,8 +531,9 @@ bool NO_INLINE decompressImpl( copy(op, match); /// copy_amount + copy_amount - 1 - 4 * 2 bytes after buffer. if (length > copy_amount * 2) { - auto * target = std::min(copy_end, output_end); - wildCopy(op + copy_amount, match + copy_amount, target); + if (unlikely(copy_end > output_end)) + return false; + wildCopy(op + copy_amount, match + copy_amount, copy_end); } op = copy_end; From afa748c717a4d38503dddeee10d5e7c95916f36b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Aug 2021 17:15:11 +0300 Subject: [PATCH 09/40] Refactor NotJoined pt1 --- src/Interpreters/HashJoin.cpp | 67 +++++++------------ src/Interpreters/MergeJoin.cpp | 67 +++++-------------- src/Interpreters/MergeJoin.h | 1 + src/Interpreters/TableJoin.cpp | 18 +++++ src/Interpreters/TableJoin.h | 1 + src/Interpreters/join_common.cpp | 58 ++++++---------- src/Interpreters/join_common.h | 60 ++++++++++++----- .../Transforms/JoiningTransform.cpp | 1 - 8 files changed, 125 insertions(+), 148 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index e0aad706966..4130431485e 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1468,40 +1468,23 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. -class NonJoinedBlockInputStream : private NotJoined, public IBlockInputStream +class NonJoinedBlockInputStream final : public NotJoined { public: - NonJoinedBlockInputStream(const HashJoin & parent_, const Block & result_sample_block_, UInt64 max_block_size_) - : NotJoined(*parent_.table_join, - parent_.savedBlockSample(), - parent_.right_sample_block, - result_sample_block_) + NonJoinedBlockInputStream( + const HashJoin & parent_, + const Block & result_sample_block_, + size_t left_columns_count, + UInt64 max_block_size_) + : NotJoined(parent_.savedBlockSample(), result_sample_block_, + left_columns_count, parent_.table_join->leftToRightKeyRemap()) , parent(parent_) , max_block_size(max_block_size_) {} - String getName() const override { return "NonJoined"; } - Block getHeader() const override { return result_sample_block; } - protected: - Block readImpl() override + size_t fillColumns(MutableColumns & columns_right) override { - if (parent.data->blocks.empty()) - return Block(); - return createBlock(); - } - -private: - const HashJoin & parent; - UInt64 max_block_size; - - std::any position; - std::optional nulls_position; - - Block createBlock() - { - MutableColumns columns_right = saved_block_sample.cloneEmptyColumns(); - size_t rows_added = 0; auto fill_callback = [&](auto, auto strictness, auto & map) @@ -1513,22 +1496,16 @@ private: throw Exception("Logical error: unknown JOIN strictness (must be on of: ANY, ALL, ASOF)", ErrorCodes::LOGICAL_ERROR); fillNullsFromBlocks(columns_right, rows_added); - if (!rows_added) - return {}; - - Block res = result_sample_block.cloneEmpty(); - addLeftColumns(res, rows_added); - addRightColumns(res, columns_right); - copySameKeys(res); - correctLowcardAndNullability(res); - -#ifndef NDEBUG - assertBlocksHaveEqualStructure(res, result_sample_block, getName()); -#endif - - return res; + return rows_added; } +private: + const HashJoin & parent; + UInt64 max_block_size; + + std::any position; + std::optional nulls_position; + template size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right) { @@ -1610,12 +1587,14 @@ private: BlockInputStreamPtr HashJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::Asof || - table_join->strictness() == ASTTableJoin::Strictness::Semi) + table_join->strictness() == ASTTableJoin::Strictness::Semi || + !isRightOrFull(table_join->kind())) + { return {}; + } - if (isRightOrFull(table_join->kind())) - return std::make_shared(*this, result_sample_block, max_block_size); - return {}; + size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); + return std::make_shared(*this, result_sample_block, left_columns_count, max_block_size); } void HashJoin::reuseJoinedData(const HashJoin & join) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index a2c63a4693b..84d5a80cff5 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1,7 +1,8 @@ #include #include -#include +#include + #include #include #include @@ -723,15 +724,7 @@ void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) if (needConditionJoinColumn()) block.erase(deriveTempName(mask_column_name_left)); - for (const auto & column_name : lowcard_keys) - { - if (!block.has(column_name)) - continue; - if (auto & col = block.getByName(column_name); !col.type->lowCardinality()) - JoinCommon::changeLowCardinalityInplace(col); - } - - JoinCommon::restoreLowCardinalityInplace(block); + JoinCommon::restoreLowCardinalityInplace(block, lowcard_keys); } template @@ -1035,55 +1028,25 @@ void MergeJoin::initRightTableWriter() } /// Stream from not joined earlier rows of the right table. -class NonMergeJoinedBlockInputStream : private NotJoined, public IBlockInputStream +class NonMergeJoinedBlockInputStream final : public NotJoined { public: NonMergeJoinedBlockInputStream(const MergeJoin & parent_, - const Block & result_sample_block_, - const Names & key_names_right_, + const Block & result_sample_block, + size_t left_columns_count, UInt64 max_block_size_) - : NotJoined(*parent_.table_join, - parent_.modifyRightBlock(parent_.right_sample_block), - parent_.right_sample_block, - result_sample_block_, - {}, key_names_right_) + : NotJoined(parent_.modifyRightBlock(parent_.right_sample_block), + result_sample_block, + left_columns_count, + parent_.table_join->leftToRightKeyRemap()) , parent(parent_) , max_block_size(max_block_size_) {} String getName() const override { return "NonMergeJoined"; } - Block getHeader() const override { return result_sample_block; } protected: - Block readImpl() override - { - if (parent.getRightBlocksCount()) - return createBlock(); - return {}; - } - -private: - const MergeJoin & parent; - size_t max_block_size; - size_t block_number = 0; - - Block createBlock() - { - MutableColumns columns_right = saved_block_sample.cloneEmptyColumns(); - - size_t rows_added = fillColumns(columns_right); - if (!rows_added) - return {}; - - Block res = result_sample_block.cloneEmpty(); - addLeftColumns(res, rows_added); - addRightColumns(res, columns_right); - copySameKeys(res); - correctLowcardAndNullability(res); - return res; - } - - size_t fillColumns(MutableColumns & columns_right) + size_t fillColumns(MutableColumns & columns_right) override { const RowBitmaps & bitmaps = *parent.used_rows_bitmap; size_t rows_added = 0; @@ -1127,13 +1090,19 @@ private: return rows_added; } + +private: + const MergeJoin & parent; + size_t max_block_size; + size_t block_number = 0; }; BlockInputStreamPtr MergeJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const { + size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) - return std::make_shared(*this, result_sample_block, key_names_right, max_block_size); + return std::make_shared(*this, result_sample_block, left_columns_count, max_block_size); return {}; } diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 844c730de4f..4aa26ead0a0 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -78,6 +78,7 @@ private: SortDescription right_merge_description; Block right_sample_block; Block right_table_keys; + /// Columns from right side of join, both key and additional Block right_columns_to_add; SortedBlocksWriter::Blocks right_blocks; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 20e8f6b18b4..68195dd9483 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -472,6 +472,24 @@ void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) on_filter_condition_asts_right.push_back(ast); } +std::unordered_map TableJoin::leftToRightKeyRemap() const +{ + std::unordered_map left_to_right_key_remap; + if (hasUsing()) + { + const auto & required_right_keys = requiredRightKeys(); + for (size_t i = 0; i < key_names_left.size(); ++i) + { + const String & left_key_name = key_names_left[i]; + const String & right_key_name = key_names_right[i]; + + if (!required_right_keys.contains(right_key_name)) + left_to_right_key_remap[left_key_name] = right_key_name; + } + } + return left_to_right_key_remap; +} + /// Returns all conditions related to one table joined with 'and' function static ASTPtr buildJoinConditionColumn(const ASTs & on_filter_condition_asts) { diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4c8c16028f5..f0f8c3696b3 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -230,6 +230,7 @@ public: Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; String renamedRightColumnName(const String & name) const; + std::unordered_map leftToRightKeyRemap() const; }; } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index e9f3e4f3fdd..2c6a2731a0e 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -314,8 +314,16 @@ void removeLowCardinalityInplace(Block & block, const Names & names, bool change } } -void restoreLowCardinalityInplace(Block & block) +void restoreLowCardinalityInplace(Block & block, const Names & lowcard_keys) { + for (const auto & column_name : lowcard_keys) + { + if (!block.has(column_name)) + continue; + if (auto & col = block.getByName(column_name); !col.type->lowCardinality()) + JoinCommon::changeLowCardinalityInplace(col); + } + for (size_t i = 0; i < block.columns(); ++i) { auto & col = block.getByPosition(i); @@ -484,49 +492,21 @@ void splitAdditionalColumns(const Names & key_names, const Block & sample_block, } - -NotJoined::NotJoined(const TableJoin & table_join, const Block & saved_block_sample_, const Block & right_sample_block, - const Block & result_sample_block_, const Names & key_names_left_, const Names & key_names_right_) +NotJoined::NotJoined(const Block & saved_block_sample_, + const Block & result_sample_block_, + size_t left_columns_count, + const LeftToRightKeyRemap & left_to_right_key_remap) : saved_block_sample(saved_block_sample_) , result_sample_block(materializeBlock(result_sample_block_)) - , key_names_left(key_names_left_.empty() ? table_join.keyNamesLeft() : key_names_left_) - , key_names_right(key_names_right_.empty() ? table_join.keyNamesRight() : key_names_right_) { - std::vector tmp; - Block right_table_keys; - Block sample_block_with_columns_to_add; - - JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, - sample_block_with_columns_to_add); - Block required_right_keys = table_join.getRequiredRightKeys(right_table_keys, tmp); - - std::unordered_map left_to_right_key_remap; - - if (table_join.hasUsing()) - { - for (size_t i = 0; i < key_names_left.size(); ++i) - { - const String & left_key_name = key_names_left[i]; - const String & right_key_name = key_names_right[i]; - - size_t left_key_pos = result_sample_block.getPositionByName(left_key_name); - size_t right_key_pos = saved_block_sample.getPositionByName(right_key_name); - - if (!required_right_keys.has(right_key_name)) - left_to_right_key_remap[left_key_pos] = right_key_pos; - } - } - - /// result_sample_block: left_sample_block + left expressions, right not key columns, required right keys - size_t left_columns_count = result_sample_block.columns() - - sample_block_with_columns_to_add.columns() - required_right_keys.columns(); - for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) { - /// We need right 'x' for 'RIGHT JOIN ... USING(x)'. - if (left_to_right_key_remap.count(left_pos)) + /// We need right 'x' for 'RIGHT JOIN ... USING(x)' + auto left_name = result_sample_block.getByPosition(left_pos).name; + const auto & right_key = left_to_right_key_remap.find(left_name); + if (right_key != left_to_right_key_remap.end()) { - size_t right_key_pos = left_to_right_key_remap[left_pos]; + size_t right_key_pos = saved_block_sample.getPositionByName(right_key->second); setRightIndex(right_key_pos, left_pos); } else @@ -558,7 +538,7 @@ NotJoined::NotJoined(const TableJoin & table_join, const Block & saved_block_sam void NotJoined::setRightIndex(size_t right_pos, size_t result_position) { - if (!column_indices_right.count(right_pos)) + if (!column_indices_right.contains(right_pos)) { column_indices_right[right_pos] = result_position; extractColumnChanges(right_pos, result_position); diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 1f037899155..f61e110627b 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -30,7 +31,7 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); ColumnRawPtrs getRawPointers(const Columns & columns); void removeLowCardinalityInplace(Block & block); void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true); -void restoreLowCardinalityInplace(Block & block); +void restoreLowCardinalityInplace(Block & block, const Names & lowcard_keys); ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_names_right); @@ -64,40 +65,69 @@ void changeLowCardinalityInplace(ColumnWithTypeAndName & column); } /// Creates result from right table data in RIGHT and FULL JOIN when keys are not present in left table. -class NotJoined +class NotJoined : public IBlockInputStream { public: - NotJoined(const TableJoin & table_join, const Block & saved_block_sample_, const Block & right_sample_block, - const Block & result_sample_block_, const Names & key_names_left_ = {}, const Names & key_names_right_ = {}); + using LeftToRightKeyRemap = std::unordered_map; + NotJoined(const Block & saved_block_sample_, + const Block & result_sample_block_, + size_t left_columns_count, + const LeftToRightKeyRemap & left_to_right_key_remap); + + String getName() const override { return "NonJoined"; } + Block getHeader() const override { return result_sample_block; } + +protected: + Block readImpl() override final + { + Block result = saved_block_sample.cloneEmpty(); + MutableColumns columns_right = result.mutateColumns(); + + size_t rows_added = fillColumns(columns_right); + if (rows_added == 0) + return {}; + + Block res = result_sample_block.cloneEmpty(); + addLeftColumns(res, rows_added); + addRightColumns(res, columns_right); + copySameKeys(res); + correctLowcardAndNullability(res); + +#ifndef NDEBUG + assertBlocksHaveEqualStructure(res, result_sample_block, getName()); +#endif + return res; + } + + virtual size_t fillColumns(MutableColumns & columns_right) = 0; + +private: + void extractColumnChanges(size_t right_pos, size_t result_pos); void correctLowcardAndNullability(Block & block); void addLeftColumns(Block & block, size_t rows_added) const; void addRightColumns(Block & block, MutableColumns & columns_right) const; void copySameKeys(Block & block) const; -protected: + /// Right block saved in Join Block saved_block_sample; + + /// Output of join Block result_sample_block; - Names key_names_left; - Names key_names_right; - - ~NotJoined() = default; - -private: /// Indices of columns in result_sample_block that should be generated std::vector column_indices_left; /// Indices of columns that come from the right-side table: right_pos -> result_pos std::unordered_map column_indices_right; - /// + std::unordered_map same_result_keys; - /// Which right columns (saved in parent) need nullability change before placing them in result block + + /// Which right columns (saved in parent) need Nullability/LowCardinality change + /// before placing them in result block std::vector> right_nullability_changes; - /// Which right columns (saved in parent) need LowCardinality change before placing them in result block std::vector> right_lowcard_changes; void setRightIndex(size_t right_pos, size_t result_position); - void extractColumnChanges(size_t right_pos, size_t result_pos); }; } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index e402fd788bc..c1329d02fed 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include From 9d8178d04c6321ad301ee82ead42106a2bb928f9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 9 Aug 2021 17:30:37 +0300 Subject: [PATCH 10/40] Refactor NotJoined pt2: rename classes, get rig of inheritance --- src/Interpreters/HashJoin.cpp | 21 +++++++--------- src/Interpreters/HashJoin.h | 2 +- src/Interpreters/MergeJoin.cpp | 24 +++++++------------ src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/join_common.cpp | 37 +++++++++++++++++++++------- src/Interpreters/join_common.h | 41 +++++++++++++------------------- 6 files changed, 65 insertions(+), 62 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 4130431485e..4384072377d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -629,7 +629,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - /// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream + /// If RIGHT or FULL save blocks with nulls for NotJoinedInputStream UInt8 save_nullmap = 0; if (isRightOrFull(kind) && null_map) { @@ -1468,21 +1468,15 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. -class NonJoinedBlockInputStream final : public NotJoined +class NotJoinedHash final : public NotJoinedInputStream::RightColumnsFiller { public: - NonJoinedBlockInputStream( - const HashJoin & parent_, - const Block & result_sample_block_, - size_t left_columns_count, - UInt64 max_block_size_) - : NotJoined(parent_.savedBlockSample(), result_sample_block_, - left_columns_count, parent_.table_join->leftToRightKeyRemap()) - , parent(parent_) - , max_block_size(max_block_size_) + NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_) + : parent(parent_), max_block_size(max_block_size_) {} -protected: + Block getEmptyBlock() override { return parent.savedBlockSample().cloneEmpty(); } + size_t fillColumns(MutableColumns & columns_right) override { size_t rows_added = 0; @@ -1594,7 +1588,8 @@ BlockInputStreamPtr HashJoin::createStreamWithNonJoinedRows(const Block & result } size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); - return std::make_shared(*this, result_sample_block, left_columns_count, max_block_size); + auto non_joined = std::make_unique(*this, max_block_size); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } void HashJoin::reuseJoinedData(const HashJoin & join) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 65e3f5dbabe..f6e47b59d25 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -337,7 +337,7 @@ public: bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); } private: - friend class NonJoinedBlockInputStream; + friend class NotJoinedHash; friend class JoinSource; std::shared_ptr table_join; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 84d5a80cff5..611f1742fa4 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1028,24 +1028,15 @@ void MergeJoin::initRightTableWriter() } /// Stream from not joined earlier rows of the right table. -class NonMergeJoinedBlockInputStream final : public NotJoined +class NotJoinedMerge final : public NotJoinedInputStream::RightColumnsFiller { public: - NonMergeJoinedBlockInputStream(const MergeJoin & parent_, - const Block & result_sample_block, - size_t left_columns_count, - UInt64 max_block_size_) - : NotJoined(parent_.modifyRightBlock(parent_.right_sample_block), - result_sample_block, - left_columns_count, - parent_.table_join->leftToRightKeyRemap()) - , parent(parent_) - , max_block_size(max_block_size_) + NotJoinedMerge(const MergeJoin & parent_, UInt64 max_block_size_) + : parent(parent_), max_block_size(max_block_size_) {} - String getName() const override { return "NonMergeJoined"; } + Block getEmptyBlock() override { return parent.modifyRightBlock(parent.right_sample_block).cloneEmpty(); } -protected: size_t fillColumns(MutableColumns & columns_right) override { const RowBitmaps & bitmaps = *parent.used_rows_bitmap; @@ -1100,9 +1091,12 @@ private: BlockInputStreamPtr MergeJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const { - size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) - return std::make_shared(*this, result_sample_block, left_columns_count, max_block_size); + { + size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); + auto non_joined = std::make_unique(*this, max_block_size); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + } return {}; } diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 4aa26ead0a0..89371d8b13b 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -38,7 +38,7 @@ public: BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const override; private: - friend class NonMergeJoinedBlockInputStream; + friend class NotJoinedMerge; struct NotProcessed : public ExtraBlock { diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 2c6a2731a0e..c640fea3a36 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -492,11 +492,12 @@ void splitAdditionalColumns(const Names & key_names, const Block & sample_block, } -NotJoined::NotJoined(const Block & saved_block_sample_, +NotJoinedInputStream::NotJoinedInputStream(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, const LeftToRightKeyRemap & left_to_right_key_remap) - : saved_block_sample(saved_block_sample_) + : filler(std::move(filler_)) + , saved_block_sample(filler->getEmptyBlock()) , result_sample_block(materializeBlock(result_sample_block_)) { for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) @@ -536,7 +537,7 @@ NotJoined::NotJoined(const Block & saved_block_sample_, ErrorCodes::LOGICAL_ERROR); } -void NotJoined::setRightIndex(size_t right_pos, size_t result_position) +void NotJoinedInputStream::setRightIndex(size_t right_pos, size_t result_position) { if (!column_indices_right.contains(right_pos)) { @@ -547,7 +548,7 @@ void NotJoined::setRightIndex(size_t right_pos, size_t result_position) same_result_keys[result_position] = column_indices_right[right_pos]; } -void NotJoined::extractColumnChanges(size_t right_pos, size_t result_pos) +void NotJoinedInputStream::extractColumnChanges(size_t right_pos, size_t result_pos) { auto src_props = getLowcardAndNullability(saved_block_sample.getByPosition(right_pos).column); auto dst_props = getLowcardAndNullability(result_sample_block.getByPosition(result_pos).column); @@ -559,7 +560,7 @@ void NotJoined::extractColumnChanges(size_t right_pos, size_t result_pos) right_lowcard_changes.push_back({result_pos, dst_props.is_lowcard}); } -void NotJoined::correctLowcardAndNullability(Block & block) +void NotJoinedInputStream::correctLowcardAndNullability(Block & block) { for (auto & [pos, added] : right_nullability_changes) { @@ -587,7 +588,7 @@ void NotJoined::correctLowcardAndNullability(Block & block) } } -void NotJoined::addLeftColumns(Block & block, size_t rows_added) const +void NotJoinedInputStream::addLeftColumns(Block & block, size_t rows_added) const { for (size_t pos : column_indices_left) { @@ -599,7 +600,7 @@ void NotJoined::addLeftColumns(Block & block, size_t rows_added) const } } -void NotJoined::addRightColumns(Block & block, MutableColumns & columns_right) const +void NotJoinedInputStream::addRightColumns(Block & block, MutableColumns & columns_right) const { for (const auto & pr : column_indices_right) { @@ -609,7 +610,7 @@ void NotJoined::addRightColumns(Block & block, MutableColumns & columns_right) c } } -void NotJoined::copySameKeys(Block & block) const +void NotJoinedInputStream::copySameKeys(Block & block) const { for (const auto & pr : same_result_keys) { @@ -619,4 +620,24 @@ void NotJoined::copySameKeys(Block & block) const } } +Block NotJoinedInputStream::readImpl() + +{ + Block right_block = filler->getEmptyBlock(); + MutableColumns columns_right = right_block.cloneEmptyColumns(); + size_t rows_added = filler->fillColumns(columns_right); + if (rows_added == 0) + return {}; + + addLeftColumns(right_block, rows_added); + addRightColumns(right_block, columns_right); + copySameKeys(right_block); + correctLowcardAndNullability(right_block); + +#ifndef NDEBUG + assertBlocksHaveEqualStructure(right_block, result_sample_block, getName()); +#endif + return right_block; +} + } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index f61e110627b..32fa4a4ee9e 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -65,12 +65,24 @@ void changeLowCardinalityInplace(ColumnWithTypeAndName & column); } /// Creates result from right table data in RIGHT and FULL JOIN when keys are not present in left table. -class NotJoined : public IBlockInputStream +class NotJoinedInputStream : public IBlockInputStream { public: using LeftToRightKeyRemap = std::unordered_map; - NotJoined(const Block & saved_block_sample_, + /// Returns non joined columns from right part of join + class RightColumnsFiller + { + public: + /// Create empty block for right part + virtual Block getEmptyBlock() = 0; + /// Fill columns from right part of join with not joined rows + virtual size_t fillColumns(MutableColumns & columns_right) = 0; + + virtual ~RightColumnsFiller() = default; + }; + + NotJoinedInputStream(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, const LeftToRightKeyRemap & left_to_right_key_remap); @@ -79,28 +91,7 @@ public: Block getHeader() const override { return result_sample_block; } protected: - Block readImpl() override final - { - Block result = saved_block_sample.cloneEmpty(); - MutableColumns columns_right = result.mutateColumns(); - - size_t rows_added = fillColumns(columns_right); - if (rows_added == 0) - return {}; - - Block res = result_sample_block.cloneEmpty(); - addLeftColumns(res, rows_added); - addRightColumns(res, columns_right); - copySameKeys(res); - correctLowcardAndNullability(res); - -#ifndef NDEBUG - assertBlocksHaveEqualStructure(res, result_sample_block, getName()); -#endif - return res; - } - - virtual size_t fillColumns(MutableColumns & columns_right) = 0; + Block readImpl() override final; private: void extractColumnChanges(size_t right_pos, size_t result_pos); @@ -109,6 +100,8 @@ private: void addRightColumns(Block & block, MutableColumns & columns_right) const; void copySameKeys(Block & block) const; + std::unique_ptr filler; + /// Right block saved in Join Block saved_block_sample; From 2dfbbe364b357699e12888093540e1b6431a8e7a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 17 Aug 2021 16:30:01 +0300 Subject: [PATCH 11/40] Do not use BlockInputStream for NonJoined --- src/Interpreters/HashJoin.cpp | 9 ++--- src/Interpreters/HashJoin.h | 3 +- src/Interpreters/IJoin.h | 4 +- src/Interpreters/JoinSwitcher.h | 38 +------------------ src/Interpreters/MergeJoin.cpp | 8 ++-- src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/join_common.cpp | 18 ++++----- src/Interpreters/join_common.h | 11 ++---- .../Transforms/JoiningTransform.cpp | 8 ++-- src/Processors/Transforms/JoiningTransform.h | 5 +-- 10 files changed, 32 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 4384072377d..6abaddd6270 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -21,7 +21,6 @@ #include -#include #include #include @@ -629,7 +628,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - /// If RIGHT or FULL save blocks with nulls for NotJoinedInputStream + /// If RIGHT or FULL save blocks with nulls for NotJoinedBlocks UInt8 save_nullmap = 0; if (isRightOrFull(kind) && null_map) { @@ -1468,7 +1467,7 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. -class NotJoinedHash final : public NotJoinedInputStream::RightColumnsFiller +class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller { public: NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_) @@ -1578,7 +1577,7 @@ private: }; -BlockInputStreamPtr HashJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const +std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::Asof || table_join->strictness() == ASTTableJoin::Strictness::Semi || @@ -1589,7 +1588,7 @@ BlockInputStreamPtr HashJoin::createStreamWithNonJoinedRows(const Block & result size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); auto non_joined = std::make_unique(*this, max_block_size); - return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } void HashJoin::reuseJoinedData(const HashJoin & join) diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index f6e47b59d25..2e691f189c4 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -20,7 +20,6 @@ #include #include -#include #include @@ -164,7 +163,7 @@ public: * Use only after all calls to joinBlock was done. * left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside). */ - BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const override; + std::shared_ptr getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; /// Number of keys in all built JOIN maps. size_t getTotalRowCount() const final; diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 8fa85de4951..2215402e1d4 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -5,7 +5,6 @@ #include #include -#include namespace DB { @@ -15,6 +14,7 @@ struct ExtraBlock; using ExtraBlockPtr = std::shared_ptr; class TableJoin; +class NotJoinedBlocks; class IJoin { @@ -43,7 +43,7 @@ public: /// Different query plan is used for such joins. virtual bool isFilled() const { return false; } - virtual BlockInputStreamPtr createStreamWithNonJoinedRows(const Block &, UInt64) const { return {}; } + virtual std::shared_ptr getNonJoinedBlocks(const Block &, UInt64) const = 0; }; using JoinPtr = std::shared_ptr; diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index a89ac6d5d98..e750bc5eed0 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -56,9 +56,9 @@ public: return join->alwaysReturnsEmptySet(); } - BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & block, UInt64 max_block_size) const override + std::shared_ptr getNonJoinedBlocks(const Block & block, UInt64 max_block_size) const override { - return join->createStreamWithNonJoinedRows(block, max_block_size); + return join->getNonJoinedBlocks(block, max_block_size); } private: @@ -74,38 +74,4 @@ private: void switchJoin(); }; - -/// Creates NonJoinedBlockInputStream on the first read. Allows to swap join algo before it. -class LazyNonJoinedBlockInputStream : public IBlockInputStream -{ -public: - LazyNonJoinedBlockInputStream(const IJoin & join_, const Block & block, UInt64 max_block_size_) - : join(join_) - , result_sample_block(block) - , max_block_size(max_block_size_) - {} - - String getName() const override { return "LazyNonMergeJoined"; } - Block getHeader() const override { return result_sample_block; } - -protected: - Block readImpl() override - { - if (!stream) - { - stream = join.createStreamWithNonJoinedRows(result_sample_block, max_block_size); - if (!stream) - return {}; - } - - return stream->read(); - } - -private: - BlockInputStreamPtr stream; - const IJoin & join; - Block result_sample_block; - UInt64 max_block_size; -}; - } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 611f1742fa4..0150bbe1d93 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1028,7 +1028,7 @@ void MergeJoin::initRightTableWriter() } /// Stream from not joined earlier rows of the right table. -class NotJoinedMerge final : public NotJoinedInputStream::RightColumnsFiller +class NotJoinedMerge final : public NotJoinedBlocks::RightColumnsFiller { public: NotJoinedMerge(const MergeJoin & parent_, UInt64 max_block_size_) @@ -1089,15 +1089,15 @@ private: }; -BlockInputStreamPtr MergeJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const +std::shared_ptr MergeJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) { size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); auto non_joined = std::make_unique(*this, max_block_size); - return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } - return {}; + return nullptr; } bool MergeJoin::needConditionJoinColumn() const diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 89371d8b13b..9e765041846 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -35,7 +35,7 @@ public: /// Has to be called only after setTotals()/mergeRightBlocks() bool alwaysReturnsEmptySet() const override { return (is_right || is_inner) && min_max_right_blocks.empty(); } - BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const override; + std::shared_ptr getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; private: friend class NotJoinedMerge; diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index c640fea3a36..349ba56e74a 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -492,7 +492,7 @@ void splitAdditionalColumns(const Names & key_names, const Block & sample_block, } -NotJoinedInputStream::NotJoinedInputStream(std::unique_ptr filler_, +NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, const LeftToRightKeyRemap & left_to_right_key_remap) @@ -537,7 +537,7 @@ NotJoinedInputStream::NotJoinedInputStream(std::unique_ptr f ErrorCodes::LOGICAL_ERROR); } -void NotJoinedInputStream::setRightIndex(size_t right_pos, size_t result_position) +void NotJoinedBlocks::setRightIndex(size_t right_pos, size_t result_position) { if (!column_indices_right.contains(right_pos)) { @@ -548,7 +548,7 @@ void NotJoinedInputStream::setRightIndex(size_t right_pos, size_t result_positio same_result_keys[result_position] = column_indices_right[right_pos]; } -void NotJoinedInputStream::extractColumnChanges(size_t right_pos, size_t result_pos) +void NotJoinedBlocks::extractColumnChanges(size_t right_pos, size_t result_pos) { auto src_props = getLowcardAndNullability(saved_block_sample.getByPosition(right_pos).column); auto dst_props = getLowcardAndNullability(result_sample_block.getByPosition(result_pos).column); @@ -560,7 +560,7 @@ void NotJoinedInputStream::extractColumnChanges(size_t right_pos, size_t result_ right_lowcard_changes.push_back({result_pos, dst_props.is_lowcard}); } -void NotJoinedInputStream::correctLowcardAndNullability(Block & block) +void NotJoinedBlocks::correctLowcardAndNullability(Block & block) { for (auto & [pos, added] : right_nullability_changes) { @@ -588,7 +588,7 @@ void NotJoinedInputStream::correctLowcardAndNullability(Block & block) } } -void NotJoinedInputStream::addLeftColumns(Block & block, size_t rows_added) const +void NotJoinedBlocks::addLeftColumns(Block & block, size_t rows_added) const { for (size_t pos : column_indices_left) { @@ -600,7 +600,7 @@ void NotJoinedInputStream::addLeftColumns(Block & block, size_t rows_added) cons } } -void NotJoinedInputStream::addRightColumns(Block & block, MutableColumns & columns_right) const +void NotJoinedBlocks::addRightColumns(Block & block, MutableColumns & columns_right) const { for (const auto & pr : column_indices_right) { @@ -610,7 +610,7 @@ void NotJoinedInputStream::addRightColumns(Block & block, MutableColumns & colum } } -void NotJoinedInputStream::copySameKeys(Block & block) const +void NotJoinedBlocks::copySameKeys(Block & block) const { for (const auto & pr : same_result_keys) { @@ -620,7 +620,7 @@ void NotJoinedInputStream::copySameKeys(Block & block) const } } -Block NotJoinedInputStream::readImpl() +Block NotJoinedBlocks::read() { Block right_block = filler->getEmptyBlock(); @@ -635,7 +635,7 @@ Block NotJoinedInputStream::readImpl() correctLowcardAndNullability(right_block); #ifndef NDEBUG - assertBlocksHaveEqualStructure(right_block, result_sample_block, getName()); + assertBlocksHaveEqualStructure(right_block, result_sample_block, "NotJoinedBlocks"); #endif return right_block; } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 32fa4a4ee9e..ec2e1d3bd50 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -65,7 +64,7 @@ void changeLowCardinalityInplace(ColumnWithTypeAndName & column); } /// Creates result from right table data in RIGHT and FULL JOIN when keys are not present in left table. -class NotJoinedInputStream : public IBlockInputStream +class NotJoinedBlocks final { public: using LeftToRightKeyRemap = std::unordered_map; @@ -82,16 +81,12 @@ public: virtual ~RightColumnsFiller() = default; }; - NotJoinedInputStream(std::unique_ptr filler_, + NotJoinedBlocks(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, const LeftToRightKeyRemap & left_to_right_key_remap); - String getName() const override { return "NonJoined"; } - Block getHeader() const override { return result_sample_block; } - -protected: - Block readImpl() override final; + Block read(); private: void extractColumnChanges(size_t right_pos, size_t result_pos); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index c1329d02fed..95a12e2291d 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -113,7 +113,7 @@ void JoiningTransform::work() } else { - if (!non_joined_stream) + if (!non_joined_blocks) { if (!finish_counter || !finish_counter->isLast()) { @@ -121,15 +121,15 @@ void JoiningTransform::work() return; } - non_joined_stream = join->createStreamWithNonJoinedRows(outputs.front().getHeader(), max_block_size); - if (!non_joined_stream) + non_joined_blocks = join->getNonJoinedBlocks(outputs.front().getHeader(), max_block_size); + if (!non_joined_blocks) { process_non_joined = false; return; } } - auto block = non_joined_stream->read(); + Block block = non_joined_blocks->read(); if (!block) { process_non_joined = false; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 98038946f3b..96c4032dabc 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -8,8 +8,7 @@ namespace DB class IJoin; using JoinPtr = std::shared_ptr; -class IBlockInputStream; -using BlockInputStreamPtr = std::shared_ptr; +class NotJoinedBlocks; /// Join rows to chunk form left table. /// This transform usually has two input ports and one output. @@ -76,7 +75,7 @@ private: ExtraBlockPtr not_processed; FinishCounterPtr finish_counter; - BlockInputStreamPtr non_joined_stream; + std::shared_ptr non_joined_blocks; size_t max_block_size; Block readExecute(Chunk & chunk); From ca67bf0e5a4e54ade19765a76f5c7aceaa4acb69 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 18 Aug 2021 11:18:33 +0300 Subject: [PATCH 12/40] Fix NotJoinedBlocks::read, add logging --- src/Interpreters/HashJoin.cpp | 7 +++++++ src/Interpreters/join_common.cpp | 29 +++++++++++++++++------------ 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 6abaddd6270..46f8c9aac79 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -193,6 +193,13 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); + LOG_DEBUG(log, "Right keys: [{}] (required: [{}]), left keys: [{}]", + fmt::join(key_names_right, ", "), + fmt::join(required_right_keys.getNames(), ", "), + fmt::join(table_join->keyNamesLeft(), ", ")); + + LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); + std::tie(condition_mask_column_name_left, condition_mask_column_name_right) = table_join->joinConditionColumnNames(); JoinCommon::removeLowCardinalityInplace(right_table_keys); diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 349ba56e74a..b230d8d1957 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -500,6 +500,9 @@ NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, , saved_block_sample(filler->getEmptyBlock()) , result_sample_block(materializeBlock(result_sample_block_)) { + LOG_DEBUG(&Poco::Logger::get("NotJoinedBlocks"), "saved_block_sample {}",saved_block_sample.dumpStructure()); + LOG_DEBUG(&Poco::Logger::get("NotJoinedBlocks"), "result_sample_block {}",result_sample_block.dumpStructure()); + for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) { /// We need right 'x' for 'RIGHT JOIN ... USING(x)' @@ -621,23 +624,25 @@ void NotJoinedBlocks::copySameKeys(Block & block) const } Block NotJoinedBlocks::read() - { - Block right_block = filler->getEmptyBlock(); - MutableColumns columns_right = right_block.cloneEmptyColumns(); - size_t rows_added = filler->fillColumns(columns_right); - if (rows_added == 0) - return {}; + Block result_block = result_sample_block.cloneEmpty(); + { + Block right_block = filler->getEmptyBlock(); + MutableColumns columns_right = right_block.cloneEmptyColumns(); + size_t rows_added = filler->fillColumns(columns_right); + if (rows_added == 0) + return {}; - addLeftColumns(right_block, rows_added); - addRightColumns(right_block, columns_right); - copySameKeys(right_block); - correctLowcardAndNullability(right_block); + addLeftColumns(result_block, rows_added); + addRightColumns(result_block, columns_right); + } + copySameKeys(result_block); + correctLowcardAndNullability(result_block); #ifndef NDEBUG - assertBlocksHaveEqualStructure(right_block, result_sample_block, "NotJoinedBlocks"); + assertBlocksHaveEqualStructure(result_block, result_sample_block, "NotJoinedBlocks"); #endif - return right_block; + return result_block; } } From 941cbc43f358562bbf7067d0e49050a6d0410399 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Aug 2021 11:28:22 +0000 Subject: [PATCH 13/40] Fix --- src/Common/filesystemHelpers.cpp | 19 +++++++++++++++++++ src/Common/filesystemHelpers.h | 2 ++ src/Dictionaries/LibraryDictionarySource.cpp | 9 ++++++--- 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 4855500b776..730099f4476 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int CANNOT_STATVFS; extern const int PATH_ACCESS_DENIED; extern const int CANNOT_CREATE_FILE; + extern const int BAD_ARGUMENTS; } @@ -122,6 +123,17 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p return path_starts_with_prefix_path; } +bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path) +{ + auto absolute_path = std::filesystem::absolute(path); + auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path); + + auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end()); + + bool path_starts_with_prefix_path = (prefix_path_mismatch_it == absolute_prefix_path.end()); + return path_starts_with_prefix_path; +} + bool pathStartsWith(const String & path, const String & prefix_path) { auto filesystem_path = std::filesystem::path(path); @@ -130,6 +142,13 @@ bool pathStartsWith(const String & path, const String & prefix_path) return pathStartsWith(filesystem_path, filesystem_prefix_path); } +bool symlinkStartsWith(const String & path, const String & prefix_path) +{ + auto filesystem_path = std::filesystem::path(path); + auto filesystem_prefix_path = std::filesystem::path(prefix_path); + + return symlinkStartsWith(filesystem_path, filesystem_prefix_path); +} } diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index b7525a64fae..71ef7844ef7 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -35,6 +35,8 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p /// Returns true if path starts with prefix path bool pathStartsWith(const String & path, const String & prefix_path); +bool symlinkStartsWith(const String & path, const String & prefix_path); + } namespace FS diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 288abde8788..f2c5cefa543 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -41,10 +41,13 @@ LibraryDictionarySource::LibraryDictionarySource( , sample_block{sample_block_} , context(Context::createCopy(context_)) { - if (fs::path(path).is_relative()) - path = fs::canonical(path); + bool path_checked = false; + if (fs::is_symlink(path)) + path_checked = symlinkStartsWith(path, context->getDictionariesLibPath()); + else + path_checked = pathStartsWith(path, context->getDictionariesLibPath()); - if (created_from_ddl && !pathStartsWith(path, context->getDictionariesLibPath())) + if (created_from_ddl && !path_checked) throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, context->getDictionariesLibPath()); if (!fs::exists(path)) From c819880aa786c95b90cdd9664c72c3fb0c94878c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 18 Aug 2021 16:34:57 +0300 Subject: [PATCH 14/40] Add test for clickhouse-keeper start after conversion --- docs/en/operations/clickhouse-keeper.md | 2 +- tests/integration/helpers/cluster.py | 5 + .../configs/keeper_config1.xml | 37 ++++++ .../configs/keeper_config2.xml | 37 ++++++ .../configs/keeper_config3.xml | 37 ++++++ .../configs/logs_conf.xml | 12 ++ .../test.py | 120 ++++++++++++++++++ 7 files changed, 249 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml create mode 100644 tests/integration/test_keeper_snapshot_small_distance/test.py diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 5fc1baa003c..389cf2361f8 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -114,5 +114,5 @@ Seamlessly migration from ZooKeeper to `clickhouse-keeper` is impossible you hav clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 --zookeeper-snapshots-dir /var/lib/zookeeper/version-2 --output-dir /path/to/clickhouse/keeper/snapshots ``` -4. Copy snapshot to `clickhouse-server` nodes with configured `keeper` or start `clickhouse-keeper` instead of ZooKeeper. Snapshot must persist only on leader node, leader will sync it automatically to other nodes. +4. Copy snapshot to `clickhouse-server` nodes with configured `keeper` or start `clickhouse-keeper` instead of ZooKeeper. Snapshot must persist on all nodes, otherwise empty nodes can be faster and one of them can becamse leader. diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6fe01b5df03..11af6ec6348 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -474,6 +474,11 @@ class ClickHouseCluster: cmd += " client" return cmd + def copy_file_from_container_to_container(self, src_node, src_path, dst_node, dst_path): + fname = os.path.basename(src_path) + run_and_check([f"docker cp {src_node.docker_id}:{src_path} {self.instances_dir}"], shell=True) + run_and_check([f"docker cp {self.instances_dir}/{fname} {dst_node.docker_id}:{dst_path}"], shell=True) + def setup_zookeeper_secure_cmd(self, instance, env_variables, docker_compose_yml_dir): logging.debug('Setup ZooKeeper Secure') zookeeper_docker_compose_path = p.join(docker_compose_yml_dir, 'docker_compose_zookeeper_secure.yml') diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml new file mode 100644 index 00000000000..b3c010fed24 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml new file mode 100644 index 00000000000..f2de2c96f06 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml new file mode 100644 index 00000000000..609c9045259 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/keeper_config3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 75 + 5 + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + 3 + + + 2 + node2 + 44444 + 2 + + + 3 + node3 + 44444 + 1 + + + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml b/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/configs/logs_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_keeper_snapshot_small_distance/test.py b/tests/integration/test_keeper_snapshot_small_distance/test.py new file mode 100644 index 00000000000..4acd76806b4 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_small_distance/test.py @@ -0,0 +1,120 @@ +#!/usr/bin/env python3 +##!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from kazoo.client import KazooClient, KazooState +import random +import string +import os +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/keeper_config1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/keeper_config2.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/keeper_config3.xml'], stay_alive=True) + +def start_zookeeper(node): + node1.exec_in_container(['bash', '-c', '/opt/zookeeper/bin/zkServer.sh start']) + +def stop_zookeeper(node): + node.exec_in_container(['bash', '-c', '/opt/zookeeper/bin/zkServer.sh stop']) + +def clear_zookeeper(node): + node.exec_in_container(['bash', '-c', 'rm -fr /zookeeper/*']) + +def restart_and_clear_zookeeper(node): + stop_zookeeper(node) + clear_zookeeper(node) + start_zookeeper(node) + +def clear_clickhouse_data(node): + node.exec_in_container(['bash', '-c', 'rm -fr /var/lib/clickhouse/coordination/logs/* /var/lib/clickhouse/coordination/snapshots/*']) + +def convert_zookeeper_data(node): + cmd = '/usr/bin/clickhouse keeper-converter --zookeeper-logs-dir /zookeeper/version-2/ --zookeeper-snapshots-dir /zookeeper/version-2/ --output-dir /var/lib/clickhouse/coordination/snapshots' + node.exec_in_container(['bash', '-c', cmd]) + return os.path.join('/var/lib/clickhouse/coordination/snapshots', node.exec_in_container(['bash', '-c', 'ls /var/lib/clickhouse/coordination/snapshots']).strip()) + +def stop_clickhouse(node): + node.stop_clickhouse() + +def start_clickhouse(node): + node.start_clickhouse() + +def copy_zookeeper_data(make_zk_snapshots, node): + stop_zookeeper(node) + + if make_zk_snapshots: # force zookeeper to create snapshot + start_zookeeper(node) + stop_zookeeper(node) + + stop_clickhouse(node) + clear_clickhouse_data(node) + convert_zookeeper_data(node) + start_zookeeper(node) + start_clickhouse(node) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + +def get_genuine_zk(node, timeout=30.0): + _genuine_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":2181", timeout=timeout) + _genuine_zk_instance.start() + return _genuine_zk_instance + + +def test_snapshot_and_load(started_cluster): + restart_and_clear_zookeeper(node1) + genuine_connection = get_genuine_zk(node1) + for node in [node1, node2, node3]: + print("Stop and clear", node.name, "with dockerid", node.docker_id) + stop_clickhouse(node) + clear_clickhouse_data(node) + + for i in range(1000): + genuine_connection.create("/test" + str(i), b"data") + + print("Data loaded to zookeeper") + + stop_zookeeper(node1) + start_zookeeper(node1) + stop_zookeeper(node1) + + print("Data copied to node1") + resulted_path = convert_zookeeper_data(node1) + print("Resulted path", resulted_path) + for node in [node2, node3]: + print("Copy snapshot from", node1.name, "to", node.name) + cluster.copy_file_from_container_to_container(node1, resulted_path, node, '/var/lib/clickhouse/coordination/snapshots') + + print("Starting clickhouses") + + p = Pool(3) + result = p.map_async(start_clickhouse, [node1, node2, node3]) + result.wait() + + print("Loading additional data") + fake_zks = [get_fake_zk(node) for node in [node1, node2, node3]] + for i in range(1000): + fake_zk = random.choice(fake_zks) + try: + fake_zk.create("/test" + str(i + 1000), b"data") + except Exception as ex: + print("Got exception:" + str(ex)) + + print("Final") + fake_zks[0].create("/test10000", b"data") From 94d68ee8ac5fdb24e7b805142c90c345d3b39280 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Aug 2021 14:14:53 +0000 Subject: [PATCH 15/40] Fix, add test --- src/Common/filesystemHelpers.cpp | 1 + tests/integration/test_library_bridge/test.py | 44 +++++++++++++++++++ 2 files changed, 45 insertions(+) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 730099f4476..86ae7a046be 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -126,6 +126,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path) { auto absolute_path = std::filesystem::absolute(path); + absolute_path = absolute_path.lexically_normal(); auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path); auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end()); diff --git a/tests/integration/test_library_bridge/test.py b/tests/integration/test_library_bridge/test.py index 97b2ccfbdbe..12a967ebaa4 100644 --- a/tests/integration/test_library_bridge/test.py +++ b/tests/integration/test_library_bridge/test.py @@ -44,6 +44,11 @@ def ch_cluster(): '/usr/bin/g++ -shared -o /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.so -fPIC /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.cpp'], user='root') + instance.exec_in_container( + ['bash', '-c', + '/usr/bin/g++ -shared -o /dict_lib_copy.so -fPIC /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib.cpp'], user='root') + instance.exec_in_container(['bash', '-c', 'ln -s /dict_lib_copy.so /etc/clickhouse-server/config.d/dictionaries_lib/dict_lib_symlink.so']) + yield cluster finally: @@ -59,6 +64,7 @@ def test_load_all(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + instance.query('DROP DICTIONARY IF EXISTS lib_dict') instance.query(''' CREATE DICTIONARY lib_dict (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) PRIMARY KEY key @@ -128,6 +134,7 @@ def test_load_keys(ch_cluster): if instance.is_built_with_memory_sanitizer(): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + instance.query('DROP DICTIONARY IF EXISTS lib_dict_ckc') instance.query(''' CREATE DICTIONARY lib_dict_ckc (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) PRIMARY KEY key @@ -148,6 +155,7 @@ def test_load_all_many_rows(ch_cluster): pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") num_rows = [1000, 10000, 100000, 1000000] + instance.query('DROP DICTIONARY IF EXISTS lib_dict') for num in num_rows: instance.query(''' CREATE DICTIONARY lib_dict (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) @@ -267,6 +275,42 @@ def test_bridge_dies_with_parent(ch_cluster): instance.query('DROP DICTIONARY lib_dict_c') +def test_path_validation(ch_cluster): + if instance.is_built_with_memory_sanitizer(): + pytest.skip("Memory Sanitizer cannot work with third-party shared libraries") + + instance.query('DROP DICTIONARY IF EXISTS lib_dict_c') + instance.query(''' + CREATE DICTIONARY lib_dict_c (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) + PRIMARY KEY key SOURCE(library(PATH '/etc/clickhouse-server/config.d/dictionaries_lib/dict_lib_symlink.so')) + LAYOUT(CACHE( + SIZE_IN_CELLS 10000000 + BLOCK_SIZE 4096 + FILE_SIZE 16777216 + READ_BUFFER_SIZE 1048576 + MAX_STORED_KEYS 1048576)) + LIFETIME(2) ; + ''') + + result = instance.query('''select dictGet(lib_dict_c, 'value1', toUInt64(1));''') + assert(result.strip() == '101') + + instance.query('DROP DICTIONARY IF EXISTS lib_dict_c') + instance.query(''' + CREATE DICTIONARY lib_dict_c (key UInt64, value1 UInt64, value2 UInt64, value3 UInt64) + PRIMARY KEY key SOURCE(library(PATH '/etc/clickhouse-server/config.d/dictionaries_lib/../../../../dict_lib_copy.so')) + LAYOUT(CACHE( + SIZE_IN_CELLS 10000000 + BLOCK_SIZE 4096 + FILE_SIZE 16777216 + READ_BUFFER_SIZE 1048576 + MAX_STORED_KEYS 1048576)) + LIFETIME(2) ; + ''') + result = instance.query_and_get_error('''select dictGet(lib_dict_c, 'value1', toUInt64(1));''') + assert('DB::Exception: File path /etc/clickhouse-server/config.d/dictionaries_lib/../../../../dict_lib_copy.so is not inside /etc/clickhouse-server/config.d/dictionaries_lib' in result) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 4d4dae79f0ecb122b3fb9304ffb85eee7edc0c4d Mon Sep 17 00:00:00 2001 From: ubuntu Date: Thu, 19 Aug 2021 00:47:40 +0800 Subject: [PATCH 16/40] fix: build issue --- .../functions/type-conversion-functions.md | 8 ++-- src/Functions/FunctionSnowflake.h | 44 +++++++++---------- src/Functions/dateTime64ToSnowflake.cpp | 14 ------ src/Functions/dateTimeToSnowflake.cpp | 14 ------ src/Functions/snowflake.cpp | 34 ++++++++++++++ src/Functions/snowflakeToDateTime.cpp | 14 ------ src/Functions/snowflakeToDateTime64.cpp | 14 ------ 7 files changed, 58 insertions(+), 84 deletions(-) delete mode 100644 src/Functions/dateTime64ToSnowflake.cpp delete mode 100644 src/Functions/dateTimeToSnowflake.cpp create mode 100644 src/Functions/snowflake.cpp delete mode 100644 src/Functions/snowflakeToDateTime.cpp delete mode 100644 src/Functions/snowflakeToDateTime64.cpp diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4f1a2d49d23..ad6edaea312 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1342,7 +1342,7 @@ Result: ## snowflakeToDateTime {#snowflakeToDateTime} -extract time from snowflake id as DateTime format. +Extract time from snowflake id as DateTime format. **Syntax** @@ -1378,7 +1378,7 @@ Result: ## snowflakeToDateTime64 {#snowflakeToDateTime64} -extract time from snowflake id as DateTime64 format. +Extract time from snowflake id as DateTime64 format. **Syntax** @@ -1414,7 +1414,7 @@ Result: ## dateTimeToSnowflake {#dateTimeToSnowflake} -convert DateTime to the first snowflake id at the giving time. +Convert DateTime to the first snowflake id at the giving time. **Syntax** @@ -1452,7 +1452,7 @@ Result: ## dateTime64ToSnowflake {#dateTime64ToSnowflake} -convert DateTime64 to the first snowflake id at the giving time. +Convert DateTime64 to the first snowflake id at the giving time. **Syntax** diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 3dd378e4956..3f0f404f7e4 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -12,23 +12,28 @@ namespace DB { - namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } +/** According to Twitter's post on Snowflake, we can extract the timestamp for a snowflake ID by right shifting + * the snowflake ID by 22 bits(10 bits machine ID and 12 bits sequence ID) and adding the Twitter epoch time of 1288834974657. + * https://en.wikipedia.org/wiki/Snowflake_ID + * https://blog.twitter.com/engineering/en_us/a/2010/announcing-snowflake + * https://ws-dl.blogspot.com/2019/08/2019-08-03-tweetedat-finding-tweet.html +*/ +static constexpr long snowflake_epoch = 1288834974657L; +static constexpr int time_shift = 22; class FunctionDateTimeToSnowflake : public IFunction { private: const char * name; + public: - FunctionDateTimeToSnowflake(const char * name_) - : name(name_) - { - } + FunctionDateTimeToSnowflake(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -54,7 +59,7 @@ public: const auto & source_data = typeid_cast(col).getData(); for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (int64_t(source_data[i])*1000-1288834974657)<<22; + result_data[i] = (Int64(source_data[i]) * 1000 - snowflake_epoch) << time_shift; } return res_column; @@ -66,11 +71,9 @@ class FunctionSnowflakeToDateTime : public IFunction { private: const char * name; + public: - FunctionSnowflakeToDateTime(const char * name_) - : name(name_) - { - } + FunctionSnowflakeToDateTime(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -104,9 +107,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = ((source_data[i]>>22)+1288834974657)/1000; + result_data[i] = ((source_data[i] >> time_shift) + snowflake_epoch) / 1000; } - return res_column; } }; @@ -116,11 +118,9 @@ class FunctionDateTime64ToSnowflake : public IFunction { private: const char * name; + public: - FunctionDateTime64ToSnowflake(const char * name_) - : name(name_) - { - } + FunctionDateTime64ToSnowflake(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -146,7 +146,7 @@ public: const auto & source_data = typeid_cast &>(col).getData(); for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (source_data[i]-1288834974657)<<22; + result_data[i] = (source_data[i] - snowflake_epoch) << time_shift; } return res_column; @@ -158,11 +158,9 @@ class FunctionSnowflakeToDateTime64 : public IFunction { private: const char * name; + public: - FunctionSnowflakeToDateTime64(const char * name_) - : name(name_) - { - } + FunctionSnowflakeToDateTime64(const char * name_) : name(name_) { } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -171,7 +169,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1 || arguments.size() > 2) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", name); @@ -197,9 +194,8 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - result_data[i] = (source_data[i]>>22)+1288834974657; + result_data[i] = (source_data[i] >> time_shift) + snowflake_epoch; } - return res_column; } }; diff --git a/src/Functions/dateTime64ToSnowflake.cpp b/src/Functions/dateTime64ToSnowflake.cpp deleted file mode 100644 index 87e35c25371..00000000000 --- a/src/Functions/dateTime64ToSnowflake.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDateTime64ToSnowflake(FunctionFactory & factory) -{ - factory.registerFunction("dateTime64ToSnowflake", - [](ContextPtr){ return std::make_unique( - std::make_shared("dateTime64ToSnowflake")); }); -} - -} diff --git a/src/Functions/dateTimeToSnowflake.cpp b/src/Functions/dateTimeToSnowflake.cpp deleted file mode 100644 index 246f35cc1dc..00000000000 --- a/src/Functions/dateTimeToSnowflake.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerDateTimeToSnowflake(FunctionFactory & factory) -{ - factory.registerFunction("dateTimeToSnowflake", - [](ContextPtr){ return std::make_unique( - std::make_shared("dateTimeToSnowflake")); }); -} - -} diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp new file mode 100644 index 00000000000..5ac1d229d17 --- /dev/null +++ b/src/Functions/snowflake.cpp @@ -0,0 +1,34 @@ +#include +#include + +namespace DB +{ + +void registerDateTimeToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTimeToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTimeToSnowflake")); }); +} + +void registerDateTime64ToSnowflake(FunctionFactory & factory) +{ + factory.registerFunction("dateTime64ToSnowflake", + [](ContextPtr){ return std::make_unique( + std::make_shared("dateTime64ToSnowflake")); }); +} + +void registerSnowflakeToDateTime(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime")); }); +} +void registerSnowflakeToDateTime64(FunctionFactory & factory) +{ + factory.registerFunction("snowflakeToDateTime64", + [](ContextPtr){ return std::make_unique( + std::make_shared("snowflakeToDateTime64")); }); +} + +} diff --git a/src/Functions/snowflakeToDateTime.cpp b/src/Functions/snowflakeToDateTime.cpp deleted file mode 100644 index 37f5e07512f..00000000000 --- a/src/Functions/snowflakeToDateTime.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerSnowflakeToDateTime(FunctionFactory & factory) -{ - factory.registerFunction("snowflakeToDateTime", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime")); }); -} - -} diff --git a/src/Functions/snowflakeToDateTime64.cpp b/src/Functions/snowflakeToDateTime64.cpp deleted file mode 100644 index ef9502a224e..00000000000 --- a/src/Functions/snowflakeToDateTime64.cpp +++ /dev/null @@ -1,14 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerSnowflakeToDateTime64(FunctionFactory & factory) -{ - factory.registerFunction("snowflakeToDateTime64", - [](ContextPtr){ return std::make_unique( - std::make_shared("snowflakeToDateTime64")); }); -} - -} From f5a91e5e5f63b6ab3c2e77f950b96b227ad6b318 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 18 Aug 2021 23:17:00 +0300 Subject: [PATCH 17/40] Fix style check --- src/Common/filesystemHelpers.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 86ae7a046be..5bed3ea1120 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes extern const int CANNOT_STATVFS; extern const int PATH_ACCESS_DENIED; extern const int CANNOT_CREATE_FILE; - extern const int BAD_ARGUMENTS; } From 3b09640d941d7d0bd1133821204e715a56fe1b67 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 12 Aug 2021 18:16:55 +0300 Subject: [PATCH 18/40] Use sessions more. --- programs/local/LocalServer.cpp | 14 +++--- .../CassandraDictionarySource.cpp | 2 +- .../ClickHouseDictionarySource.cpp | 31 +++++++----- src/Dictionaries/ClickHouseDictionarySource.h | 4 +- src/Dictionaries/DictionaryFactory.cpp | 15 +++--- src/Dictionaries/DictionaryFactory.h | 6 +-- src/Dictionaries/DictionarySourceFactory.cpp | 4 +- src/Dictionaries/DictionarySourceFactory.h | 4 +- src/Dictionaries/DictionarySourceHelpers.cpp | 48 +++++++++++-------- src/Dictionaries/DictionarySourceHelpers.h | 7 ++- src/Dictionaries/DirectDictionary.cpp | 2 +- .../ExecutableDictionarySource.cpp | 8 ++-- .../ExecutablePoolDictionarySource.cpp | 12 ++--- src/Dictionaries/FileDictionarySource.cpp | 6 +-- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 6 +-- src/Dictionaries/HashedDictionary.cpp | 8 ++-- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 4 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 +- .../PolygonDictionaryImplementations.cpp | 2 +- .../PostgreSQLDictionarySource.cpp | 8 ++-- src/Dictionaries/RangeHashedDictionary.cpp | 2 +- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 10 ++-- .../registerCacheDictionaries.cpp | 22 ++++----- src/Interpreters/Context.cpp | 21 -------- src/Interpreters/Context.h | 8 +--- .../ExternalDictionariesLoader.cpp | 8 ++++ 29 files changed, 131 insertions(+), 141 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 44e9880fabb..957bda4d75c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -374,14 +375,13 @@ void LocalServer::processQueries() if (!parse_res.second) throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); - /// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) - /// so we can't reuse it safely as a query context and need a copy here - auto context = Context::createCopy(global_context); + /// Authenticate and create a context to execute queries. + Session session{global_context, ClientInfo::Interface::TCP}; + session.authenticate("default", "", Poco::Net::SocketAddress{}); - context->makeSessionContext(); - context->makeQueryContext(); - - context->authenticate("default", "", Poco::Net::SocketAddress{}); + /// Use the same context for all queries. + auto context = session.makeQueryContext(); + context->makeSessionContext(); /// initial_create_query requires a session context to be set. context->setCurrentQueryId(""); applyCmdSettings(context); diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index aa8d6107508..d9a4dd0fd22 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -17,7 +17,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /*created_from_ddl*/) -> DictionarySourcePtr { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index d4f01dee8b2..b09a7b14cc4 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -63,19 +64,18 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextPtr context_) + ContextMutablePtr context_, + std::shared_ptr local_session_) : update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , configuration{configuration_} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} - , context(Context::createCopy(context_)) + , local_session(local_session_) + , context(context_) , pool{createPool(configuration)} , load_all_query{query_builder.composeLoadAllQuery()} { - /// Query context is needed because some code in executeQuery function may assume it exists. - /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. - context->makeQueryContext(); } ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other) @@ -85,11 +85,11 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , invalidate_query_response{other.invalidate_query_response} , query_builder{dict_struct, configuration.db, "", configuration.table, configuration.query, configuration.where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} + , local_session(other.local_session) , context(Context::createCopy(other.context)) , pool{createPool(configuration)} , load_all_query{other.load_all_query} { - context->makeQueryContext(); } std::string ClickHouseDictionarySource::getUpdateFieldAndDate() @@ -222,14 +222,13 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & default_database [[maybe_unused]], bool /* created_from_ddl */) -> DictionarySourcePtr { bool secure = config.getBool(config_prefix + ".secure", false); - auto context_copy = Context::createCopy(context); - UInt16 default_port = getPortFromContext(context_copy, secure); + UInt16 default_port = getPortFromContext(global_context, secure); std::string settings_config_prefix = config_prefix + ".clickhouse"; std::string host = config.getString(settings_config_prefix + ".host", "localhost"); @@ -252,12 +251,18 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) .secure = config.getBool(settings_config_prefix + ".secure", false) }; - /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). + ContextMutablePtr context; + std::shared_ptr local_session; if (configuration.is_local) { - context_copy->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0)); - context_copy = copyContextAndApplySettings(config_prefix, context_copy, config); + /// Start local session in case when the dictionary is loaded in-process (without TCP communication). + local_session = std::make_shared(global_context, ClientInfo::Interface::TCP); + local_session->authenticate(configuration.user, configuration.password, Poco::Net::SocketAddress{"127.0.0.1", 0}); + context = local_session->makeQueryContext(); + context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix)); } + else + context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); String dictionary_name = config.getString(".dictionary.name", ""); String dictionary_database = config.getString(".dictionary.database", ""); @@ -265,7 +270,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) if (dictionary_name == configuration.table && dictionary_database == configuration.db) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); - return std::make_unique(dict_struct, configuration, sample_block, context_copy); + return std::make_unique(dict_struct, configuration, sample_block, context, local_session); }; factory.registerSource("clickhouse", create_table_source); diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index 2daa296af3e..58243e43b15 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -39,7 +39,8 @@ public: const DictionaryStructure & dict_struct_, const Configuration & configuration_, const Block & sample_block_, - ContextPtr context); + ContextMutablePtr context_, + std::shared_ptr local_session_); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); @@ -81,6 +82,7 @@ private: mutable std::string invalidate_query_response; ExternalQueryBuilder query_builder; Block sample_block; + std::shared_ptr local_session; ContextMutablePtr context; ConnectionPoolWithFailoverPtr pool; const std::string load_all_query; diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 62b28ed7d14..4cab42c9445 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -31,7 +31,7 @@ DictionaryPtr DictionaryFactory::create( const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) const { Poco::Util::AbstractConfiguration::Keys keys; @@ -45,12 +45,9 @@ DictionaryPtr DictionaryFactory::create( const DictionaryStructure dict_struct{config, config_prefix}; DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create( - name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), created_from_ddl); + name, config, config_prefix + ".source", dict_struct, global_context, config.getString(config_prefix + ".database", ""), created_from_ddl); LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name); - if (context->hasQueryContext() && context->getSettingsRef().log_queries) - context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, name); - const auto & layout_type = keys.front(); { @@ -58,7 +55,7 @@ DictionaryPtr DictionaryFactory::create( if (found != registered_layouts.end()) { const auto & layout_creator = found->second.layout_create_function; - return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl); + return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); } } @@ -68,10 +65,10 @@ DictionaryPtr DictionaryFactory::create( layout_type); } -DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr context) const +DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, ContextPtr global_context) const { - auto configuration = getDictionaryConfigurationFromAST(ast, context); - return DictionaryFactory::create(name, *configuration, "dictionary", context, true); + auto configuration = getDictionaryConfigurationFromAST(ast, global_context); + return DictionaryFactory::create(name, *configuration, "dictionary", global_context, true); } bool DictionaryFactory::isComplex(const std::string & layout_type) const diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index b717009aa8a..b1dad340f4b 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -36,13 +36,13 @@ public: const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) const; /// Create dictionary from DDL-query DictionaryPtr create(const std::string & name, const ASTCreateQuery & ast, - ContextPtr context) const; + ContextPtr global_context) const; using LayoutCreateFunction = std::function; bool isComplex(const std::string & layout_type) const; diff --git a/src/Dictionaries/DictionarySourceFactory.cpp b/src/Dictionaries/DictionarySourceFactory.cpp index 1992c87d31f..80b60f29e37 100644 --- a/src/Dictionaries/DictionarySourceFactory.cpp +++ b/src/Dictionaries/DictionarySourceFactory.cpp @@ -80,7 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config) const { @@ -99,7 +99,7 @@ DictionarySourcePtr DictionarySourceFactory::create( { const auto & create_source = found->second; auto sample_block = createSampleBlock(dict_struct); - return create_source(dict_struct, config, config_prefix, sample_block, context, default_database, check_config); + return create_source(dict_struct, config, config_prefix, sample_block, global_context, default_database, check_config); } throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, diff --git a/src/Dictionaries/DictionarySourceFactory.h b/src/Dictionaries/DictionarySourceFactory.h index bb583927ac4..f4c3fa12163 100644 --- a/src/Dictionaries/DictionarySourceFactory.h +++ b/src/Dictionaries/DictionarySourceFactory.h @@ -35,7 +35,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config)>; @@ -48,7 +48,7 @@ public: const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const DictionaryStructure & dict_struct, - ContextPtr context, + ContextPtr global_context, const std::string & default_database, bool check_config) const; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 092e7187e8f..cf003dceb8e 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -59,30 +59,36 @@ Block blockForKeys( return block; } -ContextMutablePtr copyContextAndApplySettings( - const std::string & config_prefix, - ContextPtr context, - const Poco::Util::AbstractConfiguration & config) + +SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { - auto local_context = Context::createCopy(context); - if (config.has(config_prefix + ".settings")) + if (!config.has(config_prefix + ".settings")) + return {}; + + const auto prefix = config_prefix + ".settings"; + + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(prefix, config_keys); + + SettingsChanges changes; + + for (const std::string & key : config_keys) { - const auto prefix = config_prefix + ".settings"; - - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(prefix, config_keys); - - SettingsChanges changes; - - for (const std::string & key : config_keys) - { - const auto value = config.getString(prefix + "." + key); - changes.emplace_back(key, value); - } - - local_context->applySettingsChanges(changes); + const auto value = config.getString(prefix + "." + key); + changes.emplace_back(key, value); } - return local_context; + + return changes; +} + + +ContextMutablePtr copyContextAndApplySettingsFromDictionaryConfig( + const ContextPtr & context, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) +{ + auto context_copy = Context::createCopy(context); + auto changes = readSettingsFromDictionaryConfig(config, config_prefix); + context_copy->applySettingsChanges(changes); + return context_copy; } static Block transformHeader(Block header, Block block_to_add) diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index b955b6ffb66..5470321745a 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -14,6 +14,7 @@ namespace DB { struct DictionaryStructure; +class SettingsChanges; /// For simple key @@ -29,10 +30,8 @@ Block blockForKeys( const std::vector & requested_rows); /// Used for applying settings to copied context in some register[...]Source functions -ContextMutablePtr copyContextAndApplySettings( - const std::string & config_prefix, - ContextPtr context, - const Poco::Util::AbstractConfiguration & config); +SettingsChanges readSettingsFromDictionaryConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); +ContextMutablePtr copyContextAndApplySettingsFromDictionaryConfig(const ContextPtr & context, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); /** A stream, adds additional columns to each block that it will read from inner stream. * diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index e12100a556d..10e7414b42f 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -307,7 +307,7 @@ namespace const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /* created_from_ddl */) { const auto * layout_name = dictionary_key_type == DictionaryKeyType::Simple ? "direct" : "complex_key_direct"; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 8802d04ff30..5c6add34f1f 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -275,7 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -285,10 +285,10 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) /// Executable dictionaries may execute arbitrary commands. /// It's OK for dictionaries created by administrator from xml-file, but /// maybe dangerous for dictionaries created from DDL-queries. - if (created_from_ddl && context->getApplicationType() != Context::ApplicationType::LOCAL) + if (created_from_ddl && global_context->getApplicationType() != Context::ApplicationType::LOCAL) throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query"); - auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); std::string settings_config_prefix = config_prefix + ".executable"; @@ -301,7 +301,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false) }; - return std::make_unique(dict_struct, configuration, sample_block, context_local_copy); + return std::make_unique(dict_struct, configuration, sample_block, context); }; factory.registerSource("executable", create_table_source); diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index e97b4253407..e18664e477c 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -279,7 +279,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -289,17 +289,15 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) /// Executable dictionaries may execute arbitrary commands. /// It's OK for dictionaries created by administrator from xml-file, but /// maybe dangerous for dictionaries created from DDL-queries. - if (created_from_ddl && context->getApplicationType() != Context::ApplicationType::LOCAL) + if (created_from_ddl && global_context->getApplicationType() != Context::ApplicationType::LOCAL) throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query"); - auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + ContextMutablePtr context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); /** Currently parallel parsing input format cannot read exactly max_block_size rows from input, * so it will be blocked on ReadBufferFromFileDescriptor because this file descriptor represent pipe that does not have eof. */ - auto settings_no_parallel_parsing = context_local_copy->getSettings(); - settings_no_parallel_parsing.input_format_parallel_parsing = false; - context_local_copy->setSettings(settings_no_parallel_parsing); + context->setSetting("input_format_parallel_parsing", Field{false}); String settings_config_prefix = config_prefix + ".executable_pool"; @@ -319,7 +317,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .implicit_key = config.getBool(settings_config_prefix + ".implicit_key", false), }; - return std::make_unique(dict_struct, configuration, sample_block, context_local_copy); + return std::make_unique(dict_struct, configuration, sample_block, context); }; factory.registerSource("executable_pool", create_table_source); diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index bea14d88d1e..54ce5e4a448 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -77,7 +77,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { @@ -87,9 +87,9 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) const auto filepath = config.getString(config_prefix + ".file.path"); const auto format = config.getString(config_prefix + ".file.format"); - auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + const auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); - return std::make_unique(filepath, format, sample_block, context_local_copy, created_from_ddl); + return std::make_unique(filepath, format, sample_block, context, created_from_ddl); }; factory.registerSource("file", create_table_source); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 26667db1081..5ecf3299ea6 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -557,7 +557,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /* created_from_ddl */) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 4a80ebdf975..b5cf59b4474 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -213,13 +213,13 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { if (dict_struct.has_expressions) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions"); - auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config); + auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); const auto & settings_config_prefix = config_prefix + ".http"; const auto & credentials_prefix = settings_config_prefix + ".credentials"; @@ -258,7 +258,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) .header_entries = std::move(header_entries) }; - return std::make_unique(dict_struct, configuration, credentials, sample_block, context_local_copy, created_from_ddl); + return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); }; factory.registerSource("http", create_table_source); } diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d462631fba8..fd5865e24c0 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -756,13 +756,13 @@ void registerDictionaryHashed(DictionaryFactory & factory) using namespace std::placeholders; factory.registerLayout("hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ false); }, false); factory.registerLayout("sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple, /* sparse = */ true); }, false); factory.registerLayout("complex_key_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ false); }, true); factory.registerLayout("complex_key_sparse_hashed", - [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true); + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex, /* sparse = */ true); }, true); } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index fbe911c1d49..b688362d048 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -954,7 +954,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (!dict_struct.key || dict_struct.key->size() != 1) diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 288abde8788..602e0c5b08e 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -183,11 +183,11 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool created_from_ddl) -> DictionarySourcePtr { - return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, context, created_from_ddl); + return std::make_unique(dict_struct, config, config_prefix + ".library", sample_block, global_context, created_from_ddl); }; factory.registerSource("library", create_table_source); diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 2eebb6970d0..bd53c1e60a7 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -31,11 +31,11 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) [[maybe_unused]] const Poco::Util::AbstractConfiguration & config, [[maybe_unused]] const std::string & config_prefix, [[maybe_unused]] Block & sample_block, - [[maybe_unused]] ContextPtr context, + [[maybe_unused]] ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { #if USE_MYSQL - StreamSettings mysql_input_stream_settings(context->getSettingsRef() + StreamSettings mysql_input_stream_settings(global_context->getSettingsRef() , config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false) , false , config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss); diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index 7c3eb421a4a..72869ad57ba 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -167,7 +167,7 @@ DictionaryPtr createLayout(const std::string & , const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) { const String database = config.getString(config_prefix + ".database", ""); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 5a546820959..3fe9e899cd9 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -182,7 +182,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { @@ -190,8 +190,8 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const auto settings_config_prefix = config_prefix + ".postgresql"; auto pool = std::make_shared( config, settings_config_prefix, - context->getSettingsRef().postgresql_connection_pool_size, - context->getSettingsRef().postgresql_connection_pool_wait_timeout); + global_context->getSettingsRef().postgresql_connection_pool_size, + global_context->getSettingsRef().postgresql_connection_pool_wait_timeout); PostgreSQLDictionarySource::Configuration configuration { @@ -211,7 +211,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) (void)config; (void)config_prefix; (void)sample_block; - (void)context; + (void)global_context; throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support."); #endif diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index ea0af493bdf..390871661c7 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -688,7 +688,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr /* context */, + ContextPtr /* global_context */, bool /*created_from_ddl*/) -> DictionaryPtr { if (dict_struct.key) diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 6561a122e9d..24a14d8cc80 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -12,7 +12,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const String & config_prefix, Block & sample_block, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { return std::make_unique(dict_struct, config, config_prefix + ".redis", sample_block); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index e79e55910b7..9fc7e92634b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -234,12 +234,12 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Block & sample_block, - ContextPtr context, + ContextPtr global_context, const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { #if USE_ODBC BridgeHelperPtr bridge = std::make_shared>( - context, context->getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); + global_context, global_context->getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); std::string settings_config_prefix = config_prefix + ".odbc"; @@ -255,13 +255,13 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory) .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1) }; - return std::make_unique(dict_struct, configuration, sample_block, context, bridge); + return std::make_unique(dict_struct, configuration, sample_block, global_context, bridge); #else (void)dict_struct; (void)config; (void)config_prefix; (void)sample_block; - (void)context; + (void)global_context; throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Dictionary source of type `odbc` is disabled because poco library was built without ODBC support."); #endif @@ -276,7 +276,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, Block & /* sample_block */, - ContextPtr /* context */, + ContextPtr /* global_context */, const std::string & /* default_database */, bool /* created_from_ddl */) -> DictionarySourcePtr { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Dictionaries/registerCacheDictionaries.cpp b/src/Dictionaries/registerCacheDictionaries.cpp index 64c1c55e0ba..69197f992f0 100644 --- a/src/Dictionaries/registerCacheDictionaries.cpp +++ b/src/Dictionaries/registerCacheDictionaries.cpp @@ -154,7 +154,7 @@ DictionaryPtr createCacheDictionaryLayout( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context [[maybe_unused]], + ContextPtr global_context [[maybe_unused]], bool created_from_ddl [[maybe_unused]]) { String layout_type; @@ -213,8 +213,8 @@ DictionaryPtr createCacheDictionaryLayout( else { auto storage_configuration = parseSSDCacheStorageConfiguration(config, full_name, layout_type, dictionary_layout_prefix, dict_lifetime); - if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, context->getUserFilesPath())) - throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, context->getUserFilesPath()); + if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, global_context->getUserFilesPath())) + throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, global_context->getUserFilesPath()); storage = std::make_shared>(storage_configuration); } @@ -239,10 +239,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); }; factory.registerLayout("cache", create_simple_cache_layout, false); @@ -252,10 +252,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); }; factory.registerLayout("complex_key_cache", create_complex_key_cache_layout, true); @@ -267,10 +267,10 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); }; factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false); @@ -280,9 +280,9 @@ void registerDictionaryCache(DictionaryFactory & factory) const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, DictionarySourcePtr source_ptr, - ContextPtr context, + ContextPtr global_context, bool created_from_ddl) -> DictionaryPtr { - return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), std::move(context), created_from_ddl); + return createCacheDictionaryLayout(full_name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); }; factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a634c19dcd6..84a858d8603 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -589,27 +589,6 @@ ConfigurationPtr Context::getUsersConfig() } -void Context::authenticate(const String & name, const String & password, const Poco::Net::SocketAddress & address) -{ - authenticate(BasicCredentials(name, password), address); -} - -void Context::authenticate(const Credentials & credentials, const Poco::Net::SocketAddress & address) -{ - auto authenticated_user_id = getAccessControlManager().login(credentials, address.host()); - - client_info.current_user = credentials.getUserName(); - client_info.current_address = address; - -#if defined(ARCADIA_BUILD) - /// This is harmful field that is used only in foreign "Arcadia" build. - if (const auto * basic_credentials = dynamic_cast(&credentials)) - client_info.current_password = basic_credentials->getPassword(); -#endif - - setUser(authenticated_user_id); -} - void Context::setUser(const UUID & user_id_) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 4e378dacf01..1b636deb532 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -362,13 +362,9 @@ public: void setUsersConfig(const ConfigurationPtr & config); ConfigurationPtr getUsersConfig(); - /// Sets the current user, checks the credentials and that the specified address is allowed to connect from. - /// The function throws an exception if there is no such user or password is wrong. - void authenticate(const String & user_name, const String & password, const Poco::Net::SocketAddress & address); - void authenticate(const Credentials & credentials, const Poco::Net::SocketAddress & address); - /// Sets the current user assuming that he/she is already authenticated. - /// WARNING: This function doesn't check password! Don't use until it's necessary! + /// WARNING: This function doesn't check password! + /// Normally you shouldn't call this function. Use the Session class to do authentication instead. void setUser(const UUID & user_id_); UserPtr getUser() const; diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 83931649443..cbb0e52b91b 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -45,12 +45,20 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create( ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); + + if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) + local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + return std::static_pointer_cast(load(resolved_dictionary_name)); } ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::tryGetDictionary(const std::string & dictionary_name, ContextPtr local_context) const { std::string resolved_dictionary_name = resolveDictionaryName(dictionary_name, local_context->getCurrentDatabase()); + + if (local_context->hasQueryContext() && local_context->getSettingsRef().log_queries) + local_context->addQueryFactoriesInfo(Context::QueryLogFactories::Dictionary, resolved_dictionary_name); + return std::static_pointer_cast(tryLoad(resolved_dictionary_name)); } From 3149e80c5934bbe46bfd11882180141e17873a62 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Aug 2021 09:54:13 +0300 Subject: [PATCH 19/40] Commit missed file --- tests/integration/test_keeper_snapshot_small_distance/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_keeper_snapshot_small_distance/__init__.py diff --git a/tests/integration/test_keeper_snapshot_small_distance/__init__.py b/tests/integration/test_keeper_snapshot_small_distance/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From 781b8123a591b4599b96cde27ab3693c7cf42761 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Thu, 19 Aug 2021 09:55:18 +0300 Subject: [PATCH 20/40] Remove logging from NotJoinedBlocks --- src/Interpreters/join_common.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index b230d8d1957..9890a130c33 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -500,8 +500,6 @@ NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, , saved_block_sample(filler->getEmptyBlock()) , result_sample_block(materializeBlock(result_sample_block_)) { - LOG_DEBUG(&Poco::Logger::get("NotJoinedBlocks"), "saved_block_sample {}",saved_block_sample.dumpStructure()); - LOG_DEBUG(&Poco::Logger::get("NotJoinedBlocks"), "result_sample_block {}",result_sample_block.dumpStructure()); for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) { From df0303f9ba18a36eee4f87b0d6daf03b69b87f24 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Aug 2021 11:19:23 +0300 Subject: [PATCH 21/40] Update filesystemHelpers.cpp --- src/Common/filesystemHelpers.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 5bed3ea1120..95913e6106c 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -124,8 +124,13 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path) { + /// Differs from pathStartsWith in how `path` is normalized before comparison. + /// Make `path` absolute if it was relative and put it into normalized form: remove + /// `.` and `..` and extra `/`. Path is not canonized because otherwise path will + /// not be a path of a symlink itself. + auto absolute_path = std::filesystem::absolute(path); - absolute_path = absolute_path.lexically_normal(); + absolute_path = absolute_path.lexically_normal(); /// Normalize path. auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path); auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end()); From b03f851cba20ee54548d0067f8b5ab1733f67e9c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Aug 2021 11:39:50 +0300 Subject: [PATCH 22/40] Update filesystemHelpers.cpp --- src/Common/filesystemHelpers.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 95913e6106c..d846f56c584 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -131,7 +131,8 @@ bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem auto absolute_path = std::filesystem::absolute(path); absolute_path = absolute_path.lexically_normal(); /// Normalize path. - auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path); + auto absolute_prefix_path = std::filesystem::absolute(prefix_path); + absolute_pefix_path = absolute_prefix_path.lexically_normal(); /// Normalize path. auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end()); From c68793cd1291d9cb39ff76cde17ab1a8779ec4c3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Aug 2021 12:14:41 +0300 Subject: [PATCH 23/40] Update filesystemHelpers.cpp --- src/Common/filesystemHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index d846f56c584..9c3db0f3e30 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -132,7 +132,7 @@ bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem auto absolute_path = std::filesystem::absolute(path); absolute_path = absolute_path.lexically_normal(); /// Normalize path. auto absolute_prefix_path = std::filesystem::absolute(prefix_path); - absolute_pefix_path = absolute_prefix_path.lexically_normal(); /// Normalize path. + absolute_prefix_path = absolute_prefix_path.lexically_normal(); /// Normalize path. auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end()); From 3d05014da1d191a389a92a5818c062696fc151e8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 19 Aug 2021 12:54:03 +0300 Subject: [PATCH 24/40] Update FunctionSnowflake.h --- src/Functions/FunctionSnowflake.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Functions/FunctionSnowflake.h b/src/Functions/FunctionSnowflake.h index 3f0f404f7e4..95d02de3a2b 100644 --- a/src/Functions/FunctionSnowflake.h +++ b/src/Functions/FunctionSnowflake.h @@ -39,6 +39,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -79,6 +80,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -126,6 +128,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool isVariadic() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { @@ -166,6 +169,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { From 1e195cbacf12408784dab71c4a2198d92ab9c247 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 14:48:10 +0300 Subject: [PATCH 25/40] Update LZ4_decompress_faster.cpp --- src/Compression/LZ4_decompress_faster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index 21a2cc01a12..ad06900f45c 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,7 +439,7 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while (ip < input_end && unlikely(s == 255)); + } while while (unlikely(s == 255 && ip < input_end)); }; /// Get literal length. From 1697a6fa22d239a570f0570762c864cb6b09f8a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 19 Aug 2021 15:10:55 +0300 Subject: [PATCH 26/40] Fix --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 19560cec9ea..4f31c8dfb52 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -110,7 +110,7 @@ void insertPostgreSQLValue( readDateTime64Text(time, 6, in, assert_cast(data_type.get())->getTimeZone()); if (time < 0) time = 0; - assert_cast &>(column).insertValue(time); + assert_cast(column).insertValue(time); break; } case ExternalResultDescription::ValueType::vtDecimal32: [[fallthrough]]; From 2272c9896c7064cb7a6e3bc3da72c7d55ebe7c82 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Aug 2021 16:53:39 +0300 Subject: [PATCH 27/40] Update LZ4_decompress_faster.cpp --- src/Compression/LZ4_decompress_faster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/LZ4_decompress_faster.cpp b/src/Compression/LZ4_decompress_faster.cpp index ad06900f45c..28a285f00f4 100644 --- a/src/Compression/LZ4_decompress_faster.cpp +++ b/src/Compression/LZ4_decompress_faster.cpp @@ -439,7 +439,7 @@ bool NO_INLINE decompressImpl( { s = *ip++; length += s; - } while while (unlikely(s == 255 && ip < input_end)); + } while (unlikely(s == 255 && ip < input_end)); }; /// Get literal length. From 9fecda940dbb1a093d2693e808bdbe221a33320f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 19 Aug 2021 17:09:44 +0300 Subject: [PATCH 28/40] Fix shutdown of NamedSessionStorage. --- programs/server/Server.cpp | 2 -- src/Interpreters/Context.cpp | 3 +++ src/Interpreters/Session.cpp | 51 ++++++++++++++++++++++-------------- src/Interpreters/Session.h | 5 ++-- 4 files changed, 37 insertions(+), 24 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c30ef52f46a..4d68a8be4e4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -53,7 +53,6 @@ #include #include #include -#include #include #include #include @@ -1429,7 +1428,6 @@ if (ThreadFuzzer::instance().isEffective()) /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. async_metrics.start(); - Session::startupNamedSessions(); { String level_str = config().getString("text_log.level", ""); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a634c19dcd6..348ca84ee6f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -59,6 +59,7 @@ #include #include #include +#include #include #include #include @@ -273,6 +274,8 @@ struct ContextSharedPart return; shutdown_called = true; + Session::shutdownNamedSessions(); + /** After system_logs have been shut down it is guaranteed that no system table gets created or written to. * Note that part changes at shutdown won't be logged to part log. */ diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 7334f2e7640..c0e08395eff 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -54,17 +54,17 @@ class NamedSessionsStorage public: using Key = NamedSessionKey; + static NamedSessionsStorage & instance() + { + static NamedSessionsStorage the_instance; + return the_instance; + } + ~NamedSessionsStorage() { try { - { - std::lock_guard lock{mutex}; - quit = true; - } - - cond.notify_one(); - thread.join(); + shutdown(); } catch (...) { @@ -72,6 +72,20 @@ public: } } + void shutdown() + { + { + std::lock_guard lock{mutex}; + sessions.clear(); + if (!thread.joinable()) + return; + quit = true; + } + + cond.notify_one(); + thread.join(); + } + /// Find existing session or create a new. std::pair, bool> acquireSession( const ContextPtr & global_context, @@ -94,6 +108,10 @@ public: auto context = Context::createCopy(global_context); it = sessions.insert(std::make_pair(key, std::make_shared(key, context, timeout, *this))).first; const auto & session = it->second; + + if (!thread.joinable()) + thread = ThreadFromGlobalPool{&NamedSessionsStorage::cleanThread, this}; + return {session, true}; } else @@ -156,11 +174,9 @@ private: { setThreadName("SessionCleaner"); std::unique_lock lock{mutex}; - - while (true) + while (!quit) { auto interval = closeSessions(lock); - if (cond.wait_for(lock, interval, [this]() -> bool { return quit; })) break; } @@ -208,8 +224,8 @@ private: std::mutex mutex; std::condition_variable cond; - std::atomic quit{false}; - ThreadFromGlobalPool thread{&NamedSessionsStorage::cleanThread, this}; + ThreadFromGlobalPool thread; + bool quit = false; }; @@ -218,13 +234,12 @@ void NamedSessionData::release() parent.releaseSession(*this); } -std::optional Session::named_sessions = std::nullopt; - -void Session::startupNamedSessions() +void Session::shutdownNamedSessions() { - named_sessions.emplace(); + NamedSessionsStorage::instance().shutdown(); } + Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_) : global_context(global_context_) { @@ -317,15 +332,13 @@ ContextMutablePtr Session::makeSessionContext(const String & session_id_, std::c throw Exception("Session context already exists", ErrorCodes::LOGICAL_ERROR); if (query_context_created) throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); - if (!named_sessions) - throw Exception("Support for named sessions is not enabled", ErrorCodes::LOGICAL_ERROR); /// Make a new session context OR /// if the `session_id` and `user_id` were used before then just get a previously created session context. std::shared_ptr new_named_session; bool new_named_session_created = false; std::tie(new_named_session, new_named_session_created) - = named_sessions->acquireSession(global_context, user_id.value_or(UUID{}), session_id_, timeout_, session_check_); + = NamedSessionsStorage::instance().acquireSession(global_context, user_id.value_or(UUID{}), session_id_, timeout_, session_check_); auto new_session_context = new_named_session->context; new_session_context->makeSessionContext(); diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 58370aad2d0..0e816324dad 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -28,9 +28,8 @@ using UserPtr = std::shared_ptr; class Session { public: - /// Allow to use named sessions. The thread will be run to cleanup sessions after timeout has expired. - /// The method must be called at the server startup. - static void startupNamedSessions(); + /// Stops using named sessions. The method must be called at the server shutdown. + static void shutdownNamedSessions(); Session(const ContextPtr & global_context_, ClientInfo::Interface interface_); Session(Session &&); From a9d83c1eec7fd5723a29d0182929d73970a8b713 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 19 Aug 2021 18:15:44 +0300 Subject: [PATCH 29/40] fix postgres like cast with negative numbers --- src/Parsers/ExpressionElementParsers.cpp | 30 ++++++++++++++----- src/Parsers/ExpressionListParsers.cpp | 6 ++-- .../01852_cast_operator_3.reference | 10 +++++++ .../0_stateless/01852_cast_operator_3.sql | 14 +++++++++ .../01852_cast_operator_bad_cases.reference | 8 +++++ .../01852_cast_operator_bad_cases.sh | 10 +++++++ 6 files changed, 69 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/01852_cast_operator_3.reference create mode 100644 tests/queries/0_stateless/01852_cast_operator_3.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 16f2b720b4a..a79b3e51e16 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -850,15 +850,24 @@ static bool isOneOf(TokenType token) return ((token == tokens) || ...); } - bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - /// Parse numbers (including decimals), strings and arrays of them. + /// Parse numbers (including decimals), strings, arrays and tuples of them. const char * data_begin = pos->begin; const char * data_end = pos->end; bool is_string_literal = pos->type == TokenType::StringLiteral; - if (pos->type == TokenType::Number || is_string_literal) + + if (pos->type == TokenType::Minus) + { + ++pos; + if (pos->type != TokenType::Number) + return false; + + data_end = pos->end; + ++pos; + } + else if (pos->type == TokenType::Number || is_string_literal) { ++pos; } @@ -876,7 +885,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::ClosingSquareBracket) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; if (stack.empty() || stack.back() != TokenType::OpeningSquareBracket) return false; @@ -884,7 +893,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::ClosingRoundBracket) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; if (stack.empty() || stack.back() != TokenType::OpeningRoundBracket) return false; @@ -892,10 +901,15 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (pos->type == TokenType::Comma) { - if (isOneOf(last_token)) + if (isOneOf(last_token)) return false; } - else if (isOneOf(pos->type)) + else if (pos->type == TokenType::Number) + { + if (!isOneOf(last_token)) + return false; + } + else if (isOneOf(pos->type)) { if (!isOneOf(last_token)) return false; @@ -915,6 +929,8 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (!stack.empty()) return false; } + else + return false; ASTPtr type_ast; if (ParserToken(TokenType::DoubleColon).ignore(pos, expected) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 58f5e766905..3aa5c82884b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -664,10 +664,12 @@ bool ParserUnaryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (pos->type == TokenType::Minus) { - ParserLiteral lit_p; Pos begin = pos; + if (ParserCastOperator().parse(pos, node, expected)) + return true; - if (lit_p.parse(pos, node, expected)) + pos = begin; + if (ParserLiteral().parse(pos, node, expected)) return true; pos = begin; diff --git a/tests/queries/0_stateless/01852_cast_operator_3.reference b/tests/queries/0_stateless/01852_cast_operator_3.reference new file mode 100644 index 00000000000..a1e54797d60 --- /dev/null +++ b/tests/queries/0_stateless/01852_cast_operator_3.reference @@ -0,0 +1,10 @@ +-1 +SELECT CAST(\'-1\', \'Int32\') +-0.1 +SELECT CAST(\'-0.1\', \'Decimal(38, 38)\') +-0.111 +SELECT CAST(\'-0.111\', \'Float64\') +[-1,2,-3] +SELECT CAST(\'[-1, 2, -3]\', \'Array(Int32)\') +[-1.1,2,-3] +SELECT CAST(\'[-1.1, 2, -3]\', \'Array(Float64)\') diff --git a/tests/queries/0_stateless/01852_cast_operator_3.sql b/tests/queries/0_stateless/01852_cast_operator_3.sql new file mode 100644 index 00000000000..1ad015a8dc4 --- /dev/null +++ b/tests/queries/0_stateless/01852_cast_operator_3.sql @@ -0,0 +1,14 @@ +SELECT -1::Int32; +EXPLAIN SYNTAX SELECT -1::Int32; + +SELECT -0.1::Decimal(38, 38); +EXPLAIN SYNTAX SELECT -0.1::Decimal(38, 38); + +SELECT -0.111::Float64; +EXPLAIN SYNTAX SELECT -0.111::Float64; + +SELECT [-1, 2, -3]::Array(Int32); +EXPLAIN SYNTAX SELECT [-1, 2, -3]::Array(Int32); + +SELECT [-1.1, 2, -3]::Array(Float64); +EXPLAIN SYNTAX SELECT [-1.1, 2, -3]::Array(Float64); diff --git a/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference b/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference index 2c4517e0eda..b179e5e927a 100644 --- a/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference +++ b/tests/queries/0_stateless/01852_cast_operator_bad_cases.reference @@ -8,3 +8,11 @@ Syntax error Syntax error Syntax error Code: 6 +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error +Syntax error diff --git a/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh b/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh index f2f566b78c4..6c578a0996c 100755 --- a/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh +++ b/tests/queries/0_stateless/01852_cast_operator_bad_cases.sh @@ -15,3 +15,13 @@ $CLICKHOUSE_CLIENT --query="SELECT [1 2]::Array(UInt8)" 2>&1 | grep -o -m1 'Syn $CLICKHOUSE_CLIENT --query="SELECT 1 4::UInt32" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_CLIENT --query="SELECT '1' '4'::UInt32" 2>&1 | grep -o -m1 'Syntax error' $CLICKHOUSE_CLIENT --query="SELECT '1''4'::UInt32" 2>&1 | grep -o -m1 'Code: 6' + +$CLICKHOUSE_CLIENT --query="SELECT ::UInt32" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT ::String" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT -::Int32" 2>&1 | grep -o 'Syntax error' + +$CLICKHOUSE_CLIENT --query="SELECT [1, -]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [1, 3-]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [-, 2]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [--, 2]::Array(Int32)" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_CLIENT --query="SELECT [1, 2]-::Array(Int32)" 2>&1 | grep -o 'Syntax error' From f3ff3aee0e749e3e8242ebedeba0e867519b2b56 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 19 Aug 2021 18:49:39 +0300 Subject: [PATCH 30/40] Remove tmp folders from tests --- .../test_abxi8n/parquet_decimal0.parquet | Bin 8849 -> 0 bytes .../test_abxi8n/parquet_decimal1.parquet | Bin 29278 -> 0 bytes .../test_abxi8n/parquet_decimal2.parquet | Bin 6038 -> 0 bytes .../test_abxi8n/parquet_decimal3_1.parquet | Bin 559 -> 0 bytes .../test_abxi8n/parquet_decimal3_2.parquet | Bin 777 -> 0 bytes .../test_abxi8n/parquet_decimal3_3.parquet | Bin 3049 -> 0 bytes .../0_stateless/test_dozlem/arrays.arrow | Bin 4834 -> 0 bytes .../test_dozlem/arrow_all_types_1.arrow | Bin 3600 -> 0 bytes .../test_dozlem/arrow_all_types_2.arrow | Bin 1872 -> 0 bytes .../test_dozlem/arrow_all_types_5.arrow | Bin 1816 -> 0 bytes .../0_stateless/test_dozlem/dicts.arrow | Bin 20030554 -> 0 bytes .../test_dozlem/nullable_arrays.arrow | Bin 1322 -> 0 bytes 12 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal0.parquet delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal1.parquet delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal2.parquet delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal3_1.parquet delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal3_2.parquet delete mode 100644 tests/queries/0_stateless/test_abxi8n/parquet_decimal3_3.parquet delete mode 100644 tests/queries/0_stateless/test_dozlem/arrays.arrow delete mode 100644 tests/queries/0_stateless/test_dozlem/arrow_all_types_1.arrow delete mode 100644 tests/queries/0_stateless/test_dozlem/arrow_all_types_2.arrow delete mode 100644 tests/queries/0_stateless/test_dozlem/arrow_all_types_5.arrow delete mode 100644 tests/queries/0_stateless/test_dozlem/dicts.arrow delete mode 100644 tests/queries/0_stateless/test_dozlem/nullable_arrays.arrow diff --git a/tests/queries/0_stateless/test_abxi8n/parquet_decimal0.parquet b/tests/queries/0_stateless/test_abxi8n/parquet_decimal0.parquet deleted file mode 100644 index cfa6cc2e0b68618e3e064d1825c6aa83dddc7be2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8849 zcmdT~U1$_n6uvXA+jSF{WxIE>OID3xJ*I7&NFF2v5s^|v#D`Ey2qE;L4J~iB!bG zk(hIa3C6 z?I;@|tyB}5c_O5xDwwXScp**1NqT)mNQ*HTi&cb>CO;%8TqQz!lYmEwvJuj4gV5|J zLRw40>tw|X=~GA-LWHOKx0 zJq7B}9NtT4QbS$c1F!bD^4&b5l6E0OmFaJ^N=>pikxmhJIq5yklJuyd{#y@I^|E|R zpHZ2<7l!&`KS_Za>H@$MD4RNX_hUkn8tRW9!F+=&q|JZpDgTp&ge3p4+$a$y2}x9q zBJOg8^rnfVM-BB(BfR=V7EiuJ|-0I4A^N32?h3cS8e`AG< zN_9}AQ^Z|PdJAnNJ!+`;PQct(vV2RQQJKCMb?~24Bn4`yw@<1A)ezR z4vr+wRm3q8;y98x{bqa-C-I*+17>^?Cu_&i1DKhPhaGG1v#$(6 zc6Gtaek}OeHH9F1mf&T#5&Y~4LXaIf*jj7NSlnsoxNME7SduYZ!Cp4Wz|U3|1ldRe zFIzz1XQKv!Y?FYO4G!?L4FN$m0bpypd20a$|y6|;d=Y_K!7t`bT7Ds_UHj1yEbmGe>?*Z@DRF41v diff --git a/tests/queries/0_stateless/test_abxi8n/parquet_decimal1.parquet b/tests/queries/0_stateless/test_abxi8n/parquet_decimal1.parquet deleted file mode 100644 index 19d5e387f8da6bc9675ea6fa63d2f021662c3e1f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 29278 zcmdU24{#LK8Q-@#XrLk9(Cqf;q#6}J(Ln~RSQTtzYg_-MlomT!br|_4pw;3()LO@w z#uOt)M2s;;h%{nMuQWwOs*%YsDaABmN-2X%8HQ4d7*mQ&hzw);eYf}azP)|N-Eq5Y zxMA|{-oAbB_wD<=?|a|A_dTXw^^*xk$Os!nWtfCUYFd7A$J|Rr=gnW$49ys73?EDR z_+@BuBt*H95M4L5G3=M7X?gh~PlpCkY!r@~7}AE0iinwpZidVy!dx!kGRQyC{iCc_ zbZRqX=vW^nCB|q8Dm?k{S=W}`*r18eDr!2`171%CmoKkG@?}{BsmQg#_T@3KvtnS| zgxMkBunueW;)vqj6g4X0mm1?F{1Q#OLTHoQu6ud@!_z8e9KF5hRz%^JAxyda6A35Z zg9tJxg>_gRvhhny#@R6$8+5Zqhhsyabx3g}gSs+oQTYs9yc!r7IO^)U5alL#mH1_g z(KVcjF^BQ$z%bY_44IX<2jQ|`ni7_SS@L;f74&h#%a7@J?Ef+{%YAQ(8kO+f%!=cZiCYFmKg+^QrVL8iG9C^y8E<3ET5|^MD^xP;BwJKI z0~fRCk-C{hxd~<^e&IK3fRq<8G3GI5HGBnnMk2Ek_YkuZztqf%!P4W3lnT?w1&j5+ zRYKF8{V#JdMVKKCwwes){20urFxLs_G@#yaFiPH=qDJM@j1342c6hR^K&CGE3Yy5W zGR4L-D21gh@s@mziMJpo-f>}8>aZmOiz7-h$(CfAVT(%fbz^9mVTmsk-6gb$pbm-jLq2Fk5;wS%8&hrr*gUDV_%92D zuE7^)G7w~Nb+b~S&4v0i;qccj#i*`93CTfUYK}fGSW*oR*b0&0DROO?U1u|xWNLNO zqVJXxs4GGGS}mb?_O}!@Dq%b5E31;J3w~H$k!i%0gN?iL+@);ss?THMk(Jd=i@uxB zg{pIrzOp5mx^psYQ7OI`L|QtAi9i-s5-EBOmB*VH^{WjH)Ek&aqH8)^|PV1(8-wU?Pw~ zl|EJ|Yndm!_90t{GK*T^qkjaRpHu-C>V3^6E>LwY~&dZ_sax|!pd(!Jcdf4(o zCp=inwdQo#e0(-XdHAtrB8`(72+M=D3~JjXSU$;~57G@uH7FrD3@Y739~Z1F^&rj% zd%wkCTE44gP!C=SZ(W(Q9xOGkwtd%Mt?PJw`YR7jdc5!cf3_ofoFO?sE^!oePOd{u z=j2k@4)zMZSLpf<6W4ZQn{-%w6*QMB>6~0whAk?4(V-Vfc$dhGN=dYk#M<^fCW7U{ zTEyzT8a7{@HL;W~tbFm|g2xCZ%OFfeuKmPYuVHYx3mg3b^nBkzurxI)pQvNTMo*M~ zT<{fy;^pbPx_gsL;csi0_g>G$v%FVp#@2Ej9K2RZsN}jbY*8t`7lb-8g^6H!t`?z| zPJshI9CU=*_G1Ru@?fpEZCj^8&D7JLP)mQx#N!@p*ECo%&38gM46a)}IlWOi2>Pcp z%n1)RIvuK~XYaw%_>d#lo+78wi#X-2xQT(_9_)&rLEnv8)A)o2*M`M3pB&aseJFD= zwPxnsw=kI8gYCK*8g6#bSKgbVMx{c?NFUnpJ=TZV3pKYg@wf-uIRmO^_)g!H`p_0R zJ&~5rW+HG8wsjUPo|PMsqzW5;9fTn3YSu)F8LnmI@$*;e8k5y_paegWwq zlDtW#F8JQPn2_V7aRC#Ld$7CaLBl*GlEflYcMcg=c~z$FQfl$NAkyleGZDB4yXg+7 zzauvyN!{S;iR?Afk-Hd7?!g|q6WZ@Q?KRTcdzg6KgY8`iYZm&hk<W!EU_|qW7ILgWB`}1Hlby`y!}YG-&91_#p<98`RzhVdI0RJ$+jrVd8Ov+Ve0( zANHNTUg*K~<$N`tThwijLd&CrhDlYw<~X@Yt*8*N?J?mknUY^DxKw#KxkSX3LnbbL zDSPtSS;ZyfR<-JJ0h=pP!VId*uuxRALRrAR%?Q=;-~vfr|V zBjvVr^^*ensy$|uoD|5^%_wRfqMrtVUB#d2qQpXL&B^V(7);NVllC|Qk6 z-8mUHs_eUyGO)Xrao7n9yL6d=-e(ZB(=IC;PD$kgd)bvFPW_g+K!cv;90wWMg7~`Y zV%+Ss=UI=bY$MBZmWW8(pXW%)&KAU%UfXH~EU7(3RJJAXj#V)#qlA?&atX=Q7Q|Oy z+g}h+{(}Eh_JUDMS8yrF*h)r4UlLIH(%@m#zDACe%&lb9o|OVxSDyZiYHs2ZlDU

hQR`miQjocojOt!3pn3J+VbtOlj+D%;WK`vA0=75j&Zt|BDP_2&hW59P zFHN3!4ba|9Y2;*zHW-KXSmQ*BE4d=IIZQ5vErbg6yr$vM|9dVs8Q}tRqi(k1_S2|< zEx%LqEV;f6TUFXZwGp;uc3M-ZaQC-z*kptYVhgqPtpfW0fS{eXDH~3yGHnk#^;A&K zVfxl`9Ng1gwif3wfApB(HnJ>@FuoV)e1jw9&hBR1!dJU4cY-@!`$-PV^c2h|YyQL~ zE(P~^*S{^G=Iz15sNRhnDff8~z9ZnRcTRss zt=+^WN0FpsiV~9);26oyD4FJY~`>MK5zef z0(QKIpq;lV8%}9%v)CE6y_@48a~q2LyocV$*~**SO^)wPQwC37nowdErLbsy!s9}<*@2=zD>)ZQbQ(am0Uti ztMsKT4T;i2Ll$>xSkD_m^HZhTp^n`o83I?|BoOJ%u!YfGKRFTmly`d?fMvbmC0 zoAzR0#NNR}t7HG>NXh0(TJ`P2_xk(1pcUK5ncOM%ggXA^6fkV}$6R8v!7+y6PR{xR zIB)%r{|r;xV4Ge}V;Zx;NU5$K4x4OnDCY8(_u$m%69nzNEjx{A+y6AwMn`g2FUP@M z-u_PobbsnGwQXcs+F(T5`ah18yS&ju0*?PTcWMXi@*X+DCFCw|>t_NE9VRL#k1%(R zVmhg?)LAM^yWi3?s=S{|!Cl^lqc~UU^M+CG>Ahr>u!CV~I z3uyeDF)Dqf$`N`ZM!6+O?%b3273566?h7s9^msPYM<^b@jRRMTTR z+xoy=-u7}G>dG;)J~b_ssQg;y?f*4L%3aW+@GX^rf(J zT5B1d2i?zd>8(}bJ-XSZL-jHpmOn!yA$@%rwyJzOFaEOKWA(-jL!svd z4&7Qq-YU!;0`}MGP+ONf7fX$B8SdvGDwb&4B7D#UJa(Rqy}XsI`i7}##-+#aic*-1 zuNd>f(C`wM%UU@`2O4L*J73fx`XaF`eRagL^rh4qMa?o%weqA`wzL2?HFA-xWn_zG zyH?_XKP&xV8BMpG10q|N%^3n4e#@a->&h0()~?cF@5{NfEF=4#l5aWMcij7Od^MNQ zT0yqr*fzX^I=xB3x07#FsntS8z8q(INV8|jw#-7Hu|`72()c(DLCk-&_cd zujz2S8BsfLQ?#s)YMZDh@D;c+r#`Y<w!fO^VkIl^3Vgw?eqe(h3s(SWb;^IQHaK{m*)_G zpQi-`8mkXpM%ja(vG5RNBszE*Lk@mMdP9(L);R`DTXwkOCbGjrX$*TA`V4*sC_|9p z!Qg3NFZdjO3jqheg2!R2;B#;(1RUB4Imc*P!@B7K8rJzVtUZpl^gAXp=!it{IF=B6 zj`)Ls+Obq?*R;Y2VU+w z>CEmh`roILMHEE3)5VQ}14{IB&`s9OYy3G$XuC0pC zC67g6@uv?WMSPGDq>V+S1noocp%m$Zh9V_|KC~jz1}O?!jpy9ixpQXk-Da`7?2_EQ z^Ub~A`A+UP=g!U8ftQ9I+lf0pMaOdNln{2$#>U2qBb-jByFeIrICy^PkMQ6r~jXRcWxpOi1bPR?de!uyd0>-c%OJGd;67-dH(t)6O7DLlV z7Zvw!pfJ@#ajGApO62MmCh~|Bxe#+N#NcNO=B;cvBka{-G|J97`9OB^nX0(pOP8Mv zZmBdh1cPgsry(l6Pab0FO~~}_#oRdyu64kfj;uaYM;eT1Oh00yn|~gnC9Sa$p`J}G z29n?Yl}h2WZkEDPW}|Pr;G-@yQe6h4q==FJ8n`|>|0)`+R#rT?pe_n zi5cm)t+2Eejik@5!)TNpG1A(k*LU7tDS!C!t=Cb2EDNHgWT}{CkDXK{I7>tr4 zM#2a7kG{%)z7kY>Q?1+)7b>^S1Uc7N> z|80|zYDcL0pytrBZazw-cPYctn`TD3w+)uIg%wHh|L=&#^l@to3bJM2nbmA$-I?|9 z7n7>?l<>tY6Fwt_m)&<^bvs<#o~1c8^NismK1S+2ngE+nPqyUa@Jb0;-ob?MeRZuD zzUf6DhjVHejFKWg#;NqtWySp)C{6W{T=hc~$NAfTXDIjRLT5&50lAv*0Z}| zy&sLF&#l8~lpQfr-+qjmKIQZJd?E}JPpC%8m5m@>D?G)>G7TQ(*mpx zpplvu(U^Y3NaDGRV_M2Kp;%S0B^!iSDurK$SPFb+og0GdPf$gw$6%BcF%kypT08!e zz7kYFQ?1+&tD1EHrV(+t8@D!!}RfGDV zMyQIR5HgZa6Y}VzaD-ox?morR<2&oQ5mpVFzRXRTIq=l2fBdow<=VRR{Bvp8gS`3JI zlJFxE&r3Y-<2k}7Bwmzw(Z?mvQN&Y{2V2HdLOkyf54Mcw7~(~Tc(7$WlZYb|;@C2t zDalVfsR+-s5| znt5D;Rvv1gokt3I@MD2{_hHw?9 zX1<(3D_^0Yoi8x(;4dNmbG*ERo?}b&?<|j>c=PSjTZ0qja_*hdiIdZB%;tvoj_e&C cJbduL*x?rry_{Ld3UQwwaSiwOG5nAGe^7mF-2eap diff --git a/tests/queries/0_stateless/test_abxi8n/parquet_decimal3_1.parquet b/tests/queries/0_stateless/test_abxi8n/parquet_decimal3_1.parquet deleted file mode 100644 index 39acd4a8c14b4c0432621d7fcfbe8f4bc13a2c17..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 559 zcmWG=3^EjD5oHi%@&OVoLJSNHYHBi~e3GUt3~Z7TjESO5Vhmz3VsdH>Y8;v}K&33A zOh85GdiofVG$kQvD!`-Z2ePJQBu%#%@#uQOgk)6;lCEP+$hw#$V@w$&*ou=gQgahU zIT%FwL^(v6fId>?0{T=)Ck8}rXeRUqODAn|0dxDQy|2Ovt*%!*ZrFfeE+6(klFmZp~ICKnVal%*CGXXfWA i80wko8R`c6IR*rK_=iaP$S^R#5rZEi1495XsQ>`uct6?z diff --git a/tests/queries/0_stateless/test_abxi8n/parquet_decimal3_2.parquet b/tests/queries/0_stateless/test_abxi8n/parquet_decimal3_2.parquet deleted file mode 100644 index 0de83a7eeb6422516f92630c7129dc558c5143c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 777 zcmWG=3^EjD5oHi%@&OVoLJSNHK$=ffNJo?jD9$FzCS%3~6l9kGkwC1LBqPcvY0ARD zCMm&~D9R+pBo-nTrKYCF0a7Tb1JuMK$^ljA6vxjO*Kui@xFL77TMrQmPaV>V3}IsF#$Z82JE1OV8VLpV&l6ClP+9Xj#OZsHThh$BWsk_dz z?`d!c%->O*GLED(N^oS+5YZ4uJ#mfG5_*>9!x`NbebTl+G9l1ON#h>+jvFrhG@ zF7{oNKh`CFW6tmDaG6Si-!UZq(VRb7g27S}{Bc_12XPVW(@n#r5zDt&VP*@{#pM>i zs5Q*s2HInOh8CHhhbs1yt#Y|6O(_W3SB;-(@Xdq)n&lUS81ACMZD*vJguxxHKz}86 zaTLsXzAEv<;GV9+G?N6sos;-sa7S6_XOrNcTM|F~xK1sY=3@CpR+^c|x_51nSravp zHM@1)YIa1y;77D*FMM7$+1X8L>OpAVVM?MfU4#Cb8+|^i{NcEZny|~InT960U@&B{ zi>zS6$-2Z3P10M3E1m?u>qz|2Bo{Ue?Iie<4T&Fqq`eKea+2V8U5PKP|0WDKWBAlL zs1n;aX*_8@t20wnD3ejFd$(_3xyqLh#1RdNx+U~OUoZK3DbR~TZ}__B>t3MyoTHyO z+;lw0OR&cNMcD-PN(k!3jlWNEiZ9APrwWANJFC|(_?Pts)Jq1aR{{P-tim|WN2vmC zVLVVz^q`)*;eX~#P)~#Wt&XeI&hZa=K^Lo}p6Nh6d4YO90`(Nb$GC&W7vkS(!5~|= zTJ__XubZ#78?9FHP4oEV>5J#Z%1(8svc30Ux3+ix;lAVHPHp}tpKz;Q!%hAVm)RJU diff --git a/tests/queries/0_stateless/test_dozlem/arrays.arrow b/tests/queries/0_stateless/test_dozlem/arrays.arrow deleted file mode 100644 index 9ab37ff58909f6b01e1336deecf64e1a08f073b6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4834 zcmeHLK}b|V82;b7+G1NQBBCI%f`=_Oc$(E_61zpL%EK#wCg*KfbKnX=@R0$?qsuk-!N)a1;^V8K_hhaN^4aV8t)6tH<5=@CB((! zp*VD-uDCL1pwxUg^H7CnQ-vR!Me8NRwxP^Gx9r6B78>(UXxe`7KM(vo@Z8Su-J78c zDy>y8<|Syxi^q&`%mj3CCuYI5@Ew{c^y|grp60k2=$1=}JMYA^ICjTw(Rnz8!}S4`-2yuIIe$kG{XBNg zNg>_loP;L&`6vEWhQIT-1(gmLT}z(w;a)y?@O@#Hu2uehg~Z4CkIheYIs^OF=%~W0 za^EHoNLe+oALDRx^gQw|&lgf|&f0UWe;b_7+1STzf)P#`=UcZ47Jr@_w+R-1+fAY^ zSp1^~FWAXpSX%}!Sp0hpUagAly>`{+O&M5#k*Lb)LD7;5(SPQHb2B^%fubn-5e!5}x!W;YZxqSTWgQ1s88x67Tw+|N% zJ>H5rnu}4c-Elq~I~#tkr^?;m-7p;Ke%jTZS+P6V2UEr80QKJV<894f+QnM(ifC*PTB L1_>ml{6y%82!?uNr*u+6bT|FgF^?Usg{yKC5S_Da_o>Hg;o=&6>NlV?a-k^2f+@G zox~p@D8<21oU-WbB#2AE>hqj??}^VFLw3nm?)&cf&UxRx@80{p*Ca`j6tiX(qm`^| zd7HH&daz6m*no<-=EBwNJkZIESqp6eZICg0hjxtCMysO@pm~q>0(dT(XfJ|ZU`%MMf88hFijsJ(F=(LA0v zk3U+*Ui*98nxylSf7;zOZ9P>2pFo*QDXrFz42gr?X;n?hF;& ziLHe5?4P>^Ui(1rZig>ITeO0^4=}?#7&fV*9_G-emF*(NU`3Q&X3OT>894Mbt|j0y z^81=O!`bKj;f6Lp?2UG$# zFD$+p@xsc#8u7xq|9->^i(d=8a68fqi|+*fLh!qhURdk5AMwJPKWV-fwO_=v{|)cE zr%GK$0yY}4Ke6Z4gEzeUy>7`(9fPKZzI^zVh5 z^j` z_te)#T(^5ZZ20luEC~T$ovwadgmpv#J#5ci;V0B7e{76=PZ_MaFRj_>_xUC@%t!wl DG${*- diff --git a/tests/queries/0_stateless/test_dozlem/arrow_all_types_2.arrow b/tests/queries/0_stateless/test_dozlem/arrow_all_types_2.arrow deleted file mode 100644 index 04e1214ef847f695b4ed47d72d8f6394f7143569..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1872 zcmds2J#Q015S@!J=aaF(MHHP7imsrbz={n~lmd{2sQ3jGE{GHZmP|n8km%wf1w{%Z zQlxYhgOTqi;uL%NXwr&zCje-tczU+m5$`_9R3|JKR(r5>4_FYRx@Sn;#~ diff --git a/tests/queries/0_stateless/test_dozlem/arrow_all_types_5.arrow b/tests/queries/0_stateless/test_dozlem/arrow_all_types_5.arrow deleted file mode 100644 index 29de25ee984f5d4510ab4eed2692df87dc5f5984..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1816 zcmds1u}&L75S_CR`$VkB1&Umt=t`GPf{;rW6etxX1qDbLummPbE)r!*N{SSet|FI| z6ci~aTw1AIAxbK03X;uxdo$*=;(owm&CR`iZ)a!ccI}*VNmZmNv}I{XO_n6V8!U=} z1t`HY)~Co@pixC+2i}H1MIyKGpYUCH2mS=j1bzuWf_LFDeEiorB(8A1RUJd{x%c<7 zel|Gi6W>9XBF+l>7BF+r7^&y`@;~R8FGgh1VeU8J4hCoa{cqXV!SGOK*mI{uTZ2|* zSc5<8GuKv`w^&1+d5=%`vh_9jgPbB2IWK5a@K*0!`{^S_YuIlMKUyR=`5fPqVt(@9 z2ScWpM`(J$4<*_qa4MCV*GNQt%h#3n`?cIG=QA`Zv>PQ_&Fw2+^BRe$&r0-G_lI?n zU(m2`qY|y=rc^F8H}%tkzIaZkXHUAmhq3P=PiIqBd{68C#5VOj=lf2<+jn5}(-9MB z6RE?!MT}V1wQwI+@-ZmhVGZ7hhR-b3*q?!W<#}gXV0b%tXP$BPSwF(u-p}7&#&ylP zj&o=>kKc-BIQ8fSw-w|1dFyC2eOIyZy@20TY<<}Z`0aq-4frF)wvR6Xe-`jR0{&dF zt$z{lmx?X_DB#Bdf2G)hVe7|rp#N<`jfPD>(Q)hFRI%|h#a2Hd{}+v4)TrBD*ueYG G&HV=)4Vsbw diff --git a/tests/queries/0_stateless/test_dozlem/dicts.arrow b/tests/queries/0_stateless/test_dozlem/dicts.arrow deleted file mode 100644 index c2af6dfedeef8163be1745e3cf5b28be78702d67..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 20030554 zcmeF)b-0{qeK+vmopVm)WRlJ9X0urrc5&z#p98_&DJ{jVfWwUy3hN4l0%aBO5eYEvwWWG`evS)d2Hs)BX>T(d*&v0xyzmI)~}jpcy?9o zNUD~rR~u`oHfJnDTdc+0yf4#qw=LP3?5N$DqH0s7-jPQR&s&S#b2m=TPP*y4p_}Ic z$>u|IEFSu|cj$gE*tk_(cnBo`-FORkYzE4fbc zBT4>rHB83IG+9lqpWGn1QF4>yX2~s*TP3$iZkOC4xl{6^$&V#>P41T5Be_>{pX7eY z1CpOgekOTP@{r_V$s>}VO@1!6YBzu#6$)(BV$+ME@ zB+pA;kh~~)N%FGf70IiT*Cek^em!}8^2X%N$y<}RC%>8ecJhwooyohC_az@p_9uUk z97sN%d@A{)COm36hKDkqJm*lR=-IIGI z_e~y<{B-i5fsO)f|-O0Je%Gr3N(k_?kcvYPBk zZj{_KxkYm8zVeDb8^smU)V&rJ3umnF|io}0WNd2#Zx2ZOplfiT_8>|jj zhW+7SI2?|KqtSRY8BIsC(du|*+#e6d!|`Z59#6*8@oc<0 zS()@FgUN6*nv5rt$#gQCtWH;^{pnyjoQ|gB>0~;c&Zeugm05o_mgvjBHfS{~Tg`%2Gtp|cd8OOzY-Kho+mKae4Ow93&*X<*Qum_rs`9e(y7I#E%JS0s ztNp=B_sk%#aM&M=`h#(QFzFAb{lTn1SnUs2x|xP~L-KM*{o%Mjob-p&{&3bGuJ%VO z-SS3*{wVKPUi-K|n)FB0{%F=8t@g(&-Kxfe{&?6Q=k3fJG3k$|{qd|nUhPj-x~-TD z`jcUQGU`wAZs%Q@_9wIcWVJtC={9gW=ue0J>8L*)_ow+|p(hO1JH^L4P*v z&qn>(xIdfpXZc&@{mR?3l9!b?slPhxua5ewf)9`b=0 ztPX}N-CLMvhWTKO2E*}SI2jD5gW+s2Tpf&7x_37-jq*9lr(`@BO$MXsU^E+yRtMvi z?v2m##>2rlAG3U9CWG;GFrE#@tAoi(_czI^Cd0vGG??U5n9t92FqsV|tApuE_vgx1 zOoxN%XfPcQrumTO!!#RAR|m6|?k|}QoDB!F(O@JKtoUU{q-uz9ns_AGr9S^73d9&NihSSyIY^D2L=g*a`n2mu29z>Gp>Fk+bb9qtVKEv@#j3 zOh+rT(Mmpr*)6lL1aF~jaNsLm2Q8{$1v}1W}4)vm1EL$G?|SitE1^kw=3rpnm0bnn~q1*9Mp1v znvJHbquENgm*>Nqze!d#8;@p_(JaTj9IbM2TFC(*pX>a&vK2X2jaDb4)#+$8hs5lk zd7X#bqiZ=e9BudQ`oG(K3yW!e=G;E-E+D&MIropY^Y-MCuAO&q-aU7C$R*MLbJ%}m zIfvVB$IjQy)N+x!ke1w1wxVmTZpr$#W_{agd)je3(vCYN?YNz3$L&fxZg<*oXQmx@ zcG_|0rX6>F+Hn`A9d}XMaaT<{?&@jBT{G>tYo{G|-L&IY(vBOX9XColZjyG~EbX}K zr5(2??YJAJ9e3lj<8GRE+|ARDyJgyOw@y3mwrR)RKJB9rr6~$2~LcxV5z7E=fD?vb5u_NIUM? zX~#V`?YQTs9rwbt<6fM0+)LArdwJS%uS`4c)oI85YT9wXmUi6h(vEvW+Hr46JMJxM z$Gt7>xZg-S?zhs8`<=AoemCv7ccmToo_cRua__GXq%HTM`fyrvzgNGX_S_%VN7ADE zXnibgx=++6)2jP)eJ1U?KdwJX%kIzWvuWFXu0EgE-52XiY2SUNzM2-^!TQUz@&3C0 zCat``tFNV<_l^2yT6*8E@1(8w_w~KB_WrT{Deb*~seet2@89b`(&qcG`tP**@)2sS zZD26=d^R% zdF}jmLA$VBrCro6ZdYwrYgcdAXxD7lYS(VpY1eH((pFl(4cf4c+PF>Hw9VRTyI#9~ z+tY5)ZrEJU<3+)%%FSSRv$F#?`$F;||C$uNFC$%THr?jWGr?scIUv9tBp3$D!_O`XQ zuU*nEZI`vn+ZF9u?b+=)?YZrF?fLBm?S<_{?ZxdS?WOHy?d9zi?Un6S?bYoy?N{4t z+po1>Z?9{wZ*ORCY;S6BZf|LCZEtIDZ@_WAaO_Qm$4_T~1K_SNndS=l|jS zKb-$>_y6twzuo`0b2YBU)wmiz=jZ&KpR1EmC!FIp)V->ERrjjyReihq zcJ=M*+x277k3~Ng{aEx@(_c-0HT~7}OVuw`zf}EF^*`7DT>o?Z&wbB?@0svD6TVNz z_sRG^8Q&-4d!u}Bl<$r5y-~g&&iBLlemLI`=X)T152Wvb^gWQi@74Fc`o34+_v(9{ zeXq0cb@si^zQ5h~xBLEf-{0pZopurxZWu z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`FS;d z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^XQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAJA;;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax&nQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSz`Hi3RbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QNbIpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzmCw)X&@j)ima(0O?^dsN)y7(?%^B-;k1Z_L?%W)hxAiHObH7)MnPNGQ z>`8Vc2j}Zg=Vrsv(c{MUA6ZUSx5Q)T>n3iQdm&3-%2sU3I<{mjTeH4xjUQfNAdcC)odOh`e>h;v?sn=7lr(RFJ zo_am?dg}Gm>#5gMucuy5y`FkK^?K^{)a$9&Q?I8^hM)6ue$LPNIX~y;{G6Y!G=9#{ z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^WSXz zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H+ zEPl?<`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^A8q3=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`G*@n=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{L{tH`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAEp3#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luM>#n1UUKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}ODQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq^fV=dL@jP<(578YxFZVt@b`V`B#->bz;v7ATtBs-FW^Yy25v*GCI zabx?BET^hl;<58}6SvI0kfkqWD>h{vTe6m|S>LvDz2SPp^@i&W*Bh=kTyLCs*Bj2~ zduy%s)g^UlT~?RZ73F;HeC~YieC~YieC~X%{$BmP`g`^F>hIOxtG`!&ul`>Bz509g z_v-J}->bh@f3N=jyQThK{k{5o_4n%U)!(a!QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=Z|XqoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J)Yy6y_^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=T9zv&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS$zie$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaD~g}QdKj-KCoS*Y^e$LPNIX~y;{QRWG&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^etu5zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^viLba=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-JKZv32|^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=hrEI&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqvJ4(9C94GG9HKQ=X-0d_SGeIX#5gMucuy5y`FkK^?K^{)a$9&Q?I99PraUcJ@tC(_0;RB*Hf>j zUQeA2Kj-KCoS*Y^e$LPNIY0lA#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luKz#n1UUKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lq{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}NqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}Nc@pFF8&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxKhyX*Kj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;Unzdh&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aazi9lNpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqyl%OFVYIZt_+eY9UKs%2sU3I<{mj zTeCjb8?HB8Z@AuYz2SPp^~Q;Kz2SVmx7KQ3T~e3UWp#NupF5vBpF5vBpF5vBpR2!D zf3N;t{k{5o_4n%U)!(bXSAVbmUj4oLd-eC~@73R{zyD6DzgK^+{$BlkO08OJ8`?r! zY)frp+tfCQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T5^Be$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaClx>E=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T748Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzQyM?#=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`FXGSIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LM?FMiI?`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^X-kF^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luLxji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{OscA{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{0z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxuU-6{pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqF0H5m}gka*v`Xut5>^fV=dL@ zjP<(578YxFZVt@b`V`B#->bz;v7ATtBs-FW^Yy25v*GCIabx?BET^hl;<58}6SvI0 zkfkqWD>h{vTe6m|S>LwWUMFQuC)X);YPR;YZ0G6Ok~8YeI;$Lq9ETi-PR!$w{d{k& z)xNrT=cPs>@ZEt1eevuDV=xx$1J& z<*Lh7m#Z#UU9P%Zb-C(t)#a+oRVTyG`8hx5=lqQdKj-KCoS*Y^e$LO|-uO8`=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{DzI6^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMGji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{PxAq z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBG z;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0k&QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS%QW_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax@7wq}Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxA71>NpYwBm&d>QdKj-KCoS*Y^e$LPN`HAW0b!eDpSj*VX!*{D! zyJ}-C)#i-#y2ln4YjWgT0xmaSRew%T4NWlbm7DRpYL_Oxv0>DiJq>dZQ;9ETi-9EVQK z`n7x}+|x%SyXKyF$Cd`P}*3`P}(jU9P%Zb-C(t)#a+oRhO$SS6!~UTy?qX za@FOk%T<@FE>~Udw@Y2Fx?FX+>T=cPs>@X;!_WCSKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LOo+xR&@=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KFUHqJ%^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=TC0@oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-ILi=Xpze$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{IcTb{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~al_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxuW0<7pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSz%Zs1$bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR88&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^etxau=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxPgFmzL&H47TE=!B zzFWQ8RU2!mHfOBYJ+`n|yK{43-qxpB&i!63W{TxJvM1S*9GtH|otq6uM~@rZe`GmT z-4c(Tuba4K?u9IUDO<5A>)4XDxZZHR;d;aMhU*R28?HA_)awoB^S!lJ`|6Uqw4BeK z&z;Ym&z;Ym&z;ZJ-@m2Q->bh@f3N;t{k{5o_4n%U)!(bXSAVbmUj4oLd-eC~@73R{ zzgK^+{$BmP`g`?o{G6ZjbAHax`8hx5=luLSji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{B4b&^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNo`8CKJ)D7!Kb>q58-L!62H?LdN zE$dcw>$*+dwr*EkjjM4ruEy248du|LT#c)7HLk|hxEfdEYFv%0aW$^S)wmj0<7!-u zt8q21_P)|DRlijIQu#SQ=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%Q7@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`5hWR=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`Mrvt^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luLj#n1UUKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAn#?SdVKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0ka@pFF8&-pn&=jZ&KpYwBm&d>Qd zKR;3ZybcZX3~L$NdH8PiYFBNnrP`daUia9-V(re&fq7e>VmbGFwU{ZE^T?iLM{;n! z{&a3O934GwZ2ytvRCP-{cD`=nmbn+Q^rdXYrmSO2*5Z1@^@i&W*Bh=kTyMDEI8m=R zoX_{xTJ5V#>e6yPcRqJMcRqJMcRqJMSAVbmUj4oLd-eC~@73R{zgK^+{$BmP`g`^F z>hIOxtG`!&um1iKP5r(4d-eC~@73R{hvVn`oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{QL>U&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*UD!&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqPA$^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=a&{g=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aamlZ$f=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5z6pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR25&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqJi}VXb{@W4z1me9YpFJ8 ztk*rZuvoiub70=qr&!MYUM*&dfkDafZxMl8z zEPW|ku_^1=lC^Bj`nJ{fIw@;9xlXB5v$dyXJ5SG+oKbwA@AG}W&-eNMwHx2(`+T48 z^L@V0_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1uEzKI{*~AF`T5>jt9^Az+0X6g_H%VI z>SWZ(sFP7AqfSPh%v+l}8Fe!1WYo#1lTjz5PDY)KIvI5`>SWZ(sFP7AqfSPhj5-;0 zGU{a1$*7az=lqQdKj-KCoS*Y^e$LO|Tl}1#^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=X)AI=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`E84z^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNR#n1UUKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBG#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0km@pFF8&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%QW@pFF8 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq)4XDY|Z+%)%H3mYdX13sZ+DHr)4`&&z77~t~Xq7xZZHR;d;aMhU<<0f3G*3 z&-d0^?W;@5{}cLuLjOvq@euG?L=yKZ;guAhy5Hu~AQd zKj-KCoS*Y^e$LPNIX~y;{QTREpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*T}u&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e!ivnIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LM?DSpn+`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^R11a^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM##?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS(n6_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax&uaXfpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;*C>9@&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqD+8MI(ppL{v*q&>XvxyeBHz? zb1!7+OWBG|S%>40h;v?sn=7lcde#gPraUc zJ@tC(_0;RB*Hf>jUQfNAdOh`e>h;v?sn=7lr(RFJo_am?dg}Gm$?$W2&d>QdKj-KC zoS*Y^e*Tum&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=kFQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^Y=A=&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`NxW%^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luM3ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{1c6z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luL0#n1UUKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX}NwQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{0$@pFF8&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq=jOn?txvI>`@LGs6w7&JPqHI9 zIA4D{Hye(Q9yhlC$a1Q>B_2CpH*w3{3t9S7wqjG(;d;aMhU*R28?HB8Z@At#k*_zL z&-d0^?JMVV=X2+C=X2+C=X2+C_4n%U)!(bXSAVbmUj4oLd-eC~@73R{zgK^+{$BmP z`g`^F>hB-c)ZeSWSAVbmUj4m#IDXF0`8hx5=lqQd zKj-KCoS*Y^e$LOo)%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KFQT&{r^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=TB(-oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J0i=Xpze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCd|&Z%e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS!c@e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaOB+Au=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6eR=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-N%3=j&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNd*7!L;=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IfEPl?<`8hx5=lqQdKj-KCoS*Y^etzZl^Ex!lGpuE7=i$56t6jCRmTGgxdfj6S zi?us92j*>kisjtz)ncYt&Lex09m&D@`qR1DaCG#zvHeGuQ`If;*!jANTjpNK(wDLo zo3f5ASy$b*TYFlz^Ym=V8FglzRcF^Zb#9$k=hp>wVO^yzs*CHY zb+zJt-0zx=`*A<+$Njh;_v3!tkNa^y?#KPOANS*a+>iTlKkmoQdKj-KCd{+FNpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa>otDP&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaTNgj)=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0kSpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QTpM zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*XF5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hwoXXEGmoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmTIm=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-aPf0~&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqVmbGFwU{ZE^T?iLM{;n!{&a3O934Gw zZ2ytvRCP-{cD`=nmbn+Q^rdXYrmSO2*0MG0+g97_q^#-WI;Bp{)}EH_JUv@-Mx9w_ z)!B7Uom=PC`E@~ESXZfw>f*X;U9DVixZZHR;d;aMhU*R28$a0V4d?T{wN^ZU2k-#b z?L2@7@Br;|?Q`vO?Q`vO?Q`vO?Q{L<^rzFGPJcT6>GY@5pH6=|{ps|lduY?2PJcT6 z>GY@5pH4j-Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zk1c-A&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa$2We?&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=UW;-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{Pg1I{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX{1K@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxuU7n=pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqh;v?sn=7lr(W+GO}(CaJ@tC(_0;RB*Hf>jUQfNAdOh`e>h;v?sn=7l zr(RFJo_am?dg}Gm>#39B=lqQdKj-KCoS*Y^e$LPNIX~y;{CrycoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-J~Dt^w-`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^VP=B`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAw#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX^#8{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;w`u&GpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;cPoC* z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz zJsLmf=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`S}+bKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzgNmQ?bAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QPUh&-pn&=jZ&KpYwBm&d>QdKj-KC{6zQjIyB5PtYvKH;k(tVUA3{6YIDYV z-D3-jwL3Ql=52k7<=pSpVy0NmBYTn^$-(*h)4ADjbo98f{YREl)h+SZ`MQZ)=3dCs zm$DVEH(YPH-f+F)dc*aG>x~ordc*mAZ#kbkpF5vBpF5vBpF5wczgK^+{$BmP`g`^F z>hIOxtG`!&ul`>Bz509g_v-J}->bh@fB%rC{$BmP`g`^F>hIOV@pFF8&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAf#?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lm;^+LFpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~Z6{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;dyAj*bAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{Csoc=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw|*Z4U<=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCe0SsL z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{0% zQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS(nG_&Go4=lqQdKj-KCoS*Y^e$LPN`IX<#>(DUI zu$Hl%hwoOecGbpOs?8beb&oA9*6!RKn78#QmUF*XiUC(b40^ z_8(bJRky@r=j$eJnR_8iU&>Z&$~v}WEnBm`ZMD5l%9>8DQ|i=g?P=N0)3YUK)R}cw zon7bDxpiKhUl-Jcb(OlPE-voJ{jT1)ANS*a+>iTlKkmoSWZ(sFP7AqfSPhj5-;0GU{a1$?$W2&d>QdKj-KCoS*Y^e$LPN`K0(c zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zv&PT)IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*W&p&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmSPabAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZPy76;<&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZvMDcTe&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZPz43E?&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6XZu=qJY=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqVmbGFwU{ZE^T?iL zM{;n!{&a3O934GwZ2ytvRCP-{cD`=nmbn+Q^rdXYrmSO2*0MG0+g97_q^#-WI;Bp{ z)}EH_JUv@-Mx9w_)!B7Uom=PC`E@~ESXZfw>f&;};d;aMhU*R28?HB8Z~Wk|H=NIT z01w~+uG@J458wgX=i2Am=i2Am=i2Am=i2A`)9FvAKb`(``qSx8r$3$kbo$fjPxs)a zKb`(``qSx8r#~G(=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luNZji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{L#hF`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJA>#n1UUKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAG;5{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;&n$k<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-H!EPl?<`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^V1tY=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`NhT0`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJB1;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=RXX7UWbNxhP90CJbbr$wW~JPQfyiCj*cETw*Sa-s=6f}J6|_(%iIfDy5o@JkmJw~!*OV1 zZOXA^OO7U6b6nY0+v}tpQBJN?>eL*S)bXg}QOBc>M;(tko~t!=JnDGV@u=fb$D@u% z9gjL5bv){L)bXg}QOBc>M;(tk9(6qGc+~Nz<59=M&-pn&=jZ&KpYwBm&d>QdKYv5x z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8huy z7eD9c{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`R^7#=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS)yK_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax|FHNuKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax4>W$x&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzXN#ZnbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QPc>pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPH%pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqyiCj*cETw*Sa-s=6f}J6|_(%iIfDx^{(jg?5E@g?5E@g?5E@g?5E@g?7b>(5~26 zo6QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA{#?SdVKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9W#n1UUKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAG;9{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;&nSM*&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKi^sWoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-H!D1Oe*`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^IeUf^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLdji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{G#IL z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mP z;^+LFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=RX{NUWbNxhP90C zJbbr$wW~JPQfyiCj*cETw*Sa- zs=6f}J6|_(%iIfD`ck%HQ`WI1YuTFhZL95dQr2{Gol>V}YfsB|o}Mi^qt2|e>g+nF z&aLz6{JNkntg96F<9=6d+>iTlKkmoiJuPf?V_3V01J-41$&oBLE^_$gimY?%;e*XH#&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAGO0DnIAv{G6ZjbAHaxN5#+i zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*TW) z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwI zG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^LI9W&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`5zQN=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`9~T*|3B>9XK*CldKd8j(ab1}T48s!TJ0NeurbcLaF8=pIcF$v&gGnO zfpgYK+VH;by-?tspujmnfwSdMa?ViBmIIV?gPfK$pRmq=kZTNLnSD)@V9X{+# zpQGRZS$@vX`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Y^clbF!=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KFtnqVx&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmWb(bAHax`8hx5=lqQdKj-KC zoS*aaSB{_Obm?oERcq^PH(j=~m`Rqa)eXl+++|-K;#Ex>SpC$`8hx5=lqQdKj-KCoS*aa|K0dGKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;|0n#MpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxN8#uEoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmWq;bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Xp8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`4=~S&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaf7|#uKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauM0ot=lqQdKj-KCoS*Y^e$LPNIX{0~{5+>iU(2joTW7oJvYo|DvRtihI5rw* zr#8oZoA;R3ewT6enAVZqS~qJw>E|EUX5pfFrFC?Eon*M;<@1KltKEl&u1AhXjz^A1jz^A1j^|aI9FH829FH829FH829FH82 z9FH829FH829FH829FH829FH829FH8291lO|=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzcZQ$y zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QQj@ zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz ze{KAnpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa9}Yj~=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T0L-{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;Kic>?Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;zZibb&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax|3~BJ{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l<@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4`* z=Q&;aT4vSSI@?W`?JQ=J@Tw_R%&*zTJwap ztY@QU2-k8w-sj%u-sj%u-sj%u-sj%u>TJ~6sIyULqs~U1jXE23HtKBD*{HKoXQR$W zoy~_gbvEj3)Y+)BQD-9$Cl4nNClANZ`8hx5=lqQd zKj-KCoS*aa|JC?8Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;e-(bt&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax7sAi^IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*OjF=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8huyH-65~`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^Dk=roS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Jy!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS$zse$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaf7AFmKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aauMI!v=lqQdKj-KCoS*Y^e$LPNIX{1V{5+>iU(2joTW7oJvYo|DvRtihI5rw*r#8oZ zoA;R3ewT6enAVZqS~qJw>E|EUX5pfFrFC?Eon*M;<@1Klt$pYQX1zW*wX@AG}W&-eL0-{<>$pYQX1zR&mhKHum2e4p?0eZJ54 z`94?U`+WcL@qK>2ms$36ki*=`-Q3IlJeTM5Lijm9=jZ&KpY!wo(D*q&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax)t~coe$LPNIX~y;{QM2V&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYydf&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5(1 zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*TXeKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzk2HSH&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzFNB}-bAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QSKeKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzFExJ7&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSz?}VT8bAHax`8hx5=lqSz zgX8BpUHV#P)!I7SO_%K~W|HMq$TVxHbzH z%`2^=^XnwT6)&GRbZ&L_sIES#r&y|ctkhjr>warl&qm$pTFfiVE6gj*E6gj*E6gj* zE6gj*E6gh%oOy+D`(9?*&p{4zCwFr%_w!ty&kHebH*Pm>H*Pm>H*Pm>H*Pm>H*Pm> zS5K#&PCcD^I`wqw>D1Gyr&CX-p6gm+esi%{NQ%|R!PCcF6D?jJw{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMW!_WCSKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l@#?SdVKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*XF4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8huyHGa;|`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Dk`toS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-Jy!q53RKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aamp6XS z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqay)W8 zay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ay)W8{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;uOEKS&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqHKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz8#aE<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;Zxepb&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Y^UE}BcoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXyz&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmYmgbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^SL5gWoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKmWzX&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYCUbAHax`8hx5=lqTJ~6sIyULqs~U1jXE23HtKBD z*{HKoXY-*=osBvhbvEj3)Y-_x$-~LR$;0t;e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`S~Y>pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*P(qpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T6ICpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hw|3_s`R{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`SaoD{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}PN_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxU)K0JKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauMR)w=lqQdKj-KCoS*Y^e$LPNIX{2A{5+>iU(2joTW7oJvYo|DvRtihI5rw* zr#8oZoA;R3ewT6enAVZqS~qJw>E|EUX5pfFrFC?Eon*M;<@1KltiTlKkmo< zxF7f9e%z1yaX;?I{kR|Z<9^(a`*A<+$Ne5J_v31NnPoo*In15h&Ar^ub5S>|ZdTna zKj-KC{A(LO=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`MJ7Oe$LPN zIX~y;{G6Y^Uidjb=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-If(D*q&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Y^b@(|y=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-If+xR&@=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^d-yp&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~+4wm>=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEQ204N=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNdxqhD0rLSdHt*x`&blJ{gCRwglHyj&{vs0Vn zzRi0~Yro65dQ9ucZmpZOp7isNYqN0CywW;4zfLk-@$z{?=T>Kr>gtnvilw^8O5J6( z?zfipY}B2u<$7+^)84G-yj4$fJ5S`vJe8;OOrDK-g?WW}g?WW}g?WW}g?WW}g?WW} z#VdDSVcfo#S@v^~!`#W;+{^tu7w*UXxS#pC`MLSI`MLSI`MLSI`MLSI`MLSI`MJ7T zb+hVb)y=A#RX6(~P2H@zS#`7OX4TEAn^iZfZdTo_+$%rl=lqQdKj-KCoS*Y^e*Qa+pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Wj-=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTQdKj-KCoS*Y^e*Tw@pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e%=~C=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^NsLxe$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS#1ze$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaYmJ}tbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QQN+&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqE`2StYHgkErptB~Gs$wb zy5ZPpoSoVn_if%|TKiqb)ni&mc5B_N^`xJFT$_c9=9Sjb`E`=vikHtDI=4D|R9BzW zQ!LdzR_ZRRb-%T&hwt-!zR&mhKHq$pYQX1zR&mhKHum2 ze4p?0eZJ54`99x&+QdKj-KCoS*Y^_2>MYpYwBm&d>QdKY!iubAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEYxp@o=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEd*kQ) zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYy$6 zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zv zK=?U7=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6X3HGa;|`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^LGnB=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{5={!=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^A8L^=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lq@*1B2iNk9L%HVYTc zE3KpR>mpg#)YGY_Q%|R!PVSYT^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIY0mH#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIY0li@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l4#?SdVKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX_S1=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTyvEP@IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e!d=l&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaXTs0QdKj-KCoS*Y^e*Tr==lqQdKj-KCoS*Y^e$LPNIX~y;kDs6Cbm?oERcq^PH(j=~m`Rqa z)eXl+$ zpYQX1zR&mhKHq=*e4n51WtROMQdKUaUw&-pn&=jZ&KpY!wA2|wrO{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`L~3h^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNj8b9af{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`L{KG&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`S*vP z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMw z8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`42XJ&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`Ok!(^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luNL8$ajg{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`Oh_e&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`EQ1w^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN`Ge=@ zIbHf%X4Tp{+fA44EM}7BYIVc0(KtJ`IquuM$F%mljH}1Aj_lUDS?ftZ|F|{_7tJfJ zqx0(|!xb-|H*{`w_NcBtsi#<~d#u!5R_lIiF|RPMFt0GLFt0GLFt0GLFt0GLFt2#< z<`u^6dzocF2RY20+|9ihw;Q(`w;Q(`w;Q(`w;Q(`w;Q(`x2va9Pp6(vJ)L?w^>pg# z)YGY_Q&0CnO+B4@I`wqw>D1Gyr&CX-o=!cT+$%rl=lqQdKj-KCoS*Y^ety#UIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7KKz`Y^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=YJZ0&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=YQV#IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7JN%rV^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Y9A&Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QO?{IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LM~8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`E!k*^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNX@N<67&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq@*1B2iN&O>w#j@~<#^19&I=@abT=DXGL-&@2jOyx>dWxmG z$4cF0weH9F`99z0`+T48Kil{|-{<>$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48 z^L@V0_a8^!=jVHwWj_Zw%$?i~Kj-KCoS*Y^e*RUBpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lop#IX~y;{G6ZjbAHaxUpxGqpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa*KPcqpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-y-~+pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaw`%;HpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-zEH< zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zcWeBdpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax-#`4EpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqe$LPNIX~y;{G6ZjbAHax`8hx5=lqufh&wzHT?maEkb$42Aq)aJNv^B&XM?=r3)(>k(S>t?MdHTiqRvha$=-?KeBzfLk- z@$z{?_m+i>>gtnvilw^8O5J6(?q^D1Gyr&CX-o=!cTdOG!V>ghhP zsi#vQdKj-Hk7kQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^FIkc=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^FM3+oS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-J46@Jdo`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Zyrq&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=UwQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-I9g`e|t ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{L917 z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QS}M^PDby zEwgHEo$aQ}b`~?qa<#hQ*l3)c+8p<7-eX$(UB=a8T1R$k-K_PbpMPANg^T8u*3tQO zlHrP%&l@_oI(t-CpVU(<)jd|~E~|CFwXA2O?sP5JbEBU2W$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_xV2G=W2YP??0Np&(HTV z%YF`Wm^<-3_dWMLmy?l`k(2q#CMP2&BPSy#BPSy#BPSy#BPSy#BPSy#BPSy#BPSy# zBPSy#BPSy#BPSy#BPSy#BPYYp`8hx5=lqSzH-(?`bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPwqKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzw={mv&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEZ}>Ss=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq4~bAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEf8*!;oS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKY!=&bAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvRQNeR z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqE2 zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zv zOylSLoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKYzdQbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZvdiXg%=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6XZn0}tqrLSdHt*x`& zblJ{gCRwglHyj&{vs0VnzRi0~Yro65dQ9ucZmpZOp7isNYqN0CywW;4zfLk-@$z{? z=T>Kr>gtnvilw^8O5J6(?zfipY}B2u<$7+^)84G-yj4$fJ5S`v7;hMF7;hMF7;hMF z7;ik7;|=fgz09(ogB<2g{GHI>3H_bWxZSwjxZSwjxZSwjxZSv2osBvhbvEj3)Y+)B zQD>vhMxBj1n-6H}Y}DDPvr%WG&PJV$IvaI1>TKlU_&Go4=lqQdKj-KCoS*aa-)#JxpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aaKMp_V=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T55;e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaKW+S+pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aazY9O-=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1uze$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aa)$ntE&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzmxZ76bAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`J?LRIbHf%X4Tp{+fA44 zEM}7BYIVc0(KtJ`IquuM$F%mljH}1Aj_lUDS?ftZ|F|{_7tJfJqx0(|!xb-|H*{`w z_NcBtsi#<~d#u!5{0{jY@;mgX{to$`?`4+#9ON**=f3B@=kj{;dQUfbJ$XHOJ$XHO zJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJ$XHOJvkYE&d>QdKj-KC z{3{wi=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{2Rm1`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJBXji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{F@s;=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{CmRB`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAJ97ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{QDX|=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{3pZD`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9Vji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{HGf~=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{MW+I`8hx5=lqSzZ!~_+&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzABCUubAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QTn@Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzpEQ2X&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSz--e&_bAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QR>TKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzH2j>O^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=ZE3v{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~|jKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSz-Nw)PIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Q%GIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOoH2j>O^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxA6-As>C)FStJc=pZn|t|F_SD;s~e7u#@VUO zao^@WrnTQ?Ts@|BWVhDMT2K1<$F*6wXkKX@onI#zu6X&pp>wOVM|Jf{J;hSpW2Nr0 zTK8MadN%4#*K$2K>S=G*bKa^axgEaG_n&HfpYQX1zR&mhKHum2e4p?0eZJ54`99z0 z`+T48^L@V0_xV2G=lgu0tMPrl|LFQYKi|tN`#Fg3x$n8}xtxrgjGWAuH#r$O895m_ z895m_895m_895m_895m_895m_895m_895m_895m_895m_895m_895n#&d>QdKj-KC zoS(l&_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxzajjbpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHaxzp3$ae$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(m0_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxzdQV#pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T0ABpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*VslpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6^VpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqTJ~6sIyULqs~U1jXE23HtKBNuc@<9XQR$WosBvhbvEj3)Y+)Bk%!~w z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqia{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;AKUmjKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqk2@pFF8&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqiKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqiQ{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pV|01 zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^VP=B`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAEoO@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T3WGpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq@*1B2iNk9L% zHVYTcE3KpR>m&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM z>&eORbAHax`8hx5=U>+NIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LNdJ^Y-X^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=U*Rw&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=ik`)IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|H2j>O^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=ie26&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=ik%#IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|A^e=5^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=RY2P&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Reu_IX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPTMff>C=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqV_HXcYu&8%q@RCWn}v(!mDbVub&}zVm(LqIw>o=NSD(~VEY&@XH;gxoH;gxo zH;gxoHy-BkhWGhiX4#MTx%avEx%avEx%avEx%|ESz5KoWz5KoWz5KoWz5KoWz5KoW zz5KoW{rfigd-;3$d-;3$d-;3$d-;3$dwDp1&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=luNF8b9af{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPN`A3DH^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luK+!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=luMS8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`M(Z7=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{4*Lq=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax_ruTmIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^em-sdoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-I%ji2*#e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCd?);zpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LMyZ9mWH($_Mp*4Ei>x@>1LlPp)O z8;*^}*{RKO-{w80wcll2J*IVJx7N*CPx|@CwOP1mUTGbjUnd!^c=^1cbE~sQb@fR- z#Zuj4rS7s?_gl+)HtJ5-ay>WdX>ZnZ-l`{2AJ6ysKHum2e4p?0eZJ54`99z0`+T48 z^L@V0_xV2G=lgu0@AG}W&-eL0SL6G9|Izk+e!iDke9wK)eb41&{682^G&vbL895m_ z895m_895m_895m_895m_895m_895m_895m_895m_895m_895m_895m_8Gg>s`8hx5 z=luLj8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`9BOl=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`D-+O&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=Wh~z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^EYe!oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J)-S{~_=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-H^7Jkmp`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^LK3goS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-H^(fBz(=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`TI0} z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=N}n<&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`A0W?&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=l?4FoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KFH~gHR^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=bzsAIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOA!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T0uY=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hv7X#AX?^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=eNSo`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBs!q55n!|mreUHV#P)!I7SO_%K~W|HMwOVM|Jf{J%#ax z@rLn+@rLn+@y6|Dyzy|4H@wgH;(hLY?tSik?tSik{w2-(T>f7EUjAPGUjAPGUjAPG zUjAPGUjAPGUjAPGUjAPGUjAPGUjAPGUjAPGUjAMlj-T^$e$LPNIX~y;uNHpJ&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaS8x2B zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z|55lkKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqi__&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax-?Z^_e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS%PJQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l`F* zy9};m+wKGY-#Q@as0A~l(bZ*o&q+CEN-B{V6Eg?Q7%*eV3_2RR@SY)+n7Lxcq!N=% z3YalqkU3y>;6swmYUx}_ReDwS7vGwh`|GN%-rZ+6e3+xN_xi2X!_WCSKj-KCoS*aa zw+%n%=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T0@!IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPTYUAhpoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmWIlpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*O#L=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT8;zgybAHax`8hx5=lq2A90WSU7< zs?`n0M&s<(=BRIr9<$o-GOix8I&e%XuP0wmzMgzN`Fis8QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T2*3pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*SkFKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSze{KAnpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa{~dnL&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxr{U-PoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKi>~O=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTg79;G&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq2A90WSU7pA^D65fL$kLOvhMxBj18+A7PoS*Y^e$LPNIX~y;{QPUe&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5s1 ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zHwr)J=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T2K+pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax-zNN=pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaw`=^IpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqX@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;f4%W@e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-P#?SdVKj-KCoS*Y^ex8%1uVr4X?bF?K*~v7M ztW>KTj*Z6It<6#27CmOQ-(_4qW_4t*)~#BP`^Cq#SvqfCY8{?kCmF7I@w}mPtFuRS z^+`R&a@}LK?y^?*ThB%|>rPj4HP`BCuh(-vP*3t;9?HXcB#-8?Jf0`=WS+{?d0Ac_ z9>4>501w~+Jb(xA03N^tcmNOJ0X%>Q@PH2s58wejfCumZ9>4>{+tt&lr&CYI)zs6e zr&CYI&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LN-EBu_F^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=YPBLbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zvm&VWeIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*Q_}=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTU*YHcoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmYrUpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Un=G=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT z{P1&r&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LOou<>(#&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq2A90WSU7QdKj-KCoS*Y^e$LPNIX~y;{G6Y^VfZ;e=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IvvGH?$&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXzI zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^ zZR6+soS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKR;^xoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-J~8Gg>s`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^PdYp=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^IvHEoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk8h*~t`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^WO|V=jVU2pXX%hYnfMT`*b&5b~4Q*E7j_TW213)Yjf1MMUPqS zcNtfYSsmG{b*t9ne(`Z_md=}(T8C%XNro$4Ja6dS>g-Wno%?Y=?#KPOANS*a+>iTl zKkmoecu;Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KFGyI&N^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=bzm8IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LOw;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS#n{Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSze&gr-oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKi>{N=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCe5dhqe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%O|QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;uh#fEKj-KCoS*Y^e$LPN`HSJ_Ia&Hz z=GEFh-A$LBOf$(!wYuThXq?^J9QAF{V^;fJ#?@n1NA_yns`a>Ed|aEQ^X8@2;n{VP z;ffc}8#=c-dsJ7S)Ke_iJyz>3YjwZ%Y-F?UbR}1Ft)BLJJ?8`UBoF4HJe)`JXdcVs zc_L5dsXQI?3iAr{imz$r73LM@73LM@73LM@73LK$#=OF~eLwRYSoo=%7;@ot8P}^th`r#&d>QdKj-KCoS*aa?+8EV=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S}|)e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa?`r&QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa9|}L`=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5&4e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaA8GuYpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aapAA3f z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T2V_ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zpKtt}pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa-v~eF=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T2)6e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aa-)j7vpYwBm&d>QdKYwZXc}|wTmU*?dPj}O0C(}%_Qmt+{HX3KQHb;G1 z^qAFtmvQx&)sekgw`x7^7a!MV>AZQVb$E82WVqtR^M=l?&K}j(C-oG|b&u7$%Ua!U zJsa7qJ6*}uT&t(OUeEbJJ;{T4C=ch6JetSyc%I0Uc`8qb`*A<+$Njh;_v3!tkNa^y z?#KPOANS*a+>iTlKkmoQdKj-KCoS*Y^e$LPTNBB8E=jZ&K zpYwBdsr;Ou^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{QQ#|Kj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqR||1b5k z!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxuZ5rUbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QQ;S=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hwo-uO8`=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6XZ9e&Qw`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^REm)=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`Byi7&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`L~Ck^K*XA&-pn&e{uXgCre+;yjt6*yXmr%X(m~z zRyQ0Qjk8;uqrNSA%xb^OxO&X$$X=~mwI272k887Z-n`U0JiAUZT=C+0L+4g!ixY|y ziW7QdKj-KCoS*Y^e$LPNIX~y;KNx<_ z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zKiv2^Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;KNEh=&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;KiBvQdKj-KC zoS*Y^e$LPNIX~y;zaDQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;zuEXXKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;e;j_!&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=YLv1&&kr)GOyP5>2A90WSU7iTlzdx<}O|x8fw_19)R*tuxjck_SUCGsmw~M!n zw~M!nw~M!nw~M!nw~M!nw~M!nw~M!nw~M!nw~M!nw~M!nw~M!nw~M!XPo?)%eq!^U zO7&{$)zqu0SL5gWoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{CwQ_IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LN7r}1-s&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSz%NjrD=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T18he$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa*9$-A=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4hnpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqf6OzxcQ|OXtl?t;4hHB*PUio;P%E zb@r&PKB=czu6wN3UDoP;>)FU=-RVli55y0|55y0|55y0|55y0|55y0|55A*`ABZ1_ zABZ2kSn&h%^Zm?okQ=#~Te+P(c_z>1xjY~9bMtfabMtfabMtfabMtfabMtfa^S6lk zx%s*Ix%s*Jc=hq>f`0Ts*hJ6uRdOWJ3r^={G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX{25@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lP@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq5I{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0ln@N<67&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqEd|aEQ^X8@2;n{VP;ffc}8#fh$X4K89n^iZ<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lp!s_&Go4=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHaxKfCdBe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*yu&z}!J=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqsMB`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA0_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzdZb$pYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauWbCBpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaZw)`^=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6TNe$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*aaZ*TmZpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{BH8|oGg7U^J;CM z?xxF5rkP}=THSDLG|p~qj{3IfF{}M9++|-K@GQdKj-KCoS*aaA8h=bpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aap9(+c=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4sye$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aapK1J@ zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zUkyL!=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`S}Moe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aaUvK=JpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaKMX(T=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T55;e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaKW_Y-pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq#fPiWkotI`e(L&-eL0-{<>$pYQX1zR&mhKHum2e4p?0 zeZJ54`99z0`+T48^L@V0_xV2G=lgvBqVG?$Tt>fIO21ZKzn+b3meg}KuEy248du|L zT#c)7HLk|hxEfdEYFv%0aW$^S)wmj0<7!-ut8q21#?`pmXGC49x>R+k{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LOA8b9af{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPN`DZnL&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hwo+W0v? z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOI z2tViN{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`Im*C^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luMs#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luM2!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX{2B#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=XaN%=Va+?nOAH3bT?ggGR-6_)#`?0qj7d?bJVv*k6G<^8CQ>4 z9oegOtJdRw@o{aI&YPE7hiBJGhAUn?Z|L0W>``5PQctm5_gJmFtkwP2vysiZ)0JG! zwR+m?^_&mXlRTJ*@^Bu>qj@Zk$9Thd!+67Z!+67Z!+7KF8gKYL-_JY;xsjW>mD{s3B?J;360y0+l||e+l||e+l||e+utneY}DDPvr%WG&PJV$IvaI1>TJ~6sIyUL zqs~U1jXE23HtKBD*{HLT5692>IX~y;{G6ZjbAHax`8hx5=ieWG&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=ReT+IX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|CH$P9^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=RX;K&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Re)} zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN7 zApD%4^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=f~ma{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0lP#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luL*!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0ln#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqg-WneNs=cT=!V5 zyR6my*0Yh#y3>_h&9!>k>-C%u)RR1zhw^Y9$)kBJkB9H`eZJ54`99z0`+T48^L@V0 z_xV2G=lgu0@AG}W&-eL0-{<>$|I@?w`99z0YJ8vX-(SAZ&-XLWL2l$`Zsm6FQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T1uye$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^|Npu7w_j=eoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-JqH-65~`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^T)%_`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA%;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS%PrQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lq#de$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS%Pl_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxU$^mde$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%PdQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%PR_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZPZGN7UrLSdPt?kp@blJ%? zldM#$8;*^}*{#h{-xfV)wcll2J!W-euhy+vkNd^PwOKlEUTPhlT_+i?c=5cUbE~sQ zb@fR-#d6(aweGT3_gl|K%qz?*%qz?*%qz?*%qz?*%qz?*%q#A;d4+NNe&#vIjoi$w z+|Hdm6XSN{cH?&AcH?&AcH?&AcH?&AcH?&Sbn5Beys4*CPp6(vJ)L?w^>pg#)YGY_ zQ%|R!PCcD^I`wqw>D1Gyr<3=}&-pn&=jZ&KpYwBm&d>QdKj-KCoSz>ye$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aacMd=2=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0+TpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0*Z ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z_YXhk=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T4JepYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T4Ile$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaj}AZQ=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T6gMpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`S~9QdKj-KCoS*aaPYXZi=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`FR>Yf2sL-PL{rwd9}7rchhAj(@e5bt!_9r8fUjQM}1rL znALularKzhk-b{CYCY~3AJ=B-ym_g0cy^s+xZ=h0hR&_d9@W(+^%To>kJY-%THS9w z8{zwWpYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_xV2G=lgu0@AG}W&-Xv0 zz0`c4pYLa$gWSl?+{*3T$ur^S{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO;pY!vr#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX{0a{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;UlM-K&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSzmoQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzH-(?`bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPwqKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzw={mv&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz_lBSIbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{QS)tKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSz_cwmd&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5=F=Q&yWTISW-KHW{1 zolG;yO0~M-*l3*H+8p(5(PLKoUB=a8R!8<~-KzDtUwmAfrSs;c*5TQ8lHrOM&l@_o zI(t-CpVU(<*F9G2E^Bqa^_W+fSD06rSD06rSD06rSD06rSD07aee(+A_WjIrkQ=#~ zTe+P(F>W_*H*Pm>H*Pm>H*Pm>H*Pm>H*Qx?r=CtdoqD==ih4Trbn5BU)2XLZPp6(v zJ)L?w^>pg#)YGY_Q%|R!PTng&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN`HzR6^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL#8b9af z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`A;@} z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`7ei`^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=luKw8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`Eldt{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0lX@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIY0l@@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmYW`&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqu2U~PsLwB+H*{}V%BZeBsi#=3d#u)7*6M!i z;ro1_@AG}W&-eL0-{<>$pYQX1zR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_g{*> z&(HTW&p~eFW^Uzn?u4K7bAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX_o_&d)a+Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzvyGqgbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QS}IbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6YEariku=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-IP+W0v?=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-J)7=F&r`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Ve?toS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-J)-1s>^=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J)6MoLm`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^EYk$oS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-J)*Z4U<=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Xax@=Va+?nOAH3bT?ggGR-6_ z)#`?0qj7d?bJVv*k6G<^8CQ>49oegOtJdS1{JpR&z0i0S+rzW#B*PUio;P%FS<0xc zKB=czu6wN3UDoP;<`w1@<`w1@<`w1@<`w1@<`w1@<`sA2yu!GBKl2>qMsDU-ZpXOY zxZSwjxZSwjxZSwjxZSwjxZSv2J)L?w^>pg#-ZARw)YGY_Q%|R!PCcD^I`wqw>D1Gy zr&CX-o~}|VTWd>g)W&Vnrfs>cwAD6iYprYRZKH8DuEy248du|LT#c)7HLk|hxEfdE zYFzEd!qvDMSL142jjM4ruEy248du|LT+Mqby-!A6DnIAv{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^Pgz^oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J~8-C8u`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^Ir--=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^IvKFoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk8Gg>s`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Zj^WP0W=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^WSg$oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-J45`NCl`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Zyrq&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=Uw1&x+Yx{IJ zU3N0fBrDbGhGU~~c58Fgw?&Ux?RObhk69hrt97f^<9b=i3(L|AjaRWfJiAUZT=C+0 zL-&@YjOyx>dWz+`$7&e%X zuP0wmzMgzN`Fis8QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzBjM-#oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmVfebAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6XZ*7!L;=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Y^M))~D=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-JK)%ZC-=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zfg`e|te$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{JX=?`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9Nji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{CgWe=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lq9nCre+;yjt6*yXmr%X(m~zRyQ0Qjk8;uqrNSA%xb^O zxO&X$$X=~mwI272k887Z-n`U0JiAUZT=C+0L+4g!kLv1^dWz+`$7QdKj-KCoS*Y^e$LPNIX~y;{QR97Kj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSzk2ikK&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzFNUAQdKj-KCoS*Y^e$LPNIX~y;{QUhIKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqSzFE@V9&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz?}VT8bAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QRRDKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz?=^nT z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa z{~LbJ&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;YvJeooS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKfe`z&d>QdKj-KC{ND8QoGg7U^J;CM?p9{pPNtb;rCQx^Y&6bpZI1f3=rODP zF5~Jkt0Q~0Zq<6+FFvl#(s}bz>+tM4$#BJs=M9}(ojt0nPwFX_>mI9h7vl}%jrC@{ zVZ33yVZ33yac_<{{GRV;o`c-T&GQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTXW{4koS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYz6GbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEapULw zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmQBi z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT z`tWmp&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxT#>UV2IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*Ra&&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e*VUdpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T5(2pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{QMmoKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T70o=Q&yW zTISW-KHW{1olG;yO0~M-*l3*H+8p(5(PLKoUB=a8R!8<~-KzDtUwmAfrSs;c*5TQ8 zlHrOM&l@_oI(t-CpVU(<*F9G2F8&VrJLK=s{rY#v_k2I|9OOoB#`oO!-1l6*o_sy| zdh+$;>&e%XuP0wmzMgzN`Fis8QdKj-KCoS*Y^e$LPTR`@wT=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|xAAj+&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTd*SE&oS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYIH=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT$i~n4 zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*U|S zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QUofpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`S~pToS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H+!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lq|~lrR;twv$42Aq*5;^hiypJu z?=r3)vpTX@>sGDD{o>==ES)zmwGPj&lMGk9c;3*t)!C!E`lOy>x$d!Ack%zlc;AhD zJ>w1I4dV^tjeB*x;rDz$^Bm+x#1F&|#19^d-*dm`e$V}$%fFX@FaKWtz5ILm_ww)M z-^;(3e=q-D{=NKr`SSzBaNT)bAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QQd=Kj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSztB0TS zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QRGX zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`S~|Ae$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aazZ`zf&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*aay~fY^IX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*WE!pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QTdApYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T37Fe$LPNIX~y;{G6ZjbAHax`8hx5=lqG?T1Us~e7u#@VgSQQsClX0_jCTs>xWWUtn(T95n1$F*5HZ(eF0o?Ryy zu6Xghp>wOVM|Jf{J;id}W3}$GR`*-aMmFnCS8_Gi>S?dnb3RZ{@?iKr-{<>$pYQX1 zzR&mhKHum2e4p?0eZJ54|7Q3;-{<>$pYQX1zR&mhKHum2T#fJZ{rlJV`T2h4ImnIp zp8KBrp39SwCnHZro{T&hc{1{3QdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y>}| zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Ja z*!Vd==jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPTTlhIY=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZvPUGkNoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmUi}=lqQdKj-KFpS`;bu4G%!1N<&+ zkhIl;nbBx8hw)xJV8%o-bHGeVC1y%0F;n8iZ0*4V_ZkCc445%sTPE2dGbNQ+W;?0G z6mqK4`C8gnQk7i(d*;kMeXFZi_g)P@?$%zbpZA-?&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*aa{~LbJ&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;tKsMToS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKff7%&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS$D0Kj-KCoS*Y^e$LPDT|dw9($_Mt*7nJ6y6j|{Nmi=W4aY{~ z?AGS6Z;KwY+V3*19pg#)YGY_Q%|R!PCcD^I`wqw>D1Gyr;~f-=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-?eKGc&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPTWaH=joS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmR9QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*Qh-=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxTvyGqgbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QO@ue$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aacMd=2=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T389pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-#`4EpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqF1;TlARKewT6enAMTJTDNLF>K7l^X6dYX zp>=S2on*M;`SXU(t$pYQX1zR&mhKHum2e4p?0eZJ54 z`99z0`+WbeG``RG`99z0`+T48^Zf_e_xbsL=D87m&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luLXHh#{}`8hx5=lqQdKmVxkbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZvcKA6z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTuZ^GcbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOhH&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmR+8pYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=QqO7`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbACQ+{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;w;Dg^=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T1`6IX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LOoIQ*QS^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=MOc0&d>QdKj-K7 zvY+R8>1&x+Yx`t3U3N0fBrDbGhGU~~c58Fkw?&Ux?RObhk69hrt97f^qki#mZI;fO z7g`6W*GYyeo(0fAf360ymC-k21?>FOi<96eA<96eAbvEj3)Y+)B zQD>vhMxBj18+A77Y}DDPvr%WG&PJV$IvaI1>TJ~6$iwk-e$LPNIX~y;{QR}T&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmQYr zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzcZZ+zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QS=}e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaf8O{xKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;KN^0{&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;|GM#We$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS(m6_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxeQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqSz{q5&DUiw<* z)!IJUO_!ZaGs#M|y5ZPpoZZ?S_HEH)R{LGX)nisi_G;a#^{8KbT$`n{=7rY5>2;Ff zis#Q8I=4D|R9BzWQ!Lj#R_iWnb-(p&WV7ycDVKAlp7v@z=e2s0>v<>-=aD>`$MSfd z$dh?0Pv@CDn^)wO;eOnY`*A<+$Njh;_v3!tkNa^y?#KPO-+yS_kNa^y?#KPOANS*a z-nVl895m_895m_895m_895m_8Gg>s`8hx5=lqQdKj-KCoS*aa-wHqH=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T1XO{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;|E2MBe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%P6_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax|4;ZiKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqjLQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0EjoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-JGji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{AT0l{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX}M^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*aaFA6{B=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6z6&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq%TA`5WTjf&aBMWL|N2sL*tbQGS?zZjSC3g8*{gM{)}wy$ac!2) znipCJr`Jh_E1o}Z=-le;QC)pfPvQN6_Xpk|cz@vif%gaAA9#Q8%6Na^{ekxf-XC~> z;QfL32QTgW1M~C!n4g=Uo1dGXo1dGXo1dGXe_hPa&Ckux&Ckux&Ckux)yJ!kS0ArF zUVXg!c=hq>QdKj-KC{IwfD=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^EVAY=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{LLFb z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^LGqC=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax-?{N~e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*+#QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*-E_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax-@oy5e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*+<QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqiTlKkmoQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`S~Y@pYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOfJKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzH2j>O z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lkL3 z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX_=% z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; zHyS_Z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T0)xIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LOoF#MdK^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=ev!c^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=luMO8$ajg{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`76TD`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAJA{;phCEpYwBm&d*f*D|lx_Q`I#>|~lrR;twv$42Aq*5sGBt{o>==ES)tkv<^x$d!AcUi0Z zt!E>fb*D?YoGbOTSL->i)stM$LwPulD1Gyr&CX-o=!cT zdOG!V>gm+esi#vQdKj-KCoS*Y^e$LPNIX{1s@N<67&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{1g@N<67 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX{1&@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0mK@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq&ZKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqqj@Zk=ZQR-r}A{3$+O{p+>iTl zKkmoQdKj-KCoS*Y^e$LPNIX~y;{QQ4~ zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`S~RLoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-J47kQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^X10R`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbACQ={G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;SHsWwIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*OjF=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hwo*7!L;=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6XZ8-C8u`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^REd%=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`PVgm&d>QdKY!`{JjY94%e-3KC%fsglW8Vdsa7`} z8;!GDo5Q{>ddzCS%eZ>X>d0QLTeTkbi;ruwbk@AkIyk*fGF z;QfL32i_lef8hOr_Xpk|cz@vif%gY*9PbajKk)wGrGI}g&2qg1Sgp4JYxO>0Jsa7q zHvyM&Ialg!f^oZXyK%d5yK%d5yK%d5yK(z>#JJtK-MC%dth!lsv+8Em&8nMKH>++| z-K@GQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y3c>oS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-IfAAZiy`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^B)dB=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^TWo^ z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBb z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS(mMQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJ9|;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq5e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%P0QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAJAb;phCEpHFhK^tH^ZxxLu$vXf~hSxHy_t7SCKZfy?xw&*dd z{VwC`F{>kcwQkjV)Gt1+&C*%(LhIo4I>~Uw^XCnnTb(_stMh%n&-eL0-{<>$pYQX1 zzR&mhKHum2e4p?0eZJ54`99z0`+T48^L@V0_xV2G|DWOee4n55bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0m8#?SdVKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T6IEpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv- zCj6YA^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=U*Lu&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^RI3EoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-If6n@Un`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^KTD7=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqg-WneNs=cT=!V5yR6my*0Yh#y3?gx&Xs!FtM#1M z>PfEWp*);N@@O8*<9Q-a=BYd#e~0`X@^|Rv@ps7gd_VKt$j#i!?cB-TJeTM5LSD?P z^6HpZm{*ur_&xV~?)Tj9xm>PX?wd5ZT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAi zT)AAiTz)Q>E0-&mD<{Lx`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^S28>=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{2dxU=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y;or=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{Cyff=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^A8O_ z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC z{KFeR=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^G^sr=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqg-WneNs=c zT=!V5yR6my*0Yh#y3?gx&Xs!FtM#1M>PfEWp*);N@@O8*<9Q-a=BYd#e~0`X@^|Rv z@ps7gd_VKt$j#i!?cB-TJeTM5LSD?P^6HpZm{*ur_&xV~?)Tj9xm>PXu3WBMu3WBM zu3WBMu3WBMu3WBMu3WBMu3WBMu3WBM?k6?5Tz)Q>E0-&mD<{Lx`8hx5=lqQdKj-KCoS*Y^e$LPNx%%_Z2|wrO{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`K0l4e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS%PQQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq1V&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmUTp&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-JqHh#{} z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^RH?A zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-If z7=F&r`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^KT13=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{27 z_<4?(zLt5lwoi7`Whc{2vQn*XI5rw*w>F1;TlARKewT6enAMTJTDNLF>K7l^X6dYX zp>=S2on*M;`SXU(tQdKj-KCoS*Y^e*S~u=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-`^L}t zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*VLa zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*V+p=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hv-@5ayhIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*UwKpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e*UZB=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxTu*T2%IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Wu?pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*OpH=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxT#KzD0IX~y;4~n1XcV{*Z zadvBS*tbQGS?zZjSC3g8*{gM{)}wy$ac!2)nipCJr`Jh_E1o}Z=-ldTe~0`X@^|P# z`8zbta{Vn?t-mH~^><}G8`-SCD3@|May)W8ay)W8ay)W8ay)W8ay)W8ay)W8ay)W8 zay)W8ay)W8ay)W8ay)W8ay)W8ay)W8ayQdKj-KCoS*Y^em-veoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-J4+xR&@=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QRl#bAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEW%xNi=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP-S{~_ z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqY zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J) z8h*~t`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^EYbzoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-J)-uO8`=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqddzCS%eZ>X>d0QLTeTkbi;ruwbk@Ak zIyk*fGFQd zKj-KC{D&Gp=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{HMat`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAJ9_ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{AU_J=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC{8z%y`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJAzji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{MQ;k=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{P)Ap`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ8_ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCyoI0hbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QR@R&-pn&=jZ&KpWi=zp5vvjWnQiA zlihUL$uyI!RI3|~jmFun&0*gbJ!ZAvWn4XGb!4yBty+)z#mBW-I%{5N9h_b#8LoK# zyrHx4hVjOz8E+VG7;hMF7;oG^BK{6yI~-bKi5{bGcl( zT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)AAiT)A91895m_na^o* zGW?vM^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvV zfBs_lIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LOYG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^H(>1&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX{28@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lq{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX{2>@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{2v@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPYjKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqG?T1Us~e7u#@VgSVc!-#X0_jCTs>xWWUtn( zT95k0$F*5HYhGv_oL(mxu6X{up|kOZ@rLn+@rLn+@rLon{WRW~X1T@;t2Js^tMS8n zHnLeG2j6qwbKi5{bGcl(T)AAiT)AAiT)AAiT)AAiT)EsIY;w7BxpKL3xpKL3xpKL3 zxpKL3xpKL3xpFdcGIBCQdKj-J~9)8Zx`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Pdbq=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Pg_~ zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk z6n@Un`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^Ir}>=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^IvWJoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk8-C8u`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^WO_U=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^FL_(oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J48Gg>s`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^QFem`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJBWji2*#e$LPDCqK{e($_Mt*7nJ6y6j|{Nmi=W4aY{~ z?AGS6Z;KwY+V3*19|vBE>|vBE>|vBE>|vB zE>|vBE>|vBE>|vBE>|vBE>|vBE>|vBE>|vBE>|vBE>|v>pT7`(&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{8f#g^K*XA&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMM@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;U)lIM zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z-yD9<&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;-`e;&Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;-yeR?&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;KhXF&Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=l7SN=XmLB znOAH3WH()QGR-6_)#`?0qj7d?bJ({49oegOtJb4_@o{aI&YBlm2dCFb zhAW;wZ|L0W>``5PQctm5_gJmFtkwP2vysiZ)1_R_m3rE%^_$pYQYicMIR=`+T48^L@V0_xV2G=lgu0@AG}W&-b|+-{<@H zm+$lQ{mgSCH*+hub0>H6T%M2bx$n8}xtxrgjGT;|jGT;|jGT;|jGT;|jGT;|jGW9* zL{3IdMovafMovafMovafMovafMovafMovafMoxyG^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS(l(QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAJAT;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`S}Moe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaUupcDpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aa-wi+K=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`T55+e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa-*5b!pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa*7!L;=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6X}hM)6ue$LPNIX~y;{G6ZjbAEnr z`FW0)zLt5lwoi7`Whc{2vQn*XI5rw*w>F1;TlARKewT6enAMTJTDNLF>K7l^X6dYX zp>=S2on*M;`SXU(t{y31PKZ#^3^uXsM@73LM@73LM@73LM@73LM@ z73LN9*1W>FeLwTu$j#i!?cB-TJQw5kOU=05xZSwjxZSwjxZSwjxZSwjxLrM+dOG!V z>gm+esi#vlKt{G6Y^*!Vd==jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{K>}8`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ96ji2*# ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{B^_6 z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9! z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAJ9Uji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{4K-J`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJAP;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAEo%_&Go4=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax-!=T4pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq49oegOtJb4_@o{aI&YBlm2dCFbhAW;wZ|L0W z>``5PQctm5_gJmFtkwP2vk|_}_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1zR&mHz43j% z&-eL0-{<>$pYQX1zR&mh{)6WG{Cq$2+{n$`%I)0A-8>h5&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxf3op&e$LPNIX~y;{G6ZjbAHax`MLUYe$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC{FlPd`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJ9oji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{FfU)=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{CC37`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA@ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{P!9^=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCJdL08bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QNT;Kj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzM))~D=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-K7nxE%*>1&x+Yx`t3U3N0fBrDbGhGU~~c58Fkw?&Ux?RObhk69hrt97f^qki#m zZI;fO7g`6W*GYyeogm+esi#vQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaFK_&u zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa zZwx=@=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T6TLe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aaZ*KgYpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aa?+ria=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T1Kle$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa?{EB^pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;|33VjpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxf1>eoe$LPNIX~y;{G6ZjbAHaxA3Q(L@zU2auh#a- zZo2Gbnn_lw)eXl+$pYQX1zR&mhKHum2e4p?0eZJ54 z|62Gy-{<>$pYQX1zR&mhKHq=ve4n51XPz6mnOnJ?JGmQv&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxT&*A6%oS*Y^e$LPNIX~y;{9OGxKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;M~$EJbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPf)pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QP4YKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSz?>2tU&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaKKz`Y^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=bzE|IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LO=!_WCSKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*S3qIX~y;{G6Zj zbAEpB`FW0)zLt5lwoi7`Whc{2vQn*XI5rw*w>F1;TlARKewT6enAMTJTDNLFs>$DL zmZjG;{)p|t>2;Ffis#Q8y0s=xZSwjxZSw@@n+m^+-}@%+-}@%+^(KZJ)L?w z^>pg#)YGY_Q%|R!PCcD^I`wqw>D1Gyr&CX-o=!cTdOEpRe$LPNIX~y;{QS!rKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzpAJ9g z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0K! zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz z>o$JQ&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;e=hu-pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;Z`t@cKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*-@@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lq?{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mL#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJAx!q53RKj-KCoS*Y^e$LMyL_g2*($_Mt*7nJ6 zy6j|{Nmi=W4aY{~?AGS6Z;KwY+V3*19QdKj-KC{FfR(=jZ&KpYwBm&d>R|`g4BH&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6Zv&G2)6&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEbmQmzoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKmVP^&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*XW$&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq5ZTBbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{JaZ4=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzBjM-#oS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmUi}=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-tnqVx&d>Qd zKff3KJjY94%e-3KC%fsglW8Vdsa7`}8;!GDo5Q{>ddzCS%eZ>X>d0QLTeTkbi;ruw zbk@AkIyk*fGF?T}PjWpE z<>44_7;hMFd}%Y@Fy1iUFy6Qq#~Xgn_cPCp+{~@qj`xJ#6M9c*+-}@%+-}_dr()c0 z+-}@%+^)_>osBvhbvEj3)Y+)BQD>vhMxBj18+A77Y}DDPvr%WG&PJV$IvaU7e$LPN zIX~y;{G6Zj^M4wC&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=ik`)IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPL!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJ97ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{CgWe=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`M(W6=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPT zKYRE6TS>p)1^oT>ImhLA+;+WQZ+V`5>b)r-NN*0w?InO9y@4RTfgrtsAkAmXQ}0a= z2+|t}(i;ez<`h7X-asb3fgquShR@hXwk7`qyVh%@(R^p-JKvdKX6EDbde1ID=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIY0mNji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{NFTw&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`LBha^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLHH-65~`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Z(HJIX~y;{G6ZjbAHax`8hx5 z=aKZP}}Nv*x3IzPlC+ zXUz+(gVS-6VZ`&t4V_yZJ*v?sbrnl>j+HvgYMpN_>)EI?ZRJX?*419C>%3l9aw9i$ zD}0~t^L@V0_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1zW*uV`+T48^L@V0_xV0oQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxT&yAn+bAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{QUF7&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*U)`Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzYWO)n=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lp!F@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*ReF=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT`;DLTbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QT|1&-pn&e>nX-$5UU^ ztXkVAtLd_n#Z0nXt!~&h8fUi_hkcv(nAUoiarKzimc5!cYd-4dyKAv<*1XUas;@jiz3V&BToPSrS z+xIie-Q3IlsN2=;>UMRzx?SC_ZdbRf+y7)!x2xNIPv?6&-_!Y?&i8b_r}I6X@9BI` z=X*Ne)A^px_jJCe^F5vK>3mP;dph6K$-VM(e$LPNIX~y;{G6ZjbAJAh8$ajg{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`FqvBTKnGl zGkKr9Z(fyG=QVk4UYFPB4S8eUlsD)7!qvDMSL142jjM4ruEy248du|LT#c)7HLk|h zxEfdEYFv%0aW$^S)wmj0<7z(?-=+F4)px1QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAImsKmY0QbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvxyH}=IX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*Ui;Kj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzM}(jAbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOtL&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4(Z{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pB#S9&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzUu*oF zpYwBm&d>QdKj-I{)z5Q0^)=0^wSBUhE<0JwB+J$6hHay9c589iw|S3gt#=t$k7;e$ zt9i5Lqkg`-77J(13$26Gagt%g^T!RHTOB>B(I<5kOLdNwI*aF!=aA>nWql60&-XLS z-Q3IlxX<0^?sIuPc|CbOc|CbOc|CbOc|CbOc|CbOc|CbOc|CbOc|CbOc|CbOc|CbO zc|Cc(|J3C5QdKj-KCoS*Y^ ze$LPNIX~y;pBH}4&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*aazt#9TKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hwoAAZiy`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^RDr8e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS#3}_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxZ-$@qbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QRrK&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmWGybAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZviN??QIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-IT8-+Yx`t1U3RjVNtUbC4ckWJ z?AGG2Z}T40TJJKh9@E;gSMz4gNBw+vEf&t27g`6W<0Qj~=Z_mYw>o-MqfhE8mg*cU zb(YmS-&)qQQD@r9m0Yc>y;j$Gy{_a&Jcm4oJcl0L=aBn+KeOD;z4*Jr-xdC@@ILoG z_dfSNm&=vQmCOA{O)gh1S1wmBS1wmBS1wmBS1wmBS1wmBS1wmBS1wmBS1wmBS1wmB zS5AhX^K*XA&-pn&=jZ&KpYwBm{+{9I{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lw@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax-yD9<&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzpKbh{pYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqb@pFF8&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;AKCahKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;pVIg_ zKj-KCoS*Y^e$LPNIX~y;{QS}N^BhloO|xokpRA_KP8KuCa<#f)+i0BKS{(Lm-eX$p zUB=a8T3hyN-mLkkpYN{4!dde|>)>>pWEk=MaYN@;M~`asNnOQKonxiWvRda`%X&8I zOk25N>C2mE4Hukmr!+(4+eta-Z*Kmbas!rv9%=icYu=icXXxpKL3 zxpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xqsN?a^-U6a^-U6WcWEh=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=bsyX&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^UrVmoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hw;phCEpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC z{8r=V{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0lJ#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0l_@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*aaN7>JFJoPorsV|EjadvBQ*tdC)X{~n|SC46J*{gZ8=A(YT zyA}&)%?quA({Ykv#Pi1uom(9}s?jHP6-#vv^@e&wy`kPvZ>Tra8;^3m;eEcJS?f7EUjF{uoBX}}z5KoWz5KoWz5KoWz5KoWz5KoWz5KoWz5KoW zz5KoWz5KoWy*wO0=jZ&KpYwBm&d>QdKj-KC{5`_Y`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ8<;phCEpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqR`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9qji2*#e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{7vEK{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIY0l&@N<67&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mD#?SdVKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0m9@N<67 z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l^ z#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0ls@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq2ilh>2ilh>2ilh>2ilh>2ilh>2ilh>2ilh>2ilh>2i zlh>2ilh^x$Ca)*2C$A^3Cnv+t`8hx5=lqQdKj-KC zoS*Y^e$LPNIY0lL@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS%PQQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`T5=ObAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Xp8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`TfSv`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAG-Xe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aauM9uu=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5Pp&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPag&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYzQ%&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=a06Z=XmOCnpJE2WHnuOvY1JhtJMwL zM&s<(;;?V?9@ARyGOixe+Ok*kX3aMEA% z94mE})jHo=*0WJ(+RBw&t*gCO*Ll6J$pYQX1zR&mhKHum2e4p?0eZJ54{oc;^`99z0YJ8vXKia;} z&-XKn_qq4E_qq4E_qq4E_qjZrJe)k7JlwlS9!?%k9!?%k9!?%k9!?%k9!?%k9!?%k z9!?%k9!?%k9!?%k9!?&PpYwBm&d>QdKj-KCoS*Y^e$LOoKm44Z^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=kMA0IX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LN-u<>(#&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H^5q{3k`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^EWqs&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=N}q= z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`G+@t&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=bsRM&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPN`6o4g&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj^M~8db3FAm&8oG1vYIYC zSKZP}}Nv*x3IzPlC+XUz+(gVS-6VZ`&t4V_yZ zJ*v?sbrnl>j+HvgYMpN_>)EI?ZRJX?*419C>%3l9aw9i$E4TAl9?zXTktg$1p3XCQ zHm}Gl<2mFx2Yx^B`+@hl-w*tL;P(S{ySiQ7u5OpVm%o?4m%o?4 zm%o?4m%o?4m%o?4m%o?4m%o?4|NSO^FMlt8FMls5!_WCSKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*-%@N<67&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX_SzEc~3G^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=PQk$^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luL$QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzjmFRUIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e*RUBpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmX?NbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^ZR6+s zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqB(I<5k>J9aVdPBXT-cWC-Hy-tR!~1+c-sj%u-sj%u-sj%u-skf7^7r!h z-`eEwQdKj-KCoS*aa?+ZWY=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`T2V^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaA87oXpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSzVfZ;e=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-vhj0%&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-G5nmL^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=O5PiIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-x$$#;&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-C;XhB^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=bza4IX~y;{G6ZjbAHax zFT0=TcQdKj-KCoS*Y^e*RhE=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTpW)~HoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmVM@&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^ZoF1e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS!c@e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aayN#do zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{Cp?; zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP z9)8Zx`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^WDbJ`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAJAnji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{1xHn{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0lV@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*aaN8itLJoPorsV|EjadvBQ*tdC)X{~n|SC46J*{gZ8=A(YT zyA}&)%?quA({Ykv#Pi1uom(9}s?jHP6-#xFl{(96oo_Ac*{CyZQdKj-KC zoS*aa?+ria=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T4sye$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aa?{EB^pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzkB6W0bAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{QQlLpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`S}NjpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*U42pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T56%pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*OuKpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{QTkf^BhloO|xokpRA_KP8KuCa<#f)+i0BKS{(Lm-eX$pUB=a8T3hyN-mLkk zpYN{4!dde|>)>>pWEk=MaYN@;M~`asNnOQKonxiWvRda`%X&8IOk25N>C2 zmE6e9+{*1dmdA4^Pvprwm8bJeo{i^_=aA>n!~Yy|pZop5?+1QA@ILqZf!`1OexPnw zx2xOL?eh2X_wx7h_wx7h_wx7h_wx7h_wx7h_wx7h_wx6@+vM-%@8$31@A)}D=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLL!_WCSKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBWji2*#e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCd=h@n&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzQsd|R zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKc6*z z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS$EB{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;U(xtEKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzH-?|{bAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{QQ-TpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzyM&+fbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QP^u&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QUbGKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqSz>%!0ZIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*RQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-L*wWC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKR;~z zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk z6n@Un`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^Pdkt=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqG=9#{`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6Zj^IvTIoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Hk8-C8u`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^WO?T=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqwi zxm>wixm>wixm>wixm>wixm>wixm>wixm>wixm>y2-)VBWa=CK3a=CIcax!uQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=bsUN&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^UrGhoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-J;@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSze&gr- zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKi>{N z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC ze5dhqe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS%PrQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaS2TXk&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hwo9DbhTsjq2Pt?iT5blJ&bCRwglH*6b?vs;V9zRi0~YrV_3dQ5A}Ud@{|ANBLy zwOBZ7UT7Vhj*|={oQdKj-KCoS*Y^e$LPN`MWlL&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=dTSv=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{Pm5W^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM~8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`3Hud^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNV!q53RKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK^8b9af{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`NxEx^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLP!_WCSKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LMy4nNQF)Ymkt*7nJ2y6j{zlPp)O8@7$c*{#K4 z-{w80wccf1J*Kr~ujb8~kNWxUS}dG3FSHI$$4Q0}&mT8*ZgupiMxWGGEY&$y>MW~u zzO}4pqt3LIE4f-%d#$eXdR@tl+{~@q&SQBzck)D@%u{(f>J9aVdPBXT-cWC-Hy)09 z!~1+cv)s+S+|PqNm*?|BUd&5*Id92Z<99;86Z)M{-L7s|x2xOL?do=QySm+XHomj* zosI8od}rf38{gUZ&c=5(zO(V2&2Kl~+4#=JcQ(GWk(2SAjqhxHXCn{C&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj^G^>y=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`DZqM&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxzbO2ipYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QPS8IX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LO|8h*~t`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^J|Tt^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luN38b9af z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`P1R& z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mU z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqM^Y?do^#?eALf(*J9zUd7*W1 zI!-c-c>cJdbE~7(8|n@9hI&K2q25q$T$XxcF-ui5tW?#oTJ^(P*0WKS!&a`ueeOPY zpS#cHa^-U6a=)?3<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC<;vyC z$;ipb$;ip@bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;@6z}=Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqgpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa*EN35&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;9}s@d&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz2Q_}q&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;A02+q&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz$2NY> z&-pn&=jZ&KpYwBm&d>QdKff$~p5v*nX;!W6lht(D$zmp1u2wf}8;!GDi^IOndrWJ+ z%eZ<>Ys+5En>8Qx^WC*rIBQ;L9h{Dn3?rUDZs^?VX!VAAL%pHiP;aO=)Ek$j-dN01 z)eI|DHLOQd zKj-KCoS*Y^e$LPNIX~y;pB8@3&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzXEc7!&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEVfZ;e=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMO8$ajg{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`IYcQdKj-KCoS(lX{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;+l`;|bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QOHBKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqSzQ{m_QoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmWS$bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZ)A%_*=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNg_<4?}zNT5Vwog{mWhaZ7 zWVu@1ux&KXZY>V`Ht#X5^)BP;F|93oHE-5@)X#U=RNPBM&m{1z;phCEpYwBm&d>SzJBFY0bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QNt^&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QSEcKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqSztHaOvIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*VMZ=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-ZR6+soS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKmXCj&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmVEVbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEVB_cfoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKmWPL&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYacbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEOylSL zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-bo@NWQ(x1pTH7b9>9Ui>OtM_9ZrC;&XSWuI zeVg}~)_Rw5^_bR{y_z>`KI-SYYq4QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLr!_WCSKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0mO#?SdV zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*OjF z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8huy zHGa;|`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^Dk=roS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-IL;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqwb|pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzTN^*;=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T3LK=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT+VFFJ&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{4)7@j;Fq+S+%xLR?}rCiTra8|sbARBtS1scMFmsv1_Sept(T zHmY)PpS#c9=k9a4T)AAi+^=tPxpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3 zxpKL3xpKL3GIBCQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzJ2igJ&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;uL?is=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0TkIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LNd)A%_*=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zv#qe`}&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEK;!59 zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXar z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze*PcB&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqW8(gXQL_y_qqGreeOP&%azNO%azNO%azNO%azNO%azNO%azNO z%azNO%azNO%azNO%l(Zemn)Ymmn)YmCnF~#CnG1r&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAJ9e!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA5ji2*#e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC{Er(y=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`9kC8{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lx#?SdVKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZ5q{3k`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^M4qA&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS$DVKhN>h*EFlv_Q`6x z>|`;MELW==wvEQwt;J#A<~^ph-ep`prnP0S=FOUq`uXl!ESxnjv<^)j3w`EUR_CwXA2O&a{;)xms6yt*-NWUCE8y%&pwcV|hGx!uR<;-~YPC z_xV2G=lgu0@AG}W&-eL0-{<>$pYQX1zR&mhKHum2e4p?0eZJ4t_&(piT)xlG_cP1g z+{^tu$a8rQdKj-KCoS*aae;R(y&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;-`V&%Kj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;|1$iXpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;Kiv2^ zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqis z_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z|84j=Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIY0j^;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T0jRe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zvm+*6b&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTqsGtqIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*SmD&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`4=>P z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKfe=x&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^CufW=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqTs@|>Wv}MVnveSV z?piFIH7~RdPRB`x5ziktbZ&L@s79aERV>vxR_ZLPb-uN%$8*SY$aCnjJ%`-q`r>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM>&ffM z>&ffM>&ffM>&ffM>&ffM>&ffM$?$W2&d>QdKj-KC{7;3S^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMigrDQdKj-KC{2dxU=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Y;lq=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`KuZ~=jZ&K zpYwBm&d>QdKj-KC{Qt9a-@lb~|6Rb}$G6zN9(%pJUaxoe>E-)P2SIwA1c~&HAiaUo zI|$Mnh$Oz=@;u*`-j&`#KzacS5~l|Q>CFhz8;F(y!)NUKwJkY+z+>w*(rC`iJ#%J$ znK>Wd*ZZ2E^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN`9b67{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX{2T@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqi=QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLb z!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0kK7l^X6dYXp>=RNPBM&m{=A`ctFuQn`lRk+ zxvsHVS6Qp;t!E?Z4fTe4L%pHiP;aO=E?d1}KHtkc`?;S7d6-9eAuq;!Zaz1ko6pVX z=5zD8{Js3W{Js3W{Js3W{Js3W{Js3W{Js3W{Js3W{QZA!^7r!h^7r!h^7r!h^7r!h z@^Ji|pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTr0{co&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTLF4EAoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmYHIpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^exAn9 z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9$ z8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN z`9}CTKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzX5;7loS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYwN8=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-BK(}6^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=U*Lu&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS$DkKhN>f*D|lx*2!+VY-gHDR;twv$42Aq)aI~niypJu?=r3)vpTX{>t?M- z{o>==ES)tkv<^`0?4|xw=z4wskd@u9t z=YAgKVIJj$c+NfNo^yFUc|Cc(uWj;r@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m@_O=m z@_O=m@_O=m@_O=m@_O=m@_KSI{G6ZjbAHax`8hv-qwsTn&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTYUAhpoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmX^ApYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Poj=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT8;zgybAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QTcEe$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa_Xt1d z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5U= zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T4(Z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;9}<4f&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSz-);PypYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;m(S00y!5rqtF?8qn=ad# zW|Ea^b;GgII6JjD?AxNptoFN%tH-R4?AE$j>rub>xHd~?%?quA({Ykv#PjD3om-tf zs?jHP7t3{x)w;@BU2i?=4fTe4L%pHiP;aO=E?>Q2KHtkc`?;S7d6-9eA?9=Qx%u3D zZaz1ko6qI%QdKj-Iv9DdHv`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^S|HtIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTTjS^a zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmVNY zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zn z;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lpy<{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;FNB}-bAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{QO$u=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-MdRoEoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKYu*@oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-IP6@Jdo`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zj^CufW=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqONYpm8)*6MoRL*7H)Ls#)VBV zJo~wy2YHxB@tk|kJ?HXz@_O=mU(@9EQdKmX3~bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZvmB!EcIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*VuIKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqSzTZfQdKj-KCoS*Y^e$LPNIX~y;{QQT*&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QO@xe$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aacMm`3 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5U; zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T5T^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*aa4-P-)=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`T4JhpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzW%TnLFMTcZYHgkDrptDw znPjC}-EeF)&Q5I(`?lyYtNkwH>M^S$yR~lCdQ?T>YnP?hHvW?B!Ra{3Fyi_1hVCs( z8P({Mx{Kwy#%f(QdKmVifbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6YEQsd|RoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmU`)&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmUK>=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxT+{VxOIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e%^(j^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luLp_&Go4=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxHyS_Z=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T2{DpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOIX#AX?^K*XA&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U?3}qo3z^>1&x+YwKh;UA8mL zBrDbGhGU~~c4~9jw?&Ux?RObhk69hrt#z~3qki#mZI;fO7g`6W<0Qj~=g%8Dw>o=N zqfhECmg^d;b(OWc-g-82t*&%EH*&M?_Ez2J?YfgYxtn|O9`YXY9=e?GAQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMQ8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`FAyb&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX{25@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lP z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lq5I{G6ZjbAHax`8hx5=lqlHwQkmW)Gt1+&C*%(LhImkoMagB{CPv?R%efD^hw>t za$RG!uCi9wThB(W)s?R2MsC*K-m3e&U3YRPcXKb^L*7H)LznYCBVJo~wy2YDEO zSNOZa-xcO_^SSxld@h$Omn)Ymmn)Ymmn)Ymmn)Ymmn)Ymmn)Ymmn)Ymm;3RN%azNO z%azNO%azNO%axPi=lqQdKj-KCoS*Y^e$LPNIY0ly z@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIY0m7#?SdVKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIY0lu;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAG-Se$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aahvDb^oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKVNVBoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-H!G=9#{`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^Lydv{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l3@N<67&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqE`Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luM#`gx9*zLt5lwoZ1_WjoVMvQn*XI5rw*r#6Ru zTlARKewT6enAMTpS~qJw>K7l^X6dYXp>=RNPBM&m{=A`ctFuQn`lRk+xvsHVS5a@M zH`E*I4fTe4L%nfX>kaeyUgp`){XB^I+QdKj-KC{0$pF z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj z^S25==jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{H+^5=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^LGtD=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KC{M{Qr=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Zj^A8L^=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KC{DT`m=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqlHwQkmW)Gt1+&C*%(LhImkoMagB z{CPv?R%efD^hw>ta$RG!uHrr9J>)%fRo_FN^S#WopZj?b&$;K^b1ttZuP3i3uP3i3 zuP3i3uP3i3uP3i3uP3i3uP3i3uP3i3uP3i3ulI40*OS+i*OS+i*OS+i*OQas=lqQdKj-KCoS*Y^e$LPNIY0k{@N<67&-pn&=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0lS#?SdVKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIY0l4@N<67&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqN{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX|C;pYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJA) z@pFF8&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`T3WJpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*Tq>pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lq@T{ zzxcQ|OJ~gst%K8Xl3~R2=M9}(ojt12Cv_Lgb&b`!%357-JsY`JSGt}XxmkC6tM2o5 z-N~JJ4|xxH4_)5(kmr0a^X%t-{9WPi3V&Ca&&}uNbMyJ@MJ`t^S1wmBS1wmBS1wmB zS1wmBS1wmBS1wmBS1wmBS1wmBS1wmBS1wmBS1wmhhM)6ue$LPNIX~y;{QTR(&-pn& z=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QNr_ zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz zTZW(WbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QL*P&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Y^OZYiI=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-J~*7!L;=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6YEK=?U7=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-Hk)c83+=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lq zm%f&HwYE-n(`7r;OtMm~Za6j?XQwuYeOvUH)qa<8^_bO>-C8$mJ?a-9*JkOgd7*W1 zI!-c-c>cVhbE~sQHTtCPV!5udT31=C>#b)a*Xl~wb0atFZg16n-mW{j6YnAKA@8Bf z`yTR~?`59-+>gI2{9WPi3iG-7+Qd zKj-KCoS*Y^e$LN7zVUN@&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxTjPP@Q&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LN7tMPMw&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=luMB_&Go4=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax*BU?P=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T4`f&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=U*0n&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KC{3{wi=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqbJ({49oemQv(}@2@o{aI&YBlm2dCpC!-(h48#=c-dsL%O z>MoY+8tM)8hI&K2q25q$s5dTiyQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hxT_QucoIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*PBW=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hxT{_t~t&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxT!N$+|IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*Vtk=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxTiSToN&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hxTsm9OwIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*XU9=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxTQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT)yB{HIX~y;{G6ZjbAHax`8hwo%6^{X zrLSdPt*w*YblJ`{ldM#$8;*^}*{RK8-xfV)wcll2J!W-ex7N*CkNU;OwOKlAUT7Vh zj*|={oQdKj-Hk*Z4U<=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEdiXg%=jZ&KpYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-J4+4wm>=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAG-be$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aaS>xyY zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKY!5p zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN# z!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX}PK_&Go4=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxzr68te$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KC{Brwwj+efcd9}7qcGG1$(@e5bt!_9r8fT|AhkaZ0nALularKzhk=wOVM>YDS?qa#Fv07JItLv?2BiHIm*K;E`>uzt=ecrA+ zxs$uOm&fvWp2(AVDo^K`Je%k8d|sB9=XJvO`99z0`+T48^L@V0_xV2G=lgu0@AG}W z&-eL0-{<>$pYQX1-?#I9zR&l$8sF#pm)rOG`CjHRpPSFk=jLQdKj-KC zoS*Y^e$LOot?_ey&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LOoFZ`UJ^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=Wp5gIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-pz(8l&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LN-Jp7!W^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=kL<^IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-vhj0%&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-Dg2zD^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=O57cIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN-rSWrq&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxU)p}2xW zWVhDMT95k0$F*5HYhGv_oQ{(WBc4BR=-le;QH?&SyI8JktkzZ5>U!(h$hErC_1wtK zy4zcIpSSBy?&NOn<*_`TC-P*T%F}r!&*r&2pO@w3@gDLX@*a9=zlS{Mdzr`g2fjb> z{ek)1_XoZ|@cn_hUEQv3SGUXG%iqi2%iqi2%iqi2%iqi2%iqi2%in)=QdKj-KCoS*Y^e$LN-H~gHR z^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=O5en zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LN- zzwvW^&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPTH2j>O^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=bzE|IX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LOA!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`T1(&=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hwo-}pH{=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luLj!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%PLQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqlHwQkmW)Gt1+&C*%(LhImkoMagB{CPv?R%efD^hw=?dPBXT-cWC-H`E*I zjmutdn9uiOJ~yA6&&}uNbMv|R{N<6qm%o?4m%o?4m%o?4m%o?4m%o?4m%o?4m%o?4 zm%o?4m%o?4m%o?4m%o?4mxtr${G6ZjbAHax`8hxT=J0cV&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT*2d5IIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e*R|R=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT-tcpN&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hxT{>IPwIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*TW(=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv73_s`R z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`8zj$ z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=kFVS&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPN`TIA1&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqV^&9YYu&8%s9$_so29enh1S98ILR>L`SXU(tQdKj-KC zoS*Y^e$LN-C;XhB^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=O5GfIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LN-ukmw!&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPTPxv`M=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;{G6Xp!_WCSKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJBC;phCEpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEpL{XEA@U(38&TPM5evYlxrS*cbx z92dvvk(H&^kCBCmBXOf8Nl!)!CyOeNuO^ zT-R8wtE|=a*0Ygob*1aMk(+h5x9UD`*PYzS-Q3G#c|1?#$vl;(^Gu%2b9p{53*YDa ze4p?0eZJ54`99z0`+T48^L@V0_xV2G=lgu0@AG}W&-Z=b&iDB~-{)$4pYLCO-#4F| z&&}uNbMv|R+QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luLz8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`J0BH^K*XA&-pn&=jZ&KpYwBm&d>Qd zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM8!q53RKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luNp8b9af{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPN`8$N4^K*XA&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM~!q53RKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luM~8$ajg{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`TK;Q^K*XA z&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luK^!q53R zKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=luMa z8b9af{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHaxe>?n~pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHaxf4A{-e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS%P6_&Go4=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax|9AK~Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*OjF=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hvdZv32|^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=kvzT`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAEm+{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;Ule}M&-pn&=jZ&KpYwBm&d>QdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHaxuY#ZFcV{*Zadv8R*tbQGS?zZjSC3g8*{yZ6)}wy$ac!2)nipCJr{g5Ui0986I=4D|RHLgm z)Envz^@e&wy`kQ?3iZY`%T+V1R@Jap^}~8La;++d>$#DeRegBQJ?EZt&-wF_%azNO z%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO%azNO%axOnlaZ5=li}z5 zoS*Y^e*TT&=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hv-dE@8&oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmX>&&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKmYFVbAHax`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6Y^S>xyYoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmXpw&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmXD2bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Y^W8>%ioS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKR;~zoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-J~9e&Qw`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^Pdkt=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj^IvTIoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hwo3VxpBrLSdPt*w*YblJ`{ldM#$8;*^}*{RK8-xfV)wcll2J!W-ex7N*C zkNU;OwOKlAUT7Vhj*|={oTra8|n@9hI->F)Em<*SIw|mRl{1< z59`^;wW=Jh=SFT;_2D`9oO{ka=W@AnxpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3 zxgQa^T)AAiT)AAiT)AAiTsavz895m_8Gg>s`8hx5=lqQdKj-Ja6@Jdo`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6Zj^N(u$oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-Ja)A%_*=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KFEBu_F^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=bzg6IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LOw;phCEpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS#n{Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzUgPKdoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKi>*J=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCe7o^;e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS%PjQdKj-KCoS*Y^ ze$LPNIX~y;{G6ZjbAHax`8hx5=lqlHwQkmW)Gt1+&C*%(LhImkoMagB{CPv?R%efD^hw>t za$RG!uCi9wThB(W)s?R2MsC*K-m3e&U3YRPcXKa~QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{QQl>&-pn&=jZ&K zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*UJ7pYwBm z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T5(0 zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{QMmnKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSzk2QYI&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQd zKj-KCoS*aa{}g`C&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;ztH$OKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqM^S$yR~lCdekpIuFcX} z^Fr(3bev=u@%(v1=T>KrYV=9n#d2L^wXU*O*IUm>UMRz-`V(`jo;b$osHkw_??a4+4!B!hsW=1{LaSj zZ2Zo~?`-_e#_w$8Wc<#??`-_eMjnoz^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^ ze$LPTYxp@o=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZvcH`&#oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmSMJ=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hxTpW)~HoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKmU}*&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN`R9e7^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hx5=lp!q_&Go4=lqQdKj-KC zoS*Y^e$LPNIX~y;{G6ZjbAHaxzo7ARe$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS)weKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqSzYr@a@IX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^etxU*bAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6YEQRCSzRq^v2FMTcZYHgkDrptDwnPjC}-EeF) z&Q5I(`?lyYtNkwH>M^S$yR~lCdekpIuFcX}^Fr(3bev=u@%(v1=T>K{H`E*I4fTe4 zL%pHixGMF=G|N>ptX9>qR`tVrHgc^hhwHf!&$;K^bM85RHgdUgxpKL3xpKL3xpKL3 zxpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3xpKL3GIBCQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=g&8O z&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=YKu?oS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KFGW?vM^K*XA&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=Wo*ZIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e$LOoyYX{=&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^ ze$LPNIX~y;{G6Zv+wgOK&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hxT(ZQdKj-KCoS*Y^e*U+^&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*WH#pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=lqtr`wwlmEnE7j_TW213)YIE4PMUPqScNtfYSsmG}b+gu^e(`Z_md=_N zS_h}&B*Td3&l@_oI$OP=-cWC-H`E*I4fV!VsW+xsu9{)Bs)n_yAJ(&xYgIX1&y9G_ zJ?EZt&$(Q#T&`TMT&`TMT&`TMT&`TMT&`TMT&`TMT&`TMT<(WOE>|vBE>|vBE>|vB zE>})QPDV~fPKKZJbAHax`8hx5=lqSze+fV5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`T0jQe$LPN zIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*aa-)j7v zpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z|0DdIpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHaxm%`8aIX~y;{G6ZjbAHax`8hx5=lqQdKj-KC zoS*Y^e*RCx&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAHax`8hx5=bzvBIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LNtgrDQdKj-KC{MF&-{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~ZO{G6ZjbAHax`8hx5=lqQd zKj-KCoS*Y^e$LPNIX~y;U)cCLKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqX(m~zRyQ0Qjk8mm!@ezg%xb^O zxO&X$$ZoBhwI210k887Z*1XU+v4)9`YW#I`1LR`CjJP&;2~e!#v6hc`>iZEAy&&&OPUz^JgNjC$A^3C$A^3C$A^3 zC$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$A^3C$A^3$Is>UQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAJ9@ zQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6Zj zbAJBU!q53RKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAJBDji2*#e$LPNIX~y;{G6Zv|90&*G>*Iu;P`*ry>_j+W}BwzHkYQ`G;OzO znr@%d=QJD=G>Aw%4hbR>jzfZoghMo*=NS)?C?Y{bf`|lB3343aJSB)o@QENI z!6$-<1QiJ)*8gq}s=ePkUmbiVOn2s+nc002IQdKmSgFpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KCoS*Y^e$LPNIX~y;{G6XZTHxpWoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5 z=lqQdKmS32pYwBm&d>QdKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=lqQdKmS?c=lqQdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hv-qQKAjIX~y;{G6ZjbAHax`8hx5 z=lqQdKj-KCoS*Y^e*TLBKj-KCoS*Y^e$LPNIX~y; z{G6ZjbAHax`8hx5=luMQM-fG|8O)aS-?Me@&Qr>E5w4KFN$uFyq7hXGJ zPu_1y#eBq`TyEsGoY#$o*M6N}cHG~5-PZlQdKj-KCoS*Y^e$LPNIX~y; z{QRi`Kj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqSz9}4`OpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqQd zKj-KCoS*aaza@Uo&-pn&=jZ&KpYwBm&d>QdKj-KCoS*Y^e$LPNIX~y;{G6ZjbAHax z`8hx5=lqQdKj-KC zoS*Y^e$LPNIX~y;g~ZSKIX~y;{G6ZjbAHax`8hx5=lqQdKj-KCoS*Y^e*SFY=lqQdKj-KCoS*Y^e$LPN zIX~y;{G6ZjbAHax`8hw|S>WgVoS*Y^e$LPNIX~y;{G6ZjbAHax`8hx5=lqg~2Ys-Q?JI}ZS=`qa_EKAW*vH#C!oJj2 z343{eXV@q9cZGd_dlcg?;YOP}nyQ4TpWC^U<)+cfKX; zTb(0eAARJlVPAOUv9PDEw}pMIYc%YOU3bG?I{ZY~#}B_F>`RBo!d~uvSJ)@I-yQbl z?)QYfa%4R0lSiHm`^u50!d|VuFYHs*iLkF$-yinc(GP@u`sfG4zIJpn?Dd|f!#>mV zOxV|ZJ{0!BV^d+DJ@(9BA1J{$It;~xwA{PB;6ee3v4*hf!1 z7xsk{&xbvod?D;(CuhUHc=E-tmujC2`*`j1VPC4vg}r?0i(#KQ^`)>cpZapxD}D1} zpX~ce*jM_#8usexuZ4Z;^g`HIPk%k^wf=8}eY*dfVPETC414{|x57Si=G$RkKl9zN z4-PDaeRkk`Vc!^dDeOaMUk>}++2ycro_!_kBlTCqK41S~*thB{VIMvBqp&ZW`*GOQ z`JaS+?EGrj7tjAR?4=7o5BvCqUxt0@!dlqN7k?G@iHpAu`|`!#guQZUJ?xX0ejE0c zOTQ0$_3|IXK6QB`?5mgm6!zMcKZkw#%3s31c4af{^})Y}eP;0QVP7BoN7x6iZiRjJ z>OaH2arIweAG-GMu+Lpf%?JMN|3%aN=l^Rv!aj1n820(=yTZP8y%hG*8+*dOaHA#c z>E_>$F%3QoGSU7YD&AC-&Wh3-(T}?vGKd1ymmEq^4eG$=e#$srREvM=D#h?{g%AF aySd*utZ}_!^Cw@wHKjXu@4o$==l%mHi9*x> diff --git a/tests/queries/0_stateless/test_dozlem/nullable_arrays.arrow b/tests/queries/0_stateless/test_dozlem/nullable_arrays.arrow deleted file mode 100644 index 2e497d358c6da6916baae0127b59487ae1f55ec9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1322 zcmeHHyG{c^44lhdLKGG$#R*YRTnQ!6P!eh+P*Ok%5)q^bpd$qZ1tn!Zg97mdd;|rb z!9RqVJzJc@tDwcvu6Jzj*z3JbuixK4+yG{j;{@nJ(m_BAO|-e9BpES?+MF|+ft2W( z<1|`wNB_G_tD`Q@2oQ3&S@0zVyFFT`s`l&a0tZ5;3qielN8b6;Qi)ScAL}AL= zJ(H*>l7MCU)#(sfk!G>xY31hJAr8!6-NhR%%BSADEcxOKtyqa`oll~^NZhV7*Tu2> zx6SGMcMSVEjtnoFJ~!>kvGd)S!};zEyI)XuUq7d8ewPga%6ERZyRM=??{^OIYBVj- z>+(GFado?VI=2tu!R2^ZTu;i;#hG9f)iR&QT*#l+ceU2{A+0_HekFPZfk^L9yLAt` b8}X>dD=T#Bnzp+9zuCXNS-sU>{EvJGKXY;* From 7e2be8f93c234ec0a8a3af796b79b644cf78f635 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 19:02:26 +0300 Subject: [PATCH 31/40] blog article about perf tests --- website/blog/en/2021/performance-test-1.md | 83 ++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 website/blog/en/2021/performance-test-1.md diff --git a/website/blog/en/2021/performance-test-1.md b/website/blog/en/2021/performance-test-1.md new file mode 100644 index 00000000000..8c2259e59d7 --- /dev/null +++ b/website/blog/en/2021/performance-test-1.md @@ -0,0 +1,83 @@ +--- +title: 'Testing the Performance of ClickHouse' +image: 'https://blog-images.clickhouse.tech/en/2021/code-review/two-ducks.jpg' +date: '2021-08-19' +author: '[Alexander Kuzmenkov](https://github.com/akuzm)' +tags: ['testing', 'performance'] +--- + +One of the main selling points of ClickHouse is that it's very fast, in many cases utilizing the hardware up to the theoretical limits. This was noted by many independent benchmark such as [this one](http://brandonharris.io/redshift-clickhouse-time-series/). This speed boils down to a right combination of architectural choices and algorithmic optimizations, sprinkled with a dash of pixie dust. There is an [overview of these factors](https://clickhouse.tech/docs/en/faq/general/why-clickhouse-is-so-fast) on our website, or a talk by the ClickHouse lead developer Alexey Milovidov ["The secrets of ClickHouse performance optimizations"](https://www.youtube.com/watch?v=ZOZQCQEtrz8). But this is a static picture of "how the things are". Software is a living and changing organism, and ClickHouse is changing very fast — to give you a scale, in July 2021 we merged 319 pull requests made by 60 different authors ([live statistics here](https://gh-api.clickhouse.tech/play?user=play#c2VsZWN0IGRhdGVfdHJ1bmMoJ21vbnRoJywgY3JlYXRlZF9hdCkgbW9udGgsIHVuaXEoY3JlYXRvcl91c2VyX2xvZ2luKSBhdXRob3JzLCB1bmlxKG51bWJlcikgcHJzIGZyb20gZ2l0aHViX2V2ZW50cyB3aGVyZSByZXBvX25hbWUgPSAnQ2xpY2tIb3VzZS9DbGlja0hvdXNlJyBhbmQgbm90IGhhc0FueShsYWJlbHMsIFsncHItYmFja3BvcnQnLCAncHItZG9jdW1lbnRhdGlvbicsICdwci1jaGVycnlwaWNrJ10pIGFuZCBtZXJnZWQgYW5kIGNyZWF0ZWRfYXQgYmV0d2VlbiAnMjAyMC0wOS0wMScgYW5kICcyMDIxLTA5LTAxJyBncm91cCBieSBtb250aA==)). Any quality that is not actively selected for is going to be lost in this endless stream of changes, and the performance is no exception. For this reason, we have to have some process that allows us to ensure than ClickHouse always stays fast. + +# Measuring and Comparing the Performance + +How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test varions layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). + +Micro-benchmarks are normally ran by a developer while working on the code, but it is not practical to manually run the entire battery of the end-to-end tests for each change. We use an automated system that does this for each pull request as part of continuous integration checks. It measures whether the code changes introduced by a pull request influenced the performance, for which kinds of queries and by how much, and alerts the developer if there is a regression. Here is how a typical report looks. + + + +To talk about "changes in performance", we first have to measure this performance. The most natural measure for a single query is elapsed time. It is susceptible to random variations, so we have to take several measurements and average them in some way. From the application point of view, the most interesting statistic is maximum. We want to guarantee that e.g. an analytical dashboard built on ClickHouse is responsive. However, the query time can grow almost without limit due to random factor such as sudden disk load spikes or network delays, so using the maximum is not practical. The minimum is also interesting — after all, there is a theoretical bound on it. We know that the particular algorithm can run only so fast on the particular hardware, in ideal conditions. But if we only look at the minimum, we are going to miss cases where some runs of the query are slow and some are not (e.g. boundary effects in some cache). So we compromise by measuring the median. It is a robust statistic that is reasonably sensitive to outliers and stable enough against noise. + +After measuring the performance, how do we determine that it has changed? Due to various random and systematic factors, the query time always drifts, so the number always changes, but the question is whether this change is meaningful. If we have an old version of the server, and a new version of the server, are they going to consistently give a different result for this query, or was it just a fluke? To answer this, we have to employ some statistical method. The core idea of these methods is comparing the observed values to some reference distribution, and deciding whether what we observed can plausibly belong to this distribution, or, on the contrary, it cannot, which means that the performance characteristics of the patched server are indeed different. + +Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works for well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. Howewer, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this troughput. + +For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance task goes to a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. + +We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is thorougly described in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. + +This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of mediand _D_: +1. abs(_D_) <= _T_ — the changes are not statistically significant, +2. abs(_D_) <= 5% — the changes are too small to be important, +3. abs(_T_) >= 10% — the test query has excessive run time variance that leads to poor sensitivity, +4. finally, abs(_D_) >= _T_ and abs(_D_) >= 5% — there are statistically significant changes of significant magnitude. + +The most interesting case are the unstable queries _(3)_. When the query run time varies significantly even on the same version of server, it means we won't be able to detect any changes of performance of this query, because these changes are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? + +# Understanding the Reasons Behind the Changes + +An investigation of code performance often starts with applying a profiler. On Linux, you would use `perf`, a sampling profiler that periodically collects the stack trace of the process, so that you can then see an aggregate picture of where your program spends the most time. In ClickHouse, we actually have a built-in sampling profiler that saves results into a system table, so no external tools are needed. It can be enabled for all queries or for a particular one, by passing the settings [as described in the docs](https://clickhouse.tech/docs/en/operations/optimizing-performance/sampling-query-profiler/). It is on by default, so if you use a recent version of ClickHouse, you already have a combined profile of your production server load. To visualize it, we can use a well-known script for building [flamegraphs](https://github.com/brendangregg/FlameGraph): +``` +clickhouse-client -q "SELECT + arrayStringConcat( + arrayMap( + x -> concat(splitByChar('/', addressToLine(x))[-1], + '#', demangle(addressToSymbol(x))), + trace), + ';') AS stack, + count(*) AS samples +FROM system.trace_log +WHERE trace_type = 'Real' +AND query_id = '4aac5305-b27f-4a5a-91c3-61c0cf52ec2a' +GROUP BY trace" \ +| flamegraph.pl + +``` + +As an example, let's use the test run we've seen above. The tested [pull request](https://github.com/ClickHouse/ClickHouse/pull/26248) is supposed to speed up the `sum` aggregate function for nullable integer types. Let's look at the query #8 of the test 'sum': `SELECT sum(toNullable(number)) FROM numbers(100000000)`. The test system reported that its performance increased by 38.5%, and built a "differential" variant of flamegraph for it, that shows the relative time spent in different functions. We can see that the function that calculates the sum, `DB::AggregateFunctionSumData::addManyNotNull`, now takes 15% less time. + + + +To get more leads into why the performance has changed, we can check how the various query metrics have changed between the old and the new servers. This includes all the metrics from `system.query_log.ProfileEvents`, such as `SelectedRows` or `RealTimeMicroseconds`. ClickHouse also tracks the hardware CPU metrics such as the number of branch or cache misses, using the Linux `perf_event_open` API. After downloading the test output archive, we can use a simple ad hoc [script](https://gist.github.com/akuzm/bb28a442f882349e0a5ec2b5262b97d0) to build some statistics and graphs of these metrics. + + + +This graph shows the number of branch instructions per second, on the old and the new server. We can see that the number of branch instructions has dramatically decreased, which might explain the performance difference. The tested pull request removes some `if`s and replaces them with multiplication, so this explanation sounds plausible. + +While side-to-side comparison is more robust against the systemic errors, the historical data is still very valuable for finding where a regression was introduced or investigating the unstable test queries. This is why we save the results of all test runs into a ClickHouse database. Let's consider the same query #8 from the `sum` test. We can build the history of performance changes with this [SQL query](https://play-ci.clickhouse.tech/play?user=play#V0lUSCAwLjA1IEFTIHMKU0VMRUNUIG9sZF9zaGEsIG5ld19zaGEsIGV2ZW50X3RpbWUsIG1lc3NhZ2UsIG9sZF92YWx1ZSBBUyBgb2xkIHNlcnZlcmAsICAgbmV3X3ZhbHVlIEFTIGBuZXcgc2VydmVyYCwgYmVmb3JlIEFTIGBwcmV2IDExIHJ1bnNgLCBhZnRlciBBUyBgbmV4dCAxMSBydW5zYCwgICAgZGlmZiBBUyBgZGlmZiwgcmF0aW9gLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsIEFTIGBzdGF0IHRocmVzaG9sZCwgcmF0aW8sIGhpc3RvcmljYWxgLCBzdGF0X3RocmVzaG9sZCBBUyBgc3RhdCB0aHJlc2hvbGQsIHJhdGlvLCBwZXItcnVuYCwgY3B1X21vZGVsLHF1ZXJ5X2Rpc3BsYXlfbmFtZQpGUk9NIAooU0VMRUNUICosIHJ1bl9hdHRyaWJ1dGVzX3YxLnZhbHVlIEFTIGNwdV9tb2RlbCwKICAgICAgICBtZWRpYW4ob2xkX3ZhbHVlKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIDExIFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIGJlZm9yZSwKICAgICAgICBtZWRpYW4obmV3X3ZhbHVlKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIENVUlJFTlQgUk9XIEFORCAxMSBGT0xMT1dJTkcpIEFTIGFmdGVyLAogICAgICAgIHF1YW50aWxlRXhhY3QoMC45NSkoYWJzKGRpZmYpKSBPVkVSIChQQVJUSVRJT04gQlkgcnVuX2F0dHJpYnV0ZXNfdjEudmFsdWUsIHRlc3QsIHF1ZXJ5X2luZGV4LCBxdWVyeV9kaXNwbGF5X25hbWUgT1JERVIgQlkgZXZlbnRfZGF0ZSBBU0MgUk9XUyBCRVRXRUVOIDM3IFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIHN0YXRfdGhyZXNob2xkX2hpc3RvcmljYWwKICAgIEZST00gcGVyZnRlc3QucXVlcnlfbWV0cmljc192MgogICAgTEVGVCBKT0lOIHBlcmZ0ZXN0LnJ1bl9hdHRyaWJ1dGVzX3YxIFVTSU5HIChvbGRfc2hhLCBuZXdfc2hhKQogICAgV0hFUkUgKGF0dHJpYnV0ZSA9ICdsc2NwdS1tb2RlbC1uYW1lJykgQU5EIChtZXRyaWMgPSAnY2xpZW50X3RpbWUnKQogICAgICAgIC0tIG9ubHkgZm9yIGNvbW1pdHMgaW4gbWFzdGVyCiAgICAgICAgQU5EIChwcl9udW1iZXIgPSAwKQogICAgICAgIC0tIHNlbGVjdCB0aGUgcXVlcmllcyB3ZSBhcmUgaW50ZXJlc3RlZCBpbgogICAgICAgIEFORCAodGVzdCA9ICdzdW0nKSBBTkQgKHF1ZXJ5X2luZGV4ID0gOCkKKSBBUyB0CkFOWSBMRUZUIEpPSU4gYGdoLWRhdGFgLmNvbW1pdHMgT04gbmV3X3NoYSA9IHNoYQpXSEVSRQogICAgLS0gQ2hlY2sgZm9yIGEgcGVyc2lzdGVudCBhbmQgc2lnbmlmaWNhbnQgY2hhbmdlIGluIHF1ZXJ5IHJ1biB0aW1lLCBpbnRyb2R1Y2VkIGJ5IGEgY29tbWl0OgogICAgLS0gMSkgb24gYSBoaXN0b3JpY2FsIGdyYXBoIG9mIHF1ZXJ5IHJ1biB0aW1lLCB0aGVyZSBpcyBhIHN0ZXAgYmV0d2VlbiB0aGUgYWRqYWNlbnQgY29tbWl0cywKICAgIC0tIHRoYXQgaXMgaGlnaGVyIHRoYW4gdGhlIG5vcm1hbCB2YXJpYW5jZSwKICAgICgoKGFicyhhZnRlciAtIGJlZm9yZSkgLyBpZihhZnRlciA+IGJlZm9yZSwgYWZ0ZXIsIGJlZm9yZSkpIEFTIHN0ZXBfaGVpZ2h0KSA+PSBncmVhdGVzdChzLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsKSkKICAgIC0tIDIpIGluIHNpZGUtdG8tc2lkZSBjb21wYXJpc29uIG9mIHRoZXNlIHR3byBjb21taXRzLCB0aGVyZSB3YXMgYSBzdGF0aXN0aWNhbGx5IHNpZ25pZmljYW50IGRpZmZlcmVuY2UKICAgIC0tIHRoYXQgaXMgYWxzbyBoaWdoZXIgdGhhbiB0aGUgbm9ybWFsIHZhcmlhbmNlLAogICAgICAgIEFORCAoYWJzKGRpZmYpID49IGdyZWF0ZXN0KHN0YXRfdGhyZXNob2xkLCBzdGF0X3RocmVzaG9sZF9oaXN0b3JpY2FsLCBzKSkKICAgIC0tIDMpIGZpbmFsbHksIHRoaXMgc2lkZS10by1zaWRlIGRpZmZlcmVuY2UgaXMgb2YgbWFnbml0dWRlIGNvbXBhcmFibGUgdG8gdGhlIHN0ZXAgaW4gaGlzdG9yaWNhbCBncmFwaHMuCiAgICAgICAgQU5EIChhYnMoZGlmZikgPj0gKDAuNyAqIHN0ZXBfaGVpZ2h0KSkKb3JkZXIgYnkgZXZlbnRfdGltZSBkZXNjCmZvcm1hdCBWZXJ0aWNhbAoKCg==) to the live ClickHouse CI database. Open the link and run the query so that you can examine the query and see the result for yourself. There were three significant changes of performance throughout the test history. The most recent is a speedup in PR we started with. The second speedup is related to fully switching to clang 11. Curiously, there is also a small slowdown introduced by a PR that was supposed to speed it up instead. + +# Usability Considerations + +Regardless of how it works inside, a test system must be actually usable as a part of the development process. First and foremost, the false positive rate should be as low as possible. False positives are costly to investigate, and if they happen often, developers perceive the test as generally unreliable and tend to ignore the true positives as well. The test must also provide a concise report that makes it obvious what went wrong. We have not really succeeded in this. This test has many more failure modes than a plain functional test, and worse, some of these failures are quantitative, not binary. Much of the complexity is essential, and we try to alleviate it by providing good documentation and linking to the relevant parts of it right from the report page. Another important thing is that the user must be able to investigate a problematic query post-mortem, without running it again locally. This is why we try to export every metric and every intermediate result we have, in easily-manipulated plain text formats. + +Organizationally, it is hard to prevent devolving into a system that does a lot of busywork to just show a green check without giving any insight. I like to call this process "mining the green check", by analogy to cryptocurrencies. Our previous system did just that. It used increasingly complex heuristics tailored to each test query to prevent false positives, restarted itself many times if the results didn't look good, and so on. Ultimately, it wasted a lot of processing power without giving the real picture of the server performance. If you wanted to be sure that the performance did or did not change, you had to recheck by hand. This sorry state is the result of how the incentives are aligned around development — most of the time, the developers just want to merge their pull requests and not be bothered by some obscure test failures. Writing a good performance test query is also not always simple. Just any other query won't do — it has to give predictable performance, be not too fast and not too slow, actually measure something, and so on. After gathering more precise statistics, we discovered that several hundred of our test queries don't measure anything meaningful, e.g. they give a result that varies by 100% between runs. Another problem is that the performance often changes in statistically significant ways (true positive) with no relevant code changes (due to e.g. random differences in layout of the executable). Given all these difficulties, a working performance test system is bound to add noticeable friction to the development process. Most of the "obvious" ways to remove this friction ultimately boil down to "mining the green check". + +Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unresonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. + +This concludes the overview of the ClickHouse performance test system. Stay tuned for the next article where we will discuss the most problematic kind of a performance test failure — the unstable query run time. + +_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm)_ + +References: + +1. Box, Hunter, Hunter, 2005. Statistics for experimenters, p. 78: A Randomized Design Used in the Comparison of Standard and Modified Fertilizer Mixtures for Tomato Plants. \ No newline at end of file From 6fa539e9529465c6a5a9beb5205097908928da92 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 20:17:25 +0300 Subject: [PATCH 32/40] fixes --- website/blog/en/2021/performance-test-1.md | 26 +++++++++++----------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/website/blog/en/2021/performance-test-1.md b/website/blog/en/2021/performance-test-1.md index 8c2259e59d7..3d15a7ea3ec 100644 --- a/website/blog/en/2021/performance-test-1.md +++ b/website/blog/en/2021/performance-test-1.md @@ -1,6 +1,6 @@ --- title: 'Testing the Performance of ClickHouse' -image: 'https://blog-images.clickhouse.tech/en/2021/code-review/two-ducks.jpg' +image: 'https://blog-images.clickhouse.tech/en/2021/performance-testing-1/chebu-crop.jpg' date: '2021-08-19' author: '[Alexander Kuzmenkov](https://github.com/akuzm)' tags: ['testing', 'performance'] @@ -10,7 +10,7 @@ One of the main selling points of ClickHouse is that it's very fast, in many cas # Measuring and Comparing the Performance -How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test varions layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). +How do we know it is fast, in the first place? We do a lot of benchmarks, many kinds of them. The most basic kind of a benchmark is a micro-benchmark, that doesn't use the full code of the server and tests a particular algorithm in isolation. We use them to choose a better inner loop for some aggregate function, or to test various layouts of hash tables, and so on. For example, when we discovered that a competing database engine completes a query with `sum` aggregate function twice as fast, we tested a couple of dozen implementations of `sum` to ultimately find the one that gives the best performance (see [a talk](https://www.youtube.com/watch?v=MJJfWoWJq0o) about this, in Russian). But testing a particular algorithm by itself is not enough to say how the entire query is going to work. We have to also make end-to-end measurements of entire queries, often using the real production data, because the particulars of the data (e.g. the cardinality and the distribution of values) heavily influence the performance. Currently we have about 3000 end-to-end test queries organized into about 200 [tests](https://github.com/ClickHouse/ClickHouse/tree/6c4c3df96e41425185beb0c471a8dde0ce6f25a7/tests/performance). Many of them use real data sets, such as the [production data of Yandex.Metrica](https://clickhouse.tech/docs/en/getting-started/example-datasets/metrica/), obfuscated with `clickhouse-obfuscator` as described [here](https://habr.com/ru/company/yandex/blog/485096/). Micro-benchmarks are normally ran by a developer while working on the code, but it is not practical to manually run the entire battery of the end-to-end tests for each change. We use an automated system that does this for each pull request as part of continuous integration checks. It measures whether the code changes introduced by a pull request influenced the performance, for which kinds of queries and by how much, and alerts the developer if there is a regression. Here is how a typical report looks. @@ -20,19 +20,19 @@ To talk about "changes in performance", we first have to measure this performanc After measuring the performance, how do we determine that it has changed? Due to various random and systematic factors, the query time always drifts, so the number always changes, but the question is whether this change is meaningful. If we have an old version of the server, and a new version of the server, are they going to consistently give a different result for this query, or was it just a fluke? To answer this, we have to employ some statistical method. The core idea of these methods is comparing the observed values to some reference distribution, and deciding whether what we observed can plausibly belong to this distribution, or, on the contrary, it cannot, which means that the performance characteristics of the patched server are indeed different. -Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works for well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. Howewer, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this troughput. +Choosing the reference distribution is the starting point. One way to obtain it is to build a mathematical model of the process. This works well for simple things like tossing a coin a fixed number of times. We can analytically deduce that the number of heads we get follows the binomial distribution, and get a confidence interval on this number, given the required [level of significance](https://en.wikipedia.org/wiki/P-value#Definition_and_interpretation). If the observed number of heads doesn't belong to this interval, we can conclude that the coin is biased. However, modeling the query execution from first principles is too complex. The best we can do is to use the hardware capabilities to estimate how fast the query could run, in principle, and try to achieve this throughput. -For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance task goes to a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. +For complex processes which resist modeling, a practical option is to use the historical data from the same process. We actually used to do this for ClickHouse. For each tested commit, we measured the run times for each test query and saved them into a database. We could compare the patched server to these reference values, build graphs of changes over time and so on. The main problem with this approach is systematic errors induced by environment. Sometimes the performance testing task ends up on a machine with dying HDD, or they update `atop` to a broken version that slows every kernel call in half, et cetera, ad infinitum. This is why now we employ another approach. -We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is thorougly described in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. +We run the reference version of the server process and the tested version, simultaneously on the same machine, and run the test queries on each of them in turn, one by one. This way we eliminate most systematic errors, because both servers are equally influenced by them. We can then compare the set of results we got from the reference server process, and the set from the test server process, to see whether they look the same. Comparing the distributions using two samples is a very interesting problem in itself. We use a non-parametric bootstrap method to build a randomization distribution for the observed difference of median query run times. This method is described in detail in [[1]](#ref1), where they apply it to see how changing a fertilizer mixture changes the yield of tomato plants. ClickHouse is not much different from tomatoes, only we have to check how the changes in code influence the performance. -This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of mediand _D_: -1. abs(_D_) <= _T_ — the changes are not statistically significant, -2. abs(_D_) <= 5% — the changes are too small to be important, -3. abs(_T_) >= 10% — the test query has excessive run time variance that leads to poor sensitivity, -4. finally, abs(_D_) >= _T_ and abs(_D_) >= 5% — there are statistically significant changes of significant magnitude. +This method ultimately gives a single threshold number _T_: what is the largest difference in median query run times between old and new server, that we can observe even if nothing has changed. Then we have a simple decision protocol given this threshold _T_ and the measured difference of medians _D_: +1. _abs(D) <= T_ — the changes are not statistically significant, +2. _abs(D) <= 5%_ — the changes are too small to be important, +3. _abs(T) >= 10%_ — the test query has excessive run time variance that leads to poor sensitivity, +4. finally, _abs(D) >= T and abs(D) >= 5%_ — there are statistically significant changes of significant magnitude. -The most interesting case are the unstable queries _(3)_. When the query run time varies significantly even on the same version of server, it means we won't be able to detect any changes of performance of this query, because these changes are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? +The most interesting case are the unstable queries _(3)_. When the elapsed time changes significantly between runs even on the same version of server, it means we won't be able to detect the changes of performance, because they are going to be drowned out by the noise. Such queries tend to be the most difficult to debug, because there is no straightforward way to compare "good" and "bad" server. This topic deserves its own article which we will publish next. For now, let's consider the happy path _(4)_. This is the case of real and notable changes in performance that this system is intended to catch. What do we do next? # Understanding the Reasons Behind the Changes @@ -72,11 +72,11 @@ Regardless of how it works inside, a test system must be actually usable as a pa Organizationally, it is hard to prevent devolving into a system that does a lot of busywork to just show a green check without giving any insight. I like to call this process "mining the green check", by analogy to cryptocurrencies. Our previous system did just that. It used increasingly complex heuristics tailored to each test query to prevent false positives, restarted itself many times if the results didn't look good, and so on. Ultimately, it wasted a lot of processing power without giving the real picture of the server performance. If you wanted to be sure that the performance did or did not change, you had to recheck by hand. This sorry state is the result of how the incentives are aligned around development — most of the time, the developers just want to merge their pull requests and not be bothered by some obscure test failures. Writing a good performance test query is also not always simple. Just any other query won't do — it has to give predictable performance, be not too fast and not too slow, actually measure something, and so on. After gathering more precise statistics, we discovered that several hundred of our test queries don't measure anything meaningful, e.g. they give a result that varies by 100% between runs. Another problem is that the performance often changes in statistically significant ways (true positive) with no relevant code changes (due to e.g. random differences in layout of the executable). Given all these difficulties, a working performance test system is bound to add noticeable friction to the development process. Most of the "obvious" ways to remove this friction ultimately boil down to "mining the green check". -Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unresonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. +Implementation-wise, our system is peculiar in that it doesn't rely on well-known statistical packages, but instead heavily uses `clickhouse-local`, a tool that turns the ClickHouse SQL query processor into a [command line utility](https://altinity.com/blog/2019/6/11/clickhouse-local-the-power-of-clickhouse-sql-in-a-single-command). Doing all the computations in ClickHouse SQL helped us find bugs and usability problems with `clickhouse-local`. The performance test continues to work in dual purpose as a heavy SQL test, and sometimes catches newly introduced bugs in complex joins and the like. The query profiler is always on in the performance tests, and this finds bugs in our fork of `libunwind`. To run the test queries, we use a third-party [Python driver](https://github.com/mymarilyn/clickhouse-driver). This is the only use of this driver in our CI, and it also helped us find some bugs in native protocol handling. A not so honorable fact is that the scaffolding consists of an unreasonable amount of bash, but this at least served to convince us that running [shellcheck](https://github.com/koalaman/shellcheck) in CI is very helpful. This concludes the overview of the ClickHouse performance test system. Stay tuned for the next article where we will discuss the most problematic kind of a performance test failure — the unstable query run time. -_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm)_ +_2021-08-20 [Alexander Kuzmenkov](https://github.com/akuzm). Title photo by [Alexander Tokmakov](https://github.com/tavplubix)_ References: From 9013892a965d186e229bb1e64c28cd125d5cdd8f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 20:56:16 +0300 Subject: [PATCH 33/40] make the sql-standard window functions case insensitive --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 1b8406682ea..132bdb7b327 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1764,21 +1764,21 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("dense_rank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("lagInFrame", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) @@ -1799,7 +1799,7 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared( name, argument_types, parameters); - }, properties}); + }, properties}, AggregateFunctionFactory::CaseInsensitive); } } From 941eba908c406bbfadc90ae8ed01987603512f57 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Aug 2021 21:38:06 +0300 Subject: [PATCH 34/40] Bump librdkafka (to fix metadata cache destroying) This should fix CI under TSan [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/0/9292869c4f92664a28b8c9ddef1e62ddfd13b285/integration_tests_(thread).html Refs: edenhill/librdkafka#3279 --- contrib/librdkafka | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 43491d33ca2..b8554f16820 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 43491d33ca2826531d1e3cae70d4bf1e5249e3c9 +Subproject commit b8554f1682062c85ba519eb54ef2f90e02b812cb From 30dd965e45ed63deffaece4893309934990e68e2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 21:39:06 +0300 Subject: [PATCH 35/40] boop From a7d405759cabc85f7a3a5ada99943102eb32274c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 19 Aug 2021 21:43:43 +0300 Subject: [PATCH 36/40] fix decimal formatting settings in perf test --- docker/test/performance-comparison/compare.sh | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index e5c9f349ce3..c97e8a6ed2b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -628,9 +628,6 @@ cat analyze/errors.log >> report/errors.log ||: cat profile-errors.log >> report/errors.log ||: clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view query_display_names as select * from file('analyze/query-display-names.tsv', TSV, 'test text, query_index int, query_display_name text') @@ -644,6 +641,7 @@ create view partial_query_times as select * from -- Report for partial queries that we could only run on the new server (e.g. -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') + settings output_format_decimal_trailing_zeros = 1 as select toDecimal64(time_median, 3) time, toDecimal64(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name @@ -716,8 +714,9 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') order by test, query_index, metric_name ; -create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as - with +create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion -- of NaN to decimal. @@ -733,8 +732,9 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') as changed_fail, test, query_index, query_display_name from queries where changed_show order by abs(diff) desc; -create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') as - select +create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3), toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name from queries where unstable_show order by stat_threshold desc; @@ -764,8 +764,9 @@ create view total_speedup as from test_speedup ; -create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') as - with +create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with (times_speedup >= 1 ? '-' || toString(toDecimal64(times_speedup, 3)) || 'x' : '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x') @@ -791,8 +792,9 @@ create view total_client_time_per_query as select * from file('analyze/client-times.tsv', TSV, 'test text, query_index int, client float, server float'); -create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') as - select client, server, toDecimal64(client/server, 3) p, +create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select client, server, toDecimal64(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) where p > toDecimal64(1.02, 3) order by p desc; @@ -877,8 +879,9 @@ create view test_times_view_total as from test_times_view ; -create table test_times_report engine File(TSV, 'report/test-times.tsv') as - select +create table test_times_report engine File(TSV, 'report/test-times.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select test, toDecimal64(real, 3), toDecimal64(total_client_time, 3), @@ -896,8 +899,9 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as ; -- report for all queries page, only main metric -create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as - with +create table all_tests_report engine File(TSV, 'report/all-queries.tsv') + settings output_format_decimal_trailing_zeros = 1 + as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion -- of NaN to decimal. @@ -978,9 +982,6 @@ for version in {right,left} do rm -rf data clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view query_profiles as with 0 as left, 1 as right select * from file('analyze/query-profiles.tsv', TSV, @@ -1063,9 +1064,10 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, ; create table metric_devation engine File(TSVWithNamesAndTypes, - 'report/metric-deviation.$version.tsv') as + 'report/metric-deviation.$version.tsv') + settings output_format_decimal_trailing_zeros = 1 -- first goes the key used to split the file with grep - select test, query_index, query_display_name, + as select test, query_index, query_display_name, toDecimal64(d, 3) d, q, metric from ( select @@ -1176,9 +1178,6 @@ rm -rf metrics ||: mkdir metrics clickhouse-local --query " --- We use decimals specifically to get fixed-point, fixed-width formatting. -set output_format_decimal_trailing_zeros = 1; - create view right_async_metric_log as select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes, '$(cat right-async-metric-log.tsv.columns)') @@ -1196,8 +1195,9 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as ; -- Show metrics that have changed -create table changes engine File(TSV, 'metrics/changes.tsv') as - select metric, left, right, +create table changes engine File(TSV, 'metrics/changes.tsv') + settings output_format_decimal_trailing_zeros = 1 + as select metric, left, right, toDecimal64(diff, 3), toDecimal64(times_diff, 3) from ( select metric, median(left) as left, median(right) as right, From c263523f50ede57e3d8fe36a486723eadf6c91fa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 19 Aug 2021 22:45:40 +0300 Subject: [PATCH 37/40] Remove unused declaration. --- src/Interpreters/Session.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 0e816324dad..d104e250099 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -82,8 +82,6 @@ private: String session_id; std::shared_ptr named_session; bool named_session_created = false; - - static std::optional named_sessions; }; } From 64bfe21a1be96afe31df48c53df4ca408113d776 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 20 Aug 2021 00:25:14 +0300 Subject: [PATCH 38/40] Fix test 00443_preferred_block_size_bytes.sh (#27846) * Update 00443_preferred_block_size_bytes.sh * Update clickhouse-test * Update clickhouse-test * Update database_replicated.xml --- tests/clickhouse-test | 4 +++- tests/config/config.d/database_replicated.xml | 6 +++--- .../queries/0_stateless/00443_preferred_block_size_bytes.sh | 4 ++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c627810a550..0d833e5fbe6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -44,15 +44,17 @@ DISTRIBUTED_DDL_TIMEOUT_MSG = "is executing longer than distributed_ddl_task_tim MESSAGES_TO_RETRY = [ "DB::Exception: ZooKeeper session has been expired", - "DB::Exception: Connection loss", "Coordination::Exception: Session expired", "Coordination::Exception: Connection loss", "Coordination::Exception: Operation timeout", + "DB::Exception: Session expired", + "DB::Exception: Connection loss", "DB::Exception: Operation timeout", "Operation timed out", "ConnectionPoolWithFailover: Connection failed at try", "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", + "DB::Exception: Cannot enqueue query", "Shutdown is called for table", # It happens in SYSTEM SYNC REPLICA query if session with ZooKeeper is being reinitialized. DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME ] diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 9a3b4d68ea6..e51d90dd4d4 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -22,9 +22,9 @@ 10000 30000 1000 - 2000 - 4000 - trace + 4000 + 5000 + information false 1000000000000000 diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index 724630057d9..399a4677a44 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -42,10 +42,10 @@ popd > /dev/null #SCRIPTDIR=`dirname "$SCRIPTPATH"` SCRIPTDIR=$SCRIPTPATH -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout 2>&1 +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=10 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED -cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout 2>&1 +cat "$SCRIPTDIR"/00282_merging.sql | $CLICKHOUSE_CLIENT --preferred_block_size_bytes=20 -n > "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout cmp "$SCRIPTDIR"/00282_merging.reference "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout && echo PASSED || echo FAILED rm "${CLICKHOUSE_TMP}"/preferred_block_size_bytes.stdout From 0aa800122d6b058513ad9d19f964f0caca02bc28 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 20 Aug 2021 12:28:23 +0300 Subject: [PATCH 39/40] Update menus.md --- docs/en/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 87e4c75d0d4..8f330f39226 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -105,7 +105,7 @@ We use `Decimal` data type to store prices. Everything else is quite straightfor ## Import Data -Upload data into ClickHouse in parallel: +Upload data into ClickHouse: ``` clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv From c3c31e2895ee443b3503e573aaba39a3ae29ca65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 20 Aug 2021 12:56:39 +0300 Subject: [PATCH 40/40] Move function to appropriate place to make code more readable --- src/Server/TCPHandler.cpp | 48 +++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b2db65e22bc..beca726e95f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -49,27 +49,6 @@ namespace DB { -namespace -{ -std::string formatHTTPErrorResponse(const Poco::Util::AbstractConfiguration& config) -{ - std::string result = fmt::format( - "HTTP/1.0 400 Bad Request\r\n\r\n" - "Port {} is for clickhouse-client program\r\n", - config.getString("tcp_port")); - - if (config.has("http_port")) - { - result += fmt::format( - "You must use port {} for HTTP.\r\n", - config.getString("http_port")); - } - - return result; -} -} - - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -925,6 +904,29 @@ bool TCPHandler::receiveProxyHeader() } +namespace +{ + +std::string formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(const Poco::Util::AbstractConfiguration& config) +{ + std::string result = fmt::format( + "HTTP/1.0 400 Bad Request\r\n\r\n" + "Port {} is for clickhouse-client program\r\n", + config.getString("tcp_port")); + + if (config.has("http_port")) + { + result += fmt::format( + "You must use port {} for HTTP.\r\n", + config.getString("http_port")); + } + + return result; +} + +} + + void TCPHandler::receiveHello() { /// Receive `hello` packet. @@ -940,9 +942,7 @@ void TCPHandler::receiveHello() */ if (packet_type == 'G' || packet_type == 'P') { - writeString(formatHTTPErrorResponse(server.config()), - *out); - + writeString(formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(server.config()), *out); throw Exception("Client has connected to wrong port", ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT); } else