From 2b90141c42574cedf48aa285b24e7d53b142ede8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 Feb 2023 13:17:31 +0000 Subject: [PATCH 001/327] Integrate DiskObjectStorage with IO Scheduler --- src/Common/CurrentThread.h | 8 ++++ src/Disks/DiskLocal.cpp | 4 +- .../registerDiskAzureBlobStorage.cpp | 4 +- .../ObjectStorages/DiskObjectStorage.cpp | 47 +++++++++++++++++-- src/Disks/ObjectStorages/DiskObjectStorage.h | 11 ++++- .../ObjectStorages/HDFS/registerDiskHDFS.cpp | 5 +- .../ObjectStorages/S3/registerDiskS3.cpp | 4 +- .../Web/registerDiskWebServer.cpp | 4 +- src/IO/IResourceManager.h | 2 +- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 4 ++ 11 files changed, 85 insertions(+), 12 deletions(-) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index f36b92e319d..1571f52558c 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -93,6 +93,14 @@ public: return current_thread->getQueryId(); } + /// Returns attached query context + static ContextPtr getQueryContext() + { + if (isInitialized()) + return get().getQueryContext(); + return {}; + } + /// Non-master threads call this method in destructor automatically static void detachQuery(); static void detachQueryIfNotDetached(); diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index a3b7e413014..088e44357b3 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -613,7 +613,9 @@ DiskObjectStoragePtr DiskLocal::createDiskObjectStorage() metadata_storage, object_storage, false, - /* threadpool_size */16 + /* threadpool_size */ 16, + /* read_resource_name */ "", + /* write_resource_name */ "" ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 562b2b2fec0..e0c0a525d49 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -43,7 +43,9 @@ void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access std::move(metadata_storage), std::move(azure_object_storage), send_metadata, - copy_thread_pool_size + copy_thread_pool_size, + config.getString(config_prefix + ".read_resource", ""), + config.getString(config_prefix + ".write_resource", "") ); bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index d55b1c91c07..825ed03b1ba 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -109,7 +110,9 @@ DiskObjectStorage::DiskObjectStorage( MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, bool send_metadata_, - uint64_t thread_pool_size_) + uint64_t thread_pool_size_, + const String & read_resource_name_, + const String & write_resource_name_) : IDisk(name_, getAsyncExecutor(log_name, thread_pool_size_)) , object_storage_root_path(object_storage_root_path_) , log (&Poco::Logger::get("DiskObjectStorage(" + log_name + ")")) @@ -117,6 +120,8 @@ DiskObjectStorage::DiskObjectStorage( , object_storage(std::move(object_storage_)) , send_metadata(send_metadata_) , threadpool_size(thread_pool_size_) + , read_resource_name(read_resource_name_) + , write_resource_name(write_resource_name_) , metadata_helper(std::make_unique(this, ReadSettings{})) {} @@ -517,7 +522,9 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() metadata_storage, object_storage, send_metadata, - threadpool_size); + threadpool_size, + getReadResourceName(), + getWriteResourceName()); } void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name) @@ -546,6 +553,32 @@ NameSet DiskObjectStorage::getCacheLayersNames() const return cache_layers; } +template +static inline Settings updateResourceLink(const Settings & settings, const String & resource_name) +{ + if (resource_name.empty()) + return settings; + if (auto query_context = CurrentThread::getQueryContext()) + { + Settings result(settings); + result.resource_link = query_context->getClassifier()->get(resource_name); + return result; + } + return settings; +} + +String DiskObjectStorage::getReadResourceName() const +{ + std::unique_lock lock(resource_mutex); + return read_resource_name; +} + +String DiskObjectStorage::getWriteResourceName() const +{ + std::unique_lock lock(resource_mutex); + return write_resource_name; +} + std::unique_ptr DiskObjectStorage::readFile( const String & path, const ReadSettings & settings, @@ -554,7 +587,7 @@ std::unique_ptr DiskObjectStorage::readFile( { return object_storage->readObjects( metadata_storage->getStorageObjects(path), - object_storage->getAdjustedSettingsFromMetadataFile(settings, path), + object_storage->getAdjustedSettingsFromMetadataFile(updateResourceLink(settings, getReadResourceName()), path), read_hint, file_size); } @@ -572,7 +605,7 @@ std::unique_ptr DiskObjectStorage::writeFile( path, buf_size, mode, - object_storage->getAdjustedSettingsFromMetadataFile(settings, path)); + object_storage->getAdjustedSettingsFromMetadataFile(updateResourceLink(settings, getWriteResourceName()), path)); return result; } @@ -585,6 +618,12 @@ void DiskObjectStorage::applyNewSettings( if (AsyncThreadPoolExecutor * exec = dynamic_cast(&getExecutor())) exec->setMaxThreads(config.getInt(config_prefix + ".thread_pool_size", 16)); + + std::unique_lock lock(resource_mutex); + if (String new_read_resource_name = config.getString(config_prefix + ".read_resource", ""); new_read_resource_name != read_resource_name) + read_resource_name = new_read_resource_name; + if (String new_write_resource_name = config.getString(config_prefix + ".write_resource", ""); new_write_resource_name != write_resource_name) + write_resource_name = new_write_resource_name; } void DiskObjectStorage::restoreMetadataIfNeeded( diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index a24acc270c0..f5b9fdf54d3 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -34,7 +34,9 @@ public: MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, bool send_metadata_, - uint64_t thread_pool_size_); + uint64_t thread_pool_size_, + const String & read_resource_name_, + const String & write_resource_name_); /// Create fake transaction DiskTransactionPtr createTransaction() override; @@ -211,6 +213,9 @@ private: /// execution. DiskTransactionPtr createObjectStorageTransaction(); + String getReadResourceName() const; + String getWriteResourceName() const; + const String object_storage_root_path; Poco::Logger * log; @@ -226,6 +231,10 @@ private: const bool send_metadata; size_t threadpool_size; + mutable std::mutex resource_mutex; + String read_resource_name; + String write_resource_name; + std::unique_ptr metadata_helper; }; diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index 693b966caf2..b7f703b278d 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -54,7 +54,10 @@ void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check) std::move(metadata_storage), std::move(hdfs_storage), /* send_metadata = */ false, - copy_thread_pool_size); + copy_thread_pool_size, + config.getString(config_prefix + ".read_resource", ""), + config.getString(config_prefix + ".write_resource", "") + ); disk->startup(context, skip_access_check); return disk; diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 1c192a0d89c..bce1e640a7c 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -163,7 +163,9 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) std::move(metadata_storage), std::move(s3_storage), send_metadata, - copy_thread_pool_size); + copy_thread_pool_size, + config.getString(config_prefix + ".read_resource", ""), + config.getString(config_prefix + ".write_resource", "")); s3disk->startup(context, skip_access_check); diff --git a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp index 8a54de81815..19005727fa7 100644 --- a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp @@ -53,7 +53,9 @@ void registerDiskWebServer(DiskFactory & factory, bool global_skip_access_check) metadata_storage, object_storage, /* send_metadata */false, - /* threadpool_size */16); + /* threadpool_size */16, + config.getString(config_prefix + ".read_resource", ""), + config.getString(config_prefix + ".write_resource", "")); disk->startup(context, skip_access_check); return disk; }; diff --git a/src/IO/IResourceManager.h b/src/IO/IResourceManager.h index f084a903cb1..ff372698ed5 100644 --- a/src/IO/IResourceManager.h +++ b/src/IO/IResourceManager.h @@ -23,7 +23,7 @@ class IClassifier : private boost::noncopyable public: virtual ~IClassifier() {} - /// Returns ResouceLink that should be used to access resource. + /// Returns ResourceLink that should be used to access resource. /// Returned link is valid until classifier destruction. virtual ResourceLink get(const String & resource_name) = 0; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 33505e41789..72a472fcb4a 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1165,7 +1165,9 @@ ResourceManagerPtr Context::getResourceManager() const ClassifierPtr Context::getClassifier() const { auto lock = getLock(); - return getResourceManager()->acquire(getSettingsRef().workload); + if (!classifier) + classifier = getResourceManager()->acquire(getSettingsRef().workload); + return classifier; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 19bb6868331..9ea8d253439 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -402,6 +402,10 @@ private: /// Temporary data for query execution accounting. TemporaryDataOnDiskScopePtr temp_data_on_disk; + /// Resource classifier for a query, holds smart pointers required for ResourceLink + /// NOTE: all resource links became invalid after `classifier` destruction + mutable ClassifierPtr classifier; + public: /// Some counters for current query execution. /// Most of them are workarounds and should be removed in the future. From 21b6ccc677ba040acc62fc0c63cadbe53634ce3f Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 2 Mar 2023 15:35:28 +0000 Subject: [PATCH 002/327] fix comment --- src/IO/ResourceRequest.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/ResourceRequest.h b/src/IO/ResourceRequest.h index 989349148cf..3d2230746f9 100644 --- a/src/IO/ResourceRequest.h +++ b/src/IO/ResourceRequest.h @@ -45,8 +45,7 @@ class ResourceRequest { public: /// Cost of request execution; should be filled before request enqueueing. - /// NOTE: If cost is not known in advance, credit model can be used: - /// NOTE: for the first request use 1 and + /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; /// Request outcome From 7e3b0c5f86046a03f30b484cc99481269d4dfebd Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Thu, 8 Jun 2023 08:43:22 -0700 Subject: [PATCH 003/327] Add new exceptions to 4xx error --- src/Server/HTTPHandler.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 5a2bf0bad6c..09063d93a59 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -70,12 +70,15 @@ namespace ErrorCodes extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_COMPILE_REGEXP; - + extern const int DUPLICATE_COLUMN; + extern const int ILLEGAL_COLUMN; + extern const int THERE_IS_NO_COLUMN; extern const int UNKNOWN_ELEMENT_IN_AST; extern const int UNKNOWN_TYPE_OF_AST_NODE; extern const int TOO_DEEP_AST; extern const int TOO_BIG_AST; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; extern const int SYNTAX_ERROR; @@ -186,7 +189,9 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_FORBIDDEN; } - else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + else if (exception_code == ErrorCodes::BAD_ARGUMENTS || + exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP || + exception_code == ErrorCodes::CANNOT_PARSE_TEXT || exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || exception_code == ErrorCodes::CANNOT_PARSE_DATE || @@ -196,14 +201,19 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::CANNOT_PARSE_IPV4 || exception_code == ErrorCodes::CANNOT_PARSE_IPV6 || exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::CANNOT_PARSE_UUID || + exception_code == ErrorCodes::DUPLICATE_COLUMN || + exception_code == ErrorCodes::ILLEGAL_COLUMN || exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::THERE_IS_NO_COLUMN || exception_code == ErrorCodes::TOO_DEEP_AST || exception_code == ErrorCodes::TOO_BIG_AST || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || exception_code == ErrorCodes::SYNTAX_ERROR || exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH) + exception_code == ErrorCodes::TYPE_MISMATCH || + exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) { return HTTPResponse::HTTP_BAD_REQUEST; } From ae28c549a76f500a72426211a75ef2cfbb7eb7cd Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 9 Jun 2023 09:10:06 -0700 Subject: [PATCH 004/327] Fix style --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 09063d93a59..c99134c86d2 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -212,7 +212,7 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || exception_code == ErrorCodes::SYNTAX_ERROR || exception_code == ErrorCodes::INCORRECT_DATA || - exception_code == ErrorCodes::TYPE_MISMATCH || + exception_code == ErrorCodes::TYPE_MISMATCH || exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE) { return HTTPResponse::HTTP_BAD_REQUEST; From 003565d08e1a4bf0d29711edbf8fe32a2d7ca36a Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Mon, 12 Jun 2023 07:28:21 -0700 Subject: [PATCH 005/327] Add undefs --- src/Server/HTTPHandler.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c99134c86d2..9906438f995 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -57,6 +57,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; @@ -67,6 +68,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; extern const int CANNOT_PARSE_IPV4; extern const int CANNOT_PARSE_IPV6; + extern const int CANNOT_PARSE_UUID; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_COMPILE_REGEXP; From 48c3c5a347236f741a449cf1414aac3850bc877e Mon Sep 17 00:00:00 2001 From: CuiShuoGuo <129303239+bakam412@users.noreply.github.com> Date: Thu, 20 Jul 2023 11:25:18 +0800 Subject: [PATCH 006/327] Update drop.md --- docs/en/sql-reference/statements/drop.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index b6208c2fd52..a8658dd49fc 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -32,6 +32,14 @@ Syntax: DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] [SYNC] ``` +You can also drop multiple tables at the same time. + +Syntax: + +```sql +DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name_1[,[db.]name_2][,[db.]name_3]... [ON CLUSTER cluster] [SYNC] +``` + ## DROP DICTIONARY Deletes the dictionary. From e851be78bce89b5cf652c710714277824c4f85e8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 31 Jul 2023 23:56:07 +0200 Subject: [PATCH 007/327] fix fs_error on attempt to read unavailable dirs --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cbd32460f7e..7e214b76e3d 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -122,7 +122,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, return; const fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls); it != end; ++it) + for (fs::directory_iterator it(path_for_ls, std::filesystem::directory_options::skip_permission_denied); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); From c8d995e42e4e2595981fed21191a20da00f80192 Mon Sep 17 00:00:00 2001 From: UnamedRus Date: Tue, 1 Aug 2023 01:21:46 +0300 Subject: [PATCH 008/327] add optimize to bitmap write method --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 7ea1ebe7749..f92f8c1b5e5 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -151,6 +151,7 @@ public: } else if (BitmapKind::Bitmap == kind) { + roaring_bitmap->runOptimize(); auto size = roaring_bitmap->getSizeInBytes(); writeVarUInt(size, out); std::unique_ptr buf(new char[size]); From f9d7fe9ab9a3cd36bcc40ec9f529c0bd4980cd63 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Aug 2023 14:32:10 +0000 Subject: [PATCH 009/327] Fix misleading error message in OUTFILE with CapnProto/Protobuf --- src/Client/ClientBase.cpp | 4 +++- .../02842_capn_proto_outfile_without_schema.reference | 1 + .../02842_capn_proto_outfile_without_schema.sh | 10 ++++++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.reference create mode 100755 tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 06dabf96c28..1434498e549 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1037,7 +1037,9 @@ void ClientBase::receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, b } catch (const LocalFormatError &) { - local_format_error = std::current_exception(); + /// Remember the first exception. + if (!local_format_error) + local_format_error = std::current_exception(); connection->sendCancel(); } } diff --git a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.reference b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh new file mode 100755 index 00000000000..fde08484026 --- /dev/null +++ b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +$CLICKHOUSE_LOCAL -q "select * from numbers(10) into outfile '$CLICKHOUSE_TEST_UNIQUE_NAME.capnp'" 2>&1 | grep "The format CapnProto requires a schema" -c +rm $CLICKHOUSE_TEST_UNIQUE_NAME.capnp + From c0f9dbbacb34fabbb2ce8431476f53d3086a1eb9 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 2 Aug 2023 15:51:27 +0200 Subject: [PATCH 010/327] shorten --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7e214b76e3d..c6c9fefac97 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -122,7 +122,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, return; const fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls, std::filesystem::directory_options::skip_permission_denied); it != end; ++it) + for (fs::directory_iterator it(path_for_ls, fs::directory_options::skip_permission_denied); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); From e797d877480386cb38dcdd5034129b9f4a469ef1 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 3 Aug 2023 22:43:08 +0800 Subject: [PATCH 011/327] More precise Integer type inference --- src/DataTypes/DataTypesNumber.h | 16 ++++ src/DataTypes/FieldToDataType.cpp | 22 ++++- src/DataTypes/IDataType.h | 2 + src/DataTypes/getLeastSupertype.cpp | 132 ++++++++++++++++++++++++++++ src/DataTypes/getLeastSupertype.h | 2 + 5 files changed, 171 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypesNumber.h b/src/DataTypes/DataTypesNumber.h index 5843086248c..8511955e193 100644 --- a/src/DataTypes/DataTypesNumber.h +++ b/src/DataTypes/DataTypesNumber.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -9,10 +10,20 @@ namespace DB { +using DataTypes = std::vector; + template class DataTypeNumber final : public DataTypeNumberBase { public: + DataTypeNumber() = default; + + explicit DataTypeNumber(DataTypes data_types) + : DataTypeNumberBase() + , possible_data_types(std::move(data_types)) + { + } + bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool canBeUsedAsVersion() const override { return true; } @@ -32,6 +43,11 @@ public: { return std::make_shared>(); } + + DataTypes getPossiblePtr() const override { return possible_data_types; } + +private: + DataTypes possible_data_types; }; using DataTypeUInt8 = DataTypeNumber; diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 210dab9921e..0a48c43ceb2 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -33,18 +33,34 @@ DataTypePtr FieldToDataType::operator() (const Null &) const template DataTypePtr FieldToDataType::operator() (const UInt64 & x) const { + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); return std::make_shared(); } template DataTypePtr FieldToDataType::operator() (const Int64 & x) const { - if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); - if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); - if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); + if (x >= 0) + { + if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); + if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); + if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); + } + else + { + if (x >= std::numeric_limits::min()) return std::make_shared(); + if (x >= std::numeric_limits::min()) return std::make_shared(); + if (x >= std::numeric_limits::min()) return std::make_shared(); + } return std::make_shared(); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4adafe5d212..330836f9d92 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -73,6 +73,8 @@ public: DataTypePtr getPtr() const { return shared_from_this(); } + virtual DataTypes getPossiblePtr() const { return { shared_from_this() }; } + /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; /// Name of corresponding data type in MySQL (exampe: Bigint, Blob, etc) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 9d42d82ce91..3a7a1560af1 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -592,6 +592,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For numeric types, the most complicated part. { + optimizeTypeIds(types, type_ids); auto numeric_type = getNumericType(type_ids); if (numeric_type) return numeric_type; @@ -601,6 +602,137 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); } +void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) +{ + // Determine whether the type_id is UInt + auto is_unsigned = [](const TypeIndex & type_id) + { + switch (type_id) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::UInt64: + return true; + default: + return false; + } + }; + + auto maximize = [](size_t & what, size_t value, bool & only_unsigned, bool & only_signed, bool & both) + { + if (value > what) + { + what = value; + only_unsigned = false; + only_signed = false; + both = false; + return true; + }else if (value == what) + { + return true; + } + + return false; + }; + + size_t max_bits_of_integer = 0; + bool only_unsigned = false; + bool only_signed = false; + bool both = false; + + // Determine the distribution of maximum signed and unsigned, Example: + // Int64, Int64 = only_signed. + // UInt64, UInt64 = only_unsigned. + // UInt64(possible: Int64), Int64(possible: UInt64) = both. + // UInt64(possible: Int64), Int64 = both, only_signed. + for (const auto & type : types) + { + TypeIndex type_id = type->getTypeId(); + bool is_max_bits = false; + if (type_id == TypeIndex::UInt8 || type_id == TypeIndex::Int8) + is_max_bits = maximize(max_bits_of_integer, 8, only_unsigned, only_signed, both); + else if (type_id == TypeIndex::UInt16 || type_id == TypeIndex::Int16) + is_max_bits = maximize(max_bits_of_integer, 16, only_unsigned, only_signed, both); + else if (type_id == TypeIndex::UInt32 || type_id == TypeIndex::Int32) + is_max_bits = maximize(max_bits_of_integer, 32, only_unsigned, only_signed, both); + else if (type_id == TypeIndex::UInt64 || type_id == TypeIndex::Int64) + is_max_bits = maximize(max_bits_of_integer, 64, only_unsigned, only_signed, both); + + if (is_max_bits) + { + bool type_is_unsigned = is_unsigned(type_id); + bool type_is_both = false; + for (const auto & possible_type : type->getPossiblePtr()) + { + if (type_is_unsigned != is_unsigned(possible_type->getTypeId())) + { + type_is_both = true; + break; + } + } + + if (type_is_both) + both = true; + else if (type_is_unsigned) + only_unsigned = true; + else + only_signed = true; + } + } + + auto optimize_type_id = [&is_unsigned](const DataTypePtr & type, bool try_change_unsigned) + { + switch (type_id) + { + case TypeIndex::UInt8: + case TypeIndex::UInt16: + case TypeIndex::UInt32: + case TypeIndex::UInt64: + if (try_change_unsigned) + return type_id; + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::Int64: + if (!try_change_unsigned) + return type_id; + default: + return type_id; + } + + for (const auto & other_type : type->getPossiblePtr()) + { + TypeIndex other_type_id = other_type->getTypeId(); + if ((try_change_unsigned && is_unsigned(other_type_id)) + || (!try_change_unsigned && !is_unsigned(other_type_id))) + { + return other_type_id; + } + } + + return type_id; + }; + + // optimize type_ids, Example: + // if only_signed. UInt64(possible: Int64), Int64 = Int64, Int64 + // if only_unsigned. Int64(possible: UInt64), UInt64 = UInt64, UInt64 + if (!(only_unsigned && only_signed) && (both || only_unsigned || only_signed)) { + type_ids.clear(); + for (const auto & type : types) + { + if (only_unsigned) + { + type_ids.insert(optimize_type_id(type, true)); + } + else if (both || only_signed) + { + type_ids.insert(optimize_type_id(type, false)); + } + } + } +} + DataTypePtr getLeastSupertypeOrString(const DataTypes & types) { return getLeastSupertype(types); diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 2ef4a0e6850..0a3fa3c2536 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -29,6 +29,8 @@ DataTypePtr tryGetLeastSupertype(const DataTypes & types); using TypeIndexSet = std::unordered_set; +void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids); + template DataTypePtr getLeastSupertype(const TypeIndexSet & types); From 446465307d78ddaa99b8c625f0d6a8584ebccd67 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Fri, 4 Aug 2023 09:26:42 +0800 Subject: [PATCH 012/327] add test. fix style. --- src/DataTypes/getLeastSupertype.cpp | 4 +++- .../0_stateless/02832_integer_type_inference.reference | 1 + tests/queries/0_stateless/02832_integer_type_inference.sql | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02832_integer_type_inference.reference create mode 100644 tests/queries/0_stateless/02832_integer_type_inference.sql diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 3a7a1560af1..07a3de4760b 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -683,6 +683,7 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) auto optimize_type_id = [&is_unsigned](const DataTypePtr & type, bool try_change_unsigned) { + TypeIndex type_id = type->getTypeId(); switch (type_id) { case TypeIndex::UInt8: @@ -717,7 +718,8 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) // optimize type_ids, Example: // if only_signed. UInt64(possible: Int64), Int64 = Int64, Int64 // if only_unsigned. Int64(possible: UInt64), UInt64 = UInt64, UInt64 - if (!(only_unsigned && only_signed) && (both || only_unsigned || only_signed)) { + if (!(only_unsigned && only_signed) && (both || only_unsigned || only_signed)) + { type_ids.clear(); for (const auto & type : types) { diff --git a/tests/queries/0_stateless/02832_integer_type_inference.reference b/tests/queries/0_stateless/02832_integer_type_inference.reference new file mode 100644 index 00000000000..2d072437c90 --- /dev/null +++ b/tests/queries/0_stateless/02832_integer_type_inference.reference @@ -0,0 +1 @@ +[-4741124612489978151,-3236599669630092879,5607475129431807682] diff --git a/tests/queries/0_stateless/02832_integer_type_inference.sql b/tests/queries/0_stateless/02832_integer_type_inference.sql new file mode 100644 index 00000000000..9734bc2966f --- /dev/null +++ b/tests/queries/0_stateless/02832_integer_type_inference.sql @@ -0,0 +1 @@ +select [-4741124612489978151, -3236599669630092879, 5607475129431807682]; From 57d69a10e360fd4cccb5acd0759aa9f44a21f7f6 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Fri, 4 Aug 2023 09:55:22 +0800 Subject: [PATCH 013/327] add switch break --- src/DataTypes/getLeastSupertype.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 07a3de4760b..5a24a59a60e 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -692,12 +692,14 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) case TypeIndex::UInt64: if (try_change_unsigned) return type_id; + break ; case TypeIndex::Int8: case TypeIndex::Int16: case TypeIndex::Int32: case TypeIndex::Int64: if (!try_change_unsigned) return type_id; + break ; default: return type_id; } From 5cb856d167a37c46ac31d441e5a688b45ddb0434 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 11:53:22 +0000 Subject: [PATCH 014/327] Try batching multiple flush requests in Keeper --- src/Coordination/Changelog.cpp | 78 +++++++++++++++++++------ src/Coordination/KeeperStateManager.cpp | 3 + 2 files changed, 63 insertions(+), 18 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 94062140bac..7f972f67846 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1014,8 +1014,60 @@ void Changelog::writeThread() { WriteOperation write_operation; bool batch_append_ok = true; - while (write_operations.pop(write_operation)) + size_t pending_appends = 0; + bool try_batch_flush = false; + + /// turn into setting + static constexpr size_t max_flush_batch_size = 1000; + + const auto flush_logs = [&](const auto & flush) { + LOG_INFO(log, "Flushing {} logs", pending_appends); + + { + std::lock_guard writer_lock(writer_mutex); + current_writer->flush(); + } + + { + std::lock_guard lock{durable_idx_mutex}; + last_durable_idx = flush.index; + } + + pending_appends = 0; + }; + + const auto notify_append_completion = [&] + { + durable_idx_cv.notify_all(); + + // we need to call completion callback in another thread because it takes a global lock for the NuRaft server + // NuRaft will in some places wait for flush to be done while having the same global lock leading to deadlock + // -> future write operations are blocked by flush that cannot be completed because it cannot take NuRaft lock + // -> NuRaft won't leave lock until its flush is done + if (!append_completion_queue.push(batch_append_ok)) + LOG_WARNING(log, "Changelog is shut down"); + }; + + while (true) + { + if (try_batch_flush) + { + try_batch_flush = false; + if (!write_operations.tryPop(write_operation)) + { + chassert(batch_append_ok); + const auto & flush = std::get(write_operation); + flush_logs(flush); + notify_append_completion(); + continue; + } + } + else if (!write_operations.pop(write_operation)) + { + break; + } + assert(initialized); if (auto * append_log = std::get_if(&write_operation)) @@ -1027,6 +1079,7 @@ void Changelog::writeThread() assert(current_writer); batch_append_ok = current_writer->appendRecord(buildRecord(append_log->index, append_log->log_entry)); + ++pending_appends; } else { @@ -1034,30 +1087,19 @@ void Changelog::writeThread() if (batch_append_ok) { + /// we can try batching more logs for flush + if (pending_appends < max_flush_batch_size) { - std::lock_guard writer_lock(writer_mutex); - current_writer->flush(); - } - - { - std::lock_guard lock{durable_idx_mutex}; - last_durable_idx = flush.index; + try_batch_flush = true; + continue; } + flush_logs(flush); } else { *flush.failed = true; } - - durable_idx_cv.notify_all(); - - // we need to call completion callback in another thread because it takes a global lock for the NuRaft server - // NuRaft will in some places wait for flush to be done while having the same global lock leading to deadlock - // -> future write operations are blocked by flush that cannot be completed because it cannot take NuRaft lock - // -> NuRaft won't leave lock until its flush is done - if (!append_completion_queue.push(batch_append_ok)) - LOG_WARNING(log, "Changelog is shut down"); - + notify_append_completion(); batch_append_ok = true; } } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cf1bad8c5fa..cc13c755629 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -184,6 +184,9 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur total_servers++; } + /// this will only apply to fresh clusters + result.cluster_config->set_async_replication(true); + if (!result.config && !allow_without_us) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); From 41e8345169616ea052e09dcd5f54aea8c5253c62 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Sat, 5 Aug 2023 17:35:25 +0800 Subject: [PATCH 015/327] Add the optimize_type_ids parameter to control whether optimization is enabled. --- src/DataTypes/FieldToDataType.cpp | 2 +- src/DataTypes/getLeastSupertype.cpp | 7 ++++--- src/DataTypes/getLeastSupertype.h | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 0a48c43ceb2..9b8e476067e 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -155,7 +155,7 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const for (const Field & elem : x) element_types.emplace_back(applyVisitor(*this, elem)); - return std::make_shared(getLeastSupertype(element_types)); + return std::make_shared(getLeastSupertype(element_types, true)); } template diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 5a24a59a60e..f3b396ba895 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -201,7 +201,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) } template -DataTypePtr getLeastSupertype(const DataTypes & types) +DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids) { /// Trivial cases @@ -592,7 +592,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For numeric types, the most complicated part. { - optimizeTypeIds(types, type_ids); + if (optimize_type_ids) + optimizeTypeIds(types, type_ids); auto numeric_type = getNumericType(type_ids); if (numeric_type) return numeric_type; @@ -798,7 +799,7 @@ DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types) return getLeastSupertype(types); } -template DataTypePtr getLeastSupertype(const DataTypes & types); +template DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids); template DataTypePtr getLeastSupertype(const TypeIndexSet & types); } diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 0a3fa3c2536..ba43b3966f9 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -18,7 +18,7 @@ enum class LeastSupertypeOnError * Examples: there is no least common supertype for Array(UInt8), Int8. */ template -DataTypePtr getLeastSupertype(const DataTypes & types); +DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids = false); /// Same as above but return String type instead of throwing exception. /// All types can be casted to String, because they can be serialized to String. From ad49798f4550338c314d0620e283aee800fa7664 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Sat, 5 Aug 2023 20:17:28 +0800 Subject: [PATCH 016/327] fix optimizeTypeIds, Some scenarios do not require optimization: if UInt64(possible: Int64), UInt64(possible: Int64) = UInt64, UInt64 if Int64(possible: UInt32), Int64(possible: UInt32) = Int64, Int64 --- src/DataTypes/getLeastSupertype.cpp | 32 ++++++++++++++++++----------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index f3b396ba895..c534f3c6edc 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -641,6 +641,8 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) bool only_unsigned = false; bool only_signed = false; bool both = false; + bool has_unsigned = false; + bool has_signed = false; // Determine the distribution of maximum signed and unsigned, Example: // Int64, Int64 = only_signed. @@ -673,6 +675,11 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) } } + if (type_is_unsigned) + has_unsigned = true; + else + has_signed = true; + if (type_is_both) both = true; else if (type_is_unsigned) @@ -718,22 +725,23 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) return type_id; }; - // optimize type_ids, Example: - // if only_signed. UInt64(possible: Int64), Int64 = Int64, Int64 - // if only_unsigned. Int64(possible: UInt64), UInt64 = UInt64, UInt64 - if (!(only_unsigned && only_signed) && (both || only_unsigned || only_signed)) + // optimize type_ids + if (both) { - type_ids.clear(); - for (const auto & type : types) + // Example: UInt64(possible: Int64), Int64 = Int64, Int64 + if (only_unsigned && !only_signed) { - if (only_unsigned) - { + type_ids.clear(); + for (const auto & type : types) type_ids.insert(optimize_type_id(type, true)); - } - else if (both || only_signed) - { + } + // Example: Int64(possible: UInt32), UInt64 = UInt32, UInt64 + // Int64(possible: UInt32), UInt64(possible: Int64) = Int64, Int64 + else if ((only_signed && !only_unsigned) || (has_unsigned && has_signed && !only_signed && !only_unsigned)) + { + type_ids.clear(); + for (const auto & type : types) type_ids.insert(optimize_type_id(type, false)); - } } } } From 75a8512017027d92a2ef7311e89d84ff627629c5 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Sun, 6 Aug 2023 09:52:53 +0800 Subject: [PATCH 017/327] modify comments --- src/DataTypes/getLeastSupertype.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index c534f3c6edc..c04130a58e4 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -728,14 +728,14 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) // optimize type_ids if (both) { - // Example: UInt64(possible: Int64), Int64 = Int64, Int64 + // Example: Int64(possible: UInt32), UInt64 = UInt32, UInt64 if (only_unsigned && !only_signed) { type_ids.clear(); for (const auto & type : types) type_ids.insert(optimize_type_id(type, true)); } - // Example: Int64(possible: UInt32), UInt64 = UInt32, UInt64 + // Example: UInt64(possible: Int64), Int64 = Int64, Int64 // Int64(possible: UInt32), UInt64(possible: Int64) = Int64, Int64 else if ((only_signed && !only_unsigned) || (has_unsigned && has_signed && !only_signed && !only_unsigned)) { From 1540c02ccabaf4921ca8327d5e3779bce3da03d4 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Mon, 7 Aug 2023 17:56:51 +0800 Subject: [PATCH 018/327] temporarily disable the new Integer type inference feature to verify if it is causing the zk validation failure issue. --- src/DataTypes/FieldToDataType.cpp | 2 +- .../queries/0_stateless/02832_integer_type_inference.reference | 2 +- tests/queries/0_stateless/02832_integer_type_inference.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 9b8e476067e..5947eff7ea2 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -155,7 +155,7 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const for (const Field & elem : x) element_types.emplace_back(applyVisitor(*this, elem)); - return std::make_shared(getLeastSupertype(element_types, true)); + return std::make_shared(getLeastSupertype(element_types, false)); } template diff --git a/tests/queries/0_stateless/02832_integer_type_inference.reference b/tests/queries/0_stateless/02832_integer_type_inference.reference index 2d072437c90..e1bfb315d40 100644 --- a/tests/queries/0_stateless/02832_integer_type_inference.reference +++ b/tests/queries/0_stateless/02832_integer_type_inference.reference @@ -1 +1 @@ -[-4741124612489978151,-3236599669630092879,5607475129431807682] +[-4741124612489978151,-3236599669630092879] diff --git a/tests/queries/0_stateless/02832_integer_type_inference.sql b/tests/queries/0_stateless/02832_integer_type_inference.sql index 9734bc2966f..9c558b3ae67 100644 --- a/tests/queries/0_stateless/02832_integer_type_inference.sql +++ b/tests/queries/0_stateless/02832_integer_type_inference.sql @@ -1 +1 @@ -select [-4741124612489978151, -3236599669630092879, 5607475129431807682]; +select [-4741124612489978151, -3236599669630092879]; From 4b3e399f7d450ed50b01741543e2888d5f28234c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Aug 2023 09:05:21 +0000 Subject: [PATCH 019/327] Wait for read requests --- src/Coordination/KeeperDispatcher.cpp | 92 +++++++++++++++---------- src/Coordination/KeeperDispatcher.h | 4 +- src/Coordination/KeeperStateMachine.cpp | 7 +- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStateManager.cpp | 2 - 5 files changed, 65 insertions(+), 42 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 99c28674273..4608e918042 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -12,6 +12,7 @@ #include #include +#include #include #include #include @@ -73,7 +74,6 @@ void KeeperDispatcher::requestThread() auto coordination_settings = configuration_and_settings->coordination_settings; uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds(); - uint64_t max_batch_size = coordination_settings->max_requests_batch_size; uint64_t max_batch_bytes_size = coordination_settings->max_requests_batch_bytes_size; /// The code below do a very simple thing: batch all write (quorum) requests into vector until @@ -136,12 +136,9 @@ void KeeperDispatcher::requestThread() return false; }; - /// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size - size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; - while (!shutdown_called && !has_read_request && - !has_reconfig_request && - current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size && - try_get_request()) + size_t max_batch_size = coordination_settings->max_requests_batch_size; + while (!shutdown_called && current_batch.size() < max_batch_size && !has_reconfig_request + && current_batch_bytes_size < max_batch_bytes_size && try_get_request()) ; const auto prev_result_done = [&] @@ -152,10 +149,8 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && - !has_reconfig_request && !prev_result_done() && - current_batch.size() <= max_batch_size - && current_batch_bytes_size < max_batch_bytes_size) + while (!shutdown_called && !has_reconfig_request && !prev_result_done() && current_batch.size() <= max_batch_size + && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); } @@ -166,9 +161,10 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; + nuraft::ptr result_buf = nullptr; /// Forcefully process all previous pending requests if (prev_result) - forceWaitAndProcessResult(prev_result, prev_batch); + result_buf = forceWaitAndProcessResult(prev_result, prev_batch); /// Process collected write requests batch if (!current_batch.empty()) @@ -177,13 +173,7 @@ void KeeperDispatcher::requestThread() auto result = server->putRequestBatch(current_batch); - if (result) - { - /// If we will execute read or reconfig next, we have to process result now - if (has_read_request || has_reconfig_request) - forceWaitAndProcessResult(result, current_batch); - } - else + if (!result) { addErrorResponses(current_batch, Coordination::Error::ZCONNECTIONLOSS); current_batch.clear(); @@ -194,6 +184,28 @@ void KeeperDispatcher::requestThread() prev_result = result; } + /// If we will execute read or reconfig next, we have to process result now + if (has_read_request || has_reconfig_request) + { + if (prev_result) + result_buf = forceWaitAndProcessResult(prev_result, current_batch); + + if (result_buf) + { + nuraft::buffer_serializer bs(result_buf); + auto log_idx = bs.get_u64(); + + while (true) + { + auto current_last_committed_idx = last_committed_log_idx.load(std::memory_order_relaxed); + if (current_last_committed_idx >= log_idx) + break; + + last_committed_log_idx.wait(current_last_committed_idx); + } + } + } + if (has_reconfig_request) server->getKeeperStateMachine()->reconfigure(request); @@ -360,28 +372,33 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshots_queue, keeper_context, snapshot_s3, - [this](const KeeperStorage::RequestForSession & request_for_session) + [this](uint64_t log_idx, const KeeperStorage::RequestForSession & request_for_session) { - /// check if we have queue of read requests depending on this request to be committed - std::lock_guard lock(read_request_queue_mutex); - if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - auto & xid_to_request_queue = it->second; - - if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); - request_queue_it != xid_to_request_queue.end()) + /// check if we have queue of read requests depending on this request to be committed + std::lock_guard lock(read_request_queue_mutex); + if (auto it = read_request_queue.find(request_for_session.session_id); it != read_request_queue.end()) { - for (const auto & read_request : request_queue_it->second) - { - if (server->isLeaderAlive()) - server->putLocalReadRequest(read_request); - else - addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); - } + auto & xid_to_request_queue = it->second; - xid_to_request_queue.erase(request_queue_it); + if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); + request_queue_it != xid_to_request_queue.end()) + { + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } + + xid_to_request_queue.erase(request_queue_it); + } } } + + last_committed_log_idx.store(log_idx, std::memory_order_relaxed); + last_committed_log_idx.notify_all(); }); try @@ -636,7 +653,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession } } -void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) +nuraft::ptr KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) { if (!result->has_result()) result->get(); @@ -647,8 +664,11 @@ void KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, Keep else if (result->get_result_code() != nuraft::cmd_result_code::OK) addErrorResponses(requests_for_sessions, Coordination::Error::ZCONNECTIONLOSS); + auto result_buf = result->get(); + result = nullptr; requests_for_sessions.clear(); + return result_buf; } int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index ae592ae3fa1..31a8f80f252 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -102,11 +102,13 @@ private: /// Forcefully wait for result and sets errors if something when wrong. /// Clears both arguments - void forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); + nuraft::ptr forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); public: std::mutex read_request_queue_mutex; + std::atomic last_committed_log_idx = 0; + /// queue of read requests that can be processed after a request with specific session ID and XID is committed std::unordered_map> read_request_queue; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 8f2e3c3ac0e..5a8cf88ea73 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -167,7 +167,10 @@ nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nur request_for_session->zxid = log_idx; preprocess(*request_for_session); - return nullptr; + auto result = nuraft::buffer::alloc(8); + nuraft::buffer_serializer ss(result); + ss.put_u64(log_idx); + return result; } std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) @@ -433,7 +436,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n last_committed_idx = log_idx; if (commit_callback) - commit_callback(*request_for_session); + commit_callback(log_idx, *request_for_session); return nullptr; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 116fa9257a0..aad5d3aafd4 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -20,7 +20,7 @@ using SnapshotsQueue = ConcurrentBoundedQueue; class KeeperStateMachine : public nuraft::state_machine { public: - using CommitCallback = std::function; + using CommitCallback = std::function; KeeperStateMachine( ResponsesQueue & responses_queue_, diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cc13c755629..902c2ef595f 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -184,9 +184,7 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur total_servers++; } - /// this will only apply to fresh clusters result.cluster_config->set_async_replication(true); - if (!result.config && !allow_without_us) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); From 12d262ed7ad779e6f2ed9961ee26719eded8d91f Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Tue, 8 Aug 2023 11:48:08 +0800 Subject: [PATCH 020/327] getPossiblePtr no longer has a default value, optimize the optimizeTypeIds method. --- src/DataTypes/FieldToDataType.cpp | 2 +- src/DataTypes/IDataType.h | 2 +- src/DataTypes/getLeastSupertype.cpp | 67 +++++-------------- .../02832_integer_type_inference.reference | 3 +- .../02832_integer_type_inference.sql | 3 +- 5 files changed, 24 insertions(+), 53 deletions(-) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 5947eff7ea2..9b8e476067e 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -155,7 +155,7 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const for (const Field & elem : x) element_types.emplace_back(applyVisitor(*this, elem)); - return std::make_shared(getLeastSupertype(element_types, false)); + return std::make_shared(getLeastSupertype(element_types, true)); } template diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 330836f9d92..52962462d7e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -73,7 +73,7 @@ public: DataTypePtr getPtr() const { return shared_from_this(); } - virtual DataTypes getPossiblePtr() const { return { shared_from_this() }; } + virtual DataTypes getPossiblePtr() const { return {}; } /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index c04130a58e4..d8be5fab91e 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -620,24 +620,6 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) } }; - auto maximize = [](size_t & what, size_t value, bool & only_unsigned, bool & only_signed, bool & both) - { - if (value > what) - { - what = value; - only_unsigned = false; - only_signed = false; - both = false; - return true; - }else if (value == what) - { - return true; - } - - return false; - }; - - size_t max_bits_of_integer = 0; bool only_unsigned = false; bool only_signed = false; bool both = false; @@ -652,41 +634,28 @@ void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) for (const auto & type : types) { TypeIndex type_id = type->getTypeId(); - bool is_max_bits = false; - if (type_id == TypeIndex::UInt8 || type_id == TypeIndex::Int8) - is_max_bits = maximize(max_bits_of_integer, 8, only_unsigned, only_signed, both); - else if (type_id == TypeIndex::UInt16 || type_id == TypeIndex::Int16) - is_max_bits = maximize(max_bits_of_integer, 16, only_unsigned, only_signed, both); - else if (type_id == TypeIndex::UInt32 || type_id == TypeIndex::Int32) - is_max_bits = maximize(max_bits_of_integer, 32, only_unsigned, only_signed, both); - else if (type_id == TypeIndex::UInt64 || type_id == TypeIndex::Int64) - is_max_bits = maximize(max_bits_of_integer, 64, only_unsigned, only_signed, both); - - if (is_max_bits) + bool type_is_unsigned = is_unsigned(type_id); + bool type_is_both = false; + for (const auto & possible_type : type->getPossiblePtr()) { - bool type_is_unsigned = is_unsigned(type_id); - bool type_is_both = false; - for (const auto & possible_type : type->getPossiblePtr()) + if (type_is_unsigned != is_unsigned(possible_type->getTypeId())) { - if (type_is_unsigned != is_unsigned(possible_type->getTypeId())) - { - type_is_both = true; - break; - } + type_is_both = true; + break; } - - if (type_is_unsigned) - has_unsigned = true; - else - has_signed = true; - - if (type_is_both) - both = true; - else if (type_is_unsigned) - only_unsigned = true; - else - only_signed = true; } + + if (type_is_unsigned) + has_unsigned = true; + else + has_signed = true; + + if (type_is_both) + both = true; + else if (type_is_unsigned) + only_unsigned = true; + else + only_signed = true; } auto optimize_type_id = [&is_unsigned](const DataTypePtr & type, bool try_change_unsigned) diff --git a/tests/queries/0_stateless/02832_integer_type_inference.reference b/tests/queries/0_stateless/02832_integer_type_inference.reference index e1bfb315d40..5a01bd4cd11 100644 --- a/tests/queries/0_stateless/02832_integer_type_inference.reference +++ b/tests/queries/0_stateless/02832_integer_type_inference.reference @@ -1 +1,2 @@ -[-4741124612489978151,-3236599669630092879] +[-4741124612489978151,-3236599669630092879,5607475129431807682] +[100,-100,5607475129431807682,5607475129431807683] diff --git a/tests/queries/0_stateless/02832_integer_type_inference.sql b/tests/queries/0_stateless/02832_integer_type_inference.sql index 9c558b3ae67..221e929d705 100644 --- a/tests/queries/0_stateless/02832_integer_type_inference.sql +++ b/tests/queries/0_stateless/02832_integer_type_inference.sql @@ -1 +1,2 @@ -select [-4741124612489978151, -3236599669630092879]; +select [-4741124612489978151, -3236599669630092879, 5607475129431807682]; +select [100, -100, 5607475129431807682, 5607475129431807683]; From 1dbe007ee58337b7ae8894f8fbf1f8d9078bec27 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 8 Aug 2023 13:12:07 +0000 Subject: [PATCH 021/327] Add coordination setting for async replication --- src/Coordination/Changelog.cpp | 11 +++++------ src/Coordination/Changelog.h | 8 ++++++++ src/Coordination/CoordinationSettings.h | 6 ++++-- src/Coordination/KeeperLogStore.cpp | 5 ++--- src/Coordination/KeeperLogStore.h | 2 +- src/Coordination/KeeperServer.cpp | 13 +++++++++---- src/Coordination/KeeperStateManager.cpp | 18 +++++++++++++----- src/Coordination/KeeperStateManager.h | 4 ++-- 8 files changed, 44 insertions(+), 23 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 7f972f67846..2af68173588 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -586,13 +586,15 @@ private: std::unique_ptr read_buf; }; -Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, KeeperContextPtr keeper_context_) +Changelog::Changelog( + Poco::Logger * log_, LogFileSettings log_file_settings, FlushSettings flush_settings_, KeeperContextPtr keeper_context_) : changelogs_detached_dir("detached") , rotate_interval(log_file_settings.rotate_interval) , log(log_) , write_operations(std::numeric_limits::max()) , append_completion_queue(std::numeric_limits::max()) , keeper_context(std::move(keeper_context_)) + , flush_settings(flush_settings_) { if (auto latest_log_disk = getLatestLogDisk(); log_file_settings.force_sync && dynamic_cast(latest_log_disk.get()) == nullptr) @@ -1017,12 +1019,9 @@ void Changelog::writeThread() size_t pending_appends = 0; bool try_batch_flush = false; - /// turn into setting - static constexpr size_t max_flush_batch_size = 1000; - const auto flush_logs = [&](const auto & flush) { - LOG_INFO(log, "Flushing {} logs", pending_appends); + LOG_TRACE(log, "Flushing {} logs", pending_appends); { std::lock_guard writer_lock(writer_mutex); @@ -1088,7 +1087,7 @@ void Changelog::writeThread() if (batch_append_ok) { /// we can try batching more logs for flush - if (pending_appends < max_flush_batch_size) + if (pending_appends < flush_settings.max_flush_batch_size) { try_batch_flush = true; continue; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 9789a6b03dd..dddcb9aa218 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -82,6 +82,11 @@ struct LogFileSettings uint64_t overallocate_size = 0; }; +struct FlushSettings +{ + uint64_t max_flush_batch_size = 1000; +}; + /// Simplest changelog with files rotation. /// No compression, no metadata, just entries with headers one by one. /// Able to read broken files/entries and discard them. Not thread safe. @@ -91,6 +96,7 @@ public: Changelog( Poco::Logger * log_, LogFileSettings log_file_settings, + FlushSettings flush_settings, KeeperContextPtr keeper_context_); Changelog(Changelog &&) = delete; @@ -229,6 +235,8 @@ private: KeeperContextPtr keeper_context; + const FlushSettings flush_settings; + bool initialized = false; }; diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 220265b6cb0..80a7c245de3 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -38,8 +38,9 @@ struct Settings; M(UInt64, stale_log_gap, 10000, "When node became stale and should receive snapshots from leader", 0) \ M(UInt64, fresh_log_gap, 200, "When node became fresh", 0) \ M(UInt64, max_request_queue_size, 100000, "Maximum number of request that can be in queue for processing", 0) \ - M(UInt64, max_requests_batch_size, 100, "Max size of batch of requests that can be sent to RAFT", 0) \ + M(UInt64, max_requests_batch_size, 1000, "Max size of batch of requests that can be sent to RAFT", 0) \ M(UInt64, max_requests_batch_bytes_size, 100*1024, "Max size in bytes of batch of requests that can be sent to RAFT", 0) \ + M(UInt64, max_flush_batch_size, 1000, "Max size of batch of requests that can be flushed together", 0) \ M(UInt64, max_requests_quick_batch_size, 100, "Max size of batch of requests to try to get before proceeding with RAFT. Keeper will not wait for requests but take only requests that are already in queue" , 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ @@ -48,7 +49,8 @@ struct Settings; M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) \ M(UInt64, max_log_file_size, 50 * 1024 * 1024, "Max size of the Raft log file. If possible, each created log file will preallocate this amount of bytes on disk. Set to 0 to disable the limit", 0) \ M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) \ - M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) + M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \ + M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index a9153475dbc..b55b083cb5c 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -6,9 +6,8 @@ namespace DB { -KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context) - : log(&Poco::Logger::get("KeeperLogStore")) - , changelog(log, log_file_settings, keeper_context) +KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, FlushSettings flush_settings, KeeperContextPtr keeper_context) + : log(&Poco::Logger::get("KeeperLogStore")), changelog(log, log_file_settings, flush_settings, keeper_context) { if (log_file_settings.force_sync) LOG_INFO(log, "force_sync enabled"); diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 6e71d8c55cf..de9205241bd 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -14,7 +14,7 @@ namespace DB class KeeperLogStore : public nuraft::log_store { public: - KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context); + KeeperLogStore(LogFileSettings log_file_settings, FlushSettings flush_settings, KeeperContextPtr keeper_context); /// Read log storage from filesystem starting from last_commited_log_index void init(uint64_t last_commited_log_index, uint64_t logs_to_keep); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 88b9f1cedb4..06d396e46ce 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -208,28 +208,33 @@ void KeeperServer::loadLatestConfig() { auto latest_snapshot_config = state_machine->getClusterConfig(); auto latest_log_store_config = state_manager->getLatestConfigFromLogStore(); + auto async_replication = coordination_settings->async_replication; if (latest_snapshot_config && latest_log_store_config) { if (latest_snapshot_config->get_log_idx() > latest_log_store_config->get_log_idx()) { LOG_INFO(log, "Will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); + latest_snapshot_config->set_async_replication(async_replication); state_manager->save_config(*latest_snapshot_config); } else { - LOG_INFO(log, "Will use config from log store with log index {}", latest_snapshot_config->get_log_idx()); + LOG_INFO(log, "Will use config from log store with log index {}", latest_log_store_config->get_log_idx()); + latest_log_store_config->set_async_replication(async_replication); state_manager->save_config(*latest_log_store_config); } } else if (latest_snapshot_config) { LOG_INFO(log, "No config in log store, will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); + latest_snapshot_config->set_async_replication(async_replication); state_manager->save_config(*latest_snapshot_config); } else if (latest_log_store_config) { LOG_INFO(log, "No config in snapshot, will use config from log store with log index {}", latest_log_store_config->get_log_idx()); + latest_log_store_config->set_async_replication(async_replication); state_manager->save_config(*latest_log_store_config); } else @@ -413,7 +418,7 @@ void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, boo loadLatestConfig(); - last_local_config = state_manager->parseServersConfiguration(config, true).cluster_config; + last_local_config = state_manager->parseServersConfiguration(config, true, coordination_settings->async_replication).cluster_config; launchRaftServer(config, enable_ipv6); @@ -837,12 +842,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) { - auto diff = state_manager->getRaftConfigurationDiff(config); + auto diff = state_manager->getRaftConfigurationDiff(config, coordination_settings); if (!diff.empty()) { std::lock_guard lock{server_write_mutex}; - last_local_config = state_manager->parseServersConfiguration(config, true).cluster_config; + last_local_config = state_manager->parseServersConfiguration(config, true, coordination_settings->async_replication).cluster_config; } return diff; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 902c2ef595f..f52d0749b10 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "Coordination/CoordinationSettings.h" namespace DB { @@ -74,7 +75,7 @@ std::unordered_map getClientPorts(const Poco::Util::Abstrac /// 4. No duplicate IDs /// 5. Our ID present in hostnames list KeeperStateManager::KeeperConfigurationWrapper -KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const +KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us, bool enable_async_replication) const { const bool hostname_checks_enabled = config.getBool(config_prefix + ".hostname_checks_enabled", true); @@ -184,7 +185,8 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur total_servers++; } - result.cluster_config->set_async_replication(true); + result.cluster_config->set_async_replication(enable_async_replication); + if (!result.config && !allow_without_us) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); @@ -221,6 +223,7 @@ KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, , secure(false) , log_store(nuraft::cs_new( LogFileSettings{.force_sync = false, .compress_logs = false, .rotate_interval = 5000}, + FlushSettings{}, keeper_context_)) , server_state_file_name("state") , keeper_context(keeper_context_) @@ -243,7 +246,7 @@ KeeperStateManager::KeeperStateManager( : my_server_id(my_server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) - , configuration_wrapper(parseServersConfiguration(config, false)) + , configuration_wrapper(parseServersConfiguration(config, false, coordination_settings->async_replication)) , log_store(nuraft::cs_new( LogFileSettings { @@ -253,6 +256,10 @@ KeeperStateManager::KeeperStateManager( .max_size = coordination_settings->max_log_file_size, .overallocate_size = coordination_settings->log_file_overallocate_size }, + FlushSettings + { + .max_flush_batch_size = coordination_settings->max_flush_batch_size, + }, keeper_context_)) , server_state_file_name(server_state_file_name_) , keeper_context(keeper_context_) @@ -452,9 +459,10 @@ nuraft::ptr KeeperStateManager::read_state() return nullptr; } -ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const +ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff( + const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) const { - auto new_configuration_wrapper = parseServersConfiguration(config, true); + auto new_configuration_wrapper = parseServersConfiguration(config, true, coordination_settings->async_replication); std::unordered_map new_ids, old_ids; for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers()) diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 5abeea604b5..e402143c179 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -93,7 +93,7 @@ public: ClusterConfigPtr getLatestConfigFromLogStore() const; // TODO (myrrc) This should be removed once "reconfig" is stabilized - ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) const; private: const String & getOldServerStatePath(); @@ -131,7 +131,7 @@ private: public: /// Parse configuration from xml config. - KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const; + KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us, bool enable_async_replication) const; }; } From d8904ffa694500676734c039fb04ddda3b7e00d0 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 8 Aug 2023 23:48:23 +0200 Subject: [PATCH 022/327] Simplified prepared statements handling for MySQL interface --- src/Core/MySQL/PacketsGeneric.h | 3 + src/Core/MySQL/PacketsPreparedStatements.cpp | 40 +++++++++++ src/Core/MySQL/PacketsPreparedStatements.h | 35 +++++++++ src/Server/MySQLHandler.cpp | 76 +++++++++++++++++++- src/Server/MySQLHandler.h | 10 +++ 5 files changed, 163 insertions(+), 1 deletion(-) create mode 100644 src/Core/MySQL/PacketsPreparedStatements.cpp create mode 100644 src/Core/MySQL/PacketsPreparedStatements.h diff --git a/src/Core/MySQL/PacketsGeneric.h b/src/Core/MySQL/PacketsGeneric.h index cf990bbf15c..969716dfc7a 100644 --- a/src/Core/MySQL/PacketsGeneric.h +++ b/src/Core/MySQL/PacketsGeneric.h @@ -54,6 +54,9 @@ enum Command COM_CHANGE_USER = 0x11, COM_BINLOG_DUMP = 0x12, COM_REGISTER_SLAVE = 0x15, + COM_STMT_PREPARE = 0x16, + COM_STMT_EXECUTE = 0x17, + COM_STMT_CLOSE = 0x19, COM_RESET_CONNECTION = 0x1f, COM_DAEMON = 0x1d, COM_BINLOG_DUMP_GTID = 0x1e diff --git a/src/Core/MySQL/PacketsPreparedStatements.cpp b/src/Core/MySQL/PacketsPreparedStatements.cpp new file mode 100644 index 00000000000..eddded585d5 --- /dev/null +++ b/src/Core/MySQL/PacketsPreparedStatements.cpp @@ -0,0 +1,40 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace MySQLProtocol +{ + namespace PreparedStatements + { + size_t PrepareStatementResponseOK::getPayloadSize() const + { + return 13; + } + + void PrepareStatementResponseOK::writePayloadImpl(WriteBuffer & buffer) const + { + buffer.write(reinterpret_cast(&status), 1); + buffer.write(reinterpret_cast(&statement_id), 4); + buffer.write(reinterpret_cast(&num_columns), 2); + buffer.write(reinterpret_cast(&num_params), 2); + buffer.write(reinterpret_cast(&reserved_1), 1); + buffer.write(reinterpret_cast(&warnings_count), 2); + buffer.write(0x0); // RESULTSET_METADATA_NONE + } + + void PrepareStatementResponseOK::readPayloadImpl([[maybe_unused]] ReadBuffer & payload) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PrepareStatementResponseOK::readPayloadImpl is not implemented"); + } + + PrepareStatementResponseOK::PrepareStatementResponseOK( + uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_) + : statement_id(statement_id_), num_columns(num_columns_), num_params(num_params_), warnings_count(warnings_count_) + { + } + } +} +} diff --git a/src/Core/MySQL/PacketsPreparedStatements.h b/src/Core/MySQL/PacketsPreparedStatements.h new file mode 100644 index 00000000000..7f180d4d142 --- /dev/null +++ b/src/Core/MySQL/PacketsPreparedStatements.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ +namespace MySQLProtocol +{ + namespace PreparedStatements + { + // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_prepare.html#sect_protocol_com_stmt_prepare_response_ok + class PrepareStatementResponseOK : public IMySQLWritePacket, public IMySQLReadPacket + { + public: + uint8_t status = 0x00; + uint32_t statement_id; + uint16_t num_columns; + uint16_t num_params; + uint8_t reserved_1 = 0; + uint16_t warnings_count; + + protected: + size_t getPayloadSize() const override; + + void readPayloadImpl(ReadBuffer & payload) override; + + void writePayloadImpl(WriteBuffer & buffer) const override; + + public: + PrepareStatementResponseOK(uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_); + }; + } +} +} diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index f98b86e6cf8..a0018d0dc27 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +41,7 @@ using namespace MySQLProtocol; using namespace MySQLProtocol::Generic; using namespace MySQLProtocol::ProtocolText; using namespace MySQLProtocol::ConnectionPhase; +using namespace MySQLProtocol::PreparedStatements; #if USE_SSL using Poco::Net::SecureStreamSocket; @@ -181,6 +183,15 @@ void MySQLHandler::run() case COM_PING: comPing(); break; + case COM_STMT_PREPARE: + comStmtPrepare(payload); + break; + case COM_STMT_EXECUTE: + comStmtExecute(payload); + break; + case COM_STMT_CLOSE: + comStmtClose(payload); + break; default: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Command {} is not implemented.", command); } @@ -254,7 +265,8 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl { try { - // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible (if password is specified using double SHA1). Otherwise SHA256 plugin is used. + // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible + // (if password is specified using double SHA1). Otherwise, SHA256 plugin is used. if (session->getAuthenticationTypeOrLogInFailure(user_name) == DB::AuthenticationType::SHA256_PASSWORD) { authPluginSSL(); @@ -371,6 +383,68 @@ void MySQLHandler::comQuery(ReadBuffer & payload) } } +void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) +{ + String query; + readStringUntilEOF(query, payload); + + uint32_t statement_id = current_prepared_statement_id; + if (current_prepared_statement_id == std::numeric_limits::max()) + { + current_prepared_statement_id = 0; + } + else + { + current_prepared_statement_id++; + } + + // Key collisions should not happen here, as we remove the elements from the map with COM_STMT_CLOSE, + // and we have quite a big range of available identifiers with 32-bit unsigned integer + if (prepared_statements_map.contains(statement_id)) [[unlikely]] + { + LOG_ERROR( + log, + "Failed to store a new statement `{}` with id {}; it is already taken by `{}`", + query, + statement_id, + prepared_statements_map.at(statement_id)); + packet_endpoint->sendPacket(ERRPacket(), true); + return; + } + prepared_statements_map.emplace(statement_id, query); + + packet_endpoint->sendPacket(PrepareStatementResponseOK(statement_id, 0, 0, 0), true); +} + +void MySQLHandler::comStmtExecute(ReadBuffer & payload) +{ + uint32_t statement_id; + payload.readStrict(reinterpret_cast(&statement_id), 4); + + if (!prepared_statements_map.contains(statement_id)) [[unlikely]] + { + LOG_ERROR(log, "Could not find prepared statement with id {}", statement_id); + packet_endpoint->sendPacket(ERRPacket(), true); + return; + } + + // Temporary workaround as we work only with queries that do not bind any parameters atm + ReadBufferFromString com_query_payload(prepared_statements_map.at(statement_id)); + MySQLHandler::comQuery(com_query_payload); +}; + +void MySQLHandler::comStmtClose([[maybe_unused]] ReadBuffer & payload) { + uint32_t statement_id; + payload.readStrict(reinterpret_cast(&statement_id), 4); + + if (prepared_statements_map.contains(statement_id)) { + prepared_statements_map.erase(statement_id); + } + + // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_close.html + // No response packet is sent back to the client. +}; + void MySQLHandler::authPluginSSL() { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 3366e8792c9..6b8cc56a46e 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -56,6 +56,12 @@ protected: void authenticate(const String & user_name, const String & auth_plugin_name, const String & auth_response); + void comStmtPrepare(ReadBuffer & payload); + + void comStmtExecute(ReadBuffer & payload); + + void comStmtClose(ReadBuffer & payload); + virtual void authPluginSSL(); virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); @@ -76,6 +82,10 @@ protected: using Replacements = std::unordered_map; Replacements replacements; + uint32_t current_prepared_statement_id = 0; + using PreparedStatementsMap = std::unordered_map; + PreparedStatementsMap prepared_statements_map; + std::unique_ptr auth_plugin; std::shared_ptr in; std::shared_ptr out; From ef3e0f7b836864d1ddcc9d114d319f639654e661 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Aug 2023 06:25:47 +0000 Subject: [PATCH 023/327] Use async replication in tests --- tests/config/config.d/keeper_port.xml | 2 ++ .../helpers/0_common_enable_keeper_async_replication.xml | 7 +++++++ tests/integration/helpers/cluster.py | 2 ++ tests/integration/helpers/keeper_config1.xml | 2 ++ tests/integration/helpers/keeper_config2.xml | 2 ++ tests/integration/helpers/keeper_config3.xml | 2 ++ tests/jepsen.clickhouse/resources/keeper_config.xml | 1 + tests/jepsen.clickhouse/resources/keeper_config_solo.xml | 1 + 8 files changed, 19 insertions(+) create mode 100644 tests/integration/helpers/0_common_enable_keeper_async_replication.xml diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 7db174c5419..beac507304f 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -19,6 +19,8 @@ 0 0 0 + + 1 diff --git a/tests/integration/helpers/0_common_enable_keeper_async_replication.xml b/tests/integration/helpers/0_common_enable_keeper_async_replication.xml new file mode 100644 index 00000000000..4ecada09444 --- /dev/null +++ b/tests/integration/helpers/0_common_enable_keeper_async_replication.xml @@ -0,0 +1,7 @@ + + + + 1 + + + \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eff44de842a..1f3cd473e29 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4258,6 +4258,8 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) + write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) + logging.debug("Generate and write macros file") macros = self.macros.copy() macros["instance"] = self.name diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index f40ed9ac6fa..7702aecba9c 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -20,6 +20,8 @@ false 2000 4000 + + 1 diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index d5bdb92a79d..2a1a1c1003c 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -20,6 +20,8 @@ false 2000 4000 + + 1 diff --git a/tests/integration/helpers/keeper_config3.xml b/tests/integration/helpers/keeper_config3.xml index aa69b554660..035da1bbd22 100644 --- a/tests/integration/helpers/keeper_config3.xml +++ b/tests/integration/helpers/keeper_config3.xml @@ -20,6 +20,8 @@ false 2000 4000 + + 1 diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 1972ef6b917..6bc4ad89839 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -47,6 +47,7 @@ {snapshot_distance} {stale_log_gap} {reserved_log_items} + 1 diff --git a/tests/jepsen.clickhouse/resources/keeper_config_solo.xml b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml index b20592545cd..0054cad8f85 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config_solo.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml @@ -21,6 +21,7 @@ 1000 2000 4000 + 1 From 7fdb414793e533444808d111fd29f8fda904e15e Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 9 Aug 2023 15:18:05 +0200 Subject: [PATCH 024/327] One more [[unlikely]] --- src/Server/MySQLHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index a0018d0dc27..868575b701f 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -389,7 +389,7 @@ void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) readStringUntilEOF(query, payload); uint32_t statement_id = current_prepared_statement_id; - if (current_prepared_statement_id == std::numeric_limits::max()) + if (current_prepared_statement_id == std::numeric_limits::max()) [[unlikely]] { current_prepared_statement_id = 0; } From efb041c4d5482a9acc26aa718d5c9b47e1ad95ab Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Wed, 9 Aug 2023 23:03:50 +0800 Subject: [PATCH 025/327] Optimize the code and streamline the functionality by retaining only the feature of automatically recognizing UInt64 as Int64. --- src/Common/ErrorCodes.cpp | 3 + src/DataTypes/DataTypesNumber.h | 19 +++-- src/DataTypes/FieldToDataType.cpp | 23 +----- src/DataTypes/IDataType.cpp | 6 ++ src/DataTypes/IDataType.h | 11 ++- src/DataTypes/getLeastSupertype.cpp | 113 ++++++---------------------- 6 files changed, 61 insertions(+), 114 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ae8d5f8796d..3df92c941e8 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -582,6 +582,9 @@ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ M(699, INVALID_REDIS_TABLE_STRUCTURE) \ + M(700, USER_SESSION_LIMIT_EXCEEDED) \ + M(701, CLUSTER_DOESNT_EXIST) \ + M(702, OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/DataTypes/DataTypesNumber.h b/src/DataTypes/DataTypesNumber.h index 8511955e193..6be07db1062 100644 --- a/src/DataTypes/DataTypesNumber.h +++ b/src/DataTypes/DataTypesNumber.h @@ -18,9 +18,10 @@ class DataTypeNumber final : public DataTypeNumberBase public: DataTypeNumber() = default; - explicit DataTypeNumber(DataTypes data_types) + explicit DataTypeNumber(DataTypePtr opposite_sign_data_type_) : DataTypeNumberBase() - , possible_data_types(std::move(data_types)) + , opposite_sign_data_type(std::move(opposite_sign_data_type_)) + , has_opposite_sign_data_type(true) { } @@ -39,15 +40,23 @@ public: return std::make_shared(); } + bool hasOppositeSignDataType() const override { return has_opposite_sign_data_type; } + DataTypePtr oppositeSignDataType() const override + { + if (!has_opposite_sign_data_type) + IDataType::oppositeSignDataType(); + + return opposite_sign_data_type; + } + SerializationPtr doGetDefaultSerialization() const override { return std::make_shared>(); } - DataTypes getPossiblePtr() const override { return possible_data_types; } - private: - DataTypes possible_data_types; + DataTypePtr opposite_sign_data_type; + bool has_opposite_sign_data_type = false; }; using DataTypeUInt8 = DataTypeNumber; diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 9b8e476067e..2f759e79661 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -33,34 +33,19 @@ DataTypePtr FieldToDataType::operator() (const Null &) const template DataTypePtr FieldToDataType::operator() (const UInt64 & x) const { - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); + if (x <= std::numeric_limits::max()) return std::make_shared(std::make_shared()); return std::make_shared(); } template DataTypePtr FieldToDataType::operator() (const Int64 & x) const { - if (x >= 0) - { - if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); - if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); - if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(DataTypes{ std::make_shared() }); - } - else - { - if (x >= std::numeric_limits::min()) return std::make_shared(); - if (x >= std::numeric_limits::min()) return std::make_shared(); - if (x >= std::numeric_limits::min()) return std::make_shared(); - } + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); return std::make_shared(); } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 4ffe82039b2..221254e575e 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int DATA_TYPE_CANNOT_BE_PROMOTED; + extern const int OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND; extern const int ILLEGAL_COLUMN; } @@ -71,6 +72,11 @@ DataTypePtr IDataType::promoteNumericType() const throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED, "Data type {} can't be promoted.", getName()); } +DataTypePtr IDataType::oppositeSignDataType() const +{ + throw Exception(ErrorCodes::OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND, "Opposite sign data type not found for {}.", getName()); +} + size_t IDataType::getSizeOfValueInMemory() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of type {} in memory is not of fixed size.", getName()); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 52962462d7e..867ed6df5e3 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -73,8 +73,6 @@ public: DataTypePtr getPtr() const { return shared_from_this(); } - virtual DataTypes getPossiblePtr() const { return {}; } - /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; /// Name of corresponding data type in MySQL (exampe: Bigint, Blob, etc) @@ -160,6 +158,15 @@ public: */ virtual DataTypePtr promoteNumericType() const; + /** The data type has an opposite sign DataTypePtr type. + * Data types that can have an opposite sign are typically signed or unsigned types. + */ + virtual bool hasOppositeSignDataType() const { return false; } + + /** Return the opposite sign data type of the current data type. Throw an exception if `hasOppositeSignDataType() == false`. + */ + virtual DataTypePtr oppositeSignDataType() const; + /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overridden if data type default value differs from column default value (example: Enum data types). */ diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index d8be5fab91e..4c399e99caa 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -605,113 +605,50 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids) void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) { - // Determine whether the type_id is UInt - auto is_unsigned = [](const TypeIndex & type_id) + auto is_signed_int = [](const TypeIndex & type_id) { switch (type_id) { - case TypeIndex::UInt8: - case TypeIndex::UInt16: - case TypeIndex::UInt32: - case TypeIndex::UInt64: + case TypeIndex::Int8: + case TypeIndex::Int16: + case TypeIndex::Int32: + case TypeIndex::Int64: return true; default: return false; } }; - bool only_unsigned = false; - bool only_signed = false; - bool both = false; - bool has_unsigned = false; - bool has_signed = false; + bool has_signed_int = false; + bool has_uint64_and_has_opposite = false; + TypeIndexSet opposite_type_ids; - // Determine the distribution of maximum signed and unsigned, Example: - // Int64, Int64 = only_signed. - // UInt64, UInt64 = only_unsigned. - // UInt64(possible: Int64), Int64(possible: UInt64) = both. - // UInt64(possible: Int64), Int64 = both, only_signed. + // Determine whether UInt64 in type_ids needs to change its sign. for (const auto & type : types) { - TypeIndex type_id = type->getTypeId(); - bool type_is_unsigned = is_unsigned(type_id); - bool type_is_both = false; - for (const auto & possible_type : type->getPossiblePtr()) + auto type_id = type->getTypeId(); + + if (!has_signed_int) + has_signed_int = is_signed_int(type_id); + + if (type_id == TypeIndex::UInt64) { - if (type_is_unsigned != is_unsigned(possible_type->getTypeId())) + if (!type->hasOppositeSignDataType()) { - type_is_both = true; - break; + has_uint64_and_has_opposite = false; + break ; + }else + { + has_uint64_and_has_opposite = true; + opposite_type_ids.insert(type->oppositeSignDataType()->getTypeId()); } } - - if (type_is_unsigned) - has_unsigned = true; - else - has_signed = true; - - if (type_is_both) - both = true; - else if (type_is_unsigned) - only_unsigned = true; - else - only_signed = true; } - auto optimize_type_id = [&is_unsigned](const DataTypePtr & type, bool try_change_unsigned) + if (has_uint64_and_has_opposite && has_signed_int) { - TypeIndex type_id = type->getTypeId(); - switch (type_id) - { - case TypeIndex::UInt8: - case TypeIndex::UInt16: - case TypeIndex::UInt32: - case TypeIndex::UInt64: - if (try_change_unsigned) - return type_id; - break ; - case TypeIndex::Int8: - case TypeIndex::Int16: - case TypeIndex::Int32: - case TypeIndex::Int64: - if (!try_change_unsigned) - return type_id; - break ; - default: - return type_id; - } - - for (const auto & other_type : type->getPossiblePtr()) - { - TypeIndex other_type_id = other_type->getTypeId(); - if ((try_change_unsigned && is_unsigned(other_type_id)) - || (!try_change_unsigned && !is_unsigned(other_type_id))) - { - return other_type_id; - } - } - - return type_id; - }; - - // optimize type_ids - if (both) - { - // Example: Int64(possible: UInt32), UInt64 = UInt32, UInt64 - if (only_unsigned && !only_signed) - { - type_ids.clear(); - for (const auto & type : types) - type_ids.insert(optimize_type_id(type, true)); - } - // Example: UInt64(possible: Int64), Int64 = Int64, Int64 - // Int64(possible: UInt32), UInt64(possible: Int64) = Int64, Int64 - else if ((only_signed && !only_unsigned) || (has_unsigned && has_signed && !only_signed && !only_unsigned)) - { - type_ids.clear(); - for (const auto & type : types) - type_ids.insert(optimize_type_id(type, false)); - } + type_ids.erase(TypeIndex::UInt64); + type_ids.insert(opposite_type_ids.begin(), opposite_type_ids.end()); } } From da2a041d6891058bee5bbed621a312526be8ef21 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 10 Aug 2023 11:23:43 +0800 Subject: [PATCH 026/327] Temporarily modify ErrorCodes to avoid conflicts. --- src/Common/ErrorCodes.cpp | 3 --- src/DataTypes/IDataType.cpp | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3df92c941e8..ae8d5f8796d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -582,9 +582,6 @@ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ M(699, INVALID_REDIS_TABLE_STRUCTURE) \ - M(700, USER_SESSION_LIMIT_EXCEEDED) \ - M(701, CLUSTER_DOESNT_EXIST) \ - M(702, OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 221254e575e..092b9ed8e0e 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -23,7 +23,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int DATA_TYPE_CANNOT_BE_PROMOTED; - extern const int OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND; extern const int ILLEGAL_COLUMN; } @@ -74,7 +73,8 @@ DataTypePtr IDataType::promoteNumericType() const DataTypePtr IDataType::oppositeSignDataType() const { - throw Exception(ErrorCodes::OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND, "Opposite sign data type not found for {}.", getName()); + // TODO: Should use ErrorCodes::OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND. + throw Exception(702, "Opposite sign data type not found for {}.", getName()); } size_t IDataType::getSizeOfValueInMemory() const From e4a1780bd74ad9de0dbce1e97509c396911a500b Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 10 Aug 2023 15:29:27 +0300 Subject: [PATCH 027/327] update docs according to the PR + fix the same for HDFS --- docs/en/sql-reference/table-functions/file.md | 3 ++- docs/en/sql-reference/table-functions/hdfs.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/hdfs.md | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 8 ++++++-- 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 4db9494502e..508e58ed3ca 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -141,7 +141,8 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored. + - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 680ac54ee78..b4f63b35592 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -45,7 +45,7 @@ Multiple path components can have globs. For being processed file should exists - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored. - `{N..M}` — Substitutes any number in range from N to M including both borders. Constructions with `{}` are similar to the [remote table function](../../sql-reference/table-functions/remote.md)). diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 83ef115aacd..d7107ba5950 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). diff --git a/docs/ru/sql-reference/table-functions/hdfs.md b/docs/ru/sql-reference/table-functions/hdfs.md index b0d182eef14..350d483b592 100644 --- a/docs/ru/sql-reference/table-functions/hdfs.md +++ b/docs/ru/sql-reference/table-functions/hdfs.md @@ -43,7 +43,7 @@ LIMIT 2 - `*` — Заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — Заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы. - `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index aa99917d533..da7b1e3fe73 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -92,9 +92,13 @@ namespace HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); - if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + if (ls.file_info == nullptr && errno != ENOENT && errno != EACCES) // NOLINT { - // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. + // ignore: + // file not found (as in LSWithRegexpMatching) + // permission denied (there is no easy way to determine + // if we really need access or just scanning all dirs while doing recursive search), + // keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. throw Exception( ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); } From 086bf6cb8ce6912e3d19afa3b2cc0f1842c1af91 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 11 Aug 2023 15:49:21 +0300 Subject: [PATCH 028/327] added ignore_eacces_multidirectory_globs --- src/Core/Settings.h | 6 ++- src/Storages/HDFS/StorageHDFS.cpp | 38 +++++++++-------- src/Storages/StorageFile.cpp | 33 +++++++++------ ...771_complex_globs_in_storage_file_path.sql | 12 ------ ...ltidirectory_globs_storage_file.reference} | 3 ++ ...02771_multidirectory_globs_storage_file.sh | 42 +++++++++++++++++++ 6 files changed, 91 insertions(+), 43 deletions(-) delete mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql rename tests/queries/0_stateless/{02771_complex_globs_in_storage_file_path.reference => 02771_multidirectory_globs_storage_file.reference} (76%) create mode 100755 tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6c3d339b4be..aa2a78f00af 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -785,8 +785,10 @@ class IColumn; M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ - M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0)\ - // End of COMMON_SETTINGS + M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ + M(Bool, ignore_eacces_multidirectory_globs, false, "Ignore access denied errors when processing multi-directory globs.", 0)\ + +// End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. #define MAKE_OBSOLETE(M, TYPE, NAME, DEFAULT) \ diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index da7b1e3fe73..1163a200bf7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -67,7 +67,8 @@ namespace /// Forward-declared to use in LSWithFoldedRegexpMatching w/o circular dependency. std::vector LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, - const String & for_match); + const String & for_match, + bool ignore_eacces_multidirectory_globs); /* * When `{...}` has any `/`s, it must be processed in a different way: @@ -83,7 +84,8 @@ namespace const String & suffix_with_globs, re2::RE2 & matcher, const size_t max_depth, - const size_t next_slash_after_glob_pos) + const size_t next_slash_after_glob_pos, + bool ignore_eacces_multidirectory_globs) { /// We don't need to go all the way in every directory if max_depth is reached /// as it is upper limit of depth by simply counting `/`s in curly braces @@ -92,15 +94,14 @@ namespace HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); - if (ls.file_info == nullptr && errno != ENOENT && errno != EACCES) // NOLINT + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT { - // ignore: - // file not found (as in LSWithRegexpMatching) - // permission denied (there is no easy way to determine - // if we really need access or just scanning all dirs while doing recursive search), + // ignore file not found (as in LSWithRegexpMatching) // keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. - throw Exception( - ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); + // ignore permission denied if ignore_eacces_multidirectory_globs is true + if (!(ignore_eacces_multidirectory_globs && errno == EACCES)) + throw Exception( + ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); } std::vector result; @@ -126,15 +127,16 @@ namespace else { std::vector result_part = LSWithRegexpMatching( - fs::path(full_path) / "" , fs, suffix_with_globs.substr(next_slash_after_glob_pos)); + fs::path(full_path) / "" , fs, suffix_with_globs.substr(next_slash_after_glob_pos), + ignore_eacces_multidirectory_globs); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } else if (is_directory) { std::vector result_part = LSWithFoldedRegexpMatching( - fs::path(full_path), fs, processed_suffix + dir_or_file_name, - suffix_with_globs, matcher, max_depth - 1, next_slash_after_glob_pos); + fs::path(full_path), fs, processed_suffix + dir_or_file_name, suffix_with_globs, + matcher, max_depth - 1, next_slash_after_glob_pos, ignore_eacces_multidirectory_globs); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -147,7 +149,8 @@ namespace std::vector LSWithRegexpMatching( const String & path_for_ls, const HDFSFSPtr & fs, - const String & for_match) + const String & for_match, + bool ignore_eacces_multidirectory_globs) { const size_t first_glob_pos = for_match.find_first_of("*?{"); const bool has_glob = first_glob_pos != std::string::npos; @@ -189,8 +192,8 @@ namespace if (slashes_in_glob) { - return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, "", suffix_with_globs, - matcher, slashes_in_glob, next_slash_after_glob_pos); + return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, "", suffix_with_globs, matcher, + slashes_in_glob, next_slash_after_glob_pos, ignore_eacces_multidirectory_globs); } HDFSFileInfo ls; @@ -223,7 +226,8 @@ namespace { if (re2::RE2::FullMatch(file_name, matcher)) { - std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash_after_glob_pos)); + std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, + suffix_with_globs.substr(next_slash_after_glob_pos), ignore_eacces_multidirectory_globs); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } @@ -251,7 +255,7 @@ namespace HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - return LSWithRegexpMatching("/", fs, path_from_uri); + return LSWithRegexpMatching("/", fs, path_from_uri, context->getSettingsRef().ignore_eacces_multidirectory_globs); } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a70095cf808..3b0646039ac 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -100,6 +100,7 @@ void listFilesWithRegexpMatchingImpl( const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read, + bool ignore_eacces_multidirectory_globs, std::vector & result, bool recursive = false); @@ -118,13 +119,17 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, size_t & total_bytes_to_read, const size_t max_depth, const size_t next_slash_after_glob_pos, + bool ignore_eacces_multidirectory_globs, std::vector & result) { if (!max_depth) return; const fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls, fs::directory_options::skip_permission_denied); it != end; ++it) + fs::directory_iterator it = ignore_eacces_multidirectory_globs + ? fs::directory_iterator(path_for_ls, fs::directory_options::skip_permission_denied) + : fs::directory_iterator(path_for_ls); + for (; it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); @@ -141,14 +146,15 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, { listFilesWithRegexpMatchingImpl(fs::path(full_path) / "" , suffix_with_globs.substr(next_slash_after_glob_pos), - total_bytes_to_read, result); + total_bytes_to_read, ignore_eacces_multidirectory_globs, result); } } else if (it->is_directory()) { listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path), processed_suffix + dir_or_file_name, suffix_with_globs, matcher, total_bytes_to_read, - max_depth - 1, next_slash_after_glob_pos, result); + max_depth - 1, next_slash_after_glob_pos, + ignore_eacces_multidirectory_globs, result); } } @@ -161,6 +167,7 @@ void listFilesWithRegexpMatchingImpl( const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read, + bool ignore_eacces_multidirectory_globs, std::vector & result, bool recursive) { @@ -203,7 +210,7 @@ void listFilesWithRegexpMatchingImpl( re2::RE2 matcher(regexp); if (!matcher.ok()) throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); + "Cannot compile regex from gllistFilesWithFoldedRegexpMatchingImplob ({}): {}", for_match, matcher.error()); bool skip_regex = current_glob == "/*" ? true : false; if (!recursive) @@ -218,9 +225,9 @@ void listFilesWithRegexpMatchingImpl( if (slashes_in_glob) { - listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, - matcher, total_bytes_to_read, slashes_in_glob, - next_slash_after_glob_pos, result); + listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, matcher, + total_bytes_to_read, slashes_in_glob, next_slash_after_glob_pos, + ignore_eacces_multidirectory_globs, result); return; } @@ -246,11 +253,12 @@ void listFilesWithRegexpMatchingImpl( { listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "", looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob, - total_bytes_to_read, result, recursive); + total_bytes_to_read, ignore_eacces_multidirectory_globs, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), + total_bytes_to_read, ignore_eacces_multidirectory_globs, result); } } } @@ -258,10 +266,11 @@ void listFilesWithRegexpMatchingImpl( std::vector listFilesWithRegexpMatching( const std::string & path_for_ls, const std::string & for_match, - size_t & total_bytes_to_read) + size_t & total_bytes_to_read, + bool ignore_eacces_multidirectory_globs) { std::vector result; - listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, result); + listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, ignore_eacces_multidirectory_globs, result); return result; } @@ -450,7 +459,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user else { /// We list only non-directory files. - paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read); + paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read, context->getSettingsRef().ignore_eacces_multidirectory_globs); can_be_directory = false; } diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql deleted file mode 100644 index 1d125920220..00000000000 --- a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql +++ /dev/null @@ -1,12 +0,0 @@ --- Tags: no-replicated-database, no-parallel - -SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta/non_existing.csv', CSV); -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} - -INSERT INTO TABLE FUNCTION file('02771/dir1/subdir11/data1.csv', 'CSV', 's String') SELECT 'This is file data1' SETTINGS engine_file_truncate_on_insert=1; -INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's String') SELECT 'This is file data2' SETTINGS engine_file_truncate_on_insert=1; - -SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); -SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); - -SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; -SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference similarity index 76% rename from tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference rename to tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference index e1b420ecf37..a2fa4d7fe3d 100644 --- a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference @@ -1,4 +1,7 @@ +OK This is file data1 data1.csv This is file data2 data2.csv This is file data1 data1.csv This is file data2 data2.csv +OK +This is file data1 data1.csv diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh new file mode 100755 index 00000000000..f8f7e69e524 --- /dev/null +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# Tags: no-replicated-database, no-parallel + +#SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; +#SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/* +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ + +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta/non_existing.csv', CSV);" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo 'OK' || echo 'FAIL' + +# Create two files in different directories +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/ +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/ + +echo 'This is file data1' > ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir1/subdir11/data1.csv +echo 'This is file data2' > ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir2/subdir22/data2.csv + +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV);" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV);" + +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv';" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv';" + +# Add a directory to test against permission_denied +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && chmod 000 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ + +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0;" 2>&1 | grep -q "Permission denied" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1;" + +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} From 1983c7ce956621a88b4677a1c167d6498f518add Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 11 Aug 2023 16:53:17 +0300 Subject: [PATCH 029/327] update docs --- docs/en/operations/settings/settings.md | 40 ++++++++++++++++++ docs/en/sql-reference/table-functions/file.md | 7 ++-- docs/en/sql-reference/table-functions/hdfs.md | 5 ++- docs/ru/operations/settings/settings.md | 41 +++++++++++++++++++ docs/ru/sql-reference/table-functions/file.md | 3 +- docs/ru/sql-reference/table-functions/hdfs.md | 6 ++- .../aspell-ignore/en/aspell-dict.txt | 2 + 7 files changed, 95 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index de3cc00e4c9..9411ec84e78 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4614,3 +4614,43 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars │ 1.7091 │ 15008753 │ └─────────────────────┴──────────────────────────┘ ``` + +## ignore_eacces_multidirectory_globs {#ignore_eacces_multidirectory_globs} + +Allows to ignore 'permission denied' errors when using multi-directory `{}` globs for [File](../../sql-reference/table-functions/file.md#globs_in_path) and [HDFS](../../sql-reference/table-functions/hdfs.md) storages. + +Possible values: `0`, `1`. + +Default value: `0`. + +### Example + +Having the following structure in `user_files`: +``` +my_directory/ +├── data1 +│ ├── f1.csv +├── data2 +│ ├── f2.csv +└── test_root +``` +where `data1`, `data2` directories are accessible, but one has no rights to read `test_root` directories. + +For a query like `SELECT *, _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV)` an exception will be thrown: +`Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied`. +It happens because a multi-directory glob requires a recursive search in _all_ available directories under `my_directory`. + +If this setting is on, all inaccessible directories will be silently skipped, even if they are explicitly specified inside `{}`. + +```sql +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0; + +Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied +``` +```sql +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1; + +┌─_path───────────────────┬─_file───────┐ +│ │ +└─────────────────────────┴─────────────┘ +``` diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 508e58ed3ca..e3ee9c4f999 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -135,14 +135,13 @@ Getting data from table in table.csv, located in archive1.zip or/and archive2.zi SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ``` -## Globs in Path +## Globs in Path {#globs_in_path} Multiple path components can have globs. For being processed file must exist and match to the whole path pattern (not only suffix or prefix). - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored. - +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. @@ -211,7 +210,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - [engine_file_allow_create_multiple_files](/docs/en/operations/settings/settings.md#engine_file_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. - [engine_file_skip_empty_files](/docs/en/operations/settings/settings.md#engine_file_skip_empty_files) - allows to skip empty files while reading. Disabled by default. - [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: read, pread, mmap (only for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. - +- [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. **See Also** diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index b4f63b35592..0e4ee632b71 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -39,13 +39,13 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` -**Globs in path** +## Globs in path {#globs_in_path} Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern (not only suffix or prefix). - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting. - `{N..M}` — Substitutes any number in range from N to M including both borders. Constructions with `{}` are similar to the [remote table function](../../sql-reference/table-functions/remote.md)). @@ -102,6 +102,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. - [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. - [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. **See Also** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index d3db890ad7a..a230efcd8c6 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4206,3 +4206,44 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars │ 1.7091 │ 15008753 │ └─────────────────────┴──────────────────────────┘ ``` + +## ignore_eacces_multidirectory_globs {#ignore_eacces_multidirectory_globs} + +Позволяет игнорировать ошибку 'permission denied', возникающую при использовании шаблона `{}`, содержащего `/` внутри себя. +Работает для [File](../../sql-reference/table-functions/file.md#globs_in_path) и [HDFS](../../sql-reference/table-functions/hdfs.md). + +Возможные значения: `0`, `1`. + +Значение по умолчанию: `0`. + +### Пример + +Пусть в `user_files` имеется следующая структура: +``` +my_directory/ +├── data1 +│ ├── f1.csv +├── data2 +│ ├── f2.csv +└── test_root +``` +Пусть также директории `data1`, `data2` могут быть прочитаны, но прав на чтение `test_root` нет. + +На запрос `SELECT *, _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV)` будет выброшено исключение: +`Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied`. +Это происходит, так как для обработки такого шаблона необходимо выполнить рекурсивный поиск по _всем_ директориям, находящимся внутри `my_directory`. + +Если данная настройка имеет значение 1, то недоступные директории будут тихо пропущены, даже если они явно указаны внутри `{}`. + +```sql +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0; + +Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied +``` +```sql +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1; + +┌─_path───────────────────┬─_file───────┐ +│ │ +└─────────────────────────┴─────────────┘ +``` diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index d7107ba5950..32d98bd19ee 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs). - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). @@ -123,6 +123,7 @@ SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, - `_path` — путь к файлу. - `_file` — имя файла. + **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) diff --git a/docs/ru/sql-reference/table-functions/hdfs.md b/docs/ru/sql-reference/table-functions/hdfs.md index 350d483b592..d74248bd93c 100644 --- a/docs/ru/sql-reference/table-functions/hdfs.md +++ b/docs/ru/sql-reference/table-functions/hdfs.md @@ -39,11 +39,11 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` -**Шаблоны в пути** +## Шаблоны поиска в компонентах пути {#globs-in-path} - `*` — Заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — Заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы. +- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs). - `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). @@ -61,3 +61,5 @@ LIMIT 2 **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) +- Параметр [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs) + diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index aefb14d098f..a65c294b1d8 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1385,6 +1385,7 @@ dragonbox dropoff dumpColumnStructure durations +eacces ecto embeddings emptyArray @@ -1793,6 +1794,7 @@ monthName moscow msgpack msgpk +multidirectory multiFuzzyMatchAllIndices multiFuzzyMatchAny multiFuzzyMatchAnyIndex From e28e4be0feec36a16702e9c2361eab89bb7311d2 Mon Sep 17 00:00:00 2001 From: zvonand Date: Fri, 11 Aug 2023 18:21:03 +0300 Subject: [PATCH 030/327] update tests according to style --- .../02771_multidirectory_globs_storage_file.reference | 1 - .../0_stateless/02771_multidirectory_globs_storage_file.sh | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference index a2fa4d7fe3d..b0746e0a1be 100644 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.reference @@ -3,5 +3,4 @@ This is file data1 data1.csv This is file data2 data2.csv This is file data1 data1.csv This is file data2 data2.csv -OK This is file data1 data1.csv diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh index f8f7e69e524..4901e716be3 100755 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -15,7 +15,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/* +rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ ${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta/non_existing.csv', CSV);" 2>&1 | grep -q "CANNOT_EXTRACT_TABLE_STRUCTURE" && echo 'OK' || echo 'FAIL' @@ -34,9 +34,8 @@ ${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CL ${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv';" # Add a directory to test against permission_denied -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && chmod 000 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ +rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/dir_inaccessible/ && mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && chmod 000 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0;" 2>&1 | grep -q "Permission denied" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1;" -rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} +rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} From 74485f3e52c96702a8c85ec98a5b9999bf7db764 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Tue, 15 Aug 2023 03:25:32 +0800 Subject: [PATCH 031/327] Addressed review feedback --- src/DataTypes/DataTypesNumber.h | 27 +---------------- src/DataTypes/FieldToDataType.cpp | 5 ++-- src/DataTypes/IDataType.cpp | 6 ---- src/DataTypes/IDataType.h | 9 ------ src/DataTypes/getLeastSupertype.cpp | 45 +++++++++-------------------- src/DataTypes/getLeastSupertype.h | 2 +- 6 files changed, 19 insertions(+), 75 deletions(-) diff --git a/src/DataTypes/DataTypesNumber.h b/src/DataTypes/DataTypesNumber.h index 6be07db1062..b48c5ff1f5d 100644 --- a/src/DataTypes/DataTypesNumber.h +++ b/src/DataTypes/DataTypesNumber.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -10,21 +9,10 @@ namespace DB { -using DataTypes = std::vector; - template -class DataTypeNumber final : public DataTypeNumberBase +class DataTypeNumber : public DataTypeNumberBase { public: - DataTypeNumber() = default; - - explicit DataTypeNumber(DataTypePtr opposite_sign_data_type_) - : DataTypeNumberBase() - , opposite_sign_data_type(std::move(opposite_sign_data_type_)) - , has_opposite_sign_data_type(true) - { - } - bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } bool canBeUsedAsVersion() const override { return true; } @@ -40,23 +28,10 @@ public: return std::make_shared(); } - bool hasOppositeSignDataType() const override { return has_opposite_sign_data_type; } - DataTypePtr oppositeSignDataType() const override - { - if (!has_opposite_sign_data_type) - IDataType::oppositeSignDataType(); - - return opposite_sign_data_type; - } - SerializationPtr doGetDefaultSerialization() const override { return std::make_shared>(); } - -private: - DataTypePtr opposite_sign_data_type; - bool has_opposite_sign_data_type = false; }; using DataTypeUInt8 = DataTypeNumber; diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 2f759e79661..52a46bf55bc 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -36,7 +37,7 @@ DataTypePtr FieldToDataType::operator() (const UInt64 & x) const if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(std::make_shared()); + if (x <= std::numeric_limits::max()) return std::make_shared(std::make_shared()); return std::make_shared(); } @@ -140,7 +141,7 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const for (const Field & elem : x) element_types.emplace_back(applyVisitor(*this, elem)); - return std::make_shared(getLeastSupertype(element_types, true)); + return std::make_shared(getLeastSupertype(element_types)); } template diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 092b9ed8e0e..4ffe82039b2 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -71,12 +71,6 @@ DataTypePtr IDataType::promoteNumericType() const throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED, "Data type {} can't be promoted.", getName()); } -DataTypePtr IDataType::oppositeSignDataType() const -{ - // TODO: Should use ErrorCodes::OPPOSITE_SIGN_DATA_TYPE_NOT_FOUND. - throw Exception(702, "Opposite sign data type not found for {}.", getName()); -} - size_t IDataType::getSizeOfValueInMemory() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of type {} in memory is not of fixed size.", getName()); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index d49c163072e..54cb3d0d5c2 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -159,15 +159,6 @@ public: */ virtual DataTypePtr promoteNumericType() const; - /** The data type has an opposite sign DataTypePtr type. - * Data types that can have an opposite sign are typically signed or unsigned types. - */ - virtual bool hasOppositeSignDataType() const { return false; } - - /** Return the opposite sign data type of the current data type. Throw an exception if `hasOppositeSignDataType() == false`. - */ - virtual DataTypePtr oppositeSignDataType() const; - /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overridden if data type default value differs from column default value (example: Enum data types). */ diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 4c399e99caa..30bbf8e2786 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -201,7 +202,7 @@ DataTypePtr getNumericType(const TypeIndexSet & types) } template -DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids) +DataTypePtr getLeastSupertype(const DataTypes & types) { /// Trivial cases @@ -592,8 +593,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids) /// For numeric types, the most complicated part. { - if (optimize_type_ids) - optimizeTypeIds(types, type_ids); + optimizeTypeIds(types, type_ids); auto numeric_type = getNumericType(type_ids); if (numeric_type) return numeric_type; @@ -603,49 +603,32 @@ DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids) return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); } +// Convert the UInt64 type to Int64 in order to cover other signed_integer types and obtain the least super type of all ints. +// Example, UInt64(both Int64), Int8 = Int64, Int8. void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) { - auto is_signed_int = [](const TypeIndex & type_id) - { - switch (type_id) - { - case TypeIndex::Int8: - case TypeIndex::Int16: - case TypeIndex::Int32: - case TypeIndex::Int64: - return true; - default: - return false; - } - }; + if ((!type_ids.contains(TypeIndex::Int8) && !type_ids.contains(TypeIndex::Int16) && !type_ids.contains(TypeIndex::Int32) && !type_ids.contains(TypeIndex::Int64)) || !type_ids.contains(TypeIndex::UInt64)) + return; - bool has_signed_int = false; - bool has_uint64_and_has_opposite = false; + bool has_opposite = false; TypeIndexSet opposite_type_ids; - // Determine whether UInt64 in type_ids needs to change its sign. for (const auto & type : types) { auto type_id = type->getTypeId(); - if (!has_signed_int) - has_signed_int = is_signed_int(type_id); - if (type_id == TypeIndex::UInt64) { - if (!type->hasOppositeSignDataType()) + if (const auto * uint64_with_opposite = typeid_cast(type.get())) { - has_uint64_and_has_opposite = false; - break ; + has_opposite = true; + opposite_type_ids.insert(uint64_with_opposite->getOppositeSignDataType()->getTypeId()); }else - { - has_uint64_and_has_opposite = true; - opposite_type_ids.insert(type->oppositeSignDataType()->getTypeId()); - } + return; } } - if (has_uint64_and_has_opposite && has_signed_int) + if (has_opposite) { type_ids.erase(TypeIndex::UInt64); type_ids.insert(opposite_type_ids.begin(), opposite_type_ids.end()); @@ -713,7 +696,7 @@ DataTypePtr tryGetLeastSupertype(const TypeIndexSet & types) return getLeastSupertype(types); } -template DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids); +template DataTypePtr getLeastSupertype(const DataTypes & types); template DataTypePtr getLeastSupertype(const TypeIndexSet & types); } diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index ba43b3966f9..0a3fa3c2536 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -18,7 +18,7 @@ enum class LeastSupertypeOnError * Examples: there is no least common supertype for Array(UInt8), Int8. */ template -DataTypePtr getLeastSupertype(const DataTypes & types, bool optimize_type_ids = false); +DataTypePtr getLeastSupertype(const DataTypes & types); /// Same as above but return String type instead of throwing exception. /// All types can be casted to String, because they can be serialized to String. From 34b1811a53f0299c6a2a2845137a7558000d0c9d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 15 Aug 2023 19:10:14 +0300 Subject: [PATCH 032/327] Update src/Storages/StorageFile.cpp seems like typo Co-authored-by: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 3b0646039ac..ddb6b8a3844 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -210,7 +210,7 @@ void listFilesWithRegexpMatchingImpl( re2::RE2 matcher(regexp); if (!matcher.ok()) throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from gllistFilesWithFoldedRegexpMatchingImplob ({}): {}", for_match, matcher.error()); + "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); bool skip_regex = current_glob == "/*" ? true : false; if (!recursive) From d161a87a34123591c0ac7412f3189d1e60be318d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 15 Aug 2023 19:10:40 +0300 Subject: [PATCH 033/327] Update docs/en/sql-reference/table-functions/file.md Co-authored-by: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> --- docs/en/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index e3ee9c4f999..f12ac732c22 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -141,7 +141,7 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting for file & HDFS. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. From 72574f8376529be17c69dfd36f9775054edd3ada Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 15 Aug 2023 19:10:49 +0300 Subject: [PATCH 034/327] Update src/Core/Settings.h Co-authored-by: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e4053ca80..cec400a2ada 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -786,7 +786,7 @@ class IColumn; M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, ignore_eacces_multidirectory_globs, false, "Ignore access denied errors when processing multi-directory globs.", 0)\ + M(Bool, ignore_eacces_multidirectory_globs, false, "Ignore access denied errors when processing multi-directory globs for file & HDFS.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From 26833856ca6b86cab6f350d71f369eb5e6c746c0 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Wed, 16 Aug 2023 02:38:25 +0800 Subject: [PATCH 035/327] Perform optimization of LeastSupertype within FieldToDataType, eliminating modifications to DataType. --- src/DataTypes/DataTypesNumber.h | 2 +- src/DataTypes/FieldToDataType.cpp | 74 +++++++++++++++++++++++++++-- src/DataTypes/getLeastSupertype.cpp | 34 ------------- src/DataTypes/getLeastSupertype.h | 2 - 4 files changed, 70 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/DataTypesNumber.h b/src/DataTypes/DataTypesNumber.h index b48c5ff1f5d..5843086248c 100644 --- a/src/DataTypes/DataTypesNumber.h +++ b/src/DataTypes/DataTypesNumber.h @@ -10,7 +10,7 @@ namespace DB { template -class DataTypeNumber : public DataTypeNumberBase +class DataTypeNumber final : public DataTypeNumberBase { public: bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); } diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 52a46bf55bc..3a99552089c 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -37,7 +36,6 @@ DataTypePtr FieldToDataType::operator() (const UInt64 & x) const if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); - if (x <= std::numeric_limits::max()) return std::make_shared(std::make_shared()); return std::make_shared(); } @@ -138,8 +136,36 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const DataTypes element_types; element_types.reserve(x.size()); + auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, + const DataTypePtr & type, const Field & elem) { + if (uint64_could_opposite) { + has_signed_int |= WhichDataType(type).isNativeInt(); + + if (type->getTypeId() == TypeIndex::UInt64) { + has_uint64 = true; + uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); + } + } + }; + + bool has_signed_int = false; + bool has_uint64 = false; + bool uint64_could_opposite = true; for (const Field & elem : x) - element_types.emplace_back(applyVisitor(*this, elem)); + { + DataTypePtr type = applyVisitor(*this, elem); + element_types.emplace_back(type); + checkIfConversionSigned(has_signed_int, has_uint64, uint64_could_opposite, type, elem); + } + + // Convert the UInt64 type to Int64 in order to cover other signed_integer types + // and obtain the least super type of all ints. + if (has_signed_int && has_uint64 && uint64_could_opposite) + { + for (auto & type : element_types) + if (type->getTypeId() == TypeIndex::UInt64) + type = std::make_shared(); + } return std::make_shared(getLeastSupertype(element_types)); } @@ -167,14 +193,52 @@ DataTypePtr FieldToDataType::operator() (const Map & map) const key_types.reserve(map.size()); value_types.reserve(map.size()); + auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, + const DataTypePtr & type, const Field & elem) { + if (uint64_could_opposite) { + has_signed_int |= WhichDataType(type).isNativeInt(); + + if (type->getTypeId() == TypeIndex::UInt64) { + has_uint64 = true; + uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); + } + } + }; + + auto updateUInt64Types = [](DataTypes types) { + for (auto& type : types) { + if (type->getTypeId() == TypeIndex::UInt64) { + type = std::make_shared(); + } + } + }; + + bool k_has_signed_int = false; + bool k_has_uint64 = false; + bool k_uint64_could_opposite = true; + bool v_has_signed_int = false; + bool v_has_uint64 = false; + bool v_uint64_could_opposite = true; for (const auto & elem : map) { const auto & tuple = elem.safeGet(); assert(tuple.size() == 2); - key_types.push_back(applyVisitor(*this, tuple[0])); - value_types.push_back(applyVisitor(*this, tuple[1])); + DataTypePtr k_type = applyVisitor(*this, tuple[0]); + key_types.push_back(k_type); + checkIfConversionSigned(k_has_signed_int, k_has_uint64, k_uint64_could_opposite, k_type, tuple[0]); + DataTypePtr v_type = applyVisitor(*this, tuple[1]); + value_types.push_back(v_type); + checkIfConversionSigned(v_has_signed_int, v_has_uint64, v_uint64_could_opposite, v_type, tuple[1]); } + // Convert the UInt64 type to Int64 in order to cover other signed_integer types + // and obtain the least super type of all ints. + if (k_has_signed_int && k_has_uint64 && k_uint64_could_opposite) + updateUInt64Types(key_types); + + if (v_has_signed_int && v_has_uint64 && v_uint64_could_opposite) + updateUInt64Types(value_types); + return std::make_shared( getLeastSupertype(key_types), getLeastSupertype(value_types)); diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 30bbf8e2786..9d42d82ce91 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include @@ -593,7 +592,6 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For numeric types, the most complicated part. { - optimizeTypeIds(types, type_ids); auto numeric_type = getNumericType(type_ids); if (numeric_type) return numeric_type; @@ -603,38 +601,6 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return throwOrReturn(types, "", ErrorCodes::NO_COMMON_TYPE); } -// Convert the UInt64 type to Int64 in order to cover other signed_integer types and obtain the least super type of all ints. -// Example, UInt64(both Int64), Int8 = Int64, Int8. -void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids) -{ - if ((!type_ids.contains(TypeIndex::Int8) && !type_ids.contains(TypeIndex::Int16) && !type_ids.contains(TypeIndex::Int32) && !type_ids.contains(TypeIndex::Int64)) || !type_ids.contains(TypeIndex::UInt64)) - return; - - bool has_opposite = false; - TypeIndexSet opposite_type_ids; - - for (const auto & type : types) - { - auto type_id = type->getTypeId(); - - if (type_id == TypeIndex::UInt64) - { - if (const auto * uint64_with_opposite = typeid_cast(type.get())) - { - has_opposite = true; - opposite_type_ids.insert(uint64_with_opposite->getOppositeSignDataType()->getTypeId()); - }else - return; - } - } - - if (has_opposite) - { - type_ids.erase(TypeIndex::UInt64); - type_ids.insert(opposite_type_ids.begin(), opposite_type_ids.end()); - } -} - DataTypePtr getLeastSupertypeOrString(const DataTypes & types) { return getLeastSupertype(types); diff --git a/src/DataTypes/getLeastSupertype.h b/src/DataTypes/getLeastSupertype.h index 0a3fa3c2536..2ef4a0e6850 100644 --- a/src/DataTypes/getLeastSupertype.h +++ b/src/DataTypes/getLeastSupertype.h @@ -29,8 +29,6 @@ DataTypePtr tryGetLeastSupertype(const DataTypes & types); using TypeIndexSet = std::unordered_set; -void optimizeTypeIds(const DataTypes & types, TypeIndexSet & type_ids); - template DataTypePtr getLeastSupertype(const TypeIndexSet & types); From a8cdc9472072e08e76e73ce91597634f0d6ec319 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 15 Aug 2023 23:42:04 +0500 Subject: [PATCH 036/327] update docs acc to review --- docs/en/operations/settings/settings.md | 1 + docs/ru/operations/settings/settings.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9411ec84e78..4f284aa3354 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4618,6 +4618,7 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars ## ignore_eacces_multidirectory_globs {#ignore_eacces_multidirectory_globs} Allows to ignore 'permission denied' errors when using multi-directory `{}` globs for [File](../../sql-reference/table-functions/file.md#globs_in_path) and [HDFS](../../sql-reference/table-functions/hdfs.md) storages. +This setting is only applicable when multi directory `{}` glob is used. Possible values: `0`, `1`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a230efcd8c6..3f0bc4ee2ec 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4211,6 +4211,7 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars Позволяет игнорировать ошибку 'permission denied', возникающую при использовании шаблона `{}`, содержащего `/` внутри себя. Работает для [File](../../sql-reference/table-functions/file.md#globs_in_path) и [HDFS](../../sql-reference/table-functions/hdfs.md). +Работает _только_ для указанных выше шаблонов `{}`. Возможные значения: `0`, `1`. From 3eed7dc3cc59437c5dfb18cf6dfedafb209c6e34 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Wed, 16 Aug 2023 09:26:39 +0800 Subject: [PATCH 037/327] fix style --- src/DataTypes/FieldToDataType.cpp | 27 ++++++++++++++++----------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 3a99552089c..e25a2f546f2 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -137,11 +137,14 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const element_types.reserve(x.size()); auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, - const DataTypePtr & type, const Field & elem) { - if (uint64_could_opposite) { + const DataTypePtr & type, const Field & elem) + { + if (uint64_could_opposite) + { has_signed_int |= WhichDataType(type).isNativeInt(); - if (type->getTypeId() == TypeIndex::UInt64) { + if (type->getTypeId() == TypeIndex::UInt64) + { has_uint64 = true; uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); } @@ -194,23 +197,25 @@ DataTypePtr FieldToDataType::operator() (const Map & map) const value_types.reserve(map.size()); auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, - const DataTypePtr & type, const Field & elem) { - if (uint64_could_opposite) { + const DataTypePtr & type, const Field & elem) + { + if (uint64_could_opposite) + { has_signed_int |= WhichDataType(type).isNativeInt(); - if (type->getTypeId() == TypeIndex::UInt64) { + if (type->getTypeId() == TypeIndex::UInt64) + { has_uint64 = true; uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); } } }; - auto updateUInt64Types = [](DataTypes types) { - for (auto& type : types) { - if (type->getTypeId() == TypeIndex::UInt64) { + auto updateUInt64Types = [](DataTypes types) + { + for (auto& type : types) + if (type->getTypeId() == TypeIndex::UInt64) type = std::make_shared(); - } - } }; bool k_has_signed_int = false; From 7fa72f5cdfd61de0c4b79e4275f142c452ce60e6 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Aug 2023 12:47:26 +0000 Subject: [PATCH 038/327] Optimize group by constant keys --- src/Common/BoolArgsToTemplateArgsDispatcher.h | 40 +++++ src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 3 +- src/Interpreters/Aggregator.cpp | 142 ++++++++++++------ src/Interpreters/Aggregator.h | 8 +- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Planner/Planner.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 1 + .../TTL/TTLAggregationAlgorithm.cpp | 3 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +- .../02845_group_by_constant_keys.reference | 4 + .../02845_group_by_constant_keys.sql | 5 + 12 files changed, 161 insertions(+), 51 deletions(-) create mode 100644 src/Common/BoolArgsToTemplateArgsDispatcher.h create mode 100644 tests/queries/0_stateless/02845_group_by_constant_keys.reference create mode 100644 tests/queries/0_stateless/02845_group_by_constant_keys.sql diff --git a/src/Common/BoolArgsToTemplateArgsDispatcher.h b/src/Common/BoolArgsToTemplateArgsDispatcher.h new file mode 100644 index 00000000000..086b1e4b0f4 --- /dev/null +++ b/src/Common/BoolArgsToTemplateArgsDispatcher.h @@ -0,0 +1,40 @@ +#pragma once +#include + +namespace DB +{ + + +/// Special struct that helps to convert bool variables to function template bool arguments. +/// It can be used to avoid multiple nested if/else on bool arguments. How to use it: +/// Imagine you have template function +/// template return_type foo(...); +/// and bool variables b1, b2, ... bn. To pass these variables as template for foo you can do the following: +/// +/// auto call_foo = []() +/// { +/// return foo(...); +/// } +/// +/// BoolArgsToTemplateArgsDispatcher::call(call_foo, b1, b2, ..., bn); + +template +struct BoolArgsToTemplateArgsDispatcher +{ + template + static auto call(Func f, Args1&&... args) + { + return f.template operator()(std::forward(args)...); + } + + template + static auto call(Func f, bool b, Args1&&... ar1) + { + if (b) + return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); + else + return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); + } +}; + +} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6c3d339b4be..418ae08ac7f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -650,6 +650,7 @@ class IColumn; M(SetOperationMode, except_default_mode, SetOperationMode::ALL, "Set default mode in EXCEPT query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ + M(Bool, optimize_group_by_constant_keys, true, "Optimize GROUP BY when all keys in block are constant", 0) \ M(Bool, legacy_column_name_of_tuple_literal, false, "List all names of element of large tuple literals in their column names instead of hash. This settings exists only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher.", 0) \ \ M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index dcb67165add..eaa932a3520 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,8 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, + {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}, + {"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}}}, {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 91cd574708a..72ea2c8ec42 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include @@ -1036,11 +1037,12 @@ void Aggregator::executeImpl( ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, bool no_more_keys, + bool all_keys_are_const, AggregateDataPtr overflow_row) const { #define M(NAME, IS_TWO_LEVEL) \ else if (result.type == AggregatedDataVariants::Type::NAME) \ - executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, overflow_row); + executeImpl(*result.NAME, result.aggregates_pool, row_begin, row_end, key_columns, aggregate_instructions, no_more_keys, all_keys_are_const, overflow_row); if (false) {} // NOLINT APPLY_FOR_AGGREGATED_VARIANTS(M) @@ -1060,44 +1062,35 @@ void NO_INLINE Aggregator::executeImpl( ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, bool no_more_keys, + bool all_keys_are_const, AggregateDataPtr overflow_row) const { typename Method::State state(key_columns, key_sizes, aggregation_state_cache); + auto call_execute_impl_batch = [&]() + { + executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); + }; + + bool use_compiled_functions = false; +#if USE_EMBEDDED_COMPILER + use_compiled_functions = compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions); +#endif + if (!no_more_keys) { /// Prefetching doesn't make sense for small hash tables, because they fit in caches entirely. const bool prefetch = Method::State::has_cheap_key_calculation && params.enable_prefetch && (method.data.getBufferSizeInBytes() > min_bytes_for_prefetch); - -#if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions)) - { - if (prefetch) - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - else - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - } - else -#endif - { - if (prefetch) - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - else - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - } + BoolArgsToTemplateArgsDispatcher::call(call_execute_impl_batch, no_more_keys, use_compiled_functions, prefetch, all_keys_are_const); } else { - executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); + BoolArgsToTemplateArgsDispatcher::call(call_execute_impl_batch, no_more_keys, false, false, all_keys_are_const); } } -template +template void NO_INLINE Aggregator::executeImplBatch( Method & method, typename Method::State & state, @@ -1121,27 +1114,34 @@ void NO_INLINE Aggregator::executeImplBatch( /// For all rows. AggregateDataPtr place = aggregates_pool->alloc(0); - for (size_t i = row_begin; i < row_end; ++i) + if (all_keys_are_const) { - if constexpr (prefetch && HasPrefetchMemberFunc) + state.emplaceKey(method.data, 0, *aggregates_pool).setMapped(place); + } + else + { + for (size_t i = row_begin; i < row_end; ++i) { - if (i == row_begin + prefetching.iterationsToMeasure()) - prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); - - if (i + prefetch_look_ahead < row_end) + if constexpr (prefetch && HasPrefetchMemberFunc) { - auto && key_holder = state.getKeyHolder(i + prefetch_look_ahead, *aggregates_pool); - method.data.prefetch(std::move(key_holder)); - } - } + if (i == row_begin + prefetching.iterationsToMeasure()) + prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); - state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place); + if (i + prefetch_look_ahead < row_end) + { + auto && key_holder = state.getKeyHolder(i + prefetch_look_ahead, *aggregates_pool); + method.data.prefetch(std::move(key_holder)); + } + } + + state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place); + } } return; } /// Optimization for special case when aggregating by 8bit key. - if constexpr (!no_more_keys && std::is_same_v) + if constexpr (!no_more_keys && !all_keys_are_const && std::is_same_v) { /// We use another method if there are aggregate functions with -Array combinator. bool has_arrays = false; @@ -1183,7 +1183,20 @@ void NO_INLINE Aggregator::executeImplBatch( std::unique_ptr places(new AggregateDataPtr[row_end]); /// For all rows. - for (size_t i = row_begin; i < row_end; ++i) + size_t start, end; + /// If all keys are const, key columns contain only 1 row. + if constexpr (all_keys_are_const) + { + start = 0; + end = 1; + } + else + { + start = row_begin; + end = row_end; + } + + for (size_t i = start; i < end; ++i) { AggregateDataPtr aggregate_data = nullptr; @@ -1279,8 +1292,16 @@ void NO_INLINE Aggregator::executeImplBatch( columns_data.emplace_back(getColumnData(inst->batch_arguments[argument_index])); } - auto add_into_aggregate_states_function = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function; - add_into_aggregate_states_function(row_begin, row_end, columns_data.data(), places.get()); + if constexpr (all_keys_are_const) + { + auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place; + add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), places[0]); + } + else + { + auto add_into_aggregate_states_function = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function; + add_into_aggregate_states_function(row_begin, row_end, columns_data.data(), places.get()); + } } #endif @@ -1295,12 +1316,24 @@ void NO_INLINE Aggregator::executeImplBatch( AggregateFunctionInstruction * inst = aggregate_instructions + i; - if (inst->offsets) - inst->batch_that->addBatchArray(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool); - else if (inst->has_sparse_arguments) - inst->batch_that->addBatchSparse(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); + if constexpr (all_keys_are_const) + { + if (inst->offsets) + inst->batch_that->addBatchSinglePlace(inst->offsets[static_cast(row_begin) - 1], inst->offsets[row_end - 1], places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); + else if (inst->has_sparse_arguments) + inst->batch_that->addBatchSparseSinglePlace(row_begin, row_end, places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); + else + inst->batch_that->addBatchSinglePlace(row_begin, row_end, places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); + } else - inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); + { + if (inst->offsets) + inst->batch_that->addBatchArray(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool); + else if (inst->has_sparse_arguments) + inst->batch_that->addBatchSparse(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); + else + inst->batch_that->addBatch(row_begin, row_end, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); + } } } @@ -1540,12 +1573,27 @@ bool Aggregator::executeOnBlock(Columns columns, * To make them work anyway, we materialize them. */ Columns materialized_columns; + bool all_keys_are_const = false; + if (params.optimize_group_by_constant_keys) + { + all_keys_are_const = true; + for (size_t i = 0; i < params.keys_size; ++i) + all_keys_are_const &= isColumnConst(*columns.at(keys_positions[i])); + } /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) { - materialized_columns.push_back(recursiveRemoveSparse(columns.at(keys_positions[i]))->convertToFullColumnIfConst()); - key_columns[i] = materialized_columns.back().get(); + if (all_keys_are_const) + { + key_columns[i] = assert_cast(*columns.at(keys_positions[i])).getDataColumnPtr().get(); + } + else + { + materialized_columns.push_back(recursiveRemoveSparse(columns.at(keys_positions[i]))->convertToFullColumnIfConst()); + key_columns[i] = materialized_columns.back().get(); + } + if (!result.isLowCardinality()) { @@ -1590,7 +1638,7 @@ bool Aggregator::executeOnBlock(Columns columns, { /// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`. AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr; - executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), no_more_keys, overflow_row_ptr); + executeImpl(result, row_begin, row_end, key_columns, aggregate_functions_instructions.data(), no_more_keys, all_keys_are_const, overflow_row_ptr); } size_t result_size = result.sizeWithoutOverflowRow(); diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4f2c86606c5..dc0391c4289 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1020,6 +1020,8 @@ public: bool enable_prefetch; + bool optimize_group_by_constant_keys; + struct StatsCollectingParams { StatsCollectingParams(); @@ -1057,6 +1059,7 @@ public: size_t max_block_size_, bool enable_prefetch_, bool only_merge_, // true for projections + bool optimize_group_by_constant_keys_, const StatsCollectingParams & stats_collecting_params_ = {}) : keys(keys_) , aggregates(aggregates_) @@ -1077,6 +1080,7 @@ public: , max_block_size(max_block_size_) , only_merge(only_merge_) , enable_prefetch(enable_prefetch_) + , optimize_group_by_constant_keys(optimize_group_by_constant_keys_) , stats_collecting_params(stats_collecting_params_) { } @@ -1276,6 +1280,7 @@ private: ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, bool no_more_keys = false, + bool all_keys_are_const = false, AggregateDataPtr overflow_row = nullptr) const; /// Process one data block, aggregate the data into a hash table. @@ -1288,10 +1293,11 @@ private: ColumnRawPtrs & key_columns, AggregateFunctionInstruction * aggregate_instructions, bool no_more_keys, + bool all_keys_are_const, AggregateDataPtr overflow_row) const; /// Specialization for a particular value no_more_keys. - template + template void executeImplBatch( Method & method, typename Method::State & state, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 20fca1b1e76..8839bd8f7e2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2579,6 +2579,7 @@ static Aggregator::Params getAggregatorParams( settings.max_block_size, settings.enable_software_prefetch_in_aggregation, /* only_merge */ false, + settings.optimize_group_by_constant_keys, stats_collecting_params }; } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9f6c22f90f3..a75031898c4 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -290,6 +290,7 @@ Aggregator::Params getAggregatorParams(const PlannerContextPtr & planner_context settings.max_block_size, settings.enable_software_prefetch_in_aggregation, /* only_merge */ false, + settings.optimize_group_by_constant_keys, stats_collecting_params); return aggregator_params; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index eebbfc04304..c68a070f816 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -230,6 +230,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B transform_params->params.max_block_size, transform_params->params.enable_prefetch, /* only_merge */ false, + transform_params->params.optimize_group_by_constant_keys, transform_params->params.stats_collecting_params}; auto transform_params_for_set = std::make_shared(src_header, std::move(params_for_set), final); diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index cdd6e3e917f..fa3436ec55d 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -41,7 +41,8 @@ TTLAggregationAlgorithm::TTLAggregationAlgorithm( settings.min_count_to_compile_aggregate_expression, settings.max_block_size, settings.enable_software_prefetch_in_aggregation, - false /* only_merge */); + false /* only_merge */, + settings.optimize_group_by_constant_keys); aggregator = std::make_unique(header, params); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ee515106591..8f9d54d942d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -328,7 +328,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( settings.min_count_to_compile_aggregate_expression, settings.max_block_size, settings.enable_software_prefetch_in_aggregation, - only_merge); + only_merge, + settings.optimize_group_by_constant_keys); return std::make_pair(params, only_merge); }; diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.reference b/tests/queries/0_stateless/02845_group_by_constant_keys.reference new file mode 100644 index 00000000000..60e40ea54a7 --- /dev/null +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.reference @@ -0,0 +1,4 @@ +10000000 1 2 3 +10000000 1 2 3 +10000000 1 2 3 +10000000 1 2 3 diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.sql b/tests/queries/0_stateless/02845_group_by_constant_keys.sql new file mode 100644 index 00000000000..0223cf1df60 --- /dev/null +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.sql @@ -0,0 +1,5 @@ +SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 0; +SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions = 1; +SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 1; + From 1f5e209897b4d45dbaa6c9e2588bcc7a536702a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Aug 2023 12:50:04 +0000 Subject: [PATCH 039/327] Better comment --- src/Common/BoolArgsToTemplateArgsDispatcher.h | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Common/BoolArgsToTemplateArgsDispatcher.h b/src/Common/BoolArgsToTemplateArgsDispatcher.h index 086b1e4b0f4..469bc34cfbf 100644 --- a/src/Common/BoolArgsToTemplateArgsDispatcher.h +++ b/src/Common/BoolArgsToTemplateArgsDispatcher.h @@ -4,36 +4,35 @@ namespace DB { - /// Special struct that helps to convert bool variables to function template bool arguments. /// It can be used to avoid multiple nested if/else on bool arguments. How to use it: /// Imagine you have template function /// template return_type foo(...); -/// and bool variables b1, b2, ... bn. To pass these variables as template for foo you can do the following: +/// and bool variables b1, b2, ..., bool bn. To pass these variables as template for foo you can do the following: /// -/// auto call_foo = []() +/// auto call_foo = []() /// { /// return foo(...); /// } /// /// BoolArgsToTemplateArgsDispatcher::call(call_foo, b1, b2, ..., bn); -template +template struct BoolArgsToTemplateArgsDispatcher { template - static auto call(Func f, Args1&&... args) + static auto call(Functor f, Args1&&... args) { return f.template operator()(std::forward(args)...); } template - static auto call(Func f, bool b, Args1&&... ar1) + static auto call(Functor f, bool b, Args1&&... ar1) { if (b) - return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); + return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); else - return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); + return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); } }; From aac513185952a8d12f6e9e0247623a63d3c39ee4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Aug 2023 13:21:16 +0000 Subject: [PATCH 040/327] Fix for no_more_keys --- src/Common/BoolArgsToTemplateArgsDispatcher.h | 4 ++-- src/Interpreters/Aggregator.cpp | 12 ++++++++++-- src/Storages/StorageFile.cpp | 4 ++-- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Common/BoolArgsToTemplateArgsDispatcher.h b/src/Common/BoolArgsToTemplateArgsDispatcher.h index 469bc34cfbf..3e72db49cd2 100644 --- a/src/Common/BoolArgsToTemplateArgsDispatcher.h +++ b/src/Common/BoolArgsToTemplateArgsDispatcher.h @@ -7,8 +7,8 @@ namespace DB /// Special struct that helps to convert bool variables to function template bool arguments. /// It can be used to avoid multiple nested if/else on bool arguments. How to use it: /// Imagine you have template function -/// template return_type foo(...); -/// and bool variables b1, b2, ..., bool bn. To pass these variables as template for foo you can do the following: +/// template return_type foo(...); +/// and bool variables b1, b2, ..., bn. To pass these variables as template for foo you can do the following: /// /// auto call_foo = []() /// { diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 72ea2c8ec42..6392428c5e6 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1180,7 +1180,7 @@ void NO_INLINE Aggregator::executeImplBatch( /// - this affects only optimize_aggregation_in_order, /// - this is just a pointer, so it should not be significant, /// - and plus this will require other changes in the interface. - std::unique_ptr places(new AggregateDataPtr[row_end]); + std::unique_ptr places(new AggregateDataPtr[all_keys_are_const ? 1 : row_end]); /// For all rows. size_t start, end; @@ -1202,7 +1202,7 @@ void NO_INLINE Aggregator::executeImplBatch( if constexpr (!no_more_keys) { - if constexpr (prefetch && HasPrefetchMemberFunc) + if constexpr (prefetch && !all_keys_are_const && HasPrefetchMemberFunc) { if (i == row_begin + prefetching.iterationsToMeasure()) prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); @@ -1267,9 +1267,17 @@ void NO_INLINE Aggregator::executeImplBatch( /// Add only if the key already exists. auto find_result = state.findKey(method.data, i, *aggregates_pool); if (find_result.isFound()) + { aggregate_data = find_result.getMapped(); + } else + { + /// If all keys are constant and this is new key + /// we don't need to do anything and just skip the whole block. + if constexpr (all_keys_are_const) + return; aggregate_data = overflow_row; + } } places[i] = aggregate_data; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e3908c75a58..57e04f547d4 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -965,13 +965,13 @@ public: { if (virtual_column.name == "_path") { - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, current_path)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, current_path)); } else if (virtual_column.name == "_file") { size_t last_slash_pos = current_path.find_last_of('/'); auto file_name = current_path.substr(last_slash_pos + 1); - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_name)->convertToFullColumnIfConst()); + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_name)); } } From 0da8fe2793b88dbc3fd7465644e154b872a75000 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Aug 2023 13:22:36 +0000 Subject: [PATCH 041/327] Add missing constexpr --- src/Interpreters/Aggregator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 6392428c5e6..0d4ebfd1081 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1114,7 +1114,7 @@ void NO_INLINE Aggregator::executeImplBatch( /// For all rows. AggregateDataPtr place = aggregates_pool->alloc(0); - if (all_keys_are_const) + if constexpr (all_keys_are_const) { state.emplaceKey(method.data, 0, *aggregates_pool).setMapped(place); } From 2d4167c61206e256a36e647c878d3f98f40a2724 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 18 Aug 2023 14:50:07 +0000 Subject: [PATCH 042/327] FRemove unneded changes --- src/Storages/StorageFile.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 57e04f547d4..e3908c75a58 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -965,13 +965,13 @@ public: { if (virtual_column.name == "_path") { - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, current_path)); + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, current_path)->convertToFullColumnIfConst()); } else if (virtual_column.name == "_file") { size_t last_slash_pos = current_path.find_last_of('/'); auto file_name = current_path.substr(last_slash_pos + 1); - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_name)); + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_name)->convertToFullColumnIfConst()); } } From cceb2b2d570bf97de5421c17d8825d9df48139da Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 23 Aug 2023 08:12:05 +0000 Subject: [PATCH 043/327] Disable integration with async replication --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1f3cd473e29..9f8fa390be3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4258,7 +4258,7 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) + #write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) logging.debug("Generate and write macros file") macros = self.macros.copy() From ee5d7b1d955b6c2a8d564b1085e0168a0ffbdb14 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 23 Aug 2023 08:22:54 +0000 Subject: [PATCH 044/327] Automatic style fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9f8fa390be3..35a656aa4a4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4258,7 +4258,7 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - #write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) + # write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) logging.debug("Generate and write macros file") macros = self.macros.copy() From 625be86dc0d254447429a8ed4737cca4bb7137e3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 23 Aug 2023 08:59:41 +0000 Subject: [PATCH 045/327] Fix tests --- src/Coordination/tests/gtest_coordination.cpp | 210 +++++++++++++----- utils/keeper-data-dumper/main.cpp | 4 +- 2 files changed, 162 insertions(+), 52 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 08d31bf34f6..759304b40ba 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -298,7 +298,9 @@ TEST_P(CoordinationTest, ChangelogTestSimple) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -328,7 +330,9 @@ TEST_P(CoordinationTest, ChangelogTestFile) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -360,7 +364,9 @@ TEST_P(CoordinationTest, ChangelogReadWrite) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -375,7 +381,9 @@ TEST_P(CoordinationTest, ChangelogReadWrite) waitDurableLogs(changelog); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); @@ -398,7 +406,9 @@ TEST_P(CoordinationTest, ChangelogWriteAt) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -421,7 +431,9 @@ TEST_P(CoordinationTest, ChangelogWriteAt) EXPECT_EQ(changelog.next_slot(), 8); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), changelog.size()); @@ -438,7 +450,9 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 7; ++i) { @@ -455,7 +469,9 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 7); @@ -520,7 +536,9 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 3; ++i) @@ -570,7 +588,9 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(7, 0); EXPECT_EQ(changelog_reader.size(), 1); @@ -586,7 +606,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -602,7 +624,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) auto entries = changelog.pack(1, 5); DB::KeeperLogStore apply_changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); apply_changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -639,7 +663,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) nuraft::ptr entries; { DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -658,7 +684,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) ChangelogDirTest test1("./logs1"); setLogDirectory("./logs1"); DB::KeeperLogStore changelog_new( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog_new.init(1, 0); EXPECT_EQ(changelog_new.size(), 0); @@ -681,7 +709,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) EXPECT_EQ(changelog_new.next_slot(), 11); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(5, 0); } @@ -693,7 +723,9 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -735,7 +767,9 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); DB::KeeperLogStore changelog_read( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 7); EXPECT_EQ(changelog_read.start_index(), 1); @@ -750,7 +784,9 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -792,7 +828,9 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); DB::KeeperLogStore changelog_read( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 11); EXPECT_EQ(changelog_read.start_index(), 1); @@ -807,7 +845,9 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) { @@ -855,7 +895,9 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -877,7 +919,9 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); @@ -924,7 +968,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) setLogDirectory(log_folder); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -949,7 +995,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) plain_buf.truncate(0); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); changelog_reader.end_of_append_batch(0, 0); @@ -983,7 +1031,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) assertBrokenLogRemoved(log_folder, "changelog_31_35.bin" + params.extension); DB::KeeperLogStore changelog_reader2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 11); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -996,7 +1046,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1015,7 +1067,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) plain_buf.truncate(30); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 0); @@ -1031,7 +1085,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); DB::KeeperLogStore changelog_reader2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}, + DB::FlushSettings(), + keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 1); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -1044,7 +1100,9 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1061,7 +1119,9 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) fs::remove("./logs/changelog_1_20.bin" + params.extension); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::FlushSettings(), + keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); assertBrokenLogRemoved("./logs", "changelog_21_40.bin" + params.extension); @@ -1074,7 +1134,9 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1095,7 +1157,9 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) fs::remove("./logs/changelog_21_30.bin" + params.extension); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::FlushSettings(), + keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); @@ -1555,6 +1619,7 @@ void testLogAndStateMachine( DB::KeeperLogStore changelog( DB::LogFileSettings{ .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + DB::FlushSettings(), keeper_context); changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1599,6 +1664,7 @@ void testLogAndStateMachine( DB::KeeperLogStore restore_changelog( DB::LogFileSettings{ .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + DB::FlushSettings(), keeper_context); restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items); @@ -1851,7 +1917,9 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) setLogDirectory("./logs"); { DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1870,7 +1938,9 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); DB::KeeperLogStore changelog_1( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::FlushSettings(), + keeper_context); changelog_1.init(0, 50); for (size_t i = 0; i < 55; ++i) { @@ -1887,7 +1957,9 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); DB::KeeperLogStore changelog_2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}, + DB::FlushSettings(), + keeper_context); changelog_2.init(98, 55); for (size_t i = 0; i < 17; ++i) @@ -1911,7 +1983,9 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + params.extension)); DB::KeeperLogStore changelog_3( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::FlushSettings(), + keeper_context); changelog_3.init(116, 3); for (size_t i = 0; i < 17; ++i) { @@ -1961,7 +2035,9 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) ChangelogDirTest logs("./logs"); setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -1976,7 +2052,9 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog); DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog1.init(0, 3); for (size_t i = 55; i < 70; ++i) { @@ -1990,7 +2068,9 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog1); DB::KeeperLogStore changelog2( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog2.init(0, 3); for (size_t i = 70; i < 80; ++i) { @@ -2056,7 +2136,10 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) setLogDirectory("./logs"); { LOG_INFO(log, "================First time====================="); - DB::KeeperLogStore changelog(DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::KeeperLogStore changelog( + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2068,7 +2151,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Second time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2080,7 +2165,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Third time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2092,7 +2179,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) { LOG_INFO(log, "================Fourth time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2112,7 +2201,9 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) { LOG_INFO(log, "================First time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); for (size_t j = 0; j < 7; ++j) { @@ -2124,7 +2215,9 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) } DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(1, 0); EXPECT_EQ(changelog.next_slot(), 36 * 7 + 1); } @@ -2137,7 +2230,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================First time====================="); DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog1.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog1.append(entry); @@ -2149,7 +2244,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Second time====================="); DB::KeeperLogStore changelog2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog2.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog2.append(entry); @@ -2161,7 +2258,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Third time====================="); DB::KeeperLogStore changelog3( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog3.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog3.append(entry); @@ -2173,7 +2272,9 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) { LOG_INFO(log, "================Fourth time====================="); DB::KeeperLogStore changelog4( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog4.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog4.append(entry); @@ -2235,7 +2336,9 @@ TEST_P(CoordinationTest, TestLogGap) setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2248,7 +2351,9 @@ TEST_P(CoordinationTest, TestLogGap) } DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, keeper_context); + DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::FlushSettings(), + keeper_context); changelog1.init(61, 3); /// Logs discarded @@ -2597,6 +2702,7 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) DB::KeeperLogStore changelog( DB::LogFileSettings{ .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}, + DB::FlushSettings(), keeper_context); changelog.init(1, 0); @@ -2616,6 +2722,7 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) DB::KeeperLogStore changelog( DB::LogFileSettings{ .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + DB::FlushSettings(), keeper_context); changelog.init(1, 0); @@ -2637,6 +2744,7 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) DB::KeeperLogStore changelog( DB::LogFileSettings{ .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + DB::FlushSettings(), keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 9f089a26360..aa8c0efbb26 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -74,7 +74,9 @@ int main(int argc, char *argv[]) LOG_INFO(logger, "Last committed index: {}", last_commited_index); DB::KeeperLogStore changelog( - LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}, keeper_context); + LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}, + FlushSettings(), + keeper_context); changelog.init(last_commited_index, 10000000000UL); /// collect all logs if (changelog.size() == 0) LOG_INFO(logger, "Changelog empty"); From 59ca2374b583efdeaf81e30417b756213850fae8 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 23 Aug 2023 19:39:45 +0000 Subject: [PATCH 046/327] Better --- src/Common/BoolArgsToTemplateArgsDispatcher.h | 39 -------------- src/Interpreters/Aggregator.cpp | 54 +++++++++++-------- src/Interpreters/Aggregator.h | 3 +- .../02845_group_by_constant_keys.reference | 36 +++++++++++++ .../02845_group_by_constant_keys.sql | 32 +++++++++-- 5 files changed, 98 insertions(+), 66 deletions(-) delete mode 100644 src/Common/BoolArgsToTemplateArgsDispatcher.h diff --git a/src/Common/BoolArgsToTemplateArgsDispatcher.h b/src/Common/BoolArgsToTemplateArgsDispatcher.h deleted file mode 100644 index 3e72db49cd2..00000000000 --- a/src/Common/BoolArgsToTemplateArgsDispatcher.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -/// Special struct that helps to convert bool variables to function template bool arguments. -/// It can be used to avoid multiple nested if/else on bool arguments. How to use it: -/// Imagine you have template function -/// template return_type foo(...); -/// and bool variables b1, b2, ..., bn. To pass these variables as template for foo you can do the following: -/// -/// auto call_foo = []() -/// { -/// return foo(...); -/// } -/// -/// BoolArgsToTemplateArgsDispatcher::call(call_foo, b1, b2, ..., bn); - -template -struct BoolArgsToTemplateArgsDispatcher -{ - template - static auto call(Functor f, Args1&&... args) - { - return f.template operator()(std::forward(args)...); - } - - template - static auto call(Functor f, bool b, Args1&&... ar1) - { - if (b) - return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); - else - return BoolArgsToTemplateArgsDispatcher::call(f, std::forward(ar1)...); - } -}; - -} diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d9e9c66e4e2..90f6fc7b978 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include @@ -1066,30 +1065,40 @@ void NO_INLINE Aggregator::executeImpl( { typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - auto call_execute_impl_batch = [&]() - { - executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, overflow_row); - }; - - bool use_compiled_functions = false; -#if USE_EMBEDDED_COMPILER - use_compiled_functions = compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions); -#endif - if (!no_more_keys) { /// Prefetching doesn't make sense for small hash tables, because they fit in caches entirely. const bool prefetch = Method::State::has_cheap_key_calculation && params.enable_prefetch && (method.data.getBufferSizeInBytes() > min_bytes_for_prefetch); - BoolArgsToTemplateArgsDispatcher::call(call_execute_impl_batch, no_more_keys, use_compiled_functions, prefetch, all_keys_are_const); + +#if USE_EMBEDDED_COMPILER + if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions)) + { + if (prefetch) + executeImplBatch( + method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); + else + executeImplBatch( + method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); + } + else +#endif + { + if (prefetch) + executeImplBatch( + method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); + else + executeImplBatch( + method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); + } } else { - BoolArgsToTemplateArgsDispatcher::call(call_execute_impl_batch, no_more_keys, false, false, all_keys_are_const); + executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); } } -template +template void NO_INLINE Aggregator::executeImplBatch( Method & method, typename Method::State & state, @@ -1097,6 +1106,7 @@ void NO_INLINE Aggregator::executeImplBatch( size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions, + bool all_keys_are_const, AggregateDataPtr overflow_row) const { using KeyHolder = decltype(state.getKeyHolder(0, std::declval())); @@ -1113,7 +1123,7 @@ void NO_INLINE Aggregator::executeImplBatch( /// For all rows. AggregateDataPtr place = aggregates_pool->alloc(0); - if constexpr (all_keys_are_const) + if (all_keys_are_const) { state.emplaceKey(method.data, 0, *aggregates_pool).setMapped(place); } @@ -1140,7 +1150,7 @@ void NO_INLINE Aggregator::executeImplBatch( } /// Optimization for special case when aggregating by 8bit key. - if constexpr (!no_more_keys && !all_keys_are_const && std::is_same_v) + if constexpr (!no_more_keys && std::is_same_v) { /// We use another method if there are aggregate functions with -Array combinator. bool has_arrays = false; @@ -1153,7 +1163,7 @@ void NO_INLINE Aggregator::executeImplBatch( } } - if (!has_arrays && !hasSparseArguments(aggregate_instructions)) + if (!has_arrays && !hasSparseArguments(aggregate_instructions) && !all_keys_are_const) { for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { @@ -1184,7 +1194,7 @@ void NO_INLINE Aggregator::executeImplBatch( /// For all rows. size_t start, end; /// If all keys are const, key columns contain only 1 row. - if constexpr (all_keys_are_const) + if (all_keys_are_const) { start = 0; end = 1; @@ -1201,7 +1211,7 @@ void NO_INLINE Aggregator::executeImplBatch( if constexpr (!no_more_keys) { - if constexpr (prefetch && !all_keys_are_const && HasPrefetchMemberFunc) + if constexpr (prefetch && HasPrefetchMemberFunc) { if (i == row_begin + prefetching.iterationsToMeasure()) prefetch_look_ahead = prefetching.calcPrefetchLookAhead(); @@ -1273,7 +1283,7 @@ void NO_INLINE Aggregator::executeImplBatch( { /// If all keys are constant and this is new key /// we don't need to do anything and just skip the whole block. - if constexpr (all_keys_are_const) + if (all_keys_are_const) return; aggregate_data = overflow_row; } @@ -1299,7 +1309,7 @@ void NO_INLINE Aggregator::executeImplBatch( columns_data.emplace_back(getColumnData(inst->batch_arguments[argument_index])); } - if constexpr (all_keys_are_const) + if (all_keys_are_const) { auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place; add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), places[0]); @@ -1323,7 +1333,7 @@ void NO_INLINE Aggregator::executeImplBatch( AggregateFunctionInstruction * inst = aggregate_instructions + i; - if constexpr (all_keys_are_const) + if (all_keys_are_const) { if (inst->offsets) inst->batch_that->addBatchSinglePlace(inst->offsets[static_cast(row_begin) - 1], inst->offsets[row_end - 1], places[0] + inst->state_offset, inst->batch_arguments, aggregates_pool); diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index dc0391c4289..d5b5abddd84 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1297,7 +1297,7 @@ private: AggregateDataPtr overflow_row) const; /// Specialization for a particular value no_more_keys. - template + template void executeImplBatch( Method & method, typename Method::State & state, @@ -1305,6 +1305,7 @@ private: size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions, + bool all_keys_are_const, AggregateDataPtr overflow_row) const; /// For case when there are no keys (all aggregate into one row). diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.reference b/tests/queries/0_stateless/02845_group_by_constant_keys.reference index 60e40ea54a7..67cbdf0c025 100644 --- a/tests/queries/0_stateless/02845_group_by_constant_keys.reference +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.reference @@ -2,3 +2,39 @@ 10000000 1 2 3 10000000 1 2 3 10000000 1 2 3 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 data.Parquet +10 data.2.Parquet +10 data.1.Parquet +10 data.Parquet +10 data.2.Parquet +10 data.1.Parquet +10 data.Parquet +10 data.2.Parquet +10 data.1.Parquet +10 data.Parquet +10 data.2.Parquet +10 data.1.Parquet +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.sql b/tests/queries/0_stateless/02845_group_by_constant_keys.sql index 0223cf1df60..109a5a9a730 100644 --- a/tests/queries/0_stateless/02845_group_by_constant_keys.sql +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.sql @@ -1,5 +1,29 @@ -SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; -SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 0; -SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions = 1; -SELECT count(number), 1 AS k1, 2 as k2, 3 as k3 FROM numbers_mt(10000000) GROUP BY k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 1; +select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group by k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group by k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 0; +select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group by k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions = 1; +select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group by k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 1; + +drop table if exists test; +create table test (x UInt64) engine=File(Parquet); +set engine_file_allow_create_multiple_files = 1; +insert into test select * from numbers(10); +insert into test select * from numbers(10); +insert into test select * from numbers(10); + +select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; + +select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; + +select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; + +drop table test; From 27fb1b5ced804a37fee6c9fd110e029bb6cc01c5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 23 Aug 2023 22:18:30 +0200 Subject: [PATCH 047/327] Fix test --- tests/queries/0_stateless/02845_group_by_constant_keys.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.sql b/tests/queries/0_stateless/02845_group_by_constant_keys.sql index 109a5a9a730..e16fd3fd1d2 100644 --- a/tests/queries/0_stateless/02845_group_by_constant_keys.sql +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.sql @@ -4,7 +4,7 @@ select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group select count(number), 1 AS k1, 2 as k2, 3 as k3 from numbers_mt(10000000) group by k1, k2, k3 settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions = 1; drop table if exists test; -create table test (x UInt64) engine=File(Parquet); +create table test (x UInt64) engine=File(JSON); set engine_file_allow_create_multiple_files = 1; insert into test select * from numbers(10); insert into test select * from numbers(10); From 734ffd916c6786ffd27479cace1298cab8e4cda7 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 25 Aug 2023 20:31:21 +0200 Subject: [PATCH 048/327] WIP prepared statements --- src/Core/MySQL/PacketsGeneric.cpp | 475 +++++++++--------- src/Core/MySQL/PacketsProtocolBinary.cpp | 230 +++++++++ src/Core/MySQL/PacketsProtocolBinary.h | 45 ++ src/Core/MySQL/PacketsProtocolText.cpp | 388 +++++++------- src/DataTypes/DataTypesDecimal.cpp | 26 +- src/Formats/FormatSettings.h | 16 +- .../InterpreterShowColumnsQuery.cpp | 2 +- .../Formats/Impl/MySQLOutputFormat.cpp | 95 ++-- .../Formats/Impl/MySQLOutputFormat.h | 3 +- src/Server/MySQLHandler.cpp | 142 ++++-- src/Server/MySQLHandler.h | 32 +- .../InformationSchema/key_column_usage.sql | 38 ++ .../referential_constraints.sql | 25 + .../System/InformationSchema/schemata.sql | 55 +- .../System/InformationSchema/tables.sql | 50 +- .../System/attachInformationSchemaTables.cpp | 5 +- 16 files changed, 1051 insertions(+), 576 deletions(-) create mode 100644 src/Core/MySQL/PacketsProtocolBinary.cpp create mode 100644 src/Core/MySQL/PacketsProtocolBinary.h create mode 100644 src/Storages/System/InformationSchema/key_column_usage.sql create mode 100644 src/Storages/System/InformationSchema/referential_constraints.sql diff --git a/src/Core/MySQL/PacketsGeneric.cpp b/src/Core/MySQL/PacketsGeneric.cpp index af80797d5c1..88183890399 100644 --- a/src/Core/MySQL/PacketsGeneric.cpp +++ b/src/Core/MySQL/PacketsGeneric.cpp @@ -8,254 +8,263 @@ namespace DB namespace MySQLProtocol { -namespace Generic -{ - -static const size_t MYSQL_ERRMSG_SIZE = 512; - -void SSLRequest::readPayloadImpl(ReadBuffer & buf) -{ - buf.readStrict(reinterpret_cast(&capability_flags), 4); - buf.readStrict(reinterpret_cast(&max_packet_size), 4); - buf.readStrict(reinterpret_cast(&character_set), 1); -} - -OKPacket::OKPacket(uint32_t capabilities_) - : header(0x00), capabilities(capabilities_), affected_rows(0x00), last_insert_id(0x00), status_flags(0x00) -{ -} - -OKPacket::OKPacket( - uint8_t header_, uint32_t capabilities_, uint64_t affected_rows_, uint32_t status_flags_, int16_t warnings_, - String session_state_changes_, String info_) - : header(header_), capabilities(capabilities_), affected_rows(affected_rows_), last_insert_id(0), warnings(warnings_), - status_flags(status_flags_), session_state_changes(std::move(session_state_changes_)), info(std::move(info_)) -{ -} - -size_t OKPacket::getPayloadSize() const -{ - size_t result = 2 + getLengthEncodedNumberSize(affected_rows); - - if (capabilities & CLIENT_PROTOCOL_41) + namespace Generic { - result += 4; - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - result += 2; - } - if (capabilities & CLIENT_SESSION_TRACK) - { - result += getLengthEncodedStringSize(info); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - result += getLengthEncodedStringSize(session_state_changes); - } - else - { - result += info.size(); - } + static const size_t MYSQL_ERRMSG_SIZE = 512; - return result; -} - -void OKPacket::readPayloadImpl(ReadBuffer & payload) - -{ - payload.readStrict(reinterpret_cast(&header), 1); - affected_rows = readLengthEncodedNumber(payload); - last_insert_id = readLengthEncodedNumber(payload); - - if (capabilities & CLIENT_PROTOCOL_41) - { - payload.readStrict(reinterpret_cast(&status_flags), 2); - payload.readStrict(reinterpret_cast(&warnings), 2); - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - payload.readStrict(reinterpret_cast(&status_flags), 2); - } - - if (capabilities & CLIENT_SESSION_TRACK) - { - readLengthEncodedString(info, payload); - if (status_flags & SERVER_SESSION_STATE_CHANGED) + void SSLRequest::readPayloadImpl(ReadBuffer & buf) { - readLengthEncodedString(session_state_changes, payload); + buf.readStrict(reinterpret_cast(&capability_flags), 4); + buf.readStrict(reinterpret_cast(&max_packet_size), 4); + buf.readStrict(reinterpret_cast(&character_set), 1); } - } - else - { - readString(info, payload); - } -} -void OKPacket::writePayloadImpl(WriteBuffer & buffer) const + OKPacket::OKPacket(uint32_t capabilities_) + : header(0x00), capabilities(capabilities_), affected_rows(0x00), last_insert_id(0x00), status_flags(0x00) + { + } -{ - buffer.write(header); - writeLengthEncodedNumber(affected_rows, buffer); - writeLengthEncodedNumber(last_insert_id, buffer); /// last insert-id + OKPacket::OKPacket( + uint8_t header_, + uint32_t capabilities_, + uint64_t affected_rows_, + uint32_t status_flags_, + int16_t warnings_, + String session_state_changes_, + String info_) + : header(header_) + , capabilities(capabilities_) + , affected_rows(affected_rows_) + , last_insert_id(0) + , warnings(warnings_) + , status_flags(status_flags_) + , session_state_changes(std::move(session_state_changes_)) + , info(std::move(info_)) + { + } - if (capabilities & CLIENT_PROTOCOL_41) - { - buffer.write(reinterpret_cast(&status_flags), 2); - buffer.write(reinterpret_cast(&warnings), 2); - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - buffer.write(reinterpret_cast(&status_flags), 2); - } + size_t OKPacket::getPayloadSize() const + { + size_t result = 2 + getLengthEncodedNumberSize(affected_rows); - if (capabilities & CLIENT_SESSION_TRACK) - { - writeLengthEncodedString(info, buffer); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - writeLengthEncodedString(session_state_changes, buffer); - } - else - { - writeString(info, buffer); - } -} - -EOFPacket::EOFPacket() : warnings(0x00), status_flags(0x00) -{ -} - -EOFPacket::EOFPacket(int warnings_, int status_flags_) - : warnings(warnings_), status_flags(status_flags_) -{ -} - -size_t EOFPacket::getPayloadSize() const -{ - return 5; -} - -void EOFPacket::readPayloadImpl(ReadBuffer & payload) -{ - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xfe); - payload.readStrict(reinterpret_cast(&warnings), 2); - payload.readStrict(reinterpret_cast(&status_flags), 2); -} - -void EOFPacket::writePayloadImpl(WriteBuffer & buffer) const -{ - buffer.write(header); // EOF header - buffer.write(reinterpret_cast(&warnings), 2); - buffer.write(reinterpret_cast(&status_flags), 2); -} - -void AuthSwitchPacket::readPayloadImpl(ReadBuffer & payload) -{ - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xfe); - readStringUntilEOF(plugin_name, payload); -} - -ERRPacket::ERRPacket() : error_code(0x00) -{ -} - -ERRPacket::ERRPacket(int error_code_, String sql_state_, String error_message_) - : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) -{ -} - -size_t ERRPacket::getPayloadSize() const -{ - return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); -} - -void ERRPacket::readPayloadImpl(ReadBuffer & payload) -{ - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xff); - - payload.readStrict(reinterpret_cast(&error_code), 2); - - /// SQL State [optional: # + 5bytes string] - UInt8 sharp = static_cast(*payload.position()); - if (sharp == 0x23) - { - payload.ignore(1); - sql_state.resize(5); - payload.readStrict(reinterpret_cast(sql_state.data()), 5); - } - readString(error_message, payload); -} - -void ERRPacket::writePayloadImpl(WriteBuffer & buffer) const -{ - buffer.write(header); - buffer.write(reinterpret_cast(&error_code), 2); - buffer.write('#'); - buffer.write(sql_state.data(), sql_state.length()); - buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); -} - -ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) - : ok(OKPacket(server_capability_flags_)) -{ -} - -ResponsePacket::ResponsePacket(UInt32 server_capability_flags_, bool is_handshake_) - : ok(OKPacket(server_capability_flags_)), is_handshake(is_handshake_) -{ -} - -void ResponsePacket::readPayloadImpl(ReadBuffer & payload) -{ - UInt16 header = static_cast(*payload.position()); - switch (header) - { - case PACKET_OK: - packetType = PACKET_OK; - ok.readPayloadWithUnpacked(payload); - break; - case PACKET_ERR: - packetType = PACKET_ERR; - err.readPayloadWithUnpacked(payload); - break; - case PACKET_EOF: - if (is_handshake) + if (capabilities & CLIENT_PROTOCOL_41) { - packetType = PACKET_AUTH_SWITCH; - auth_switch.readPayloadWithUnpacked(payload); + result += 4; + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + result += 2; + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + result += getLengthEncodedStringSize(info); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + result += getLengthEncodedStringSize(session_state_changes); } else { - packetType = PACKET_EOF; - eof.readPayloadWithUnpacked(payload); + result += info.size(); } - break; - case PACKET_LOCALINFILE: - packetType = PACKET_LOCALINFILE; - break; - default: - packetType = PACKET_OK; - column_length = readLengthEncodedNumber(payload); + + return result; + } + + void OKPacket::readPayloadImpl(ReadBuffer & payload) + + { + payload.readStrict(reinterpret_cast(&header), 1); + affected_rows = readLengthEncodedNumber(payload); + last_insert_id = readLengthEncodedNumber(payload); + + if (capabilities & CLIENT_PROTOCOL_41) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + payload.readStrict(reinterpret_cast(&warnings), 2); + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + readLengthEncodedString(info, payload); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + { + readLengthEncodedString(session_state_changes, payload); + } + } + else + { + readString(info, payload); + } + } + + void OKPacket::writePayloadImpl(WriteBuffer & buffer) const + + { + buffer.write(header); + writeLengthEncodedNumber(affected_rows, buffer); + writeLengthEncodedNumber(last_insert_id, buffer); /// last insert-id + + if (capabilities & CLIENT_PROTOCOL_41) + { + buffer.write(reinterpret_cast(&status_flags), 2); + buffer.write(reinterpret_cast(&warnings), 2); + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + buffer.write(reinterpret_cast(&status_flags), 2); + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + writeLengthEncodedString(info, buffer); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + writeLengthEncodedString(session_state_changes, buffer); + } + else + { + writeString(info, buffer); + } + } + + EOFPacket::EOFPacket() : warnings(0x00), status_flags(0x00) + { + } + + EOFPacket::EOFPacket(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) + { + } + + size_t EOFPacket::getPayloadSize() const + { + return 5; + } + + void EOFPacket::readPayloadImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + payload.readStrict(reinterpret_cast(&warnings), 2); + payload.readStrict(reinterpret_cast(&status_flags), 2); + } + + void EOFPacket::writePayloadImpl(WriteBuffer & buffer) const + { + buffer.write(header); // EOF header + buffer.write(reinterpret_cast(&warnings), 2); + buffer.write(reinterpret_cast(&status_flags), 2); + } + + void AuthSwitchPacket::readPayloadImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + readStringUntilEOF(plugin_name, payload); + } + + ERRPacket::ERRPacket() : error_code(0x00) + { + } + + ERRPacket::ERRPacket(int error_code_, String sql_state_, String error_message_) + : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) + { + } + + size_t ERRPacket::getPayloadSize() const + { + return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); + } + + void ERRPacket::readPayloadImpl(ReadBuffer & payload) + { + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xff); + + payload.readStrict(reinterpret_cast(&error_code), 2); + + /// SQL State [optional: # + 5bytes string] + UInt8 sharp = static_cast(*payload.position()); + if (sharp == 0x23) + { + payload.ignore(1); + sql_state.resize(5); + payload.readStrict(reinterpret_cast(sql_state.data()), 5); + } + readString(error_message, payload); + } + + void ERRPacket::writePayloadImpl(WriteBuffer & buffer) const + { + buffer.write(header); + buffer.write(reinterpret_cast(&error_code), 2); + buffer.write('#'); + buffer.write(sql_state.data(), sql_state.length()); + buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); + } + + ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) : ok(OKPacket(server_capability_flags_)) + { + } + + ResponsePacket::ResponsePacket(UInt32 server_capability_flags_, bool is_handshake_) + : ok(OKPacket(server_capability_flags_)), is_handshake(is_handshake_) + { + } + + void ResponsePacket::readPayloadImpl(ReadBuffer & payload) + { + UInt16 header = static_cast(*payload.position()); + switch (header) + { + case PACKET_OK: + packetType = PACKET_OK; + ok.readPayloadWithUnpacked(payload); + break; + case PACKET_ERR: + packetType = PACKET_ERR; + err.readPayloadWithUnpacked(payload); + break; + case PACKET_EOF: + if (is_handshake) + { + packetType = PACKET_AUTH_SWITCH; + auth_switch.readPayloadWithUnpacked(payload); + } + else + { + packetType = PACKET_EOF; + eof.readPayloadWithUnpacked(payload); + } + break; + case PACKET_LOCALINFILE: + packetType = PACKET_LOCALINFILE; + break; + default: + packetType = PACKET_OK; + column_length = readLengthEncodedNumber(payload); + } + } + + LengthEncodedNumber::LengthEncodedNumber(uint64_t value_) : value(value_) + { + } + + size_t LengthEncodedNumber::getPayloadSize() const + { + return getLengthEncodedNumberSize(value); + } + + void LengthEncodedNumber::writePayloadImpl(WriteBuffer & buffer) const + { + writeLengthEncodedNumber(value, buffer); + } + } -} - -LengthEncodedNumber::LengthEncodedNumber(uint64_t value_) : value(value_) -{ -} - -size_t LengthEncodedNumber::getPayloadSize() const -{ - return getLengthEncodedNumberSize(value); -} - -void LengthEncodedNumber::writePayloadImpl(WriteBuffer & buffer) const -{ - writeLengthEncodedNumber(value, buffer); -} - -} } diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp new file mode 100644 index 00000000000..287dda269e6 --- /dev/null +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -0,0 +1,230 @@ +#include +#include +#include +#include +#include +#include +#include "Columns/ColumnLowCardinality.h" +#include "Columns/ColumnVector.h" +#include "DataTypes/DataTypeLowCardinality.h" +#include "DataTypes/DataTypeNullable.h" +#include "Formats/FormatSettings.h" +#include "IO/WriteBufferFromString.h" +#include "base/types.h" + +namespace DB +{ + +namespace MySQLProtocol +{ + + namespace ProtocolBinary + { + ResultSetRow::ResultSetRow( + const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_) + : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) + { + /// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row + payload_size = 1 + null_bitmap_size; + // LOG_TRACE(&Poco::Logger::get("ResultSetRow"), "Null bitmap size: {}", null_bitmap_size); + FormatSettings format_settings; + for (size_t i = 0; i < columns.size(); ++i) + { + ColumnPtr col = columns[i]; + + if (col->isNullAt(row_num)) + { + null_bitmap[i / 8] |= 1 << i % 8; + } + + TypeIndex type_index = removeNullable(removeLowCardinality(data_types[i]))->getTypeId(); + switch (type_index) + { + case TypeIndex::Int8: + case TypeIndex::UInt8: + payload_size += 1; + break; + case TypeIndex::Int16: + case TypeIndex::UInt16: + payload_size += 2; + break; + case TypeIndex::Int32: + case TypeIndex::UInt32: + case TypeIndex::Float32: + payload_size += 4; + break; + case TypeIndex::Int64: + case TypeIndex::UInt64: + case TypeIndex::Float64: + payload_size += 8; + break; + case TypeIndex::Date: { + UInt64 value = col->get64(row_num); + if (value == 0) + { + payload_size += 1; // length only, no other fields + } + else + { + payload_size += 5; + } + break; + } + case TypeIndex::DateTime: { + UInt64 value = col->get64(row_num); + if (value == 0) + { + payload_size += 1; // length only, no other fields + } + else + { + Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); + if (dt.second() == 0 && dt.minute() == 0 && dt.hour() == 0) + { + payload_size += 5; + } + else + { + payload_size += 8; + } + } + break; + } + default: + WriteBufferFromOwnString ostr; + serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); + payload_size += getLengthEncodedStringSize(ostr.str()); + serialized[i] = std::move(ostr.str()); + break; + } + } + } + + void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const + { + buffer.write(static_cast(0x00)); + buffer.write(null_bitmap.data(), null_bitmap_size); + for (size_t i = 0; i < columns.size(); ++i) + { + ColumnPtr col = columns[i]; + if (col->isNullAt(row_num)) + { + continue; // NULLs are stored in the null bitmap only + } + + TypeIndex type_index = removeNullable(removeLowCardinality(data_types[i]))->getTypeId(); + switch (type_index) + { + case TypeIndex::UInt8: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 1); + break; + } + case TypeIndex::UInt16: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 2); + break; + } + case TypeIndex::UInt32: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::UInt64: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::Int8: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 1); + break; + } + case TypeIndex::Int16: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 2); + break; + } + case TypeIndex::Int32: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::Int64: { + UInt64 value = col->get64(row_num); + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::Float32: { + Float32 value = col->getFloat32(row_num); + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::Float64: { + Float64 value = col->getFloat64(row_num); + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::Date: { + UInt64 value = col->get64(row_num); + if (value != 0) + { + Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); + buffer.write(static_cast(4)); // bytes_following + int year = dt.year(); + int month = dt.month(); + int day = dt.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + } + else + { + buffer.write(static_cast(0)); + } + break; + } + case TypeIndex::DateTime: { + UInt64 value = col->get64(row_num); + if (value != 0) + { + Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); + bool is_date_time = !(dt.hour() == 0 && dt.minute() == 0 && dt.second() == 0); + size_t bytes_following = is_date_time ? 7 : 4; + buffer.write(reinterpret_cast(&bytes_following), 1); + int year = dt.year(); + int month = dt.month(); + int day = dt.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + if (is_date_time) + { + int hour = dt.hourAMPM(); + int minute = dt.minute(); + int second = dt.second(); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + } + } + else + { + buffer.write(static_cast(0)); + } + break; + } + default: + writeLengthEncodedString(serialized[i], buffer); + break; + } + } + } + + size_t ResultSetRow::getPayloadSize() const + { + return payload_size; + }; + } +} +} diff --git a/src/Core/MySQL/PacketsProtocolBinary.h b/src/Core/MySQL/PacketsProtocolBinary.h new file mode 100644 index 00000000000..69936e527c1 --- /dev/null +++ b/src/Core/MySQL/PacketsProtocolBinary.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include +#include "DataTypes/IDataType.h" +#include "DataTypes/Serializations/ISerialization.h" + +namespace DB +{ + +namespace MySQLProtocol +{ + + namespace ProtocolBinary + { + class ResultSetRow : public IMySQLWritePacket + { + private: + TypeIndex getTypeIndex(DataTypePtr data_type, const ColumnPtr & col) const; + + protected: + int row_num; + const Columns & columns; + const DataTypes & data_types; + const Serializations & serializations; + + std::vector serialized = std::vector(columns.size()); + + size_t null_bitmap_size = (columns.size() + 7) / 8; + std::vector null_bitmap = std::vector(null_bitmap_size, 0); + + size_t payload_size = 0; + + size_t getPayloadSize() const override; + + void writePayloadImpl(WriteBuffer & buffer) const override; + + public: + ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_); + }; + } +} +} diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 728e8061e87..9c5bf6b6e05 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -1,7 +1,8 @@ #include -#include #include +#include #include +#include "Core/MySQL/IMySQLWritePacket.h" namespace DB { @@ -9,197 +10,212 @@ namespace DB namespace MySQLProtocol { -namespace ProtocolText -{ - -ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) - : columns(columns_), row_num(row_num_) -{ - for (size_t i = 0; i < columns.size(); ++i) + namespace ProtocolText { - if (columns[i]->isNullAt(row_num)) + + ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) + : columns(columns_), row_num(row_num_) { - payload_size += 1; - serialized.emplace_back("\xfb"); + for (size_t i = 0; i < columns.size(); ++i) + { + if (columns[i]->isNullAt(row_num)) + { + payload_size += 1; + serialized.emplace_back("\xfb"); + } + else + { + WriteBufferFromOwnString ostr; + serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings()); + payload_size += getLengthEncodedStringSize(ostr.str()); + serialized.push_back(std::move(ostr.str())); + } + } } - else + + size_t ResultSetRow::getPayloadSize() const { - WriteBufferFromOwnString ostr; - serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings()); - payload_size += getLengthEncodedStringSize(ostr.str()); - serialized.push_back(std::move(ostr.str())); + return payload_size; } + + void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const + { + for (size_t i = 0; i < columns.size(); ++i) + { + if (columns[i]->isNullAt(row_num)) + buffer.write(serialized[i].data(), 1); + else + writeLengthEncodedString(serialized[i], buffer); + } + } + + void ComFieldList::readPayloadImpl(ReadBuffer & payload) + { + // Command byte has been already read from payload. + readNullTerminated(table, payload); + readStringUntilEOF(field_wildcard, payload); + } + + ColumnDefinition::ColumnDefinition() : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) + { + } + + ColumnDefinition::ColumnDefinition( + String schema_, + String table_, + String org_table_, + String name_, + String org_name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_, + bool with_defaults_) + : schema(std::move(schema_)) + , table(std::move(table_)) + , org_table(std::move(org_table_)) + , name(std::move(name_)) + , org_name(std::move(org_name_)) + , character_set(character_set_) + , column_length(column_length_) + , column_type(column_type_) + , flags(flags_) + , decimals(decimals_) + , is_comm_field_list_response(with_defaults_) + { + } + + ColumnDefinition::ColumnDefinition( + String name_, uint16_t character_set_, uint32_t column_length_, ColumnType column_type_, uint16_t flags_, uint8_t decimals_) + : ColumnDefinition("", "", "", std::move(name_), "", character_set_, column_length_, column_type_, flags_, decimals_) + { + } + + size_t ColumnDefinition::getPayloadSize() const + { + return 12 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) + + getLengthEncodedStringSize(org_table) + getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) + + getLengthEncodedNumberSize(next_length) + is_comm_field_list_response; + } + + void ColumnDefinition::readPayloadImpl(ReadBuffer & payload) + { + String def; + readLengthEncodedString(def, payload); + assert(def == "def"); + readLengthEncodedString(schema, payload); + readLengthEncodedString(table, payload); + readLengthEncodedString(org_table, payload); + readLengthEncodedString(name, payload); + readLengthEncodedString(org_name, payload); + next_length = readLengthEncodedNumber(payload); + payload.readStrict(reinterpret_cast(&character_set), 2); + payload.readStrict(reinterpret_cast(&column_length), 4); + payload.readStrict(reinterpret_cast(&column_type), 1); + payload.readStrict(reinterpret_cast(&flags), 2); + payload.readStrict(reinterpret_cast(&decimals), 1); + payload.ignore(2); + } + + void ColumnDefinition::writePayloadImpl(WriteBuffer & buffer) const + { + writeLengthEncodedString(std::string("def"), buffer); /// always "def" + writeLengthEncodedString(schema, buffer); + writeLengthEncodedString(table, buffer); + writeLengthEncodedString(org_table, buffer); + writeLengthEncodedString(name, buffer); + writeLengthEncodedString(org_name, buffer); + writeLengthEncodedNumber(next_length, buffer); + buffer.write(reinterpret_cast(&character_set), 2); + buffer.write(reinterpret_cast(&column_length), 4); + buffer.write(reinterpret_cast(&column_type), 1); + buffer.write(reinterpret_cast(&flags), 2); + buffer.write(reinterpret_cast(&decimals), 1); + writeChar(0x0, 2, buffer); + if (is_comm_field_list_response) + { + /// We should write length encoded int with string size + /// followed by string with some "default values" (possibly it's column defaults). + /// But we just send NULL for simplicity. + writeChar(0xfb, buffer); + } + } + + ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex type_index) + { + ColumnType column_type; + CharacterSet charset = CharacterSet::binary; + int flags = 0; + uint8_t decimals = 0; + switch (type_index) + { + case TypeIndex::UInt8: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::Int8: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Float32: + column_type = ColumnType::MYSQL_TYPE_FLOAT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + decimals = 31; + break; + case TypeIndex::Float64: + column_type = ColumnType::MYSQL_TYPE_DOUBLE; + flags = ColumnDefinitionFlags::BINARY_FLAG; + decimals = 31; + break; + case TypeIndex::Date: + column_type = ColumnType::MYSQL_TYPE_DATE; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::DateTime: + column_type = ColumnType::MYSQL_TYPE_DATETIME; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Decimal32: + case TypeIndex::Decimal64: + case TypeIndex::Decimal128: + /// MySQL Decimal has max 65 precision and 30 scale. Thus, Decimal256 is reported as a string + column_type = ColumnType::MYSQL_TYPE_DECIMAL; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + default: + column_type = ColumnType::MYSQL_TYPE_STRING; + charset = CharacterSet::utf8_general_ci; + break; + } + return ColumnDefinition(column_name, charset, 0, column_type, flags, decimals); + } + } -} - -size_t ResultSetRow::getPayloadSize() const -{ - return payload_size; -} - -void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const -{ - for (size_t i = 0; i < columns.size(); ++i) - { - if (columns[i]->isNullAt(row_num)) - buffer.write(serialized[i].data(), 1); - else - writeLengthEncodedString(serialized[i], buffer); - } -} - -void ComFieldList::readPayloadImpl(ReadBuffer & payload) -{ - // Command byte has been already read from payload. - readNullTerminated(table, payload); - readStringUntilEOF(field_wildcard, payload); -} - -ColumnDefinition::ColumnDefinition() - : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) -{ -} - -ColumnDefinition::ColumnDefinition( - String schema_, String table_, String org_table_, String name_, String org_name_, uint16_t character_set_, uint32_t column_length_, - ColumnType column_type_, uint16_t flags_, uint8_t decimals_, bool with_defaults_) - : schema(std::move(schema_)), table(std::move(table_)), org_table(std::move(org_table_)), name(std::move(name_)), - org_name(std::move(org_name_)), character_set(character_set_), column_length(column_length_), column_type(column_type_), - flags(flags_), decimals(decimals_), is_comm_field_list_response(with_defaults_) -{ -} - -ColumnDefinition::ColumnDefinition( - String name_, uint16_t character_set_, uint32_t column_length_, ColumnType column_type_, uint16_t flags_, uint8_t decimals_) - : ColumnDefinition("", "", "", std::move(name_), "", character_set_, column_length_, column_type_, flags_, decimals_) -{ -} - -size_t ColumnDefinition::getPayloadSize() const -{ - return 12 + - getLengthEncodedStringSize("def") + - getLengthEncodedStringSize(schema) + - getLengthEncodedStringSize(table) + - getLengthEncodedStringSize(org_table) + - getLengthEncodedStringSize(name) + - getLengthEncodedStringSize(org_name) + - getLengthEncodedNumberSize(next_length) + - is_comm_field_list_response; -} - -void ColumnDefinition::readPayloadImpl(ReadBuffer & payload) -{ - String def; - readLengthEncodedString(def, payload); - assert(def == "def"); - readLengthEncodedString(schema, payload); - readLengthEncodedString(table, payload); - readLengthEncodedString(org_table, payload); - readLengthEncodedString(name, payload); - readLengthEncodedString(org_name, payload); - next_length = readLengthEncodedNumber(payload); - payload.readStrict(reinterpret_cast(&character_set), 2); - payload.readStrict(reinterpret_cast(&column_length), 4); - payload.readStrict(reinterpret_cast(&column_type), 1); - payload.readStrict(reinterpret_cast(&flags), 2); - payload.readStrict(reinterpret_cast(&decimals), 1); - payload.ignore(2); -} - -void ColumnDefinition::writePayloadImpl(WriteBuffer & buffer) const -{ - writeLengthEncodedString(std::string("def"), buffer); /// always "def" - writeLengthEncodedString(schema, buffer); - writeLengthEncodedString(table, buffer); - writeLengthEncodedString(org_table, buffer); - writeLengthEncodedString(name, buffer); - writeLengthEncodedString(org_name, buffer); - writeLengthEncodedNumber(next_length, buffer); - buffer.write(reinterpret_cast(&character_set), 2); - buffer.write(reinterpret_cast(&column_length), 4); - buffer.write(reinterpret_cast(&column_type), 1); - buffer.write(reinterpret_cast(&flags), 2); - buffer.write(reinterpret_cast(&decimals), 1); - writeChar(0x0, 2, buffer); - if (is_comm_field_list_response) - { - /// We should write length encoded int with string size - /// followed by string with some "default values" (possibly it's column defaults). - /// But we just send NULL for simplicity. - writeChar(0xfb, buffer); - } -} - -ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex type_index) -{ - ColumnType column_type; - CharacterSet charset = CharacterSet::binary; - int flags = 0; - switch (type_index) - { - case TypeIndex::UInt8: - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::UInt16: - column_type = ColumnType::MYSQL_TYPE_SHORT; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::UInt32: - column_type = ColumnType::MYSQL_TYPE_LONG; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::UInt64: - column_type = ColumnType::MYSQL_TYPE_LONGLONG; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::Int8: - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int16: - column_type = ColumnType::MYSQL_TYPE_SHORT; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int32: - column_type = ColumnType::MYSQL_TYPE_LONG; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int64: - column_type = ColumnType::MYSQL_TYPE_LONGLONG; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Float32: - column_type = ColumnType::MYSQL_TYPE_FLOAT; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Float64: - column_type = ColumnType::MYSQL_TYPE_DOUBLE; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Date: - column_type = ColumnType::MYSQL_TYPE_DATE; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::DateTime: - column_type = ColumnType::MYSQL_TYPE_DATETIME; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::String: - case TypeIndex::FixedString: - column_type = ColumnType::MYSQL_TYPE_STRING; - charset = CharacterSet::utf8_general_ci; - break; - default: - column_type = ColumnType::MYSQL_TYPE_STRING; - charset = CharacterSet::utf8_general_ci; - break; - } - return ColumnDefinition(column_name, charset, 0, column_type, flags, 0); -} - -} } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index fa044d4ac9c..2af216529e5 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -1,13 +1,13 @@ #include #include -#include #include #include #include #include #include #include +#include #include @@ -31,6 +31,12 @@ std::string DataTypeDecimal::doGetName() const template std::string DataTypeDecimal::getSQLCompatibleName() const { + /// See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html + /// DECIMAL(M,D) + /// M is the maximum number of digits (the precision). It has a range of 1 to 65. + /// D is the number of digits to the right of the decimal point (the scale). It has a range of 0 to 30 and must be no larger than M. + if (this->precision > 65 || this->scale > 30) + return "TEXT"; return fmt::format("DECIMAL({}, {})", this->precision, this->scale); } @@ -75,14 +81,14 @@ SerializationPtr DataTypeDecimal::doGetDefaultSerialization() const static DataTypePtr create(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Decimal data type family must have exactly two arguments: precision and scale"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Decimal data type family must have exactly two arguments: precision and scale"); const auto * precision = arguments->children[0]->as(); const auto * scale = arguments->children[1]->as(); - if (!precision || precision->value.getType() != Field::Types::UInt64 || - !scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) + if (!precision || precision->value.getType() != Field::Types::UInt64 || !scale + || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal data type family must have two numbers as its arguments"); UInt64 precision_value = precision->value.get(); @@ -95,13 +101,15 @@ template static DataTypePtr createExact(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have exactly one arguments: scale"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have exactly one arguments: scale"); const auto * scale_arg = arguments->children[0]->as(); if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); UInt64 precision = DecimalUtils::max_precision; UInt64 scale = scale_arg->value.get(); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 2c283dcc2b7..56359fd0bea 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include #include @@ -48,9 +48,9 @@ struct FormatSettings enum class DateTimeInputFormat { - Basic, /// Default format for fast parsing: YYYY-MM-DD hh:mm:ss (ISO-8601 without fractional part and timezone) or NNNNNNNNNN unix timestamp. - BestEffort, /// Use sophisticated rules to parse whatever possible. - BestEffortUS /// Use sophisticated rules to parse American style: mm/dd/yyyy + Basic, /// Default format for fast parsing: YYYY-MM-DD hh:mm:ss (ISO-8601 without fractional part and timezone) or NNNNNNNNNN unix timestamp. + BestEffort, /// Use sophisticated rules to parse whatever possible. + BestEffortUS /// Use sophisticated rules to parse American style: mm/dd/yyyy }; DateTimeInputFormat date_time_input_format = DateTimeInputFormat::Basic; @@ -282,6 +282,14 @@ struct FormatSettings uint32_t client_capabilities = 0; size_t max_packet_size = 0; uint8_t * sequence_id = nullptr; /// Not null if it's MySQLWire output format used to handle MySQL protocol connections. + /** + * COM_QUERY uses Text ResultSet + * https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_query_response_text_resultset.html + * COM_STMT_EXECUTE uses Binary Protocol ResultSet + * https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_execute_response.html + * By default, use Text ResultSet. + */ + bool use_binary_result_set = false; } mysql_wire; struct diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index c86d3c753c4..922f9887a82 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -37,7 +37,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() SELECT name AS field, type AS type, - startsWith(type, 'Nullable') AS null, + if(startsWith(type, 'Nullable'), 'YES', 'NO') AS null, trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index f2157f63c25..3dafe560281 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -1,11 +1,12 @@ -#include #include +#include #include #include #include #include #include - +#include +#include "Common/logger_useful.h" namespace DB { @@ -13,17 +14,18 @@ namespace DB using namespace MySQLProtocol; using namespace MySQLProtocol::Generic; using namespace MySQLProtocol::ProtocolText; - +using namespace MySQLProtocol::ProtocolBinary; MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) - : IOutputFormat(header_, out_) - , client_capabilities(settings_.mysql_wire.client_capabilities) + : IOutputFormat(header_, out_), client_capabilities(settings_.mysql_wire.client_capabilities) { /// MySQlWire is a special format that is usually used as output format for MySQL protocol connections. /// In this case we have a correct `sequence_id` stored in `settings_.mysql_wire`. /// But it's also possible to specify MySQLWire as output format for clickhouse-client or clickhouse-local. /// There is no `sequence_id` stored in `settings_.mysql_wire` in this case, so we create a dummy one. sequence_id = settings_.mysql_wire.sequence_id ? settings_.mysql_wire.sequence_id : &dummy_sequence_id; + /// Switch between Text (COM_QUERY) and Binary (COM_EXECUTE_STMT) ResultSet + use_binary_result_set = settings_.mysql_wire.use_binary_result_set; const auto & header = getPort(PortKind::Main).getHeader(); data_types = header.getDataTypes(); @@ -54,7 +56,7 @@ void MySQLOutputFormat::writePrefix() packet_endpoint->sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); } - if (!(client_capabilities & Capability::CLIENT_DEPRECATE_EOF)) + if (!(client_capabilities & Capability::CLIENT_DEPRECATE_EOF) && !use_binary_result_set) { packet_endpoint->sendPacket(EOFPacket(0, 0)); } @@ -63,39 +65,67 @@ void MySQLOutputFormat::writePrefix() void MySQLOutputFormat::consume(Chunk chunk) { - for (size_t i = 0; i < chunk.getNumRows(); ++i) + if (!use_binary_result_set) { - ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), static_cast(i)); - packet_endpoint->sendPacket(row_packet); + for (size_t i = 0; i < chunk.getNumRows(); ++i) + { + ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), static_cast(i)); + packet_endpoint->sendPacket(row_packet); + } + } + else + { + for (size_t i = 0; i < chunk.getNumRows(); ++i) + { + ProtocolBinary::ResultSetRow row_packet(serializations, data_types, chunk.getColumns(), static_cast(i)); + packet_endpoint->sendPacket(row_packet); + } } } void MySQLOutputFormat::finalizeImpl() { - size_t affected_rows = 0; - std::string human_readable_info; - if (QueryStatusPtr process_list_elem = getContext()->getProcessListElement()) + if (!use_binary_result_set) { - CurrentThread::finalizePerformanceCounters(); - QueryStatusInfo info = process_list_elem->getInfo(); - affected_rows = info.written_rows; - double elapsed_seconds = static_cast(info.elapsed_microseconds) / 1000000.0; - human_readable_info = fmt::format( - "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", - info.read_rows, - ReadableSize(info.read_bytes), - elapsed_seconds, - static_cast(info.read_rows / elapsed_seconds), - ReadableSize(info.read_bytes / elapsed_seconds)); - } + size_t affected_rows = 0; + std::string human_readable_info; + if (QueryStatusPtr process_list_elem = getContext()->getProcessListElement()) + { + CurrentThread::finalizePerformanceCounters(); + QueryStatusInfo info = process_list_elem->getInfo(); + affected_rows = info.written_rows; + double elapsed_seconds = static_cast(info.elapsed_microseconds) / 1000000.0; + human_readable_info = fmt::format( + "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", + info.read_rows, + ReadableSize(info.read_bytes), + elapsed_seconds, + static_cast(info.read_rows / elapsed_seconds), + ReadableSize(info.read_bytes / elapsed_seconds)); + } - const auto & header = getPort(PortKind::Main).getHeader(); - if (header.columns() == 0) - packet_endpoint->sendPacket(OKPacket(0x0, client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); - else if (client_capabilities & CLIENT_DEPRECATE_EOF) - packet_endpoint->sendPacket(OKPacket(0xfe, client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); + const auto & header = getPort(PortKind::Main).getHeader(); + if (header.columns() == 0) + packet_endpoint->sendPacket(OKPacket(0x0, client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); + else if (client_capabilities & CLIENT_DEPRECATE_EOF) + packet_endpoint->sendPacket(OKPacket(0xfe, client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); + else + packet_endpoint->sendPacket(EOFPacket(0, 0), true); + } else - packet_endpoint->sendPacket(EOFPacket(0, 0), true); + { + size_t affected_rows = 0; + if (QueryStatusPtr process_list_elem = getContext()->getProcessListElement()) + { + CurrentThread::finalizePerformanceCounters(); + QueryStatusInfo info = process_list_elem->getInfo(); + affected_rows = info.written_rows; + } + if (client_capabilities & CLIENT_DEPRECATE_EOF) + packet_endpoint->sendPacket(OKPacket(0xfe, client_capabilities, affected_rows, 0, 0, "", ""), true); + else + packet_endpoint->sendPacket(EOFPacket(0, 0), true); + } } void MySQLOutputFormat::flush() @@ -107,9 +137,8 @@ void registerOutputFormatMySQLWire(FormatFactory & factory) { factory.registerOutputFormat( "MySQLWire", - [](WriteBuffer & buf, - const Block & sample, - const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); + [](WriteBuffer & buf, const Block & sample, const FormatSettings & settings) + { return std::make_shared(buf, sample, settings); }); } } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.h b/src/Processors/Formats/Impl/MySQLOutputFormat.h index 9481ef67070..6161b6bdc14 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include #include @@ -39,6 +39,7 @@ private: MySQLProtocol::PacketEndpointPtr packet_endpoint; DataTypes data_types; Serializations serializations; + bool use_binary_result_set = false; }; } diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 868575b701f..3715dfea9f7 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -1,29 +1,29 @@ #include "MySQLHandler.h" #include -#include -#include +#include +#include +#include #include #include -#include #include #include -#include -#include -#include #include #include #include +#include #include #include -#include +#include +#include +#include #include #include -#include -#include -#include -#include #include +#include +#include +#include +#include #include "config_version.h" @@ -67,10 +67,7 @@ static String killConnectionIdReplacementQuery(const String & query); static String selectLimitReplacementQuery(const String & query); MySQLHandler::MySQLHandler( - IServer & server_, - TCPServer & tcp_server_, - const Poco::Net::StreamSocket & socket_, - bool ssl_enabled, uint32_t connection_id_) + IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, uint32_t connection_id_) : Poco::Net::TCPServerConnection(socket_) , server(server_) , tcp_server(tcp_server_) @@ -78,7 +75,8 @@ MySQLHandler::MySQLHandler( , connection_id(connection_id_) , auth_plugin(new MySQLProtocol::Authentication::Native41()) { - server_capabilities = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; + server_capabilities = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA + | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; if (ssl_enabled) server_capabilities |= CLIENT_SSL; @@ -104,8 +102,13 @@ void MySQLHandler::run() try { - Handshake handshake(server_capabilities, connection_id, VERSION_STRING + String("-") + VERSION_NAME, - auth_plugin->getName(), auth_plugin->getAuthPluginData(), CharacterSet::utf8_general_ci); + Handshake handshake( + server_capabilities, + connection_id, + VERSION_STRING + String("-") + VERSION_NAME, + auth_plugin->getName(), + auth_plugin->getAuthPluginData(), + CharacterSet::utf8_general_ci); packet_endpoint->sendPacket(handshake, true); LOG_TRACE(log, "Sent handshake"); @@ -115,8 +118,10 @@ void MySQLHandler::run() client_capabilities = handshake_response.capability_flags; max_packet_size = handshake_response.max_packet_size ? handshake_response.max_packet_size : MAX_PACKET_LENGTH; - LOG_TRACE(log, - "Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: {}", + LOG_TRACE( + log, + "Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: " + "{}", handshake_response.capability_flags, handshake_response.max_packet_size, static_cast(handshake_response.character_set), @@ -160,8 +165,8 @@ void MySQLHandler::run() // For commands which are executed without MemoryTracker. LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet."); - LOG_DEBUG(log, "Received command: {}. Connection id: {}.", - static_cast(static_cast(command)), connection_id); + LOG_DEBUG( + log, "Received command: {}. Connection id: {}.", static_cast(static_cast(command)), connection_id); if (!tcp_server.isOpen()) return; @@ -175,7 +180,7 @@ void MySQLHandler::run() comInitDB(limited_payload); break; case COM_QUERY: - comQuery(payload); + comQuery(payload, false); break; case COM_FIELD_LIST: comFieldList(limited_payload); @@ -227,13 +232,15 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp size_t pos = 0; /// Reads at least count and at most packet_size bytes. - auto read_bytes = [this, &buf, &pos, &packet_size](size_t count) -> void { + auto read_bytes = [this, &buf, &pos, &packet_size](size_t count) -> void + { while (pos < count) { int ret = socket().receiveBytes(buf + pos, static_cast(packet_size - pos)); if (ret == 0) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Bytes read: {}. Bytes expected: 3", std::to_string(pos)); + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Bytes read: {}. Bytes expected: 3", std::to_string(pos)); } pos += ret; } @@ -272,7 +279,8 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl authPluginSSL(); } - std::optional auth_response = auth_plugin_name == auth_plugin->getName() ? std::make_optional(initial_auth_response) : std::nullopt; + std::optional auth_response + = auth_plugin_name == auth_plugin->getName() ? std::make_optional(initial_auth_response) : std::nullopt; auth_plugin->authenticate(user_name, *session, auth_response, packet_endpoint, secure_connection, socket().peerAddress()); } catch (const Exception & exc) @@ -304,8 +312,17 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAll()) { ColumnDefinition column_definition( - database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0, true - ); + database, + packet.table, + packet.table, + column.name, + column.name, + CharacterSet::binary, + 100, + ColumnType::MYSQL_TYPE_STRING, + 0, + 0, + true); packet_endpoint->sendPacket(column_definition); } packet_endpoint->sendPacket(OKPacket(0xfe, client_capabilities, 0, 0, 0), true); @@ -318,7 +335,7 @@ void MySQLHandler::comPing() static bool isFederatedServerSetupSetCommand(const String & query); -void MySQLHandler::comQuery(ReadBuffer & payload) +void MySQLHandler::comQuery(ReadBuffer & payload, bool use_binary_protocol_result_set) { String query = String(payload.position(), payload.buffer().end()); @@ -350,20 +367,22 @@ void MySQLHandler::comQuery(ReadBuffer & payload) query_context->setCurrentQueryId(fmt::format("mysql:{}:{}", connection_id, toString(UUIDHelpers::generateV4()))); CurrentThread::QueryScope query_scope{query_context}; - std::atomic affected_rows {0}; + std::atomic affected_rows{0}; auto prev = query_context->getProgressCallback(); - query_context->setProgressCallback([&, my_prev = prev](const Progress & progress) - { - if (my_prev) - my_prev(progress); + query_context->setProgressCallback( + [&, my_prev = prev](const Progress & progress) + { + if (my_prev) + my_prev(progress); - affected_rows += progress.written_rows; - }); + affected_rows += progress.written_rows; + }); FormatSettings format_settings; format_settings.mysql_wire.client_capabilities = client_capabilities; format_settings.mysql_wire.max_packet_size = max_packet_size; format_settings.mysql_wire.sequence_id = &sequence_id; + format_settings.mysql_wire.use_binary_result_set = use_binary_protocol_result_set; auto set_result_details = [&with_output](const QueryResultDetails & details) { @@ -385,11 +404,18 @@ void MySQLHandler::comQuery(ReadBuffer & payload) void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) { + if (prepared_statements_map.size() > 10000) /// Shouldn't happen in reality as COM_STMT_CLOSE cleans up the elements + { + LOG_ERROR(log, "Too many prepared statements"); + packet_endpoint->sendPacket(ERRPacket(), true); + return; + } + String query; readStringUntilEOF(query, payload); uint32_t statement_id = current_prepared_statement_id; - if (current_prepared_statement_id == std::numeric_limits::max()) [[unlikely]] + if (current_prepared_statement_id == std::numeric_limits::max()) { current_prepared_statement_id = 0; } @@ -400,7 +426,7 @@ void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) // Key collisions should not happen here, as we remove the elements from the map with COM_STMT_CLOSE, // and we have quite a big range of available identifiers with 32-bit unsigned integer - if (prepared_statements_map.contains(statement_id)) [[unlikely]] + if (prepared_statements_map.contains(statement_id)) { LOG_ERROR( log, @@ -411,8 +437,8 @@ void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) packet_endpoint->sendPacket(ERRPacket(), true); return; } - prepared_statements_map.emplace(statement_id, query); + prepared_statements_map.emplace(statement_id, query); packet_endpoint->sendPacket(PrepareStatementResponseOK(statement_id, 0, 0, 0), true); } @@ -421,7 +447,7 @@ void MySQLHandler::comStmtExecute(ReadBuffer & payload) uint32_t statement_id; payload.readStrict(reinterpret_cast(&statement_id), 4); - if (!prepared_statements_map.contains(statement_id)) [[unlikely]] + if (!prepared_statements_map.contains(statement_id)) { LOG_ERROR(log, "Could not find prepared statement with id {}", statement_id); packet_endpoint->sendPacket(ERRPacket(), true); @@ -430,14 +456,16 @@ void MySQLHandler::comStmtExecute(ReadBuffer & payload) // Temporary workaround as we work only with queries that do not bind any parameters atm ReadBufferFromString com_query_payload(prepared_statements_map.at(statement_id)); - MySQLHandler::comQuery(com_query_payload); + MySQLHandler::comQuery(com_query_payload, true); }; -void MySQLHandler::comStmtClose([[maybe_unused]] ReadBuffer & payload) { +void MySQLHandler::comStmtClose(ReadBuffer & payload) +{ uint32_t statement_id; payload.readStrict(reinterpret_cast(&statement_id), 4); - if (prepared_statements_map.contains(statement_id)) { + if (prepared_statements_map.contains(statement_id)) + { prepared_statements_map.erase(statement_id); } @@ -447,13 +475,17 @@ void MySQLHandler::comStmtClose([[maybe_unused]] ReadBuffer & payload) { void MySQLHandler::authPluginSSL() { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "ClickHouse was built without SSL support. Try specifying password using double SHA1 in users.xml."); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "ClickHouse was built without SSL support. Try specifying password using double SHA1 in users.xml."); } void MySQLHandler::finishHandshakeSSL( - [[maybe_unused]] size_t packet_size, [[maybe_unused]] char * buf, [[maybe_unused]] size_t pos, - [[maybe_unused]] std::function read_bytes, [[maybe_unused]] MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) + [[maybe_unused]] size_t packet_size, + [[maybe_unused]] char * buf, + [[maybe_unused]] size_t pos, + [[maybe_unused]] std::function read_bytes, + [[maybe_unused]] MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Client requested SSL, while it is disabled."); } @@ -467,10 +499,9 @@ MySQLHandlerSSL::MySQLHandlerSSL( uint32_t connection_id_, RSA & public_key_, RSA & private_key_) - : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_) - , public_key(public_key_) - , private_key(private_key_) -{} + : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_), public_key(public_key_), private_key(private_key_) +{ +} void MySQLHandlerSSL::authPluginSSL() { @@ -478,7 +509,10 @@ void MySQLHandlerSSL::authPluginSSL() } void MySQLHandlerSSL::finishHandshakeSSL( - size_t packet_size, char *buf, size_t pos, std::function read_bytes, + size_t packet_size, + char * buf, + size_t pos, + std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) { read_bytes(packet_size); /// Reading rest SSLRequest. @@ -508,8 +542,8 @@ static bool isFederatedServerSetupSetCommand(const String & query) "|(^(SET AUTOCOMMIT(.*)))" "|(^(SET sql_mode(.*)))" "|(^(SET @@(.*)))" - "|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))" - , std::regex::icase}; + "|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))", + std::regex::icase}; return 1 == std::regex_match(query, expr); } diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 6b8cc56a46e..a412b647ae2 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -1,12 +1,12 @@ #pragma once -#include -#include -#include #include -#include #include +#include #include +#include +#include +#include #include "IServer.h" #include "config.h" @@ -19,7 +19,7 @@ namespace CurrentMetrics { - extern const Metric MySQLConnection; +extern const Metric MySQLConnection; } namespace DB @@ -32,11 +32,7 @@ class MySQLHandler : public Poco::Net::TCPServerConnection { public: MySQLHandler( - IServer & server_, - TCPServer & tcp_server_, - const Poco::Net::StreamSocket & socket_, - bool ssl_enabled, - uint32_t connection_id_); + IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, uint32_t connection_id_); void run() final; @@ -46,7 +42,7 @@ protected: /// Enables SSL, if client requested. void finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResponse &); - void comQuery(ReadBuffer & payload); + void comQuery(ReadBuffer & payload, bool use_binary_protocol_result_set); void comFieldList(ReadBuffer & payload); @@ -63,7 +59,12 @@ protected: void comStmtClose(ReadBuffer & payload); virtual void authPluginSSL(); - virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); + virtual void finishHandshakeSSL( + size_t packet_size, + char * buf, + size_t pos, + std::function read_bytes, + MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); IServer & server; TCPServer & tcp_server; @@ -109,8 +110,11 @@ private: void authPluginSSL() override; void finishHandshakeSSL( - size_t packet_size, char * buf, size_t pos, - std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) override; + size_t packet_size, + char * buf, + size_t pos, + std::function read_bytes, + MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) override; RSA & public_key; RSA & private_key; diff --git a/src/Storages/System/InformationSchema/key_column_usage.sql b/src/Storages/System/InformationSchema/key_column_usage.sql new file mode 100644 index 00000000000..43630b8c8b9 --- /dev/null +++ b/src/Storages/System/InformationSchema/key_column_usage.sql @@ -0,0 +1,38 @@ +ATTACH VIEW key_column_usage + ( + `referenced_table_schema` Nullable(String), + `referenced_table_name` Nullable(String), + `referenced_column_name` Nullable(String), + `table_schema` String, + `table_name` String, + `column_name` Nullable(String), + `ordinal_position` UInt32, + `constraint_name` Nullable(String), + `REFERENCED_TABLE_SCHEMA` Nullable(String), + `REFERENCED_TABLE_NAME` Nullable(String), + `REFERENCED_COLUMN_NAME` Nullable(String), + `TABLE_SCHEMA` String, + `TABLE_NAME` String, + `COLUMN_NAME` Nullable(String), + `ORDINAL_POSITION` UInt32, + `CONSTRAINT_NAME` Nullable(String) + ) AS +SELECT NULL AS `referenced_table_schema`, + NULL AS `referenced_table_name`, + NULL AS `referenced_column_name`, + database AS `table_schema`, + table AS `table_name`, + name AS `column_name`, + position AS `ordinal_position`, + 'PRIMARY' AS `constraint_name`, + + `referenced_table_schema` AS `REFERENCED_TABLE_SCHEMA`, + `referenced_table_name` AS `REFERENCED_TABLE_NAME`, + `referenced_column_name` AS `REFERENCED_COLUMN_NAME`, + `table_schema` AS `TABLE_SCHEMA`, + `table_name` AS `TABLE_NAME`, + `column_name` AS `COLUMN_NAME`, + `ordinal_position` AS `ORDINAL_POSITION`, + `constraint_name` AS `CONSTRAINT_NAME` +FROM system.columns +WHERE is_in_primary_key; \ No newline at end of file diff --git a/src/Storages/System/InformationSchema/referential_constraints.sql b/src/Storages/System/InformationSchema/referential_constraints.sql new file mode 100644 index 00000000000..8216b8fff83 --- /dev/null +++ b/src/Storages/System/InformationSchema/referential_constraints.sql @@ -0,0 +1,25 @@ +ATTACH VIEW referential_constraints + ( + `constraint_name` Nullable(String), + `constraint_schema` String, + `table_name` String, + `update_rule` String, + `delete_rule` String, + `CONSTRAINT_NAME` Nullable(String), + `CONSTRAINT_SCHEMA` String, + `TABLE_NAME` String, + `UPDATE_RULE` String, + `DELETE_RULE` String + ) AS +SELECT NULL AS `constraint_name`, + '' AS `constraint_schema`, + '' AS `table_name`, + '' AS `update_rule`, + '' AS `delete_rule`, + + NULL AS `CONSTRAINT_NAME`, + '' AS `CONSTRAINT_SCHEMA`, + '' AS `TABLE_NAME`, + '' AS `UPDATE_RULE`, + '' AS `DELETE_RULE` +WHERE false; \ No newline at end of file diff --git a/src/Storages/System/InformationSchema/schemata.sql b/src/Storages/System/InformationSchema/schemata.sql index 9686fcbf4fa..ca4ad4f7310 100644 --- a/src/Storages/System/InformationSchema/schemata.sql +++ b/src/Storages/System/InformationSchema/schemata.sql @@ -1,26 +1,33 @@ ATTACH VIEW schemata -( - `catalog_name` String, - `schema_name` String, - `schema_owner` String, - `default_character_set_catalog` Nullable(String), - `default_character_set_schema` Nullable(String), - `default_character_set_name` Nullable(String), - `sql_path` Nullable(String), - `CATALOG_NAME` String ALIAS catalog_name, - `SCHEMA_NAME` String ALIAS schema_name, - `SCHEMA_OWNER` String ALIAS schema_owner, - `DEFAULT_CHARACTER_SET_CATALOG` Nullable(String) ALIAS default_character_set_catalog, - `DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String) ALIAS default_character_set_schema, - `DEFAULT_CHARACTER_SET_NAME` Nullable(String) ALIAS default_character_set_name, - `SQL_PATH` Nullable(String) ALIAS sql_path -) AS -SELECT - name AS catalog_name, - name AS schema_name, - 'default' AS schema_owner, - NULL AS default_character_set_catalog, - NULL AS default_character_set_schema, - NULL AS default_character_set_name, - NULL AS sql_path + ( + `catalog_name` String, + `schema_name` String, + `schema_owner` String, + `default_character_set_catalog` Nullable(String), + `default_character_set_schema` Nullable(String), + `default_character_set_name` Nullable(String), + `sql_path` Nullable(String), + `CATALOG_NAME` String, + `SCHEMA_NAME` String, + `SCHEMA_OWNER` String, + `DEFAULT_CHARACTER_SET_CATALOG` Nullable(String), + `DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String), + `DEFAULT_CHARACTER_SET_NAME` Nullable(String), + `SQL_PATH` Nullable(String) + ) AS +SELECT name AS `catalog_name`, + name AS `schema_name`, + 'default' AS `schema_owner`, + NULL AS `default_character_set_catalog`, + NULL AS `default_character_set_schema`, + NULL AS `default_character_set_name`, + NULL AS `sql_path`, + + catalog_name AS `CATALOG_NAME`, + schema_name AS `SCHEMA_NAME`, + schema_owner AS `SCHEMA_OWNER`, + NULL AS `DEFAULT_CHARACTER_SET_CATALOG`, + NULL AS `DEFAULT_CHARACTER_SET_SCHEMA`, + NULL AS `DEFAULT_CHARACTER_SET_NAME`, + NULL AS `SQL_PATH` FROM system.databases diff --git a/src/Storages/System/InformationSchema/tables.sql b/src/Storages/System/InformationSchema/tables.sql index 8eea3713923..b3bbfa72517 100644 --- a/src/Storages/System/InformationSchema/tables.sql +++ b/src/Storages/System/InformationSchema/tables.sql @@ -1,17 +1,35 @@ ATTACH VIEW tables -( - `table_catalog` String, - `table_schema` String, - `table_name` String, - `table_type` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5), - `TABLE_CATALOG` String ALIAS table_catalog, - `TABLE_SCHEMA` String ALIAS table_schema, - `TABLE_NAME` String ALIAS table_name, - `TABLE_TYPE` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5) ALIAS table_type -) AS -SELECT - database AS table_catalog, - database AS table_schema, - name AS table_name, - multiIf(is_temporary, 4, engine like '%View', 2, engine LIKE 'System%', 5, has_own_data = 0, 3, 1) AS table_type -FROM system.tables + ( + `table_catalog` String, + `table_schema` String, + `table_name` String, + `table_type` String, + `table_comment` String, + `table_collation` String, + `TABLE_CATALOG` String, + `TABLE_SCHEMA` String, + `TABLE_NAME` String, + `TABLE_TYPE` String, + `TABLE_COMMENT` String, + `TABLE_COLLATION` String + ) AS +SELECT database AS `table_catalog`, + database AS `table_schema`, + name AS `table_name`, + comment AS `table_comment`, + multiIf( + is_temporary, 'LOCAL TEMPORARY', + engine LIKE '%View', 'VIEW', + engine LIKE 'System%', 'SYSTEM VIEW', + has_own_data = 0, 'FOREIGN TABLE', + 'BASE TABLE' + ) AS `table_type`, + 'utf8mb4_0900_ai_ci' AS `table_collation`, + + table_catalog AS `TABLE_CATALOG`, + table_schema AS `TABLE_SCHEMA`, + table_name AS `TABLE_NAME`, + table_comment AS `TABLE_COMMENT`, + table_type AS `TABLE_TYPE`, + table_collation AS `TABLE_COLLATION` +FROM system.tables \ No newline at end of file diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 074a648d235..d4775bf0d4a 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -12,7 +12,8 @@ INCBIN(resource_schemata_sql, SOURCE_DIR "/src/Storages/System/InformationSchema INCBIN(resource_tables_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/tables.sql"); INCBIN(resource_views_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/views.sql"); INCBIN(resource_columns_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/columns.sql"); - +INCBIN(resource_key_column_usage_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/key_column_usage.sql"); +INCBIN(resource_referential_constraints_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/referential_constraints.sql"); namespace DB { @@ -66,6 +67,8 @@ void attachInformationSchema(ContextMutablePtr context, IDatabase & information_ createInformationSchemaView(context, information_schema_database, "tables", std::string_view(reinterpret_cast(gresource_tables_sqlData), gresource_tables_sqlSize)); createInformationSchemaView(context, information_schema_database, "views", std::string_view(reinterpret_cast(gresource_views_sqlData), gresource_views_sqlSize)); createInformationSchemaView(context, information_schema_database, "columns", std::string_view(reinterpret_cast(gresource_columns_sqlData), gresource_columns_sqlSize)); + createInformationSchemaView(context, information_schema_database, "key_column_usage", std::string_view(reinterpret_cast(gresource_key_column_usage_sqlData), gresource_key_column_usage_sqlSize)); + createInformationSchemaView(context, information_schema_database, "referential_constraints", std::string_view(reinterpret_cast(gresource_referential_constraints_sqlData), gresource_referential_constraints_sqlSize)); } } From 2447eb027c37a68f409fec1352e1abce0e6cbf54 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 28 Aug 2023 15:12:44 +0300 Subject: [PATCH 049/327] Update StorageHDFS.cpp --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index dafdba15549..09c205930dc 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -261,7 +261,7 @@ namespace HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto res = LSWithRegexpMatching("/", fs, path_from_uri, context->getSettingsRef().ignore_eacces_multidirectory_globs) + auto res = LSWithRegexpMatching("/", fs, path_from_uri, context->getSettingsRef().ignore_eacces_multidirectory_globs); return res; } } From b6ff1b78c7acbb32d80082d71bf342b346f448f2 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 28 Aug 2023 21:35:16 +0300 Subject: [PATCH 050/327] Update 02771_multidirectory_globs_storage_file.sh --- .../0_stateless/02771_multidirectory_globs_storage_file.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh index 4901e716be3..8c8faafba82 100755 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -1,9 +1,6 @@ #!/usr/bin/env bash # Tags: no-replicated-database, no-parallel -#SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; -#SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; - set -eu CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 1c4d49c46e2f87505f45e30c02648898b3d37849 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 29 Aug 2023 09:05:03 +0300 Subject: [PATCH 051/327] Update tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh Co-authored-by: SmitaRKulkarni <64093672+SmitaRKulkarni@users.noreply.github.com> --- .../0_stateless/02771_multidirectory_globs_storage_file.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh index 8c8faafba82..984ae47d222 100755 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # Data preparation. -# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# Now we can get the user_files_path by using the file function, we can also get it by this query: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') From e9b0d3e4a2775bd3a9df5b167452e7971fcbc0ce Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 29 Aug 2023 09:41:18 +0000 Subject: [PATCH 052/327] do not implicitly read pk and version columns in lonely parts if nnot necessary --- .../QueryPlan/ReadFromMergeTree.cpp | 13 ++++---- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- ...ross_partitions_final_all_lonely.reference | 21 ++++++++++++ ...rge_across_partitions_final_all_lonely.sql | 32 +++++++++++++++++++ 4 files changed, 60 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference create mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0b5eb94dbac..02b3d7a07fa 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -979,7 +979,7 @@ static void addMergingFinal( Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & column_names, ActionsDAGPtr & out_projection) + RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection) { const auto & settings = context->getSettingsRef(); const auto data_settings = data.getSettings(); @@ -1141,17 +1141,16 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); - auto pipe = read(std::move(lonely_parts), column_names, ReadFromMergeTree::ReadType::Default, + auto pipe = read(std::move(lonely_parts), partition_pipes.empty() ? origin_column_names : column_names, ReadFromMergeTree::ReadType::Default, num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache); /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) out_projection = createProjection(pipe.getHeader()); - pipe.addSimpleTransform([sorting_expr](const Block & header) - { - return std::make_shared(header, sorting_expr); - }); + if (!partition_pipes.empty()) + pipe.addSimpleTransform([sorting_expr](const Block & header) + { return std::make_shared(header, sorting_expr); }); partition_pipes.emplace_back(std::move(pipe)); } @@ -1742,7 +1741,7 @@ Pipe ReadFromMergeTree::spreadMarkRanges( ::sort(column_names_to_read.begin(), column_names_to_read.end()); column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end()); - return spreadMarkRangesAmongStreamsFinal(std::move(parts_with_ranges), num_streams, column_names_to_read, result_projection); + return spreadMarkRangesAmongStreamsFinal(std::move(parts_with_ranges), num_streams, result.column_names_to_read, column_names_to_read, result_projection); } else if (input_order_info) { diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index cb2a3a8ddf9..2a4b6022d49 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -314,7 +314,7 @@ private: const InputOrderInfoPtr & input_order_info); Pipe spreadMarkRangesAmongStreamsFinal( - RangesInDataParts && parts, size_t num_streams, const Names & column_names, ActionsDAGPtr & out_projection); + RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); ReadFromMergeTree::AnalysisResult getAnalysisResult() const; MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr; diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference new file mode 100644 index 00000000000..9f4677b0f16 --- /dev/null +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference @@ -0,0 +1,21 @@ +(Expression) +ExpressionTransform × 16 +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 4 → 16 + Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 4 + Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 4 → 4 + Header × 4 : val UInt64: val UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + (ReadFromMergeTree) + ExpressionTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + MergeTreeThread × 4 0 → 1 + Header: val UInt64: val UInt64 UInt64(size = 0) diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql new file mode 100644 index 00000000000..d2c40bb002a --- /dev/null +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS all_lonely; + +CREATE TABLE all_lonely +( + `id` UInt64, + `dt` Date, + `val` UInt64, + `version` UInt64 +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY dt +ORDER BY (id); + +INSERT INTO all_lonely SELECT number, '2022-10-28', number*10, 0 FROM numbers(10000); +INSERT INTO all_lonely SELECT number+500000, '2022-10-28', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE all_lonely PARTITION '2022-10-28' FINAL; + + +INSERT INTO all_lonely SELECT number, '2022-10-29', number*10, 0 FROM numbers(10000); +INSERT INTO all_lonely SELECT number+500000, '2022-10-29', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE all_lonely PARTITION '2022-10-29' FINAL; + +INSERT INTO all_lonely SELECT number, '2022-10-30', number*10, 0 FROM numbers(10000); +INSERT INTO all_lonely SELECT number+500000, '2022-10-30', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE all_lonely PARTITION '2022-10-30' FINAL; + + +INSERT INTO all_lonely SELECT number, '2022-10-31', number*10, 0 FROM numbers(10000); +INSERT INTO all_lonely SELECT number+500000, '2022-10-31', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE all_lonely PARTITION '2022-10-31' FINAL; + +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM all_lonely FINAL SETTINGS do_not_merge_across_partitions_select_final = 1, max_threads = 16; From 11c94246c774217bbf3b64b8e6166e22b8f0b3ab Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 30 Aug 2023 05:05:57 +0000 Subject: [PATCH 053/327] do not implicitly read pk + version and compute sort expression for lonely parts Signed-off-by: Duc Canh Le --- .../QueryPlan/ReadFromMergeTree.cpp | 115 ++++++---- ...ross_partitions_final_all_lonely.reference | 21 -- ...rge_across_partitions_final_all_lonely.sql | 32 --- ...oss_partitions_final_with_lonely.reference | 202 ++++++++++++++++++ ...ge_across_partitions_final_with_lonely.sql | 49 +++++ 5 files changed, 321 insertions(+), 98 deletions(-) delete mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference delete mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql create mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference create mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 02b3d7a07fa..e868cb3845a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -982,7 +982,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts_with_ranges, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection) { const auto & settings = context->getSettingsRef(); - const auto data_settings = data.getSettings(); + const auto & data_settings = data.getSettings(); PartRangesReadInfo info(parts_with_ranges, settings, *data_settings); @@ -1017,7 +1017,8 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( parts_to_merge_ranges.push_back(parts_with_ranges.end()); } - Pipes partition_pipes; + Pipes merging_pipes; + Pipes no_merging_pipes; /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 /// we will store lonely parts with level > 0 to use parallel select on them. @@ -1028,20 +1029,21 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) { + /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition + /// with level > 0 then we won't post-process this part, and if num_streams > 1 we + /// can use parallel select on such parts. + bool no_merging_final = settings.do_not_merge_across_partitions_select_final && + std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && + parts_to_merge_ranges[range_index]->data_part->info.level > 0 && + data.merging_params.is_deleted_column.empty(); Pipes pipes; { RangesInDataParts new_parts; - /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition - /// with level > 0 then we won't postprocess this part and if num_streams > 1 we - /// can use parallel select on such parts. We save such parts in one vector and then use - /// MergeTreeReadPool and MergeTreeThreadSelectProcessor for parallel select. - if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && - std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && - parts_to_merge_ranges[range_index]->data_part->info.level > 0 - && data.merging_params.is_deleted_column.empty()) + if (no_merging_final) { - sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); + if (num_streams > 1) + sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); continue; } @@ -1091,16 +1093,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( out_projection = createProjection(pipes.front().getHeader()); } - /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition - /// with level > 0 then we won't postprocess this part - if (settings.do_not_merge_across_partitions_select_final && - std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && - parts_to_merge_ranges[range_index]->data_part->info.level > 0 && - data.merging_params.is_deleted_column.empty()) - { - partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes))); - continue; - } Names sort_columns = metadata_for_reading->getSortingKeyColumns(); SortDescription sort_description; @@ -1123,39 +1115,72 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( partition_key_columns, max_block_size); - partition_pipes.emplace_back(Pipe::unitePipes(std::move(pipes))); + merging_pipes.emplace_back(Pipe::unitePipes(std::move(pipes))); } if (!lonely_parts.empty()) { - size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); + Pipe pipe; + if (num_streams > 1) + { + size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); - const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - info.index_granularity_bytes, - sum_marks_in_lonely_parts); + const size_t min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + info.index_granularity_bytes, + sum_marks_in_lonely_parts); - /// Reduce the number of num_streams_for_lonely_parts if the data is small. - if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) - num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + /// Reduce the number of num_streams_for_lonely_parts if the data is small. + if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read + && lonely_parts.size() < num_streams_for_lonely_parts) + num_streams_for_lonely_parts = std::max( + (sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, + lonely_parts.size()); - auto pipe = read(std::move(lonely_parts), partition_pipes.empty() ? origin_column_names : column_names, ReadFromMergeTree::ReadType::Default, - num_streams_for_lonely_parts, min_marks_for_concurrent_read, info.use_uncompressed_cache); - - /// Drop temporary columns, added by 'sorting_key_expr' - if (!out_projection) - out_projection = createProjection(pipe.getHeader()); - - if (!partition_pipes.empty()) - pipe.addSimpleTransform([sorting_expr](const Block & header) - { return std::make_shared(header, sorting_expr); }); - - partition_pipes.emplace_back(std::move(pipe)); + pipe = read( + std::move(lonely_parts), + origin_column_names, + ReadFromMergeTree::ReadType::Default, + num_streams_for_lonely_parts, + min_marks_for_concurrent_read, + info.use_uncompressed_cache); + } + else + { + pipe = read( + std::move(lonely_parts), + origin_column_names, + ReadFromMergeTree::ReadType::InOrder, + num_streams, + 0, + info.use_uncompressed_cache); + } + no_merging_pipes.emplace_back(std::move(pipe)); } - return Pipe::unitePipes(std::move(partition_pipes)); + if (!merging_pipes.empty() && !no_merging_pipes.empty()) + { + out_projection = nullptr; /// We do projection here + Pipes pipes; + pipes.resize(2); + pipes[0] = Pipe::unitePipes(std::move(merging_pipes)); + pipes[1] = Pipe::unitePipes(std::move(no_merging_pipes)); + auto conversion_action = ActionsDAG::makeConvertingActions( + pipes[0].getHeader().getColumnsWithTypeAndName(), + pipes[1].getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + pipes[0].addSimpleTransform( + [conversion_action](const Block & header) + { + auto converting_expr = std::make_shared(conversion_action); + return std::make_shared(header, converting_expr); + }); + return Pipe::unitePipes(std::move(pipes)); + } + else + return merging_pipes.empty() ? Pipe::unitePipes(std::move(no_merging_pipes)) : Pipe::unitePipes(std::move(merging_pipes)); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference deleted file mode 100644 index 9f4677b0f16..00000000000 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.reference +++ /dev/null @@ -1,21 +0,0 @@ -(Expression) -ExpressionTransform × 16 -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 4 → 16 - Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 4 - Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 4 → 4 - Header × 4 : val UInt64: val UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - (ReadFromMergeTree) - ExpressionTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - MergeTreeThread × 4 0 → 1 - Header: val UInt64: val UInt64 UInt64(size = 0) diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql deleted file mode 100644 index d2c40bb002a..00000000000 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_all_lonely.sql +++ /dev/null @@ -1,32 +0,0 @@ -DROP TABLE IF EXISTS all_lonely; - -CREATE TABLE all_lonely -( - `id` UInt64, - `dt` Date, - `val` UInt64, - `version` UInt64 -) -ENGINE = ReplacingMergeTree(version) -PARTITION BY dt -ORDER BY (id); - -INSERT INTO all_lonely SELECT number, '2022-10-28', number*10, 0 FROM numbers(10000); -INSERT INTO all_lonely SELECT number+500000, '2022-10-28', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE all_lonely PARTITION '2022-10-28' FINAL; - - -INSERT INTO all_lonely SELECT number, '2022-10-29', number*10, 0 FROM numbers(10000); -INSERT INTO all_lonely SELECT number+500000, '2022-10-29', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE all_lonely PARTITION '2022-10-29' FINAL; - -INSERT INTO all_lonely SELECT number, '2022-10-30', number*10, 0 FROM numbers(10000); -INSERT INTO all_lonely SELECT number+500000, '2022-10-30', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE all_lonely PARTITION '2022-10-30' FINAL; - - -INSERT INTO all_lonely SELECT number, '2022-10-31', number*10, 0 FROM numbers(10000); -INSERT INTO all_lonely SELECT number+500000, '2022-10-31', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE all_lonely PARTITION '2022-10-31' FINAL; - -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM all_lonely FINAL SETTINGS do_not_merge_across_partitions_select_final = 1, max_threads = 16; diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference new file mode 100644 index 00000000000..a4b37bc423b --- /dev/null +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference @@ -0,0 +1,202 @@ +(Expression) +ExpressionTransform × 16 +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 6 → 16 + Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 6 + Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 6 → 6 + Header × 6 : val UInt64: val UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 6 + Header: val UInt64: val UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeThread × 4 0 → 1 + Header: val UInt64: val UInt64 UInt64(size = 0) + ExpressionTransform × 2 + Header: val UInt64: val UInt64 UInt64(size = 0) + ReplacingSorted 2 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + FilterSortedStreamByRange × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + Description: filter values in [(50000), +inf) + ExpressionTransform × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + MergeTreeInOrder × 2 0 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + ReplacingSorted 2 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + FilterSortedStreamByRange × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + Description: filter values in [-inf, (50000)) + ExpressionTransform × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + MergeTreeInOrder × 2 0 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) +(Expression) +ExpressionTransform × 16 +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 4 → 16 + Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 4 + Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 4 → 4 + Header × 4 : val UInt64: val UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + (Filter) + FilterTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeThread × 4 0 → 1 + Header: dt Date: dt Date UInt16(size = 0) + val UInt64: val UInt64 UInt64(size = 0) +(Expression) +ExpressionTransform +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 4 → 1 + Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 4 + Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 4 → 4 + Header × 4 : val UInt64: val UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + (Filter) + FilterTransform × 4 + Header: val UInt64: val UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeInOrder × 4 0 → 1 + Header: dt Date: dt Date UInt16(size = 0) + val UInt64: val UInt64 UInt64(size = 0) +(Expression) +ExpressionTransform × 16 +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 6 → 16 + Header × 16 : max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 6 + Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 6 → 6 + Header × 6 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 6 + Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeThread × 4 0 → 1 + Header: val UInt64: val UInt64 UInt64(size = 0) + ExpressionTransform × 2 + Header: val UInt64: val UInt64 UInt64(size = 0) + ReplacingSorted 2 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + FilterSortedStreamByRange × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + Description: filter values in [(50000), +inf) + ExpressionTransform × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + MergeTreeInOrder × 2 0 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + ReplacingSorted 2 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + FilterSortedStreamByRange × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + Description: filter values in [-inf, (50000)) + ExpressionTransform × 2 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) + MergeTreeInOrder × 2 0 → 1 + Header: id UInt64: id UInt64 UInt64(size = 0) + val UInt64: val UInt64 UInt64(size = 0) + version UInt64: version UInt64 UInt64(size = 0) +(Expression) +ExpressionTransform × 16 +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 4 → 16 + Header × 16 : max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 4 + Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 4 → 4 + Header × 4 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 4 + Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (Filter) + FilterTransform × 4 + Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeThread × 4 0 → 1 + Header: val UInt64: val UInt64 UInt64(size = 0) + dt Date: dt Date UInt16(size = 0) +(Expression) +ExpressionTransform +Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + (Aggregating) + Resize 4 → 1 + Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + AggregatingTransform × 4 + Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) + count() UInt64: count() UInt64 UInt64(size = 0) + StrictResize 4 → 4 + Header × 4 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (Expression) + ExpressionTransform × 4 + Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (Filter) + FilterTransform × 4 + Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) + (ReadFromMergeTree) + MergeTreeInOrder × 4 0 → 1 + Header: val UInt64: val UInt64 UInt64(size = 0) + dt Date: dt Date UInt16(size = 0) diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql new file mode 100644 index 00000000000..526c9204d96 --- /dev/null +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql @@ -0,0 +1,49 @@ +DROP TABLE IF EXISTS with_lonely; + +CREATE TABLE with_lonely +( + `id` UInt64, + `dt` Date, + `val` UInt64, + `version` UInt64 +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY dt +ORDER BY (id); + +INSERT INTO with_lonely SELECT number, '2022-10-28', number*10, 0 FROM numbers(10000); +INSERT INTO with_lonely SELECT number+500000, '2022-10-28', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE with_lonely PARTITION '2022-10-28' FINAL; + + +INSERT INTO with_lonely SELECT number, '2022-10-29', number*10, 0 FROM numbers(10000); +INSERT INTO with_lonely SELECT number+500000, '2022-10-29', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE with_lonely PARTITION '2022-10-29' FINAL; + +INSERT INTO with_lonely SELECT number, '2022-10-30', number*10, 0 FROM numbers(10000); +INSERT INTO with_lonely SELECT number+500000, '2022-10-30', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE with_lonely PARTITION '2022-10-30' FINAL; + + +INSERT INTO with_lonely SELECT number, '2022-10-31', number*10, 0 FROM numbers(10000); +INSERT INTO with_lonely SELECT number+500000, '2022-10-31', number*10, 1 FROM numbers(10000); +OPTIMIZE TABLE with_lonely PARTITION '2022-10-31' FINAL; + +SYSTEM STOP MERGES with_lonely; +INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(1000); +INSERT INTO with_lonely SELECT number+50000, '2022-11-01', number*10, 1 FROM numbers(1000); +INSERT INTO with_lonely SELECT number+60000, '2022-11-01', number*10, 2 FROM numbers(1000); + +SET do_not_merge_across_partitions_select_final = 1, max_threads = 16; + +-- mix lonely parts and non-lonely parts +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL SETTINGS allow_experimental_analyzer = 0; +-- only lonely parts +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS allow_experimental_analyzer = 0; +-- only lonely parts but max_thread = 1, so reading lonely parts with in-order +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS max_threads = 1, allow_experimental_analyzer = 0; + + +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL SETTINGS allow_experimental_analyzer = 1; +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS allow_experimental_analyzer = 1; +EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS max_threads = 1, allow_experimental_analyzer = 1; From f4dd6be8a861e13b63e7de4cc8013d4c60a2bffe Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 30 Aug 2023 18:59:39 +0200 Subject: [PATCH 054/327] WIP prepared statements --- .../PreparedStatementsTest.java | 155 +++++++++++++ src/Core/MySQL/PacketsProtocolBinary.cpp | 60 +++-- src/Core/MySQL/PacketsProtocolBinary.h | 4 +- src/Core/MySQL/PacketsProtocolText.cpp | 33 ++- src/Core/MySQL/PacketsProtocolText.h | 209 ++++++++++-------- .../Formats/Impl/MySQLOutputFormat.cpp | 2 +- tests/integration/requirements.txt | 30 +++ .../prepared_statements.reference | 1 + tests/integration/test_mysql_protocol/test.py | 122 ++++++++-- 9 files changed, 478 insertions(+), 138 deletions(-) create mode 100644 docker/test/integration/mysql_java_client/PreparedStatementsTest.java create mode 100644 tests/integration/requirements.txt create mode 100644 tests/integration/test_mysql_protocol/prepared_statements.reference diff --git a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java new file mode 100644 index 00000000000..0534f2b761d --- /dev/null +++ b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java @@ -0,0 +1,155 @@ +import com.mysql.cj.MysqlType; + +import java.sql.*; + +public class PreparedStatementsTest { + public static void main(String[] args) { + int i = 0; + String host = "127.0.0.1"; + String port = "9004"; + String user = "default"; + String password = ""; + String database = "default"; + while (i < args.length) { + switch (args[i]) { + case "--host": + host = args[++i]; + break; + case "--port": + port = args[++i]; + break; + case "--user": + user = args[++i]; + break; + case "--password": + password = args[++i]; + break; + case "--database": + database = args[++i]; + break; + default: + i++; + break; + } + } + + try { + Class.forName("com.mysql.cj.jdbc.Driver"); + /* + * useServerPrepStmts allows us to send COM_STMT_PREPARE and COM_STMT_EXECUTE to + * test the binary protocol + */ + String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true", + host, port, database); + Connection conn = DriverManager.getConnection(jdbcUrl, user, password); + testSimpleDataTypes(conn); + testStringTypes(conn); + testDecimalTypes(conn); + testMiscTypes(conn); + // testDateTypes(conn); + conn.close(); + } catch (Exception e) { + e.printStackTrace(); + System.exit(1); + } + } + + private static void testSimpleDataTypes(Connection conn) throws SQLException { + System.out.println("### testSimpleDataTypes"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_simple_data_types").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i8"), rs.getInt("i8")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i16"), rs.getInt("i16")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i32"), rs.getInt("i32")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i64"), rs.getLong("i64")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "i128"), rs.getString("i128")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "i256"), rs.getString("i256")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui8"), rs.getInt("ui8")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui16"), rs.getInt("ui16")); + System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui32"), rs.getLong("ui32")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui64"), rs.getString("ui64")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui128"), rs.getString("ui128")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui256"), rs.getString("ui256")); + System.out.printf("Type: %s, value: %f\n", getMysqlType(rs, "f32"), rs.getFloat("f32")); + System.out.printf("Type: %s, value: %f\n", getMysqlType(rs, "f64"), rs.getFloat("f64")); + System.out.printf("Type: %s, value: %b\n", getMysqlType(rs, "b"), rs.getBoolean("b")); + } + System.out.println(); + } + + private static void testStringTypes(Connection conn) throws SQLException { + System.out.println("### testStringTypes"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_string_types").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "s"), rs.getString("s")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "sn"), rs.getString("sn")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "lc"), rs.getString("lc")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "nlc"), rs.getString("nlc")); + } + System.out.println(); + } + + private static void testDecimalTypes(Connection conn) throws SQLException { + System.out.println("### testDecimalTypes"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_decimal_types").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("Type: %s, value: %s\n", + getMysqlType(rs, "d32"), + rs.getBigDecimal("d32").toPlainString()); + System.out.printf("Type: %s, value: %s\n", + getMysqlType(rs, "d64"), + rs.getBigDecimal("d64").toPlainString()); + System.out.printf("Type: %s, value: %s\n", + getMysqlType(rs, "d128_native"), + rs.getBigDecimal("d128_native").toPlainString()); + System.out.printf("Type: %s, value: %s\n", + getMysqlType(rs, "d128_text"), + rs.getString("d128_text")); + System.out.printf("Type: %s, value: %s\n", + getMysqlType(rs, "d256"), + rs.getString("d256")); + } + System.out.println(); + } + + private static void testDateTypes(Connection conn) throws SQLException { + System.out.println("### testDateTypes"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_date_types").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d32"), rs.getDate("d32")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64"), rs.getTimestamp("dt64")); + } + System.out.println(); + } + + private static void testMiscTypes(Connection conn) throws SQLException { + System.out.println("### testMiscTypes"); + ResultSet rs = conn.prepareStatement("SELECT d FROM ps_misc_types").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "a"), rs.getString("a")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "u"), rs.getString("u")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "t"), rs.getString("t")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "m"), rs.getString("m")); + } + System.out.println(); + } + + private static MysqlType getMysqlType(ResultSet rs, String columnLabel) throws SQLException { + ResultSetMetaData meta = rs.getMetaData(); + return MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel))); + + } + +} diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 287dda269e6..2b5dce78ab3 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -3,11 +3,14 @@ #include #include #include +#include #include +#include "Common/logger_useful.h" #include "Columns/ColumnLowCardinality.h" #include "Columns/ColumnVector.h" #include "DataTypes/DataTypeLowCardinality.h" #include "DataTypes/DataTypeNullable.h" +#include "DataTypes/DataTypesNumber.h" #include "Formats/FormatSettings.h" #include "IO/WriteBufferFromString.h" #include "base/types.h" @@ -26,24 +29,37 @@ namespace MySQLProtocol { /// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row payload_size = 1 + null_bitmap_size; - // LOG_TRACE(&Poco::Logger::get("ResultSetRow"), "Null bitmap size: {}", null_bitmap_size); + LOG_TRACE(&Poco::Logger::get("ResultSetRow"), "Null bitmap size: {}", null_bitmap_size); FormatSettings format_settings; for (size_t i = 0; i < columns.size(); ++i) { ColumnPtr col = columns[i]; - + LOG_TRACE(&Poco::Logger::get("col->isNullAt"), "isNullAt: {}, {}", row_num, col->isNullAt(row_num)); if (col->isNullAt(row_num)) { - null_bitmap[i / 8] |= 1 << i % 8; + size_t byte = (i + 2) / 8; + int bit = 1 << ((i + 2) % 8); + null_bitmap[byte] |= bit; + continue; // NULLs are stored in the null bitmap only } - TypeIndex type_index = removeNullable(removeLowCardinality(data_types[i]))->getTypeId(); + DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); + TypeIndex type_index = data_type->getTypeId(); switch (type_index) { case TypeIndex::Int8: - case TypeIndex::UInt8: payload_size += 1; break; + case TypeIndex::UInt8: + if (data_type->getName() == "Bool") + { + payload_size += 2; // BIT MySQL type is string in binary + } + else + { + payload_size += 1; + } + break; case TypeIndex::Int16: case TypeIndex::UInt16: payload_size += 2; @@ -91,6 +107,7 @@ namespace MySQLProtocol break; } default: + LOG_TRACE(&Poco::Logger::get("Type default"), "{} is {}", col->getName(), data_type->getName()); WriteBufferFromOwnString ostr; serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); payload_size += getLengthEncodedStringSize(ostr.str()); @@ -109,64 +126,69 @@ namespace MySQLProtocol ColumnPtr col = columns[i]; if (col->isNullAt(row_num)) { - continue; // NULLs are stored in the null bitmap only + continue; } - TypeIndex type_index = removeNullable(removeLowCardinality(data_types[i]))->getTypeId(); + DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); + TypeIndex type_index = data_type->getTypeId(); switch (type_index) { case TypeIndex::UInt8: { - UInt64 value = col->get64(row_num); + UInt8 value = assert_cast &>(*col).getData()[row_num]; + if (data_type->getName() == "Bool") + { + buffer.write(static_cast(1)); + } buffer.write(reinterpret_cast(&value), 1); break; } case TypeIndex::UInt16: { - UInt64 value = col->get64(row_num); + UInt16 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 2); break; } case TypeIndex::UInt32: { - UInt64 value = col->get64(row_num); + UInt32 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 4); break; } case TypeIndex::UInt64: { - UInt64 value = col->get64(row_num); + UInt64 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 8); break; } case TypeIndex::Int8: { - UInt64 value = col->get64(row_num); + Int8 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 1); break; } case TypeIndex::Int16: { - UInt64 value = col->get64(row_num); + Int16 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 2); break; } case TypeIndex::Int32: { - UInt64 value = col->get64(row_num); + Int32 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 4); break; } case TypeIndex::Int64: { - UInt64 value = col->get64(row_num); + Int64 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 8); break; } case TypeIndex::Float32: { - Float32 value = col->getFloat32(row_num); + Float32 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 4); break; } case TypeIndex::Float64: { - Float64 value = col->getFloat64(row_num); + Float64 value = assert_cast &>(*col).getData()[row_num]; buffer.write(reinterpret_cast(&value), 8); break; } case TypeIndex::Date: { - UInt64 value = col->get64(row_num); + UInt64 value = assert_cast &>(*col).getData()[row_num]; if (value != 0) { Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); @@ -185,7 +207,7 @@ namespace MySQLProtocol break; } case TypeIndex::DateTime: { - UInt64 value = col->get64(row_num); + UInt64 value = assert_cast &>(*col).getData()[row_num]; if (value != 0) { Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); diff --git a/src/Core/MySQL/PacketsProtocolBinary.h b/src/Core/MySQL/PacketsProtocolBinary.h index 69936e527c1..1a201ce3a4a 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.h +++ b/src/Core/MySQL/PacketsProtocolBinary.h @@ -28,8 +28,8 @@ namespace MySQLProtocol std::vector serialized = std::vector(columns.size()); - size_t null_bitmap_size = (columns.size() + 7) / 8; - std::vector null_bitmap = std::vector(null_bitmap_size, 0); + size_t null_bitmap_size = (columns.size() + 7 + 2) / 8; + std::vector null_bitmap = std::vector(null_bitmap_size, static_cast(0)); size_t payload_size = 0; diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 9c5bf6b6e05..bb5412bb5a5 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -2,7 +2,9 @@ #include #include #include +#include "Common/assert_cast.h" #include "Core/MySQL/IMySQLWritePacket.h" +#include "DataTypes/DataTypesDecimal.h" namespace DB { @@ -142,17 +144,25 @@ namespace MySQLProtocol } } - ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex type_index) + ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type) { ColumnType column_type; CharacterSet charset = CharacterSet::binary; int flags = 0; uint8_t decimals = 0; + TypeIndex type_index = data_type->getTypeId(); switch (type_index) { case TypeIndex::UInt8: - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + if (data_type->getName() == "Bool") + { + column_type = ColumnType::MYSQL_TYPE_BIT; + } + else + { + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + } break; case TypeIndex::UInt16: column_type = ColumnType::MYSQL_TYPE_SHORT; @@ -202,11 +212,24 @@ namespace MySQLProtocol break; case TypeIndex::Decimal32: case TypeIndex::Decimal64: - case TypeIndex::Decimal128: - /// MySQL Decimal has max 65 precision and 30 scale. Thus, Decimal256 is reported as a string column_type = ColumnType::MYSQL_TYPE_DECIMAL; flags = ColumnDefinitionFlags::BINARY_FLAG; break; + case TypeIndex::Decimal128: { + // MySQL Decimal has max 65 precision and 30 scale + const auto & type = assert_cast(*data_type); + if (type.getPrecision() > 65 || type.getScale() > 30) + { + column_type = ColumnType::MYSQL_TYPE_STRING; + charset = CharacterSet::utf8_general_ci; + } + else + { + column_type = ColumnType::MYSQL_TYPE_DECIMAL; + flags = ColumnDefinitionFlags::BINARY_FLAG; + } + break; + } default: column_type = ColumnType::MYSQL_TYPE_STRING; charset = CharacterSet::utf8_general_ci; diff --git a/src/Core/MySQL/PacketsProtocolText.h b/src/Core/MySQL/PacketsProtocolText.h index aeeedc4dbf8..d9ce913494b 100644 --- a/src/Core/MySQL/PacketsProtocolText.h +++ b/src/Core/MySQL/PacketsProtocolText.h @@ -12,122 +12,135 @@ namespace DB namespace MySQLProtocol { -namespace ProtocolText -{ + namespace ProtocolText + { -enum CharacterSet -{ - utf8_general_ci = 33, - binary = 63 -}; + enum CharacterSet + { + utf8_general_ci = 33, + binary = 63 + }; -// https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html -enum ColumnDefinitionFlags -{ - UNSIGNED_FLAG = 32, - BINARY_FLAG = 128 -}; + // https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html + enum ColumnDefinitionFlags + { + UNSIGNED_FLAG = 32, + BINARY_FLAG = 128 + }; -enum ColumnType -{ - MYSQL_TYPE_DECIMAL = 0x00, - MYSQL_TYPE_TINY = 0x01, - MYSQL_TYPE_SHORT = 0x02, - MYSQL_TYPE_LONG = 0x03, - MYSQL_TYPE_FLOAT = 0x04, - MYSQL_TYPE_DOUBLE = 0x05, - MYSQL_TYPE_NULL = 0x06, - MYSQL_TYPE_TIMESTAMP = 0x07, - MYSQL_TYPE_LONGLONG = 0x08, - MYSQL_TYPE_INT24 = 0x09, - MYSQL_TYPE_DATE = 0x0a, - MYSQL_TYPE_TIME = 0x0b, - MYSQL_TYPE_DATETIME = 0x0c, - MYSQL_TYPE_YEAR = 0x0d, - MYSQL_TYPE_NEWDATE = 0x0e, - MYSQL_TYPE_VARCHAR = 0x0f, - MYSQL_TYPE_BIT = 0x10, - MYSQL_TYPE_TIMESTAMP2 = 0x11, - MYSQL_TYPE_DATETIME2 = 0x12, - MYSQL_TYPE_TIME2 = 0x13, - MYSQL_TYPE_JSON = 0xf5, - MYSQL_TYPE_NEWDECIMAL = 0xf6, - MYSQL_TYPE_ENUM = 0xf7, - MYSQL_TYPE_SET = 0xf8, - MYSQL_TYPE_TINY_BLOB = 0xf9, - MYSQL_TYPE_MEDIUM_BLOB = 0xfa, - MYSQL_TYPE_LONG_BLOB = 0xfb, - MYSQL_TYPE_BLOB = 0xfc, - MYSQL_TYPE_VAR_STRING = 0xfd, - MYSQL_TYPE_STRING = 0xfe, - MYSQL_TYPE_GEOMETRY = 0xff -}; + enum ColumnType + { + MYSQL_TYPE_DECIMAL = 0x00, + MYSQL_TYPE_TINY = 0x01, + MYSQL_TYPE_SHORT = 0x02, + MYSQL_TYPE_LONG = 0x03, + MYSQL_TYPE_FLOAT = 0x04, + MYSQL_TYPE_DOUBLE = 0x05, + MYSQL_TYPE_NULL = 0x06, + MYSQL_TYPE_TIMESTAMP = 0x07, + MYSQL_TYPE_LONGLONG = 0x08, + MYSQL_TYPE_INT24 = 0x09, + MYSQL_TYPE_DATE = 0x0a, + MYSQL_TYPE_TIME = 0x0b, + MYSQL_TYPE_DATETIME = 0x0c, + MYSQL_TYPE_YEAR = 0x0d, + MYSQL_TYPE_NEWDATE = 0x0e, + MYSQL_TYPE_VARCHAR = 0x0f, + MYSQL_TYPE_BIT = 0x10, + MYSQL_TYPE_TIMESTAMP2 = 0x11, + MYSQL_TYPE_DATETIME2 = 0x12, + MYSQL_TYPE_TIME2 = 0x13, + MYSQL_TYPE_JSON = 0xf5, + MYSQL_TYPE_NEWDECIMAL = 0xf6, + MYSQL_TYPE_ENUM = 0xf7, + MYSQL_TYPE_SET = 0xf8, + MYSQL_TYPE_TINY_BLOB = 0xf9, + MYSQL_TYPE_MEDIUM_BLOB = 0xfa, + MYSQL_TYPE_LONG_BLOB = 0xfb, + MYSQL_TYPE_BLOB = 0xfc, + MYSQL_TYPE_VAR_STRING = 0xfd, + MYSQL_TYPE_STRING = 0xfe, + MYSQL_TYPE_GEOMETRY = 0xff + }; -class ResultSetRow : public IMySQLWritePacket -{ -protected: - const Columns & columns; - int row_num; - size_t payload_size = 0; - std::vector serialized; + class ResultSetRow : public IMySQLWritePacket + { + protected: + const Columns & columns; + int row_num; + size_t payload_size = 0; + std::vector serialized; - size_t getPayloadSize() const override; + size_t getPayloadSize() const override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; -public: - ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_); -}; + public: + ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_); + }; -class ComFieldList : public LimitedReadPacket -{ -public: - String table, field_wildcard; + class ComFieldList : public LimitedReadPacket + { + public: + String table, field_wildcard; - void readPayloadImpl(ReadBuffer & payload) override; -}; + void readPayloadImpl(ReadBuffer & payload) override; + }; -class ColumnDefinition : public IMySQLWritePacket, public IMySQLReadPacket -{ -public: - String schema; - String table; - String org_table; - String name; - String org_name; - size_t next_length = 0x0c; - uint16_t character_set; - uint32_t column_length; - ColumnType column_type; - uint16_t flags; - uint8_t decimals = 0x00; - /// https://dev.mysql.com/doc/internals/en/com-query-response.html#column-definition - /// There are extra fields in the packet for column defaults - bool is_comm_field_list_response = false; + class ColumnDefinition : public IMySQLWritePacket, public IMySQLReadPacket + { + public: + String schema; + String table; + String org_table; + String name; + String org_name; + size_t next_length = 0x0c; + uint16_t character_set; + uint32_t column_length; + ColumnType column_type; + uint16_t flags; + uint8_t decimals = 0x00; + /// https://dev.mysql.com/doc/internals/en/com-query-response.html#column-definition + /// There are extra fields in the packet for column defaults + bool is_comm_field_list_response = false; -protected: - size_t getPayloadSize() const override; + protected: + size_t getPayloadSize() const override; - void readPayloadImpl(ReadBuffer & payload) override; + void readPayloadImpl(ReadBuffer & payload) override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; -public: - ColumnDefinition(); + public: + ColumnDefinition(); - ColumnDefinition( - String schema_, String table_, String org_table_, String name_, String org_name_, uint16_t character_set_, uint32_t column_length_, - ColumnType column_type_, uint16_t flags_, uint8_t decimals_, bool with_defaults_ = false); + ColumnDefinition( + String schema_, + String table_, + String org_table_, + String name_, + String org_name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_, + bool with_defaults_ = false); - /// Should be used when column metadata (original name, table, original table, database) is unknown. - ColumnDefinition( - String name_, uint16_t character_set_, uint32_t column_length_, ColumnType column_type_, uint16_t flags_, uint8_t decimals_); + /// Should be used when column metadata (original name, table, original table, database) is unknown. + ColumnDefinition( + String name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_); + }; -}; + ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type); -ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex index); - -} + } } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 3dafe560281..d25b68507fc 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -53,7 +53,7 @@ void MySQLOutputFormat::writePrefix() for (size_t i = 0; i < header.columns(); ++i) { const auto & column_name = header.getColumnsWithTypeAndName()[i].name; - packet_endpoint->sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); + packet_endpoint->sendPacket(getColumnDefinition(column_name, data_types[i])); } if (!(client_capabilities & Capability::CLIENT_DEPRECATE_EOF) && !use_binary_result_set) diff --git a/tests/integration/requirements.txt b/tests/integration/requirements.txt new file mode 100644 index 00000000000..11efc43eded --- /dev/null +++ b/tests/integration/requirements.txt @@ -0,0 +1,30 @@ +PyMySQL +aerospike +avro +cassandra-driver +confluent-kafka +dicttoxml +docker +docker-compose +grpcio +grpcio-tools +kafka-python +kazoo +minio +lz4 +protobuf +psycopg2-binary +pymongo +pytz +pytest +pytest-timeout +redis +tzlocal==2.1 +urllib3 +requests-kerberos +dict2xml +hypothesis +pyhdfs +pika +meilisearch +nats-py diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference new file mode 100644 index 00000000000..257cc5642cb --- /dev/null +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -0,0 +1 @@ +foo diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index a5c82e1a2c6..c0bc178c5bb 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -10,7 +10,6 @@ import logging import docker import pymysql.connections import pytest -from docker.models.containers import Container from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -150,7 +149,6 @@ def java_container(): def test_mysql_client(started_cluster): - # type: (ClickHouseCluster) -> None code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( """ mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba @@ -651,7 +649,6 @@ def test_python_client(started_cluster): def test_golang_client(started_cluster, golang_container): - # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, "golang.reference"), "rb") as fp: reference = fp.read() @@ -687,7 +684,6 @@ def test_golang_client(started_cluster, golang_container): def test_php_client(started_cluster, php_container): - # type: (str, Container) -> None code, (stdout, stderr) = php_container.exec_run( "php -f test.php {host} {port} default 123".format( host=started_cluster.get_instance_ip("node"), port=server_port @@ -764,7 +760,6 @@ def test_mysqljs_client(started_cluster, nodejs_container): def test_java_client(started_cluster, java_container): - # type: (str, Container) -> None with open(os.path.join(SCRIPT_DIR, "java.reference")) as fp: reference = fp.read() @@ -810,6 +805,107 @@ def test_java_client(started_cluster, java_container): assert stdout.decode() == reference +def test_prepared_statements(started_cluster, java_container): + with open(os.path.join(SCRIPT_DIR, "prepared_statements.reference")) as fp: + reference = fp.read() + + node.query(""" + CREATE TABLE ps_simple_data_types + ( + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64, + i128 Int128, + i256 Int256, + ui8 UInt8, + ui16 UInt16, + ui32 UInt32, + ui64 UInt64, + ui128 UInt128, + ui256 UInt256, + f32 Float32, + f64 Float64, + b Boolean + ) ENGINE MergeTree ORDER BY i8; + """, settings={"password": "123"}) + node.query(""" + INSERT INTO ps_simple_data_types + VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, + 57896044618658097711785492504343953926634992332820282019728792003956564819967, + 255, 65535, 4294967295, 18446744073709551615, 340282366920938463463374607431768211455, + 115792089237316195423570985008687907853269984665640564039457584007913129639935, + 1.234, 3.35245141223232, FALSE), + (-128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, + -57896044618658097711785492504343953926634992332820282019728792003956564819968, + 120, 1234, 51234, 421342, 15324355, 41345135123432, + -0.7968956, -0.113259, TRUE); + """, settings={"password": "123"}) + + node.query(""" + CREATE TABLE ps_string_types + ( + s String, + sn Nullable(String), + lc LowCardinality(String), + nlc LowCardinality(Nullable(String)) + ) ENGINE MergeTree ORDER BY s; + """, settings={"password": "123"}) + node.query(""" + INSERT INTO ps_string_types + VALUES ('foo', 'bar', 'qaz', 'qux'), + ('42', NULL, 'test', NULL); + """, settings={"password": "123"}) + + node.query(""" + CREATE TABLE ps_decimal_types + ( + d32 Decimal(9, 2), + d64 Decimal(18, 3), + d128_native Decimal(30, 10), + d128_text Decimal(38, 31), + d256 Decimal(76, 20) + ) ENGINE MergeTree ORDER BY d32; + """, settings={"password": "123"}) + node.query(""" + INSERT INTO ps_decimal_types + VALUES (1234567.89, + 123456789123456.789, + 12345678912345678912.1234567891, + 1234567.8912345678912345678911234567891, + 12345678912345678912345678911234567891234567891234567891.12345678911234567891), + (-1.55, 6.03, 5, -1224124.23423, -54342.3); + """, settings={"password": "123"}) + + node.query(""" + CREATE TABLE ps_misc_types + ( + + a Array(String), + u UUID, + t Tuple(Int32, String), + m Map(String, Int32) + ) ENGINE MergeTree ORDER BY u; + """, settings={"password": "123"}) + node.query(""" + SELECT * + FROM ps_misc_types; + INSERT INTO ps_misc_types + VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255}), + ([], '9a0ccc06-2578-4861-8534-631c9d40f3f7', (0, ''), {}); + """, settings={"password": "123"}) + + code, (stdout, stderr) = java_container.exec_run( + "java PreparedStatementsTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database " + "default".format( + host=started_cluster.get_instance_ip("node"), port=server_port + ), + demux=True, + ) + assert code == 0 + assert stdout.decode() == reference + + def test_types(started_cluster): client = pymysql.connections.Connection( host=started_cluster.get_instance_ip("node"), @@ -844,14 +940,14 @@ def test_types(started_cluster): result = cursor.fetchall()[0] expected = [ - ("Int8_column", -(2**7)), - ("UInt8_column", 2**8 - 1), - ("Int16_column", -(2**15)), - ("UInt16_column", 2**16 - 1), - ("Int32_column", -(2**31)), - ("UInt32_column", 2**32 - 1), - ("Int64_column", -(2**63)), - ("UInt64_column", 2**64 - 1), + ("Int8_column", -(2 ** 7)), + ("UInt8_column", 2 ** 8 - 1), + ("Int16_column", -(2 ** 15)), + ("UInt16_column", 2 ** 16 - 1), + ("Int32_column", -(2 ** 31)), + ("UInt32_column", 2 ** 32 - 1), + ("Int64_column", -(2 ** 63)), + ("UInt64_column", 2 ** 64 - 1), ("String_column", "тест"), ("FixedString_column", "тест"), ("Float32_column", 1.5), From 06afe0c2aa43828d830bca354cf99c3ea65b4534 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 31 Aug 2023 06:27:06 +0000 Subject: [PATCH 055/327] more stable stateless test + add perf. test --- tests/performance/final_with_lonely_parts.xml | 77 +++++++ ...oss_partitions_final_with_lonely.reference | 208 +----------------- ...rge_across_partitions_final_with_lonely.sh | 60 +++++ ...ge_across_partitions_final_with_lonely.sql | 49 ----- 4 files changed, 143 insertions(+), 251 deletions(-) create mode 100644 tests/performance/final_with_lonely_parts.xml create mode 100755 tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh delete mode 100644 tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql diff --git a/tests/performance/final_with_lonely_parts.xml b/tests/performance/final_with_lonely_parts.xml new file mode 100644 index 00000000000..075f639956b --- /dev/null +++ b/tests/performance/final_with_lonely_parts.xml @@ -0,0 +1,77 @@ + + + 1 + 1 + + + + CREATE TABLE with_lonely + ( + `id` UInt64, + `dt` Date, + `val` UInt64, + `version` UInt64 + ) + ENGINE = ReplacingMergeTree(version) + PARTITION BY dt + ORDER BY (id); + + + + INSERT INTO with_lonely SELECT number, '2022-10-28', number*10, 0 FROM numbers(3000000); + + + INSERT INTO with_lonely SELECT number+500000, '2022-10-28', number*10, 1 FROM + numbers(3000000); + + OPTIMIZE TABLE with_lonely PARTITION '2022-10-28' FINAL; + + + INSERT INTO with_lonely SELECT number, '2022-10-29', number*10, 0 FROM numbers(3000000); + + + INSERT INTO with_lonely SELECT number+500000, '2022-10-29', number*10, 1 FROM + numbers(3000000); + + OPTIMIZE TABLE with_lonely PARTITION '2022-10-29' FINAL; + + + INSERT INTO with_lonely SELECT number, '2022-10-30', number*10, 0 FROM numbers(3000000); + + + INSERT INTO with_lonely SELECT number+500000, '2022-10-30', number*10, 1 FROM + numbers(3000000); + + OPTIMIZE TABLE with_lonely PARTITION '2022-10-30' FINAL; + + + INSERT INTO with_lonely SELECT number, '2022-10-31', number*10, 0 FROM numbers(3000000); + + + INSERT INTO with_lonely SELECT number+500000, '2022-10-31', number*10, 1 FROM + numbers(3000000); + + OPTIMIZE TABLE with_lonely PARTITION '2022-10-31' FINAL; + + + INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(500000); + + + INSERT INTO with_lonely SELECT number+50000, '2022-11-01', number*10, 1 FROM + numbers(500000); + + + INSERT INTO with_lonely SELECT number+60000, '2022-11-01', number*10, 2 FROM + numbers(500000); + + + INSERT INTO with_lonely SELECT number+70000, '2022-11-01', number*10, 3 FROM + numbers(500000); + + + + SELECT max(val), count(*) FROM with_lonely FINAL; + SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt != '2022-11-01'; + + DROP TABLE IF EXISTS with_lonely; + diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference index a4b37bc423b..72837206fac 100644 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference @@ -1,202 +1,6 @@ -(Expression) -ExpressionTransform × 16 -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 6 → 16 - Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 6 - Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 6 → 6 - Header × 6 : val UInt64: val UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 6 - Header: val UInt64: val UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeThread × 4 0 → 1 - Header: val UInt64: val UInt64 UInt64(size = 0) - ExpressionTransform × 2 - Header: val UInt64: val UInt64 UInt64(size = 0) - ReplacingSorted 2 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - FilterSortedStreamByRange × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - Description: filter values in [(50000), +inf) - ExpressionTransform × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - MergeTreeInOrder × 2 0 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - ReplacingSorted 2 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - FilterSortedStreamByRange × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - Description: filter values in [-inf, (50000)) - ExpressionTransform × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - MergeTreeInOrder × 2 0 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) -(Expression) -ExpressionTransform × 16 -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 4 → 16 - Header × 16 : max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 4 - Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 4 → 4 - Header × 4 : val UInt64: val UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - (Filter) - FilterTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeThread × 4 0 → 1 - Header: dt Date: dt Date UInt16(size = 0) - val UInt64: val UInt64 UInt64(size = 0) -(Expression) -ExpressionTransform -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 4 → 1 - Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 4 - Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 4 → 4 - Header × 4 : val UInt64: val UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - (Filter) - FilterTransform × 4 - Header: val UInt64: val UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeInOrder × 4 0 → 1 - Header: dt Date: dt Date UInt16(size = 0) - val UInt64: val UInt64 UInt64(size = 0) -(Expression) -ExpressionTransform × 16 -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 6 → 16 - Header × 16 : max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 6 - Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 6 → 6 - Header × 6 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 6 - Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeThread × 4 0 → 1 - Header: val UInt64: val UInt64 UInt64(size = 0) - ExpressionTransform × 2 - Header: val UInt64: val UInt64 UInt64(size = 0) - ReplacingSorted 2 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - FilterSortedStreamByRange × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - Description: filter values in [(50000), +inf) - ExpressionTransform × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - MergeTreeInOrder × 2 0 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - ReplacingSorted 2 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - FilterSortedStreamByRange × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - Description: filter values in [-inf, (50000)) - ExpressionTransform × 2 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) - MergeTreeInOrder × 2 0 → 1 - Header: id UInt64: id UInt64 UInt64(size = 0) - val UInt64: val UInt64 UInt64(size = 0) - version UInt64: version UInt64 UInt64(size = 0) -(Expression) -ExpressionTransform × 16 -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 4 → 16 - Header × 16 : max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 4 - Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 4 → 4 - Header × 4 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 4 - Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (Filter) - FilterTransform × 4 - Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeThread × 4 0 → 1 - Header: val UInt64: val UInt64 UInt64(size = 0) - dt Date: dt Date UInt16(size = 0) -(Expression) -ExpressionTransform -Header: max(val) UInt64: max(val) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - (Aggregating) - Resize 4 → 1 - Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - AggregatingTransform × 4 - Header: max(val_0) UInt64: max(val_0) UInt64 UInt64(size = 0) - count() UInt64: count() UInt64 UInt64(size = 0) - StrictResize 4 → 4 - Header × 4 : val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (Expression) - ExpressionTransform × 4 - Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (Filter) - FilterTransform × 4 - Header: val_0 UInt64: val_0 UInt64 UInt64(size = 0) - (ReadFromMergeTree) - MergeTreeInOrder × 4 0 → 1 - Header: val UInt64: val UInt64 UInt64(size = 0) - dt Date: dt Date UInt16(size = 0) +90 110 +90 80 +90 80 +90 110 +90 80 +90 80 diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh new file mode 100755 index 00000000000..d01e4f94350 --- /dev/null +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh @@ -0,0 +1,60 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -nm -q """ +DROP TABLE IF EXISTS with_lonely; + +CREATE TABLE with_lonely +( + id UInt64, + dt Date, + val UInt64, + version UInt64 +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY dt +ORDER BY (id); +""" + +create_optimize_partition() { + ${CLICKHOUSE_CLIENT} -nm -q """ + INSERT INTO with_lonely SELECT number, '$1', number*10, 0 FROM numbers(10); + INSERT INTO with_lonely SELECT number+500000, '$1', number*10, 1 FROM numbers(10); + """ + cnt=$(${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'with_lonely' AND partition = '$1' AND active") + while [ "$cnt" -ne "1" ]; do + ${CLICKHOUSE_CLIENT} -q "OPTIMIZE TABLE with_lonely PARTITION '$1' FINAL;" + cnt=$(${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 'with_lonely' AND partition = '$1' AND active") + done +} + +create_optimize_partition "2022-10-28" +create_optimize_partition "2022-10-29" +create_optimize_partition "2022-10-30" +create_optimize_partition "2022-10-31" + +${CLICKHOUSE_CLIENT} -nm -q """ +SYSTEM STOP MERGES with_lonely; + +INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(10); +INSERT INTO with_lonely SELECT number+50000, '2022-11-01', number*10, 1 FROM numbers(10); +INSERT INTO with_lonely SELECT number+60000, '2022-11-01', number*10, 2 FROM numbers(10); +""" + +CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --do_not_merge_across_partitions_select_final 1" + +# mix lonely parts and non-lonely parts +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_bytes_to_read 2000 -q "SELECT max(val), count(*) FROM with_lonely FINAL;" +# only lonely parts +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" +# only lonely parts but max_thread = 1, so reading lonely parts with in-order +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_threads 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" + +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_bytes_to_read 2000 -q "SELECT max(val), count(*) FROM with_lonely FINAL;" +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" +${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_threads 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql deleted file mode 100644 index 526c9204d96..00000000000 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sql +++ /dev/null @@ -1,49 +0,0 @@ -DROP TABLE IF EXISTS with_lonely; - -CREATE TABLE with_lonely -( - `id` UInt64, - `dt` Date, - `val` UInt64, - `version` UInt64 -) -ENGINE = ReplacingMergeTree(version) -PARTITION BY dt -ORDER BY (id); - -INSERT INTO with_lonely SELECT number, '2022-10-28', number*10, 0 FROM numbers(10000); -INSERT INTO with_lonely SELECT number+500000, '2022-10-28', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE with_lonely PARTITION '2022-10-28' FINAL; - - -INSERT INTO with_lonely SELECT number, '2022-10-29', number*10, 0 FROM numbers(10000); -INSERT INTO with_lonely SELECT number+500000, '2022-10-29', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE with_lonely PARTITION '2022-10-29' FINAL; - -INSERT INTO with_lonely SELECT number, '2022-10-30', number*10, 0 FROM numbers(10000); -INSERT INTO with_lonely SELECT number+500000, '2022-10-30', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE with_lonely PARTITION '2022-10-30' FINAL; - - -INSERT INTO with_lonely SELECT number, '2022-10-31', number*10, 0 FROM numbers(10000); -INSERT INTO with_lonely SELECT number+500000, '2022-10-31', number*10, 1 FROM numbers(10000); -OPTIMIZE TABLE with_lonely PARTITION '2022-10-31' FINAL; - -SYSTEM STOP MERGES with_lonely; -INSERT INTO with_lonely SELECT number, '2022-11-01', number*10, 0 FROM numbers(1000); -INSERT INTO with_lonely SELECT number+50000, '2022-11-01', number*10, 1 FROM numbers(1000); -INSERT INTO with_lonely SELECT number+60000, '2022-11-01', number*10, 2 FROM numbers(1000); - -SET do_not_merge_across_partitions_select_final = 1, max_threads = 16; - --- mix lonely parts and non-lonely parts -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL SETTINGS allow_experimental_analyzer = 0; --- only lonely parts -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS allow_experimental_analyzer = 0; --- only lonely parts but max_thread = 1, so reading lonely parts with in-order -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS max_threads = 1, allow_experimental_analyzer = 0; - - -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL SETTINGS allow_experimental_analyzer = 1; -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS allow_experimental_analyzer = 1; -EXPLAIN PIPELINE header=1 SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01' SETTINGS max_threads = 1, allow_experimental_analyzer = 1; From 8483d1c701c56db00296ea5ad712e35f527c0579 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Thu, 31 Aug 2023 13:26:27 +0200 Subject: [PATCH 056/327] Binary protocol tests --- .../integration/mysql_java_client/Dockerfile | 13 +-- .../PreparedStatementsTest.java | 36 ++++---- .../integration/mysql_java_client/Test.java | 3 +- src/Core/MySQL/PacketsProtocolBinary.cpp | 84 ++++++++++++------ src/Core/MySQL/PacketsProtocolText.cpp | 1 + .../Serializations/SerializationDate32.cpp | 14 +-- .../prepared_statements.reference | 82 ++++++++++++++++- .../prepared_statements_test.sql | 86 ++++++++++++++++++ tests/integration/test_mysql_protocol/test.py | 88 +------------------ 9 files changed, 262 insertions(+), 145 deletions(-) create mode 100644 tests/integration/test_mysql_protocol/prepared_statements_test.sql diff --git a/docker/test/integration/mysql_java_client/Dockerfile b/docker/test/integration/mysql_java_client/Dockerfile index 0abf50cd493..75e3c9de138 100644 --- a/docker/test/integration/mysql_java_client/Dockerfile +++ b/docker/test/integration/mysql_java_client/Dockerfile @@ -1,10 +1,10 @@ # docker build -t clickhouse/mysql-java-client . # MySQL Java client docker container -FROM ubuntu:18.04 +FROM ubuntu:22.04 RUN apt-get update && \ - apt-get install -y software-properties-common build-essential openjdk-8-jdk libmysql-java curl + apt-get install -y software-properties-common build-essential openjdk-8-jdk curl RUN rm -rf \ /var/lib/apt/lists/* \ @@ -12,10 +12,11 @@ RUN rm -rf \ /tmp/* \ RUN apt-get clean -ARG ver=5.1.46 -RUN curl -L -o /mysql-connector-java-${ver}.jar https://repo1.maven.org/maven2/mysql/mysql-connector-java/${ver}/mysql-connector-java-${ver}.jar -ENV CLASSPATH=$CLASSPATH:/mysql-connector-java-${ver}.jar +ARG ver=8.1.0 +RUN curl -L -o /mysql-connector-j-${ver}.jar https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/${ver}/mysql-connector-j-${ver}.jar +ENV CLASSPATH=$CLASSPATH:/mysql-connector-j-${ver}.jar WORKDIR /jdbc COPY Test.java Test.java -RUN javac Test.java +COPY PreparedStatementsTest.java PreparedStatementsTest.java +RUN javac Test.java PreparedStatementsTest.java diff --git a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java index 0534f2b761d..cb479e13099 100644 --- a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java +++ b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java @@ -33,20 +33,19 @@ public class PreparedStatementsTest { } } + // useServerPrepStmts allows us to send COM_STMT_PREPARE and COM_STMT_EXECUTE + // to test the binary protocol implementation + String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true", + host, port, database); + try { Class.forName("com.mysql.cj.jdbc.Driver"); - /* - * useServerPrepStmts allows us to send COM_STMT_PREPARE and COM_STMT_EXECUTE to - * test the binary protocol - */ - String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true", - host, port, database); Connection conn = DriverManager.getConnection(jdbcUrl, user, password); testSimpleDataTypes(conn); testStringTypes(conn); testDecimalTypes(conn); testMiscTypes(conn); - // testDateTypes(conn); + testDateTypes(conn); conn.close(); } catch (Exception e) { e.printStackTrace(); @@ -99,20 +98,15 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %s\n", - getMysqlType(rs, "d32"), + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d32"), rs.getBigDecimal("d32").toPlainString()); - System.out.printf("Type: %s, value: %s\n", - getMysqlType(rs, "d64"), + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d64"), rs.getBigDecimal("d64").toPlainString()); - System.out.printf("Type: %s, value: %s\n", - getMysqlType(rs, "d128_native"), + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d128_native"), rs.getBigDecimal("d128_native").toPlainString()); - System.out.printf("Type: %s, value: %s\n", - getMysqlType(rs, "d128_text"), + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d128_text"), rs.getString("d128_text")); - System.out.printf("Type: %s, value: %s\n", - getMysqlType(rs, "d256"), + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d256"), rs.getString("d256")); } System.out.println(); @@ -127,14 +121,16 @@ public class PreparedStatementsTest { System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d32"), rs.getDate("d32")); System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64"), rs.getTimestamp("dt64")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getString("dt64_3")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_6"), rs.getString("dt64_6")); + System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_9"), rs.getString("dt64_9")); } System.out.println(); } private static void testMiscTypes(Connection conn) throws SQLException { System.out.println("### testMiscTypes"); - ResultSet rs = conn.prepareStatement("SELECT d FROM ps_misc_types").executeQuery(); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_misc_types").executeQuery(); int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); @@ -149,7 +145,5 @@ public class PreparedStatementsTest { private static MysqlType getMysqlType(ResultSet rs, String columnLabel) throws SQLException { ResultSetMetaData meta = rs.getMetaData(); return MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel))); - } - } diff --git a/docker/test/integration/mysql_java_client/Test.java b/docker/test/integration/mysql_java_client/Test.java index 2e256d5dc44..752441718a9 100644 --- a/docker/test/integration/mysql_java_client/Test.java +++ b/docker/test/integration/mysql_java_client/Test.java @@ -46,6 +46,7 @@ class JavaConnectorTest { Connection conn = null; Statement stmt = null; try { + Class.forName("com.mysql.cj.jdbc.Driver"); conn = DriverManager.getConnection(jdbcUrl, user, password); stmt = conn.createStatement(); stmt.executeUpdate(CREATE_TABLE_SQL); @@ -69,7 +70,7 @@ class JavaConnectorTest { stmt.close(); conn.close(); - } catch (SQLException e) { + } catch (Exception e) { e.printStackTrace(); System.exit(1); } diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 2b5dce78ab3..ebb256cb462 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -2,17 +2,18 @@ #include #include #include -#include -#include -#include -#include "Common/logger_useful.h" +#include "Common/LocalDate.h" +#include "Common/LocalDateTime.h" #include "Columns/ColumnLowCardinality.h" #include "Columns/ColumnVector.h" +#include "Columns/ColumnsDateTime.h" #include "DataTypes/DataTypeLowCardinality.h" #include "DataTypes/DataTypeNullable.h" #include "DataTypes/DataTypesNumber.h" #include "Formats/FormatSettings.h" #include "IO/WriteBufferFromString.h" +#include "base/DayNum.h" +#include "base/Decimal.h" #include "base/types.h" namespace DB @@ -29,12 +30,10 @@ namespace MySQLProtocol { /// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row payload_size = 1 + null_bitmap_size; - LOG_TRACE(&Poco::Logger::get("ResultSetRow"), "Null bitmap size: {}", null_bitmap_size); FormatSettings format_settings; for (size_t i = 0; i < columns.size(); ++i) { ColumnPtr col = columns[i]; - LOG_TRACE(&Poco::Logger::get("col->isNullAt"), "isNullAt: {}, {}", row_num, col->isNullAt(row_num)); if (col->isNullAt(row_num)) { size_t byte = (i + 2) / 8; @@ -75,7 +74,19 @@ namespace MySQLProtocol payload_size += 8; break; case TypeIndex::Date: { - UInt64 value = col->get64(row_num); + UInt16 value = assert_cast &>(*col).getData()[row_num]; + if (value == 0) + { + payload_size += 1; // length only, no other fields + } + else + { + payload_size += 5; + } + break; + } + case TypeIndex::Date32: { + Int32 value = assert_cast &>(*col).getData()[row_num]; if (value == 0) { payload_size += 1; // length only, no other fields @@ -87,15 +98,15 @@ namespace MySQLProtocol break; } case TypeIndex::DateTime: { - UInt64 value = col->get64(row_num); + UInt32 value = assert_cast &>(*col).getData()[row_num]; if (value == 0) { payload_size += 1; // length only, no other fields } else { - Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); - if (dt.second() == 0 && dt.minute() == 0 && dt.hour() == 0) + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); + if (ldt.second() == 0 && ldt.minute() == 0 && ldt.hour() == 0) { payload_size += 5; } @@ -107,7 +118,6 @@ namespace MySQLProtocol break; } default: - LOG_TRACE(&Poco::Logger::get("Type default"), "{} is {}", col->getName(), data_type->getName()); WriteBufferFromOwnString ostr; serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); payload_size += getLengthEncodedStringSize(ostr.str()); @@ -188,14 +198,33 @@ namespace MySQLProtocol break; } case TypeIndex::Date: { - UInt64 value = assert_cast &>(*col).getData()[row_num]; + UInt16 value = assert_cast &>(*col).getData()[row_num]; if (value != 0) { - Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); + LocalDate ld = LocalDate(DayNum(value)); buffer.write(static_cast(4)); // bytes_following - int year = dt.year(); - int month = dt.month(); - int day = dt.day(); + auto year = ld.year(); + auto month = ld.month(); + auto day = ld.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + } + else + { + buffer.write(static_cast(0)); + } + break; + } + case TypeIndex::Date32: { + Int32 value = assert_cast &>(*col).getData()[row_num]; + if (value != 0) + { + LocalDate ld = LocalDate(ExtendedDayNum(value)); + buffer.write(static_cast(4)); // bytes_following + auto year = ld.year(); + auto month = ld.month(); + auto day = ld.day(); buffer.write(reinterpret_cast(&year), 2); buffer.write(reinterpret_cast(&month), 1); buffer.write(reinterpret_cast(&day), 1); @@ -207,24 +236,24 @@ namespace MySQLProtocol break; } case TypeIndex::DateTime: { - UInt64 value = assert_cast &>(*col).getData()[row_num]; + UInt32 value = assert_cast &>(*col).getData()[row_num]; if (value != 0) { - Poco::DateTime dt = Poco::DateTime(Poco::Timestamp(value * 1000 * 1000)); - bool is_date_time = !(dt.hour() == 0 && dt.minute() == 0 && dt.second() == 0); - size_t bytes_following = is_date_time ? 7 : 4; + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); + int year = ldt.year(); + int month = ldt.month(); + int day = ldt.day(); + int hour = ldt.hour(); + int minute = ldt.minute(); + int second = ldt.second(); + bool has_time = !(hour == 0 && minute == 0 && second == 0); + size_t bytes_following = has_time ? 7 : 4; buffer.write(reinterpret_cast(&bytes_following), 1); - int year = dt.year(); - int month = dt.month(); - int day = dt.day(); buffer.write(reinterpret_cast(&year), 2); buffer.write(reinterpret_cast(&month), 1); buffer.write(reinterpret_cast(&day), 1); - if (is_date_time) + if (has_time) { - int hour = dt.hourAMPM(); - int minute = dt.minute(); - int second = dt.second(); buffer.write(reinterpret_cast(&hour), 1); buffer.write(reinterpret_cast(&minute), 1); buffer.write(reinterpret_cast(&second), 1); @@ -236,6 +265,7 @@ namespace MySQLProtocol } break; } + default: writeLengthEncodedString(serialized[i], buffer); break; diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index bb5412bb5a5..712b203bc2a 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -203,6 +203,7 @@ namespace MySQLProtocol decimals = 31; break; case TypeIndex::Date: + case TypeIndex::Date32: column_type = ColumnType::MYSQL_TYPE_DATE; flags = ColumnDefinitionFlags::BINARY_FLAG; break; diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index 851710de839..68213a14afc 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -28,12 +28,14 @@ void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } -void SerializationDate32::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextEscaped( + const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); } -void SerializationDate32::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextQuoted( + const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('\'', ostr); serializeText(column, row_num, ostr, settings); @@ -46,10 +48,11 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i assertChar('\'', istr); readDateText(x, istr, time_zone); assertChar('\'', istr); - assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } -void SerializationDate32::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextJSON( + const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); serializeText(column, row_num, ostr, settings); @@ -65,7 +68,8 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist assert_cast(column).getData().push_back(x); } -void SerializationDate32::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextCSV( + const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); serializeText(column, row_num, ostr, settings); diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference index 257cc5642cb..3e23620c07e 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements.reference +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -1 +1,81 @@ -foo +### testSimpleDataTypes +Row #1 +Type: TINYINT, value: -128 +Type: SMALLINT, value: -32768 +Type: INT, value: -2147483648 +Type: BIGINT, value: -9223372036854775808 +Type: CHAR, value: -170141183460469231731687303715884105728 +Type: CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968 +Type: TINYINT, value: 120 +Type: SMALLINT, value: 1234 +Type: INT, value: 51234 +Type: BIGINT, value: 421342 +Type: CHAR, value: 15324355 +Type: CHAR, value: 41345135123432 +Type: FLOAT, value: -0.796896 +Type: DOUBLE, value: -0.113259 +Type: BIT, value: true +Row #2 +Type: TINYINT, value: 127 +Type: SMALLINT, value: 32767 +Type: INT, value: 2147483647 +Type: BIGINT, value: 9223372036854775807 +Type: CHAR, value: 170141183460469231731687303715884105727 +Type: CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967 +Type: TINYINT, value: 255 +Type: SMALLINT, value: 65535 +Type: INT, value: 4294967295 +Type: BIGINT, value: 18446744073709551615 +Type: CHAR, value: 340282366920938463463374607431768211455 +Type: CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 +Type: FLOAT, value: 1.234000 +Type: DOUBLE, value: 3.352451 +Type: BIT, value: false + +### testStringTypes +Row #1 +Type: CHAR, value: 42 +Type: CHAR, value: null +Type: CHAR, value: test +Type: CHAR, value: null +Row #2 +Type: CHAR, value: foo +Type: CHAR, value: bar +Type: CHAR, value: qaz +Type: CHAR, value: qux + +### testDecimalTypes +Row #1 +Type: DECIMAL, value: -1.55 +Type: DECIMAL, value: 6.03 +Type: DECIMAL, value: 5 +Type: CHAR, value: -1224124.23423 +Type: CHAR, value: -54342.3 +Row #2 +Type: DECIMAL, value: 1234567.89 +Type: DECIMAL, value: 123456789123456.789 +Type: DECIMAL, value: 12345678912345678912.1234567891 +Type: CHAR, value: 1234567.8912345678912345678911234567891 +Type: CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891 + +### testMiscTypes +Row #1 +Type: CHAR, value: [] +Type: CHAR, value: 9a0ccc06-2578-4861-8534-631c9d40f3f7 +Type: CHAR, value: (0,'') +Type: CHAR, value: {} +Row #2 +Type: CHAR, value: ['foo','bar'] +Type: CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 +Type: CHAR, value: (42,'qaz') +Type: CHAR, value: {'qux':144,'text':255} + +### testDateTypes +Row #1 +Type: DATE, value: 2149-06-06 +Type: DATE, value: 2178-04-16 +Type: TIMESTAMP, value: 2106-02-07 06:28:15.0 +Type: CHAR, value: 2106-02-07 06:28:15.123 +Type: CHAR, value: 2106-02-07 06:28:15.123456 +Type: CHAR, value: 2106-02-07 06:28:15.123456789 + diff --git a/tests/integration/test_mysql_protocol/prepared_statements_test.sql b/tests/integration/test_mysql_protocol/prepared_statements_test.sql new file mode 100644 index 00000000000..4fb53a14a61 --- /dev/null +++ b/tests/integration/test_mysql_protocol/prepared_statements_test.sql @@ -0,0 +1,86 @@ +CREATE TABLE ps_simple_data_types +( + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64, + i128 Int128, + i256 Int256, + ui8 UInt8, + ui16 UInt16, + ui32 UInt32, + ui64 UInt64, + ui128 UInt128, + ui256 UInt256, + f32 Float32, + f64 Float64, + b Boolean +) ENGINE MergeTree ORDER BY i8; + +INSERT INTO ps_simple_data_types +VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, + 57896044618658097711785492504343953926634992332820282019728792003956564819967, + 255, 65535, 4294967295, 18446744073709551615, 340282366920938463463374607431768211455, + 115792089237316195423570985008687907853269984665640564039457584007913129639935, + 1.234, 3.35245141223232, FALSE), + (-128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, + -57896044618658097711785492504343953926634992332820282019728792003956564819968, + 120, 1234, 51234, 421342, 15324355, 41345135123432, + -0.7968956, -0.113259, TRUE); + +CREATE TABLE ps_string_types +( + s String, + sn Nullable(String), + lc LowCardinality(String), + nlc LowCardinality(Nullable(String)) +) ENGINE MergeTree ORDER BY s; + +INSERT INTO ps_string_types +VALUES ('foo', 'bar', 'qaz', 'qux'), + ('42', NULL, 'test', NULL); + +CREATE TABLE ps_decimal_types +( + d32 Decimal(9, 2), + d64 Decimal(18, 3), + d128_native Decimal(30, 10), + d128_text Decimal(38, 31), + d256 Decimal(76, 20) +) ENGINE MergeTree ORDER BY d32; + +INSERT INTO ps_decimal_types +VALUES (1234567.89, + 123456789123456.789, + 12345678912345678912.1234567891, + 1234567.8912345678912345678911234567891, + 12345678912345678912345678911234567891234567891234567891.12345678911234567891), + (-1.55, 6.03, 5, -1224124.23423, -54342.3); + +CREATE TABLE ps_misc_types +( + a Array(String), + u UUID, + t Tuple(Int32, String), + m Map(String, Int32) +) ENGINE MergeTree ORDER BY u; + +INSERT INTO ps_misc_types +VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255}), + ([], '9a0ccc06-2578-4861-8534-631c9d40f3f7', (0, ''), {}); + +CREATE TABLE ps_date_types +( + d Date, + d32 Date32, + dt DateTime, + dt64_3 DateTime64(3, 'UTC'), + dt64_6 DateTime64(6, 'UTC'), + dt64_9 DateTime64(9, 'UTC') +) ENGINE MergeTree ORDER BY d; + +INSERT INTO ps_date_types +VALUES ('2149-06-06', '2178-04-16', '2106-02-07 06:28:15', + '2106-02-07 06:28:15.123', + '2106-02-07 06:28:15.123456', + '2106-02-07 06:28:15.123456789'); diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index c0bc178c5bb..19a52da1158 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -809,91 +809,11 @@ def test_prepared_statements(started_cluster, java_container): with open(os.path.join(SCRIPT_DIR, "prepared_statements.reference")) as fp: reference = fp.read() - node.query(""" - CREATE TABLE ps_simple_data_types - ( - i8 Int8, - i16 Int16, - i32 Int32, - i64 Int64, - i128 Int128, - i256 Int256, - ui8 UInt8, - ui16 UInt16, - ui32 UInt32, - ui64 UInt64, - ui128 UInt128, - ui256 UInt256, - f32 Float32, - f64 Float64, - b Boolean - ) ENGINE MergeTree ORDER BY i8; - """, settings={"password": "123"}) - node.query(""" - INSERT INTO ps_simple_data_types - VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, - 57896044618658097711785492504343953926634992332820282019728792003956564819967, - 255, 65535, 4294967295, 18446744073709551615, 340282366920938463463374607431768211455, - 115792089237316195423570985008687907853269984665640564039457584007913129639935, - 1.234, 3.35245141223232, FALSE), - (-128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, - -57896044618658097711785492504343953926634992332820282019728792003956564819968, - 120, 1234, 51234, 421342, 15324355, 41345135123432, - -0.7968956, -0.113259, TRUE); - """, settings={"password": "123"}) + with open(os.path.join(SCRIPT_DIR, "prepared_statements_test.sql")) as sql: + statements = list(filter(lambda s: s != '', map(lambda s: s.strip().replace('\n', ' '), sql.read().split(';')))) - node.query(""" - CREATE TABLE ps_string_types - ( - s String, - sn Nullable(String), - lc LowCardinality(String), - nlc LowCardinality(Nullable(String)) - ) ENGINE MergeTree ORDER BY s; - """, settings={"password": "123"}) - node.query(""" - INSERT INTO ps_string_types - VALUES ('foo', 'bar', 'qaz', 'qux'), - ('42', NULL, 'test', NULL); - """, settings={"password": "123"}) - - node.query(""" - CREATE TABLE ps_decimal_types - ( - d32 Decimal(9, 2), - d64 Decimal(18, 3), - d128_native Decimal(30, 10), - d128_text Decimal(38, 31), - d256 Decimal(76, 20) - ) ENGINE MergeTree ORDER BY d32; - """, settings={"password": "123"}) - node.query(""" - INSERT INTO ps_decimal_types - VALUES (1234567.89, - 123456789123456.789, - 12345678912345678912.1234567891, - 1234567.8912345678912345678911234567891, - 12345678912345678912345678911234567891234567891234567891.12345678911234567891), - (-1.55, 6.03, 5, -1224124.23423, -54342.3); - """, settings={"password": "123"}) - - node.query(""" - CREATE TABLE ps_misc_types - ( - - a Array(String), - u UUID, - t Tuple(Int32, String), - m Map(String, Int32) - ) ENGINE MergeTree ORDER BY u; - """, settings={"password": "123"}) - node.query(""" - SELECT * - FROM ps_misc_types; - INSERT INTO ps_misc_types - VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255}), - ([], '9a0ccc06-2578-4861-8534-631c9d40f3f7', (0, ''), {}); - """, settings={"password": "123"}) + for statement in statements: + node.query(statement, settings={"password": "123"}) code, (stdout, stderr) = java_container.exec_run( "java PreparedStatementsTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database " From 7739dd7157b1cd04d05746514aa9ce83f1b437a0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 31 Aug 2023 14:52:42 +0000 Subject: [PATCH 057/327] Fix integration tests --- programs/server/Server.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 4 ++-- src/Coordination/CoordinationSettings.cpp | 3 +++ tests/integration/helpers/cluster.py | 2 +- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6f94e483e20..6d49fde1691 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1425,7 +1425,7 @@ try const auto interserver_listen_hosts = getInterserverListenHosts(config()); const auto listen_try = getListenTry(config()); - if (config().has("keeper_server")) + if (config().has("keeper_server.server_id")) { #if USE_NURAFT //// If we don't have configured connection probably someone trying to use clickhouse-server instead diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 8b719265869..d099b09827c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1427,7 +1427,7 @@ void validateZooKeeperConfig(const Poco::Util::AbstractConfiguration & config) bool hasZooKeeperConfig(const Poco::Util::AbstractConfiguration & config) { - return config.has("zookeeper") || config.has("keeper") || (config.has("keeper_server") && config.getBool("keeper_server.use_cluster", true)); + return config.has("zookeeper") || config.has("keeper") || (config.has("keeper_server.raft_configuration") && config.getBool("keeper_server.use_cluster", true)); } String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config) @@ -1438,7 +1438,7 @@ String getZooKeeperConfigName(const Poco::Util::AbstractConfiguration & config) if (config.has("keeper")) return "keeper"; - if (config.has("keeper_server") && config.getBool("keeper_server.use_cluster", true)) + if (config.has("keeper_server.raft_configuration") && config.getBool("keeper_server.use_cluster", true)) return "keeper_server"; throw DB::Exception(DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG, "There is no Zookeeper configuration in server config"); diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 61e08f9fde3..aadedf19204 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -152,6 +152,9 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const writeText("raft_limits_reconnect_limit=", buf); write_int(static_cast(coordination_settings->raft_limits_reconnect_limit)); + + writeText("async_replication=", buf); + write_bool(coordination_settings->async_replication); } KeeperConfigurationAndSettingsPtr diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c48bd4303c0..708614ed48c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4260,7 +4260,7 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - # write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) + write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) logging.debug("Generate and write macros file") macros = self.macros.copy() From 3999994eb4d973ef56af829dac0fba1352d54478 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 31 Aug 2023 15:07:26 +0000 Subject: [PATCH 058/327] Automatic style fix --- tests/integration/helpers/cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 708614ed48c..1e92bdaa788 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4260,7 +4260,9 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - write_embedded_config("0_common_enable_keeper_async_replication.xml", self.config_d_dir) + write_embedded_config( + "0_common_enable_keeper_async_replication.xml", self.config_d_dir + ) logging.debug("Generate and write macros file") macros = self.macros.copy() From 5f1a75ceae6960598a4f440aa62a04b85bfacf56 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 1 Sep 2023 07:39:36 +0000 Subject: [PATCH 059/327] Fix tests --- docker/test/upgrade/run.sh | 12 +++++++++++ src/Coordination/Changelog.cpp | 2 +- src/Coordination/CoordinationSettings.cpp | 21 ++++++++++++------- tests/integration/helpers/cluster.py | 7 ++++--- .../test_keeper_four_word_command/test.py | 4 +++- 5 files changed, 33 insertions(+), 13 deletions(-) diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 0b30ab9dbf7..886efe3ff33 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -60,6 +60,12 @@ install_packages previous_release_package_folder # available for dump via clickhouse-local configure +# async_replication setting doesn't exist on some older versions +sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ + | sed "/1<\/async_replication>/d" \ + > /etc/clickhouse-server/config.d/keeper_port.xml.tmp +sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml + # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -82,6 +88,12 @@ sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ > /etc/clickhouse-server/config.d/keeper_port.xml.tmp sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml +# async_replication setting doesn't exist on some older versions +sudo cat /etc/clickhouse-server/config.d/keeper_port.xml \ + | sed "/1<\/async_replication>/d" \ + > /etc/clickhouse-server/config.d/keeper_port.xml.tmp +sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml + # But we still need default disk because some tables loaded only into it sudo cat /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml \ | sed "s|
s3
|
s3
default|" \ diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 2af68173588..7030775904c 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1021,7 +1021,7 @@ void Changelog::writeThread() const auto flush_logs = [&](const auto & flush) { - LOG_TRACE(log, "Flushing {} logs", pending_appends); + LOG_TEST(log, "Flushing {} logs", pending_appends); { std::lock_guard writer_lock(writer_mutex); diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index aadedf19204..8688f6f5a40 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -22,16 +22,19 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_elem, config_keys); - try + for (const String & key : config_keys) { - for (const String & key : config_keys) + try + { set(key, config.getString(config_elem + "." + key)); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_SETTING) - e.addMessage("in Coordination settings config"); - throw; + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + LOG_WARNING(&Poco::Logger::get("CoordinationSettings"), "Found unknown coordination setting in config: '{}'", key); + else + throw; + } } } @@ -134,6 +137,8 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const write_int(coordination_settings->max_requests_batch_size); writeText("max_requests_batch_bytes_size=", buf); write_int(coordination_settings->max_requests_batch_bytes_size); + writeText("max_flush_batch_size=", buf); + write_int(coordination_settings->max_flush_batch_size); writeText("max_request_queue_size=", buf); write_int(coordination_settings->max_request_queue_size); writeText("max_requests_quick_batch_size=", buf); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1e92bdaa788..7ee86d74d35 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4260,9 +4260,10 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - write_embedded_config( - "0_common_enable_keeper_async_replication.xml", self.config_d_dir - ) + if self.tag == "latest": + write_embedded_config( + "0_common_enable_keeper_async_replication.xml", self.config_d_dir + ) logging.debug("Generate and write macros file") macros = self.macros.copy() diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 1d5bc6a6541..0de7de8c3be 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -279,8 +279,9 @@ def test_cmd_conf(started_cluster): assert result["stale_log_gap"] == "10000" assert result["fresh_log_gap"] == "200" - assert result["max_requests_batch_size"] == "100" + assert result["max_requests_batch_size"] == "1000" assert result["max_requests_batch_bytes_size"] == "102400" + assert result["max_flush_batch_size"] == "1000" assert result["max_request_queue_size"] == "100000" assert result["max_requests_quick_batch_size"] == "100" assert result["quorum_reads"] == "false" @@ -290,6 +291,7 @@ def test_cmd_conf(started_cluster): assert result["compress_snapshots_with_zstd_format"] == "true" assert result["configuration_change_tries_count"] == "20" + assert result["async_replication"] == "true" finally: close_keeper_socket(client) From ef4dcf909fa0e8b447fb881e273021b256e85ae2 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 1 Sep 2023 16:23:15 +0000 Subject: [PATCH 060/327] fix build --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index b755a4e0b2a..8d7e9c60e31 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -66,8 +66,8 @@ DiskObjectStorage::DiskObjectStorage( , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(config.getBool(config_prefix + ".send_metadata", false)) - , read_resource_name(config.getString(config_prefix + ".read_resource", "") - , write_resource_name(config.getString(config_prefix + ".write_resource", "") + , read_resource_name(config.getString(config_prefix + ".read_resource", "")) + , write_resource_name(config.getString(config_prefix + ".write_resource", "")) , metadata_helper(std::make_unique(this, ReadSettings{})) {} From 14e2032d929f325ad2188032df50a2ab199d69a8 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 1 Sep 2023 16:28:03 +0000 Subject: [PATCH 061/327] add documentation --- .../mergetree-family/mergetree.md | 4 + docs/en/operations/workload-scheduling.md | 149 ++++++++++++++++++ 2 files changed, 153 insertions(+) create mode 100644 docs/en/operations/workload-scheduling.md diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4f506126682..5d0f9f5d6c6 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1139,6 +1139,8 @@ Optional parameters: - `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. +- `read_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of read requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). +- `write_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of write requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). ### Configuring the cache @@ -1251,6 +1253,8 @@ Other parameters: * `cache_enabled` - Allows to cache mark and index files on local FS. Default value is `true`. * `cache_path` - Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. * `skip_access_check` - If true, disk access checks will not be performed on disk start-up. Default value is `false`. +* `read_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of read requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). +* `write_resource` — Resource name to be used for [scheduling](/docs/en/operations/workload-scheduling.md) of write requests to this disk. Default value is empty string (IO scheduling is not enabled for this disk). Examples of working configurations can be found in integration tests directory (see e.g. [test_merge_tree_azure_blob_storage](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml) or [test_azure_blob_storage_zero_copy_replication](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml)). diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md new file mode 100644 index 00000000000..1761f99b469 --- /dev/null +++ b/docs/en/operations/workload-scheduling.md @@ -0,0 +1,149 @@ +--- +slug: /en/operations/workload-scheduling +sidebar_position: 69 +sidebar_label: "Workload scheduling" +title: "Workload scheduling" +--- + +When ClickHouse execute multiple queries simultaneously, they may be using shared resources (e.g. disks). Scheduling constraints and policies can be applied to regulate how resources are utilized and shared between different workloads. For every resource a scheduling hierarchy can be configured. Hierarchy root represents a resource, while leafs are queues, holding requests that exceed resource capacity. + +:::note +Currently only remote disk IO can be scheduled using described method. For CPU scheduling see settings about thread pools and [`concurrent_threads_soft_limit_num`](server-configuration-parameters/settings.md#concurrent_threads_soft_limit_num). For flexible memory limits see [Memory overcommit](settings/memory-overcommit.md) +::: + +## Disk configuration + +To enable IO scheduling for a specific disk, you have to specify `read_resource` and/or `write_resource` in storage configuration. It says ClickHouse what resource should be used for every read and write requests with given disk. Read and write resource can refer to the same resource name, which is useful for local SSDs or HDDs. Multiple different disks also can refer to the same resource, which is useful for remote disks: if you want to be able to allow fair division of network bandwidth between e.g. "production" and "development" workloads. + +Example: +```xml + + + ... + + + s3 + https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/root-path/ + your_access_key_id + your_secret_access_key + network_read + network_write + + + + + +
+ s3 +
+
+
+
+
+
+``` + +## Workload markup {#workload_markup} + +Queries can be marked with setting `workload` to distinguish different workloads. If `workload` is not set, than value "default" is used. Note that you are able to specify the other value using settings profiles. Setting constraints can be used to make `workload` constant if you want all queries from the user to be marked with fixed value of `workload` setting. + +Let's consider an example of a system with two different workloads: "production" and "development". + +```sql +SELECT count() FROM my_table WHERE value = 42 SETTINGS workload = 'production' +SELECT count() FROM my_table WHERE value = 13 SETTINGS workload = 'development' +``` + +## Resource scheduling hierarchy + +From the standpoint of scheduling subsystem a resource represents a hierarchy of scheduling nodes. + +```mermaid +graph TD + subgraph network_read + nr_root(("/")) + -->|100 concurrent requests| nr_fair("fair") + -->|75% bandwidth| nr_prod["prod"] + nr_fair + -->|25% bandwidth| nr_dev["dev"] + end + + subgraph network_write + nw_root(("/")) + -->|100 concurrent requests| nw_fair("fair") + -->|75% bandwidth| nw_prod["prod"] + nw_fair + -->|25% bandwidth| nw_dev["dev"] + end +``` + +**Possible node types:** +* inflight_limit (constraint) - blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost`; must have a single child. +* fair (policy) - selects the next request to serve from one of its children nodes according to max-min fairness; children nodes can specify `weight` (default is 1). +* priority (policy) - selects the next request to serve from one of its children nodes according to static priorities (lower value means higher priority); children nodes can specify `priority` (default is 0). +* fifo (queue) - leaf of the hierarchy capable of holding requests that exceed resource capacity. + +The following example shows how to define IO scheduling hierarchies shown in the picture: + +```xml + + + + + inflight_limit + 100 + + + fair + + + fifo + 3 + + + fifo + + + + + inflight_limit + 100 + + + fair + + + fifo + 3 + + + fifo + + + + +``` + +## Workload classifiers + +Workload classifiers are used to define mapping from `workload` specified by a query into leaf-queues that should be used for specific resources. At the moment, workload classification is simple: only static mapping is available. + +Example: +```xml + + + + /fair/prod + /fair/prod + + + /fair/dev + /fair/dev + + + /fair/dev + /fair/dev + + + +``` From 096b44e1e5dcf46e081b9d8a123462c222c8fd0b Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 1 Sep 2023 23:34:47 -0400 Subject: [PATCH 062/327] Implementing `reconfig` command. Parser improvements --- programs/keeper-client/Commands.cpp | 93 ++++++++++++++++++------- programs/keeper-client/Commands.h | 17 +++++ programs/keeper-client/KeeperClient.cpp | 63 ++++++++++------- programs/keeper-client/KeeperClient.h | 4 +- programs/keeper-client/Parser.cpp | 33 ++++----- src/Common/ZooKeeper/ZooKeeper.cpp | 39 +++++++++++ src/Common/ZooKeeper/ZooKeeper.h | 13 ++++ 7 files changed, 189 insertions(+), 73 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index c822a631798..e265d7f4858 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -9,11 +9,11 @@ namespace DB bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return true; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -42,11 +42,11 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con bool CDCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return true; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -64,11 +64,12 @@ void CDCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con bool SetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return false; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); + String arg; if (!parseKeeperArg(pos, expected, arg)) return false; node->args.push_back(std::move(arg)); @@ -93,11 +94,12 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return false; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); + String arg; if (!parseKeeperArg(pos, expected, arg)) return false; node->args.push_back(std::move(arg)); @@ -143,10 +145,10 @@ void TouchCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) bool GetCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return false; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -158,11 +160,11 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return true; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -325,10 +327,10 @@ void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return false; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -340,10 +342,10 @@ void RMCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { - String arg; - if (!parseKeeperPath(pos, expected, arg)) + String path; + if (!parseKeeperPath(pos, expected, path)) return false; - node->args.push_back(std::move(arg)); + node->args.push_back(std::move(path)); return true; } @@ -355,6 +357,49 @@ void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co [client, path]{ client->zookeeper->removeRecursive(path); }); } +bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr & node, DB::Expected & expected) const +{ + ReconfigCommand::Operation operation; + if (ParserKeyword{"ADD"}.ignore(pos, expected)) + operation = ReconfigCommand::Operation::ADD; + else if (ParserKeyword{"REMOVE"}.ignore(pos, expected)) + operation = ReconfigCommand::Operation::REMOVE; + else if (ParserKeyword{"SET"}.ignore(pos, expected)) + operation = ReconfigCommand::Operation::SET; + else + return false; + + node->args.push_back(operation); + ParserToken{TokenType::Whitespace}.ignore(pos); + + String arg; + if (!parseKeeperArg(pos, expected, arg)) + return false; + node->args.push_back(std::move(arg)); + + return true; +} + +void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient * client) const +{ + String joining; + String leaving; + String new_members; + + auto operation = query->args[0].get(); + if (operation == static_cast(ReconfigCommand::Operation::ADD)) + joining = query->args[1].safeGet(); + else if (operation == static_cast(ReconfigCommand::Operation::REMOVE)) + leaving = query->args[1].safeGet(); + else if (operation == static_cast(ReconfigCommand::Operation::SET)) + new_members = query->args[1].safeGet(); + else + UNREACHABLE(); + + auto response = client->zookeeper->reconfig(joining, leaving, new_members); + std::cout << response.value << '\n'; +} + bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const { return true; diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 7d64fef1738..c2f7bd9b896 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -177,6 +177,23 @@ class RMRCommand : public IKeeperClientCommand String getHelpMessage() const override { return "{} -- Recursively deletes path. Confirmation required"; } }; +class ReconfigCommand : public IKeeperClientCommand +{ + enum class Operation : Int64 { + ADD = 0, + REMOVE = 1, + SET = 2, + }; + + String getName() const override { return "reconfig"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "{} \"\" [version] -- Reconfigures a ZooKeeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration"; } +}; + class HelpCommand : public IKeeperClientCommand { String getName() const override { return "help"; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 893be4d25d1..021baf75f64 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -83,8 +83,11 @@ std::vector KeeperClient::getCompletions(const String & prefix) const void KeeperClient::askConfirmation(const String & prompt, std::function && callback) { + if (!ask_confirmation) + return callback(); + std::cout << prompt << " Continue?\n"; - need_confirmation = true; + waiting_confirmation = true; confirmation_callback = callback; } @@ -184,6 +187,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), }); @@ -216,17 +220,6 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) EventNotifier::init(); } -void KeeperClient::executeQuery(const String & query) -{ - std::vector queries; - boost::algorithm::split(queries, query, boost::is_any_of(";")); - - for (const auto & query_text : queries) - { - if (!query_text.empty()) - processQueryText(query_text); - } -} bool KeeperClient::processQueryText(const String & text) { @@ -235,29 +228,44 @@ bool KeeperClient::processQueryText(const String & text) try { - if (need_confirmation) + if (waiting_confirmation) { - need_confirmation = false; + waiting_confirmation = false; if (text.size() == 1 && (text == "y" || text == "Y")) confirmation_callback(); return true; } KeeperParser parser; - String message; const char * begin = text.data(); - ASTPtr res = tryParseQuery(parser, begin, begin + text.size(), message, true, "", false, 0, 0, false); + const char * end = begin + text.size(); - if (!res) + while (begin < end) { - std::cerr << message << "\n"; - return true; + String message; + ASTPtr res = tryParseQuery( + parser, + begin, + end, + /* out_error_message = */ message, + /* hilite = */ true, + /* description = */ "", + /* allow_multi_statements = */ true, + /* max_query_size = */ 0, + /* max_parser_depth = */ 0, + /* skip_insignificant = */ false); + + if (!res) + { + std::cerr << message << "\n"; + return true; + } + + auto * query = res->as(); + + auto command = KeeperClient::commands.find(query->command); + command->second->execute(query, this); } - - auto * query = res->as(); - - auto command = KeeperClient::commands.find(query->command); - command->second->execute(query, this); } catch (Coordination::Exception & err) { @@ -286,7 +294,7 @@ void KeeperClient::runInteractive() while (true) { String prompt; - if (need_confirmation) + if (waiting_confirmation) prompt = "[y/n] "; else prompt = cwd.string() + " :) "; @@ -320,7 +328,10 @@ int KeeperClient::main(const std::vector & /* args */) zookeeper = std::make_unique(zk_args); if (config().has("query")) - executeQuery(config().getString("query")); + { + ask_confirmation = false; + processQueryText(config().getString("query")); + } else runInteractive(); diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index e7fa5cdc5fe..4a20a45de6e 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -50,7 +50,6 @@ public: protected: void runInteractive(); bool processQueryText(const String & text); - void executeQuery(const String & query); void loadCommands(std::vector && new_commands); @@ -61,7 +60,8 @@ protected: zkutil::ZooKeeperArgs zk_args; - bool need_confirmation = false; + bool ask_confirmation = true; + bool waiting_confirmation = false; std::vector registered_commands_and_four_letter_words; }; diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp index 3420ccb2219..228ce93b78d 100644 --- a/programs/keeper-client/Parser.cpp +++ b/programs/keeper-client/Parser.cpp @@ -7,43 +7,34 @@ namespace DB bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result) { - expected.add(pos, getTokenName(TokenType::BareWord)); - - if (pos->type == TokenType::BareWord) + if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) { - result = String(pos->begin, pos->end); - ++pos; + if (!parseIdentifierOrStringLiteral(pos, expected, result)) + return false; + ParserToken{TokenType::Whitespace}.ignore(pos); - return true; } - bool status = parseIdentifierOrStringLiteral(pos, expected, result); - ParserToken{TokenType::Whitespace}.ignore(pos); - return status; -} - -bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path) -{ - expected.add(pos, "path"); - - if (pos->type == TokenType::QuotedIdentifier || pos->type == TokenType::StringLiteral) - return parseIdentifierOrStringLiteral(pos, expected, path); - - String result; - while (pos->type != TokenType::Whitespace && pos->type != TokenType::EndOfStream) + while (pos->type != TokenType::Whitespace && pos->type != TokenType::EndOfStream && pos->type != TokenType::Semicolon) { result.append(pos->begin, pos->end); ++pos; } + ParserToken{TokenType::Whitespace}.ignore(pos); if (result.empty()) return false; - path = result; return true; } +bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path) +{ + expected.add(pos, "path"); + return parseKeeperArg(pos, expected, path); +} + bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto query = std::make_shared(); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 8b719265869..adc78035abb 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -877,6 +877,24 @@ void ZooKeeper::handleEphemeralNodeExistence(const std::string & path, const std } } +Coordination::ReconfigResponse ZooKeeper::reconfig( + const std::string & joining, + const std::string & leaving, + const std::string & new_members, + int32_t version) +{ + auto future_result = asyncReconfig(joining, leaving, new_members, version); + + if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + { + impl->finalize(fmt::format("Operation timeout on {}", Coordination::OpNum::Reconfig)); + throw KeeperException(Coordination::Error::ZOPERATIONTIMEOUT); + } + + return future_result.get(); +} + + ZooKeeperPtr ZooKeeper::startNewSession() const { return std::make_shared(args, zk_log); @@ -1226,6 +1244,27 @@ std::future ZooKeeper::asyncSync(const std::string & return future; } +std::future ZooKeeper::asyncReconfig( + const std::string & joining, + const std::string & leaving, + const std::string & new_members, + int32_t version) +{ + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::ReconfigResponse & response) mutable + { + if (response.error != Coordination::Error::ZOK) + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise->set_value(response); + }; + + impl->reconfig(joining, leaving, new_members, version, std::move(callback)); + return future; +} + void ZooKeeper::finalize(const String & reason) { impl->finalize(reason); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 1126cc189b5..4b598147301 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -449,6 +449,12 @@ public: /// disappear automatically after 3x session_timeout. void handleEphemeralNodeExistence(const std::string & path, const std::string & fast_delete_if_equal_value); + Coordination::ReconfigResponse reconfig( + const std::string & joining, + const std::string & leaving, + const std::string & new_members, + int32_t version = -1); + /// Async interface (a small subset of operations is implemented). /// /// Usage: @@ -529,6 +535,13 @@ public: const std::string & path, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); + using FutureReconfig = std::future; + FutureReconfig asyncReconfig( + const std::string & joining, + const std::string & leaving, + const std::string & new_members, + int32_t version = -1); + void finalize(const String & reason); void setZooKeeperLog(std::shared_ptr zk_log_); From adcab639a47edc19e2c336aa21c2a00ce4b8e24f Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 3 Sep 2023 21:48:33 +0000 Subject: [PATCH 063/327] add `system.scheduler` table for introspection --- src/IO/IResourceManager.h | 5 +- src/IO/ISchedulerConstraint.h | 3 + src/IO/ISchedulerNode.h | 11 +- src/IO/ISchedulerQueue.h | 6 + src/IO/Resource/DynamicResourceManager.cpp | 30 ++++- src/IO/Resource/DynamicResourceManager.h | 3 +- src/IO/Resource/FairPolicy.h | 24 ++++ src/IO/Resource/FifoQueue.h | 18 +++ src/IO/Resource/PriorityPolicy.h | 9 ++ src/IO/Resource/SemaphoreConstraint.h | 29 ++++- src/IO/Resource/StaticResourceManager.h | 5 + src/IO/ResourceBudget.h | 5 + src/IO/SchedulerRoot.h | 8 ++ .../System/StorageSystemScheduler.cpp | 103 ++++++++++++++++++ src/Storages/System/StorageSystemScheduler.h | 22 ++++ src/Storages/System/attachSystemTables.cpp | 2 + 16 files changed, 276 insertions(+), 7 deletions(-) create mode 100644 src/Storages/System/StorageSystemScheduler.cpp create mode 100644 src/Storages/System/StorageSystemScheduler.h diff --git a/src/IO/IResourceManager.h b/src/IO/IResourceManager.h index ff372698ed5..7a078c565d8 100644 --- a/src/IO/IResourceManager.h +++ b/src/IO/IResourceManager.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB { @@ -46,6 +46,9 @@ public: /// Obtain a classifier instance required to get access to resources. /// Note that it holds resource configuration, so should be destructed when query is done. virtual ClassifierPtr acquire(const String & classifier_name) = 0; + + /// For introspection, see `system.scheduler` table + virtual void forEachNode(std::function visitor) = 0; }; using ResourceManagerPtr = std::shared_ptr; diff --git a/src/IO/ISchedulerConstraint.h b/src/IO/ISchedulerConstraint.h index 47f6905e265..16a421f1240 100644 --- a/src/IO/ISchedulerConstraint.h +++ b/src/IO/ISchedulerConstraint.h @@ -44,6 +44,9 @@ public: } } + /// For introspection of current state (true = satisfied, false = violated) + virtual bool isSatisfied(); + protected: // Reference to nearest parent that is also derived from ISchedulerConstraint. // Request can traverse through multiple constraints while being dequeue from hierarchy, diff --git a/src/IO/ISchedulerNode.h b/src/IO/ISchedulerNode.h index 1c33c033744..943d41d95d2 100644 --- a/src/IO/ISchedulerNode.h +++ b/src/IO/ISchedulerNode.h @@ -3,6 +3,7 @@ #include #include #include +#include "base/types.h" #include #include @@ -176,7 +177,10 @@ public: /// Returns true iff node is active virtual bool isActive() = 0; - /// Returns the first request to be executed as the first component of resuting pair. + /// Returns number of active children + virtual size_t activeChildren() = 0; + + /// Returns the first request to be executed as the first component of resulting pair. /// The second pair component is `true` iff node is still active after dequeueing. virtual std::pair dequeueRequest() = 0; @@ -215,6 +219,11 @@ public: String basename; SchedulerNodeInfo info; ISchedulerNode * parent = nullptr; + + /// Introspection + std::atomic dequeued_requests{0}; + std::atomic dequeued_cost{0}; + std::atomic busy_periods{0}; }; using SchedulerNodePtr = std::shared_ptr; diff --git a/src/IO/ISchedulerQueue.h b/src/IO/ISchedulerQueue.h index fc2f3943d26..2e190a529e3 100644 --- a/src/IO/ISchedulerQueue.h +++ b/src/IO/ISchedulerQueue.h @@ -50,6 +50,12 @@ public: /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void enqueueRequest(ResourceRequest * request) = 0; + /// For introspection + ResourceCost getBudget() const + { + return budget.get(); + } + private: // Allows multiple consumers to synchronize with common "debit/credit" balance. // 1) (positive) to avoid wasting of allocated but not used resource (e.g in case of a failure); diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index df0de6575f4..964f4f662e6 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -9,6 +9,7 @@ #include #include +#include namespace DB { @@ -217,13 +218,36 @@ void DynamicResourceManager::updateConfiguration(const Poco::Util::AbstractConfi ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) { // Acquire a reference to the current state - StatePtr state_; + StatePtr state_ref; { std::lock_guard lock{mutex}; - state_ = state; + state_ref = state; } - return std::make_shared(state_, classifier_name); + return std::make_shared(state_ref, classifier_name); +} + +void DynamicResourceManager::forEachNode(std::function visitor) +{ + // Acquire a reference to the current state + StatePtr state_ref; + { + std::lock_guard lock{mutex}; + state_ref = state; + } + + std::promise promise; + auto future = promise.get_future(); + scheduler.event_queue->enqueue([state_ref, visitor, &promise] + { + for (auto & [name, resource] : state_ref->resources) + for (auto & [path, node] : resource->nodes) + visitor(name, path, node.type, node.ptr); + promise.set_value(); + }); + + // Block until execution is done in the scheduler thread + future.get(); } void registerDynamicResourceManager(ResourceManagerFactory & factory) diff --git a/src/IO/Resource/DynamicResourceManager.h b/src/IO/Resource/DynamicResourceManager.h index aa1147f1fb2..10d784503ca 100644 --- a/src/IO/Resource/DynamicResourceManager.h +++ b/src/IO/Resource/DynamicResourceManager.h @@ -19,7 +19,7 @@ namespace DB * `ClassifierPtr` is acquired and held. * * Manager can update configuration after initialization. During update, new version of resources are also - * attached to scheduler, so multiple version can coexist for a short perid. This will violate constraints + * attached to scheduler, so multiple version can coexist for a short period. This will violate constraints * (e.g. in-fly-limit), because different version have independent nodes to impose constraints, the same * violation will apply to fairness. Old version exists as long as there is at least one classifier * instance referencing it. Classifiers are typically attached to queries and will be destructed with them. @@ -30,6 +30,7 @@ public: DynamicResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & classifier_name) override; + void forEachNode(std::function visitor) override; private: /// Holds everything required to work with one specific configuration diff --git a/src/IO/Resource/FairPolicy.h b/src/IO/Resource/FairPolicy.h index 9c0c78f057c..7e632efc8fd 100644 --- a/src/IO/Resource/FairPolicy.h +++ b/src/IO/Resource/FairPolicy.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -176,8 +177,11 @@ public: max_vruntime = 0; } system_vruntime = max_vruntime; + busy_periods++; } + dequeued_requests++; + dequeued_cost += request->cost; return {request, heap_size > 0}; } @@ -186,12 +190,32 @@ public: return heap_size > 0; } + size_t activeChildren() override + { + return heap_size; + } + void activateChild(ISchedulerNode * child) override { // Find this child; this is O(1), thanks to inactive index we hold in `parent.idx` activateChildImpl(child->info.parent.idx); } + // For introspection + double getSystemVRuntime() const + { + return system_vruntime; + } + + std::optional getChildVRuntime(ISchedulerNode * child) const + { + for (const auto & item : items) { + if (child == item.child) + return item.vruntime; + } + return std::nullopt; + } + private: void activateChildImpl(size_t inactive_idx) { diff --git a/src/IO/Resource/FifoQueue.h b/src/IO/Resource/FifoQueue.h index f3ff15ad461..e0689e3c8da 100644 --- a/src/IO/Resource/FifoQueue.h +++ b/src/IO/Resource/FifoQueue.h @@ -39,6 +39,7 @@ public: { std::unique_lock lock(mutex); request->enqueue_ns = clock_gettime_ns(); + queue_cost += request->cost; bool was_empty = requests.empty(); requests.push_back(request); if (was_empty) @@ -52,6 +53,11 @@ public: return {nullptr, false}; ResourceRequest * result = requests.front(); requests.pop_front(); + if (requests.empty()) + busy_periods++; + queue_cost -= result->cost; + dequeued_requests++; + dequeued_cost += result->cost; return {result, !requests.empty()}; } @@ -61,6 +67,11 @@ public: return !requests.empty(); } + size_t activeChildren() override + { + return 0; + } + void activateChild(ISchedulerNode *) override { assert(false); // queue cannot have children @@ -83,8 +94,15 @@ public: return nullptr; } + std::pair getQueueLengthAndCost() + { + std::unique_lock lock(mutex); + return {requests.size(), queue_cost}; + } + private: std::mutex mutex; + Int64 queue_cost = 0; std::deque requests; }; diff --git a/src/IO/Resource/PriorityPolicy.h b/src/IO/Resource/PriorityPolicy.h index 3c091dcc85a..0a06a4fdc09 100644 --- a/src/IO/Resource/PriorityPolicy.h +++ b/src/IO/Resource/PriorityPolicy.h @@ -113,8 +113,12 @@ public: { std::pop_heap(items.begin(), items.end()); items.pop_back(); + if (items.empty()) + busy_periods++; } + dequeued_requests++; + dequeued_cost += request->cost; return {request, !items.empty()}; } @@ -123,6 +127,11 @@ public: return !items.empty(); } + size_t activeChildren() override + { + return items.size(); + } + void activateChild(ISchedulerNode * child) override { bool activate_parent = items.empty(); diff --git a/src/IO/Resource/SemaphoreConstraint.h b/src/IO/Resource/SemaphoreConstraint.h index 237e63eaddb..26437d3da87 100644 --- a/src/IO/Resource/SemaphoreConstraint.h +++ b/src/IO/Resource/SemaphoreConstraint.h @@ -78,7 +78,10 @@ public: requests++; cost += request->cost; child_active = child_now_active; - + if (!active()) + busy_periods++; + dequeued_requests++; + dequeued_cost += request->cost; return {request, active()}; } @@ -113,6 +116,30 @@ public: return active(); } + size_t activeChildren() override + { + std::unique_lock lock(mutex); + return child_active ? 1 : 0; + } + + bool isSatisfied() override + { + std::unique_lock lock(mutex); + return satisfied(); + } + + std::pair getInflights() + { + std::unique_lock lock(mutex); + return {requests, cost}; + } + + std::pair getLimits() + { + std::unique_lock lock(mutex); + return {max_requests, max_cost}; + } + private: bool satisfied() const { diff --git a/src/IO/Resource/StaticResourceManager.h b/src/IO/Resource/StaticResourceManager.h index 066dbf4ebf8..b8712b22b73 100644 --- a/src/IO/Resource/StaticResourceManager.h +++ b/src/IO/Resource/StaticResourceManager.h @@ -22,6 +22,11 @@ public: ClassifierPtr acquire(const String & classifier_name) override; + void forEachNode(std::function visitor) override + { + UNUSED(visitor); + } + private: struct Resource { diff --git a/src/IO/ResourceBudget.h b/src/IO/ResourceBudget.h index 7f67f9cfc10..0adad45ba91 100644 --- a/src/IO/ResourceBudget.h +++ b/src/IO/ResourceBudget.h @@ -48,6 +48,11 @@ public: available.fetch_add(estimated_cost - real_cost); } + ResourceCost get() const + { + return available.load(); + } + private: std::atomic available = 0; // requested - consumed }; diff --git a/src/IO/SchedulerRoot.h b/src/IO/SchedulerRoot.h index f9af2099b8c..75a1b3af72a 100644 --- a/src/IO/SchedulerRoot.h +++ b/src/IO/SchedulerRoot.h @@ -156,6 +156,8 @@ public: else current = current->next; // Just move round-robin pointer + dequeued_requests++; + dequeued_cost += request->cost; return {request, current != nullptr}; } @@ -164,6 +166,11 @@ public: return current != nullptr; } + size_t activeChildren() override + { + return 0; + } + void activateChild(ISchedulerNode * child) override { activate(TResource::get(child->info)); @@ -205,6 +212,7 @@ private: value->next = nullptr; value->prev = nullptr; current = nullptr; + busy_periods++; return; } else // Just move current to next to avoid invalidation diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp new file mode 100644 index 00000000000..bb73619b972 --- /dev/null +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -0,0 +1,103 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "IO/ResourceRequest.h" + + +namespace DB +{ + +NamesAndTypesList StorageSystemScheduler::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"resource", std::make_shared()}, + {"path", std::make_shared()}, + {"type", std::make_shared()}, + {"weight", std::make_shared()}, + {"priority", std::make_shared()}, + {"is_active", std::make_shared()}, + {"active_children", std::make_shared()}, + {"dequeued_requests", std::make_shared()}, + {"dequeued_cost", std::make_shared()}, + {"busy_periods", std::make_shared()}, + {"vruntime", std::make_shared(std::make_shared())}, + {"system_vruntime", std::make_shared(std::make_shared())}, + {"queue_length", std::make_shared(std::make_shared())}, + {"queue_cost", std::make_shared(std::make_shared())}, + {"budget", std::make_shared(std::make_shared())}, + {"is_satisfied", std::make_shared(std::make_shared())}, + {"inflight_requests", std::make_shared(std::make_shared())}, + {"inflight_cost", std::make_shared(std::make_shared())}, + {"max_requests", std::make_shared(std::make_shared())}, + {"max_cost", std::make_shared(std::make_shared())}, + }; + return names_and_types; +} + + +void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) { + size_t i = 0; + res_columns[i++]->insert(resource); + res_columns[i++]->insert(path); + res_columns[i++]->insert(type); + res_columns[i++]->insert(node->info.weight); + res_columns[i++]->insert(node->info.priority.value); + res_columns[i++]->insert(node->isActive()); + res_columns[i++]->insert(node->activeChildren()); + res_columns[i++]->insert(node->dequeued_requests.load()); + res_columns[i++]->insert(node->dequeued_cost.load()); + res_columns[i++]->insert(node->busy_periods.load()); + + Field vruntime; + Field system_vruntime; + Field queue_length; + Field queue_cost; + Field budget; + Field is_satisfied; + Field inflight_requests; + Field inflight_cost; + Field max_requests; + Field max_cost; + + if (auto parent = dynamic_cast(node->parent)) + vruntime = parent->getChildVRuntime(node.get()); + if (auto ptr = dynamic_cast(node.get())) + system_vruntime = ptr->getSystemVRuntime(); + if (auto ptr = dynamic_cast(node.get())) + std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); + if (auto ptr = dynamic_cast(node.get())) + budget = ptr->getBudget(); + if (auto ptr = dynamic_cast(node.get())) + is_satisfied = ptr->isSatisfied(); + if (auto ptr = dynamic_cast(node.get())) + { + std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); + std::tie(max_requests, max_cost) = ptr->getLimits(); + } + + res_columns[i++]->insert(vruntime); + res_columns[i++]->insert(system_vruntime); + res_columns[i++]->insert(queue_length); + res_columns[i++]->insert(queue_cost); + res_columns[i++]->insert(budget); + res_columns[i++]->insert(is_satisfied); + res_columns[i++]->insert(inflight_requests); + res_columns[i++]->insert(inflight_cost); + res_columns[i++]->insert(max_requests); + res_columns[i++]->insert(max_cost); + }); +} + +} diff --git a/src/Storages/System/StorageSystemScheduler.h b/src/Storages/System/StorageSystemScheduler.h new file mode 100644 index 00000000000..31d14862209 --- /dev/null +++ b/src/Storages/System/StorageSystemScheduler.h @@ -0,0 +1,22 @@ +#pragma once + +#include + + +namespace DB +{ +class Context; + +/// Implements `system.scheduler` table, which allows you to get information about scheduling nodes. +class StorageSystemScheduler final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemScheduler"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index b770f784c01..e4e19ce2e06 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -83,6 +83,7 @@ #include #include #include +#include #if USE_RDKAFKA #include @@ -148,6 +149,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "backups"); attach(context, system_database, "schema_inference_cache"); attach(context, system_database, "dropped_tables"); + attach(context, system_database, "scheduler"); #if USE_RDKAFKA attach(context, system_database, "kafka_consumers"); #endif From 8fdd118e0c338f5a34d410cdcac874de7b3bc199 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 3 Sep 2023 22:40:00 +0000 Subject: [PATCH 064/327] fix build --- src/IO/ISchedulerConstraint.h | 4 ++-- src/Storages/System/StorageSystemScheduler.cpp | 17 ++++++++++------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/IO/ISchedulerConstraint.h b/src/IO/ISchedulerConstraint.h index 16a421f1240..05bed7c3df8 100644 --- a/src/IO/ISchedulerConstraint.h +++ b/src/IO/ISchedulerConstraint.h @@ -21,7 +21,7 @@ namespace DB class ISchedulerConstraint : public ISchedulerNode { public: - ISchedulerConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + explicit ISchedulerConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) : ISchedulerNode(event_queue_, config, config_prefix) {} @@ -45,7 +45,7 @@ public: } /// For introspection of current state (true = satisfied, false = violated) - virtual bool isSatisfied(); + virtual bool isSatisfied() = 0; protected: // Reference to nearest parent that is also derived from ISchedulerConstraint. diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index bb73619b972..2c3b726433e 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -71,17 +71,20 @@ void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr c Field max_requests; Field max_cost; - if (auto parent = dynamic_cast(node->parent)) - vruntime = parent->getChildVRuntime(node.get()); - if (auto ptr = dynamic_cast(node.get())) + if (auto * parent = dynamic_cast(node->parent)) + { + if (auto value = parent->getChildVRuntime(node.get())) + vruntime = *value; + } + if (auto * ptr = dynamic_cast(node.get())) system_vruntime = ptr->getSystemVRuntime(); - if (auto ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node.get())) std::tie(queue_length, queue_cost) = ptr->getQueueLengthAndCost(); - if (auto ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node.get())) budget = ptr->getBudget(); - if (auto ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node.get())) is_satisfied = ptr->isSatisfied(); - if (auto ptr = dynamic_cast(node.get())) + if (auto * ptr = dynamic_cast(node.get())) { std::tie(inflight_requests, inflight_cost) = ptr->getInflights(); std::tie(max_requests, max_cost) = ptr->getLimits(); From 964412ee64dcf7f20c4f674215ba88747e58e08d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 4 Sep 2023 02:47:26 +0000 Subject: [PATCH 065/327] no random settings in stateless test Signed-off-by: Duc Canh Le --- ...erge_across_partitions_final_with_lonely.reference | 3 --- ...68_no_merge_across_partitions_final_with_lonely.sh | 11 ++++------- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference index 72837206fac..126459c2db0 100644 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.reference @@ -1,6 +1,3 @@ 90 110 90 80 90 80 -90 110 -90 80 -90 80 diff --git a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh index d01e4f94350..4bc29ce4233 100755 --- a/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh +++ b/tests/queries/0_stateless/02868_no_merge_across_partitions_final_with_lonely.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-settings, no-random-merge-tree-settings set -e @@ -49,12 +50,8 @@ INSERT INTO with_lonely SELECT number+60000, '2022-11-01', number*10, 2 FROM num CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --do_not_merge_across_partitions_select_final 1" # mix lonely parts and non-lonely parts -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_bytes_to_read 2000 -q "SELECT max(val), count(*) FROM with_lonely FINAL;" +${CLICKHOUSE_CLIENT} --max_bytes_to_read 2000 -q "SELECT max(val), count(*) FROM with_lonely FINAL;" # only lonely parts -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" +${CLICKHOUSE_CLIENT} --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" # only lonely parts but max_thread = 1, so reading lonely parts with in-order -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 1 --max_threads 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" - -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_bytes_to_read 2000 -q "SELECT max(val), count(*) FROM with_lonely FINAL;" -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" -${CLICKHOUSE_CLIENT} --allow_experimental_analyzer 0 --max_threads 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" +${CLICKHOUSE_CLIENT} --max_threads 1 --max_bytes_to_read 1000 -q "SELECT max(val), count(*) FROM with_lonely FINAL WHERE dt < '2022-11-01';" From b5233efc20a1ad18ae347cf90b320f0b9cf8a0d1 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 4 Sep 2023 12:35:17 +0000 Subject: [PATCH 066/327] fix system table and scheduler config reloading --- src/IO/ISchedulerNode.h | 14 ++++++++++++-- src/IO/Resource/FairPolicy.h | 2 ++ src/IO/Resource/FifoQueue.h | 2 ++ src/IO/Resource/PriorityPolicy.h | 2 ++ src/IO/Resource/SemaphoreConstraint.h | 2 ++ src/IO/SchedulerRoot.h | 2 ++ src/Storages/System/StorageSystemScheduler.cpp | 2 +- 7 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/IO/ISchedulerNode.h b/src/IO/ISchedulerNode.h index 943d41d95d2..f7fc9257128 100644 --- a/src/IO/ISchedulerNode.h +++ b/src/IO/ISchedulerNode.h @@ -69,6 +69,13 @@ struct SchedulerNodeInfo { priority.value = value; } + + // To check if configuration update required + bool equals(const SchedulerNodeInfo & o) const + { + // `parent` data is not compared intentionally (it is not part of configuration settings) + return weight == o.weight && priority == o.priority; + } }; /* @@ -158,8 +165,11 @@ public: virtual ~ISchedulerNode() {} - // Checks if two nodes configuration is equal - virtual bool equals(ISchedulerNode * other) = 0; + /// Checks if two nodes configuration is equal + virtual bool equals(ISchedulerNode * other) + { + return info.equals(other->info); + } /// Attach new child virtual void attachChild(const std::shared_ptr & child) = 0; diff --git a/src/IO/Resource/FairPolicy.h b/src/IO/Resource/FairPolicy.h index 7e632efc8fd..3c7ccfcff89 100644 --- a/src/IO/Resource/FairPolicy.h +++ b/src/IO/Resource/FairPolicy.h @@ -51,6 +51,8 @@ public: bool equals(ISchedulerNode * other) override { + if (!ISchedulerNode::equals(other)) + return false; if (auto * o = dynamic_cast(other)) return true; return false; diff --git a/src/IO/Resource/FifoQueue.h b/src/IO/Resource/FifoQueue.h index e0689e3c8da..e95e817719a 100644 --- a/src/IO/Resource/FifoQueue.h +++ b/src/IO/Resource/FifoQueue.h @@ -30,6 +30,8 @@ public: bool equals(ISchedulerNode * other) override { + if (!ISchedulerNode::equals(other)) + return false; if (auto * o = dynamic_cast(other)) return true; return false; diff --git a/src/IO/Resource/PriorityPolicy.h b/src/IO/Resource/PriorityPolicy.h index 0a06a4fdc09..86d4fadb9dc 100644 --- a/src/IO/Resource/PriorityPolicy.h +++ b/src/IO/Resource/PriorityPolicy.h @@ -42,6 +42,8 @@ public: bool equals(ISchedulerNode * other) override { + if (!ISchedulerNode::equals(other)) + return false; if (auto * o = dynamic_cast(other)) return true; return false; diff --git a/src/IO/Resource/SemaphoreConstraint.h b/src/IO/Resource/SemaphoreConstraint.h index 26437d3da87..9160d0431de 100644 --- a/src/IO/Resource/SemaphoreConstraint.h +++ b/src/IO/Resource/SemaphoreConstraint.h @@ -27,6 +27,8 @@ public: bool equals(ISchedulerNode * other) override { + if (!ISchedulerNode::equals(other)) + return false; if (auto * o = dynamic_cast(other)) return max_requests == o->max_requests && max_cost == o->max_cost; return false; diff --git a/src/IO/SchedulerRoot.h b/src/IO/SchedulerRoot.h index 75a1b3af72a..748632615bc 100644 --- a/src/IO/SchedulerRoot.h +++ b/src/IO/SchedulerRoot.h @@ -97,6 +97,8 @@ public: bool equals(ISchedulerNode * other) override { + if (!ISchedulerNode::equals(other)) + return false; if (auto * o = dynamic_cast(other)) return true; return false; diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 2c3b726433e..101914a34ae 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -26,7 +26,7 @@ NamesAndTypesList StorageSystemScheduler::getNamesAndTypes() {"weight", std::make_shared()}, {"priority", std::make_shared()}, {"is_active", std::make_shared()}, - {"active_children", std::make_shared()}, + {"active_children", std::make_shared()}, {"dequeued_requests", std::make_shared()}, {"dequeued_cost", std::make_shared()}, {"busy_periods", std::make_shared()}, From aa8e9c3cbc5822983fba3a54737be5ff8a87a601 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 4 Sep 2023 14:18:54 +0000 Subject: [PATCH 067/327] fix style --- src/IO/Resource/FairPolicy.h | 3 ++- src/Storages/System/StorageSystemScheduler.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/IO/Resource/FairPolicy.h b/src/IO/Resource/FairPolicy.h index 3c7ccfcff89..57b26344658 100644 --- a/src/IO/Resource/FairPolicy.h +++ b/src/IO/Resource/FairPolicy.h @@ -211,7 +211,8 @@ public: std::optional getChildVRuntime(ISchedulerNode * child) const { - for (const auto & item : items) { + for (const auto & item : items) + { if (child == item.child) return item.vruntime; } diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 101914a34ae..ca0a138559e 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -47,7 +47,8 @@ NamesAndTypesList StorageSystemScheduler::getNamesAndTypes() void StorageSystemScheduler::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) { + context->getResourceManager()->forEachNode([&] (const String & resource, const String & path, const String & type, const SchedulerNodePtr & node) + { size_t i = 0; res_columns[i++]->insert(resource); res_columns[i++]->insert(path); From 4f4640ec90a450a39dd41dd0f798299b88d5983f Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 20:49:29 +0200 Subject: [PATCH 068/327] Add DateTime64 support --- .../PreparedStatementsTest.java | 108 +++++--- src/Core/MySQL/PacketsProtocolBinary.cpp | 248 ++++++++++++------ src/Core/MySQL/PacketsProtocolText.cpp | 4 + .../prepared_statements.reference | 157 ++++++----- .../prepared_statements_test.sql | 33 ++- 5 files changed, 364 insertions(+), 186 deletions(-) diff --git a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java index cb479e13099..9efb3330fd5 100644 --- a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java +++ b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java @@ -46,6 +46,8 @@ public class PreparedStatementsTest { testDecimalTypes(conn); testMiscTypes(conn); testDateTypes(conn); + testUnusualDateTime64Scales(conn); + testDateTimeTimezones(conn); conn.close(); } catch (Exception e) { e.printStackTrace(); @@ -59,21 +61,21 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i8"), rs.getInt("i8")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i16"), rs.getInt("i16")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i32"), rs.getInt("i32")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "i64"), rs.getLong("i64")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "i128"), rs.getString("i128")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "i256"), rs.getString("i256")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui8"), rs.getInt("ui8")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui16"), rs.getInt("ui16")); - System.out.printf("Type: %s, value: %d\n", getMysqlType(rs, "ui32"), rs.getLong("ui32")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui64"), rs.getString("ui64")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui128"), rs.getString("ui128")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "ui256"), rs.getString("ui256")); - System.out.printf("Type: %s, value: %f\n", getMysqlType(rs, "f32"), rs.getFloat("f32")); - System.out.printf("Type: %s, value: %f\n", getMysqlType(rs, "f64"), rs.getFloat("f64")); - System.out.printf("Type: %s, value: %b\n", getMysqlType(rs, "b"), rs.getBoolean("b")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "i8"), rs.getInt("i8")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "i16"), rs.getInt("i16")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "i32"), rs.getInt("i32")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "i64"), rs.getLong("i64")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "i128"), rs.getString("i128")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "i256"), rs.getString("i256")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui8"), rs.getInt("ui8")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui16"), rs.getInt("ui16")); + System.out.printf("%s, value: %d\n", getMysqlType(rs, "ui32"), rs.getLong("ui32")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui64"), rs.getString("ui64")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui128"), rs.getString("ui128")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "ui256"), rs.getString("ui256")); + System.out.printf("%s, value: %f\n", getMysqlType(rs, "f32"), rs.getFloat("f32")); + System.out.printf("%s, value: %f\n", getMysqlType(rs, "f64"), rs.getFloat("f64")); + System.out.printf("%s, value: %b\n", getMysqlType(rs, "b"), rs.getBoolean("b")); } System.out.println(); } @@ -84,10 +86,10 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "s"), rs.getString("s")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "sn"), rs.getString("sn")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "lc"), rs.getString("lc")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "nlc"), rs.getString("nlc")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "s"), rs.getString("s")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "sn"), rs.getString("sn")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "lc"), rs.getString("lc")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "nlc"), rs.getString("nlc")); } System.out.println(); } @@ -98,16 +100,12 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d32"), - rs.getBigDecimal("d32").toPlainString()); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d64"), - rs.getBigDecimal("d64").toPlainString()); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d128_native"), + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d32"), rs.getBigDecimal("d32").toPlainString()); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d64"), rs.getBigDecimal("d64").toPlainString()); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d128_native"), rs.getBigDecimal("d128_native").toPlainString()); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d128_text"), - rs.getString("d128_text")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d256"), - rs.getString("d256")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d128_text"), rs.getString("d128_text")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d256"), rs.getString("d256")); } System.out.println(); } @@ -118,12 +116,41 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "d32"), rs.getDate("d32")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getString("dt64_3")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_6"), rs.getString("dt64_6")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "dt64_9"), rs.getString("dt64_9")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d"), rs.getDate("d")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "d32"), rs.getDate("d32")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_6"), rs.getTimestamp("dt64_6")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_9"), rs.getTimestamp("dt64_9")); + } + System.out.println(); + } + + private static void testUnusualDateTime64Scales(Connection conn) throws SQLException { + System.out.println("### testUnusualDateTime64Scales"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_unusual_datetime64_scales").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_0"), rs.getTimestamp("dt64_0")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_1"), rs.getTimestamp("dt64_1")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_2"), rs.getTimestamp("dt64_2")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_4"), rs.getTimestamp("dt64_4")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_5"), rs.getTimestamp("dt64_5")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_7"), rs.getTimestamp("dt64_7")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_8"), rs.getTimestamp("dt64_8")); + } + System.out.println(); + } + + private static void testDateTimeTimezones(Connection conn) throws SQLException { + System.out.println("### testDateTimeTimezones"); + ResultSet rs = conn.prepareStatement("SELECT * FROM ps_datetime_timezones").executeQuery(); + int rowNum = 1; + while (rs.next()) { + System.out.printf("Row #%d\n", rowNum++); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt"), rs.getTimestamp("dt")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "dt64_3"), rs.getTimestamp("dt64_3")); } System.out.println(); } @@ -134,16 +161,17 @@ public class PreparedStatementsTest { int rowNum = 1; while (rs.next()) { System.out.printf("Row #%d\n", rowNum++); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "a"), rs.getString("a")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "u"), rs.getString("u")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "t"), rs.getString("t")); - System.out.printf("Type: %s, value: %s\n", getMysqlType(rs, "m"), rs.getString("m")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "a"), rs.getString("a")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "u"), rs.getString("u")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "t"), rs.getString("t")); + System.out.printf("%s, value: %s\n", getMysqlType(rs, "m"), rs.getString("m")); } System.out.println(); } - private static MysqlType getMysqlType(ResultSet rs, String columnLabel) throws SQLException { + private static String getMysqlType(ResultSet rs, String columnLabel) throws SQLException { ResultSetMetaData meta = rs.getMetaData(); - return MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel))); + return String.format("%s type is %s", columnLabel, + MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel)))); } } diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index ebb256cb462..ca206870a76 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -7,6 +7,8 @@ #include "Columns/ColumnLowCardinality.h" #include "Columns/ColumnVector.h" #include "Columns/ColumnsDateTime.h" +#include "Core/DecimalFunctions.h" +#include "DataTypes/DataTypeDateTime64.h" #include "DataTypes/DataTypeLowCardinality.h" #include "DataTypes/DataTypeNullable.h" #include "DataTypes/DataTypesNumber.h" @@ -74,46 +76,66 @@ namespace MySQLProtocol payload_size += 8; break; case TypeIndex::Date: { - UInt16 value = assert_cast &>(*col).getData()[row_num]; - if (value == 0) - { - payload_size += 1; // length only, no other fields - } - else - { - payload_size += 5; - } + payload_size += 5; break; } case TypeIndex::Date32: { - Int32 value = assert_cast &>(*col).getData()[row_num]; - if (value == 0) - { - payload_size += 1; // length only, no other fields - } - else - { - payload_size += 5; - } + payload_size += 5; break; } case TypeIndex::DateTime: { UInt32 value = assert_cast &>(*col).getData()[row_num]; - if (value == 0) + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); + + bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); + if (has_time) { - payload_size += 1; // length only, no other fields + payload_size += 8; } else { - LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); - if (ldt.second() == 0 && ldt.minute() == 0 && ldt.hour() == 0) - { - payload_size += 5; - } - else - { - payload_size += 8; - } + payload_size += 5; + } + break; + } + case TypeIndex::DateTime64: { + const auto * date_time_type = typeid_cast(data_type.get()); + UInt32 scale = date_time_type->getScale(); + + static constexpr UInt32 MaxScale = DecimalUtils::max_precision; + scale = scale > MaxScale ? MaxScale : scale; + + const auto dt64 = assert_cast(*col).getData()[row_num]; + auto components = DecimalUtils::split(dt64, scale); + + using T = typename DateTime64::NativeType; + if (dt64.value < 0 && components.fractional) + { + components.fractional + = DecimalUtils::scaleMultiplier(scale) + (components.whole ? T(-1) : T(1)) * components.fractional; + --components.whole; + } + if (scale > 6) + { + // MySQL Timestamp has max scale of 6 + components.fractional /= static_cast(pow(10, scale - 6)); + } + + LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))); + + bool has_microseconds = components.fractional != 0; + bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); + if (has_microseconds) + { + payload_size += 12; + } + else if (has_time) + { + payload_size += 8; + } + else + { + payload_size += 5; } break; } @@ -199,73 +221,133 @@ namespace MySQLProtocol } case TypeIndex::Date: { UInt16 value = assert_cast &>(*col).getData()[row_num]; - if (value != 0) - { - LocalDate ld = LocalDate(DayNum(value)); - buffer.write(static_cast(4)); // bytes_following - auto year = ld.year(); - auto month = ld.month(); - auto day = ld.day(); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - } - else - { - buffer.write(static_cast(0)); - } + LocalDate ld = LocalDate(DayNum(value)); + buffer.write(static_cast(4)); // bytes_following + auto year = ld.year(); + auto month = ld.month(); + auto day = ld.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); break; } case TypeIndex::Date32: { Int32 value = assert_cast &>(*col).getData()[row_num]; - if (value != 0) - { - LocalDate ld = LocalDate(ExtendedDayNum(value)); - buffer.write(static_cast(4)); // bytes_following - auto year = ld.year(); - auto month = ld.month(); - auto day = ld.day(); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - } - else - { - buffer.write(static_cast(0)); - } + LocalDate ld = LocalDate(ExtendedDayNum(value)); + buffer.write(static_cast(4)); // bytes_following + auto year = ld.year(); + auto month = ld.month(); + auto day = ld.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); break; } case TypeIndex::DateTime: { UInt32 value = assert_cast &>(*col).getData()[row_num]; - if (value != 0) + String timezone = getDateTimeTimezone(*data_type); + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(timezone)); + int year = ldt.year(); + int month = ldt.month(); + int day = ldt.day(); + int hour = ldt.hour(); + int minute = ldt.minute(); + int second = ldt.second(); + bool has_time = !(hour == 0 && minute == 0 && second == 0); + size_t bytes_following = has_time ? 7 : 4; + buffer.write(reinterpret_cast(&bytes_following), 1); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + if (has_time) { - LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); - int year = ldt.year(); - int month = ldt.month(); - int day = ldt.day(); - int hour = ldt.hour(); - int minute = ldt.minute(); - int second = ldt.second(); - bool has_time = !(hour == 0 && minute == 0 && second == 0); - size_t bytes_following = has_time ? 7 : 4; - buffer.write(reinterpret_cast(&bytes_following), 1); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - if (has_time) - { - buffer.write(reinterpret_cast(&hour), 1); - buffer.write(reinterpret_cast(&minute), 1); - buffer.write(reinterpret_cast(&second), 1); - } - } - else - { - buffer.write(static_cast(0)); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); } break; } + case TypeIndex::DateTime64: { + const auto * date_time_type = typeid_cast(data_type.get()); + UInt32 scale = date_time_type->getScale(); + static constexpr UInt32 MaxScale = DecimalUtils::max_precision; + scale = scale > MaxScale ? MaxScale : scale; + + const auto dt64 = assert_cast(*col).getData()[row_num]; + auto components = DecimalUtils::split(dt64, scale); + + using T = typename DateTime64::NativeType; + if (dt64.value < 0 && components.fractional) + { + components.fractional + = DecimalUtils::scaleMultiplier(scale) + (components.whole ? T(-1) : T(1)) * components.fractional; + --components.whole; + } + + if (components.fractional != 0) + { + if (scale > 6) + { + // MySQL Timestamp has max scale of 6 + components.fractional /= static_cast(pow(10, scale - 6)); + } + else + { + // fractional == 1 is a different microsecond value depending on the scale + // Scale 1 = 100 000 + // Scale 2 = 010 000 + // Scale 3 = 001 000 + // Scale 4 = 000 100 + // Scale 5 = 000 010 + // Scale 6 = 000 001 + components.fractional *= static_cast(pow(10, 6 - scale)); + } + } + + String timezone = getDateTimeTimezone(*data_type); + std::cout << "Timezone is " << timezone << std::endl; + LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone)); + auto year = ldt.year(); + auto month = ldt.month(); + auto day = ldt.day(); + auto hour = ldt.hour(); + auto minute = ldt.minute(); + auto second = ldt.second(); + + bool has_time = !(hour == 0 && minute == 0 && second == 0); + bool has_microseconds = components.fractional != 0; + + if (has_microseconds) + { + buffer.write(static_cast(11)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + buffer.write(reinterpret_cast(&components.fractional), 4); + } + else if (has_time) + { + buffer.write(static_cast(7)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + } + else + { + buffer.write(static_cast(4)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + } + break; + } default: writeLengthEncodedString(serialized[i], buffer); break; diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 712b203bc2a..9f4de31e44d 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -211,6 +211,10 @@ namespace MySQLProtocol column_type = ColumnType::MYSQL_TYPE_DATETIME; flags = ColumnDefinitionFlags::BINARY_FLAG; break; + case TypeIndex::DateTime64: + column_type = ColumnType::MYSQL_TYPE_DATETIME; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; case TypeIndex::Decimal32: case TypeIndex::Decimal64: column_type = ColumnType::MYSQL_TYPE_DECIMAL; diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference index 3e23620c07e..402e64348ae 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements.reference +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -1,81 +1,114 @@ ### testSimpleDataTypes Row #1 -Type: TINYINT, value: -128 -Type: SMALLINT, value: -32768 -Type: INT, value: -2147483648 -Type: BIGINT, value: -9223372036854775808 -Type: CHAR, value: -170141183460469231731687303715884105728 -Type: CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968 -Type: TINYINT, value: 120 -Type: SMALLINT, value: 1234 -Type: INT, value: 51234 -Type: BIGINT, value: 421342 -Type: CHAR, value: 15324355 -Type: CHAR, value: 41345135123432 -Type: FLOAT, value: -0.796896 -Type: DOUBLE, value: -0.113259 -Type: BIT, value: true +i8 type is TINYINT, value: -128 +i16 type is SMALLINT, value: -32768 +i32 type is INT, value: -2147483648 +i64 type is BIGINT, value: -9223372036854775808 +i128 type is CHAR, value: -170141183460469231731687303715884105728 +i256 type is CHAR, value: -57896044618658097711785492504343953926634992332820282019728792003956564819968 +ui8 type is TINYINT, value: 120 +ui16 type is SMALLINT, value: 1234 +ui32 type is INT, value: 51234 +ui64 type is BIGINT, value: 421342 +ui128 type is CHAR, value: 15324355 +ui256 type is CHAR, value: 41345135123432 +f32 type is FLOAT, value: -0.796896 +f64 type is DOUBLE, value: -0.113259 +b type is BIT, value: true Row #2 -Type: TINYINT, value: 127 -Type: SMALLINT, value: 32767 -Type: INT, value: 2147483647 -Type: BIGINT, value: 9223372036854775807 -Type: CHAR, value: 170141183460469231731687303715884105727 -Type: CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967 -Type: TINYINT, value: 255 -Type: SMALLINT, value: 65535 -Type: INT, value: 4294967295 -Type: BIGINT, value: 18446744073709551615 -Type: CHAR, value: 340282366920938463463374607431768211455 -Type: CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 -Type: FLOAT, value: 1.234000 -Type: DOUBLE, value: 3.352451 -Type: BIT, value: false +i8 type is TINYINT, value: 127 +i16 type is SMALLINT, value: 32767 +i32 type is INT, value: 2147483647 +i64 type is BIGINT, value: 9223372036854775807 +i128 type is CHAR, value: 170141183460469231731687303715884105727 +i256 type is CHAR, value: 57896044618658097711785492504343953926634992332820282019728792003956564819967 +ui8 type is TINYINT, value: 255 +ui16 type is SMALLINT, value: 65535 +ui32 type is INT, value: 4294967295 +ui64 type is BIGINT, value: 18446744073709551615 +ui128 type is CHAR, value: 340282366920938463463374607431768211455 +ui256 type is CHAR, value: 115792089237316195423570985008687907853269984665640564039457584007913129639935 +f32 type is FLOAT, value: 1.234000 +f64 type is DOUBLE, value: 3.352451 +b type is BIT, value: false ### testStringTypes Row #1 -Type: CHAR, value: 42 -Type: CHAR, value: null -Type: CHAR, value: test -Type: CHAR, value: null +s type is CHAR, value: 42 +sn type is CHAR, value: null +lc type is CHAR, value: test +nlc type is CHAR, value: null Row #2 -Type: CHAR, value: foo -Type: CHAR, value: bar -Type: CHAR, value: qaz -Type: CHAR, value: qux +s type is CHAR, value: foo +sn type is CHAR, value: bar +lc type is CHAR, value: qaz +nlc type is CHAR, value: qux ### testDecimalTypes Row #1 -Type: DECIMAL, value: -1.55 -Type: DECIMAL, value: 6.03 -Type: DECIMAL, value: 5 -Type: CHAR, value: -1224124.23423 -Type: CHAR, value: -54342.3 +d32 type is DECIMAL, value: -1.55 +d64 type is DECIMAL, value: 6.03 +d128_native type is DECIMAL, value: 5 +d128_text type is CHAR, value: -1224124.23423 +d256 type is CHAR, value: -54342.3 Row #2 -Type: DECIMAL, value: 1234567.89 -Type: DECIMAL, value: 123456789123456.789 -Type: DECIMAL, value: 12345678912345678912.1234567891 -Type: CHAR, value: 1234567.8912345678912345678911234567891 -Type: CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891 +d32 type is DECIMAL, value: 1234567.89 +d64 type is DECIMAL, value: 123456789123456.789 +d128_native type is DECIMAL, value: 12345678912345678912.1234567891 +d128_text type is CHAR, value: 1234567.8912345678912345678911234567891 +d256 type is CHAR, value: 12345678912345678912345678911234567891234567891234567891.12345678911234567891 ### testMiscTypes Row #1 -Type: CHAR, value: [] -Type: CHAR, value: 9a0ccc06-2578-4861-8534-631c9d40f3f7 -Type: CHAR, value: (0,'') -Type: CHAR, value: {} +a type is CHAR, value: [] +u type is CHAR, value: 9a0ccc06-2578-4861-8534-631c9d40f3f7 +t type is CHAR, value: (0,'') +m type is CHAR, value: {} Row #2 -Type: CHAR, value: ['foo','bar'] -Type: CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 -Type: CHAR, value: (42,'qaz') -Type: CHAR, value: {'qux':144,'text':255} +a type is CHAR, value: ['foo','bar'] +u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 +t type is CHAR, value: (42,'qaz') +m type is CHAR, value: {'qux':144,'text':255} ### testDateTypes Row #1 -Type: DATE, value: 2149-06-06 -Type: DATE, value: 2178-04-16 -Type: TIMESTAMP, value: 2106-02-07 06:28:15.0 -Type: CHAR, value: 2106-02-07 06:28:15.123 -Type: CHAR, value: 2106-02-07 06:28:15.123456 -Type: CHAR, value: 2106-02-07 06:28:15.123456789 +d type is DATE, value: 1970-01-01 +d32 type is DATE, value: 1900-01-01 +dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0 +dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 +dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001 +dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 +Row #2 +d type is DATE, value: 2149-06-06 +d32 type is DATE, value: 2178-04-16 +dt type is TIMESTAMP, value: 2106-02-07 06:28:15.0 +dt64_3 type is TIMESTAMP, value: 2106-02-07 06:28:15.123 +dt64_6 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 +dt64_9 type is TIMESTAMP, value: 2106-02-07 06:28:15.123456 + +### testUnusualDateTime64Scales +Row #1 +dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 +dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 +dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.12 +dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.1234 +dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.12345 +dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 +dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.123456 +Row #2 +dt64_0 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 +dt64_1 type is TIMESTAMP, value: 2022-04-13 03:17:45.1 +dt64_2 type is TIMESTAMP, value: 2022-04-13 03:17:45.01 +dt64_4 type is TIMESTAMP, value: 2022-04-13 03:17:45.0001 +dt64_5 type is TIMESTAMP, value: 2022-04-13 03:17:45.00001 +dt64_7 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 +dt64_8 type is TIMESTAMP, value: 2022-04-13 03:17:45.0 + +### testDateTimeTimezones +Row #1 +dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0 +dt64_3 type is TIMESTAMP, value: 1969-12-31 16:00:00.0 +Row #2 +dt type is TIMESTAMP, value: 2022-09-04 20:31:05.0 +dt64_3 type is TIMESTAMP, value: 2022-09-04 20:31:05.022 diff --git a/tests/integration/test_mysql_protocol/prepared_statements_test.sql b/tests/integration/test_mysql_protocol/prepared_statements_test.sql index 4fb53a14a61..a32ce9db1d5 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements_test.sql +++ b/tests/integration/test_mysql_protocol/prepared_statements_test.sql @@ -83,4 +83,35 @@ INSERT INTO ps_date_types VALUES ('2149-06-06', '2178-04-16', '2106-02-07 06:28:15', '2106-02-07 06:28:15.123', '2106-02-07 06:28:15.123456', - '2106-02-07 06:28:15.123456789'); + '2106-02-07 06:28:15.123456789'), + ('1970-01-01', '1900-01-01', '1970-01-01 00:00:00', + '1900-01-01 00:00:00.001', + '1900-01-01 00:00:00.000001', + '1900-01-01 00:00:00.000000001');; + +CREATE TABLE ps_unusual_datetime64_scales +( + dt64_0 DateTime64(0, 'UTC'), + dt64_1 DateTime64(1, 'UTC'), + dt64_2 DateTime64(2, 'UTC'), + dt64_4 DateTime64(4, 'UTC'), + dt64_5 DateTime64(5, 'UTC'), + dt64_7 DateTime64(7, 'UTC'), + dt64_8 DateTime64(8, 'UTC') +) ENGINE MergeTree ORDER BY dt64_0; + +INSERT INTO ps_unusual_datetime64_scales +VALUES ('2022-04-13 03:17:45', + '2022-04-13 03:17:45.1', + '2022-04-13 03:17:45.12', + '2022-04-13 03:17:45.1234', + '2022-04-13 03:17:45.12345', + '2022-04-13 03:17:45.1234567', + '2022-04-13 03:17:45.12345678'), + ('2022-04-13 03:17:45', + '2022-04-13 03:17:45.1', + '2022-04-13 03:17:45.01', + '2022-04-13 03:17:45.0001', + '2022-04-13 03:17:45.00001', + '2022-04-13 03:17:45.0000001', + '2022-04-13 03:17:45.00000001'); From 2245a8df9dd1f66ab2bc1cb35be19f0b3ca97a94 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 20:58:24 +0200 Subject: [PATCH 069/327] Revert format changes --- src/Core/MySQL/PacketsGeneric.cpp | 499 ++++++++++++------------- src/Core/MySQL/PacketsProtocolText.cpp | 452 +++++++++++----------- src/DataTypes/DataTypesDecimal.cpp | 18 +- 3 files changed, 478 insertions(+), 491 deletions(-) diff --git a/src/Core/MySQL/PacketsGeneric.cpp b/src/Core/MySQL/PacketsGeneric.cpp index 88183890399..23f197bcfe8 100644 --- a/src/Core/MySQL/PacketsGeneric.cpp +++ b/src/Core/MySQL/PacketsGeneric.cpp @@ -8,264 +8,253 @@ namespace DB namespace MySQLProtocol { - namespace Generic +namespace Generic +{ + +static const size_t MYSQL_ERRMSG_SIZE = 512; + +void SSLRequest::readPayloadImpl(ReadBuffer & buf) +{ + buf.readStrict(reinterpret_cast(&capability_flags), 4); + buf.readStrict(reinterpret_cast(&max_packet_size), 4); + buf.readStrict(reinterpret_cast(&character_set), 1); +} + +OKPacket::OKPacket(uint32_t capabilities_) + : header(0x00), capabilities(capabilities_), affected_rows(0x00), last_insert_id(0x00), status_flags(0x00) +{ +} + +OKPacket::OKPacket( + uint8_t header_, uint32_t capabilities_, uint64_t affected_rows_, uint32_t status_flags_, int16_t warnings_, + String session_state_changes_, String info_) + : header(header_), capabilities(capabilities_), affected_rows(affected_rows_), last_insert_id(0), warnings(warnings_), + status_flags(status_flags_), session_state_changes(std::move(session_state_changes_)), info(std::move(info_)) +{ +} + +size_t OKPacket::getPayloadSize() const +{ + size_t result = 2 + getLengthEncodedNumberSize(affected_rows); + + if (capabilities & CLIENT_PROTOCOL_41) { - - static const size_t MYSQL_ERRMSG_SIZE = 512; - - void SSLRequest::readPayloadImpl(ReadBuffer & buf) - { - buf.readStrict(reinterpret_cast(&capability_flags), 4); - buf.readStrict(reinterpret_cast(&max_packet_size), 4); - buf.readStrict(reinterpret_cast(&character_set), 1); - } - - OKPacket::OKPacket(uint32_t capabilities_) - : header(0x00), capabilities(capabilities_), affected_rows(0x00), last_insert_id(0x00), status_flags(0x00) - { - } - - OKPacket::OKPacket( - uint8_t header_, - uint32_t capabilities_, - uint64_t affected_rows_, - uint32_t status_flags_, - int16_t warnings_, - String session_state_changes_, - String info_) - : header(header_) - , capabilities(capabilities_) - , affected_rows(affected_rows_) - , last_insert_id(0) - , warnings(warnings_) - , status_flags(status_flags_) - , session_state_changes(std::move(session_state_changes_)) - , info(std::move(info_)) - { - } - - size_t OKPacket::getPayloadSize() const - { - size_t result = 2 + getLengthEncodedNumberSize(affected_rows); - - if (capabilities & CLIENT_PROTOCOL_41) - { - result += 4; - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - result += 2; - } - - if (capabilities & CLIENT_SESSION_TRACK) - { - result += getLengthEncodedStringSize(info); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - result += getLengthEncodedStringSize(session_state_changes); - } - else - { - result += info.size(); - } - - return result; - } - - void OKPacket::readPayloadImpl(ReadBuffer & payload) - - { - payload.readStrict(reinterpret_cast(&header), 1); - affected_rows = readLengthEncodedNumber(payload); - last_insert_id = readLengthEncodedNumber(payload); - - if (capabilities & CLIENT_PROTOCOL_41) - { - payload.readStrict(reinterpret_cast(&status_flags), 2); - payload.readStrict(reinterpret_cast(&warnings), 2); - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - payload.readStrict(reinterpret_cast(&status_flags), 2); - } - - if (capabilities & CLIENT_SESSION_TRACK) - { - readLengthEncodedString(info, payload); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - { - readLengthEncodedString(session_state_changes, payload); - } - } - else - { - readString(info, payload); - } - } - - void OKPacket::writePayloadImpl(WriteBuffer & buffer) const - - { - buffer.write(header); - writeLengthEncodedNumber(affected_rows, buffer); - writeLengthEncodedNumber(last_insert_id, buffer); /// last insert-id - - if (capabilities & CLIENT_PROTOCOL_41) - { - buffer.write(reinterpret_cast(&status_flags), 2); - buffer.write(reinterpret_cast(&warnings), 2); - } - else if (capabilities & CLIENT_TRANSACTIONS) - { - buffer.write(reinterpret_cast(&status_flags), 2); - } - - if (capabilities & CLIENT_SESSION_TRACK) - { - writeLengthEncodedString(info, buffer); - if (status_flags & SERVER_SESSION_STATE_CHANGED) - writeLengthEncodedString(session_state_changes, buffer); - } - else - { - writeString(info, buffer); - } - } - - EOFPacket::EOFPacket() : warnings(0x00), status_flags(0x00) - { - } - - EOFPacket::EOFPacket(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) - { - } - - size_t EOFPacket::getPayloadSize() const - { - return 5; - } - - void EOFPacket::readPayloadImpl(ReadBuffer & payload) - { - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xfe); - payload.readStrict(reinterpret_cast(&warnings), 2); - payload.readStrict(reinterpret_cast(&status_flags), 2); - } - - void EOFPacket::writePayloadImpl(WriteBuffer & buffer) const - { - buffer.write(header); // EOF header - buffer.write(reinterpret_cast(&warnings), 2); - buffer.write(reinterpret_cast(&status_flags), 2); - } - - void AuthSwitchPacket::readPayloadImpl(ReadBuffer & payload) - { - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xfe); - readStringUntilEOF(plugin_name, payload); - } - - ERRPacket::ERRPacket() : error_code(0x00) - { - } - - ERRPacket::ERRPacket(int error_code_, String sql_state_, String error_message_) - : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) - { - } - - size_t ERRPacket::getPayloadSize() const - { - return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); - } - - void ERRPacket::readPayloadImpl(ReadBuffer & payload) - { - payload.readStrict(reinterpret_cast(&header), 1); - assert(header == 0xff); - - payload.readStrict(reinterpret_cast(&error_code), 2); - - /// SQL State [optional: # + 5bytes string] - UInt8 sharp = static_cast(*payload.position()); - if (sharp == 0x23) - { - payload.ignore(1); - sql_state.resize(5); - payload.readStrict(reinterpret_cast(sql_state.data()), 5); - } - readString(error_message, payload); - } - - void ERRPacket::writePayloadImpl(WriteBuffer & buffer) const - { - buffer.write(header); - buffer.write(reinterpret_cast(&error_code), 2); - buffer.write('#'); - buffer.write(sql_state.data(), sql_state.length()); - buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); - } - - ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) : ok(OKPacket(server_capability_flags_)) - { - } - - ResponsePacket::ResponsePacket(UInt32 server_capability_flags_, bool is_handshake_) - : ok(OKPacket(server_capability_flags_)), is_handshake(is_handshake_) - { - } - - void ResponsePacket::readPayloadImpl(ReadBuffer & payload) - { - UInt16 header = static_cast(*payload.position()); - switch (header) - { - case PACKET_OK: - packetType = PACKET_OK; - ok.readPayloadWithUnpacked(payload); - break; - case PACKET_ERR: - packetType = PACKET_ERR; - err.readPayloadWithUnpacked(payload); - break; - case PACKET_EOF: - if (is_handshake) - { - packetType = PACKET_AUTH_SWITCH; - auth_switch.readPayloadWithUnpacked(payload); - } - else - { - packetType = PACKET_EOF; - eof.readPayloadWithUnpacked(payload); - } - break; - case PACKET_LOCALINFILE: - packetType = PACKET_LOCALINFILE; - break; - default: - packetType = PACKET_OK; - column_length = readLengthEncodedNumber(payload); - } - } - - LengthEncodedNumber::LengthEncodedNumber(uint64_t value_) : value(value_) - { - } - - size_t LengthEncodedNumber::getPayloadSize() const - { - return getLengthEncodedNumberSize(value); - } - - void LengthEncodedNumber::writePayloadImpl(WriteBuffer & buffer) const - { - writeLengthEncodedNumber(value, buffer); - } - + result += 4; + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + result += 2; } + if (capabilities & CLIENT_SESSION_TRACK) + { + result += getLengthEncodedStringSize(info); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + result += getLengthEncodedStringSize(session_state_changes); + } + else + { + result += info.size(); + } + + return result; +} + +void OKPacket::readPayloadImpl(ReadBuffer & payload) + +{ + payload.readStrict(reinterpret_cast(&header), 1); + affected_rows = readLengthEncodedNumber(payload); + last_insert_id = readLengthEncodedNumber(payload); + + if (capabilities & CLIENT_PROTOCOL_41) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + payload.readStrict(reinterpret_cast(&warnings), 2); + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + payload.readStrict(reinterpret_cast(&status_flags), 2); + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + readLengthEncodedString(info, payload); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + { + readLengthEncodedString(session_state_changes, payload); + } + } + else + { + readString(info, payload); + } +} + +void OKPacket::writePayloadImpl(WriteBuffer & buffer) const + +{ + buffer.write(header); + writeLengthEncodedNumber(affected_rows, buffer); + writeLengthEncodedNumber(last_insert_id, buffer); /// last insert-id + + if (capabilities & CLIENT_PROTOCOL_41) + { + buffer.write(reinterpret_cast(&status_flags), 2); + buffer.write(reinterpret_cast(&warnings), 2); + } + else if (capabilities & CLIENT_TRANSACTIONS) + { + buffer.write(reinterpret_cast(&status_flags), 2); + } + + if (capabilities & CLIENT_SESSION_TRACK) + { + writeLengthEncodedString(info, buffer); + if (status_flags & SERVER_SESSION_STATE_CHANGED) + writeLengthEncodedString(session_state_changes, buffer); + } + else + { + writeString(info, buffer); + } +} + +EOFPacket::EOFPacket() : warnings(0x00), status_flags(0x00) +{ +} + +EOFPacket::EOFPacket(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) +{ +} + +size_t EOFPacket::getPayloadSize() const +{ + return 5; +} + +void EOFPacket::readPayloadImpl(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + payload.readStrict(reinterpret_cast(&warnings), 2); + payload.readStrict(reinterpret_cast(&status_flags), 2); +} + +void EOFPacket::writePayloadImpl(WriteBuffer & buffer) const +{ + buffer.write(header); // EOF header + buffer.write(reinterpret_cast(&warnings), 2); + buffer.write(reinterpret_cast(&status_flags), 2); +} + +void AuthSwitchPacket::readPayloadImpl(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xfe); + readStringUntilEOF(plugin_name, payload); +} + +ERRPacket::ERRPacket() : error_code(0x00) +{ +} + +ERRPacket::ERRPacket(int error_code_, String sql_state_, String error_message_) + : error_code(error_code_), sql_state(std::move(sql_state_)), error_message(std::move(error_message_)) +{ +} + +size_t ERRPacket::getPayloadSize() const +{ + return 4 + sql_state.length() + std::min(error_message.length(), MYSQL_ERRMSG_SIZE); +} + +void ERRPacket::readPayloadImpl(ReadBuffer & payload) +{ + payload.readStrict(reinterpret_cast(&header), 1); + assert(header == 0xff); + + payload.readStrict(reinterpret_cast(&error_code), 2); + + /// SQL State [optional: # + 5bytes string] + UInt8 sharp = static_cast(*payload.position()); + if (sharp == 0x23) + { + payload.ignore(1); + sql_state.resize(5); + payload.readStrict(reinterpret_cast(sql_state.data()), 5); + } + readString(error_message, payload); +} + +void ERRPacket::writePayloadImpl(WriteBuffer & buffer) const +{ + buffer.write(header); + buffer.write(reinterpret_cast(&error_code), 2); + buffer.write('#'); + buffer.write(sql_state.data(), sql_state.length()); + buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); +} + +ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) : ok(OKPacket(server_capability_flags_)) +{ +} + +ResponsePacket::ResponsePacket(UInt32 server_capability_flags_, bool is_handshake_) + : ok(OKPacket(server_capability_flags_)), is_handshake(is_handshake_) +{ +} + +void ResponsePacket::readPayloadImpl(ReadBuffer & payload) +{ + UInt16 header = static_cast(*payload.position()); + switch (header) + { + case PACKET_OK: + packetType = PACKET_OK; + ok.readPayloadWithUnpacked(payload); + break; + case PACKET_ERR: + packetType = PACKET_ERR; + err.readPayloadWithUnpacked(payload); + break; + case PACKET_EOF: + if (is_handshake) + { + packetType = PACKET_AUTH_SWITCH; + auth_switch.readPayloadWithUnpacked(payload); + } + else + { + packetType = PACKET_EOF; + eof.readPayloadWithUnpacked(payload); + } + break; + case PACKET_LOCALINFILE: + packetType = PACKET_LOCALINFILE; + break; + default: + packetType = PACKET_OK; + column_length = readLengthEncodedNumber(payload); + } +} + +LengthEncodedNumber::LengthEncodedNumber(uint64_t value_) : value(value_) +{ +} + +size_t LengthEncodedNumber::getPayloadSize() const +{ + return getLengthEncodedNumberSize(value); +} + +void LengthEncodedNumber::writePayloadImpl(WriteBuffer & buffer) const +{ + writeLengthEncodedNumber(value, buffer); +} + +} + } } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 9f4de31e44d..f01f2ff9b61 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -12,238 +12,238 @@ namespace DB namespace MySQLProtocol { - namespace ProtocolText +namespace ProtocolText +{ + +ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) + : columns(columns_), row_num(row_num_) +{ + for (size_t i = 0; i < columns.size(); ++i) { - - ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) - : columns(columns_), row_num(row_num_) + if (columns[i]->isNullAt(row_num)) { - for (size_t i = 0; i < columns.size(); ++i) - { - if (columns[i]->isNullAt(row_num)) - { - payload_size += 1; - serialized.emplace_back("\xfb"); - } - else - { - WriteBufferFromOwnString ostr; - serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings()); - payload_size += getLengthEncodedStringSize(ostr.str()); - serialized.push_back(std::move(ostr.str())); - } - } + payload_size += 1; + serialized.emplace_back("\xfb"); } - - size_t ResultSetRow::getPayloadSize() const + else { - return payload_size; + WriteBufferFromOwnString ostr; + serializations[i]->serializeText(*columns[i], row_num, ostr, FormatSettings()); + payload_size += getLengthEncodedStringSize(ostr.str()); + serialized.push_back(std::move(ostr.str())); } - - void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const - { - for (size_t i = 0; i < columns.size(); ++i) - { - if (columns[i]->isNullAt(row_num)) - buffer.write(serialized[i].data(), 1); - else - writeLengthEncodedString(serialized[i], buffer); - } - } - - void ComFieldList::readPayloadImpl(ReadBuffer & payload) - { - // Command byte has been already read from payload. - readNullTerminated(table, payload); - readStringUntilEOF(field_wildcard, payload); - } - - ColumnDefinition::ColumnDefinition() : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) - { - } - - ColumnDefinition::ColumnDefinition( - String schema_, - String table_, - String org_table_, - String name_, - String org_name_, - uint16_t character_set_, - uint32_t column_length_, - ColumnType column_type_, - uint16_t flags_, - uint8_t decimals_, - bool with_defaults_) - : schema(std::move(schema_)) - , table(std::move(table_)) - , org_table(std::move(org_table_)) - , name(std::move(name_)) - , org_name(std::move(org_name_)) - , character_set(character_set_) - , column_length(column_length_) - , column_type(column_type_) - , flags(flags_) - , decimals(decimals_) - , is_comm_field_list_response(with_defaults_) - { - } - - ColumnDefinition::ColumnDefinition( - String name_, uint16_t character_set_, uint32_t column_length_, ColumnType column_type_, uint16_t flags_, uint8_t decimals_) - : ColumnDefinition("", "", "", std::move(name_), "", character_set_, column_length_, column_type_, flags_, decimals_) - { - } - - size_t ColumnDefinition::getPayloadSize() const - { - return 12 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) - + getLengthEncodedStringSize(org_table) + getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) - + getLengthEncodedNumberSize(next_length) + is_comm_field_list_response; - } - - void ColumnDefinition::readPayloadImpl(ReadBuffer & payload) - { - String def; - readLengthEncodedString(def, payload); - assert(def == "def"); - readLengthEncodedString(schema, payload); - readLengthEncodedString(table, payload); - readLengthEncodedString(org_table, payload); - readLengthEncodedString(name, payload); - readLengthEncodedString(org_name, payload); - next_length = readLengthEncodedNumber(payload); - payload.readStrict(reinterpret_cast(&character_set), 2); - payload.readStrict(reinterpret_cast(&column_length), 4); - payload.readStrict(reinterpret_cast(&column_type), 1); - payload.readStrict(reinterpret_cast(&flags), 2); - payload.readStrict(reinterpret_cast(&decimals), 1); - payload.ignore(2); - } - - void ColumnDefinition::writePayloadImpl(WriteBuffer & buffer) const - { - writeLengthEncodedString(std::string("def"), buffer); /// always "def" - writeLengthEncodedString(schema, buffer); - writeLengthEncodedString(table, buffer); - writeLengthEncodedString(org_table, buffer); - writeLengthEncodedString(name, buffer); - writeLengthEncodedString(org_name, buffer); - writeLengthEncodedNumber(next_length, buffer); - buffer.write(reinterpret_cast(&character_set), 2); - buffer.write(reinterpret_cast(&column_length), 4); - buffer.write(reinterpret_cast(&column_type), 1); - buffer.write(reinterpret_cast(&flags), 2); - buffer.write(reinterpret_cast(&decimals), 1); - writeChar(0x0, 2, buffer); - if (is_comm_field_list_response) - { - /// We should write length encoded int with string size - /// followed by string with some "default values" (possibly it's column defaults). - /// But we just send NULL for simplicity. - writeChar(0xfb, buffer); - } - } - - ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type) - { - ColumnType column_type; - CharacterSet charset = CharacterSet::binary; - int flags = 0; - uint8_t decimals = 0; - TypeIndex type_index = data_type->getTypeId(); - switch (type_index) - { - case TypeIndex::UInt8: - if (data_type->getName() == "Bool") - { - column_type = ColumnType::MYSQL_TYPE_BIT; - } - else - { - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - } - break; - case TypeIndex::UInt16: - column_type = ColumnType::MYSQL_TYPE_SHORT; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::UInt32: - column_type = ColumnType::MYSQL_TYPE_LONG; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::UInt64: - column_type = ColumnType::MYSQL_TYPE_LONGLONG; - flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; - break; - case TypeIndex::Int8: - column_type = ColumnType::MYSQL_TYPE_TINY; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int16: - column_type = ColumnType::MYSQL_TYPE_SHORT; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int32: - column_type = ColumnType::MYSQL_TYPE_LONG; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Int64: - column_type = ColumnType::MYSQL_TYPE_LONGLONG; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Float32: - column_type = ColumnType::MYSQL_TYPE_FLOAT; - flags = ColumnDefinitionFlags::BINARY_FLAG; - decimals = 31; - break; - case TypeIndex::Float64: - column_type = ColumnType::MYSQL_TYPE_DOUBLE; - flags = ColumnDefinitionFlags::BINARY_FLAG; - decimals = 31; - break; - case TypeIndex::Date: - case TypeIndex::Date32: - column_type = ColumnType::MYSQL_TYPE_DATE; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::DateTime: - column_type = ColumnType::MYSQL_TYPE_DATETIME; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::DateTime64: - column_type = ColumnType::MYSQL_TYPE_DATETIME; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Decimal32: - case TypeIndex::Decimal64: - column_type = ColumnType::MYSQL_TYPE_DECIMAL; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; - case TypeIndex::Decimal128: { - // MySQL Decimal has max 65 precision and 30 scale - const auto & type = assert_cast(*data_type); - if (type.getPrecision() > 65 || type.getScale() > 30) - { - column_type = ColumnType::MYSQL_TYPE_STRING; - charset = CharacterSet::utf8_general_ci; - } - else - { - column_type = ColumnType::MYSQL_TYPE_DECIMAL; - flags = ColumnDefinitionFlags::BINARY_FLAG; - } - break; - } - default: - column_type = ColumnType::MYSQL_TYPE_STRING; - charset = CharacterSet::utf8_general_ci; - break; - } - return ColumnDefinition(column_name, charset, 0, column_type, flags, decimals); - } - } +} + +size_t ResultSetRow::getPayloadSize() const +{ + return payload_size; +} + +void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const +{ + for (size_t i = 0; i < columns.size(); ++i) + { + if (columns[i]->isNullAt(row_num)) + buffer.write(serialized[i].data(), 1); + else + writeLengthEncodedString(serialized[i], buffer); + } +} + +void ComFieldList::readPayloadImpl(ReadBuffer & payload) +{ + // Command byte has been already read from payload. + readNullTerminated(table, payload); + readStringUntilEOF(field_wildcard, payload); +} + +ColumnDefinition::ColumnDefinition() : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) +{ +} + +ColumnDefinition::ColumnDefinition( + String schema_, + String table_, + String org_table_, + String name_, + String org_name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_, + bool with_defaults_) + : schema(std::move(schema_)) + , table(std::move(table_)) + , org_table(std::move(org_table_)) + , name(std::move(name_)) + , org_name(std::move(org_name_)) + , character_set(character_set_) + , column_length(column_length_) + , column_type(column_type_) + , flags(flags_) + , decimals(decimals_) + , is_comm_field_list_response(with_defaults_) +{ +} + +ColumnDefinition::ColumnDefinition( + String name_, uint16_t character_set_, uint32_t column_length_, ColumnType column_type_, uint16_t flags_, uint8_t decimals_) + : ColumnDefinition("", "", "", std::move(name_), "", character_set_, column_length_, column_type_, flags_, decimals_) +{ +} + +size_t ColumnDefinition::getPayloadSize() const +{ + return 12 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) + + getLengthEncodedStringSize(org_table) + getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) + + getLengthEncodedNumberSize(next_length) + is_comm_field_list_response; +} + +void ColumnDefinition::readPayloadImpl(ReadBuffer & payload) +{ + String def; + readLengthEncodedString(def, payload); + assert(def == "def"); + readLengthEncodedString(schema, payload); + readLengthEncodedString(table, payload); + readLengthEncodedString(org_table, payload); + readLengthEncodedString(name, payload); + readLengthEncodedString(org_name, payload); + next_length = readLengthEncodedNumber(payload); + payload.readStrict(reinterpret_cast(&character_set), 2); + payload.readStrict(reinterpret_cast(&column_length), 4); + payload.readStrict(reinterpret_cast(&column_type), 1); + payload.readStrict(reinterpret_cast(&flags), 2); + payload.readStrict(reinterpret_cast(&decimals), 1); + payload.ignore(2); +} + +void ColumnDefinition::writePayloadImpl(WriteBuffer & buffer) const +{ + writeLengthEncodedString(std::string("def"), buffer); /// always "def" + writeLengthEncodedString(schema, buffer); + writeLengthEncodedString(table, buffer); + writeLengthEncodedString(org_table, buffer); + writeLengthEncodedString(name, buffer); + writeLengthEncodedString(org_name, buffer); + writeLengthEncodedNumber(next_length, buffer); + buffer.write(reinterpret_cast(&character_set), 2); + buffer.write(reinterpret_cast(&column_length), 4); + buffer.write(reinterpret_cast(&column_type), 1); + buffer.write(reinterpret_cast(&flags), 2); + buffer.write(reinterpret_cast(&decimals), 1); + writeChar(0x0, 2, buffer); + if (is_comm_field_list_response) + { + /// We should write length encoded int with string size + /// followed by string with some "default values" (possibly it's column defaults). + /// But we just send NULL for simplicity. + writeChar(0xfb, buffer); + } +} + +ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type) +{ + ColumnType column_type; + CharacterSet charset = CharacterSet::binary; + int flags = 0; + uint8_t decimals = 0; + TypeIndex type_index = data_type->getTypeId(); + switch (type_index) + { + case TypeIndex::UInt8: + if (data_type->getName() == "Bool") + { + column_type = ColumnType::MYSQL_TYPE_BIT; + } + else + { + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + } + break; + case TypeIndex::UInt16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::Int8: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Float32: + column_type = ColumnType::MYSQL_TYPE_FLOAT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + decimals = 31; + break; + case TypeIndex::Float64: + column_type = ColumnType::MYSQL_TYPE_DOUBLE; + flags = ColumnDefinitionFlags::BINARY_FLAG; + decimals = 31; + break; + case TypeIndex::Date: + case TypeIndex::Date32: + column_type = ColumnType::MYSQL_TYPE_DATE; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::DateTime: + column_type = ColumnType::MYSQL_TYPE_DATETIME; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::DateTime64: + column_type = ColumnType::MYSQL_TYPE_DATETIME; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Decimal32: + case TypeIndex::Decimal64: + column_type = ColumnType::MYSQL_TYPE_DECIMAL; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Decimal128: { + // MySQL Decimal has max 65 precision and 30 scale + const auto & type = assert_cast(*data_type); + if (type.getPrecision() > 65 || type.getScale() > 30) + { + column_type = ColumnType::MYSQL_TYPE_STRING; + charset = CharacterSet::utf8_general_ci; + } + else + { + column_type = ColumnType::MYSQL_TYPE_DECIMAL; + flags = ColumnDefinitionFlags::BINARY_FLAG; + } + break; + } + default: + column_type = ColumnType::MYSQL_TYPE_STRING; + charset = CharacterSet::utf8_general_ci; + break; + } + return ColumnDefinition(column_name, charset, 0, column_type, flags, decimals); +} + +} } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 2af216529e5..41373d1f5e7 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -81,14 +81,14 @@ SerializationPtr DataTypeDecimal::doGetDefaultSerialization() const static DataTypePtr create(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 2) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Decimal data type family must have exactly two arguments: precision and scale"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Decimal data type family must have exactly two arguments: precision and scale"); const auto * precision = arguments->children[0]->as(); const auto * scale = arguments->children[1]->as(); - if (!precision || precision->value.getType() != Field::Types::UInt64 || !scale - || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) + if (!precision || precision->value.getType() != Field::Types::UInt64 || + !scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Decimal data type family must have two numbers as its arguments"); UInt64 precision_value = precision->value.get(); @@ -101,15 +101,13 @@ template static DataTypePtr createExact(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 1) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have exactly one arguments: scale"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have exactly one arguments: scale"); const auto * scale_arg = arguments->children[0]->as(); if (!scale_arg || !(scale_arg->value.getType() == Field::Types::Int64 || scale_arg->value.getType() == Field::Types::UInt64)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Decimal32 | Decimal64 | Decimal128 | Decimal256 data type family must have a one number as its argument"); UInt64 precision = DecimalUtils::max_precision; UInt64 scale = scale_arg->value.get(); From bb0eff9669d03ddc1be52f613ccbe35f6de484d2 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 21:15:26 +0200 Subject: [PATCH 070/327] Revert format changes --- src/Core/MySQL/PacketsGeneric.cpp | 8 +- src/Core/MySQL/PacketsProtocolText.cpp | 42 ++-- src/Core/MySQL/PacketsProtocolText.h | 224 +++++++++--------- src/DataTypes/DataTypesDecimal.cpp | 2 +- .../Serializations/SerializationDate32.cpp | 14 +- src/Formats/FormatSettings.h | 8 +- .../Formats/Impl/MySQLOutputFormat.cpp | 8 +- src/Server/MySQLHandler.cpp | 92 +++---- src/Server/MySQLHandler.h | 22 +- 9 files changed, 191 insertions(+), 229 deletions(-) diff --git a/src/Core/MySQL/PacketsGeneric.cpp b/src/Core/MySQL/PacketsGeneric.cpp index 23f197bcfe8..af80797d5c1 100644 --- a/src/Core/MySQL/PacketsGeneric.cpp +++ b/src/Core/MySQL/PacketsGeneric.cpp @@ -29,7 +29,7 @@ OKPacket::OKPacket( uint8_t header_, uint32_t capabilities_, uint64_t affected_rows_, uint32_t status_flags_, int16_t warnings_, String session_state_changes_, String info_) : header(header_), capabilities(capabilities_), affected_rows(affected_rows_), last_insert_id(0), warnings(warnings_), - status_flags(status_flags_), session_state_changes(std::move(session_state_changes_)), info(std::move(info_)) + status_flags(status_flags_), session_state_changes(std::move(session_state_changes_)), info(std::move(info_)) { } @@ -124,7 +124,8 @@ EOFPacket::EOFPacket() : warnings(0x00), status_flags(0x00) { } -EOFPacket::EOFPacket(int warnings_, int status_flags_) : warnings(warnings_), status_flags(status_flags_) +EOFPacket::EOFPacket(int warnings_, int status_flags_) + : warnings(warnings_), status_flags(status_flags_) { } @@ -196,7 +197,8 @@ void ERRPacket::writePayloadImpl(WriteBuffer & buffer) const buffer.write(error_message.data(), std::min(error_message.length(), MYSQL_ERRMSG_SIZE)); } -ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) : ok(OKPacket(server_capability_flags_)) +ResponsePacket::ResponsePacket(UInt32 server_capability_flags_) + : ok(OKPacket(server_capability_flags_)) { } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index f01f2ff9b61..ebfff1c2370 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -58,33 +58,17 @@ void ComFieldList::readPayloadImpl(ReadBuffer & payload) readStringUntilEOF(field_wildcard, payload); } -ColumnDefinition::ColumnDefinition() : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) +ColumnDefinition::ColumnDefinition() + : character_set(0x00), column_length(0), column_type(MYSQL_TYPE_DECIMAL), flags(0x00) { } ColumnDefinition::ColumnDefinition( - String schema_, - String table_, - String org_table_, - String name_, - String org_name_, - uint16_t character_set_, - uint32_t column_length_, - ColumnType column_type_, - uint16_t flags_, - uint8_t decimals_, - bool with_defaults_) - : schema(std::move(schema_)) - , table(std::move(table_)) - , org_table(std::move(org_table_)) - , name(std::move(name_)) - , org_name(std::move(org_name_)) - , character_set(character_set_) - , column_length(column_length_) - , column_type(column_type_) - , flags(flags_) - , decimals(decimals_) - , is_comm_field_list_response(with_defaults_) + String schema_, String table_, String org_table_, String name_, String org_name_, uint16_t character_set_, uint32_t column_length_, + ColumnType column_type_, uint16_t flags_, uint8_t decimals_, bool with_defaults_) + : schema(std::move(schema_)), table(std::move(table_)), org_table(std::move(org_table_)), name(std::move(name_)), + org_name(std::move(org_name_)), character_set(character_set_), column_length(column_length_), column_type(column_type_), + flags(flags_), decimals(decimals_), is_comm_field_list_response(with_defaults_) { } @@ -96,9 +80,15 @@ ColumnDefinition::ColumnDefinition( size_t ColumnDefinition::getPayloadSize() const { - return 12 + getLengthEncodedStringSize("def") + getLengthEncodedStringSize(schema) + getLengthEncodedStringSize(table) - + getLengthEncodedStringSize(org_table) + getLengthEncodedStringSize(name) + getLengthEncodedStringSize(org_name) - + getLengthEncodedNumberSize(next_length) + is_comm_field_list_response; + return 12 + + getLengthEncodedStringSize("def") + + getLengthEncodedStringSize(schema) + + getLengthEncodedStringSize(table) + + getLengthEncodedStringSize(org_table) + + getLengthEncodedStringSize(name) + + getLengthEncodedStringSize(org_name) + + getLengthEncodedNumberSize(next_length) + + is_comm_field_list_response; } void ColumnDefinition::readPayloadImpl(ReadBuffer & payload) diff --git a/src/Core/MySQL/PacketsProtocolText.h b/src/Core/MySQL/PacketsProtocolText.h index d9ce913494b..1de9aabe9bb 100644 --- a/src/Core/MySQL/PacketsProtocolText.h +++ b/src/Core/MySQL/PacketsProtocolText.h @@ -12,135 +12,135 @@ namespace DB namespace MySQLProtocol { - namespace ProtocolText - { +namespace ProtocolText +{ - enum CharacterSet - { - utf8_general_ci = 33, - binary = 63 - }; +enum CharacterSet +{ + utf8_general_ci = 33, + binary = 63 +}; - // https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html - enum ColumnDefinitionFlags - { - UNSIGNED_FLAG = 32, - BINARY_FLAG = 128 - }; +// https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html +enum ColumnDefinitionFlags +{ + UNSIGNED_FLAG = 32, + BINARY_FLAG = 128 +}; - enum ColumnType - { - MYSQL_TYPE_DECIMAL = 0x00, - MYSQL_TYPE_TINY = 0x01, - MYSQL_TYPE_SHORT = 0x02, - MYSQL_TYPE_LONG = 0x03, - MYSQL_TYPE_FLOAT = 0x04, - MYSQL_TYPE_DOUBLE = 0x05, - MYSQL_TYPE_NULL = 0x06, - MYSQL_TYPE_TIMESTAMP = 0x07, - MYSQL_TYPE_LONGLONG = 0x08, - MYSQL_TYPE_INT24 = 0x09, - MYSQL_TYPE_DATE = 0x0a, - MYSQL_TYPE_TIME = 0x0b, - MYSQL_TYPE_DATETIME = 0x0c, - MYSQL_TYPE_YEAR = 0x0d, - MYSQL_TYPE_NEWDATE = 0x0e, - MYSQL_TYPE_VARCHAR = 0x0f, - MYSQL_TYPE_BIT = 0x10, - MYSQL_TYPE_TIMESTAMP2 = 0x11, - MYSQL_TYPE_DATETIME2 = 0x12, - MYSQL_TYPE_TIME2 = 0x13, - MYSQL_TYPE_JSON = 0xf5, - MYSQL_TYPE_NEWDECIMAL = 0xf6, - MYSQL_TYPE_ENUM = 0xf7, - MYSQL_TYPE_SET = 0xf8, - MYSQL_TYPE_TINY_BLOB = 0xf9, - MYSQL_TYPE_MEDIUM_BLOB = 0xfa, - MYSQL_TYPE_LONG_BLOB = 0xfb, - MYSQL_TYPE_BLOB = 0xfc, - MYSQL_TYPE_VAR_STRING = 0xfd, - MYSQL_TYPE_STRING = 0xfe, - MYSQL_TYPE_GEOMETRY = 0xff - }; +enum ColumnType +{ + MYSQL_TYPE_DECIMAL = 0x00, + MYSQL_TYPE_TINY = 0x01, + MYSQL_TYPE_SHORT = 0x02, + MYSQL_TYPE_LONG = 0x03, + MYSQL_TYPE_FLOAT = 0x04, + MYSQL_TYPE_DOUBLE = 0x05, + MYSQL_TYPE_NULL = 0x06, + MYSQL_TYPE_TIMESTAMP = 0x07, + MYSQL_TYPE_LONGLONG = 0x08, + MYSQL_TYPE_INT24 = 0x09, + MYSQL_TYPE_DATE = 0x0a, + MYSQL_TYPE_TIME = 0x0b, + MYSQL_TYPE_DATETIME = 0x0c, + MYSQL_TYPE_YEAR = 0x0d, + MYSQL_TYPE_NEWDATE = 0x0e, + MYSQL_TYPE_VARCHAR = 0x0f, + MYSQL_TYPE_BIT = 0x10, + MYSQL_TYPE_TIMESTAMP2 = 0x11, + MYSQL_TYPE_DATETIME2 = 0x12, + MYSQL_TYPE_TIME2 = 0x13, + MYSQL_TYPE_JSON = 0xf5, + MYSQL_TYPE_NEWDECIMAL = 0xf6, + MYSQL_TYPE_ENUM = 0xf7, + MYSQL_TYPE_SET = 0xf8, + MYSQL_TYPE_TINY_BLOB = 0xf9, + MYSQL_TYPE_MEDIUM_BLOB = 0xfa, + MYSQL_TYPE_LONG_BLOB = 0xfb, + MYSQL_TYPE_BLOB = 0xfc, + MYSQL_TYPE_VAR_STRING = 0xfd, + MYSQL_TYPE_STRING = 0xfe, + MYSQL_TYPE_GEOMETRY = 0xff +}; - class ResultSetRow : public IMySQLWritePacket - { - protected: - const Columns & columns; - int row_num; - size_t payload_size = 0; - std::vector serialized; +class ResultSetRow : public IMySQLWritePacket +{ +protected: + const Columns & columns; + int row_num; + size_t payload_size = 0; + std::vector serialized; - size_t getPayloadSize() const override; + size_t getPayloadSize() const override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; - public: - ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_); - }; +public: + ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_); +}; - class ComFieldList : public LimitedReadPacket - { - public: - String table, field_wildcard; +class ComFieldList : public LimitedReadPacket +{ +public: + String table, field_wildcard; - void readPayloadImpl(ReadBuffer & payload) override; - }; + void readPayloadImpl(ReadBuffer & payload) override; +}; - class ColumnDefinition : public IMySQLWritePacket, public IMySQLReadPacket - { - public: - String schema; - String table; - String org_table; - String name; - String org_name; - size_t next_length = 0x0c; - uint16_t character_set; - uint32_t column_length; - ColumnType column_type; - uint16_t flags; - uint8_t decimals = 0x00; - /// https://dev.mysql.com/doc/internals/en/com-query-response.html#column-definition - /// There are extra fields in the packet for column defaults - bool is_comm_field_list_response = false; +class ColumnDefinition : public IMySQLWritePacket, public IMySQLReadPacket +{ +public: + String schema; + String table; + String org_table; + String name; + String org_name; + size_t next_length = 0x0c; + uint16_t character_set; + uint32_t column_length; + ColumnType column_type; + uint16_t flags; + uint8_t decimals = 0x00; + /// https://dev.mysql.com/doc/internals/en/com-query-response.html#column-definition + /// There are extra fields in the packet for column defaults + bool is_comm_field_list_response = false; - protected: - size_t getPayloadSize() const override; +protected: + size_t getPayloadSize() const override; - void readPayloadImpl(ReadBuffer & payload) override; + void readPayloadImpl(ReadBuffer & payload) override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; - public: - ColumnDefinition(); +public: + ColumnDefinition(); - ColumnDefinition( - String schema_, - String table_, - String org_table_, - String name_, - String org_name_, - uint16_t character_set_, - uint32_t column_length_, - ColumnType column_type_, - uint16_t flags_, - uint8_t decimals_, - bool with_defaults_ = false); + ColumnDefinition( + String schema_, + String table_, + String org_table_, + String name_, + String org_name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_, + bool with_defaults_ = false); - /// Should be used when column metadata (original name, table, original table, database) is unknown. - ColumnDefinition( - String name_, - uint16_t character_set_, - uint32_t column_length_, - ColumnType column_type_, - uint16_t flags_, - uint8_t decimals_); - }; + /// Should be used when column metadata (original name, table, original table, database) is unknown. + ColumnDefinition( + String name_, + uint16_t character_set_, + uint32_t column_length_, + ColumnType column_type_, + uint16_t flags_, + uint8_t decimals_); +}; - ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type); - - } +ColumnDefinition getColumnDefinition(const String & column_name, const DataTypePtr & data_type); + +} } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 41373d1f5e7..e830363d9f1 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -1,13 +1,13 @@ #include #include +#include #include #include #include #include #include #include -#include #include diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index 68213a14afc..851710de839 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -28,14 +28,12 @@ void SerializationDate32::deserializeTextEscaped(IColumn & column, ReadBuffer & assert_cast(column).getData().push_back(x); } -void SerializationDate32::serializeTextEscaped( - const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { serializeText(column, row_num, ostr, settings); } -void SerializationDate32::serializeTextQuoted( - const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('\'', ostr); serializeText(column, row_num, ostr, settings); @@ -48,11 +46,10 @@ void SerializationDate32::deserializeTextQuoted(IColumn & column, ReadBuffer & i assertChar('\'', istr); readDateText(x, istr, time_zone); assertChar('\'', istr); - assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. + assert_cast(column).getData().push_back(x); /// It's important to do this at the end - for exception safety. } -void SerializationDate32::serializeTextJSON( - const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); serializeText(column, row_num, ostr, settings); @@ -68,8 +65,7 @@ void SerializationDate32::deserializeTextJSON(IColumn & column, ReadBuffer & ist assert_cast(column).getData().push_back(x); } -void SerializationDate32::serializeTextCSV( - const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void SerializationDate32::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('"', ostr); serializeText(column, row_num, ostr, settings); diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 986d79a0b7b..2f1d64e7bc6 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include #include @@ -48,9 +48,9 @@ struct FormatSettings enum class DateTimeInputFormat { - Basic, /// Default format for fast parsing: YYYY-MM-DD hh:mm:ss (ISO-8601 without fractional part and timezone) or NNNNNNNNNN unix timestamp. - BestEffort, /// Use sophisticated rules to parse whatever possible. - BestEffortUS /// Use sophisticated rules to parse American style: mm/dd/yyyy + Basic, /// Default format for fast parsing: YYYY-MM-DD hh:mm:ss (ISO-8601 without fractional part and timezone) or NNNNNNNNNN unix timestamp. + BestEffort, /// Use sophisticated rules to parse whatever possible. + BestEffortUS /// Use sophisticated rules to parse American style: mm/dd/yyyy }; DateTimeInputFormat date_time_input_format = DateTimeInputFormat::Basic; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index d25b68507fc..eaae8c6555c 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -17,7 +17,8 @@ using namespace MySQLProtocol::ProtocolText; using namespace MySQLProtocol::ProtocolBinary; MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & settings_) - : IOutputFormat(header_, out_), client_capabilities(settings_.mysql_wire.client_capabilities) + : IOutputFormat(header_, out_) + , client_capabilities(settings_.mysql_wire.client_capabilities) { /// MySQlWire is a special format that is usually used as output format for MySQL protocol connections. /// In this case we have a correct `sequence_id` stored in `settings_.mysql_wire`. @@ -137,8 +138,9 @@ void registerOutputFormatMySQLWire(FormatFactory & factory) { factory.registerOutputFormat( "MySQLWire", - [](WriteBuffer & buf, const Block & sample, const FormatSettings & settings) - { return std::make_shared(buf, sample, settings); }); + [](WriteBuffer & buf, + const Block & sample, + const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); } } diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 3715dfea9f7..79b645eafd2 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -67,7 +67,10 @@ static String killConnectionIdReplacementQuery(const String & query); static String selectLimitReplacementQuery(const String & query); MySQLHandler::MySQLHandler( - IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, uint32_t connection_id_) + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, uint32_t connection_id_) : Poco::Net::TCPServerConnection(socket_) , server(server_) , tcp_server(tcp_server_) @@ -75,8 +78,7 @@ MySQLHandler::MySQLHandler( , connection_id(connection_id_) , auth_plugin(new MySQLProtocol::Authentication::Native41()) { - server_capabilities = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA - | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; + server_capabilities = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; if (ssl_enabled) server_capabilities |= CLIENT_SSL; @@ -102,13 +104,8 @@ void MySQLHandler::run() try { - Handshake handshake( - server_capabilities, - connection_id, - VERSION_STRING + String("-") + VERSION_NAME, - auth_plugin->getName(), - auth_plugin->getAuthPluginData(), - CharacterSet::utf8_general_ci); + Handshake handshake(server_capabilities, connection_id, VERSION_STRING + String("-") + VERSION_NAME, + auth_plugin->getName(), auth_plugin->getAuthPluginData(), CharacterSet::utf8_general_ci); packet_endpoint->sendPacket(handshake, true); LOG_TRACE(log, "Sent handshake"); @@ -118,10 +115,8 @@ void MySQLHandler::run() client_capabilities = handshake_response.capability_flags; max_packet_size = handshake_response.max_packet_size ? handshake_response.max_packet_size : MAX_PACKET_LENGTH; - LOG_TRACE( - log, - "Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: " - "{}", + LOG_TRACE(log, + "Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: {}", handshake_response.capability_flags, handshake_response.max_packet_size, static_cast(handshake_response.character_set), @@ -165,8 +160,8 @@ void MySQLHandler::run() // For commands which are executed without MemoryTracker. LimitReadBuffer limited_payload(payload, 10000, /* trow_exception */ true, /* exact_limit */ {}, "too long MySQL packet."); - LOG_DEBUG( - log, "Received command: {}. Connection id: {}.", static_cast(static_cast(command)), connection_id); + LOG_DEBUG(log, "Received command: {}. Connection id: {}.", + static_cast(static_cast(command)), connection_id); if (!tcp_server.isOpen()) return; @@ -232,15 +227,13 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp size_t pos = 0; /// Reads at least count and at most packet_size bytes. - auto read_bytes = [this, &buf, &pos, &packet_size](size_t count) -> void - { + auto read_bytes = [this, &buf, &pos, &packet_size](size_t count) -> void { while (pos < count) { int ret = socket().receiveBytes(buf + pos, static_cast(packet_size - pos)); if (ret == 0) { - throw Exception( - ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Bytes read: {}. Bytes expected: 3", std::to_string(pos)); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Bytes read: {}. Bytes expected: 3", std::to_string(pos)); } pos += ret; } @@ -279,8 +272,7 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl authPluginSSL(); } - std::optional auth_response - = auth_plugin_name == auth_plugin->getName() ? std::make_optional(initial_auth_response) : std::nullopt; + std::optional auth_response = auth_plugin_name == auth_plugin->getName() ? std::make_optional(initial_auth_response) : std::nullopt; auth_plugin->authenticate(user_name, *session, auth_response, packet_endpoint, secure_connection, socket().peerAddress()); } catch (const Exception & exc) @@ -312,17 +304,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAll()) { ColumnDefinition column_definition( - database, - packet.table, - packet.table, - column.name, - column.name, - CharacterSet::binary, - 100, - ColumnType::MYSQL_TYPE_STRING, - 0, - 0, - true); + database, packet.table, packet.table, column.name, column.name, CharacterSet::binary, 100, ColumnType::MYSQL_TYPE_STRING, 0, 0, true + ); packet_endpoint->sendPacket(column_definition); } packet_endpoint->sendPacket(OKPacket(0xfe, client_capabilities, 0, 0, 0), true); @@ -367,16 +350,15 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool use_binary_protocol_resul query_context->setCurrentQueryId(fmt::format("mysql:{}:{}", connection_id, toString(UUIDHelpers::generateV4()))); CurrentThread::QueryScope query_scope{query_context}; - std::atomic affected_rows{0}; + std::atomic affected_rows {0}; auto prev = query_context->getProgressCallback(); - query_context->setProgressCallback( - [&, my_prev = prev](const Progress & progress) - { - if (my_prev) - my_prev(progress); + query_context->setProgressCallback([&, my_prev = prev](const Progress & progress) + { + if (my_prev) + my_prev(progress); - affected_rows += progress.written_rows; - }); + affected_rows += progress.written_rows; + }); FormatSettings format_settings; format_settings.mysql_wire.client_capabilities = client_capabilities; @@ -475,17 +457,13 @@ void MySQLHandler::comStmtClose(ReadBuffer & payload) void MySQLHandler::authPluginSSL() { - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "ClickHouse was built without SSL support. Try specifying password using double SHA1 in users.xml."); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "ClickHouse was built without SSL support. Try specifying password using double SHA1 in users.xml."); } void MySQLHandler::finishHandshakeSSL( - [[maybe_unused]] size_t packet_size, - [[maybe_unused]] char * buf, - [[maybe_unused]] size_t pos, - [[maybe_unused]] std::function read_bytes, - [[maybe_unused]] MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) + [[maybe_unused]] size_t packet_size, [[maybe_unused]] char * buf, [[maybe_unused]] size_t pos, + [[maybe_unused]] std::function read_bytes, [[maybe_unused]] MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Client requested SSL, while it is disabled."); } @@ -499,9 +477,10 @@ MySQLHandlerSSL::MySQLHandlerSSL( uint32_t connection_id_, RSA & public_key_, RSA & private_key_) - : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_), public_key(public_key_), private_key(private_key_) -{ -} + : MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_) + , public_key(public_key_) + , private_key(private_key_) +{} void MySQLHandlerSSL::authPluginSSL() { @@ -509,10 +488,7 @@ void MySQLHandlerSSL::authPluginSSL() } void MySQLHandlerSSL::finishHandshakeSSL( - size_t packet_size, - char * buf, - size_t pos, - std::function read_bytes, + size_t packet_size, char *buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) { read_bytes(packet_size); /// Reading rest SSLRequest. @@ -542,8 +518,8 @@ static bool isFederatedServerSetupSetCommand(const String & query) "|(^(SET AUTOCOMMIT(.*)))" "|(^(SET sql_mode(.*)))" "|(^(SET @@(.*)))" - "|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))", - std::regex::icase}; + "|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))" + , std::regex::icase}; return 1 == std::regex_match(query, expr); } diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index a412b647ae2..240057e8ba7 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -19,7 +19,7 @@ namespace CurrentMetrics { -extern const Metric MySQLConnection; + extern const Metric MySQLConnection; } namespace DB @@ -32,7 +32,11 @@ class MySQLHandler : public Poco::Net::TCPServerConnection { public: MySQLHandler( - IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, uint32_t connection_id_); + IServer & server_, + TCPServer & tcp_server_, + const Poco::Net::StreamSocket & socket_, + bool ssl_enabled, + uint32_t connection_id_); void run() final; @@ -59,12 +63,7 @@ protected: void comStmtClose(ReadBuffer & payload); virtual void authPluginSSL(); - virtual void finishHandshakeSSL( - size_t packet_size, - char * buf, - size_t pos, - std::function read_bytes, - MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); + virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); IServer & server; TCPServer & tcp_server; @@ -110,11 +109,8 @@ private: void authPluginSSL() override; void finishHandshakeSSL( - size_t packet_size, - char * buf, - size_t pos, - std::function read_bytes, - MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) override; + size_t packet_size, char * buf, size_t pos, + std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet) override; RSA & public_key; RSA & private_key; From 6e0a25436893fc0d7d401b08baf4da2d2c6ca7df Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 21:50:06 +0200 Subject: [PATCH 071/327] Update tests, update imports, remove requirements --- .../Formats/Impl/MySQLOutputFormat.cpp | 3 +- tests/integration/requirements.txt | 30 ------------------- .../prepared_statements.reference | 4 +-- .../prepared_statements_test.sql | 12 +++++++- 4 files changed, 14 insertions(+), 35 deletions(-) delete mode 100644 tests/integration/requirements.txt diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index eaae8c6555c..2f056daf8e4 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -5,8 +6,6 @@ #include #include #include -#include -#include "Common/logger_useful.h" namespace DB { diff --git a/tests/integration/requirements.txt b/tests/integration/requirements.txt deleted file mode 100644 index 11efc43eded..00000000000 --- a/tests/integration/requirements.txt +++ /dev/null @@ -1,30 +0,0 @@ -PyMySQL -aerospike -avro -cassandra-driver -confluent-kafka -dicttoxml -docker -docker-compose -grpcio -grpcio-tools -kafka-python -kazoo -minio -lz4 -protobuf -psycopg2-binary -pymongo -pytz -pytest -pytest-timeout -redis -tzlocal==2.1 -urllib3 -requests-kerberos -dict2xml -hypothesis -pyhdfs -pika -meilisearch -nats-py diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference index 402e64348ae..6ea4b218f56 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements.reference +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -74,8 +74,8 @@ m type is CHAR, value: {'qux':144,'text':255} Row #1 d type is DATE, value: 1970-01-01 d32 type is DATE, value: 1900-01-01 -dt type is TIMESTAMP, value: 1970-01-01 01:00:00.0 -dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 +dt type is TIMESTAMP, value: 1970-01-01 00:00:00.0 +dt64_3 type is TIMESTAMP, value: 1900-01-01 00:00:00.001 dt64_6 type is TIMESTAMP, value: 1900-01-01 00:00:00.000001 dt64_9 type is TIMESTAMP, value: 1900-01-01 00:00:00.0 Row #2 diff --git a/tests/integration/test_mysql_protocol/prepared_statements_test.sql b/tests/integration/test_mysql_protocol/prepared_statements_test.sql index a32ce9db1d5..0302fe5aca7 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements_test.sql +++ b/tests/integration/test_mysql_protocol/prepared_statements_test.sql @@ -73,7 +73,7 @@ CREATE TABLE ps_date_types ( d Date, d32 Date32, - dt DateTime, + dt DateTime('UTC'), dt64_3 DateTime64(3, 'UTC'), dt64_6 DateTime64(6, 'UTC'), dt64_9 DateTime64(9, 'UTC') @@ -115,3 +115,13 @@ VALUES ('2022-04-13 03:17:45', '2022-04-13 03:17:45.00001', '2022-04-13 03:17:45.0000001', '2022-04-13 03:17:45.00000001'); + +CREATE TABLE ps_datetime_timezones +( + dt DateTime('Europe/Amsterdam'), + dt64_3 DateTime64(3, 'Asia/Shanghai') +) ENGINE MergeTree ORDER BY dt; + +INSERT INTO ps_datetime_timezones +VALUES ('2022-09-04 20:31:05', '2022-09-04 20:31:05.022'), + ('1970-01-01 00:00:00', '1969-12-31 16:00:00'); From 319a51c0b70579b73dff64b40580acd939a80ab0 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 22:05:51 +0200 Subject: [PATCH 072/327] Address style checks --- src/Core/MySQL/PacketsPreparedStatements.cpp | 6 ----- src/Core/MySQL/PacketsPreparedStatements.h | 5 +--- src/Core/MySQL/PacketsProtocolBinary.cpp | 1 - tests/integration/test_mysql_protocol/test.py | 23 +++++++++++-------- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/src/Core/MySQL/PacketsPreparedStatements.cpp b/src/Core/MySQL/PacketsPreparedStatements.cpp index eddded585d5..dcff294ebad 100644 --- a/src/Core/MySQL/PacketsPreparedStatements.cpp +++ b/src/Core/MySQL/PacketsPreparedStatements.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -25,11 +24,6 @@ namespace MySQLProtocol buffer.write(0x0); // RESULTSET_METADATA_NONE } - void PrepareStatementResponseOK::readPayloadImpl([[maybe_unused]] ReadBuffer & payload) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PrepareStatementResponseOK::readPayloadImpl is not implemented"); - } - PrepareStatementResponseOK::PrepareStatementResponseOK( uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_) : statement_id(statement_id_), num_columns(num_columns_), num_params(num_params_), warnings_count(warnings_count_) diff --git a/src/Core/MySQL/PacketsPreparedStatements.h b/src/Core/MySQL/PacketsPreparedStatements.h index 7f180d4d142..5aca459fc3a 100644 --- a/src/Core/MySQL/PacketsPreparedStatements.h +++ b/src/Core/MySQL/PacketsPreparedStatements.h @@ -1,6 +1,5 @@ #pragma once -#include #include namespace DB @@ -10,7 +9,7 @@ namespace MySQLProtocol namespace PreparedStatements { // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_prepare.html#sect_protocol_com_stmt_prepare_response_ok - class PrepareStatementResponseOK : public IMySQLWritePacket, public IMySQLReadPacket + class PrepareStatementResponseOK : public IMySQLWritePacket { public: uint8_t status = 0x00; @@ -23,8 +22,6 @@ namespace MySQLProtocol protected: size_t getPayloadSize() const override; - void readPayloadImpl(ReadBuffer & payload) override; - void writePayloadImpl(WriteBuffer & buffer) const override; public: diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index ca206870a76..9e5e2993595 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -306,7 +306,6 @@ namespace MySQLProtocol } String timezone = getDateTimeTimezone(*data_type); - std::cout << "Timezone is " << timezone << std::endl; LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone)); auto year = ldt.year(); auto month = ldt.month(); diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 19a52da1158..0f8c1300cdc 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -810,7 +810,12 @@ def test_prepared_statements(started_cluster, java_container): reference = fp.read() with open(os.path.join(SCRIPT_DIR, "prepared_statements_test.sql")) as sql: - statements = list(filter(lambda s: s != '', map(lambda s: s.strip().replace('\n', ' '), sql.read().split(';')))) + statements = list( + filter( + lambda s: s != "", + map(lambda s: s.strip().replace("\n", " "), sql.read().split(";")), + ) + ) for statement in statements: node.query(statement, settings={"password": "123"}) @@ -860,14 +865,14 @@ def test_types(started_cluster): result = cursor.fetchall()[0] expected = [ - ("Int8_column", -(2 ** 7)), - ("UInt8_column", 2 ** 8 - 1), - ("Int16_column", -(2 ** 15)), - ("UInt16_column", 2 ** 16 - 1), - ("Int32_column", -(2 ** 31)), - ("UInt32_column", 2 ** 32 - 1), - ("Int64_column", -(2 ** 63)), - ("UInt64_column", 2 ** 64 - 1), + ("Int8_column", -(2**7)), + ("UInt8_column", 2**8 - 1), + ("Int16_column", -(2**15)), + ("UInt16_column", 2**16 - 1), + ("Int32_column", -(2**31)), + ("UInt32_column", 2**32 - 1), + ("Int64_column", -(2**63)), + ("UInt64_column", 2**64 - 1), ("String_column", "тест"), ("FixedString_column", "тест"), ("Float32_column", 1.5), From 9fa4850630223d9c6dd02c5b1204efb3903d338f Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 22:36:29 +0200 Subject: [PATCH 073/327] Try to resolve style check --- src/Interpreters/InterpreterShowColumnsQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 922f9887a82..6c55ef0e6e9 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -37,7 +37,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() SELECT name AS field, type AS type, - if(startsWith(type, 'Nullable'), 'YES', 'NO') AS null, + if(startsWith(type, 'Nullable'), 'YES', 'NO') AS `null`, trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; From eddf914421a964bb3ebb590f75e08ca905a74178 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Mon, 4 Sep 2023 23:00:24 +0200 Subject: [PATCH 074/327] Clear prepared statements when it's too many --- src/Server/MySQLHandler.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 79b645eafd2..219e9c15ac2 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -389,6 +389,8 @@ void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) if (prepared_statements_map.size() > 10000) /// Shouldn't happen in reality as COM_STMT_CLOSE cleans up the elements { LOG_ERROR(log, "Too many prepared statements"); + current_prepared_statement_id = 0; + prepared_statements_map.clear(); packet_endpoint->sendPacket(ERRPacket(), true); return; } From 8aeccd078bc32050ba5c173a299a6568387c4f65 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 4 Sep 2023 21:16:25 +0000 Subject: [PATCH 075/327] Try to catch timeout --- tests/ci/fast_test_check.py | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 460e17acd37..57646dec4c5 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -32,6 +32,8 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo +from report import TestResult +from subprocess import TimeoutExpired NAME = "Fast test" @@ -151,12 +153,17 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") + expired_timeout = None + with TeePopen(run_cmd, run_log_path, timeout= 65) as process: + try: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + except TimeoutExpired as timeout_ex: + logging.info(f"Timeout expired for process execution: {run_cmd}") + expired_timeout = timeout_ex.timeout subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) @@ -188,6 +195,9 @@ def main(): else: state, description, test_results, additional_logs = process_results(output_path) + if (expired_timeout is not None): + test_results.append(TestResult(f"Timeout expired for process execution: {run_cmd}", "FAIL", expired_timeout)) + ch_helper = ClickHouseHelper() s3_path_prefix = os.path.join( get_release_or_pr(pr_info, get_version_from_repo())[0], From 0f5c49f6b5e74c98aa89ecacb1b176d719f1113e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 4 Sep 2023 21:23:58 +0000 Subject: [PATCH 076/327] Automatic style fix --- tests/ci/fast_test_check.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 57646dec4c5..522c0f9f290 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -154,7 +154,7 @@ def main(): run_log_path = os.path.join(logs_path, "run.log") expired_timeout = None - with TeePopen(run_cmd, run_log_path, timeout= 65) as process: + with TeePopen(run_cmd, run_log_path, timeout=65) as process: try: retcode = process.wait() if retcode == 0: @@ -195,8 +195,14 @@ def main(): else: state, description, test_results, additional_logs = process_results(output_path) - if (expired_timeout is not None): - test_results.append(TestResult(f"Timeout expired for process execution: {run_cmd}", "FAIL", expired_timeout)) + if expired_timeout is not None: + test_results.append( + TestResult( + f"Timeout expired for process execution: {run_cmd}", + "FAIL", + expired_timeout, + ) + ) ch_helper = ClickHouseHelper() s3_path_prefix = os.path.join( From 3159560c6cc955e2e4b03e82101a1d12186ea450 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 5 Sep 2023 18:24:53 +0200 Subject: [PATCH 077/327] Fix filesystem cache test A missing % in the query prevented it from producing any result at all. --- .../02241_filesystem_cache_on_write_operations.reference | 2 ++ .../0_stateless/02241_filesystem_cache_on_write_operations.sh | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference index c6099b96d31..157837983f7 100644 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference @@ -26,6 +26,7 @@ state: DOWNLOADED 24 84045 32 167243 41 250541 +INSERT INTO test_02241 SELECT number, toString(number) FROM numbers(5000000) 0 5010500 18816 Using storage policy: local_cache @@ -56,5 +57,6 @@ state: DOWNLOADED 24 84045 32 167243 41 250541 +INSERT INTO test_02241 SELECT number, toString(number) FROM numbers(5000000) 0 5010500 18816 diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh index 3a22ddfd7f5..96f61cf61e8 100755 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh @@ -113,7 +113,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do FROM system.query_log WHERE - query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%' + query LIKE '%SELECT number, toString(number) FROM numbers(5000000)%' AND type = 'QueryFinish' AND current_database = currentDatabase() ORDER BY From 17cb6a8e3f0fa8c8f3c65834e968282aef2a343c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 5 Sep 2023 18:34:33 +0000 Subject: [PATCH 078/327] SHOW COLUMNS: update docs, refresh expected results, add comment about bug --- docs/en/sql-reference/statements/show.md | 2 +- .../InterpreterShowColumnsQuery.cpp | 3 + .../0_stateless/02706_show_columns.reference | 68 +++++++++---------- 3 files changed, 38 insertions(+), 35 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 1c399d2072b..994c2bb24c7 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -208,7 +208,7 @@ The optional keyword `FULL` causes the output to include the collation, comment The statement produces a result table with the following structure: - field - The name of the column (String) - type - The column data type (String) -- null - If the column data type is Nullable (UInt8) +- null - `YES` if the column data type is Nullable, `NO` otherwise (UInt8) - key - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String) - default - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String)) - extra - Additional information, currently unused (String) diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 6c55ef0e6e9..ff1bab062d8 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -42,6 +42,9 @@ SELECT if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; + // Known issue: Field 'null' is wrong for types like 'LowCardinality(Nullable(String))'. Can't simply replace 'startsWith' by + // `hasSubsequence` as that would return `true` for non-nullable types such as `Tuple(Nullable(String), String)`... + // TODO Interpret query.extended. It is supposed to show internal/virtual columns. Need to fetch virtual column names, see // IStorage::getVirtuals(). We can't easily do that via SQL. diff --git a/tests/queries/0_stateless/02706_show_columns.reference b/tests/queries/0_stateless/02706_show_columns.reference index 9d7e8485e19..29e5329d63d 100644 --- a/tests/queries/0_stateless/02706_show_columns.reference +++ b/tests/queries/0_stateless/02706_show_columns.reference @@ -1,47 +1,47 @@ --- Aliases of SHOW COLUMNS -int32 Nullable(Int32) 1 \N -str String 0 SOR \N -uint64 UInt64 0 PRI SOR \N -int32 Nullable(Int32) 1 \N -str String 0 SOR \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +str String NO SOR \N +uint64 UInt64 NO PRI SOR \N +int32 Nullable(Int32) YES \N +str String NO SOR \N +uint64 UInt64 NO PRI SOR \N --- EXTENDED -int32 Nullable(Int32) 1 \N -str String 0 SOR \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +str String NO SOR \N +uint64 UInt64 NO PRI SOR \N --- FULL -int32 Nullable(Int32) 1 \N \N example comment -str String 0 SOR \N \N -uint64 UInt64 0 PRI SOR \N \N +int32 Nullable(Int32) YES \N \N example comment +str String NO SOR \N \N +uint64 UInt64 NO PRI SOR \N \N --- LIKE -int32 Nullable(Int32) 1 \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +uint64 UInt64 NO PRI SOR \N --- NOT LIKE -str String 0 SOR \N +str String NO SOR \N --- ILIKE -int32 Nullable(Int32) 1 \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +uint64 UInt64 NO PRI SOR \N --- NOT ILIKE -str String 0 SOR \N +str String NO SOR \N --- WHERE -int32 Nullable(Int32) 1 \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +uint64 UInt64 NO PRI SOR \N --- LIMIT -int32 Nullable(Int32) 1 \N +int32 Nullable(Int32) YES \N --- Check with weird table names -c String 0 PRI SOR \N -c String 0 PRI SOR \N -c String 0 PRI SOR \N -c String 0 PRI SOR \N +c String NO PRI SOR \N +c String NO PRI SOR \N +c String NO PRI SOR \N +c String NO PRI SOR \N --- Original table -int32 Nullable(Int32) 1 \N -str String 0 SOR \N -uint64 UInt64 0 PRI SOR \N +int32 Nullable(Int32) YES \N +str String NO SOR \N +uint64 UInt64 NO PRI SOR \N --- Equally named table in other database -int32 Int32 0 \N -str String 0 \N -uint64 UInt64 0 PRI SOR \N +int32 Int32 NO \N +str String NO \N +uint64 UInt64 NO PRI SOR \N --- Short form -int32 Int32 0 \N -str String 0 \N -uint64 UInt64 0 PRI SOR \N +int32 Int32 NO \N +str String NO \N +uint64 UInt64 NO PRI SOR \N From 767f2472c2cee3e788222c1ff14b20ec1b1f6845 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Fri, 1 Sep 2023 14:04:01 +0200 Subject: [PATCH 079/327] Add elapsed_ns to HTTP headers X-ClickHouse-Progress and X-ClickHouse-Summary The TCP interface progress has this field. This is not a super accurate measure of server side query time, but trying to measure from the client is even worse. --- docs/en/interfaces/http.md | 18 ++++---- docs/ru/interfaces/http.md | 16 +++---- docs/zh/interfaces/http.md | 18 ++++---- src/IO/Progress.cpp | 8 +++- src/IO/Progress.h | 2 + .../WriteBufferFromHTTPServerResponse.cpp | 2 + ...0416_pocopatch_progress_in_http_headers.sh | 6 +-- .../0_stateless/02136_scalar_progress.sh | 2 +- .../02373_progress_contain_result.sh | 2 +- .../02423_insert_summary_behaviour.sh | 12 +++--- .../02457_insert_select_progress_http.sh | 2 +- .../02869_http_headers_elapsed_ns.reference | 2 + .../02869_http_headers_elapsed_ns.sh | 43 +++++++++++++++++++ 13 files changed, 94 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02869_http_headers_elapsed_ns.reference create mode 100755 tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 37821f0fee1..f2091dd0900 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -56,7 +56,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334,"peak_memory_usage":"0"} 1 ``` @@ -286,9 +286,9 @@ Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"} ``` Possible header fields: @@ -416,7 +416,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -581,7 +581,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -621,7 +621,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -673,7 +673,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -692,7 +692,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index ca7fbdc219b..ae3f100a6e8 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -50,7 +50,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} 1 ``` @@ -267,9 +267,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"} ``` Возможные поля заголовка: @@ -530,7 +530,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -570,7 +570,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -622,7 +622,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -641,7 +641,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index f84768beccc..b1501cbb5d6 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -53,7 +53,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} 1 ``` @@ -262,9 +262,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 您可以在`X-ClickHouse-Progress`响应头中收到查询进度的信息。为此,启用[Http Header携带进度](../operations/settings/settings.md#settings-send_progress_in_http_headers)。示例: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","elapsed_ns":"662334","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","elapsed_ns":"992334","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","elapsed_ns":"1232334","peak_memory_usage":"23155600"} ``` 显示字段信息: @@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -521,7 +521,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -561,7 +561,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -613,7 +613,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -632,7 +632,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index 620d2f0f762..56fb7fe317e 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -90,7 +90,8 @@ void ProgressValues::writeJSON(WriteBuffer & out, bool add_braces) const writeText(result_rows, out); writeCString("\",\"result_bytes\":\"", out); writeText(result_bytes, out); - writeCString("\"", out); + writeCString("\",\"elapsed_ns\":\"", out); + writeText(elapsed_ns, out); if (add_braces) writeCString("}", out); } @@ -239,4 +240,9 @@ void Progress::writeJSON(WriteBuffer & out, bool add_braces) const getValues().writeJSON(out, add_braces); } +void Progress::incrementElapsedNs(UInt64 elapsed_ns_) +{ + elapsed_ns.fetch_add(elapsed_ns_, std::memory_order_relaxed); +} + } diff --git a/src/IO/Progress.h b/src/IO/Progress.h index 936ed5ea342..288598cae78 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -124,6 +124,8 @@ struct Progress /// Each value separately is changed atomically (but not whole object). bool incrementPiecewiseAtomically(const Progress & rhs); + void incrementElapsedNs(UInt64 elapsed_ns_); + void reset(); ProgressValues getValues() const; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 9da24cfd9cb..62f492fa0a9 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -48,6 +48,7 @@ void WriteBufferFromHTTPServerResponse::writeHeaderProgressImpl(const char * hea void WriteBufferFromHTTPServerResponse::writeHeaderSummary() { + accumulated_progress.incrementElapsedNs(progress_watch.elapsed()); writeHeaderProgressImpl("X-ClickHouse-Summary: "); } @@ -164,6 +165,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress, In peak_memory_usage = peak_memory_usage_; if (send_progress && progress_watch.elapsed() >= send_progress_interval_ms * 1000000) { + accumulated_progress.incrementElapsedNs(progress_watch.elapsed()); progress_watch.restart(); /// Send all common headers before our special progress headers. diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index e77aa1bf250..6ec88fae351 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -10,7 +10,7 @@ result="" lines_expected=4 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | sed 's/,\"peak_mem[^}]*//') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | sed 's/,\"elapsed_ns[^}]*//') let counter=counter+1 done echo "$result" @@ -19,7 +19,7 @@ result="" lines_expected=12 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'| sed 's/,\"peak_mem[^}]*//') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'| sed 's/,\"elapsed_ns[^}]*//') let counter=counter+1 done echo "$result" @@ -46,7 +46,7 @@ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}" -H 'Accept-Encoding: gzip' -d 'CREAT result="" counter=0 while [ $counter -lt $RETRIES ] && [ -z "$result" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&http_headers_progress_interval_ms=0&send_progress_in_http_headers=1" -d 'INSERT INTO insert_number_query (record) SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Summary|^[0-9]' | sed 's/,\"peak_mem[^}]*//') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&http_headers_progress_interval_ms=0&send_progress_in_http_headers=1" -d 'INSERT INTO insert_number_query (record) SELECT number FROM system.numbers LIMIT 10' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Summary|^[0-9]' | sed 's/,\"elapsed_ns[^}]*//') let counter=counter+1 done echo "$result" diff --git a/tests/queries/0_stateless/02136_scalar_progress.sh b/tests/queries/0_stateless/02136_scalar_progress.sh index 517fe19ecd4..f2016b04f82 100755 --- a/tests/queries/0_stateless/02136_scalar_progress.sh +++ b/tests/queries/0_stateless/02136_scalar_progress.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" | sed 's/,\"peak_mem[^}]*//' +$CLICKHOUSE_CURL -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d "SELECT (SELECT max(number), count(number) FROM numbers(100000) settings max_block_size=65505);" -v 2>&1 | grep -E "X-ClickHouse-Summary|X-ClickHouse-Progress" | sed 's/,\"elapsed_ns[^}]*//' diff --git a/tests/queries/0_stateless/02373_progress_contain_result.sh b/tests/queries/0_stateless/02373_progress_contain_result.sh index c84af0ee269..c87a5ec7615 100755 --- a/tests/queries/0_stateless/02373_progress_contain_result.sh +++ b/tests/queries/0_stateless/02373_progress_contain_result.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo 'SELECT 1 FROM numbers(100)' | ${CLICKHOUSE_CURL_COMMAND} -v "${CLICKHOUSE_URL}&wait_end_of_query=1&send_progress_in_http_headers=0" --data-binary @- 2>&1 | - grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' + grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' diff --git a/tests/queries/0_stateless/02423_insert_summary_behaviour.sh b/tests/queries/0_stateless/02423_insert_summary_behaviour.sh index a0b3c519806..b184d9ccf47 100755 --- a/tests/queries/0_stateless/02423_insert_summary_behaviour.sh +++ b/tests/queries/0_stateless/02423_insert_summary_behaviour.sh @@ -11,11 +11,11 @@ $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target TO target_1 AS $CLICKHOUSE_CLIENT -q "CREATE MATERIALIZED VIEW floats_to_target_2 TO target_2 AS SELECT * FROM floats, numbers(2) n" echo "No materialized views" -${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+target_1+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' echo "With materialized views" -${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' -$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"peak_mem[^}]*//' +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats" -d "VALUES(1.0)" -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format Native | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+Native" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' +$CLICKHOUSE_LOCAL -q "SELECT number::Float64 AS v FROM numbers(10)" --format RowBinary | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query=INSERT+INTO+floats+FORMAT+RowBinary" --data-binary @- -v 2>&1 | grep 'X-ClickHouse-Summary' | sed 's/,\"elapsed_ns[^}]*//' diff --git a/tests/queries/0_stateless/02457_insert_select_progress_http.sh b/tests/queries/0_stateless/02457_insert_select_progress_http.sh index 7f7fc67ae00..ae62ee4b77e 100755 --- a/tests/queries/0_stateless/02457_insert_select_progress_http.sh +++ b/tests/queries/0_stateless/02457_insert_select_progress_http.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "insert into function null('_ Int') select * from numbers(5) settings max_block_size=1" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | sed 's/,\"peak_mem[^}]*//' + grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | sed 's/,\"elapsed_ns[^}]*//' } diff --git a/tests/queries/0_stateless/02869_http_headers_elapsed_ns.reference b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.reference new file mode 100644 index 00000000000..f89715e69fd --- /dev/null +++ b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.reference @@ -0,0 +1,2 @@ +elapsed_ns in progress are all non zero +elapsed_ns in summary is not zero diff --git a/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh new file mode 100755 index 00000000000..107d8a10345 --- /dev/null +++ b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +CURL_OUTPUT=$(echo 'SELECT number FROM numbers(10)' | \ + ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) + +ELAPSED_NS_PROGRESS=$(echo "${CURL_OUTPUT}" | \ + grep 'X-ClickHouse-Progress' | \ + awk '{print $3}' | \ + sed -E 's/.*"elapsed_ns":"?([^,"]*)"?.*/\1/' + ) + +ELAPSED_NS_SUMMARY=$(echo "${CURL_OUTPUT}" | \ + grep 'X-ClickHouse-Summary' | \ + awk '{print $3}' | \ + sed -E 's/.*"elapsed_ns":"?([^,"]*)"?.*/\1/' + ) + + +ALL_ARE_NON_ZERO=1 +while read -r line; do + if [ "$line" -eq 0 ]; then + ALL_ARE_NON_ZERO=0 + break + fi +done <<< "$ELAPSED_NS_PROGRESS" + +if [ "$ALL_ARE_NON_ZERO" -eq 1 ] && [ "$(echo "$ELAPSED_NS_SUMMARY" | wc -l)" -gt 0 ]; then + echo "elapsed_ns in progress are all non zero" +else + echo "elapsed_ns in progress are all zero!" +fi + +if [ "$ELAPSED_NS_SUMMARY" -ne 0 ]; +then + echo "elapsed_ns in summary is not zero" +else + echo "elapsed_ns in summary is zero!" +fi From 91f637b24a0b713561039fff8b980c43e344f441 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Mon, 4 Sep 2023 17:56:14 +0200 Subject: [PATCH 080/327] fix python test --- .../test_postgresql_replica_database_engine_1/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 57ee920d49c..c118080a572 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -414,6 +414,7 @@ def test_many_concurrent_queries(started_cluster): port=started_cluster.postgres_port, database=True, ) + cursor = conn.cursor() pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000) def attack(thread_id): From 3ef6d0da39488cd28a9014e6c37c3740e18b216e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 11:02:17 +0000 Subject: [PATCH 081/327] Set status and description --- tests/ci/fast_test_check.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 522c0f9f290..38e2eca3420 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -26,13 +26,12 @@ from docker_pull_helper import get_image_with_version from env_helper import S3_BUILDS_BUCKET, TEMP_PATH from get_robot_token import get_best_robot_token from pr_info import FORCE_TESTS_LABEL, PRInfo -from report import TestResults, read_test_results +from report import TestResult, TestResults, read_test_results from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo -from report import TestResult from subprocess import TimeoutExpired NAME = "Fast test" @@ -203,6 +202,8 @@ def main(): expired_timeout, ) ) + state = "failure" + description = test_results[-1].name ch_helper = ClickHouseHelper() s3_path_prefix = os.path.join( From a22e219be96ec2277278e6565447a3ad0cd62db9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 11:51:43 +0000 Subject: [PATCH 082/327] Fix --- tests/ci/fast_test_check.py | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 38e2eca3420..ce10c9dce51 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -152,17 +152,17 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - expired_timeout = None - with TeePopen(run_cmd, run_log_path, timeout=65) as process: - try: + timeout = 65 + timeout_expired = False + with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() - if retcode == 0: + if process.timeout_exceeded: + logging.info(f"Timeout expired for process execution: {run_cmd}") + timeout_expired = True + elif retcode == 0: logging.info("Run successfully") else: logging.info("Run failed") - except TimeoutExpired as timeout_ex: - logging.info(f"Timeout expired for process execution: {run_cmd}") - expired_timeout = timeout_ex.timeout subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) @@ -194,15 +194,15 @@ def main(): else: state, description, test_results, additional_logs = process_results(output_path) - if expired_timeout is not None: + if timeout_expired is not None: test_results.append( TestResult( f"Timeout expired for process execution: {run_cmd}", "FAIL", - expired_timeout, + timeout, ) ) - state = "failure" + state = "timeout" description = test_results[-1].name ch_helper = ClickHouseHelper() From f08d445a0fc083b088e6e830c4f7ca4036503fe7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Sep 2023 11:59:51 +0000 Subject: [PATCH 083/327] Automatic style fix --- tests/ci/fast_test_check.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index ce10c9dce51..4af1ca0812a 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -155,14 +155,14 @@ def main(): timeout = 65 timeout_expired = False with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: - retcode = process.wait() - if process.timeout_exceeded: - logging.info(f"Timeout expired for process execution: {run_cmd}") - timeout_expired = True - elif retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") + retcode = process.wait() + if process.timeout_exceeded: + logging.info(f"Timeout expired for process execution: {run_cmd}") + timeout_expired = True + elif retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) From f6f2e731a7ee557c0c718352797346544cc7ae83 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 12:44:24 +0000 Subject: [PATCH 084/327] Fix state --- tests/ci/fast_test_check.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 4af1ca0812a..afb3a93a617 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -32,7 +32,6 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo -from subprocess import TimeoutExpired NAME = "Fast test" @@ -202,7 +201,7 @@ def main(): timeout, ) ) - state = "timeout" + state = "failure" description = test_results[-1].name ch_helper = ClickHouseHelper() From e858f0b880688c9115c38a1a04078cbe5b9fbd2e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 13:05:56 +0000 Subject: [PATCH 085/327] Fix description length --- tests/ci/fast_test_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index bead2f2f61f..e1548bf801e 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -185,6 +185,11 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) if timeout_expired is not None: + name = f"Timeout expired for process execution: {run_cmd}" + if len(name) > 140: + name = name[:136] + name += " ..." + test_results.append( TestResult( f"Timeout expired for process execution: {run_cmd}", From babbecc646acf48b4176ef07013728100948a494 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 6 Sep 2023 15:00:32 +0000 Subject: [PATCH 086/327] Fix --- tests/ci/fast_test_check.py | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index e1548bf801e..97e4f2a8713 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -185,18 +185,13 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) if timeout_expired is not None: - name = f"Timeout expired for process execution: {run_cmd}" + name = f"Timeout expired for process execution" if len(name) > 140: name = name[:136] name += " ..." - test_results.append( - TestResult( - f"Timeout expired for process execution: {run_cmd}", - "FAIL", - timeout, - ) - ) + test_results.append(TestResult(name, "FAIL", timeout)) + state = "failure" description = test_results[-1].name From 4b5784f7d021ca03b1759de944d777f780a6538d Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Sep 2023 18:05:20 +0200 Subject: [PATCH 087/327] Fix test --- .../02845_group_by_constant_keys.sql | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.sql b/tests/queries/0_stateless/02845_group_by_constant_keys.sql index e16fd3fd1d2..053ad3ecdc4 100644 --- a/tests/queries/0_stateless/02845_group_by_constant_keys.sql +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.sql @@ -10,20 +10,19 @@ insert into test select * from numbers(10); insert into test select * from numbers(10); insert into test select * from numbers(10); -select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; -select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; -select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; -select count() from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; +select count() from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count() from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count() from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count() from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; -select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; -select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; -select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; -select count(), _file from test group by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; +select count(), _file from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count(), _file from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count(), _file from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count(), _file from test group by _file order by _file settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; -select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; -select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; -select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; -select count() from test group by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; +select count() from test group by _file, _path order by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=0; +select count() from test group by _file, _path order by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=0; +select count() from test group by _file, _path order by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=0, compile_aggregate_expressions=1; +select count() from test group by _file, _path order by _file, _path settings optimize_group_by_constant_keys=1, enable_software_prefetch_in_aggregation=1, compile_aggregate_expressions=1; drop table test; - From b10639594abf7b3bf28a27b8974e221b3700d256 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Sep 2023 18:05:43 +0200 Subject: [PATCH 088/327] Update test reference --- .../02845_group_by_constant_keys.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02845_group_by_constant_keys.reference b/tests/queries/0_stateless/02845_group_by_constant_keys.reference index 67cbdf0c025..5473729c960 100644 --- a/tests/queries/0_stateless/02845_group_by_constant_keys.reference +++ b/tests/queries/0_stateless/02845_group_by_constant_keys.reference @@ -14,18 +14,18 @@ 10 10 10 -10 data.Parquet -10 data.2.Parquet -10 data.1.Parquet -10 data.Parquet -10 data.2.Parquet -10 data.1.Parquet -10 data.Parquet -10 data.2.Parquet -10 data.1.Parquet -10 data.Parquet -10 data.2.Parquet -10 data.1.Parquet +10 data.1.JSON +10 data.2.JSON +10 data.JSON +10 data.1.JSON +10 data.2.JSON +10 data.JSON +10 data.1.JSON +10 data.2.JSON +10 data.JSON +10 data.1.JSON +10 data.2.JSON +10 data.JSON 10 10 10 From 692f4d1f6718c52030a9e63511147580e945d24a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Sep 2023 18:09:56 +0200 Subject: [PATCH 089/327] Fix test --- .../0_stateless/02842_capn_proto_outfile_without_schema.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh index fde08484026..ae57cdd7543 100755 --- a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh +++ b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From ce673d6beab8203b95b50feea8862038ab2bee51 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 6 Sep 2023 13:59:39 -0300 Subject: [PATCH 090/327] arrow-parquet account for monotonically increasing offsets across multiple batches --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 76f39b07a05..d6ae39b1542 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -447,10 +447,25 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr 0. + * If they are not monotonically increasing, it'll always be 0. + * Therefore, we subtract the previous offset from the current offset to get the corresponding CH "offset". + * + * This workaround should not be confused with the one above. That workaround deals with multiple chunks and internal representation + * of offsets, not batches. + * */ + auto previous_offset = arrow_offsets.Value(0); + for (int64_t i = 1; i < arrow_offsets.length(); ++i) { auto offset = arrow_offsets.Value(i); - offsets_data.emplace_back(start_offset + offset); + offsets_data.emplace_back(start_offset + offset - previous_offset); } } return offsets_column; From ab101eaff8ff36aa31c50bf3c20e2e4e23fe571a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 6 Sep 2023 18:21:24 +0000 Subject: [PATCH 091/327] Fix possible error 'URI contains invalid characters' in s3 table function --- src/Storages/StorageS3.cpp | 8 ++++++-- src/TableFunctions/TableFunctionS3.cpp | 9 +++++---- ...esigned_url_and_url_with_special_characters.reference | 0 ..._s3_presigned_url_and_url_with_special_characters.sql | 4 ++++ 4 files changed, 15 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.reference create mode 100644 tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 24b2279bfdc..5078d21141b 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1246,11 +1246,13 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); + String url; if (!filename.empty()) - configuration.url = S3::URI(std::filesystem::path(collection.get("url")) / filename); + url = std::filesystem::path(collection.get("url")) / filename; else - configuration.url = S3::URI(collection.get("url")); + url = collection.get("url"); + configuration.url = S3::URI(url); configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); @@ -1258,6 +1260,8 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur configuration.auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); configuration.format = collection.getOrDefault("format", configuration.format); + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(url).getPath(), true); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); configuration.structure = collection.getOrDefault("structure", "auto"); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 0ae3a19dc7f..94ce29f1116 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -152,7 +152,8 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context } /// This argument is always the first - configuration.url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + String url = checkAndGetLiteralArgument(args[0], "url"); + configuration.url = S3::URI(url); if (args_to_idx.contains("format")) { @@ -176,12 +177,12 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); configuration.auth_settings.no_sign_request = no_sign_request; + + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(url).getPath(), true); } configuration.keys = {configuration.url.key}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(configuration.url.uri.getPath()).getPath(), true); } void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.reference b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql new file mode 100644 index 00000000000..e6954fd9cbf --- /dev/null +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -0,0 +1,4 @@ +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } + +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } + From 664efaf3fa3bcd7029dcc71558e5bdbc06fcff7f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 6 Sep 2023 18:35:40 +0000 Subject: [PATCH 092/327] add test --- tests/integration/test_scheduler/__init__.py | 0 .../test_scheduler/configs/scheduler.xml | 62 ++++++++++++ tests/integration/test_scheduler/test.py | 97 +++++++++++++++++++ 3 files changed, 159 insertions(+) create mode 100644 tests/integration/test_scheduler/__init__.py create mode 100644 tests/integration/test_scheduler/configs/scheduler.xml create mode 100644 tests/integration/test_scheduler/test.py diff --git a/tests/integration/test_scheduler/__init__.py b/tests/integration/test_scheduler/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_scheduler/configs/scheduler.xml b/tests/integration/test_scheduler/configs/scheduler.xml new file mode 100644 index 00000000000..fe01cb8d662 --- /dev/null +++ b/tests/integration/test_scheduler/configs/scheduler.xml @@ -0,0 +1,62 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + network_read + network_write + + + + + +
+ s3 +
+
+
+
+
+ + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + + + + + /prio/admin + /prio/admin + + + /prio/fair/prod + /prio/fair/prod + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/dev + /prio/fair/dev + + +
diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py new file mode 100644 index 00000000000..7900e5b649e --- /dev/null +++ b/tests/integration/test_scheduler/test.py @@ -0,0 +1,97 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +import time +import threading +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + stay_alive=True, + main_configs=["configs/scheduler.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield + finally: + cluster.shutdown() + + +def test_s3_disk(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + def write_query(workload): + try: + node.query(f"insert into data select * from numbers(1e5) settings workload='{workload}'") + except QueryRuntimeException: + pass + thread1 = threading.Thread(target=write_query, args=["development"]) + thread2 = threading.Thread(target=write_query, args=["production"]) + thread3 = threading.Thread(target=write_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/admin'") + == "1\n" + ) + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/dev'") + == "1\n" + ) + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/prod'") + == "1\n" + ) + + def read_query(workload): + try: + node.query(f"select sum(key*key) from data settings workload='{workload}'") + except QueryRuntimeException: + pass + thread1 = threading.Thread(target=read_query, args=["development"]) + thread2 = threading.Thread(target=read_query, args=["production"]) + thread3 = threading.Thread(target=read_query, args=["admin"]) + + thread1.start() + thread2.start() + thread3.start() + + thread3.join() + thread2.join() + thread1.join() + + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/admin'") + == "1\n" + ) + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/dev'") + == "1\n" + ) + assert ( + node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/prod'") + == "1\n" + ) + From cd7c57f5d47ab46f3120faf115554deb9d486af8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Sep 2023 19:00:05 +0000 Subject: [PATCH 093/327] Automatic style fix --- tests/integration/test_scheduler/test.py | 31 ++++++++++++++++++------ 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 7900e5b649e..e6def99c076 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -37,9 +37,12 @@ def test_s3_disk(): def write_query(workload): try: - node.query(f"insert into data select * from numbers(1e5) settings workload='{workload}'") + node.query( + f"insert into data select * from numbers(1e5) settings workload='{workload}'" + ) except QueryRuntimeException: pass + thread1 = threading.Thread(target=write_query, args=["development"]) thread2 = threading.Thread(target=write_query, args=["production"]) thread3 = threading.Thread(target=write_query, args=["admin"]) @@ -53,15 +56,21 @@ def test_s3_disk(): thread1.join() assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/admin'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/admin'" + ) == "1\n" ) assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/dev'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/dev'" + ) == "1\n" ) assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/prod'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_write' and path='/prio/fair/prod'" + ) == "1\n" ) @@ -70,6 +79,7 @@ def test_s3_disk(): node.query(f"select sum(key*key) from data settings workload='{workload}'") except QueryRuntimeException: pass + thread1 = threading.Thread(target=read_query, args=["development"]) thread2 = threading.Thread(target=read_query, args=["production"]) thread3 = threading.Thread(target=read_query, args=["admin"]) @@ -83,15 +93,20 @@ def test_s3_disk(): thread1.join() assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/admin'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/admin'" + ) == "1\n" ) assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/dev'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/dev'" + ) == "1\n" ) assert ( - node.query(f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/prod'") + node.query( + f"select dequeued_requests>0 from system.scheduler where resource='network_read' and path='/prio/fair/prod'" + ) == "1\n" ) - From e11e5fbe2e8e50b298a05f6c8c8c8d2e9b0ffbef Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Sep 2023 21:10:30 +0200 Subject: [PATCH 094/327] Fix fasttest --- .../02873_s3_presigned_url_and_url_with_special_characters.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql index e6954fd9cbf..da76a5cb88f 100644 --- a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } From 77697211cd2cf565fd248db9842360bde40d5e7f Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 6 Sep 2023 19:30:39 +0000 Subject: [PATCH 095/327] add docs for `system.scheduler` + small fix --- docs/en/operations/system-tables/scheduler.md | 64 +++++++++++++++++++ docs/en/operations/workload-scheduling.md | 10 ++- src/IO/ISchedulerNode.h | 2 +- .../System/StorageSystemScheduler.cpp | 2 +- 4 files changed, 73 insertions(+), 5 deletions(-) create mode 100644 docs/en/operations/system-tables/scheduler.md diff --git a/docs/en/operations/system-tables/scheduler.md b/docs/en/operations/system-tables/scheduler.md new file mode 100644 index 00000000000..168bfe708c4 --- /dev/null +++ b/docs/en/operations/system-tables/scheduler.md @@ -0,0 +1,64 @@ +--- +slug: /en/operations/system-tables/scheduler +--- +# scheduler + +Contains information and status for [scheduling nodes](/docs/en/operations/workload-scheduling.md/#hierarchy) residing on the local server. +This table can be used for monitoring. The table contains a row for every scheduling node. + +Example: + +``` sql +SELECT * +FROM system.scheduler +WHERE resource = 'network_read' AND path = '/prio/fair/prod' +FORMAT Vertical +``` + +``` text +Row 1: +────── +resource: network_read +path: /prio/fair/prod +type: fifo +weight: 5 +priority: 0 +is_active: 0 +active_children: 0 +dequeued_requests: 67 +dequeued_cost: 4692272 +busy_periods: 63 +vruntime: 938454.1999999989 +system_vruntime: ᴺᵁᴸᴸ +queue_length: 0 +queue_cost: 0 +budget: -60524 +is_satisfied: ᴺᵁᴸᴸ +inflight_requests: ᴺᵁᴸᴸ +inflight_cost: ᴺᵁᴸᴸ +max_requests: ᴺᵁᴸᴸ +max_cost: ᴺᵁᴸᴸ +``` + +Columns: + +- `resource` (`String`) - Resource name +- `path` (`String`) - Path to a scheduling node within this resource scheduling hierarchy +- `type` (`String`) - Type of a scheduling node. +- `weight` (`Float64`) - Weight of a node, used by a parent node of `fair`` type. +- `priority` (`Int64`) - Priority of a node, used by a parent node of 'priority' type (Lower value means higher priority). +- `is_active` (`UInt8`) - Whether this node is currently active - has resource requests to be dequeued and constraints satisfied. +- `active_children` (`UInt64`) - The number of children in active state. +- `dequeued_requests` (`UInt64`) - The total number of resource requests dequeued from this node. +- `dequeued_cost` (`UInt64`) - The sum of costs (e.g. size in bytes) of all requests dequeued from this node. +- `busy_periods` (`UInt64`) - The total number of deactivations of this node. +- `vruntime` (`Nullable(Float64)`) - For children of `fair` nodes only. Virtual runtime of a node used by SFQ algorithm to select the next child to process in a max-min fair manner. +- `system_vruntime` (`Nullable(Float64)`) - For `fair` nodes only. Virtual runtime showing `vruntime` of the last processed resource request. Used during child activation as the new value of `vruntime`. +- `queue_length` (`Nullable(UInt64)`) - For `fifo` nodes only. Current number of resource requests residing in the queue. +- `queue_cost` (`Nullable(UInt64)`) - For `fifo` nodes only. Sum of costs (e.g. size in bytes) of all requests residing in the queue. +- `budget` (`Nullable(Int64)`) - For `fifo` nodes only. The number of available "cost units" for new resource requests. Can appear in case of discrepancy of estimated and real costs of resource requests (e.g. after read/write failure) +- `is_satisfied` (`Nullable(UInt8)`) - For constraint nodes only (e.g. `inflight_limit`). Equals `1` if all the constraint of this node are satisfied. +- `inflight_requests` (`Nullable(Int64)`) - For `inflight_limit` nodes only. The number of resource requests dequeued from this node, that are currently in consumption state. +- `inflight_cost` (`Nullable(Int64)`) - For `inflight_limit` nodes only. The sum of costs (e.g. bytes) of all resource requests dequeued from this node, that are currently in consumption state. +- `max_requests` (`Nullable(Int64)`) - For `inflight_limit` nodes only. Upper limit for `inflight_requests` leading to constraint violation. +- `max_cost` (`Nullable(Int64)`) - For `inflight_limit` nodes only. Upper limit for `inflight_cost` leading to constraint violation. diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 1761f99b469..c6529d5b81f 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -11,7 +11,7 @@ When ClickHouse execute multiple queries simultaneously, they may be using share Currently only remote disk IO can be scheduled using described method. For CPU scheduling see settings about thread pools and [`concurrent_threads_soft_limit_num`](server-configuration-parameters/settings.md#concurrent_threads_soft_limit_num). For flexible memory limits see [Memory overcommit](settings/memory-overcommit.md) ::: -## Disk configuration +## Disk configuration {#disk-config} To enable IO scheduling for a specific disk, you have to specify `read_resource` and/or `write_resource` in storage configuration. It says ClickHouse what resource should be used for every read and write requests with given disk. Read and write resource can refer to the same resource name, which is useful for local SSDs or HDDs. Multiple different disks also can refer to the same resource, which is useful for remote disks: if you want to be able to allow fair division of network bandwidth between e.g. "production" and "development" workloads. @@ -54,7 +54,7 @@ SELECT count() FROM my_table WHERE value = 42 SETTINGS workload = 'production' SELECT count() FROM my_table WHERE value = 13 SETTINGS workload = 'development' ``` -## Resource scheduling hierarchy +## Resource scheduling hierarchy {#hierarchy} From the standpoint of scheduling subsystem a resource represents a hierarchy of scheduling nodes. @@ -124,7 +124,7 @@ The following example shows how to define IO scheduling hierarchies shown in the ``` -## Workload classifiers +## Workload classifiers {#classifiers} Workload classifiers are used to define mapping from `workload` specified by a query into leaf-queues that should be used for specific resources. At the moment, workload classification is simple: only static mapping is available. @@ -147,3 +147,7 @@ Example: ``` + + +## See also + - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) diff --git a/src/IO/ISchedulerNode.h b/src/IO/ISchedulerNode.h index f7fc9257128..644cc708bca 100644 --- a/src/IO/ISchedulerNode.h +++ b/src/IO/ISchedulerNode.h @@ -232,7 +232,7 @@ public: /// Introspection std::atomic dequeued_requests{0}; - std::atomic dequeued_cost{0}; + std::atomic dequeued_cost{0}; std::atomic busy_periods{0}; }; diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index ca0a138559e..9f4307fca3a 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -28,7 +28,7 @@ NamesAndTypesList StorageSystemScheduler::getNamesAndTypes() {"is_active", std::make_shared()}, {"active_children", std::make_shared()}, {"dequeued_requests", std::make_shared()}, - {"dequeued_cost", std::make_shared()}, + {"dequeued_cost", std::make_shared()}, {"busy_periods", std::make_shared()}, {"vruntime", std::make_shared(std::make_shared())}, {"system_vruntime", std::make_shared(std::make_shared())}, From a8027be612986b060b8bd285ac503c9d2c1e08d6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 6 Sep 2023 17:25:27 -0300 Subject: [PATCH 096/327] fix --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 24 +++++-------------- 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index d6ae39b1542..fc522cd7569 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -427,26 +427,12 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr &>(*offsets_column).getData(); offsets_data.reserve(arrow_column->length()); - uint64_t start_offset = 0u; - for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); auto arrow_offsets_array = list_chunk.offsets(); auto & arrow_offsets = dynamic_cast(*arrow_offsets_array); - /* - * It seems like arrow::ListArray::values() (nested column data) might or might not be shared across chunks. - * When it is shared, the offsets will be monotonically increasing. Otherwise, the offsets will be zero based. - * In order to account for both cases, the starting offset is updated whenever a zero-based offset is found. - * More info can be found in: https://lists.apache.org/thread/rrwfb9zo2dc58dhd9rblf20xd7wmy7jm and - * https://github.com/ClickHouse/ClickHouse/pull/43297 - * */ - if (list_chunk.offset() == 0) - { - start_offset = offsets_data.back(); - } - /* * CH uses element size as "offsets", while arrow uses actual offsets as offsets. * That's why CH usually starts reading offsets with i=1 and i=0 is ignored. @@ -457,15 +443,17 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr Date: Wed, 6 Sep 2023 17:26:34 -0300 Subject: [PATCH 097/327] docs --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index fc522cd7569..2a9892b7219 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -444,7 +444,8 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr Date: Wed, 6 Sep 2023 22:49:28 +0000 Subject: [PATCH 098/327] Polishing --- tests/ci/fast_test_check.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 97e4f2a8713..96eb144da2c 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -21,6 +21,7 @@ from commit_status_helper import ( get_commit, post_commit_status, update_mergeable_check, + format_description, ) from docker_pull_helper import get_image_with_version from env_helper import S3_BUILDS_BUCKET, TEMP_PATH @@ -185,15 +186,10 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) if timeout_expired is not None: - name = f"Timeout expired for process execution" - if len(name) > 140: - name = name[:136] - name += " ..." - - test_results.append(TestResult(name, "FAIL", timeout)) - + test_result_name = "Timeout for fast test check is expired" + test_results.append(TestResult(test_result_name, "FAIL", timeout)) state = "failure" - description = test_results[-1].name + description = format_description(test_result_name) ch_helper = ClickHouseHelper() s3_path_prefix = os.path.join( From ef2350cfc7e8b6727cbe7e08f3787ee0669bf736 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 6 Sep 2023 20:36:39 -0400 Subject: [PATCH 099/327] Implementing `exists`, `sync` commands. Better tests --- .../utilities/clickhouse-keeper-client.md | 5 + programs/keeper-client/Commands.cpp | 30 ++++ programs/keeper-client/Commands.h | 27 ++- programs/keeper-client/KeeperClient.cpp | 37 +++- programs/keeper-client/KeeperClient.h | 3 + tests/integration/helpers/keeper_utils.py | 164 +++++++++++++++++- tests/integration/test_keeper_client/test.py | 147 +++++++--------- .../test_keeper_reconfig_add/test.py | 68 ++++---- .../test_keeper_reconfig_remove/test.py | 77 ++++---- .../test_keeper_reconfig_remove_many/test.py | 71 +++----- .../test.py | 51 +++--- .../__init__.py | 0 .../configs/keeper1.xml | 35 ---- .../configs/keeper2.xml | 35 ---- .../configs/keeper3.xml | 35 ---- .../configs/keeper4.xml | 21 --- .../test.py | 43 ----- 17 files changed, 430 insertions(+), 419 deletions(-) delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml delete mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 2140d22b620..9f24ba9587b 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -16,6 +16,8 @@ A client application to interact with clickhouse-keeper by its native protocol. - `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s. - `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s. - `--history-file=FILE_PATH` — Set path of history file. Default value: `~/.keeper-client-history`. +- `--log-level=LEVEL` — Set log level. Default value: `information`. +- `--no-confirmation` — If set, will not require a confirmation on several commands. Default value `false` for interactive and `true` for query - `--help` — Shows the help message. ## Example {#clickhouse-keeper-client-example} @@ -44,6 +46,7 @@ keeper foo bar - `ls [path]` -- Lists the nodes for the given path (default: cwd) - `cd [path]` -- Change the working path (default `.`) +- `exists ` -- Returns `1` if node exists, `0` otherwise - `set [version]` -- Updates the node's value. Only update if version matches (default: -1) - `create [mode]` -- Creates new node with the set value - `touch ` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists @@ -56,3 +59,5 @@ keeper foo bar - `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive - `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) +- `sync ` -- Synchronizes node between processes and leader +- `reconfig "" [version]` -- Reconfigures Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index e265d7f4858..65bc719f1be 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -158,6 +158,21 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; } +bool ExistsCommand::parse(IParser::Pos & pos, std::shared_ptr & node, DB::Expected & expected) const +{ + String path; + if (!parseKeeperPath(pos, expected, path)) + return false; + node->args.push_back(std::move(path)); + + return true; +} + +void ExistsCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient * client) const +{ + std::cout << client->zookeeper->exists(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; +} + bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String path; @@ -400,6 +415,21 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient std::cout << response.value << '\n'; } +bool SyncCommand::parse(IParser::Pos & pos, std::shared_ptr & node, DB::Expected & expected) const +{ + String path; + if (!parseKeeperPath(pos, expected, path)) + return false; + node->args.push_back(std::move(path)); + + return true; +} + +void SyncCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient * client) const +{ + std::cout << client->zookeeper->sync(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; +} + bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const { return true; diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index c2f7bd9b896..19d5ee2a516 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -101,6 +101,17 @@ class GetCommand : public IKeeperClientCommand String getHelpMessage() const override { return "{} -- Returns the node's value"; } }; +class ExistsCommand : public IKeeperClientCommand +{ + String getName() const override { return "exists"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "{} -- Returns `1` if node exists, `0` otherwise"; } +}; + class GetStatCommand : public IKeeperClientCommand { String getName() const override { return "get_stat"; } @@ -179,7 +190,8 @@ class RMRCommand : public IKeeperClientCommand class ReconfigCommand : public IKeeperClientCommand { - enum class Operation : Int64 { + enum class Operation : Int64 + { ADD = 0, REMOVE = 1, SET = 2, @@ -191,7 +203,18 @@ class ReconfigCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} \"\" [version] -- Reconfigures a ZooKeeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration"; } + String getHelpMessage() const override { return "{} \"\" [version] -- Reconfigures Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration"; } +}; + +class SyncCommand: public IKeeperClientCommand +{ + String getName() const override { return "sync"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "{} -- Synchronizes node between processes and leader"; } }; class HelpCommand : public IKeeperClientCommand diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index afaf94448d5..890b937e384 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -173,6 +173,14 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) Poco::Util::Option("log-level", "", "set log level") .argument("") .binding("log-level")); + + options.addOption( + Poco::Util::Option("no-confirmation", "", "if set, will not require a confirmation on several commands. default false for interactive and true for query") + .binding("no-confirmation")); + + options.addOption( + Poco::Util::Option("tests-mode", "", "run keeper-client in a special mode for tests. all commands output are separated by special symbols. default false") + .binding("tests-mode")); } void KeeperClient::initialize(Poco::Util::Application & /* self */) @@ -187,6 +195,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), @@ -194,6 +203,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), }); @@ -233,7 +243,6 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) EventNotifier::init(); } - bool KeeperClient::processQueryText(const String & text) { if (exit_strings.find(text) != exit_strings.end()) @@ -287,7 +296,7 @@ bool KeeperClient::processQueryText(const String & text) return true; } -void KeeperClient::runInteractive() +void KeeperClient::runInteractiveReplxx() { LineReader::Patterns query_extenders = {"\\"}; @@ -321,6 +330,26 @@ void KeeperClient::runInteractive() } } +void KeeperClient::runInteractiveInputStream() +{ + for (String input; std::getline(std::cin, input);) + { + if (!processQueryText(input)) + break; + + std::cout << "\a\a\a\a" << std::endl; + std::cerr << std::flush; + } +} + +void KeeperClient::runInteractive() +{ + if (config().hasOption("tests-mode")) + runInteractiveInputStream(); + else + runInteractiveReplxx(); +} + int KeeperClient::main(const std::vector & /* args */) { if (config().hasOption("help")) @@ -370,9 +399,11 @@ int KeeperClient::main(const std::vector & /* args */) zk_args.operation_timeout_ms = config().getInt("operation-timeout", 10) * 1000; zookeeper = std::make_unique(zk_args); + if (config().has("no-confirmation") || config().has("query")) + ask_confirmation = false; + if (config().has("query")) { - ask_confirmation = false; processQueryText(config().getString("query")); } else diff --git a/programs/keeper-client/KeeperClient.h b/programs/keeper-client/KeeperClient.h index 4a20a45de6e..0d3db3c2f02 100644 --- a/programs/keeper-client/KeeperClient.h +++ b/programs/keeper-client/KeeperClient.h @@ -49,6 +49,9 @@ public: protected: void runInteractive(); + void runInteractiveReplxx(); + void runInteractiveInputStream(); + bool processQueryText(const String & text); void loadCommands(std::vector && new_commands); diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 93ea3fa74b7..a9ee3075750 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,6 +1,164 @@ +import io +import subprocess import socket import time +import typing as tp +import contextlib +import select from kazoo.client import KazooClient +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.client import CommandRequest + + +def execute_keeper_client_query(cluster: ClickHouseCluster, node: ClickHouseInstance, query: str) -> str: + request = CommandRequest( + [ + cluster.server_bin_path, + "keeper-client", + "--host", + str(cluster.get_instance_ip(node.name)), + "--port", + str(cluster.zookeeper_port), + "-q", + query, + ], + stdin="", + ) + + return request.get_answer() + + +class KeeperException(Exception): + pass + + +class KeeperClient(object): + + SEPARATOR = b'\a\a\a\a\n' + + def __init__(self, bin_path: str, host: str, port: int): + self.bin_path = bin_path + self.host = host + self.port = port + + self.proc = subprocess.Popen( + [ + bin_path, + 'keeper-client', + '--host', + host, + '--port', + str(port), + '--log-level', + 'error', + '--tests-mode', + '--no-confirmation', + ], + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + ) + + self.poller = select.epoll() + self.poller.register(self.proc.stdout) + self.poller.register(self.proc.stderr) + + self._fd_nums = { + self.proc.stdout.fileno(): self.proc.stdout, + self.proc.stderr.fileno(): self.proc.stderr, + } + + self.stopped = False + + def execute_query(self, query: str, timeout: float = 10.) -> str: + output = io.BytesIO() + + self.proc.stdin.write(query.encode() + b'\n') + self.proc.stdin.flush() + + events = self.poller.poll(timeout) + + for fd_num, event in events: + if event & (select.EPOLLIN | select.EPOLLPRI): + file = self._fd_nums[fd_num] + + if file == self.proc.stdout: + while True: + chunk = file.readline() + if chunk.endswith(self.SEPARATOR): + break + + output.write(chunk) + + elif file == self.proc.stderr: + assert self.proc.stdout.readline() == self.SEPARATOR + raise KeeperException(self.proc.stderr.readline().strip().decode()) + + else: + raise ValueError(f'Failed to read from pipe. Flag {event}') + + data = output.getvalue().strip().decode() + return data + + def cd(self, path: str, timeout: float = 10.): + self.execute_query(f'cd {path}', timeout) + + def ls(self, path: str, timeout: float = 10.) -> list[str]: + return self.execute_query(f'ls {path}', timeout).split(' ') + + def create(self, path: str, value: str, timeout: float = 10.): + self.execute_query(f'create {path} {value}', timeout) + + def get(self, path: str, timeout: float = 10.) -> str: + return self.execute_query(f'get {path}', timeout) + + def exists(self, path: str, timeout: float = 10.) -> bool: + return bool(int(self.execute_query(f'exists {path}', timeout))) + + def stop(self): + if not self.stopped: + self.stopped = True + self.proc.communicate(b'exit\n', timeout=10.) + + def sync(self, path: str, timeout: float = 10.): + self.execute_query(f'sync {path}', timeout) + + def touch(self, path: str, timeout: float = 10.): + self.execute_query(f'touch {path}', timeout) + + def find_big_family(self, path: str, n: int = 10, timeout: float = 10.) -> str: + return self.execute_query(f'find_big_family {path} {n}', timeout) + + def find_super_nodes(self, threshold: int, timeout: float = 10.) -> str: + return self.execute_query(f'find_super_nodes {threshold}', timeout) + + def delete_stale_backups(self, timeout: float = 10.) -> str: + return self.execute_query('delete_stale_backups', timeout) + + def reconfig(self, joining: tp.Optional[str], leaving: tp.Optional[str], new_members: tp.Optional[str], timeout: float = 10.) -> str: + if bool(joining) + bool(leaving) + bool(new_members) != 1: + raise ValueError('Exactly one of joining, leaving or new_members must be specified') + + if joining is not None: + operation = 'add' + elif leaving is not None: + operation = 'remove' + elif new_members is not None: + operation = 'set' + else: + raise ValueError('At least one of joining, leaving or new_members must be specified') + + return self.execute_query(f'reconfig {operation} {joining or leaving or new_members}', timeout) + + @classmethod + @contextlib.contextmanager + def from_cluster(cls, cluster: ClickHouseCluster, keeper_node: str, port: tp.Optional[int] = None) -> 'KeeperClient': + client = cls(cluster.server_bin_path, cluster.get_instance_ip(keeper_node), port or cluster.zookeeper_port) + + try: + yield client + finally: + client.stop() def get_keeper_socket(cluster, node, port=9181): @@ -70,14 +228,14 @@ def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: return _fake -def get_config_str(zk: KazooClient) -> str: +def get_config_str(zk: KeeperClient) -> str: """ Return decoded contents of /keeper/config node """ - return zk.get("/keeper/config")[0].decode("utf-8") + return zk.get("/keeper/config") -def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = 30.0): +def wait_configs_equal(left_config: str, right_zk: KeeperClient, timeout: float = 30.0): """ Check whether get /keeper/config result in left_config is equal to get /keeper/config on right_zk ZK connection. diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 8f7056a5afd..c82917372b8 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -1,7 +1,7 @@ import pytest -from helpers.client import CommandRequest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +from helpers.keeper_utils import KeeperClient cluster = ClickHouseCluster(__file__) @@ -24,39 +24,28 @@ def started_cluster(): cluster.shutdown() -def keeper_query(query: str): - return CommandRequest( - [ - cluster.server_bin_path, - "keeper-client", - "--host", - str(cluster.get_instance_ip("zoo1")), - "--port", - str(cluster.zookeeper_port), - "-q", - query, - ], - stdin="", - ) +@pytest.fixture(scope="function") +def client(started_cluster): + with KeeperClient.from_cluster(cluster, "zoo1") as keeper_client: + yield keeper_client -def test_big_family(): - command = keeper_query( - "touch test_big_family;" - "touch test_big_family/1;" - "touch test_big_family/1/1;" - "touch test_big_family/1/2;" - "touch test_big_family/1/3;" - "touch test_big_family/1/4;" - "touch test_big_family/1/5;" - "touch test_big_family/2;" - "touch test_big_family/2/1;" - "touch test_big_family/2/2;" - "touch test_big_family/2/3;" - "find_big_family test_big_family;" - ) +def test_big_family(client: KeeperClient): + client.touch("/test_big_family") + client.touch("/test_big_family/1") + client.touch("/test_big_family/1/1") + client.touch("/test_big_family/1/2") + client.touch("/test_big_family/1/3") + client.touch("/test_big_family/1/4") + client.touch("/test_big_family/1/5") + client.touch("/test_big_family/2") + client.touch("/test_big_family/2/1") + client.touch("/test_big_family/2/2") + client.touch("/test_big_family/2/3") - assert command.get_answer() == TSV( + response = client.find_big_family("/test_big_family") + + assert response == TSV( [ ["/test_big_family/1", "5"], ["/test_big_family/2", "3"], @@ -71,34 +60,33 @@ def test_big_family(): ] ) - command = keeper_query("find_big_family test_big_family 1;") + response = client.find_big_family("/test_big_family", 1) - assert command.get_answer() == TSV( + assert response == TSV( [ ["/test_big_family/1", "5"], ] ) -def test_find_super_nodes(): - command = keeper_query( - "touch test_find_super_nodes;" - "touch test_find_super_nodes/1;" - "touch test_find_super_nodes/1/1;" - "touch test_find_super_nodes/1/2;" - "touch test_find_super_nodes/1/3;" - "touch test_find_super_nodes/1/4;" - "touch test_find_super_nodes/1/5;" - "touch test_find_super_nodes/2;" - "touch test_find_super_nodes/2/1;" - "touch test_find_super_nodes/2/2;" - "touch test_find_super_nodes/2/3;" - "touch test_find_super_nodes/2/4;" - "cd test_find_super_nodes;" - "find_super_nodes 4;" - ) +def test_find_super_nodes(client: KeeperClient): + client.touch("/test_find_super_nodes") + client.touch("/test_find_super_nodes/1") + client.touch("/test_find_super_nodes/1/1") + client.touch("/test_find_super_nodes/1/2") + client.touch("/test_find_super_nodes/1/3") + client.touch("/test_find_super_nodes/1/4") + client.touch("/test_find_super_nodes/1/5") + client.touch("/test_find_super_nodes/2") + client.touch("/test_find_super_nodes/2/1") + client.touch("/test_find_super_nodes/2/2") + client.touch("/test_find_super_nodes/2/3") + client.touch("/test_find_super_nodes/2/4") - assert command.get_answer() == TSV( + client.cd("/test_find_super_nodes") + + response = client.find_super_nodes(4) + assert response == TSV( [ ["/test_find_super_nodes/1", "5"], ["/test_find_super_nodes/2", "4"], @@ -106,41 +94,38 @@ def test_find_super_nodes(): ) -def test_delete_stale_backups(): - command = keeper_query( - "touch /clickhouse;" - "touch /clickhouse/backups;" - "touch /clickhouse/backups/1;" - "touch /clickhouse/backups/1/stage;" - "touch /clickhouse/backups/1/stage/alive123;" - "touch /clickhouse/backups/2;" - "touch /clickhouse/backups/2/stage;" - "touch /clickhouse/backups/2/stage/dead123;" - "delete_stale_backups;" - "y;" - "ls clickhouse/backups;" - ) +def test_delete_stale_backups(client: KeeperClient): + client.touch("/clickhouse") + client.touch("/clickhouse/backups") + client.touch("/clickhouse/backups/1") + client.touch("/clickhouse/backups/1/stage") + client.touch("/clickhouse/backups/1/stage/alive123") + client.touch("/clickhouse/backups/2") + client.touch("/clickhouse/backups/2/stage") + client.touch("/clickhouse/backups/2/stage/dead123") - assert command.get_answer() == ( - "You are going to delete all inactive backups in /clickhouse/backups. Continue?\n" + response = client.delete_stale_backups() + + assert response == ( 'Found backup "/clickhouse/backups/1", checking if it\'s active\n' 'Backup "/clickhouse/backups/1" is active, not going to delete\n' 'Found backup "/clickhouse/backups/2", checking if it\'s active\n' - 'Backup "/clickhouse/backups/2" is not active, deleting it\n' - "1\n" + 'Backup "/clickhouse/backups/2" is not active, deleting it' ) - -def test_base_commands(): - command = keeper_query( - "create test_create_zk_node1 testvalue1;" - "create test_create_zk_node_2 testvalue2;" - "get test_create_zk_node1;" - ) - - assert command.get_answer() == "testvalue1\n" + assert client.ls("/clickhouse/backups") == ["1"] -def test_four_letter_word_commands(): - command = keeper_query("ruok") - assert command.get_answer() == "imok\n" +def test_base_commands(client: KeeperClient): + client.create("/test_create_zk_node1", "testvalue1") + client.create("/test_create_zk_node_2", "testvalue2") + assert client.get("/test_create_zk_node1") == "testvalue1" + + client.create("/123", "1=2") + client.create("/123/321", "'foo;bar'") + assert client.get("/123") == "1=2" + assert client.get("/123/321") == "foo;bar" + + +def test_four_letter_word_commands(client: KeeperClient): + assert client.execute_query("ruok") == "imok" diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py index 2c2da7403a1..6e05766c271 100644 --- a/tests/integration/test_keeper_reconfig_add/test.py +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -1,11 +1,10 @@ #!/usr/bin/env python3 import pytest -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance import helpers.keeper_utils as ku import os -from kazoo.client import KazooClient -from kazoo.exceptions import BadArgumentsException +import typing as tp cluster = ClickHouseCluster(__file__) CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") @@ -19,11 +18,7 @@ part_of_cluster = "now this node is the part of cluster" zk1, zk2, zk3 = None, None, None -def get_fake_zk(node): - return ku.get_fake_zk(cluster, node) - - -@pytest.fixture(scope="module") +@pytest.fixture(scope="module", autouse=True) def started_cluster(): try: cluster.start() @@ -43,21 +38,26 @@ def started_cluster(): yield cluster finally: + conn: tp.Optional[ku.KeeperClient] for conn in [zk1, zk2, zk3]: - if conn: + if conn is not None: conn.stop() - conn.close() cluster.shutdown() -def test_reconfig_add(started_cluster): +def create_client(node: ClickHouseInstance): + return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) + + +def test_reconfig_add(): """ Add a node to another node. Then add another node to two. """ + global zk1, zk2, zk3 + zk1 = create_client(node1) - zk1 = get_fake_zk(node1) - config = ku.get_config_str(zk1) + config = zk1.get("/keeper/config") print("Initial config", config) assert len(config.split("\n")) == 1 @@ -65,24 +65,22 @@ def test_reconfig_add(started_cluster): assert "node2" not in config assert "node3" not in config - with pytest.raises(BadArgumentsException): + with pytest.raises(ku.KeeperException): # duplicate id with different endpoint zk1.reconfig(joining="server.1=localhost:1337", leaving=None, new_members=None) - with pytest.raises(BadArgumentsException): + with pytest.raises(ku.KeeperException): # duplicate endpoint zk1.reconfig(joining="server.8=node1:9234", leaving=None, new_members=None) for i in range(100): - zk1.create(f"/test_three_{i}", b"somedata") + zk1.create(f"/test_three_{i}", "somedata") node2.start_clickhouse() - config, _ = zk1.reconfig( + config = zk1.reconfig( joining="server.2=node2:9234", leaving=None, new_members=None ) ku.wait_until_connected(cluster, node2) - - config = config.decode("utf-8") print("After adding 2", config) assert len(config.split("\n")) == 2 @@ -90,12 +88,12 @@ def test_reconfig_add(started_cluster): assert "node2" in config assert "node3" not in config - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) ku.wait_configs_equal(config, zk2) for i in range(100): - assert zk2.exists(f"/test_three_{i}") is not None - zk2.create(f"/test_three_{100 + i}", b"somedata") + assert zk2.exists(f"/test_three_{i}") + zk2.create(f"/test_three_{100 + i}", "somedata") # Why not both? # One node will process add_srv request, other will pull out updated config, apply @@ -107,23 +105,21 @@ def test_reconfig_add(started_cluster): assert node2.contains_in_log(part_of_cluster) zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) zk1.sync("/test_three_0") for i in range(200): - assert zk1.exists(f"/test_three_{i}") is not None + assert zk1.exists(f"/test_three_{i}") for i in range(100): - zk2.create(f"/test_four_{i}", b"somedata") + zk2.create(f"/test_four_{i}", "somedata") node3.start_clickhouse() - config, _ = zk2.reconfig( + config = zk2.reconfig( joining="server.3=node3:9234", leaving=None, new_members=None ) ku.wait_until_connected(cluster, node3) - config = config.decode("utf-8") print("After adding 3", config) assert len(config.split("\n")) == 3 @@ -131,25 +127,23 @@ def test_reconfig_add(started_cluster): assert "node2" in config assert "node3" in config - zk3 = get_fake_zk(node3) + zk3 = create_client(node3) ku.wait_configs_equal(config, zk3) for i in range(100): - assert zk3.exists(f"/test_four_{i}") is not None - zk3.create(f"/test_four_{100 + i}", b"somedata") + assert zk3.exists(f"/test_four_{i}") + zk3.create(f"/test_four_{100 + i}", "somedata") zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) zk1.sync("/test_four_0") zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) zk2.sync("/test_four_0") for i in range(200): - assert zk1.exists(f"/test_four_{i}") is not None - assert zk2.exists(f"/test_four_{i}") is not None + assert zk1.exists(f"/test_four_{i}") + assert zk2.exists(f"/test_four_{i}") assert node3.contains_in_log(part_of_cluster) diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py index fb0a9472df3..daab94c59c4 100644 --- a/tests/integration/test_keeper_reconfig_remove/test.py +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -1,11 +1,11 @@ #!/usr/bin/env python3 +import subprocess import pytest -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance import helpers.keeper_utils as ku import os -from kazoo.client import KazooClient -from kazoo.exceptions import BadVersionException, BadArgumentsException +import typing as tp cluster = ClickHouseCluster(__file__) CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") @@ -23,16 +23,16 @@ def started_cluster(): cluster.start() yield cluster finally: + conn: tp.Optional[ku.KeeperClient] for conn in [zk1, zk2, zk3]: if conn: conn.stop() - conn.close() cluster.shutdown() -def get_fake_zk(node): - return ku.get_fake_zk(cluster, node) +def create_client(node: ClickHouseInstance): + return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) def test_reconfig_remove_followers_from_3(started_cluster): @@ -42,9 +42,9 @@ def test_reconfig_remove_followers_from_3(started_cluster): Check that remaining node is in standalone mode. """ - zk1 = get_fake_zk(node1) - config, _ = zk1.get("/keeper/config") - config = config.decode("utf-8") + global zk1, zk2, zk3 + zk1 = create_client(node1) + config = zk1.get("/keeper/config") print("Initial config", config) assert len(config.split("\n")) == 3 @@ -52,36 +52,33 @@ def test_reconfig_remove_followers_from_3(started_cluster): assert "node2" in config assert "node3" in config - with pytest.raises(BadVersionException): - zk1.reconfig(joining=None, leaving="1", new_members=None, from_config=20) - with pytest.raises(BadArgumentsException): + with pytest.raises(ValueError): zk1.reconfig(joining=None, leaving=None, new_members=None) - with pytest.raises(BadArgumentsException): + with pytest.raises(ku.KeeperException): # bulk reconfiguration is not supported zk1.reconfig(joining=None, leaving=None, new_members="3") - with pytest.raises(BadArgumentsException): + with pytest.raises(ValueError): zk1.reconfig(joining="1", leaving="1", new_members="3") - with pytest.raises(BadArgumentsException): + with pytest.raises(ku.KeeperException): # at least one node must be left zk1.reconfig(joining=None, leaving="1,2,3", new_members=None) for i in range(100): - zk1.create(f"/test_two_{i}", b"somedata") + zk1.create(f"/test_two_{i}", "somedata") - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) zk2.sync("/test_two_0") ku.wait_configs_equal(config, zk2) - zk3 = get_fake_zk(node3) + zk3 = create_client(node3) zk3.sync("/test_two_0") ku.wait_configs_equal(config, zk3) for i in range(100): - assert zk2.exists(f"test_two_{i}") is not None - assert zk3.exists(f"test_two_{i}") is not None + assert zk2.exists(f"test_two_{i}") + assert zk3.exists(f"test_two_{i}") - config, _ = zk1.reconfig(joining=None, leaving="3", new_members=None) - config = config.decode("utf-8") + config = zk1.reconfig(joining=None, leaving="3", new_members=None) print("After removing 3", config) assert len(config.split("\n")) == 2 @@ -90,35 +87,26 @@ def test_reconfig_remove_followers_from_3(started_cluster): assert "node3" not in config zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) ku.wait_configs_equal(config, zk2) for i in range(100): - assert zk2.exists(f"test_two_{i}") is not None - zk2.create(f"/test_two_{100 + i}", b"otherdata") + assert zk2.exists(f"test_two_{i}") + zk2.create(f"/test_two_{100 + i}", "otherdata") zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) zk1.sync("/test_two_0") for i in range(200): - assert zk1.exists(f"test_two_{i}") is not None - - with pytest.raises(Exception): - zk3.stop() - zk3.close() - zk3 = get_fake_zk(node3) - zk3.sync("/test_two_0") + assert zk1.exists(f"test_two_{i}") assert node3.contains_in_log(log_msg_removed) for i in range(100): - zk2.create(f"/test_two_{200 + i}", b"otherdata") + zk2.create(f"/test_two_{200 + i}", "otherdata") - config, _ = zk1.reconfig(joining=None, leaving="2", new_members=None) - config = config.decode("utf-8") + config = zk1.reconfig(joining=None, leaving="2", new_members=None) print("After removing 2", config) assert len(config.split("\n")) == 1 @@ -127,19 +115,12 @@ def test_reconfig_remove_followers_from_3(started_cluster): assert "node3" not in config zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) zk1.sync("/test_two_0") for i in range(300): - assert zk1.exists(f"test_two_{i}") is not None - - with pytest.raises(Exception): - zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) - zk2.sync("/test_two_0") + assert zk1.exists(f"test_two_{i}") assert not node1.contains_in_log(log_msg_removed) assert node2.contains_in_log(log_msg_removed) - assert "Mode: standalone" in zk1.command(b"stat") + assert "Mode: standalone" in zk1.execute_query('stat') diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py index ec0d8b95eff..e99327e1098 100644 --- a/tests/integration/test_keeper_reconfig_remove_many/test.py +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -1,11 +1,11 @@ #!/usr/bin/env python3 import pytest -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance import helpers.keeper_utils as ku import os -from kazoo.client import KazooClient, KazooState -from kazoo.exceptions import BadVersionException, BadArgumentsException +import typing as tp + cluster = ClickHouseCluster(__file__) CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") @@ -26,49 +26,49 @@ def started_cluster(): cluster.start() yield cluster finally: + conn: tp.Optional[ku.KeeperClient] for conn in [zk1, zk2, zk3, zk4, zk5]: if conn: conn.stop() - conn.close() cluster.shutdown() -def get_fake_zk(node): - return ku.get_fake_zk(cluster, node) +def create_client(node: ClickHouseInstance): + return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) def test_reconfig_remove_2_and_leader(started_cluster): """ Remove 2 followers from a cluster of 5. Remove leader from 3 nodes. """ + global zk1, zk2, zk3, zk4, zk5 - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) config = ku.get_config_str(zk1) print("Initial config", config) assert len(config.split("\n")) == 5 for i in range(100): - zk1.create(f"/test_two_{i}", b"somedata") + zk1.create(f"/test_two_{i}", "somedata") - zk4 = get_fake_zk(node4) + zk4 = create_client(node4) zk4.sync("/test_two_0") ku.wait_configs_equal(config, zk4) - zk5 = get_fake_zk(node5) + zk5 = create_client(node5) zk5.sync("/test_two_0") ku.wait_configs_equal(config, zk5) for i in range(100): - assert zk4.exists(f"test_two_{i}") is not None - assert zk5.exists(f"test_two_{i}") is not None + assert zk4.exists(f"test_two_{i}") + assert zk5.exists(f"test_two_{i}") - zk4.create(f"/test_two_{100 + i}", b"otherdata") + zk4.create(f"/test_two_{100 + i}", "otherdata") - zk2 = get_fake_zk(node2) - config, _ = zk2.reconfig(joining=None, leaving="4,5", new_members=None) - config = config.decode("utf-8") + zk2 = create_client(node2) + config = zk2.reconfig(joining=None, leaving="4,5", new_members=None) print("After removing 4,5", config) assert len(config.split("\n")) == 3 @@ -79,27 +79,14 @@ def test_reconfig_remove_2_and_leader(started_cluster): assert "node5" not in config zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) + zk1 = create_client(node1) zk1.sync("/test_two_0") ku.wait_configs_equal(config, zk1) for i in range(200): - assert zk1.exists(f"test_two_{i}") is not None - assert zk2.exists(f"test_two_{i}") is not None - - with pytest.raises(Exception): - zk4.stop() - zk4.close() - zk4 = get_fake_zk(node4) - zk4.sync("/test_two_0") - - with pytest.raises(Exception): - zk5.stop() - zk5.close() - zk5 = get_fake_zk(node5) - zk5.sync("/test_two_0") + assert zk1.exists(f"test_two_{i}") + assert zk2.exists(f"test_two_{i}") assert not node1.contains_in_log(log_msg_removed) assert not node2.contains_in_log(log_msg_removed) @@ -110,11 +97,10 @@ def test_reconfig_remove_2_and_leader(started_cluster): assert ku.is_leader(cluster, node1) for i in range(100): - zk1.create(f"/test_leader_{i}", b"somedata") + zk1.create(f"/test_leader_{i}", "somedata") # when a leader gets a remove request, it must yield leadership - config, _ = zk1.reconfig(joining=None, leaving="1", new_members=None) - config = config.decode("utf-8") + config = zk1.reconfig(joining=None, leaving="1", new_members=None) print("After removing 1 (leader)", config) assert len(config.split("\n")) == 2 @@ -125,24 +111,17 @@ def test_reconfig_remove_2_and_leader(started_cluster): assert "node5" not in config zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) zk2.sync("/test_leader_0") ku.wait_configs_equal(config, zk2) - zk3 = get_fake_zk(node3) + zk3 = create_client(node3) zk3.sync("/test_leader_0") ku.wait_configs_equal(config, zk3) for i in range(100): - assert zk2.exists(f"test_leader_{i}") is not None - assert zk3.exists(f"test_leader_{i}") is not None - - with pytest.raises(Exception): - zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) - zk1.sync("/test_leader_0") + assert zk2.exists(f"test_leader_{i}") + assert zk3.exists(f"test_leader_{i}") assert node1.contains_in_log(log_msg_removed) assert not node2.contains_in_log(log_msg_removed) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index ef1d5394b67..bbb5b66eaa5 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -1,11 +1,10 @@ #!/usr/bin/env python3 import pytest -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from os.path import join, dirname, realpath -import time import helpers.keeper_utils as ku -from kazoo.client import KazooClient, KazooState +import typing as tp cluster = ClickHouseCluster(__file__) CONFIG_DIR = join(dirname(realpath(__file__)), "configs") @@ -31,24 +30,26 @@ def started_cluster(): yield cluster finally: + conn: tp.Optional[ku.KeeperClient] for conn in [zk1, zk2, zk3, zk4]: if conn: conn.stop() - conn.close() cluster.shutdown() -def get_fake_zk(node): - return ku.get_fake_zk(cluster, node) +def create_client(node: ClickHouseInstance): + return ku.KeeperClient( + cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181 + ) def test_reconfig_replace_leader(started_cluster): """ Remove leader from a cluster of 3 and add a new node via two commands. """ - - zk1 = get_fake_zk(node1) + global zk1, zk2, zk3, zk4 + zk1 = create_client(node1) config = ku.get_config_str(zk1) assert len(config.split("\n")) == 3 @@ -58,23 +59,22 @@ def test_reconfig_replace_leader(started_cluster): assert "node4" not in config for i in range(100): - zk1.create(f"/test_four_{i}", b"somedata") + zk1.create(f"/test_four_{i}", "somedata") - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) zk2.sync("/test_four_0") ku.wait_configs_equal(config, zk2) - zk3 = get_fake_zk(node3) + zk3 = create_client(node3) zk3.sync("/test_four_0") ku.wait_configs_equal(config, zk3) for i in range(100): - assert zk2.exists(f"/test_four_{i}") is not None - assert zk3.exists(f"/test_four_{i}") is not None + assert zk2.exists(f"/test_four_{i}") + assert zk3.exists(f"/test_four_{i}") assert ku.is_leader(cluster, node1) - config, _ = zk2.reconfig(joining=None, leaving="1", new_members=None) - config = config.decode("utf-8") + config = zk2.reconfig(joining=None, leaving="1", new_members=None) print("After removing 1 (leader)", config) assert len(config.split("\n")) == 2 @@ -85,17 +85,10 @@ def test_reconfig_replace_leader(started_cluster): ku.wait_configs_equal(config, zk2) - with pytest.raises(Exception): - zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) - zk1.sync("/test_four_0") - node4.start_clickhouse() - config, _ = zk2.reconfig( + config = zk2.reconfig( joining="server.4=node4:9234", leaving=None, new_members=None ) - config = config.decode("utf-8") ku.wait_until_connected(cluster, node4) print("After adding 4", config) @@ -105,22 +98,20 @@ def test_reconfig_replace_leader(started_cluster): assert "node3" in config assert "node4" in config - zk4 = get_fake_zk(node4) + zk4 = create_client(node4) ku.wait_configs_equal(config, zk4) for i in range(100): - assert zk4.exists(f"test_four_{i}") is not None - zk4.create(f"/test_four_{100 + i}", b"somedata") + assert zk4.exists(f"test_four_{i}") + zk4.create(f"/test_four_{100 + i}", "somedata") zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) + zk2 = create_client(node2) zk2.sync("/test_four_0") ku.wait_configs_equal(config, zk2) zk3.stop() - zk3.close() - zk3 = get_fake_zk(node3) + zk3 = create_client(node3) zk3.sync("/test_four_0") ku.wait_configs_equal(config, zk3) diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml deleted file mode 100644 index 71f3403aca3..00000000000 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - 9181 - 1 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - true - - - 5000 - 10000 - trace - - - - - 1 - node1 - 9234 - - - 2 - node2 - 9234 - true - - - 3 - node3 - 9234 - true - - - - diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml deleted file mode 100644 index faefb4d1102..00000000000 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - 9181 - 2 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - true - - - 5000 - 10000 - trace - - - - - 1 - node1 - 9234 - - - 2 - node2 - 9234 - true - - - 3 - node3 - 9234 - true - - - - diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml deleted file mode 100644 index 80a9caa92c2..00000000000 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - 9181 - 3 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - true - - - 5000 - 10000 - trace - - - - - 1 - node1 - 9234 - - - 2 - node2 - 9234 - true - - - 3 - node3 - 9234 - true - - - - diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml deleted file mode 100644 index 9fd88fe5d63..00000000000 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - 9181 - 4 - /var/lib/clickhouse/coordination/log - /var/lib/clickhouse/coordination/snapshots - true - - - 5000 - 10000 - trace - - - - 2 node2 9234 - 3 node3 9234 - 4 node4 9234 - - - diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py deleted file mode 100644 index b099d0513e1..00000000000 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ /dev/null @@ -1,43 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -from helpers.cluster import ClickHouseCluster -from os.path import join, dirname, realpath -import time -import helpers.keeper_utils as ku -from kazoo.client import KazooClient, KazooState - -cluster = ClickHouseCluster(__file__) -CONFIG_DIR = join(dirname(realpath(__file__)), "configs") - -node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) -node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) -node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) -node4 = cluster.add_instance("node4", stay_alive=True) -zk1, zk2, zk3, zk4 = None, None, None, None - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - node4.stop_clickhouse() - node4.copy_file_to_container( - join(CONFIG_DIR, "keeper4.xml"), - "/etc/clickhouse-server/config.d/keeper.xml", - ) - - yield cluster - - finally: - for conn in [zk1, zk2, zk3, zk4]: - if conn: - conn.stop() - conn.close() - - cluster.shutdown() - - -def get_fake_zk(node): - return ku.get_fake_zk(cluster, node) From cc7892691940132fa0c5ac65bab3ccdb08e98024 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 7 Sep 2023 00:44:54 +0000 Subject: [PATCH 100/327] Automatic style fix --- tests/integration/helpers/keeper_utils.py | 105 +++++++++++------- .../test_keeper_reconfig_add/test.py | 12 +- .../test_keeper_reconfig_remove/test.py | 6 +- .../test_keeper_reconfig_remove_many/test.py | 4 +- .../test.py | 4 +- 5 files changed, 75 insertions(+), 56 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index a9ee3075750..767416a3dbc 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -10,7 +10,9 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.client import CommandRequest -def execute_keeper_client_query(cluster: ClickHouseCluster, node: ClickHouseInstance, query: str) -> str: +def execute_keeper_client_query( + cluster: ClickHouseCluster, node: ClickHouseInstance, query: str +) -> str: request = CommandRequest( [ cluster.server_bin_path, @@ -33,8 +35,7 @@ class KeeperException(Exception): class KeeperClient(object): - - SEPARATOR = b'\a\a\a\a\n' + SEPARATOR = b"\a\a\a\a\n" def __init__(self, bin_path: str, host: str, port: int): self.bin_path = bin_path @@ -44,15 +45,15 @@ class KeeperClient(object): self.proc = subprocess.Popen( [ bin_path, - 'keeper-client', - '--host', + "keeper-client", + "--host", host, - '--port', + "--port", str(port), - '--log-level', - 'error', - '--tests-mode', - '--no-confirmation', + "--log-level", + "error", + "--tests-mode", + "--no-confirmation", ], stdin=subprocess.PIPE, stdout=subprocess.PIPE, @@ -70,10 +71,10 @@ class KeeperClient(object): self.stopped = False - def execute_query(self, query: str, timeout: float = 10.) -> str: + def execute_query(self, query: str, timeout: float = 10.0) -> str: output = io.BytesIO() - self.proc.stdin.write(query.encode() + b'\n') + self.proc.stdin.write(query.encode() + b"\n") self.proc.stdin.flush() events = self.poller.poll(timeout) @@ -95,65 +96,83 @@ class KeeperClient(object): raise KeeperException(self.proc.stderr.readline().strip().decode()) else: - raise ValueError(f'Failed to read from pipe. Flag {event}') + raise ValueError(f"Failed to read from pipe. Flag {event}") data = output.getvalue().strip().decode() return data - def cd(self, path: str, timeout: float = 10.): - self.execute_query(f'cd {path}', timeout) + def cd(self, path: str, timeout: float = 10.0): + self.execute_query(f"cd {path}", timeout) - def ls(self, path: str, timeout: float = 10.) -> list[str]: - return self.execute_query(f'ls {path}', timeout).split(' ') + def ls(self, path: str, timeout: float = 10.0) -> list[str]: + return self.execute_query(f"ls {path}", timeout).split(" ") - def create(self, path: str, value: str, timeout: float = 10.): - self.execute_query(f'create {path} {value}', timeout) + def create(self, path: str, value: str, timeout: float = 10.0): + self.execute_query(f"create {path} {value}", timeout) - def get(self, path: str, timeout: float = 10.) -> str: - return self.execute_query(f'get {path}', timeout) + def get(self, path: str, timeout: float = 10.0) -> str: + return self.execute_query(f"get {path}", timeout) - def exists(self, path: str, timeout: float = 10.) -> bool: - return bool(int(self.execute_query(f'exists {path}', timeout))) + def exists(self, path: str, timeout: float = 10.0) -> bool: + return bool(int(self.execute_query(f"exists {path}", timeout))) def stop(self): if not self.stopped: self.stopped = True - self.proc.communicate(b'exit\n', timeout=10.) + self.proc.communicate(b"exit\n", timeout=10.0) - def sync(self, path: str, timeout: float = 10.): - self.execute_query(f'sync {path}', timeout) + def sync(self, path: str, timeout: float = 10.0): + self.execute_query(f"sync {path}", timeout) - def touch(self, path: str, timeout: float = 10.): - self.execute_query(f'touch {path}', timeout) + def touch(self, path: str, timeout: float = 10.0): + self.execute_query(f"touch {path}", timeout) - def find_big_family(self, path: str, n: int = 10, timeout: float = 10.) -> str: - return self.execute_query(f'find_big_family {path} {n}', timeout) + def find_big_family(self, path: str, n: int = 10, timeout: float = 10.0) -> str: + return self.execute_query(f"find_big_family {path} {n}", timeout) - def find_super_nodes(self, threshold: int, timeout: float = 10.) -> str: - return self.execute_query(f'find_super_nodes {threshold}', timeout) + def find_super_nodes(self, threshold: int, timeout: float = 10.0) -> str: + return self.execute_query(f"find_super_nodes {threshold}", timeout) - def delete_stale_backups(self, timeout: float = 10.) -> str: - return self.execute_query('delete_stale_backups', timeout) + def delete_stale_backups(self, timeout: float = 10.0) -> str: + return self.execute_query("delete_stale_backups", timeout) - def reconfig(self, joining: tp.Optional[str], leaving: tp.Optional[str], new_members: tp.Optional[str], timeout: float = 10.) -> str: + def reconfig( + self, + joining: tp.Optional[str], + leaving: tp.Optional[str], + new_members: tp.Optional[str], + timeout: float = 10.0, + ) -> str: if bool(joining) + bool(leaving) + bool(new_members) != 1: - raise ValueError('Exactly one of joining, leaving or new_members must be specified') + raise ValueError( + "Exactly one of joining, leaving or new_members must be specified" + ) if joining is not None: - operation = 'add' + operation = "add" elif leaving is not None: - operation = 'remove' + operation = "remove" elif new_members is not None: - operation = 'set' + operation = "set" else: - raise ValueError('At least one of joining, leaving or new_members must be specified') + raise ValueError( + "At least one of joining, leaving or new_members must be specified" + ) - return self.execute_query(f'reconfig {operation} {joining or leaving or new_members}', timeout) + return self.execute_query( + f"reconfig {operation} {joining or leaving or new_members}", timeout + ) @classmethod @contextlib.contextmanager - def from_cluster(cls, cluster: ClickHouseCluster, keeper_node: str, port: tp.Optional[int] = None) -> 'KeeperClient': - client = cls(cluster.server_bin_path, cluster.get_instance_ip(keeper_node), port or cluster.zookeeper_port) + def from_cluster( + cls, cluster: ClickHouseCluster, keeper_node: str, port: tp.Optional[int] = None + ) -> "KeeperClient": + client = cls( + cluster.server_bin_path, + cluster.get_instance_ip(keeper_node), + port or cluster.zookeeper_port, + ) try: yield client diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py index 6e05766c271..724bfdef492 100644 --- a/tests/integration/test_keeper_reconfig_add/test.py +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -47,7 +47,9 @@ def started_cluster(): def create_client(node: ClickHouseInstance): - return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) + return ku.KeeperClient( + cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181 + ) def test_reconfig_add(): @@ -77,9 +79,7 @@ def test_reconfig_add(): zk1.create(f"/test_three_{i}", "somedata") node2.start_clickhouse() - config = zk1.reconfig( - joining="server.2=node2:9234", leaving=None, new_members=None - ) + config = zk1.reconfig(joining="server.2=node2:9234", leaving=None, new_members=None) ku.wait_until_connected(cluster, node2) print("After adding 2", config) @@ -115,9 +115,7 @@ def test_reconfig_add(): zk2.create(f"/test_four_{i}", "somedata") node3.start_clickhouse() - config = zk2.reconfig( - joining="server.3=node3:9234", leaving=None, new_members=None - ) + config = zk2.reconfig(joining="server.3=node3:9234", leaving=None, new_members=None) ku.wait_until_connected(cluster, node3) print("After adding 3", config) diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py index daab94c59c4..d23d771edba 100644 --- a/tests/integration/test_keeper_reconfig_remove/test.py +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -32,7 +32,9 @@ def started_cluster(): def create_client(node: ClickHouseInstance): - return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) + return ku.KeeperClient( + cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181 + ) def test_reconfig_remove_followers_from_3(started_cluster): @@ -123,4 +125,4 @@ def test_reconfig_remove_followers_from_3(started_cluster): assert not node1.contains_in_log(log_msg_removed) assert node2.contains_in_log(log_msg_removed) - assert "Mode: standalone" in zk1.execute_query('stat') + assert "Mode: standalone" in zk1.execute_query("stat") diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py index e99327e1098..0f9d2fce374 100644 --- a/tests/integration/test_keeper_reconfig_remove_many/test.py +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -35,7 +35,9 @@ def started_cluster(): def create_client(node: ClickHouseInstance): - return ku.KeeperClient(cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181) + return ku.KeeperClient( + cluster.server_bin_path, cluster.get_instance_ip(node.name), 9181 + ) def test_reconfig_remove_2_and_leader(started_cluster): diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index bbb5b66eaa5..76a8eb092e2 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -86,9 +86,7 @@ def test_reconfig_replace_leader(started_cluster): ku.wait_configs_equal(config, zk2) node4.start_clickhouse() - config = zk2.reconfig( - joining="server.4=node4:9234", leaving=None, new_members=None - ) + config = zk2.reconfig(joining="server.4=node4:9234", leaving=None, new_members=None) ku.wait_until_connected(cluster, node4) print("After adding 4", config) From c2f4dc0f14f55cf9785da86cb2bf07c556e22d1e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 7 Sep 2023 01:32:39 +0000 Subject: [PATCH 101/327] Implemented peak_threads_usage --- docs/en/operations/system-tables/query_log.md | 3 +- docs/ru/operations/system-tables/query_log.md | 3 +- src/Common/ConcurrencyControl.cpp | 2 +- src/Common/ThreadStatus.h | 16 +++- src/Interpreters/ProcessList.cpp | 4 +- src/Interpreters/ProcessList.h | 1 + src/Interpreters/QueryLog.cpp | 3 + src/Interpreters/QueryLog.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 22 ++++- src/Interpreters/executeQuery.cpp | 1 + .../MergeTree/MergePlainMergeTreeTask.cpp | 1 - .../02871_peak_threads_usage.reference | 14 +++ .../0_stateless/02871_peak_threads_usage.sh | 85 +++++++++++++++++++ 13 files changed, 146 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02871_peak_threads_usage.reference create mode 100755 tests/queries/0_stateless/02871_peak_threads_usage.sh diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 7143520835f..ced97166702 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -101,7 +101,8 @@ Columns: - `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map.md)) — ProfileEvents that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) - `Settings` ([Map(String, String)](../../sql-reference/data-types/map.md)) — Settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. - `log_comment` ([String](../../sql-reference/data-types/string.md)) — Log comment. It can be set to arbitrary string no longer than [max_query_size](../../operations/settings/settings.md#settings-max_query_size). An empty string if it is not defined. -- `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Thread ids that are participating in query execution. +- `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Thread ids that are participating in query execution. These threads may not have run simultaneously. +- `peak_threads_usage` ([UInt64)](../../sql-reference/data-types/int-uint.md)) — Maximum count of simultaneous threads executing the query. - `used_aggregate_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `aggregate functions`, which were used during query execution. - `used_aggregate_function_combinators` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `aggregate functions combinators`, which were used during query execution. - `used_database_engines` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `database engines`, which were used during query execution. diff --git a/docs/ru/operations/system-tables/query_log.md b/docs/ru/operations/system-tables/query_log.md index 29793188d3d..8d79aa0eef7 100644 --- a/docs/ru/operations/system-tables/query_log.md +++ b/docs/ru/operations/system-tables/query_log.md @@ -99,7 +99,8 @@ ClickHouse не удаляет данные из таблица автомати - `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events)(#system_tables-events - `Settings` ([Map(String, String)](../../sql-reference/data-types/array.md)) — имена настроек, которые меняются, когда клиент выполняет запрос. Чтобы разрешить логирование изменений настроек, установите параметр `log_query_settings` равным 1. - `log_comment` ([String](../../sql-reference/data-types/string.md)) — комментарий к записи в логе. Представляет собой произвольную строку, длина которой должна быть не больше, чем [max_query_size](../../operations/settings/settings.md#settings-max_query_size). Если нет комментария, то пустая строка. -- `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — идентификаторы потоков, участвующих в обработке запросов. +- `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — идентификаторы потоков, участвующих в обработке запросов, эти потоки не обязательно выполняются одновременно. +- `peak_threads_usage` ([UInt64)](../../sql-reference/data-types/int-uint.md)) — максимальное количество одновременно работавших потоков, участвоваших в обработке запроса. - `used_aggregate_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — канонические имена `агрегатных функций`, использованных при выполнении запроса. - `used_aggregate_function_combinators` ([Array(String)](../../sql-reference/data-types/array.md)) — канонические имена `комбинаторов агрегатных функций`, использованных при выполнении запроса. - `used_database_engines` ([Array(String)](../../sql-reference/data-types/array.md)) — канонические имена `движков баз данных`, использованных при выполнении запроса. diff --git a/src/Common/ConcurrencyControl.cpp b/src/Common/ConcurrencyControl.cpp index de46f0e90ca..ebc1c3f9f42 100644 --- a/src/Common/ConcurrencyControl.cpp +++ b/src/Common/ConcurrencyControl.cpp @@ -57,7 +57,7 @@ ConcurrencyControl::Allocation::Allocation(ConcurrencyControl & parent_, SlotCou *waiter = this; } -// Grant single slot to allocation, returns true iff more slot(s) are required +// Grant single slot to allocation, returns true if more slot(s) are required bool ConcurrencyControl::Allocation::grant() { std::unique_lock lock{mutex}; diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 7c8dbdb68bd..02e1f50ff2d 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -107,15 +107,25 @@ public: static ThreadGroupPtr createForBackgroundProcess(ContextPtr storage_context); std::vector getInvolvedThreadIds() const; - void linkThread(UInt64 thread_it); + size_t getPeakThreadsUsage() const; + + void linkThread(UInt64 thread_id); + void unlinkThread(); private: mutable std::mutex mutex; /// Set up at creation, no race when reading - SharedData shared_data; + SharedData shared_data TSA_GUARDED_BY(mutex); + /// Set of all thread ids which has been attached to the group - std::unordered_set thread_ids; + std::unordered_set thread_ids TSA_GUARDED_BY(mutex); + + /// Count of simultaneously working threads + UInt64 active_thread_count TSA_GUARDED_BY(mutex) = 0; + + /// Peak threads count in the group + size_t peak_threads_usage TSA_GUARDED_BY(mutex) = 0; }; /** diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c299572ef41..8572470abc1 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -590,8 +590,10 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.peak_memory_usage = thread_group->memory_tracker.getPeak(); if (get_thread_list) + { res.thread_ids = thread_group->getInvolvedThreadIds(); - + res.peak_threads_usage = thread_group->getPeakThreadsUsage(); + } if (get_profile_events) res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 2eea49e1267..75a0eaa34bc 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -67,6 +67,7 @@ struct QueryStatusInfo /// Optional fields, filled by query std::vector thread_ids; + size_t peak_threads_usage; std::shared_ptr profile_counters; std::shared_ptr query_settings; std::string current_database; diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index df21e82305a..57f3968fba1 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -118,6 +118,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"log_comment", std::make_shared()}, {"thread_ids", std::make_shared(std::make_shared())}, + {"peak_threads_usage", std::make_shared()}, {"ProfileEvents", std::make_shared(low_cardinality_string, std::make_shared())}, {"Settings", std::make_shared(low_cardinality_string, low_cardinality_string)}, @@ -230,6 +231,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(threads_array); } + columns[i++]->insert(peak_threads_usage); + if (profile_counters) { auto * column = columns[i++].get(); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 5bc80280eac..733be5840e5 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -91,6 +91,7 @@ struct QueryLogElement String log_comment; std::vector thread_ids; + UInt64 peak_threads_usage{}; std::shared_ptr profile_counters; std::shared_ptr async_read_counters; std::shared_ptr query_settings; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6ee8ec987db..e0b94e8534a 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -61,10 +61,26 @@ std::vector ThreadGroup::getInvolvedThreadIds() const return res; } -void ThreadGroup::linkThread(UInt64 thread_it) +size_t ThreadGroup::getPeakThreadsUsage() const { std::lock_guard lock(mutex); - thread_ids.insert(thread_it); + return peak_threads_usage; +} + + +void ThreadGroup::linkThread(UInt64 thread_id) +{ + std::lock_guard lock(mutex); + thread_ids.insert(thread_id); + + ++active_thread_count; + peak_threads_usage = std::max(peak_threads_usage, active_thread_count); +} + +void ThreadGroup::unlinkThread() +{ + std::lock_guard lock(mutex); + --active_thread_count; } ThreadGroupPtr ThreadGroup::createForQuery(ContextPtr query_context_, std::function fatal_error_callback_) @@ -243,6 +259,8 @@ void ThreadStatus::detachFromGroup() /// Extract MemoryTracker out from query and user context memory_tracker.setParent(&total_memory_tracker); + thread_group->unlinkThread(); + thread_group.reset(); query_id_from_query_context.clear(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index bac6807b682..1bfeeaa8ad4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -241,6 +241,7 @@ addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo element.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0; element.thread_ids = info.thread_ids; + element.peak_threads_usage = info.peak_threads_usage; element.profile_counters = info.profile_counters; /// We need to refresh the access info since dependent views might have added extra information, either during diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 3f5753a0c95..c218acce903 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -40,7 +40,6 @@ bool MergePlainMergeTreeTask::executeStep() if (merge_list_entry) { switcher.emplace((*merge_list_entry)->thread_group); - } switch (state) diff --git a/tests/queries/0_stateless/02871_peak_threads_usage.reference b/tests/queries/0_stateless/02871_peak_threads_usage.reference new file mode 100644 index 00000000000..d772a2c4b4e --- /dev/null +++ b/tests/queries/0_stateless/02871_peak_threads_usage.reference @@ -0,0 +1,14 @@ +1 2 1 1 +2 2 1 1 +3 2 1 1 +4 2 1 1 +5 4 1 1 +6 6 1 1 +7 2 1 1 +8 2 1 1 +9 2 1 1 +10 6 1 1 +11 6 1 1 +12 6 1 1 +13 2 1 1 +14 2 1 1 diff --git a/tests/queries/0_stateless/02871_peak_threads_usage.sh b/tests/queries/0_stateless/02871_peak_threads_usage.sh new file mode 100755 index 00000000000..dfb3e665020 --- /dev/null +++ b/tests/queries/0_stateless/02871_peak_threads_usage.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash +# Tags: no-parallel +# Tag no-parallel: Avoid using threads in other parallel queries. + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +QUERY_OPTIONS=( + "--log_query_threads=1" + "--log_queries_min_type=QUERY_FINISH" + "--log_queries=1" + "--format=Null" + "--use_concurrency_control=0" +) + +UNIQUE_QUERY_ID="02871_1_$$" + +# TCPHandler and QueryPullPipeEx threads are always part of the query thread group, but those threads are not within the max_threads limit. +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_1" --query='SELECT 1' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_2" --query='SELECT 1 SETTINGS max_threads = 1' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_3" --query='SELECT 1 SETTINGS max_threads = 8' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_4" --query='SELECT * FROM numbers_mt(500000) SETTINGS max_threads = 1' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_5" --query='SELECT * FROM numbers_mt(500000) SETTINGS max_threads = 2' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_6" --query='SELECT * FROM numbers_mt(500000) SETTINGS max_threads = 4' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_7" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 1, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_8" --query='SELECT * FROM numbers_mt(5000), numbers(5000) SETTINGS max_threads = 4, joined_subquery_requires_alias=0' "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_9" -mn --query=""" +SELECT count() FROM + (SELECT number FROM numbers_mt(1,100000) + UNION ALL SELECT number FROM numbers_mt(10000, 200000) + UNION ALL SELECT number FROM numbers_mt(30000, 40000) + UNION ALL SELECT number FROM numbers_mt(30000, 40000) + UNION ALL SELECT number FROM numbers_mt(300000, 400000) + UNION ALL SELECT number FROM numbers_mt(300000, 400000) + UNION ALL SELECT number FROM numbers_mt(300000, 4000000) + UNION ALL SELECT number FROM numbers_mt(300000, 4000000) + ) SETTINGS max_threads = 1""" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_10" -mn --query=""" +SELECT count() FROM + (SELECT number FROM numbers_mt(1,100000) + UNION ALL SELECT number FROM numbers_mt(10000, 2000) + UNION ALL SELECT number FROM numbers_mt(30000, 40000) + UNION ALL SELECT number FROM numbers_mt(30000, 40) + UNION ALL SELECT number FROM numbers_mt(300000, 400) + UNION ALL SELECT number FROM numbers_mt(300000, 4000) + UNION ALL SELECT number FROM numbers_mt(300000, 40000) + UNION ALL SELECT number FROM numbers_mt(300000, 4000000) + ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_11" -mn --query=""" +SELECT count() FROM + (SELECT number FROM numbers_mt(1,100000) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 1) + UNION ALL SELECT number FROM numbers_mt(1, 4000000) + ) SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_12" -mn --query=""" +SELECT sum(number) FROM numbers_mt(100000) +GROUP BY number % 2 +WITH TOTALS ORDER BY number % 2 +SETTINGS max_threads = 4""" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_13" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 1" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} --query_id="${UNIQUE_QUERY_ID}_14" -mn --query="SELECT * FROM numbers(100000) SETTINGS max_threads = 4" "${QUERY_OPTIONS[@]}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +for i in {1..14} +do + ${CLICKHOUSE_CLIENT} -mn --query=""" + SELECT '${i}', + peak_threads_usage, + (select count() from system.query_thread_log WHERE system.query_thread_log.query_id = '${UNIQUE_QUERY_ID}_${i}' AND current_database = currentDatabase()) = length(thread_ids), + length(thread_ids) >= peak_threads_usage + FROM system.query_log + WHERE type = 'QueryFinish' AND query_id = '${UNIQUE_QUERY_ID}_${i}' AND current_database = currentDatabase()" +done From fef80f176ea3f88e6d8219b4448c581a0acda578 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 6 Sep 2023 21:56:14 -0400 Subject: [PATCH 102/327] Fix typo --- docs/en/operations/utilities/clickhouse-keeper-client.md | 2 +- programs/keeper-client/Commands.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 9f24ba9587b..42fe4fd196e 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -60,4 +60,4 @@ keeper foo bar - `delete_stale_backups` -- Deletes ClickHouse nodes used for backups that are now inactive - `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) - `sync ` -- Synchronizes node between processes and leader -- `reconfig "" [version]` -- Reconfigures Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration +- `reconfig "" [version]` -- Reconfigure Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 19d5ee2a516..aed98bfa0fb 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -203,7 +203,7 @@ class ReconfigCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} \"\" [version] -- Reconfigures Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration"; } + String getHelpMessage() const override { return "{} \"\" [version] -- Reconfigure Keeper cluster. See https://clickhouse.com/docs/en/guides/sre/keeper/clickhouse-keeper#reconfiguration"; } }; class SyncCommand: public IKeeperClientCommand From 1c8216b5c65b06a3c664cd57db790156f4e424f2 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 7 Sep 2023 03:27:04 +0000 Subject: [PATCH 103/327] added assertion --- src/Interpreters/ThreadStatusExt.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index e0b94e8534a..efb8c6792bc 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -80,6 +80,7 @@ void ThreadGroup::linkThread(UInt64 thread_id) void ThreadGroup::unlinkThread() { std::lock_guard lock(mutex); + chassert(active_thread_count > 0); --active_thread_count; } From f1e4ca4da5565f8be79d3aca08ac5fc65c57c58e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 7 Sep 2023 04:56:13 +0000 Subject: [PATCH 104/327] changed data_type --- src/Common/ThreadStatus.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 02e1f50ff2d..970bb5cd6e6 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -122,7 +122,7 @@ private: std::unordered_set thread_ids TSA_GUARDED_BY(mutex); /// Count of simultaneously working threads - UInt64 active_thread_count TSA_GUARDED_BY(mutex) = 0; + size_t active_thread_count TSA_GUARDED_BY(mutex) = 0; /// Peak threads count in the group size_t peak_threads_usage TSA_GUARDED_BY(mutex) = 0; From d11a9b4175e540192a781f73dced9e4ea9cdfa19 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 7 Sep 2023 08:51:35 +0000 Subject: [PATCH 105/327] update spell dict --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2c29fd9369e..231110c4f3b 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1329,10 +1329,10 @@ ddl deallocation deallocations debian +decodeHTMLComponent decodeURLComponent decodeURLFormComponent decodeXMLComponent -decodeHTMLComponent decompressor decrypt decrypted @@ -1346,17 +1346,18 @@ defaultRoles defaultValueOfArgumentType defaultValueOfTypeName delim -deltaLake -deltaSum -deltaSumTimestamp deltalake +deltaLake deltasum +deltaSum deltasumtimestamp +deltaSumTimestamp demangle denormalize denormalized denormalizing denormals +dequeued deserialization deserialized deserializing @@ -1441,6 +1442,7 @@ farmFingerprint farmHash fastops fcoverage +fifo filesystem filesystemAvailable filesystemCapacity @@ -1607,6 +1609,7 @@ incrementing indexHint indexOf infi +inflight initcap initcapUTF initialQueryID @@ -2536,6 +2539,7 @@ visitParamExtractRaw visitParamExtractString visitParamExtractUInt visitParamHas +vruntime wchc wchs webpage From c36a3fbd3900379f5c1efa80807e01a4fa1329fe Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 7 Sep 2023 12:00:25 +0300 Subject: [PATCH 106/327] renamed setting --- docs/en/operations/settings/settings.md | 6 ++--- docs/en/sql-reference/table-functions/file.md | 4 ++-- docs/en/sql-reference/table-functions/hdfs.md | 4 ++-- docs/ru/operations/settings/settings.md | 6 ++--- docs/ru/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/hdfs.md | 4 ++-- src/Core/Settings.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 20 ++++++++-------- src/Storages/StorageFile.cpp | 24 +++++++++---------- ...02771_multidirectory_globs_storage_file.sh | 2 +- 10 files changed, 37 insertions(+), 37 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ac53b46da7e..2fac021fe2d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4660,7 +4660,7 @@ The default value is `false`. true ``` -## ignore_eacces_multidirectory_globs {#ignore_eacces_multidirectory_globs} +## ignore_access_denied_multidirectory_globs {#ignore_access_denied_multidirectory_globs} Allows to ignore 'permission denied' errors when using multi-directory `{}` globs for [File](../../sql-reference/table-functions/file.md#globs_in_path) and [HDFS](../../sql-reference/table-functions/hdfs.md) storages. This setting is only applicable when multi directory `{}` glob is used. @@ -4689,12 +4689,12 @@ It happens because a multi-directory glob requires a recursive search in _all_ a If this setting is on, all inaccessible directories will be silently skipped, even if they are explicitly specified inside `{}`. ```sql -SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0; +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_access_denied_multidirectory_globs = 0; Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied ``` ```sql -SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1; +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_access_denied_multidirectory_globs = 1; ┌─_path───────────────────┬─_file───────┐ │ │ diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index f12ac732c22..06571fd724b 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -141,7 +141,7 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting for file & HDFS. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_access_denied_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) setting for file & HDFS. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. @@ -210,7 +210,7 @@ SELECT count(*) FROM file('big_dir/**/file002', 'CSV', 'name String, value UInt3 - [engine_file_allow_create_multiple_files](/docs/en/operations/settings/settings.md#engine_file_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. - [engine_file_skip_empty_files](/docs/en/operations/settings/settings.md#engine_file_skip_empty_files) - allows to skip empty files while reading. Disabled by default. - [storage_file_read_method](/docs/en/operations/settings/settings.md#engine-file-emptyif-not-exists) - method of reading data from storage file, one of: read, pread, mmap (only for clickhouse-local). Default value: `pread` for clickhouse-server, `mmap` for clickhouse-local. -- [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. +- [ignore_access_denied_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. **See Also** diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 0e4ee632b71..b50356202f7 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -45,7 +45,7 @@ Multiple path components can have globs. For being processed file should exists - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) setting. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. In case at least one of strings contains `/`, `'permission denied'` errors may be ignored using [ignore_access_denied_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) setting. - `{N..M}` — Substitutes any number in range from N to M including both borders. Constructions with `{}` are similar to the [remote table function](../../sql-reference/table-functions/remote.md)). @@ -102,7 +102,7 @@ FROM hdfs('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name Strin - [hdfs_truncate_on_insert](/docs/en/operations/settings/settings.md#hdfs-truncate-on-insert) - allows to truncate file before insert into it. Disabled by default. - [hdfs_create_multiple_files](/docs/en/operations/settings/settings.md#hdfs_allow_create_multiple_files) - allows to create a new file on each insert if format has suffix. Disabled by default. - [hdfs_skip_empty_files](/docs/en/operations/settings/settings.md#hdfs_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [ignore_eacces_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_eacces_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. +- [ignore_access_denied_multidirectory_globs](/docs/en/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) - allows to ignore permission denied errors for multi-directory globs. **See Also** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 3f0bc4ee2ec..73ffb9d6a12 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4207,7 +4207,7 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars └─────────────────────┴──────────────────────────┘ ``` -## ignore_eacces_multidirectory_globs {#ignore_eacces_multidirectory_globs} +## ignore_access_denied_multidirectory_globs {#ignore_access_denied_multidirectory_globs} Позволяет игнорировать ошибку 'permission denied', возникающую при использовании шаблона `{}`, содержащего `/` внутри себя. Работает для [File](../../sql-reference/table-functions/file.md#globs_in_path) и [HDFS](../../sql-reference/table-functions/hdfs.md). @@ -4237,12 +4237,12 @@ my_directory/ Если данная настройка имеет значение 1, то недоступные директории будут тихо пропущены, даже если они явно указаны внутри `{}`. ```sql -SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 0; +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_access_denied_multidirectory_globs = 0; Code: 1001. DB::Exception: std::__1::__fs::filesystem::filesystem_error: filesystem error: in directory_iterator::directory_iterator(...): Permission denied ``` ```sql -SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1; +SELECT _path, _file FROM file('my_directory/{data1/f1,data2/f2}.csv', CSV) SETTINGS ignore_access_denied_multidirectory_globs = 1; ┌─_path───────────────────┬─_file───────┐ │ │ diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 32d98bd19ee..4c12b7cca66 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs). +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_access_denied_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_access_denied_multidirectory_globs). - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). diff --git a/docs/ru/sql-reference/table-functions/hdfs.md b/docs/ru/sql-reference/table-functions/hdfs.md index d74248bd93c..98cb1fbfd8a 100644 --- a/docs/ru/sql-reference/table-functions/hdfs.md +++ b/docs/ru/sql-reference/table-functions/hdfs.md @@ -43,7 +43,7 @@ LIMIT 2 - `*` — Заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — Заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs). +- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. В случае, если в какой-либо из строк содержится `/`, то ошибки доступа (permission denied) к существующим, но недоступным директориям/файлам могут быть проигнорированы при помощи настройки [ignore_access_denied_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_access_denied_multidirectory_globs). - `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). @@ -61,5 +61,5 @@ LIMIT 2 **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) -- Параметр [ignore_eacces_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_eacces_multidirectory_globs) +- Параметр [ignore_access_denied_multidirectory_globs](/docs/ru/operations/settings/settings.md#ignore_access_denied_multidirectory_globs) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4706f8914f4..441498d50f8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -797,7 +797,7 @@ class IColumn; M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0) \ - M(Bool, ignore_eacces_multidirectory_globs, false, "Ignore access denied errors when processing multi-directory globs for file & HDFS.", 0)\ + M(Bool, ignore_access_denied_multidirectory_globs, false, "Ignore access denied errors when processing multi-directory globs for file & HDFS.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 09c205930dc..fc904c83413 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -74,7 +74,7 @@ namespace std::vector LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, - bool ignore_eacces_multidirectory_globs); + bool ignore_access_denied_multidirectory_globs); /* * When `{...}` has any `/`s, it must be processed in a different way: @@ -91,7 +91,7 @@ namespace re2::RE2 & matcher, const size_t max_depth, const size_t next_slash_after_glob_pos, - bool ignore_eacces_multidirectory_globs) + bool ignore_access_denied_multidirectory_globs) { /// We don't need to go all the way in every directory if max_depth is reached /// as it is upper limit of depth by simply counting `/`s in curly braces @@ -104,8 +104,8 @@ namespace { // ignore file not found (as in LSWithRegexpMatching) // keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. - // ignore permission denied if ignore_eacces_multidirectory_globs is true - if (!(ignore_eacces_multidirectory_globs && errno == EACCES)) + // ignore permission denied if ignore_access_denied_multidirectory_globs is true + if (!(ignore_access_denied_multidirectory_globs && errno == EACCES)) throw Exception( ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); } @@ -134,7 +134,7 @@ namespace { std::vector result_part = LSWithRegexpMatching( fs::path(full_path) / "" , fs, suffix_with_globs.substr(next_slash_after_glob_pos), - ignore_eacces_multidirectory_globs); + ignore_access_denied_multidirectory_globs); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -142,7 +142,7 @@ namespace { std::vector result_part = LSWithFoldedRegexpMatching( fs::path(full_path), fs, processed_suffix + dir_or_file_name, suffix_with_globs, - matcher, max_depth - 1, next_slash_after_glob_pos, ignore_eacces_multidirectory_globs); + matcher, max_depth - 1, next_slash_after_glob_pos, ignore_access_denied_multidirectory_globs); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -156,7 +156,7 @@ namespace const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, - bool ignore_eacces_multidirectory_globs) + bool ignore_access_denied_multidirectory_globs) { const size_t first_glob_pos = for_match.find_first_of("*?{"); const bool has_glob = first_glob_pos != std::string::npos; @@ -199,7 +199,7 @@ namespace if (slashes_in_glob) { return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, "", suffix_with_globs, matcher, - slashes_in_glob, next_slash_after_glob_pos, ignore_eacces_multidirectory_globs); + slashes_in_glob, next_slash_after_glob_pos, ignore_access_denied_multidirectory_globs); } HDFSFileInfo ls; @@ -233,7 +233,7 @@ namespace if (re2::RE2::FullMatch(file_name, matcher)) { std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, - suffix_with_globs.substr(next_slash_after_glob_pos), ignore_eacces_multidirectory_globs); + suffix_with_globs.substr(next_slash_after_glob_pos), ignore_access_denied_multidirectory_globs); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } @@ -261,7 +261,7 @@ namespace HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - auto res = LSWithRegexpMatching("/", fs, path_from_uri, context->getSettingsRef().ignore_eacces_multidirectory_globs); + auto res = LSWithRegexpMatching("/", fs, path_from_uri, context->getSettingsRef().ignore_access_denied_multidirectory_globs); return res; } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 84f1479881a..ee27757f1c2 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -104,7 +104,7 @@ void listFilesWithRegexpMatchingImpl( const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read, - bool ignore_eacces_multidirectory_globs, + bool ignore_access_denied_multidirectory_globs, std::vector & result, bool recursive = false); @@ -123,14 +123,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, size_t & total_bytes_to_read, const size_t max_depth, const size_t next_slash_after_glob_pos, - bool ignore_eacces_multidirectory_globs, + bool ignore_access_denied_multidirectory_globs, std::vector & result) { if (!max_depth) return; const fs::directory_iterator end; - fs::directory_iterator it = ignore_eacces_multidirectory_globs + fs::directory_iterator it = ignore_access_denied_multidirectory_globs ? fs::directory_iterator(path_for_ls, fs::directory_options::skip_permission_denied) : fs::directory_iterator(path_for_ls); for (; it != end; ++it) @@ -150,7 +150,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, { listFilesWithRegexpMatchingImpl(fs::path(full_path) / "" , suffix_with_globs.substr(next_slash_after_glob_pos), - total_bytes_to_read, ignore_eacces_multidirectory_globs, result); + total_bytes_to_read, ignore_access_denied_multidirectory_globs, result); } } else if (it->is_directory()) @@ -158,7 +158,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path), processed_suffix + dir_or_file_name, suffix_with_globs, matcher, total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, - ignore_eacces_multidirectory_globs, result); + ignore_access_denied_multidirectory_globs, result); } } @@ -171,7 +171,7 @@ void listFilesWithRegexpMatchingImpl( const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read, - bool ignore_eacces_multidirectory_globs, + bool ignore_access_denied_multidirectory_globs, std::vector & result, bool recursive) { @@ -231,7 +231,7 @@ void listFilesWithRegexpMatchingImpl( { listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, matcher, total_bytes_to_read, slashes_in_glob, next_slash_after_glob_pos, - ignore_eacces_multidirectory_globs, result); + ignore_access_denied_multidirectory_globs, result); return; } @@ -257,12 +257,12 @@ void listFilesWithRegexpMatchingImpl( { listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "", looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob, - total_bytes_to_read, ignore_eacces_multidirectory_globs, result, recursive); + total_bytes_to_read, ignore_access_denied_multidirectory_globs, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), - total_bytes_to_read, ignore_eacces_multidirectory_globs, result); + total_bytes_to_read, ignore_access_denied_multidirectory_globs, result); } } } @@ -271,10 +271,10 @@ std::vector listFilesWithRegexpMatching( const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read, - bool ignore_eacces_multidirectory_globs) + bool ignore_access_denied_multidirectory_globs) { std::vector result; - listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, ignore_eacces_multidirectory_globs, result); + listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, ignore_access_denied_multidirectory_globs, result); return result; } @@ -439,7 +439,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user else { /// We list only non-directory files. - paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read, context->getSettingsRef().ignore_eacces_multidirectory_globs); + paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read, context->getSettingsRef().ignore_access_denied_multidirectory_globs); can_be_directory = false; } diff --git a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh index 984ae47d222..b0918d4641d 100755 --- a/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh +++ b/tests/queries/0_stateless/02771_multidirectory_globs_storage_file.sh @@ -33,6 +33,6 @@ ${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CL # Add a directory to test against permission_denied rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/dir_inaccessible/ && mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ && chmod 000 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir_inaccessible/ -${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_eacces_multidirectory_globs = 1;" +${CLICKHOUSE_CLIENT} --query "SELECT *, _file FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV) SETTINGS ignore_access_denied_multidirectory_globs = 1;" rm -rf ${user_files_path:?}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} From bf3dc24471bfd54d192aafac15999d50d26b4a3c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 7 Sep 2023 11:23:42 +0200 Subject: [PATCH 107/327] Update mergetree.md --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index e22497a29b6..8c4a3682165 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -1220,7 +1220,6 @@ Configuration markup: account pass123 /var/lib/clickhouse/disks/blob_storage_disk/ - true /var/lib/clickhouse/disks/blob_storage_disk/cache/ false @@ -1248,8 +1247,6 @@ Limit parameters (mainly for internal usage): Other parameters: * `metadata_path` - Path on local FS to store metadata files for Blob Storage. Default value is `/var/lib/clickhouse/disks//`. -* `cache_enabled` - Allows to cache mark and index files on local FS. Default value is `true`. -* `cache_path` - Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. * `skip_access_check` - If true, disk access checks will not be performed on disk start-up. Default value is `false`. Examples of working configurations can be found in integration tests directory (see e.g. [test_merge_tree_azure_blob_storage](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml) or [test_azure_blob_storage_zero_copy_replication](https://github.com/ClickHouse/ClickHouse/blob/master/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml)). From c6ef811b82e4c1ccc30c224ab285255399a7bfc7 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Sep 2023 11:05:06 +0000 Subject: [PATCH 108/327] Fix tests --- src/Storages/StorageS3.cpp | 8 ++------ src/TableFunctions/TableFunctionS3.cpp | 5 +++++ 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 5078d21141b..24b2279bfdc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1246,13 +1246,11 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); auto filename = collection.getOrDefault("filename", ""); - String url; if (!filename.empty()) - url = std::filesystem::path(collection.get("url")) / filename; + configuration.url = S3::URI(std::filesystem::path(collection.get("url")) / filename); else - url = collection.get("url"); + configuration.url = S3::URI(collection.get("url")); - configuration.url = S3::URI(url); configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); @@ -1260,8 +1258,6 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur configuration.auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); configuration.format = collection.getOrDefault("format", configuration.format); - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(url).getPath(), true); configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); configuration.structure = collection.getOrDefault("structure", "auto"); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 94ce29f1116..2e9527cff6c 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -58,6 +58,11 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) { StorageS3::processNamedCollectionResult(configuration, *named_collection); + if (configuration.format == "auto") + { + String file_path = named_collection->getOrDefault("filename", Poco::URI(named_collection->get("url")).getPath()); + configuration.format = FormatFactory::instance().getFormatFromFileName(file_path, true); + } } else { From b06412ae60fb50f8bfbf6635e6b575d49f11d96e Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Sep 2023 11:15:49 +0000 Subject: [PATCH 109/327] Add addDate function for compatibility with MySQL --- .../functions/date-time-functions.md | 39 ++++++++ src/Functions/addDate.cpp | 88 +++++++++++++++++++ ...new_functions_must_be_documented.reference | 1 + .../02834_add_date_function.reference | 4 + .../0_stateless/02834_add_date_function.sql | 9 ++ 5 files changed, 141 insertions(+) create mode 100644 src/Functions/addDate.cpp create mode 100644 tests/queries/0_stateless/02834_add_date_function.reference create mode 100644 tests/queries/0_stateless/02834_add_date_function.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 87d84425029..3940aef8fbf 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -987,6 +987,10 @@ Result: └───────────────────────────────────────────────┘ ``` +**See Also** + +- [addDate](#addDate) + ## date\_sub Subtracts the time interval or date interval from the provided date or date with time. @@ -1134,6 +1138,41 @@ Result: └──────────────────────────────────────────────────────────────┘ ``` +## addDate + +Adds the time interval or date interval to the provided date or date with time. + +**Syntax** + +``` sql +addDate(date, interval) +``` + +**Arguments** + +- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +- `interval` — Interval to add. [Interval](../../sql-reference/data-types/special-data-types/interval.md). + +**Returned value** + +Date or date with time obtained by adding `value`, expressed in `unit`, to `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + +**Example** + +```sql +SELECT addDate(toDate('2018-01-01'), INTERVAL 3 YEAR); +``` + +Result: + +```text +┌─addDate(toDate('2018-01-01'), toIntervalYear(3))─┐ +│ 2021-01-01 │ +└──────────────────────────────────────────────────┘ +``` + ## now Returns the current date and time at the moment of query analysis. The function is a constant expression. diff --git a/src/Functions/addDate.cpp b/src/Functions/addDate.cpp new file mode 100644 index 00000000000..a34ded341ac --- /dev/null +++ b/src/Functions/addDate.cpp @@ -0,0 +1,88 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +class FunctionAddDate : public IFunction +{ +public: + static constexpr auto name = "addDate"; + + explicit FunctionAddDate(ContextPtr context_) : context(context_) {} + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + String getName() const override { return name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), + getName()); + + if (!isInterval(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}. Should be an interval", + arguments[0].type->getName(), + getName()); + + auto plus = FunctionFactory::instance().get("plus", context); + auto plus_build = plus->build(arguments); + + return plus_build->getResultType(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), + getName()); + + if (!isInterval(arguments[1].type)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1st argument of function {}. Should be an interval", + arguments[0].type->getName(), + getName()); + + auto plus = FunctionFactory::instance().get("plus", context); + auto plus_build = plus->build(arguments); + + auto res_type = plus_build->getResultType(); + return plus_build->execute(arguments, res_type, input_rows_count); + } + +private: + ContextPtr context; +}; + +} + + +REGISTER_FUNCTION(AddInterval) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..1a7ffdb1fa1 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -68,6 +68,7 @@ accurateCastOrDefault accurateCastOrNull acos acosh +addDate addDays addHours addMicroseconds diff --git a/tests/queries/0_stateless/02834_add_date_function.reference b/tests/queries/0_stateless/02834_add_date_function.reference new file mode 100644 index 00000000000..8912c8c8f63 --- /dev/null +++ b/tests/queries/0_stateless/02834_add_date_function.reference @@ -0,0 +1,4 @@ +2022-05-07 00:05:00 +2022-05-07 00:05:00.000 +2022-05-07 00:05:00 +2022-05-07 00:05:00.000 diff --git a/tests/queries/0_stateless/02834_add_date_function.sql b/tests/queries/0_stateless/02834_add_date_function.sql new file mode 100644 index 00000000000..1bf4cb0976e --- /dev/null +++ b/tests/queries/0_stateless/02834_add_date_function.sql @@ -0,0 +1,9 @@ +SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::DateTime, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::DateTime64, INTERVAL 5 MINUTE); + +SELECT addDate('2022-05-07'::Date); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE, 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT addDate('2022-05-07'::Date, 10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT addDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 9a3ea9d6dc7ce9c7f1217e0f15ade11e2953f44a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 7 Sep 2023 11:31:09 +0000 Subject: [PATCH 110/327] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..aa130681f50 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -982,6 +982,7 @@ acos acosh activecube activerecord +addDate addDays addHours addMinutes From 7d86d8b559739cb3d12d2dbd1ba213155cdfe7d8 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 7 Sep 2023 09:22:54 -0300 Subject: [PATCH 111/327] trigger ci From a0317e461a1f227895de22933b040cc4cd45f28c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Sep 2023 12:41:36 +0000 Subject: [PATCH 112/327] Better description --- tests/ci/fast_test_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 96eb144da2c..052aafb3395 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -145,8 +145,8 @@ def main(): logs_path.mkdir(parents=True, exist_ok=True) run_log_path = logs_path / "run.log" - timeout = 65 timeout_expired = False + timeout = 90 * 60 with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: @@ -186,7 +186,7 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) if timeout_expired is not None: - test_result_name = "Timeout for fast test check is expired" + test_result_name = "Check timeout expired" test_results.append(TestResult(test_result_name, "FAIL", timeout)) state = "failure" description = format_description(test_result_name) From c354fcaff7e5a53e2c14265aac595d6ad9f065dd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Sep 2023 12:56:22 +0000 Subject: [PATCH 113/327] Timeout report for stress_check --- tests/ci/fast_test_check.py | 4 ++-- tests/ci/stress_check.py | 20 ++++++++++++++++---- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 052aafb3395..3d6d43a2d54 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: - logging.info(f"Timeout expired for process execution: {run_cmd}") + logging.info(f"Timeout expired for command: {run_cmd}") timeout_expired = True elif retcode == 0: logging.info("Run successfully") @@ -185,7 +185,7 @@ def main(): else: state, description, test_results, additional_logs = process_results(output_path) - if timeout_expired is not None: + if timeout_expired: test_result_name = "Check timeout expired" test_results.append(TestResult(test_result_name, "FAIL", timeout)) state = "failure" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 7bac4e1d511..933bdc2ba60 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -16,12 +16,12 @@ from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, ) -from commit_status_helper import RerunHelper, get_commit, post_commit_status +from commit_status_helper import RerunHelper, get_commit, post_commit_status, format_description from docker_pull_helper import DockerImage, get_image_with_version from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from get_robot_token import get_best_robot_token from pr_info import PRInfo -from report import TestResults, read_test_results +from report import TestResult, TestResults, read_test_results from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen @@ -169,9 +169,14 @@ def run_stress_test(docker_image_name): ) logging.info("Going to run stress test: %s", run_command) - with TeePopen(run_command, run_log_path, timeout=60 * 150) as process: + timeout_expired = False + timeout = 60 * 150 + with TeePopen(run_command, run_log_path, timeout=timeout) as process: retcode = process.wait() - if retcode == 0: + if process.timeout_exceeded: + logging.info(f"Timeout expired for command: {run_command}") + timeout_expired = True + elif retcode == 0: logging.info("Run successfully") else: logging.info("Run failed") @@ -183,6 +188,13 @@ def run_stress_test(docker_image_name): state, description, test_results, additional_logs = process_results( result_path, server_log_path, run_log_path ) + + if timeout_expired: + test_result_name = "Check timeout expired" + test_results.append(TestResult(test_result_name, "FAIL", timeout)) + state = "failure" + description = format_description(test_result_name) + ch_helper = ClickHouseHelper() report_url = upload_results( From 27455a5882d95b474dbae8311c75d62449ec1990 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 7 Sep 2023 12:57:35 +0000 Subject: [PATCH 114/327] Test timeout to fail --- tests/ci/stress_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 933bdc2ba60..9f832192886 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -170,7 +170,7 @@ def run_stress_test(docker_image_name): logging.info("Going to run stress test: %s", run_command) timeout_expired = False - timeout = 60 * 150 + timeout = 60 * 1 with TeePopen(run_command, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: From a5b508d6c5d6a87c6bd77c6af4b09c16514270bc Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Thu, 7 Sep 2023 14:59:33 +0200 Subject: [PATCH 115/327] Fix json in header --- docker/test/stateless/Dockerfile | 1 + src/IO/Progress.cpp | 1 + .../0_stateless/02869_http_headers_elapsed_ns.sh | 12 ++++++------ 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 35a6e9c365b..d96531e84c2 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -40,6 +40,7 @@ RUN apt-get update -y \ cargo \ zstd \ file \ + jq \ pv \ zip \ p7zip-full \ diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index 56fb7fe317e..1dcb206496f 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -92,6 +92,7 @@ void ProgressValues::writeJSON(WriteBuffer & out, bool add_braces) const writeText(result_bytes, out); writeCString("\",\"elapsed_ns\":\"", out); writeText(elapsed_ns, out); + writeCString("\"", out); if (add_braces) writeCString("}", out); } diff --git a/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh index 107d8a10345..df6302bb82a 100755 --- a/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh +++ b/tests/queries/0_stateless/02869_http_headers_elapsed_ns.sh @@ -8,17 +8,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURL_OUTPUT=$(echo 'SELECT number FROM numbers(10)' | \ ${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" --data-binary @- 2>&1) -ELAPSED_NS_PROGRESS=$(echo "${CURL_OUTPUT}" | \ +ELAPSED_NS_PROGRESS="$(echo "${CURL_OUTPUT}" | \ grep 'X-ClickHouse-Progress' | \ awk '{print $3}' | \ - sed -E 's/.*"elapsed_ns":"?([^,"]*)"?.*/\1/' - ) + jq -cM '.elapsed_ns | tonumber' + )" -ELAPSED_NS_SUMMARY=$(echo "${CURL_OUTPUT}" | \ +ELAPSED_NS_SUMMARY="$(echo "${CURL_OUTPUT}" | \ grep 'X-ClickHouse-Summary' | \ awk '{print $3}' | \ - sed -E 's/.*"elapsed_ns":"?([^,"]*)"?.*/\1/' - ) + jq -cM '.elapsed_ns | tonumber' + )" ALL_ARE_NON_ZERO=1 From c680c951d3d371342a37d4607b1876238d3382dc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 7 Sep 2023 13:12:53 +0000 Subject: [PATCH 116/327] Automatic style fix --- tests/ci/stress_check.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 9f832192886..9e607f7da82 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -16,7 +16,12 @@ from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, ) -from commit_status_helper import RerunHelper, get_commit, post_commit_status, format_description +from commit_status_helper import ( + RerunHelper, + get_commit, + post_commit_status, + format_description, +) from docker_pull_helper import DockerImage, get_image_with_version from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from get_robot_token import get_best_robot_token From c9635ddacd6e06191b6798b91e00c051b8b645f2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 7 Sep 2023 15:48:49 +0200 Subject: [PATCH 117/327] Update src/IO/ISchedulerNode.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/IO/ISchedulerNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ISchedulerNode.h b/src/IO/ISchedulerNode.h index 644cc708bca..5cf1ae94216 100644 --- a/src/IO/ISchedulerNode.h +++ b/src/IO/ISchedulerNode.h @@ -3,7 +3,7 @@ #include #include #include -#include "base/types.h" +#include #include #include From 958f102daae2a8e33d59529918c7126c1b08afa2 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 7 Sep 2023 15:58:31 +0200 Subject: [PATCH 118/327] Update src/IO/IResourceManager.h Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/IO/IResourceManager.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/IO/IResourceManager.h b/src/IO/IResourceManager.h index 7a078c565d8..b4270cd1935 100644 --- a/src/IO/IResourceManager.h +++ b/src/IO/IResourceManager.h @@ -48,7 +48,8 @@ public: virtual ClassifierPtr acquire(const String & classifier_name) = 0; /// For introspection, see `system.scheduler` table - virtual void forEachNode(std::function visitor) = 0; + using VisitorFunc = std::function; + void forEachNode(VisitorFunc visitor) override; }; using ResourceManagerPtr = std::shared_ptr; From a94ae9a7abcf7379145d93fc9465264c2aaae5f4 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 7 Sep 2023 14:02:58 +0000 Subject: [PATCH 119/327] review fixes --- docs/en/operations/workload-scheduling.md | 14 +++++++------- src/IO/Resource/ClassifiersConfig.cpp | 4 ++-- src/IO/Resource/ClassifiersConfig.h | 4 ++-- src/IO/Resource/DynamicResourceManager.cpp | 2 +- src/IO/Resource/DynamicResourceManager.h | 2 +- src/IO/Resource/SemaphoreConstraint.h | 2 +- src/IO/Resource/StaticResourceManager.h | 2 +- .../tests/gtest_resource_manager_hierarchical.cpp | 8 ++++---- .../tests/gtest_resource_manager_static.cpp | 8 ++++---- .../test_scheduler/configs/scheduler.xml | 4 ++-- 10 files changed, 25 insertions(+), 25 deletions(-) diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index c6529d5b81f..6ed6ced41b4 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -78,10 +78,10 @@ graph TD ``` **Possible node types:** -* inflight_limit (constraint) - blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost`; must have a single child. -* fair (policy) - selects the next request to serve from one of its children nodes according to max-min fairness; children nodes can specify `weight` (default is 1). -* priority (policy) - selects the next request to serve from one of its children nodes according to static priorities (lower value means higher priority); children nodes can specify `priority` (default is 0). -* fifo (queue) - leaf of the hierarchy capable of holding requests that exceed resource capacity. +* `inflight_limit` (constraint) - blocks if either number of concurrent in-flight requests exceeds `max_requests`, or their total cost exceeds `max_cost`; must have a single child. +* `fair` (policy) - selects the next request to serve from one of its children nodes according to max-min fairness; children nodes can specify `weight` (default is 1). +* `priority` (policy) - selects the next request to serve from one of its children nodes according to static priorities (lower value means higher priority); children nodes can specify `priority` (default is 0). +* `fifo` (queue) - leaf of the hierarchy capable of holding requests that exceed resource capacity. The following example shows how to define IO scheduling hierarchies shown in the picture: @@ -124,14 +124,14 @@ The following example shows how to define IO scheduling hierarchies shown in the ``` -## Workload classifiers {#classifiers} +## Workload classifiers {#workload_classifiers} Workload classifiers are used to define mapping from `workload` specified by a query into leaf-queues that should be used for specific resources. At the moment, workload classification is simple: only static mapping is available. Example: ```xml - + /fair/prod /fair/prod @@ -144,7 +144,7 @@ Example: /fair/dev /fair/dev - + ``` diff --git a/src/IO/Resource/ClassifiersConfig.cpp b/src/IO/Resource/ClassifiersConfig.cpp index fcd4655e2e4..7dc4d517138 100644 --- a/src/IO/Resource/ClassifiersConfig.cpp +++ b/src/IO/Resource/ClassifiersConfig.cpp @@ -21,7 +21,7 @@ ClassifierDescription::ClassifierDescription(const Poco::Util::AbstractConfigura ClassifiersConfig::ClassifiersConfig(const Poco::Util::AbstractConfiguration & config) { Poco::Util::AbstractConfiguration::Keys keys; - const String config_prefix = "classifiers"; + const String config_prefix = "workload_classifiers"; config.keys(config_prefix, keys); for (const auto & key : keys) classifiers.emplace(std::piecewise_construct, @@ -34,7 +34,7 @@ const ClassifierDescription & ClassifiersConfig::get(const String & classifier_n if (auto it = classifiers.find(classifier_name); it != classifiers.end()) return it->second; else - throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown classifier '{}' to access resources", classifier_name); + throw Exception(ErrorCodes::RESOURCE_NOT_FOUND, "Unknown workload classifier '{}' to access resources", classifier_name); } } diff --git a/src/IO/Resource/ClassifiersConfig.h b/src/IO/Resource/ClassifiersConfig.h index 96e2bd0f0b9..186c49943ad 100644 --- a/src/IO/Resource/ClassifiersConfig.h +++ b/src/IO/Resource/ClassifiersConfig.h @@ -15,14 +15,14 @@ struct ClassifierDescription : std::unordered_map /* * Loads a config with the following format: - * + * * * /path/to/queue * /path/to/another/queue * * ... * ... - * + * */ class ClassifiersConfig { diff --git a/src/IO/Resource/DynamicResourceManager.cpp b/src/IO/Resource/DynamicResourceManager.cpp index 964f4f662e6..b9803d8079d 100644 --- a/src/IO/Resource/DynamicResourceManager.cpp +++ b/src/IO/Resource/DynamicResourceManager.cpp @@ -227,7 +227,7 @@ ClassifierPtr DynamicResourceManager::acquire(const String & classifier_name) return std::make_shared(state_ref, classifier_name); } -void DynamicResourceManager::forEachNode(std::function visitor) +void DynamicResourceManager::forEachNode(IResourceManager::VisitorFunc visitor) { // Acquire a reference to the current state StatePtr state_ref; diff --git a/src/IO/Resource/DynamicResourceManager.h b/src/IO/Resource/DynamicResourceManager.h index 10d784503ca..3372d40a285 100644 --- a/src/IO/Resource/DynamicResourceManager.h +++ b/src/IO/Resource/DynamicResourceManager.h @@ -30,7 +30,7 @@ public: DynamicResourceManager(); void updateConfiguration(const Poco::Util::AbstractConfiguration & config) override; ClassifierPtr acquire(const String & classifier_name) override; - void forEachNode(std::function visitor) override; + void forEachNode(VisitorFunc visitor) override; private: /// Holds everything required to work with one specific configuration diff --git a/src/IO/Resource/SemaphoreConstraint.h b/src/IO/Resource/SemaphoreConstraint.h index 9160d0431de..9c6ce43d6ea 100644 --- a/src/IO/Resource/SemaphoreConstraint.h +++ b/src/IO/Resource/SemaphoreConstraint.h @@ -121,7 +121,7 @@ public: size_t activeChildren() override { std::unique_lock lock(mutex); - return child_active ? 1 : 0; + return child_active; } bool isSatisfied() override diff --git a/src/IO/Resource/StaticResourceManager.h b/src/IO/Resource/StaticResourceManager.h index b8712b22b73..5ec6a35750b 100644 --- a/src/IO/Resource/StaticResourceManager.h +++ b/src/IO/Resource/StaticResourceManager.h @@ -22,7 +22,7 @@ public: ClassifierPtr acquire(const String & classifier_name) override; - void forEachNode(std::function visitor) override + void forEachNode(VisitorFunc visitor) override { UNUSED(visitor); } diff --git a/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp b/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp index 43773559f03..949a1ee0264 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_hierarchical.cpp @@ -24,10 +24,10 @@ TEST(IOResourceDynamicResourceManager, Smoke) fifo3 - + /fair/A /fair/B - + )CONFIG"); @@ -71,11 +71,11 @@ TEST(IOResourceDynamicResourceManager, Fairness) fifo - + /fair/A /fair/B /fair/leader - + )CONFIG"); diff --git a/src/IO/Resource/tests/gtest_resource_manager_static.cpp b/src/IO/Resource/tests/gtest_resource_manager_static.cpp index 976eac41a49..9c5e86e9ffc 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_static.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_static.cpp @@ -24,10 +24,10 @@ TEST(IOResourceStaticResourceManager, Smoke) 1 - + /prio/A /prio/B - + )CONFIG"); @@ -70,13 +70,13 @@ TEST(IOResourceStaticResourceManager, Prioritization) - + /prio/A /prio/B /prio/C /prio/D /prio/leader - + )CONFIG"); diff --git a/tests/integration/test_scheduler/configs/scheduler.xml b/tests/integration/test_scheduler/configs/scheduler.xml index fe01cb8d662..523ba1a5a98 100644 --- a/tests/integration/test_scheduler/configs/scheduler.xml +++ b/tests/integration/test_scheduler/configs/scheduler.xml @@ -41,7 +41,7 @@ fifo1 - + /prio/admin /prio/admin @@ -58,5 +58,5 @@ /prio/fair/dev /prio/fair/dev - + From 59844b0a9d5e00bdd8774fa423ff4cb01a6372ea Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Mon, 4 Sep 2023 19:50:46 -0700 Subject: [PATCH 120/327] Add SHOW FUNCTIONS support to client Accepts CH client queries in the form of SHOW FUNCTIONS [LIKE | ILIKE '< pattern>'] Retrieves all columns from `functions` table in system database. Fixes #52757 --- docs/en/sql-reference/statements/show.md | 13 ++++++ src/Interpreters/InterpreterFactory.cpp | 6 +++ .../InterpreterShowFunctionsQuery.cpp | 46 +++++++++++++++++++ .../InterpreterShowFunctionsQuery.h | 27 +++++++++++ src/Parsers/ASTShowFunctionsQuery.cpp | 25 ++++++++++ src/Parsers/ASTShowFunctionsQuery.h | 23 ++++++++++ src/Parsers/ParserQueryWithOutput.cpp | 3 ++ src/Parsers/ParserShowFunctionsQuery.cpp | 35 ++++++++++++++ src/Parsers/ParserShowFunctionsQuery.h | 19 ++++++++ .../0_stateless/00419_show_sql_queries.sh | 1 + .../02875_show_functions.reference | 0 .../0_stateless/02875_show_functions.sh | 14 ++++++ 12 files changed, 212 insertions(+) create mode 100644 src/Interpreters/InterpreterShowFunctionsQuery.cpp create mode 100644 src/Interpreters/InterpreterShowFunctionsQuery.h create mode 100644 src/Parsers/ASTShowFunctionsQuery.cpp create mode 100644 src/Parsers/ASTShowFunctionsQuery.h create mode 100644 src/Parsers/ParserShowFunctionsQuery.cpp create mode 100644 src/Parsers/ParserShowFunctionsQuery.h create mode 100644 tests/queries/0_stateless/02875_show_functions.reference create mode 100755 tests/queries/0_stateless/02875_show_functions.sh diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 1c399d2072b..e94718394de 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -638,3 +638,16 @@ Outputs the content of the [system.table_engines](../../operations/system-tables **See Also** - [system.table_engines](../../operations/system-tables/table_engines.md) table + +## SHOW FUNCTIONS + +``` sql +SHOW FUNCTIONS [LIKE | ILIKE ''] +``` + +Outputs the content of the [system.functions](../../operations/system-tables/functions.md) table. + +If either `LIKE` or `ILIKE` clause is specified, the query returns a list of system functions whose names match the provided ``. + +**See Also** +- [system.functions](../../operations/system-tables/functions.md) table diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 27c94119750..de3a3d68d39 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -80,6 +81,7 @@ #include #include #include +#include #include #include #include @@ -203,6 +205,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp new file mode 100644 index 00000000000..efadb929451 --- /dev/null +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -0,0 +1,46 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +InterpreterShowFunctionsQuery::InterpreterShowFunctionsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) + : WithMutableContext(context_), query_ptr(query_ptr_) +{ +} + +BlockIO InterpreterShowFunctionsQuery::execute() +{ + return executeQuery(getRewrittenQuery(), getContext(), true); +} + +String InterpreterShowFunctionsQuery::getRewrittenQuery() +{ + constexpr const char * functions_table = "functions"; + + const auto & query = query_ptr->as(); + + DatabasePtr systemDb = DatabaseCatalog::instance().getSystemDatabase(); + + String rewritten_query = fmt::format( + R"( +SELECT * +FROM {}.{})", + systemDb->getDatabaseName(), + functions_table); + + if (!query.like.empty()) + { + rewritten_query += " WHERE name "; + rewritten_query += query.case_insensitive_like ? "ILIKE " : "LIKE "; + rewritten_query += fmt::format("'{}'", query.like); + } + + return rewritten_query; +} + +} diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.h b/src/Interpreters/InterpreterShowFunctionsQuery.h new file mode 100644 index 00000000000..f15e1ae67b2 --- /dev/null +++ b/src/Interpreters/InterpreterShowFunctionsQuery.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Context; + +class InterpreterShowFunctionsQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterShowFunctionsQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_); + + BlockIO execute() override; + + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; + + String getRewrittenQuery(); +}; + +} diff --git a/src/Parsers/ASTShowFunctionsQuery.cpp b/src/Parsers/ASTShowFunctionsQuery.cpp new file mode 100644 index 00000000000..9253dcf5cb2 --- /dev/null +++ b/src/Parsers/ASTShowFunctionsQuery.cpp @@ -0,0 +1,25 @@ +#include + +#include + +namespace DB +{ + +ASTPtr ASTShowFunctionsQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + cloneOutputOptions(*res); + return res; +} + +void ASTShowFunctionsQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW FUNCTIONS" << (settings.hilite ? hilite_none : ""); + + if (!like.empty()) + settings.ostr << (settings.hilite ? hilite_keyword : "") << (case_insensitive_like ? " ILIKE " : " LIKE ") + << (settings.hilite ? hilite_none : "") << DB::quote << like; +} + +} diff --git a/src/Parsers/ASTShowFunctionsQuery.h b/src/Parsers/ASTShowFunctionsQuery.h new file mode 100644 index 00000000000..6993f939888 --- /dev/null +++ b/src/Parsers/ASTShowFunctionsQuery.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ASTShowFunctionsQuery : public ASTQueryWithOutput +{ +public: + bool case_insensitive_like = false; + String like; + + String getID(char) const override { return "ShowFunctions"; } + ASTPtr clone() const override; + QueryKind getQueryKind() const override { return QueryKind::Show; } + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index a2391495071..9a71bc222b5 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +41,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowTablesQuery show_tables_p; ParserShowColumnsQuery show_columns_p; ParserShowEnginesQuery show_engine_p; + ParserShowFunctionsQuery show_functions_p; ParserShowIndexesQuery show_indexes_p; ParserSelectWithUnionQuery select_p; ParserTablePropertiesQuery table_p; @@ -71,6 +73,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || show_tables_p.parse(pos, query, expected) || show_columns_p.parse(pos, query, expected) || show_engine_p.parse(pos, query, expected) + || show_functions_p.parse(pos, query, expected) || show_indexes_p.parse(pos, query, expected) || table_p.parse(pos, query, expected) || describe_cache_p.parse(pos, query, expected) diff --git a/src/Parsers/ParserShowFunctionsQuery.cpp b/src/Parsers/ParserShowFunctionsQuery.cpp new file mode 100644 index 00000000000..524d936c4f1 --- /dev/null +++ b/src/Parsers/ParserShowFunctionsQuery.cpp @@ -0,0 +1,35 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +bool ParserShowFunctionsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr like; + + auto query = std::make_shared(); + if (!ParserKeyword("SHOW FUNCTIONS").ignore(pos, expected)) + return false; + + if (bool insensitive = ParserKeyword("ILIKE").ignore(pos, expected); insensitive || ParserKeyword("LIKE").ignore(pos, expected)) + { + if (insensitive) + query->case_insensitive_like = true; + + if (!ParserStringLiteral().parse(pos, like, expected)) + return false; + } + + if (like) + query->like = like->as().value.safeGet(); + node = query; + + return true; +} + +} diff --git a/src/Parsers/ParserShowFunctionsQuery.h b/src/Parsers/ParserShowFunctionsQuery.h new file mode 100644 index 00000000000..25241d180db --- /dev/null +++ b/src/Parsers/ParserShowFunctionsQuery.h @@ -0,0 +1,19 @@ +#pragma once + +#include + +namespace DB +{ + +/** Parses queries of the form + * SHOW FUNCTIONS [LIKE | ILIKE ''] + */ +class ParserShowFunctionsQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW FUNCTIONS query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/tests/queries/0_stateless/00419_show_sql_queries.sh b/tests/queries/0_stateless/00419_show_sql_queries.sh index 607703b385a..99252eeb1ba 100755 --- a/tests/queries/0_stateless/00419_show_sql_queries.sh +++ b/tests/queries/0_stateless/00419_show_sql_queries.sh @@ -8,3 +8,4 @@ $CLICKHOUSE_CLIENT -q "SHOW PROCESSLIST" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW DATABASES" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW TABLES" &>/dev/null $CLICKHOUSE_CLIENT -q "SHOW ENGINES" &>/dev/null +$CLICKHOUSE_CLIENT -q "SHOW FUNCTIONS" &>/dev/null diff --git a/tests/queries/0_stateless/02875_show_functions.reference b/tests/queries/0_stateless/02875_show_functions.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02875_show_functions.sh b/tests/queries/0_stateless/02875_show_functions.sh new file mode 100755 index 00000000000..6f8da63ca9e --- /dev/null +++ b/tests/queries/0_stateless/02875_show_functions.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +diff -q <($CLICKHOUSE_CLIENT -q "SELECT * from system.functions") \ + <($CLICKHOUSE_CLIENT -q "SHOW FUNCTIONS") + +diff -q <($CLICKHOUSE_CLIENT -q "SELECT * FROM system.functions WHERE name ILIKE 'quantile%'") \ + <($CLICKHOUSE_CLIENT -q "SHOW FUNCTIONS ILIKE 'quantile%'") + +diff -q <($CLICKHOUSE_CLIENT -q "SELECT * FROM system.functions WHERE name LIKE 'median%'") \ + <($CLICKHOUSE_CLIENT -q "SHOW FUNCTIONS LIKE 'median%'") From 411a97ff36f9855e4c3f3e85403a9a140ee2ca33 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Sep 2023 21:15:30 +0000 Subject: [PATCH 121/327] reproduce a bug in removeSharedRecursive --- .../__init__.py | 0 .../configs/config.d/storage_conf.xml | 43 +++ .../configs/config.d/users.xml | 7 + .../test.py | 334 ++++++++++++++++++ 4 files changed, 384 insertions(+) create mode 100644 tests/integration/test_replicated_zero_copy_projection_mutation/__init__.py create mode 100644 tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml create mode 100644 tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/users.xml create mode 100644 tests/integration/test_replicated_zero_copy_projection_mutation/test.py diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/__init__.py b/tests/integration/test_replicated_zero_copy_projection_mutation/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml new file mode 100644 index 00000000000..08c5ccdde88 --- /dev/null +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml @@ -0,0 +1,43 @@ + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + true + + + s3 + http://resolver:8081/root/data/ + minio + minio123 + true + 0 + + + + + +
+ s3 +
+
+
+ + +
+ broken_s3 +
+
+
+
+
+ + + 1 + + +
diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/users.xml b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/users.xml new file mode 100644 index 00000000000..246de9ecb96 --- /dev/null +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/users.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/test.py b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py new file mode 100644 index 00000000000..10c1ce3f89a --- /dev/null +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py @@ -0,0 +1,334 @@ +import logging +import time +from contextlib import contextmanager +import pathlib + +import pytest + +from helpers.mock_servers import start_s3_mock +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + + +def args_to_dict(**kwargs): + return kwargs + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + + kwargs = args_to_dict( + main_configs=[ + "configs/config.d/storage_conf.xml", + ], + user_configs=[ + "configs/config.d/users.xml", + ], + with_minio=True, + with_zookeeper=True, + stay_alive=True, + ) + + cluster.add_instance("node1", **kwargs) + cluster.add_instance("node2", **kwargs) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(scope="module") +def all_cluster_nodes(cluster): + yield cluster.instances.values() + + +@pytest.fixture(scope="module") +def first_cluster_node(cluster): + yield cluster.instances["node1"] + + +@pytest.fixture(scope="module") +def second_cluster_node(cluster): + yield cluster.instances["node2"] + + +@pytest.fixture(scope="module") +def init_broken_s3(cluster): + yield start_s3_mock(cluster, "broken_s3", "8081") + + +@pytest.fixture(scope="function") +def broken_s3(init_broken_s3): + init_broken_s3.reset() + yield init_broken_s3 + + +def list_objects(cluster, path="data/", hint="list_objects"): + minio = cluster.minio_client + objects = list(minio.list_objects(cluster.minio_bucket, path, recursive=True)) + names = [x.object_name for x in objects] + names.sort() + logging.info(f"{hint} ({len(objects)}): {names}") + return names + + +def wait_for_delete_s3_objects(cluster, expected, timeout=30): + while timeout > 0: + if len(list_objects(cluster, "data/")) == expected: + return + timeout -= 1 + time.sleep(1) + final_listing = list_objects(cluster, "data/") + assert len(final_listing) == expected, ",".join(final_listing) + + +def remove_all_s3_objects(cluster): + minio = cluster.minio_client + for obj in list_objects(cluster, "data/"): + minio.remove_object(cluster.minio_bucket, obj) + + +@pytest.fixture(autouse=True, scope="function") +def clear_minio(cluster): + try: + # CH do some writes to the S3 at start. For example, file data/clickhouse_access_check_{server_uuid}. + # Set the timeout there as 10 sec in order to resolve the race with that file exists. + wait_for_delete_s3_objects(cluster, 0, timeout=10) + except: + # Remove extra objects to prevent tests cascade failing + remove_all_s3_objects(cluster) + + yield + + +@contextmanager +def drop_table_guard(nodes, table): + for node in nodes: + node.query(f"DROP TABLE IF EXISTS {table} SYNC") + try: + yield + finally: + for node in nodes: + node.query(f"DROP TABLE IF EXISTS {table} SYNC") + + +def test_all_projection_files_are_dropped_when_part_is_dropped(cluster, first_cluster_node): + node = first_cluster_node + + with drop_table_guard([node], "test_all_projection_files_are_dropped"): + node.query( + """ + CREATE TABLE test_all_projection_files_are_dropped(a UInt32, b UInt32) + ENGINE MergeTree() + ORDER BY a + SETTINGS storage_policy='s3', old_parts_lifetime=0 + """ + ) + + objects_empty_table = list_objects(cluster) + + node.query( + "ALTER TABLE test_all_projection_files_are_dropped ADD projection b_order (SELECT a, b ORDER BY b)") + node.query( + "ALTER TABLE test_all_projection_files_are_dropped MATERIALIZE projection b_order") + + node.query( + """ + INSERT INTO test_all_projection_files_are_dropped + VALUES (1, 105), (5, 101), (3, 103), (4, 102), (2, 104) + """ + ) + + node.query( + "ALTER TABLE test_all_projection_files_are_dropped DROP PARTITION ID 'all'" + ) + + objects_at_the_end = list_objects(cluster) + assert objects_at_the_end == objects_empty_table + + +def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, first_cluster_node, second_cluster_node): + with drop_table_guard(all_cluster_nodes, "test_hardlinks_preserved_when_projection_dropped"): + create_query = ( + """ + CREATE TABLE test_hardlinks_preserved_when_projection_dropped + ( + a UInt32, + b UInt32, + c UInt32, + PROJECTION projection_order_by_b + ( + SELECT a, b ORDER BY b + ) + ) + ENGINE ReplicatedMergeTree('/clickhouse/tables/test_projection', '{instance}') + ORDER BY a + """ + ) + + first_node_settings = ( + """ + SETTINGS + storage_policy='s3', + old_parts_lifetime=0 + """ + ) + + # big old_parts_lifetime value makes second node to hold outdated part for us, we make it as broken_on_start + second_node_settings = ( + """ + SETTINGS + storage_policy='s3', + old_parts_lifetime=10000 + """ + ) + + first_cluster_node.query(create_query + first_node_settings) + second_cluster_node.query(create_query + second_node_settings) + + first_cluster_node.query("SYSTEM FLUSH LOGS") + table_uuid = first_cluster_node.query( + """ + SELECT uuid FROM system.tables + WHERE name = 'test_hardlinks_preserved_when_projection_dropped' + """ + ).strip() + + first_cluster_node.query( + """ + INSERT INTO test_hardlinks_preserved_when_projection_dropped + VALUES (1, 105, 1), (5, 101, 1), (3, 103, 1), (4, 102, 1), (2, 104, 1) + """ + ) + + # second_cluster_node will fetch the mutated part when it is ready on first_cluster_node + second_cluster_node.query("SYSTEM STOP MERGES") + + first_cluster_node.query( + """ + ALTER TABLE test_hardlinks_preserved_when_projection_dropped + UPDATE c = 2 where c = 1 + """, + settings={"mutations_sync": "1"} + ) + + assert_eq_with_retry(first_cluster_node, "SELECT COUNT() FROM system.replication_queue", "0") + + # the mutated part is ready on first_cluster_node, second replica just fetches it + second_cluster_node.query("SYSTEM START MERGES") + + data = first_cluster_node.query( + """ + SELECT * FROM system.parts + WHERE name = 'all_0_0_0' + AND table = 'test_hardlinks_preserved_when_projection_dropped' + AND not active + FORMAT Vertical + """ + ) + + # fist node removed outdated part + assert_eq_with_retry( + first_cluster_node, + """ + SELECT removal_state FROM system.parts + WHERE name = 'all_0_0_0' AND table = 'test_hardlinks_preserved_when_projection_dropped' AND not active + """, + "", + retry_count=300, + sleep_time=1 + ) + + hardlinks = first_cluster_node.query( + f""" + SELECT value + FROM system.zookeeper + WHERE + path like '/clickhouse/zero_copy/zero_copy_s3/{table_uuid}' AND name = 'all_0_0_0' + """, + settings={"allow_unrestricted_reads_from_keeper": "1"} + ).strip().split() + assert len(hardlinks) > 0, ",".join(hardlinks) + assert any(["proj/" in x for x in hardlinks]), ",".join(hardlinks) + + logging.info("hardlinks has been planted") + + part_path_on_second_node = second_cluster_node.query( + """ + SELECT path FROM system.parts + WHERE + name = 'all_0_0_0' AND table = 'test_hardlinks_preserved_when_projection_dropped' + """ + ).strip() + + # that corrupts outdatated part all_0_0_0 + script = ( + f"INDEX_FILE={part_path_on_second_node}/primary.cidx" + """ + cp $INDEX_FILE $INDEX_FILE.backup + echo "unexpected data in metadata file" | cat > $INDEX_FILE + """ + ) + second_cluster_node.exec_in_container(["bash", "-c", script]) + + # corrupted outdatated part all_0_0_0 is detached as broken_on_start + second_cluster_node.restart_clickhouse() + + second_cluster_node.query("SYSTEM WAIT LOADING PARTS test_hardlinks_preserved_when_projection_dropped") + + second_cluster_node.query("SYSTEM FLUSH LOGS") + + broken_parts = second_cluster_node.query( + """ + SELECT name, reason, path FROM system.detached_parts + WHERE + table = 'test_hardlinks_preserved_when_projection_dropped' + """ + ).strip().split('\n') + + assert len(broken_parts) == 1, broken_parts + broken_part_name, reason, broken_part_path_on_second_node = broken_parts[0].split('\t') + assert "broken-on-start" == reason + + script = ( + f"INDEX_FILE={broken_part_path_on_second_node}/primary.cidx" + """ + mv $INDEX_FILE.backup $INDEX_FILE + """ + ) + second_cluster_node.exec_in_container(["bash", "-c", script]) + + second_cluster_node.query( + f""" + ALTER TABLE test_hardlinks_preserved_when_projection_dropped + DROP DETACHED PART '{broken_part_name}' + """, + settings={"allow_drop_detached": "1"} + ) + + res = first_cluster_node.query( + """ + CHECK TABLE test_hardlinks_preserved_when_projection_dropped + """ + ).strip() + + # corrupted + assert res == "0" + + data = first_cluster_node.query( + """ + SELECT a, b FROM test_hardlinks_preserved_when_projection_dropped + WHERE b > 104 + ORDER BY b + """ + ).split("\n") + + assert len(data) == 5, data + + + + From 44e432deabdbb1eda096004bb71ea93750f62599 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 7 Sep 2023 23:47:35 +0200 Subject: [PATCH 122/327] style fix --- .../configs/config.d/storage_conf.xml | 15 --- .../test.py | 114 +++++++++--------- 2 files changed, 59 insertions(+), 70 deletions(-) diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml index 08c5ccdde88..44d043b944f 100644 --- a/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/configs/config.d/storage_conf.xml @@ -9,14 +9,6 @@ minio123 true - - s3 - http://resolver:8081/root/data/ - minio - minio123 - true - 0 -
@@ -26,13 +18,6 @@ - - -
- broken_s3 -
-
-
diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/test.py b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py index 10c1ce3f89a..3f594158a46 100644 --- a/tests/integration/test_replicated_zero_copy_projection_mutation/test.py +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py @@ -116,7 +116,9 @@ def drop_table_guard(nodes, table): node.query(f"DROP TABLE IF EXISTS {table} SYNC") -def test_all_projection_files_are_dropped_when_part_is_dropped(cluster, first_cluster_node): +def test_all_projection_files_are_dropped_when_part_is_dropped( + cluster, first_cluster_node +): node = first_cluster_node with drop_table_guard([node], "test_all_projection_files_are_dropped"): @@ -132,9 +134,11 @@ def test_all_projection_files_are_dropped_when_part_is_dropped(cluster, first_cl objects_empty_table = list_objects(cluster) node.query( - "ALTER TABLE test_all_projection_files_are_dropped ADD projection b_order (SELECT a, b ORDER BY b)") + "ALTER TABLE test_all_projection_files_are_dropped ADD projection b_order (SELECT a, b ORDER BY b)" + ) node.query( - "ALTER TABLE test_all_projection_files_are_dropped MATERIALIZE projection b_order") + "ALTER TABLE test_all_projection_files_are_dropped MATERIALIZE projection b_order" + ) node.query( """ @@ -151,10 +155,13 @@ def test_all_projection_files_are_dropped_when_part_is_dropped(cluster, first_cl assert objects_at_the_end == objects_empty_table -def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, first_cluster_node, second_cluster_node): - with drop_table_guard(all_cluster_nodes, "test_hardlinks_preserved_when_projection_dropped"): - create_query = ( - """ +def test_hardlinks_preserved_when_projection_dropped( + cluster, all_cluster_nodes, first_cluster_node, second_cluster_node +): + with drop_table_guard( + all_cluster_nodes, "test_hardlinks_preserved_when_projection_dropped" + ): + create_query = """ CREATE TABLE test_hardlinks_preserved_when_projection_dropped ( a UInt32, @@ -167,25 +174,20 @@ def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_projection', '{instance}') ORDER BY a - """ - ) + """ - first_node_settings = ( - """ + first_node_settings = """ SETTINGS storage_policy='s3', old_parts_lifetime=0 - """ - ) + """ # big old_parts_lifetime value makes second node to hold outdated part for us, we make it as broken_on_start - second_node_settings = ( - """ + second_node_settings = """ SETTINGS storage_policy='s3', old_parts_lifetime=10000 - """ - ) + """ first_cluster_node.query(create_query + first_node_settings) second_cluster_node.query(create_query + second_node_settings) @@ -213,45 +215,43 @@ def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, ALTER TABLE test_hardlinks_preserved_when_projection_dropped UPDATE c = 2 where c = 1 """, - settings={"mutations_sync": "1"} + settings={"mutations_sync": "1"}, ) - assert_eq_with_retry(first_cluster_node, "SELECT COUNT() FROM system.replication_queue", "0") + assert_eq_with_retry( + first_cluster_node, "SELECT COUNT() FROM system.replication_queue", "0" + ) # the mutated part is ready on first_cluster_node, second replica just fetches it second_cluster_node.query("SYSTEM START MERGES") - data = first_cluster_node.query( - """ - SELECT * FROM system.parts - WHERE name = 'all_0_0_0' - AND table = 'test_hardlinks_preserved_when_projection_dropped' - AND not active - FORMAT Vertical - """ - ) - # fist node removed outdated part assert_eq_with_retry( first_cluster_node, """ SELECT removal_state FROM system.parts - WHERE name = 'all_0_0_0' AND table = 'test_hardlinks_preserved_when_projection_dropped' AND not active + WHERE name = 'all_0_0_0' + AND table = 'test_hardlinks_preserved_when_projection_dropped' + AND not active """, "", retry_count=300, - sleep_time=1 + sleep_time=1, ) - hardlinks = first_cluster_node.query( - f""" - SELECT value - FROM system.zookeeper - WHERE - path like '/clickhouse/zero_copy/zero_copy_s3/{table_uuid}' AND name = 'all_0_0_0' - """, - settings={"allow_unrestricted_reads_from_keeper": "1"} - ).strip().split() + hardlinks = ( + first_cluster_node.query( + f""" + SELECT value + FROM system.zookeeper + WHERE + path like '/clickhouse/zero_copy/zero_copy_s3/{table_uuid}' AND name = 'all_0_0_0' + """, + settings={"allow_unrestricted_reads_from_keeper": "1"}, + ) + .strip() + .split() + ) assert len(hardlinks) > 0, ",".join(hardlinks) assert any(["proj/" in x for x in hardlinks]), ",".join(hardlinks) @@ -278,20 +278,28 @@ def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, # corrupted outdatated part all_0_0_0 is detached as broken_on_start second_cluster_node.restart_clickhouse() - second_cluster_node.query("SYSTEM WAIT LOADING PARTS test_hardlinks_preserved_when_projection_dropped") + second_cluster_node.query( + "SYSTEM WAIT LOADING PARTS test_hardlinks_preserved_when_projection_dropped" + ) second_cluster_node.query("SYSTEM FLUSH LOGS") - broken_parts = second_cluster_node.query( - """ - SELECT name, reason, path FROM system.detached_parts - WHERE - table = 'test_hardlinks_preserved_when_projection_dropped' - """ - ).strip().split('\n') - + broken_parts = ( + second_cluster_node.query( + """ + SELECT name, reason, path FROM system.detached_parts + WHERE + table = 'test_hardlinks_preserved_when_projection_dropped' + """ + ) + .strip() + .split("\n") + ) assert len(broken_parts) == 1, broken_parts - broken_part_name, reason, broken_part_path_on_second_node = broken_parts[0].split('\t') + # style checker black asked to do this. It is crazy + broken_part_name, reason, broken_part_path_on_second_node = broken_parts[ + 0 + ].split("\t") assert "broken-on-start" == reason script = ( @@ -307,7 +315,7 @@ def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, ALTER TABLE test_hardlinks_preserved_when_projection_dropped DROP DETACHED PART '{broken_part_name}' """, - settings={"allow_drop_detached": "1"} + settings={"allow_drop_detached": "1"}, ) res = first_cluster_node.query( @@ -328,7 +336,3 @@ def test_hardlinks_preserved_when_projection_dropped(cluster, all_cluster_nodes, ).split("\n") assert len(data) == 5, data - - - - From 926310728f109a474c17751b5ca0a88c1e25a1ed Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 7 Sep 2023 22:48:04 -0400 Subject: [PATCH 123/327] Fix issues --- programs/keeper-client/Commands.cpp | 22 ++++++++++++++-------- programs/keeper-client/Commands.h | 2 +- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 65bc719f1be..757f90fadcf 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -402,14 +402,20 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient String new_members; auto operation = query->args[0].get(); - if (operation == static_cast(ReconfigCommand::Operation::ADD)) - joining = query->args[1].safeGet(); - else if (operation == static_cast(ReconfigCommand::Operation::REMOVE)) - leaving = query->args[1].safeGet(); - else if (operation == static_cast(ReconfigCommand::Operation::SET)) - new_members = query->args[1].safeGet(); - else - UNREACHABLE(); + switch (operation) + { + case static_cast(ReconfigCommand::Operation::ADD): + joining = query->args[1].safeGet(); + break; + case static_cast(ReconfigCommand::Operation::REMOVE): + leaving = query->args[1].safeGet(); + break; + case static_cast(ReconfigCommand::Operation::SET): + new_members = query->args[1].safeGet(); + break; + default: + UNREACHABLE(); + } auto response = client->zookeeper->reconfig(joining, leaving, new_members); std::cout << response.value << '\n'; diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index aed98bfa0fb..f9d4292584b 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -190,7 +190,7 @@ class RMRCommand : public IKeeperClientCommand class ReconfigCommand : public IKeeperClientCommand { - enum class Operation : Int64 + enum class Operation : UInt8 { ADD = 0, REMOVE = 1, From 5c2fd38f7367b8112a4f086fd140e0a7869857d8 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Fri, 8 Sep 2023 15:05:04 +0800 Subject: [PATCH 124/327] extract checkUInt64ToIn64Conversion and convertUInt64ToInt64IfPossible to separate functions --- src/DataTypes/FieldToDataType.cpp | 94 +++++++++++-------------------- src/DataTypes/FieldToDataType.h | 10 ++++ 2 files changed, 44 insertions(+), 60 deletions(-) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index e25a2f546f2..837aae6753a 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -136,39 +136,17 @@ DataTypePtr FieldToDataType::operator() (const Array & x) const DataTypes element_types; element_types.reserve(x.size()); - auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, - const DataTypePtr & type, const Field & elem) - { - if (uint64_could_opposite) - { - has_signed_int |= WhichDataType(type).isNativeInt(); - - if (type->getTypeId() == TypeIndex::UInt64) - { - has_uint64 = true; - uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); - } - } - }; - bool has_signed_int = false; - bool has_uint64 = false; - bool uint64_could_opposite = true; + bool uint64_convert_possible = true; for (const Field & elem : x) { DataTypePtr type = applyVisitor(*this, elem); element_types.emplace_back(type); - checkIfConversionSigned(has_signed_int, has_uint64, uint64_could_opposite, type, elem); + checkUInt64ToIn64Conversion(has_signed_int, uint64_convert_possible, type, elem); } - // Convert the UInt64 type to Int64 in order to cover other signed_integer types - // and obtain the least super type of all ints. - if (has_signed_int && has_uint64 && uint64_could_opposite) - { - for (auto & type : element_types) - if (type->getTypeId() == TypeIndex::UInt64) - type = std::make_shared(); - } + if (has_signed_int && uint64_convert_possible) + convertUInt64ToInt64IfPossible(element_types); return std::make_shared(getLeastSupertype(element_types)); } @@ -196,53 +174,27 @@ DataTypePtr FieldToDataType::operator() (const Map & map) const key_types.reserve(map.size()); value_types.reserve(map.size()); - auto checkIfConversionSigned = [&](bool& has_signed_int, bool& has_uint64, bool& uint64_could_opposite, - const DataTypePtr & type, const Field & elem) - { - if (uint64_could_opposite) - { - has_signed_int |= WhichDataType(type).isNativeInt(); - - if (type->getTypeId() == TypeIndex::UInt64) - { - has_uint64 = true; - uint64_could_opposite &= (elem.template get() <= std::numeric_limits::max()); - } - } - }; - - auto updateUInt64Types = [](DataTypes types) - { - for (auto& type : types) - if (type->getTypeId() == TypeIndex::UInt64) - type = std::make_shared(); - }; - bool k_has_signed_int = false; - bool k_has_uint64 = false; - bool k_uint64_could_opposite = true; + bool k_uint64_convert_possible = true; bool v_has_signed_int = false; - bool v_has_uint64 = false; - bool v_uint64_could_opposite = true; + bool v_uint64_convert_possible = true; for (const auto & elem : map) { const auto & tuple = elem.safeGet(); assert(tuple.size() == 2); DataTypePtr k_type = applyVisitor(*this, tuple[0]); key_types.push_back(k_type); - checkIfConversionSigned(k_has_signed_int, k_has_uint64, k_uint64_could_opposite, k_type, tuple[0]); + checkUInt64ToIn64Conversion(k_has_signed_int, k_uint64_convert_possible, k_type, tuple[0]); DataTypePtr v_type = applyVisitor(*this, tuple[1]); value_types.push_back(v_type); - checkIfConversionSigned(v_has_signed_int, v_has_uint64, v_uint64_could_opposite, v_type, tuple[1]); + checkUInt64ToIn64Conversion(v_has_signed_int, v_uint64_convert_possible, v_type, tuple[1]); } - // Convert the UInt64 type to Int64 in order to cover other signed_integer types - // and obtain the least super type of all ints. - if (k_has_signed_int && k_has_uint64 && k_uint64_could_opposite) - updateUInt64Types(key_types); + if (k_has_signed_int && k_uint64_convert_possible) + convertUInt64ToInt64IfPossible(key_types); - if (v_has_signed_int && v_has_uint64 && v_uint64_could_opposite) - updateUInt64Types(value_types); + if (v_has_signed_int && v_uint64_convert_possible) + convertUInt64ToInt64IfPossible(value_types); return std::make_shared( getLeastSupertype(key_types), @@ -275,6 +227,28 @@ DataTypePtr FieldToDataType::operator()(const bool &) const return DataTypeFactory::instance().get("Bool"); } +template +void FieldToDataType::checkUInt64ToIn64Conversion(bool & has_signed_int, bool & uint64_convert_possible, const DataTypePtr & type, const Field & elem) const +{ + if (uint64_convert_possible) + { + bool is_native_int = WhichDataType(type).isNativeInt(); + + if (is_native_int) + has_signed_int |= is_native_int; + else if (type->getTypeId() == TypeIndex::UInt64) + uint64_convert_possible &= (elem.template get() <= std::numeric_limits::max()); + } +} + +template +void FieldToDataType::convertUInt64ToInt64IfPossible(DataTypes & data_types) const +{ + for (auto& type : data_types) + if (type->getTypeId() == TypeIndex::UInt64) + type = std::make_shared(); +} + template class FieldToDataType; template class FieldToDataType; template class FieldToDataType; diff --git a/src/DataTypes/FieldToDataType.h b/src/DataTypes/FieldToDataType.h index 8febadc1a0d..d1a3f11e8de 100644 --- a/src/DataTypes/FieldToDataType.h +++ b/src/DataTypes/FieldToDataType.h @@ -45,6 +45,16 @@ public: DataTypePtr operator() (const UInt256 & x) const; DataTypePtr operator() (const Int256 & x) const; DataTypePtr operator() (const bool & x) const; + +private: + // The conditions for converting UInt64 to Int64 are: + // 1. The existence of Int. + // 2. The existence of UInt64, and the UInt64 value must be <= Int64.max. + void checkUInt64ToIn64Conversion(bool& has_signed_int, bool& uint64_convert_possible, const DataTypePtr & type, const Field & elem) const; + + // Convert the UInt64 type to Int64 in order to cover other signed_integer types + // and obtain the least super type of all ints. + void convertUInt64ToInt64IfPossible(DataTypes & data_types) const; }; } From 80aea109b2352f13f84a60336dac02a199a9fcd7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 8 Sep 2023 12:49:15 +0300 Subject: [PATCH 125/327] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index caf504e232d..465e7344174 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1392,7 +1392,6 @@ dragonbox dropoff dumpColumnStructure durations -eacces ecto embeddings emptyArray From 5e0caa669e0c8e9dd6494f45289c90f75f4e419a Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Sep 2023 09:56:53 +0000 Subject: [PATCH 126/327] Move sucessull checks in commit status under a spoiler --- tests/ci/commit_status_helper.py | 50 ++++++++++++++++++++++++-------- 1 file changed, 38 insertions(+), 12 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index ac1498ae745..4d6037c76a8 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -201,12 +201,10 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: comment_body = ( f"\n" - f"This is an automated comment for commit {pr_info.sha} with " - f"description of existing statuses. It's updated for the latest CI running\n" - f"The full report is available [here]({report_url})\n" - f"{worst_state}\n\n" - "\n" - "" + f"*This is an automated comment for commit {pr_info.sha} with " + f"description of existing statuses. It's updated for the latest CI running*\n" + f"📄 [Click here]({report_url}) to open a full report in a separate page\n" + f"{worst_state}\n\n" ) # group checks by the name to get the worst one per each grouped_statuses = {} # type: Dict[CheckDescription, CommitStatuses] @@ -230,17 +228,45 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: else: grouped_statuses[cd] = [status] - table_rows = [] # type: List[str] + table_header = ( + "
Check nameDescriptionStatus
\n" + "\n" + "\n" + ) + table_footer = "\n
Check nameDescriptionStatus
\n" + + details_header = "
Successfull checks\n" + details_footer = "
\n" + + visible_table_rows = [] # type: List[str] + hidden_table_rows = [] # type: List[str] for desc, gs in grouped_statuses.items(): - table_rows.append( + state = get_worst_state(gs) + table_row = ( f"{desc.name}{desc.description}" - f"{beauty_state(get_worst_state(gs))}\n" + f"{beauty_state(state)}\n" ) + if state == SUCCESS: + hidden_table_rows.append(table_row) + else: + visible_table_rows.append(table_row) - table_rows.sort() + visible_table_rows.sort() + hidden_table_rows.sort() - comment_footer = "" - return "".join([comment_body, *table_rows, comment_footer]) + return "".join( + [ + comment_body, + table_header, + *visible_table_rows, + table_footer, + details_header, + table_header, + *hidden_table_rows, + table_footer, + details_footer, + ] + ) def get_worst_state(statuses: CommitStatuses) -> str: From 66479b69660e999c0f0a58a7aa976e1714869fbb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 8 Sep 2023 10:54:25 +0000 Subject: [PATCH 127/327] code cleanup and performance improvement --- src/Functions/FunctionsConversion.h | 23 ++--------------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 79bac2d17e8..6cdf961c450 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -990,20 +990,11 @@ struct ConvertImpl(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; - if constexpr (std::is_same_v || std::is_same_v) - time_zone = &DateLUT::instance(); - /// For argument of Date or DateTime type, second argument with time zone could be specified. - if constexpr (std::is_same_v || std::is_same_v) - { - auto non_null_args = createBlockWithNestedColumns(arguments); - time_zone = &extractTimeZoneFromFunctionArguments(non_null_args, 1, 0); - } - if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) { auto col_to = ColumnString::create(); @@ -1013,17 +1004,7 @@ struct ConvertImplgetOffsets(); size_t size = vec_from.size(); - if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1)); - else if constexpr (std::is_same_v) - data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + col_from->getScale() + 1)); - else - data_to.resize(size * 3); /// Arbitrary - + data_to.resize(size * 3); offsets_to.resize(size); WriteBufferFromVector write_buffer(data_to); From 4a1e2959fa2b10f86592663fac7ba15891b73ea3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 8 Sep 2023 12:45:01 +0000 Subject: [PATCH 128/327] Apply PR comments --- src/Coordination/Changelog.cpp | 12 +++++++++++- src/Coordination/CoordinationSettings.cpp | 19 ++++++++----------- src/Coordination/KeeperDispatcher.cpp | 17 ++++++++++++----- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateManager.cpp | 13 ++++++------- 6 files changed, 39 insertions(+), 26 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 7030775904c..3c2004a1b75 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1048,18 +1048,26 @@ void Changelog::writeThread() LOG_WARNING(log, "Changelog is shut down"); }; + /// NuRaft writes a batch of request by first calling multiple store requests, i.e. AppendLog + /// finished by a flush request + /// We assume that after some number of appends, we always get flush request while (true) { if (try_batch_flush) { try_batch_flush = false; + /// we have Flush request stored in write operation + /// but we try to get new append operations + /// if there are none, we apply the currently set Flush + chassert(std::holds_alternative(write_operation)); if (!write_operations.tryPop(write_operation)) { chassert(batch_append_ok); const auto & flush = std::get(write_operation); flush_logs(flush); notify_append_completion(); - continue; + if (!write_operations.pop(write_operation)) + break; } } else if (!write_operations.pop(write_operation)) @@ -1092,10 +1100,12 @@ void Changelog::writeThread() try_batch_flush = true; continue; } + /// we need to flush because we have maximum allowed pending records flush_logs(flush); } else { + std::lock_guard lock{durable_idx_mutex}; *flush.failed = true; } notify_append_completion(); diff --git a/src/Coordination/CoordinationSettings.cpp b/src/Coordination/CoordinationSettings.cpp index 8688f6f5a40..1f27823182a 100644 --- a/src/Coordination/CoordinationSettings.cpp +++ b/src/Coordination/CoordinationSettings.cpp @@ -22,19 +22,16 @@ void CoordinationSettings::loadFromConfig(const String & config_elem, const Poco Poco::Util::AbstractConfiguration::Keys config_keys; config.keys(config_elem, config_keys); - for (const String & key : config_keys) + try { - try - { + for (const String & key : config_keys) set(key, config.getString(config_elem + "." + key)); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_SETTING) - LOG_WARNING(&Poco::Logger::get("CoordinationSettings"), "Found unknown coordination setting in config: '{}'", key); - else - throw; - } + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("in Coordination settings config"); + throw; } } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index ecd81661d8f..d93bdb21db0 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -149,7 +149,8 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_reconfig_request && !prev_result_done() && current_batch.size() <= max_batch_size + while (!shutdown_called && !has_reconfig_request && + !prev_result_done() && current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); @@ -190,18 +191,24 @@ void KeeperDispatcher::requestThread() if (prev_result) result_buf = forceWaitAndProcessResult(prev_result, current_batch); + /// In case of older version or disabled async replication, result buf will be set to value of `commit` function + /// which always returns nullptr + /// in that case we don't have to do manual wait because are already sure that the batch was committed when we get + /// the result back + /// otherwise, we need to manually wait until the batch is committed if (result_buf) { nuraft::buffer_serializer bs(result_buf); auto log_idx = bs.get_u64(); + /// we will wake up this thread on each commit so we need to run it in loop until the last request of batch is committed while (true) { - auto current_last_committed_idx = last_committed_log_idx.load(std::memory_order_relaxed); + auto current_last_committed_idx = our_last_committed_log_idx.load(std::memory_order_relaxed); if (current_last_committed_idx >= log_idx) break; - last_committed_log_idx.wait(current_last_committed_idx); + our_last_committed_log_idx.wait(current_last_committed_idx); } } } @@ -397,8 +404,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf } } - last_committed_log_idx.store(log_idx, std::memory_order_relaxed); - last_committed_log_idx.notify_all(); + our_last_committed_log_idx.store(log_idx, std::memory_order_relaxed); + our_last_committed_log_idx.notify_all(); }); try diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 31a8f80f252..f11102291f0 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -107,7 +107,7 @@ private: public: std::mutex read_request_queue_mutex; - std::atomic last_committed_log_idx = 0; + std::atomic our_last_committed_log_idx = 0; /// queue of read requests that can be processed after a request with specific session ID and XID is committed std::unordered_map> read_request_queue; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5a8cf88ea73..e753695fd46 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -167,7 +167,7 @@ nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nur request_for_session->zxid = log_idx; preprocess(*request_for_session); - auto result = nuraft::buffer::alloc(8); + auto result = nuraft::buffer::alloc(sizeof(log_idx)); nuraft::buffer_serializer ss(result); ss.put_u64(log_idx); return result; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index f52d0749b10..879a42f6258 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -250,15 +250,14 @@ KeeperStateManager::KeeperStateManager( , log_store(nuraft::cs_new( LogFileSettings { - .force_sync = coordination_settings->force_sync, - .compress_logs = coordination_settings->compress_logs, - .rotate_interval = coordination_settings->rotate_log_storage_interval, - .max_size = coordination_settings->max_log_file_size, - .overallocate_size = coordination_settings->log_file_overallocate_size - }, + .force_sync = coordination_settings->force_sync, + .compress_logs = coordination_settings->compress_logs, + .rotate_interval = coordination_settings->rotate_log_storage_interval, + .max_size = coordination_settings->max_log_file_size, + .overallocate_size = coordination_settings->log_file_overallocate_size}, FlushSettings { - .max_flush_batch_size = coordination_settings->max_flush_batch_size, + .max_flush_batch_size = coordination_settings->max_flush_batch_size, }, keeper_context_)) , server_state_file_name(server_state_file_name_) From 12ce57369bd8cd80043257ffa87d519f6add4279 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Sep 2023 15:36:33 +0200 Subject: [PATCH 129/327] Update FunctionsConversion.h --- src/Functions/FunctionsConversion.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6cdf961c450..b7e21b81b21 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -993,8 +993,6 @@ struct ConvertImpl(*col_with_type_and_name.type); - const DateLUTImpl * time_zone = nullptr; - if (const auto col_from = checkAndGetColumn(col_with_type_and_name.column.get())) { auto col_to = ColumnString::create(); @@ -1013,7 +1011,8 @@ struct ConvertImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + bool is_ok = FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); + /// We don't use timezones in this branch null_map->getData()[i] |= !is_ok; writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); @@ -1023,7 +1022,7 @@ struct ConvertImpl::template execute(vec_from[i], write_buffer, &type, time_zone); + FormatImpl::template execute(vec_from[i], write_buffer, &type, nullptr); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } From 4cfd46a0cc86709610e283b5cc9781bb7b79c170 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Fri, 8 Sep 2023 17:29:09 +0300 Subject: [PATCH 130/327] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2fac021fe2d..9a9c4a57e29 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4663,7 +4663,7 @@ The default value is `false`. ## ignore_access_denied_multidirectory_globs {#ignore_access_denied_multidirectory_globs} Allows to ignore 'permission denied' errors when using multi-directory `{}` globs for [File](../../sql-reference/table-functions/file.md#globs_in_path) and [HDFS](../../sql-reference/table-functions/hdfs.md) storages. -This setting is only applicable when multi directory `{}` glob is used. +This setting is only applicable to multi directory `{}` globs. Possible values: `0`, `1`. From b0b8255c6f53c753532abc5a676eeaee9dfdbaf7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Sep 2023 15:36:32 +0000 Subject: [PATCH 131/327] Set PR status into title --- tests/ci/commit_status_helper.py | 73 ++++++++++++++++++++------------ 1 file changed, 45 insertions(+), 28 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 4d6037c76a8..b9c0392ecc4 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -4,6 +4,7 @@ import csv import os import time from typing import Dict, List, Optional, Union +from collections import defaultdict import logging from github import Github @@ -128,6 +129,28 @@ def post_commit_status( logging.error("Failed to update the status comment, continue anyway") +STATUS_ICON_MAP = defaultdict( + str, + { + ERROR: "🔴", + FAILURE: "🔴", + PENDING: "🟡", + SUCCESS: "🟢", + }, +) + + +def update_pr_title_icon(pr: Github.PullRequest, status: str) -> None: + new_title = pr.title + new_status_icon = STATUS_ICON_MAP[status] + if new_title and new_title[-1] != new_status_icon: + if new_title[-1] in set(STATUS_ICON_MAP.values()): + new_title = new_title[:-1] + " " + new_status_icon + else: + new_title = new_title + " " + new_status_icon + pr.edit(title=new_title) + + def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: """It adds or updates the comment status to all Pull Requests but for release one, so the method does nothing for simple pushes and pull requests with @@ -167,6 +190,8 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: comment = ic break + update_pr_title_icon(pr, get_worst_state(statuses)) + if comment is None: pr.create_issue_comment(comment_body) return @@ -180,15 +205,6 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: """The method generates the comment body, as well it updates the CI report""" - def beauty_state(state: str) -> str: - if state == SUCCESS: - return f"🟢 {state}" - if state == PENDING: - return f"🟡 {state}" - if state in [ERROR, FAILURE]: - return f"🔴 {state}" - return state - report_url = create_ci_report(pr_info, statuses) worst_state = get_worst_state(statuses) if not worst_state: @@ -196,14 +212,14 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: # the function should not be used on empty statuses worst_state = "The commit doesn't have the statuses yet" else: - worst_state = f"The overall status of the commit is {beauty_state(worst_state)}" + worst_state = f"The overall status of the commit is {STATUS_ICON_MAP[worst_state]} {worst_state}" comment_body = ( f"\n" f"*This is an automated comment for commit {pr_info.sha} with " - f"description of existing statuses. It's updated for the latest CI running*\n" - f"📄 [Click here]({report_url}) to open a full report in a separate page\n" + f"description of existing statuses. It's updated for the latest CI running*\n\n" + f"[📄 Click here]({report_url}) to open a full report in a separate page\n" f"{worst_state}\n\n" ) # group checks by the name to get the worst one per each @@ -244,29 +260,30 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: state = get_worst_state(gs) table_row = ( f"{desc.name}{desc.description}" - f"{beauty_state(state)}\n" + f"{STATUS_ICON_MAP[worst_state]} {worst_state}\n" ) if state == SUCCESS: hidden_table_rows.append(table_row) else: visible_table_rows.append(table_row) - visible_table_rows.sort() - hidden_table_rows.sort() + result = [comment_body] - return "".join( - [ - comment_body, - table_header, - *visible_table_rows, - table_footer, - details_header, - table_header, - *hidden_table_rows, - table_footer, - details_footer, - ] - ) + if visible_table_rows: + visible_table_rows.sort() + result.append(table_header) + result.extend(visible_table_rows) + result.append(table_footer) + + if hidden_table_rows: + hidden_table_rows.sort() + result.append(details_header) + result.append(table_header) + result.extend(hidden_table_rows) + result.append(table_footer) + result.append(details_footer) + + return "".join(result) def get_worst_state(statuses: CommitStatuses) -> str: From 465d84aba7412082123d475df3fb83d927062f15 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Sep 2023 15:39:07 +0000 Subject: [PATCH 132/327] Set raw_logs in create_ci_report --- tests/ci/commit_status_helper.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index b9c0392ecc4..52ee80d66f0 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -298,7 +298,14 @@ def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: log_urls = None if status.target_url is not None: log_urls = [status.target_url] - test_results.append(TestResult(status.context, status.state, log_urls=log_urls)) + raw_logs = None + if status.description: + raw_logs = status.description + test_results.append( + TestResult( + status.context, status.state, log_urls=log_urls, raw_logs=raw_logs + ) + ) return upload_results( S3Helper(), pr_info.number, pr_info.sha, test_results, [], CI_STATUS_NAME ) From a46d1974ac751df34fbf047ac795b0e4ebf495fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Sep 2023 15:49:13 +0000 Subject: [PATCH 133/327] fix --- tests/ci/commit_status_helper.py | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 52ee80d66f0..207a04f06ad 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -12,6 +12,7 @@ from github.GithubObject import _NotSetType, NotSet as NotSet from github.Commit import Commit from github.CommitStatus import CommitStatus from github.IssueComment import IssueComment +from github.PullRequest import PullRequest from github.Repository import Repository from ci_config import CI_CONFIG, REQUIRED_CHECKS, CHECK_DESCRIPTIONS, CheckDescription @@ -140,9 +141,11 @@ STATUS_ICON_MAP = defaultdict( ) -def update_pr_title_icon(pr: Github.PullRequest, status: str) -> None: - new_title = pr.title +def update_pr_title_icon(pr: PullRequest, status: str) -> None: new_status_icon = STATUS_ICON_MAP[status] + if not new_status_icon: + return + new_title = pr.title if new_title and new_title[-1] != new_status_icon: if new_title[-1] in set(STATUS_ICON_MAP.values()): new_title = new_title[:-1] + " " + new_status_icon @@ -207,20 +210,14 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: report_url = create_ci_report(pr_info, statuses) worst_state = get_worst_state(statuses) - if not worst_state: - # Theoretically possible, although - # the function should not be used on empty statuses - worst_state = "The commit doesn't have the statuses yet" - else: - worst_state = f"The overall status of the commit is {STATUS_ICON_MAP[worst_state]} {worst_state}" comment_body = ( f"\n" f"*This is an automated comment for commit {pr_info.sha} with " f"description of existing statuses. It's updated for the latest CI running*\n\n" - f"[📄 Click here]({report_url}) to open a full report in a separate page\n" - f"{worst_state}\n\n" + f"[{STATUS_ICON_MAP[worst_state]} Click here]({report_url}) to open a full report in a separate page\n" + f"\n" ) # group checks by the name to get the worst one per each grouped_statuses = {} # type: Dict[CheckDescription, CommitStatuses] @@ -260,7 +257,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: state = get_worst_state(gs) table_row = ( f"{desc.name}{desc.description}" - f"{STATUS_ICON_MAP[worst_state]} {worst_state}\n" + f"{STATUS_ICON_MAP[state]} {state}\n" ) if state == SUCCESS: hidden_table_rows.append(table_row) From 6a51d9fd7d2a373741f79ac7f9100225d79ef8bd Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 8 Sep 2023 16:01:13 +0000 Subject: [PATCH 134/327] do not merge this commit --- src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- tests/ci/commit_status_helper.py | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e6edb99e301..840a77868d6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7656,8 +7656,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( entry_replace.columns_version = -1; } - if (replace) - { + if (replace) { /// Cancel concurrent inserts in range clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); /// Remove deduplication block_ids of replacing parts diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 207a04f06ad..9788e629cb8 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -248,7 +248,7 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: ) table_footer = "\n\n" - details_header = "
Successfull checks\n" + details_header = "
Successful checks\n" details_footer = "
\n" visible_table_rows = [] # type: List[str] From da69a9fda02145347424abf36e1c1be61f20da01 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 8 Sep 2023 21:41:42 +0000 Subject: [PATCH 135/327] Add a log message on replicated table drop --- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++++ .../0_stateless/01165_lost_part_empty_partition.sql | 2 ++ 2 files changed, 12 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e52b1159b40..aac85b4afc0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -981,6 +981,16 @@ void StorageReplicatedMergeTree::drop() { /// Session could expire, get it again zookeeper = getZooKeeperIfTableShutDown(); + + auto lost_part_count_path = fs::path(zookeeper_path) / "lost_part_count"; + Coordination::Stat lost_part_count_stat; + String lost_part_count_str; + if (zookeeper->tryGet(lost_part_count_path, lost_part_count_str, &lost_part_count_stat)) + { + UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); + if (lost_part_count > 0) + LOG_DEBUG(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); + } dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper); } } diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index 924798b0050..8486a6ac2e6 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -12,6 +12,7 @@ drop table rmt1; system sync replica rmt2; select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; +select count() from system.text_log where logger_name like '%' || currentDatabase() || '%' and message ilike '%table with non-zero lost_part_count equal to%'; create table rmt1 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '1') order by n partition by tuple(); @@ -24,6 +25,7 @@ drop table rmt1; system sync replica rmt2; select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; +select count() from system.text_log where logger_name like '%' || currentDatabase() || '%' and message ilike '%table with non-zero lost_part_count equal to%'; create table rmt1 (n UInt8, m Int32, d Date, t DateTime) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '1') order by n partition by (n, m, d, t); From 1b3da7fc1e1abef16564ea0c3e9b4bfd1808a23b Mon Sep 17 00:00:00 2001 From: serxa Date: Sat, 9 Sep 2023 09:26:18 +0000 Subject: [PATCH 136/327] fix build --- src/IO/IResourceManager.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/IResourceManager.h b/src/IO/IResourceManager.h index b4270cd1935..39fede0e19f 100644 --- a/src/IO/IResourceManager.h +++ b/src/IO/IResourceManager.h @@ -49,7 +49,7 @@ public: /// For introspection, see `system.scheduler` table using VisitorFunc = std::function; - void forEachNode(VisitorFunc visitor) override; + virtual void forEachNode(VisitorFunc visitor) = 0; }; using ResourceManagerPtr = std::shared_ptr; From 9b936c44db62400cca0bf3e3d53dd43898ddd85e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Sep 2023 12:29:39 +0200 Subject: [PATCH 137/327] Revert "Revert "Add settings for real-time updates during query execution"" --- docs/en/operations/settings/settings.md | 8 + src/Client/ClientBase.cpp | 36 +- src/Client/ClientBase.h | 15 + src/Core/Settings.h | 3 + src/Interpreters/Aggregator.cpp | 23 ++ src/Interpreters/Aggregator.h | 2 + src/Processors/Chunk.cpp | 3 +- .../Executors/CompletedPipelineExecutor.cpp | 4 +- src/Processors/Executors/ExecutingGraph.cpp | 1 - .../Executors/ExecutionThreadContext.h | 13 +- src/Processors/Executors/ExecutorTasks.cpp | 15 +- src/Processors/Executors/ExecutorTasks.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 5 +- src/Processors/Executors/PipelineExecutor.h | 5 +- .../PullingAsyncPipelineExecutor.cpp | 7 +- .../Executors/PullingAsyncPipelineExecutor.h | 2 +- .../Executors/PullingPipelineExecutor.cpp | 2 +- .../PushingAsyncPipelineExecutor.cpp | 2 +- .../Executors/PushingPipelineExecutor.cpp | 2 +- src/Processors/Formats/IOutputFormat.cpp | 110 ++++-- src/Processors/Formats/IOutputFormat.h | 13 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 34 +- .../Formats/Impl/PrettyBlockOutputFormat.h | 6 + .../Impl/PrettyCompactBlockOutputFormat.cpp | 3 +- src/Processors/Formats/LazyOutputFormat.h | 5 +- src/Processors/IProcessor.cpp | 5 + src/Processors/IProcessor.h | 23 ++ src/Processors/LimitTransform.cpp | 9 +- src/Processors/LimitTransform.h | 4 + .../QueryPlan/BuildQueryPipelineSettings.cpp | 7 +- .../QueryPlan/BuildQueryPipelineSettings.h | 3 + src/Processors/QueryPlan/QueryPlan.cpp | 5 + src/Processors/QueryPlan/SortingStep.h | 2 + .../AggregatingPartialResultTransform.cpp | 47 +++ .../AggregatingPartialResultTransform.h | 29 ++ .../Transforms/AggregatingTransform.cpp | 12 + .../Transforms/AggregatingTransform.h | 21 ++ .../Transforms/ExpressionTransform.cpp | 6 + .../Transforms/ExpressionTransform.h | 5 + .../LimitPartialResultTransform.cpp | 42 +++ .../Transforms/LimitPartialResultTransform.h | 36 ++ .../Transforms/LimitsCheckingTransform.cpp | 1 + .../Transforms/LimitsCheckingTransform.h | 2 + .../MergeSortingPartialResultTransform.cpp | 48 +++ .../MergeSortingPartialResultTransform.h | 28 ++ .../Transforms/MergeSortingTransform.cpp | 12 + .../Transforms/MergeSortingTransform.h | 11 + .../Transforms/PartialResultTransform.cpp | 80 +++++ .../Transforms/PartialResultTransform.h | 57 ++++ src/QueryPipeline/Pipe.cpp | 204 +++++++++--- src/QueryPipeline/Pipe.h | 25 +- src/QueryPipeline/QueryPipeline.cpp | 50 ++- src/QueryPipeline/QueryPipeline.h | 7 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 21 +- src/QueryPipeline/QueryPipelineBuilder.h | 6 + src/Server/TCPHandler.cpp | 3 +- .../0_stateless/02010_lc_native.python | 291 +++------------- .../02210_processors_profile_log.reference | 1 + .../02458_insert_select_progress_tcp.python | 244 ++------------ .../02750_settings_alias_tcp_protocol.python | 212 +----------- ...rting_result_during_query_execution.python | 93 ++++++ ...ng_result_during_query_execution.reference | 38 +++ ...l_sorting_result_during_query_execution.sh | 8 + ...ating_result_during_query_execution.python | 125 +++++++ ...ng_result_during_query_execution.reference | 88 +++++ ...gregating_result_during_query_execution.sh | 8 + .../queries/0_stateless/helpers/tcp_client.py | 313 ++++++++++++++++++ 67 files changed, 1779 insertions(+), 774 deletions(-) create mode 100644 src/Processors/Transforms/AggregatingPartialResultTransform.cpp create mode 100644 src/Processors/Transforms/AggregatingPartialResultTransform.h create mode 100644 src/Processors/Transforms/LimitPartialResultTransform.cpp create mode 100644 src/Processors/Transforms/LimitPartialResultTransform.h create mode 100644 src/Processors/Transforms/MergeSortingPartialResultTransform.cpp create mode 100644 src/Processors/Transforms/MergeSortingPartialResultTransform.h create mode 100644 src/Processors/Transforms/PartialResultTransform.cpp create mode 100644 src/Processors/Transforms/PartialResultTransform.h create mode 100755 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python create mode 100644 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference create mode 100755 tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh create mode 100644 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python create mode 100644 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference create mode 100755 tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh create mode 100644 tests/queries/0_stateless/helpers/tcp_client.py diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 66b8c923d5c..a1989ab3d13 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4644,6 +4644,14 @@ SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_pars └─────────────────────┴──────────────────────────┘ ``` +## partial_result_update_duration_ms + +Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET. + +## max_rows_in_partial_result + +Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query). + ## validate_tcp_client_information {#validate-tcp-client-information} Determines whether validation of client information enabled when query packet is received from a client using a TCP connection. diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 859665c64b3..cc60095884e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -441,7 +441,20 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (!block) return; - processed_rows += block.rows(); + if (block.rows() == 0 && partial_result_mode == PartialResultMode::Active) + { + partial_result_mode = PartialResultMode::Inactive; + if (is_interactive) + { + progress_indication.clearProgressOutput(*tty_buf); + std::cout << "Full result:" << std::endl; + progress_indication.writeProgress(*tty_buf); + } + } + + if (partial_result_mode == PartialResultMode::Inactive) + processed_rows += block.rows(); + /// Even if all blocks are empty, we still need to initialize the output stream to write empty resultset. initOutputFormat(block, parsed_query); @@ -451,13 +464,20 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) return; + if (!is_interactive && partial_result_mode == PartialResultMode::Active) + return; + /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. if (need_render_progress && tty_buf && (!select_into_file || select_into_file_and_stdout)) progress_indication.clearProgressOutput(*tty_buf); try { - output_format->write(materializeBlock(block)); + if (partial_result_mode == PartialResultMode::Active) + output_format->writePartialResult(materializeBlock(block)); + else + output_format->write(materializeBlock(block)); + written_first_block = true; } catch (const Exception &) @@ -521,6 +541,9 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) void ClientBase::initOutputFormat(const Block & block, ASTPtr parsed_query) try { + if (partial_result_mode == PartialResultMode::NotInit) + partial_result_mode = PartialResultMode::Active; + if (!output_format) { /// Ignore all results when fuzzing as they can be huge. @@ -931,6 +954,14 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; + bool has_partial_result_setting = settings.partial_result_update_duration_ms.totalMilliseconds() > 0; + + if (has_partial_result_setting) + { + partial_result_mode = PartialResultMode::NotInit; + if (is_interactive) + std::cout << "Partial result:" << std::endl; + } int retries_left = 10; while (retries_left) @@ -1736,6 +1767,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } processed_rows = 0; + partial_result_mode = PartialResultMode::Inactive; written_first_block = false; progress_indication.resetProgress(); profile_events.watch.restart(); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index eabc79b7432..e2394259f85 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -272,6 +272,21 @@ protected: size_t processed_rows = 0; /// How many rows have been read or written. bool print_num_processed_rows = false; /// Whether to print the number of processed rows at + enum class PartialResultMode: UInt8 + { + /// Query doesn't show partial result before the first block with 0 rows. + /// The first block with 0 rows initializes the output table format using its header. + NotInit, + + /// Query shows partial result after the first and before the second block with 0 rows. + /// The second block with 0 rows indicates that that receiving blocks with partial result has been completed and next blocks will be with the full result. + Active, + + /// Query doesn't show partial result at all. + Inactive, + }; + PartialResultMode partial_result_mode = PartialResultMode::Inactive; + bool print_stack_trace = false; /// The last exception that was received from the server. Is used for the /// return code in batch mode. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 620cc8fd67f..719def2a649 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -309,6 +309,9 @@ class IColumn; \ M(Bool, partial_result_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ \ + M(Milliseconds, partial_result_update_duration_ms, 0, "Interval (in milliseconds) for sending updates with partial data about the result table to the client (in interactive mode) during query execution. Setting to 0 disables partial results. Only supported for single-threaded GROUP BY without key, ORDER BY, LIMIT and OFFSET.", 0) \ + M(UInt64, max_rows_in_partial_result, 10, "Maximum rows to show in the partial result after every real-time update while the query runs (use partial result limit + OFFSET as a value in case of OFFSET in the query).", 0) \ + \ M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \ M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \ /** Settings for testing hedged requests */ \ diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 9249c3ce4ce..23ee097ebff 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2272,6 +2272,29 @@ Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_va return block; } +Block Aggregator::prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const +{ + size_t rows = 1; + bool final = true; + + auto && out_cols + = prepareOutputBlockColumns(params, aggregate_functions, getHeader(final), data_variants.aggregates_pools, final, rows); + auto && [key_columns, raw_key_columns, aggregate_columns, final_aggregate_columns, aggregate_columns_data] = out_cols; + + AggregatedDataWithoutKey & data = data_variants.without_key; + + /// Always single-thread. It's safe to pass current arena from 'aggregates_pool'. + for (size_t insert_i = 0; insert_i < params.aggregates_size; ++insert_i) + aggregate_functions[insert_i]->insertResultInto( + data + offsets_of_aggregate_states[insert_i], + *final_aggregate_columns[insert_i], + data_variants.aggregates_pool); + + Block block = finalizeBlock(params, getHeader(final), std::move(out_cols), final, rows); + + return block; +} + template Aggregator::ConvertToBlockRes Aggregator::prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4f2c86606c5..6bfaa76f9b3 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1210,6 +1210,7 @@ private: friend class ConvertingAggregatedToChunksSource; friend class ConvertingAggregatedToChunksWithMergingSource; friend class AggregatingInOrderTransform; + friend class AggregatingPartialResultTransform; /// Data structure of source blocks. Block header; @@ -1391,6 +1392,7 @@ private: std::atomic * is_cancelled = nullptr) const; Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const; + Block prepareBlockAndFillWithoutKeySnapshot(AggregatedDataVariants & data_variants) const; BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const; template diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 3839a8963b2..cd442085eca 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -14,7 +14,8 @@ namespace ErrorCodes extern const int POSITION_OUT_OF_BOUND; } -Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_) +Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) + : columns(std::move(columns_)), num_rows(num_rows_) { checkNumRowsIsConsistent(); } diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 598a51bf0c7..c30586e194e 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -75,7 +75,7 @@ void CompletedPipelineExecutor::execute() if (interactive_timeout_ms) { data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); /// Avoid passing this to lambda, copy ptr to data instead. @@ -105,7 +105,7 @@ void CompletedPipelineExecutor::execute() } else { - PipelineExecutor executor(pipeline.processors, pipeline.process_list_element); + PipelineExecutor executor(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); executor.setReadProgressCallback(pipeline.getReadProgressCallback()); executor.execute(pipeline.getNumThreads(), pipeline.getConcurrencyControl()); } diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 27f6a454b24..6a946b4a4b9 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -260,7 +260,6 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue { pid = updated_processors.top(); updated_processors.pop(); - /// In this method we have ownership on node. auto & node = *nodes[pid]; diff --git a/src/Processors/Executors/ExecutionThreadContext.h b/src/Processors/Executors/ExecutionThreadContext.h index eb048f8ab09..85788a70771 100644 --- a/src/Processors/Executors/ExecutionThreadContext.h +++ b/src/Processors/Executors/ExecutionThreadContext.h @@ -30,6 +30,12 @@ private: /// Callback for read progress. ReadProgressCallback * read_progress_callback = nullptr; + /// Timer that stops optimization of running local tasks instead of queuing them. + /// It provides local progress for each IProcessor task, allowing the partial result of the request to be always sended to the user. + Stopwatch watch; + /// Time period that limits the maximum allowed duration for optimizing the scheduling of local tasks within the executor + const UInt64 partial_result_duration_ms; + public: #ifndef NDEBUG /// Time for different processing stages. @@ -62,8 +68,13 @@ public: void setException(std::exception_ptr exception_) { exception = exception_; } void rethrowExceptionIfHas(); - explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback) + bool needWatchRestartForPartialResultProgress() { return partial_result_duration_ms != 0 && partial_result_duration_ms < watch.elapsedMilliseconds(); } + void restartWatch() { watch.restart(); } + + explicit ExecutionThreadContext(size_t thread_number_, bool profile_processors_, bool trace_processors_, ReadProgressCallback * callback, UInt64 partial_result_duration_ms_) : read_progress_callback(callback) + , watch(CLOCK_MONOTONIC) + , partial_result_duration_ms(partial_result_duration_ms_) , thread_number(thread_number_) , profile_processors(profile_processors_) , trace_processors(trace_processors_) diff --git a/src/Processors/Executors/ExecutorTasks.cpp b/src/Processors/Executors/ExecutorTasks.cpp index e61d225a968..08920592391 100644 --- a/src/Processors/Executors/ExecutorTasks.cpp +++ b/src/Processors/Executors/ExecutorTasks.cpp @@ -108,8 +108,15 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea { context.setTask(nullptr); - /// Take local task from queue if has one. - if (!queue.empty() && !context.hasAsyncTasks()) + /// If sending partial results is allowed and local tasks scheduling optimization is repeated longer than the limit + /// or new task need to send partial result later, skip optimization for this iteration. + /// Otherwise take local task from queue if has one. + if ((!queue.empty() && queue.front()->processor->isPartialResultProcessor()) + || context.needWatchRestartForPartialResultProgress()) + { + context.restartWatch(); + } + else if (!queue.empty() && !context.hasAsyncTasks()) { context.setTask(queue.front()); queue.pop(); @@ -139,7 +146,7 @@ void ExecutorTasks::pushTasks(Queue & queue, Queue & async_queue, ExecutionThrea } } -void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback) +void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms) { num_threads = num_threads_; use_threads = use_threads_; @@ -151,7 +158,7 @@ void ExecutorTasks::init(size_t num_threads_, size_t use_threads_, bool profile_ executor_contexts.reserve(num_threads); for (size_t i = 0; i < num_threads; ++i) - executor_contexts.emplace_back(std::make_unique(i, profile_processors, trace_processors, callback)); + executor_contexts.emplace_back(std::make_unique(i, profile_processors, trace_processors, callback, partial_result_duration_ms)); } } diff --git a/src/Processors/Executors/ExecutorTasks.h b/src/Processors/Executors/ExecutorTasks.h index d35f8de94d1..ab6d5e91411 100644 --- a/src/Processors/Executors/ExecutorTasks.h +++ b/src/Processors/Executors/ExecutorTasks.h @@ -58,7 +58,7 @@ public: void tryGetTask(ExecutionThreadContext & context); void pushTasks(Queue & queue, Queue & async_queue, ExecutionThreadContext & context); - void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback); + void init(size_t num_threads_, size_t use_threads_, bool profile_processors, bool trace_processors, ReadProgressCallback * callback, UInt64 partial_result_duration_ms); void fill(Queue & queue); void upscale(size_t use_threads_); diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 37af391fba3..77779e2cec2 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -33,8 +33,9 @@ namespace ErrorCodes } -PipelineExecutor::PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem) +PipelineExecutor::PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_) : process_list_element(std::move(elem)) + , partial_result_duration_ms(partial_result_duration_ms_) { if (process_list_element) { @@ -328,7 +329,7 @@ void PipelineExecutor::initializeExecution(size_t num_threads, bool concurrency_ Queue queue; graph->initializeExecution(queue); - tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get()); + tasks.init(num_threads, use_threads, profile_processors, trace_processors, read_progress_callback.get(), partial_result_duration_ms); tasks.fill(queue); if (num_threads > 1) diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index dee12dad282..6cb0e6c4ac1 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -33,7 +33,7 @@ public: /// During pipeline execution new processors can appear. They will be added to existing set. /// /// Explicit graph representation is built in constructor. Throws if graph is not correct. - explicit PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem); + explicit PipelineExecutor(std::shared_ptr & processors, QueryStatusPtr elem, UInt64 partial_result_duration_ms_ = 0); ~PipelineExecutor(); /// Execute pipeline in multiple threads. Must be called once. @@ -90,6 +90,9 @@ private: ReadProgressCallbackPtr read_progress_callback; + /// Duration between sending partial result through the pipeline + const UInt64 partial_result_duration_ms; + using Queue = std::queue; void initializeExecution(size_t num_threads, bool concurrency_control); /// Initialize executor contexts and task_queue. diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 345bec395b2..95a2022bf93 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -41,12 +41,13 @@ struct PullingAsyncPipelineExecutor::Data } }; -PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) +PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting) : pipeline(pipeline_) { if (!pipeline.pulling()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PullingAsyncPipelineExecutor must be pulling"); - lazy_format = std::make_shared(pipeline.output->getHeader()); + lazy_format = std::make_shared(pipeline.output->getHeader(), /*is_partial_result_protocol_active*/ has_partial_result_setting); + pipeline.complete(lazy_format); } @@ -103,7 +104,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (!data) { data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); data->lazy_format = lazy_format.get(); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 361bcc0155c..202ecbf281b 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -21,7 +21,7 @@ struct ProfileInfo; class PullingAsyncPipelineExecutor { public: - explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_); + explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_, bool has_partial_result_setting = false); ~PullingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..f79f15c19bf 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -44,7 +44,7 @@ bool PullingPipelineExecutor::pull(Chunk & chunk) { if (!executor) { - executor = std::make_shared(pipeline.processors, pipeline.process_list_element); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index a816ab9ca7f..f3ed24e7e96 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -167,7 +167,7 @@ void PushingAsyncPipelineExecutor::start() started = true; data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); data->executor->setReadProgressCallback(pipeline.getReadProgressCallback()); data->source = pushing_source.get(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 696932932df..f2b018792c7 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -87,7 +87,7 @@ void PushingPipelineExecutor::start() return; started = true; - executor = std::make_shared(pipeline.processors, pipeline.process_list_element); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element, pipeline.partial_result_duration_ms); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); if (!executor->executeStep(&input_wait_flag)) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 88a6fb1e92f..e691e32a7bc 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -1,40 +1,89 @@ #include #include +#include namespace DB { -IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_) - : IProcessor({header_, header_, header_}, {}), out(out_) +IOutputFormat::IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_) + : IProcessor({header_, header_, header_, header_}, {}) + , out(out_) + , is_partial_result_protocol_active(is_partial_result_protocol_active_) { } +void IOutputFormat::setCurrentChunk(InputPort & input, PortKind kind) +{ + current_chunk = input.pull(true); + current_block_kind = kind; + has_input = true; +} + +IOutputFormat::Status IOutputFormat::prepareMainAndPartialResult() +{ + bool need_data = false; + for (auto kind : {Main, PartialResult}) + { + auto & input = getPort(kind); + + if (input.isFinished()) + continue; + + if (kind == PartialResult && main_input_activated) + { + input.close(); + continue; + } + + input.setNeeded(); + need_data = true; + + if (!input.hasData()) + continue; + + setCurrentChunk(input, kind); + return Status::Ready; + } + + if (need_data) + return Status::NeedData; + + return Status::Finished; +} + +IOutputFormat::Status IOutputFormat::prepareTotalsAndExtremes() +{ + for (auto kind : {Totals, Extremes}) + { + auto & input = getPort(kind); + + if (!input.isConnected() || input.isFinished()) + continue; + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + setCurrentChunk(input, kind); + return Status::Ready; + } + + return Status::Finished; +} + IOutputFormat::Status IOutputFormat::prepare() { if (has_input) return Status::Ready; - for (auto kind : {Main, Totals, Extremes}) - { - auto & input = getPort(kind); + auto status = prepareMainAndPartialResult(); + if (status != Status::Finished) + return status; - if (kind != Main && !input.isConnected()) - continue; - - if (input.isFinished()) - continue; - - input.setNeeded(); - - if (!input.hasData()) - return Status::NeedData; - - current_chunk = input.pull(true); - current_block_kind = kind; - has_input = true; - return Status::Ready; - } + status = prepareTotalsAndExtremes(); + if (status != Status::Finished) + return status; finished = true; @@ -83,8 +132,18 @@ void IOutputFormat::work() case Main: result_rows += current_chunk.getNumRows(); result_bytes += current_chunk.allocatedBytes(); + if (is_partial_result_protocol_active && !main_input_activated && current_chunk.hasRows()) + { + /// Sending an empty block signals to the client that partial results are terminated, + /// and only data from the main pipeline will be forwarded. + consume(Chunk(current_chunk.cloneEmptyColumns(), 0)); + main_input_activated = true; + } consume(std::move(current_chunk)); break; + case PartialResult: + consumePartialResult(std::move(current_chunk)); + break; case Totals: writeSuffixIfNeeded(); if (auto totals = prepareTotals(std::move(current_chunk))) @@ -119,6 +178,15 @@ void IOutputFormat::write(const Block & block) flush(); } +void IOutputFormat::writePartialResult(const Block & block) +{ + writePrefixIfNeeded(); + consumePartialResult(Chunk(block.getColumns(), block.rows())); + + if (auto_flush) + flush(); +} + void IOutputFormat::finalize() { if (finalized) diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 58700a978ff..470d24e9a22 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -23,9 +23,9 @@ class WriteBuffer; class IOutputFormat : public IProcessor { public: - enum PortKind { Main = 0, Totals = 1, Extremes = 2 }; + enum PortKind { Main = 0, Totals = 1, Extremes = 2, PartialResult = 3 }; - IOutputFormat(const Block & header_, WriteBuffer & out_); + IOutputFormat(const Block & header_, WriteBuffer & out_, bool is_partial_result_protocol_active_ = false); Status prepare() override; void work() override; @@ -54,6 +54,7 @@ public: /// TODO: separate formats and processors. void write(const Block & block); + void writePartialResult(const Block & block); void finalize(); @@ -118,6 +119,7 @@ protected: virtual void consume(Chunk) = 0; virtual void consumeTotals(Chunk) {} virtual void consumeExtremes(Chunk) {} + virtual void consumePartialResult(Chunk) {} virtual void finalizeImpl() {} virtual void finalizeBuffers() {} virtual void writePrefix() {} @@ -166,6 +168,7 @@ protected: Chunk current_chunk; PortKind current_block_kind = PortKind::Main; + bool main_input_activated = false; bool has_input = false; bool finished = false; bool finalized = false; @@ -180,9 +183,15 @@ protected: Statistics statistics; private: + void setCurrentChunk(InputPort & input, PortKind kind); + IOutputFormat::Status prepareMainAndPartialResult(); + IOutputFormat::Status prepareTotalsAndExtremes(); + size_t rows_read_before = 0; bool are_totals_written = false; + bool is_partial_result_protocol_active = false; + /// Counters for consumed chunks. Are used for QueryLog. size_t result_rows = 0; size_t result_bytes = 0; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 14648e68f94..6fa891297f6 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -134,7 +134,8 @@ void PrettyBlockOutputFormat::write(Chunk chunk, PortKind port_kind) { if (total_rows >= format_settings.pretty.max_rows) { - total_rows += chunk.getNumRows(); + if (port_kind != PortKind::PartialResult) + total_rows += chunk.getNumRows(); return; } if (mono_block) @@ -315,7 +316,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind } writeString(bottom_separator_s, out); - total_rows += num_rows; + if (port_kind != PortKind::PartialResult) + total_rows += num_rows; } @@ -388,6 +390,34 @@ void PrettyBlockOutputFormat::consumeExtremes(Chunk chunk) write(std::move(chunk), PortKind::Extremes); } +void PrettyBlockOutputFormat::clearLastLines(size_t lines_number) +{ + /// http://en.wikipedia.org/wiki/ANSI_escape_code + #define MOVE_TO_PREV_LINE "\033[A" + #define CLEAR_TO_END_OF_LINE "\033[K" + + static const char * clear_prev_line = MOVE_TO_PREV_LINE \ + CLEAR_TO_END_OF_LINE; + + /// Move cursor to the beginning of line + writeCString("\r", out); + + for (size_t line = 0; line < lines_number; ++line) + { + writeCString(clear_prev_line, out); + } +} + +void PrettyBlockOutputFormat::consumePartialResult(Chunk chunk) +{ + if (prev_partial_block_rows > 0) + /// number of rows + header line + footer line + clearLastLines(prev_partial_block_rows + 2); + + prev_partial_block_rows = chunk.getNumRows(); + write(std::move(chunk), PortKind::PartialResult); +} + void PrettyBlockOutputFormat::writeMonoChunkIfNeeded() { diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index dfb23ac63f9..92466dce3ff 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -28,7 +28,12 @@ protected: void consumeTotals(Chunk) override; void consumeExtremes(Chunk) override; + void clearLastLines(size_t lines_number); + void consumePartialResult(Chunk) override; + size_t total_rows = 0; + size_t prev_partial_block_rows = 0; + size_t row_number_width = 7; // "10000. " const FormatSettings format_settings; @@ -55,6 +60,7 @@ protected: void resetFormatterImpl() override { total_rows = 0; + prev_partial_block_rows = 0; } private: diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 2ba9ec725e2..3a04d86b1ad 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -194,7 +194,8 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po writeBottom(max_widths); - total_rows += num_rows; + if (port_kind != PortKind::PartialResult) + total_rows += num_rows; } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 9cf609ed2d7..bbcfdbb7193 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -14,8 +14,8 @@ class LazyOutputFormat : public IOutputFormat { public: - explicit LazyOutputFormat(const Block & header) - : IOutputFormat(header, out), queue(2) {} + explicit LazyOutputFormat(const Block & header, bool is_partial_result_protocol_active = false) + : IOutputFormat(header, out, is_partial_result_protocol_active), queue(2) {} String getName() const override { return "LazyOutputFormat"; } @@ -49,6 +49,7 @@ protected: void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } + void consumePartialResult(Chunk chunk) override { consume(std::move(chunk)); } private: diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index 8b160153733..2f294a32531 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -40,5 +40,10 @@ std::string IProcessor::statusToName(Status status) UNREACHABLE(); } +ProcessorPtr IProcessor::getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) +{ + return current_processor->getPartialResultProcessor(current_processor, partial_result_limit, partial_result_duration_ms); +} + } diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index c6bef186877..51a0bb1c121 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -164,6 +164,8 @@ public: static std::string statusToName(Status status); + static ProcessorPtr getPartialResultProcessorPtr(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms); + /** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations. * * It may access input and output ports, @@ -235,6 +237,22 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'expandPipeline' is not implemented for {} processor", getName()); } + enum class PartialResultStatus + { + /// Processor currently doesn't support work with the partial result pipeline. + NotSupported, + + /// Processor can be skipped in the partial result pipeline. + SkipSupported, + + /// Processor creates a light-weight copy of itself in the partial result pipeline. + /// The copy can create snapshots of the original processor or transform small blocks of data in the same way as the original processor + FullSupported, + }; + + virtual bool isPartialResultProcessor() const { return false; } + virtual PartialResultStatus getPartialResultProcessorSupportStatus() const { return PartialResultStatus::NotSupported; } + /// In case if query was cancelled executor will wait till all processors finish their jobs. /// Generally, there is no reason to check this flag. However, it may be reasonable for long operations (e.g. i/o). bool isCancelled() const { return is_cancelled.load(std::memory_order_acquire); } @@ -369,6 +387,11 @@ public: protected: virtual void onCancel() {} + virtual ProcessorPtr getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getPartialResultProcessor' is not implemented for {} processor", getName()); + } + private: /// For: /// - elapsed_us diff --git a/src/Processors/LimitTransform.cpp b/src/Processors/LimitTransform.cpp index 5e24062d67a..b2bf3c28eee 100644 --- a/src/Processors/LimitTransform.cpp +++ b/src/Processors/LimitTransform.cpp @@ -1,5 +1,5 @@ #include - +#include namespace DB { @@ -180,7 +180,6 @@ LimitTransform::Status LimitTransform::preparePair(PortsData & data) return Status::NeedData; data.current_chunk = input.pull(true); - auto rows = data.current_chunk.getNumRows(); if (rows_before_limit_at_least && !data.input_port_has_counter) @@ -367,5 +366,11 @@ bool LimitTransform::sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort return true; } +ProcessorPtr LimitTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) +{ + const auto & header = inputs.front().getHeader(); + return std::make_shared(header, partial_result_limit, partial_result_duration_ms, limit, offset); +} + } diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index 33ff968985f..eac5f9e8d6d 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -55,6 +55,8 @@ private: ColumnRawPtrs extractSortColumns(const Columns & columns) const; bool sortColumnsEqualAt(const ColumnRawPtrs & current_chunk_sort_columns, UInt64 current_chunk_row_num) const; + ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; + public: LimitTransform( const Block & header_, UInt64 limit_, UInt64 offset_, size_t num_streams = 1, @@ -73,6 +75,8 @@ public: void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } + + PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } }; } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index fb3ed7f80fc..60ac30389a1 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -9,7 +9,12 @@ namespace DB BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from) { BuildQueryPipelineSettings settings; - settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes); + + const auto & context_settings = from->getSettingsRef(); + settings.partial_result_limit = context_settings.max_rows_in_partial_result; + settings.partial_result_duration_ms = context_settings.partial_result_update_duration_ms.totalMilliseconds(); + + settings.actions_settings = ExpressionActionsSettings::fromSettings(context_settings, CompileExpressions::yes); settings.process_list_element = from->getProcessListElement(); settings.progress_callback = from->getProgressCallback(); return settings; diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index 3b5e4e06953..0410bf925d1 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -19,6 +19,9 @@ struct BuildQueryPipelineSettings QueryStatusPtr process_list_element; ProgressCallback progress_callback = nullptr; + UInt64 partial_result_limit = 0; + UInt64 partial_result_duration_ms = 0; + const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } static BuildQueryPipelineSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 8054209c1c3..5d38bfb42c4 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -168,6 +168,8 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( QueryPipelineBuilderPtr last_pipeline; + bool has_partial_result_setting = build_pipeline_settings.partial_result_duration_ms > 0; + std::stack stack; stack.push(Frame{.node = root}); @@ -194,6 +196,9 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( } else stack.push(Frame{.node = frame.node->children[next_child]}); + + if (has_partial_result_setting && last_pipeline && !last_pipeline->isPartialResultActive()) + last_pipeline->activatePartialResult(build_pipeline_settings.partial_result_limit, build_pipeline_settings.partial_result_duration_ms); } last_pipeline->setProgressCallback(build_pipeline_settings.progress_callback); diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 371a24ac6f2..a72cab05754 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -27,6 +27,8 @@ public: size_t max_bytes_before_external_sort = 0; TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; + UInt64 partial_result_limit = 0; + UInt64 partial_result_duration_ms = 0; explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); diff --git a/src/Processors/Transforms/AggregatingPartialResultTransform.cpp b/src/Processors/Transforms/AggregatingPartialResultTransform.cpp new file mode 100644 index 00000000000..cf8ce72e096 --- /dev/null +++ b/src/Processors/Transforms/AggregatingPartialResultTransform.cpp @@ -0,0 +1,47 @@ +#include + +namespace DB +{ + +AggregatingPartialResultTransform::AggregatingPartialResultTransform( + const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_, + UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) + : PartialResultTransform(input_header, output_header, partial_result_limit_, partial_result_duration_ms_) + , aggregating_transform(std::move(aggregating_transform_)) + , transform_aggregator(input_header, aggregating_transform->params->params) + {} + +void AggregatingPartialResultTransform::transformPartialResult(Chunk & chunk) +{ + auto & params = aggregating_transform->params->params; + + bool no_more_keys = false; + AggregatedDataVariants variants; + ColumnRawPtrs key_columns(params.keys_size); + Aggregator::AggregateColumns aggregate_columns(params.aggregates_size); + + const UInt64 num_rows = chunk.getNumRows(); + transform_aggregator.executeOnBlock(chunk.detachColumns(), 0, num_rows, variants, key_columns, aggregate_columns, no_more_keys); + + auto transformed_block = transform_aggregator.convertToBlocks(variants, /*final*/ true, /*max_threads*/ 1).front(); + + chunk = convertToChunk(transformed_block); +} + +PartialResultTransform::ShaphotResult AggregatingPartialResultTransform::getRealProcessorSnapshot() +{ + std::lock_guard lock(aggregating_transform->snapshot_mutex); + if (aggregating_transform->is_generate_initialized) + return {{}, SnaphotStatus::Stopped}; + + if (aggregating_transform->variants.empty()) + return {{}, SnaphotStatus::NotReady}; + + auto & snapshot_aggregator = aggregating_transform->params->aggregator; + auto & snapshot_variants = aggregating_transform->many_data->variants; + auto block = snapshot_aggregator.prepareBlockAndFillWithoutKeySnapshot(*snapshot_variants.at(0)); + + return {convertToChunk(block), SnaphotStatus::Ready}; +} + +} diff --git a/src/Processors/Transforms/AggregatingPartialResultTransform.h b/src/Processors/Transforms/AggregatingPartialResultTransform.h new file mode 100644 index 00000000000..f7bac3a5394 --- /dev/null +++ b/src/Processors/Transforms/AggregatingPartialResultTransform.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class AggregatingPartialResultTransform : public PartialResultTransform +{ +public: + using AggregatingTransformPtr = std::shared_ptr; + + AggregatingPartialResultTransform( + const Block & input_header, const Block & output_header, AggregatingTransformPtr aggregating_transform_, + UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); + + String getName() const override { return "AggregatingPartialResultTransform"; } + + void transformPartialResult(Chunk & chunk) override; + ShaphotResult getRealProcessorSnapshot() override; + +private: + AggregatingTransformPtr aggregating_transform; + Aggregator transform_aggregator; +}; + +} diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 4bd000797a6..b4d2785bed2 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -657,6 +658,8 @@ void AggregatingTransform::consume(Chunk chunk) src_rows += num_rows; src_bytes += chunk.bytes(); + std::lock_guard lock(snapshot_mutex); + if (params->params.only_merge) { auto block = getInputs().front().getHeader().cloneWithColumns(chunk.detachColumns()); @@ -676,6 +679,7 @@ void AggregatingTransform::initGenerate() if (is_generate_initialized) return; + std::lock_guard lock(snapshot_mutex); is_generate_initialized = true; /// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation. @@ -806,4 +810,12 @@ void AggregatingTransform::initGenerate() } } +ProcessorPtr AggregatingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) +{ + const auto & input_header = inputs.front().getHeader(); + const auto & output_header = outputs.front().getHeader(); + auto aggregating_processor = std::dynamic_pointer_cast(current_processor); + return std::make_shared(input_header, output_header, std::move(aggregating_processor), partial_result_limit, partial_result_duration_ms); +} + } diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 38baa4d0394..791cd12326f 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -170,9 +170,23 @@ public: void work() override; Processors expandPipeline() override; + PartialResultStatus getPartialResultProcessorSupportStatus() const override + { + /// Currently AggregatingPartialResultTransform support only single-thread aggregation without key. + + /// TODO: check that insert results from aggregator.prepareBlockAndFillWithoutKey return values without + /// changing of the aggregator state when aggregation with keys will be supported in AggregatingPartialResultTransform. + bool is_partial_result_supported = params->params.keys_size == 0 /// Aggregation without key. + && many_data->variants.size() == 1; /// Use only one stream for aggregation. + + return is_partial_result_supported ? PartialResultStatus::FullSupported : PartialResultStatus::NotSupported; + } + protected: void consume(Chunk chunk); + ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; + private: /// To read the data that was flushed into the temporary data file. Processors processors; @@ -212,6 +226,13 @@ private: bool is_consume_started = false; + friend class AggregatingPartialResultTransform; + /// The mutex protects variables that are used for creating a snapshot of the current processor. + /// The current implementation of AggregatingPartialResultTransform uses the 'is_generate_initialized' variable to check + /// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots. + /// Additionally, the mutex protects the 'params->aggregator' and 'many_data->variants' variables, which are used to get data from them for a snapshot. + std::mutex snapshot_mutex; + void initGenerate(); }; diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 0d3341b000c..78dace56e4e 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -25,6 +25,12 @@ void ExpressionTransform::transform(Chunk & chunk) chunk.setColumns(block.getColumns(), num_rows); } +ProcessorPtr ExpressionTransform::getPartialResultProcessor(const ProcessorPtr & /*current_processor*/, UInt64 /*partial_result_limit*/, UInt64 /*partial_result_duration_ms*/) +{ + const auto & header = getInputPort().getHeader(); + return std::make_shared(header, expression); +} + ConvertingTransform::ConvertingTransform(const Block & header_, ExpressionActionsPtr expression_) : ExceptionKeepingTransform(header_, ExpressionTransform::transformHeader(header_, expression_->getActionsDAG())) , expression(std::move(expression_)) diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index 791c7d7ba73..8250f25f0f8 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -26,10 +26,15 @@ public: static Block transformHeader(Block header, const ActionsDAG & expression); + PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } + protected: void transform(Chunk & chunk) override; + ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; + private: + ExpressionActionsPtr expression; }; diff --git a/src/Processors/Transforms/LimitPartialResultTransform.cpp b/src/Processors/Transforms/LimitPartialResultTransform.cpp new file mode 100644 index 00000000000..c9eaa9dc7dd --- /dev/null +++ b/src/Processors/Transforms/LimitPartialResultTransform.cpp @@ -0,0 +1,42 @@ +#include +#include + +namespace DB +{ + +LimitPartialResultTransform::LimitPartialResultTransform( + const Block & header, + UInt64 partial_result_limit_, + UInt64 partial_result_duration_ms_, + UInt64 limit_, + UInt64 offset_) + : PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_) + , limit(limit_) + , offset(offset_) + {} + +void LimitPartialResultTransform::transformPartialResult(Chunk & chunk) +{ + UInt64 num_rows = chunk.getNumRows(); + if (num_rows < offset || limit == 0) + { + chunk = {}; + return; + } + + UInt64 length = std::min(limit, num_rows - offset); + + /// Check if some rows should be removed + if (length < num_rows) + { + UInt64 num_columns = chunk.getNumColumns(); + auto columns = chunk.detachColumns(); + + for (UInt64 i = 0; i < num_columns; ++i) + columns[i] = columns[i]->cut(offset, length); + + chunk.setColumns(std::move(columns), length); + } +} + +} diff --git a/src/Processors/Transforms/LimitPartialResultTransform.h b/src/Processors/Transforms/LimitPartialResultTransform.h new file mode 100644 index 00000000000..3a0116b624d --- /dev/null +++ b/src/Processors/Transforms/LimitPartialResultTransform.h @@ -0,0 +1,36 @@ +#pragma once + +#include + +namespace DB +{ + +class LimitTransform; + +/// Currently support only single thread implementation with one input and one output ports +class LimitPartialResultTransform : public PartialResultTransform +{ +public: + using LimitTransformPtr = std::shared_ptr; + + LimitPartialResultTransform( + const Block & header, + UInt64 partial_result_limit_, + UInt64 partial_result_duration_ms_, + UInt64 limit_, + UInt64 offset_); + + String getName() const override { return "LimitPartialResultTransform"; } + + void transformPartialResult(Chunk & chunk) override; + /// LimitsTransform doesn't have a state which can be snapshoted + ShaphotResult getRealProcessorSnapshot() override { return {{}, SnaphotStatus::Stopped}; } + +private: + UInt64 limit; + UInt64 offset; + + LimitTransformPtr limit_transform; +}; + +} diff --git a/src/Processors/Transforms/LimitsCheckingTransform.cpp b/src/Processors/Transforms/LimitsCheckingTransform.cpp index 02d2fef808c..0557f3f291e 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB diff --git a/src/Processors/Transforms/LimitsCheckingTransform.h b/src/Processors/Transforms/LimitsCheckingTransform.h index 2f96a17c17b..eabb988dab6 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/src/Processors/Transforms/LimitsCheckingTransform.h @@ -33,6 +33,8 @@ public: void setQuota(const std::shared_ptr & quota_) { quota = quota_; } + PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::SkipSupported; } + protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp b/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp new file mode 100644 index 00000000000..e4a2af2cdd8 --- /dev/null +++ b/src/Processors/Transforms/MergeSortingPartialResultTransform.cpp @@ -0,0 +1,48 @@ +#include + +namespace DB +{ + +MergeSortingPartialResultTransform::MergeSortingPartialResultTransform( + const Block & header, MergeSortingTransformPtr merge_sorting_transform_, + UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) + : PartialResultTransform(header, partial_result_limit_, partial_result_duration_ms_) + , merge_sorting_transform(std::move(merge_sorting_transform_)) + {} + +PartialResultTransform::ShaphotResult MergeSortingPartialResultTransform::getRealProcessorSnapshot() +{ + std::lock_guard lock(merge_sorting_transform->snapshot_mutex); + if (merge_sorting_transform->generated_prefix) + return {{}, SnaphotStatus::Stopped}; + + if (merge_sorting_transform->chunks.empty()) + return {{}, SnaphotStatus::NotReady}; + + /// Sort all input data + merge_sorting_transform->remerge(); + /// Add a copy of the first `partial_result_limit` rows to a generated_chunk + /// to send it later as a partial result in the next prepare stage of the current processor + auto generated_columns = merge_sorting_transform->chunks[0].cloneEmptyColumns(); + size_t total_rows = 0; + for (const auto & merged_chunk : merge_sorting_transform->chunks) + { + size_t rows = std::min(merged_chunk.getNumRows(), partial_result_limit - total_rows); + if (rows == 0) + break; + + for (size_t position = 0; position < generated_columns.size(); ++position) + { + auto column = merged_chunk.getColumns()[position]; + generated_columns[position]->insertRangeFrom(*column, 0, rows); + } + + total_rows += rows; + } + + auto partial_result = Chunk(std::move(generated_columns), total_rows, merge_sorting_transform->chunks[0].getChunkInfo()); + merge_sorting_transform->enrichChunkWithConstants(partial_result); + return {std::move(partial_result), SnaphotStatus::Ready}; +} + +} diff --git a/src/Processors/Transforms/MergeSortingPartialResultTransform.h b/src/Processors/Transforms/MergeSortingPartialResultTransform.h new file mode 100644 index 00000000000..781aa8e1265 --- /dev/null +++ b/src/Processors/Transforms/MergeSortingPartialResultTransform.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class MergeSortingPartialResultTransform : public PartialResultTransform +{ +public: + using MergeSortingTransformPtr = std::shared_ptr; + + MergeSortingPartialResultTransform( + const Block & header, MergeSortingTransformPtr merge_sorting_transform_, + UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); + + String getName() const override { return "MergeSortingPartialResultTransform"; } + + /// MergeSortingTransform always receives chunks in a sorted state, so transformation is not needed + void transformPartialResult(Chunk & /*chunk*/) override {} + ShaphotResult getRealProcessorSnapshot() override; + +private: + MergeSortingTransformPtr merge_sorting_transform; +}; + +} diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index de77711d129..e801e5e16d5 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -136,6 +137,8 @@ void MergeSortingTransform::consume(Chunk chunk) /// If there were only const columns in sort description, then there is no need to sort. /// Return the chunk as is. + std::lock_guard lock(snapshot_mutex); + if (description.empty()) { generated_chunk = std::move(chunk); @@ -213,6 +216,8 @@ void MergeSortingTransform::serialize() void MergeSortingTransform::generate() { + std::lock_guard lock(snapshot_mutex); + if (!generated_prefix) { size_t num_tmp_files = tmp_data ? tmp_data->getStreams().size() : 0; @@ -273,4 +278,11 @@ void MergeSortingTransform::remerge() sum_bytes_in_blocks = new_sum_bytes_in_blocks; } +ProcessorPtr MergeSortingTransform::getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) +{ + const auto & header = inputs.front().getHeader(); + auto merge_sorting_processor = std::dynamic_pointer_cast(current_processor); + return std::make_shared(header, std::move(merge_sorting_processor), partial_result_limit, partial_result_duration_ms); +} + } diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index e8c180b6903..67f098b4362 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -33,6 +33,8 @@ public: String getName() const override { return "MergeSortingTransform"; } + PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } + protected: void consume(Chunk chunk) override; void serialize() override; @@ -40,6 +42,8 @@ protected: Processors expandPipeline() override; + ProcessorPtr getPartialResultProcessor(const ProcessorPtr & current_processor, UInt64 partial_result_limit, UInt64 partial_result_duration_ms) override; + private: size_t max_bytes_before_remerge; double remerge_lowered_memory_bytes_ratio; @@ -59,6 +63,13 @@ private: void remerge(); ProcessorPtr external_merging_sorted; + + friend class MergeSortingPartialResultTransform; + /// The mutex protects variables that are used for creating a snapshot of the current processor. + /// The current implementation of MergeSortingPartialResultTransform uses the 'generated_prefix' variable to check + /// whether the processor has started sending data through the main pipeline, and the corresponding partial result processor should stop creating snapshots. + /// Additionally, the mutex protects the 'chunks' variable and all variables in the 'remerge' function, which is used to transition 'chunks' to a sorted state. + std::mutex snapshot_mutex; }; } diff --git a/src/Processors/Transforms/PartialResultTransform.cpp b/src/Processors/Transforms/PartialResultTransform.cpp new file mode 100644 index 00000000000..97ff79dee54 --- /dev/null +++ b/src/Processors/Transforms/PartialResultTransform.cpp @@ -0,0 +1,80 @@ +#include + +namespace DB +{ + + +PartialResultTransform::PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) + : PartialResultTransform(header, header, partial_result_limit_, partial_result_duration_ms_) {} + +PartialResultTransform::PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) + : IProcessor({input_header}, {output_header}) + , input(inputs.front()) + , output(outputs.front()) + , partial_result_limit(partial_result_limit_) + , partial_result_duration_ms(partial_result_duration_ms_) + , watch(CLOCK_MONOTONIC) + {} + +IProcessor::Status PartialResultTransform::prepare() +{ + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + + if (finished_getting_snapshots) + { + output.finish(); + return Status::Finished; + } + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + /// If input data from previous partial result processor is finished then + /// PartialResultTransform ready to create snapshots and send them as a partial result + if (input.isFinished()) + { + if (partial_result.snapshot_status == SnaphotStatus::Ready) + { + partial_result.snapshot_status = SnaphotStatus::NotReady; + output.push(std::move(partial_result.chunk)); + return Status::PortFull; + } + + return Status::Ready; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + partial_result.chunk = input.pull(); + transformPartialResult(partial_result.chunk); + if (partial_result.chunk.getNumRows() > 0) + { + output.push(std::move(partial_result.chunk)); + return Status::PortFull; + } + + return Status::NeedData; +} + +void PartialResultTransform::work() +{ + if (partial_result_duration_ms < watch.elapsedMilliseconds()) + { + partial_result = getRealProcessorSnapshot(); + if (partial_result.snapshot_status == SnaphotStatus::Stopped) + finished_getting_snapshots = true; + + watch.restart(); + } +} + +} diff --git a/src/Processors/Transforms/PartialResultTransform.h b/src/Processors/Transforms/PartialResultTransform.h new file mode 100644 index 00000000000..4fe87638f38 --- /dev/null +++ b/src/Processors/Transforms/PartialResultTransform.h @@ -0,0 +1,57 @@ +#pragma once + +#include + +namespace DB +{ + +/// Processors of this type are used to construct an auxiliary pipeline with processors corresponding to those in the main pipeline. +/// These processors work in two modes: +/// 1) Creating a snapshot of the corresponding processor from the main pipeline once per partial_result_duration_ms (period in milliseconds), and then sending the snapshot through the partial result pipeline. +/// 2) Transforming small blocks of data in the same way as the original processor and sending the transformed data through the partial result pipeline. +/// All processors of this type rely on the invariant that a new block from the previous processor of the partial result pipeline overwrites information about the previous block of the same previous processor. +class PartialResultTransform : public IProcessor +{ +public: + PartialResultTransform(const Block & header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); + PartialResultTransform(const Block & input_header, const Block & output_header, UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); + + String getName() const override { return "PartialResultTransform"; } + + Status prepare() override; + void work() override; + + bool isPartialResultProcessor() const override { return true; } + +protected: + enum class SnaphotStatus + { + NotReady, // Waiting for data from the previous partial result processor or awaiting a timer before creating the snapshot. + Ready, // Current partial result processor has received a snapshot from the processor in the main pipeline. + Stopped, // The processor from the main pipeline has started sending data, and the pipeline for partial results should use data from the next processors of the main pipeline. + }; + + struct ShaphotResult + { + Chunk chunk; + SnaphotStatus snapshot_status; + }; + + InputPort & input; + OutputPort & output; + + UInt64 partial_result_limit; + UInt64 partial_result_duration_ms; + + ShaphotResult partial_result = {{}, SnaphotStatus::NotReady}; + + bool finished_getting_snapshots = false; + + virtual void transformPartialResult(Chunk & /*chunk*/) = 0; + virtual ShaphotResult getRealProcessorSnapshot() = 0; // { return {{}, SnaphotStatus::Stopped}; } + +private: + Stopwatch watch; +}; + +} diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 91ba01c479f..293d152ea65 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -167,12 +168,9 @@ Pipe::Pipe(ProcessorPtr source) { checkSource(*source); - if (collected_processors) - collected_processors->emplace_back(source); - output_ports.push_back(&source->getOutputs().front()); header = output_ports.front()->getHeader(); - processors->emplace_back(std::move(source)); + addProcessor(std::move(source)); max_parallel_streams = 1; } @@ -319,6 +317,16 @@ Pipe Pipe::unitePipes(Pipes pipes, Processors * collected_processors, bool allow res.processors->insert(res.processors->end(), pipe.processors->begin(), pipe.processors->end()); res.output_ports.insert(res.output_ports.end(), pipe.output_ports.begin(), pipe.output_ports.end()); + if (res.isPartialResultActive() && pipe.isPartialResultActive()) + { + res.partial_result_ports.insert( + res.partial_result_ports.end(), + pipe.partial_result_ports.begin(), + pipe.partial_result_ports.end()); + } + else + res.dropPartialResult(); + res.max_parallel_streams += pipe.max_parallel_streams; if (pipe.totals_port) @@ -352,11 +360,11 @@ void Pipe::addSource(ProcessorPtr source) else assertBlocksHaveEqualStructure(header, source_header, "Pipes"); - if (collected_processors) - collected_processors->emplace_back(source); - output_ports.push_back(&source->getOutputs().front()); - processors->emplace_back(std::move(source)); + if (isPartialResultActive()) + partial_result_ports.push_back(nullptr); + + addProcessor(std::move(source)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -374,11 +382,9 @@ void Pipe::addTotalsSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); - if (collected_processors) - collected_processors->emplace_back(source); - totals_port = &source->getOutputs().front(); - processors->emplace_back(std::move(source)); + + addProcessor(std::move(source)); } void Pipe::addExtremesSource(ProcessorPtr source) @@ -394,11 +400,20 @@ void Pipe::addExtremesSource(ProcessorPtr source) assertBlocksHaveEqualStructure(header, source_header, "Pipes"); - if (collected_processors) - collected_processors->emplace_back(source); - extremes_port = &source->getOutputs().front(); - processors->emplace_back(std::move(source)); + + addProcessor(std::move(source)); +} + +void Pipe::activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_) +{ + if (is_partial_result_active) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Partial result for Pipe should be initialized only once"); + + is_partial_result_active = true; + partial_result_limit = partial_result_limit_; + partial_result_duration_ms = partial_result_duration_ms_; + partial_result_ports.assign(output_ports.size(), nullptr); } static void dropPort(OutputPort *& port, Processors & processors, Processors * collected_processors) @@ -426,6 +441,15 @@ void Pipe::dropExtremes() dropPort(extremes_port, *processors, collected_processors); } +void Pipe::dropPartialResult() +{ + for (auto & port : partial_result_ports) + dropPort(port, *processors, collected_processors); + + is_partial_result_active = false; + partial_result_ports.clear(); +} + void Pipe::addTransform(ProcessorPtr transform) { addTransform(std::move(transform), static_cast(nullptr), static_cast(nullptr)); @@ -456,6 +480,8 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort if (extremes) extremes_port = extremes; + addPartialResultTransform(transform); + size_t next_output = 0; for (auto & input : inputs) { @@ -506,10 +532,7 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - if (collected_processors) - collected_processors->emplace_back(transform); - - processors->emplace_back(std::move(transform)); + addProcessor(std::move(transform)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } @@ -546,6 +569,8 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes_port = nullptr; } + addPartialResultTransform(transform); + bool found_totals = false; bool found_extremes = false; @@ -595,14 +620,104 @@ void Pipe::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * if (extremes_port) assertBlocksHaveEqualStructure(header, extremes_port->getHeader(), "Pipes"); - if (collected_processors) - collected_processors->emplace_back(transform); - - processors->emplace_back(std::move(transform)); + addProcessor(std::move(transform)); max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); } +void Pipe::addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id) +{ + if (isPartialResultActive()) + { + auto & partial_result_port = partial_result_ports[partial_result_port_id]; + auto partial_result_status = transform->getPartialResultProcessorSupportStatus(); + + if (partial_result_status == IProcessor::PartialResultStatus::NotSupported) + dropPort(partial_result_port, *processors, collected_processors); + + if (partial_result_status != IProcessor::PartialResultStatus::FullSupported) + return; + + auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms); + + connectPartialResultPort(partial_result_port, partial_result_transform->getInputs().front()); + + partial_result_port = &partial_result_transform->getOutputs().front(); + + addProcessor(std::move(partial_result_transform)); + } +} + +void Pipe::addPartialResultTransform(const ProcessorPtr & transform) +{ + if (isPartialResultActive()) + { + size_t new_outputs_size = transform->getOutputs().size(); + auto partial_result_status = transform->getPartialResultProcessorSupportStatus(); + + if (partial_result_status == IProcessor::PartialResultStatus::SkipSupported && new_outputs_size != partial_result_ports.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot skip transform {} in the partial result part of the Pipe because it has {} output ports, but the partial result part expects {} output ports", + transform->getName(), + new_outputs_size, + partial_result_ports.size()); + + if (partial_result_status == IProcessor::PartialResultStatus::NotSupported) + { + for (auto & partial_result_port : partial_result_ports) + dropPort(partial_result_port, *processors, collected_processors); + + partial_result_ports.assign(new_outputs_size, nullptr); + } + + if (partial_result_status != IProcessor::PartialResultStatus::FullSupported) + return; + + auto partial_result_transform = IProcessor::getPartialResultProcessorPtr(transform, partial_result_limit, partial_result_duration_ms); + auto & inputs = partial_result_transform->getInputs(); + + if (inputs.size() != partial_result_ports.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot add partial result transform {} to Pipe because it has {} input ports, but {} expected", + partial_result_transform->getName(), + inputs.size(), + partial_result_ports.size()); + + size_t next_port = 0; + for (auto & input : inputs) + { + connectPartialResultPort(partial_result_ports[next_port], input); + ++next_port; + } + + partial_result_ports.assign(new_outputs_size, nullptr); + + next_port = 0; + for (auto & new_partial_result_port : partial_result_transform->getOutputs()) + { + partial_result_ports[next_port] = &new_partial_result_port; + ++next_port; + } + + addProcessor(std::move(partial_result_transform)); + } +} + +void Pipe::connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port) +{ + if (partial_result_port == nullptr) + { + auto source = std::make_shared(getHeader()); + partial_result_port = &source->getPort(); + + addProcessor(std::move(source)); + } + + connect(*partial_result_port, partial_result_transform_port); +} + void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { if (output_ports.empty()) @@ -610,7 +725,7 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) Block new_header; - auto add_transform = [&](OutputPort *& port, StreamType stream_type) + auto add_transform = [&](OutputPort *& port, size_t partial_result_port_id, StreamType stream_type) { if (!port) return; @@ -646,19 +761,22 @@ void Pipe::addSimpleTransform(const ProcessorGetterWithStreamKind & getter) { connect(*port, transform->getInputs().front()); port = &transform->getOutputs().front(); + if (stream_type == StreamType::Main) + addPartialResultSimpleTransform(transform, partial_result_port_id); - if (collected_processors) - collected_processors->emplace_back(transform); - - processors->emplace_back(std::move(transform)); + addProcessor(std::move(transform)); } }; + size_t partial_result_port_id = 0; for (auto & port : output_ports) - add_transform(port, StreamType::Main); + { + add_transform(port, partial_result_port_id, StreamType::Main); + ++partial_result_port_id; + } - add_transform(totals_port, StreamType::Totals); - add_transform(extremes_port, StreamType::Extremes); + add_transform(totals_port, 0, StreamType::Totals); + add_transform(extremes_port, 0, StreamType::Extremes); header = std::move(new_header); } @@ -679,6 +797,7 @@ void Pipe::addChains(std::vector chains) dropTotals(); dropExtremes(); + dropPartialResult(); size_t max_parallel_streams_for_chains = 0; @@ -697,18 +816,21 @@ void Pipe::addChains(std::vector chains) auto added_processors = Chain::getProcessors(std::move(chains[i])); for (auto & transform : added_processors) - { - if (collected_processors) - collected_processors->emplace_back(transform); - - processors->emplace_back(std::move(transform)); - } + addProcessor(std::move(transform)); } header = std::move(new_header); max_parallel_streams = std::max(max_parallel_streams, max_parallel_streams_for_chains); } +void Pipe::addProcessor(ProcessorPtr processor) +{ + if (collected_processors) + collected_processors->emplace_back(processor); + + processors->emplace_back(std::move(processor)); +} + void Pipe::resize(size_t num_streams, bool force, bool strict) { if (output_ports.empty()) @@ -769,6 +891,9 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) add_transform(totals_port, StreamType::Totals); add_transform(extremes_port, StreamType::Extremes); + for (auto & port : partial_result_ports) + add_transform(port, StreamType::PartialResult); + output_ports.clear(); header.clear(); } @@ -778,6 +903,9 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) if (output_ports.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot transform empty Pipe"); + /// TODO: Add functionality to work with partial result ports in transformer. + dropPartialResult(); + auto new_processors = transformer(output_ports); /// Create hash table with new processors. diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 09931e38578..70e933bcfd2 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -48,6 +48,9 @@ public: OutputPort * getOutputPort(size_t pos) const { return output_ports[pos]; } OutputPort * getTotalsPort() const { return totals_port; } OutputPort * getExtremesPort() const { return extremes_port; } + OutputPort * getPartialResultPort(size_t pos) const { return partial_result_ports.empty() ? nullptr : partial_result_ports[pos]; } + + bool isPartialResultActive() { return is_partial_result_active; } /// Add processor to list, add it output ports to output_ports. /// Processor shouldn't have input ports, output ports shouldn't be connected. @@ -58,9 +61,13 @@ public: void addTotalsSource(ProcessorPtr source); void addExtremesSource(ProcessorPtr source); - /// Drop totals and extremes (create NullSink for them). + /// Activate sending partial result during main pipeline execution + void activatePartialResult(UInt64 partial_result_limit_, UInt64 partial_result_duration_ms_); + + /// Drop totals, extremes and partial result (create NullSink for them). void dropTotals(); void dropExtremes(); + void dropPartialResult(); /// Add processor to list. It should have size() input ports with compatible header. /// Output ports should have same headers. @@ -69,11 +76,16 @@ public: void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); + void addPartialResultTransform(const ProcessorPtr & transform); + void addPartialResultSimpleTransform(const ProcessorPtr & transform, size_t partial_result_port_id); + void connectPartialResultPort(OutputPort * partial_result_port, InputPort & partial_result_transform_port); + enum class StreamType { Main = 0, /// Stream for query data. There may be several streams of this type. Totals, /// Stream for totals. No more than one. Extremes, /// Stream for extremes. No more than one. + PartialResult, /// Stream for partial result data. There may be several streams of this type. }; using ProcessorGetter = std::function; @@ -109,10 +121,17 @@ private: Block header; std::shared_ptr processors; - /// Output ports. Totals and extremes are allowed to be empty. + /// If the variable is true, then each time a processor is added pipe will try + /// to add processor which will send partial result from original processor + bool is_partial_result_active = false; + UInt64 partial_result_limit = 0; + UInt64 partial_result_duration_ms = 0; + + /// Output ports. Totals, extremes and partial results are allowed to be empty. OutputPortRawPtrs output_ports; OutputPort * totals_port = nullptr; OutputPort * extremes_port = nullptr; + OutputPortRawPtrs partial_result_ports; /// It is the max number of processors which can be executed in parallel for each step. /// Usually, it's the same as the number of output ports. @@ -128,6 +147,8 @@ private: static Pipe unitePipes(Pipes pipes, Processors * collected_processors, bool allow_empty_header); void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); + void addProcessor(ProcessorPtr processor); + friend class QueryPipelineBuilder; friend class QueryPipeline; }; diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 128972b8ff0..9a836f68da1 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -66,7 +66,8 @@ static void checkPulling( Processors & processors, OutputPort * output, OutputPort * totals, - OutputPort * extremes) + OutputPort * extremes, + OutputPort * partial_result) { if (!output || output->isConnected()) throw Exception( @@ -83,9 +84,15 @@ static void checkPulling( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its extremes port is connected"); + if (partial_result && partial_result->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its partial_result port is connected"); + bool found_output = false; bool found_totals = false; bool found_extremes = false; + bool found_partial_result = false; for (const auto & processor : processors) { for (const auto & in : processor->getInputs()) @@ -99,6 +106,8 @@ static void checkPulling( found_totals = true; else if (extremes && &out == extremes) found_extremes = true; + else if (partial_result && &out == partial_result) + found_partial_result = true; else checkOutput(out, processor); } @@ -116,6 +125,10 @@ static void checkPulling( throw Exception( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its extremes port does not belong to any processor"); + if (partial_result && !found_partial_result) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its partial result port does not belong to any processor"); } static void checkCompleted(Processors & processors) @@ -318,17 +331,20 @@ QueryPipeline::QueryPipeline( std::shared_ptr processors_, OutputPort * output_, OutputPort * totals_, - OutputPort * extremes_) + OutputPort * extremes_, + OutputPort * partial_result_) : resources(std::move(resources_)) , processors(std::move(processors_)) , output(output_) , totals(totals_) , extremes(extremes_) + , partial_result(partial_result_) { - checkPulling(*processors, output, totals, extremes); + checkPulling(*processors, output, totals, extremes, partial_result); } QueryPipeline::QueryPipeline(Pipe pipe) + : partial_result_duration_ms(pipe.partial_result_duration_ms) { if (pipe.numOutputPorts() > 0) { @@ -336,9 +352,10 @@ QueryPipeline::QueryPipeline(Pipe pipe) output = pipe.getOutputPort(0); totals = pipe.getTotalsPort(); extremes = pipe.getExtremesPort(); + partial_result = pipe.getPartialResultPort(0); processors = std::move(pipe.processors); - checkPulling(*processors, output, totals, extremes); + checkPulling(*processors, output, totals, extremes, partial_result); } else { @@ -370,6 +387,7 @@ QueryPipeline::QueryPipeline(std::shared_ptr format) auto & format_main = format->getPort(IOutputFormat::PortKind::Main); auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); + auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult); if (!totals) { @@ -385,12 +403,21 @@ QueryPipeline::QueryPipeline(std::shared_ptr format) processors->emplace_back(std::move(source)); } + if (!partial_result) + { + auto source = std::make_shared(format_partial_result.getHeader()); + partial_result = &source->getPort(); + processors->emplace_back(std::move(source)); + } + connect(*totals, format_totals); connect(*extremes, format_extremes); + connect(*partial_result, format_partial_result); input = &format_main; totals = nullptr; extremes = nullptr; + partial_result = nullptr; output_format = format.get(); @@ -418,6 +445,7 @@ void QueryPipeline::complete(std::shared_ptr sink) drop(totals, *processors); drop(extremes, *processors); + drop(partial_result, *processors); connect(*output, sink->getPort()); processors->emplace_back(std::move(sink)); @@ -433,6 +461,7 @@ void QueryPipeline::complete(Chain chain) drop(totals, *processors); drop(extremes, *processors); + drop(partial_result, *processors); processors->reserve(processors->size() + chain.getProcessors().size() + 1); for (auto processor : chain.getProcessors()) @@ -458,6 +487,7 @@ void QueryPipeline::complete(Pipe pipe) pipe.resize(1); pipe.dropExtremes(); pipe.dropTotals(); + pipe.dropPartialResult(); connect(*pipe.getOutputPort(0), *input); input = nullptr; @@ -486,11 +516,13 @@ void QueryPipeline::complete(std::shared_ptr format) addMaterializing(output, *processors); addMaterializing(totals, *processors); addMaterializing(extremes, *processors); + addMaterializing(partial_result, *processors); } auto & format_main = format->getPort(IOutputFormat::PortKind::Main); auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); + auto & format_partial_result = format->getPort(IOutputFormat::PortKind::PartialResult); if (!totals) { @@ -506,13 +538,22 @@ void QueryPipeline::complete(std::shared_ptr format) processors->emplace_back(std::move(source)); } + if (!partial_result) + { + auto source = std::make_shared(format_partial_result.getHeader()); + partial_result = &source->getPort(); + processors->emplace_back(std::move(source)); + } + connect(*output, format_main); connect(*totals, format_totals); connect(*extremes, format_extremes); + connect(*partial_result, format_partial_result); output = nullptr; totals = nullptr; extremes = nullptr; + partial_result = nullptr; initRowsBeforeLimit(format.get()); output_format = format.get(); @@ -684,6 +725,7 @@ void QueryPipeline::convertStructureTo(const ColumnsWithTypeAndName & columns) addExpression(output, actions, *processors); addExpression(totals, actions, *processors); addExpression(extremes, actions, *processors); + addExpression(partial_result, actions, *processors); } std::unique_ptr QueryPipeline::getReadProgressCallback() const diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h index f14cf61aac2..20e58bc0f59 100644 --- a/src/QueryPipeline/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -75,7 +75,8 @@ public: std::shared_ptr processors_, OutputPort * output_, OutputPort * totals_ = nullptr, - OutputPort * extremes_ = nullptr); + OutputPort * extremes_ = nullptr, + OutputPort * partial_result_ = nullptr); bool initialized() const { return !processors->empty(); } /// When initialized, exactly one of the following is true. @@ -154,6 +155,7 @@ private: OutputPort * output = nullptr; OutputPort * totals = nullptr; OutputPort * extremes = nullptr; + OutputPort * partial_result = nullptr; QueryStatusPtr process_list_element; @@ -162,6 +164,9 @@ private: size_t num_threads = 0; bool concurrency_control = false; + UInt64 partial_result_limit = 0; + UInt64 partial_result_duration_ms = 0; + friend class PushingPipelineExecutor; friend class PullingPipelineExecutor; friend class PushingAsyncPipelineExecutor; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index f9726339872..90f5ee364f3 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -110,6 +110,16 @@ void QueryPipelineBuilder::init(QueryPipeline & pipeline) pipe.header = {}; } + if (pipeline.partial_result) + { + /// Set partial result ports only after activation because when activated, it is set to nullptr + pipe.activatePartialResult(pipeline.partial_result_limit, pipeline.partial_result_duration_ms); + pipe.partial_result_ports = {pipeline.partial_result}; + } + + if (!pipeline.partial_result) + pipe.dropPartialResult(); + pipe.totals_port = pipeline.totals; pipe.extremes_port = pipeline.extremes; pipe.max_parallel_streams = pipeline.num_threads; @@ -352,6 +362,10 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped left->checkInitializedAndNotCompleted(); right->checkInitializedAndNotCompleted(); + /// TODO: Support joining of partial results from different pipelines. + left->pipe.dropPartialResult(); + right->pipe.dropPartialResult(); + left->pipe.dropExtremes(); right->pipe.dropExtremes(); if (left->getNumStreams() != 1 || right->getNumStreams() != 1) @@ -364,6 +378,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped auto joining = std::make_shared(join, inputs, out_header, max_block_size); + /// TODO: Support partial results in merge pipelines after joining support above. return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } @@ -384,6 +399,10 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe left->pipe.dropExtremes(); right->pipe.dropExtremes(); + /// TODO: Support joining of partial results from different pipelines. + left->pipe.dropPartialResult(); + right->pipe.dropPartialResult(); + left->pipe.collected_processors = collected_processors; /// Collect the NEW processors for the right pipeline. @@ -634,7 +653,7 @@ PipelineExecutorPtr QueryPipelineBuilder::execute() if (!isCompleted()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot execute pipeline because it is not completed"); - return std::make_shared(pipe.processors, process_list_element); + return std::make_shared(pipe.processors, process_list_element, pipe.partial_result_duration_ms); } Pipe QueryPipelineBuilder::getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 5d273df7068..612e7b1652f 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -85,6 +85,12 @@ public: /// Pipeline will be completed after this transformation. void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); + /// Activate building separate pipeline for sending partial result. + void activatePartialResult(UInt64 partial_result_limit, UInt64 partial_result_duration_ms) { pipe.activatePartialResult(partial_result_limit, partial_result_duration_ms); } + + /// Check if building of a pipeline for sending partial result active. + bool isPartialResultActive() { return pipe.isPartialResultActive(); } + /// Add totals which returns one chunk with single row with defaults. void addDefaultTotals(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 96c449b7e17..c687a6064b4 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -885,7 +885,8 @@ void TCPHandler::processOrdinaryQueryWithProcessors() std::unique_lock progress_lock(task_callback_mutex, std::defer_lock); { - PullingAsyncPipelineExecutor executor(pipeline); + bool has_partial_result_setting = query_context->getSettingsRef().partial_result_update_duration_ms.totalMilliseconds() > 0; + PullingAsyncPipelineExecutor executor(pipeline, has_partial_result_setting); CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread}; Block block; diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index 6c4220855c8..219fdf04472 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -1,227 +1,33 @@ #!/usr/bin/env python3 # -*- coding: utf-8 -*- -import socket import os -import uuid +import sys -CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") -CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) -CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") -CLIENT_NAME = "simple native protocol" +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -def writeVarUInt(x, ba): - for _ in range(0, 9): - byte = x & 0x7F - if x > 0x7F: - byte |= 0x80 - - ba.append(byte) - - x >>= 7 - if x == 0: - return - - -def writeStringBinary(s, ba): - b = bytes(s, "utf-8") - writeVarUInt(len(s), ba) - ba.extend(b) - - -def readStrict(s, size=1): - res = bytearray() - while size: - cur = s.recv(size) - # if not res: - # raise "Socket is closed" - size -= len(cur) - res.extend(cur) - - return res - - -def readUInt(s, size=1): - res = readStrict(s, size) - val = 0 - for i in range(len(res)): - val += res[i] << (i * 8) - return val - - -def readUInt8(s): - return readUInt(s) - - -def readUInt16(s): - return readUInt(s, 2) - - -def readUInt32(s): - return readUInt(s, 4) - - -def readUInt64(s): - return readUInt(s, 8) - - -def readVarUInt(s): - x = 0 - for i in range(9): - byte = readStrict(s)[0] - x |= (byte & 0x7F) << (7 * i) - - if not byte & 0x80: - return x - - return x - - -def readStringBinary(s): - size = readVarUInt(s) - s = readStrict(s, size) - return s.decode("utf-8") - - -def sendHello(s): - ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary(CLIENT_NAME, ba) - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary("default", ba) # database - writeStringBinary("default", ba) # user - writeStringBinary("", ba) # pwd - s.sendall(ba) - - -def receiveHello(s): - p_type = readVarUInt(s) - assert p_type == 0 # Hello - server_name = readStringBinary(s) - # print("Server name: ", server_name) - server_version_major = readVarUInt(s) - # print("Major: ", server_version_major) - server_version_minor = readVarUInt(s) - # print("Minor: ", server_version_minor) - server_revision = readVarUInt(s) - # print("Revision: ", server_revision) - server_timezone = readStringBinary(s) - # print("Timezone: ", server_timezone) - server_display_name = readStringBinary(s) - # print("Display name: ", server_display_name) - server_version_patch = readVarUInt(s) - # print("Version patch: ", server_version_patch) - - -def serializeClientInfo(ba, query_id): - writeStringBinary("default", ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary("127.0.0.1:9000", ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary("os_user", ba) # os_user - writeStringBinary("client_hostname", ba) # client_hostname - writeStringBinary(CLIENT_NAME, ba) # client_name - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary("", ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry - - -def sendQuery(s, query): - ba = bytearray() - query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query - writeStringBinary(query_id, ba) - - ba.append(1) # INITIAL_QUERY - - # client info - serializeClientInfo(ba, query_id) - - writeStringBinary("", ba) # No settings - writeStringBinary("", ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally - s.sendall(ba) - - -def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num - - -def sendEmptyBlock(s): - ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary("", ba) - serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns - s.sendall(ba) - - -def assertPacket(packet, expected): - assert packet == expected, packet - - -def readHeader(s): - packet_type = readVarUInt(s) - if packet_type == 2: # Exception - raise RuntimeError(readException(s)) - assertPacket(packet_type, 1) # Data - - readStringBinary(s) # external table name - # BlockInfo - assertPacket(readVarUInt(s), 1) # 1 - assertPacket(readUInt8(s), 0) # is_overflows - assertPacket(readVarUInt(s), 2) # 2 - assertPacket(readUInt32(s), 4294967295) # bucket_num - assertPacket(readVarUInt(s), 0) # 0 - columns = readVarUInt(s) # rows - rows = readVarUInt(s) # columns - print("Rows {} Columns {}".format(rows, columns)) - for _ in range(columns): - col_name = readStringBinary(s) - type_name = readStringBinary(s) - print("Column {} type {}".format(col_name, type_name)) - - -def readException(s): - code = readUInt32(s) - name = readStringBinary(s) - text = readStringBinary(s) - readStringBinary(s) # trace - assertPacket(readUInt8(s), 0) # has_nested - return "code {}: {}".format(code, text.replace("DB::Exception:", "")) +from tcp_client import ( + TCPClient, + CLICKHOUSE_DATABASE, + writeVarUInt, + writeStringBinary, + serializeBlockInfo, + assertPacket, +) def insertValidLowCardinalityRow(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) - sendQuery( - s, + with TCPClient() as client: + client.sendQuery( "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - sendEmptyBlock(s) - readHeader(s) + client.sendEmptyBlock() + client.readHeader() # Data ba = bytearray() @@ -240,31 +46,25 @@ def insertValidLowCardinalityRow(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (0 for 'hello') - s.sendall(ba) + client.send(ba) # Fin block - sendEmptyBlock(s) + client.sendEmptyBlock() - assertPacket(readVarUInt(s), 5) # End of stream - s.close() + assertPacket(client.readVarUInt(), 5) # End of stream def insertLowCardinalityRowWithIndexOverflow(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) - sendQuery( - s, + with TCPClient() as client: + client.sendQuery( "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - sendEmptyBlock(s) - readHeader(s) + client.sendEmptyBlock() + client.readHeader() # Data ba = bytearray() @@ -283,29 +83,23 @@ def insertLowCardinalityRowWithIndexOverflow(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 7 + [1]) # UInt64 index (overflow) - s.sendall(ba) + client.send(ba) - assertPacket(readVarUInt(s), 2) - print(readException(s)) - s.close() + assertPacket(client.readVarUInt(), 2) # Exception + print(client.readException()) def insertLowCardinalityRowWithIncorrectDictType(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) - sendQuery( - s, + with TCPClient() as client: + client.sendQuery( "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - sendEmptyBlock(s) - readHeader(s) + client.sendEmptyBlock() + client.readHeader() # Data ba = bytearray() @@ -324,29 +118,23 @@ def insertLowCardinalityRowWithIncorrectDictType(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (overflow) - s.sendall(ba) + client.send(ba) - assertPacket(readVarUInt(s), 2) - print(readException(s)) - s.close() + assertPacket(client.readVarUInt(), 2) # Exception + print(client.readException()) def insertLowCardinalityRowWithIncorrectAdditionalKeys(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) - sendQuery( - s, + with TCPClient() as client: + client.sendQuery( "insert into {}.tab settings input_format_defaults_for_omitted_fields=0 format TSV".format( CLICKHOUSE_DATABASE ), ) # external tables - sendEmptyBlock(s) - readHeader(s) + client.sendEmptyBlock() + client.readHeader() # Data ba = bytearray() @@ -365,11 +153,10 @@ def insertLowCardinalityRowWithIncorrectAdditionalKeys(): writeStringBinary("hello", ba) # key ba.extend([1] + [0] * 7) # num_indexes ba.extend([0] * 8) # UInt64 index (0 for 'hello') - s.sendall(ba) + client.send(ba) - assertPacket(readVarUInt(s), 2) - print(readException(s)) - s.close() + assertPacket(client.readVarUInt(), 2) # Exception + print(client.readException()) def main(): diff --git a/tests/queries/0_stateless/02210_processors_profile_log.reference b/tests/queries/0_stateless/02210_processors_profile_log.reference index 41543d0706a..f480236111f 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log.reference @@ -38,4 +38,5 @@ LazyOutputFormat 1 1 1 0 0 LimitsCheckingTransform 1 1 1 1 1 NullSource 1 0 0 0 0 NullSource 1 0 0 0 0 +NullSource 0 0 0 0 0 SourceFromSingleChunk 1 0 0 1 1 diff --git a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python index 92240e109c1..fdc64a8dba8 100644 --- a/tests/queries/0_stateless/02458_insert_select_progress_tcp.python +++ b/tests/queries/0_stateless/02458_insert_select_progress_tcp.python @@ -1,188 +1,30 @@ #!/usr/bin/env python3 -import socket -import os -import uuid import json +import os +import sys -CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") -CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) -CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") -CLIENT_NAME = "simple native protocol" +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -def writeVarUInt(x, ba): - for _ in range(0, 9): - byte = x & 0x7F - if x > 0x7F: - byte |= 0x80 - - ba.append(byte) - - x >>= 7 - if x == 0: - return - - -def writeStringBinary(s, ba): - b = bytes(s, "utf-8") - writeVarUInt(len(s), ba) - ba.extend(b) - - -def readStrict(s, size=1): - res = bytearray() - while size: - cur = s.recv(size) - # if not res: - # raise "Socket is closed" - size -= len(cur) - res.extend(cur) - - return res - - -def readUInt(s, size=1): - res = readStrict(s, size) - val = 0 - for i in range(len(res)): - val += res[i] << (i * 8) - return val - - -def readUInt8(s): - return readUInt(s) - - -def readUInt16(s): - return readUInt(s, 2) - - -def readUInt32(s): - return readUInt(s, 4) - - -def readUInt64(s): - return readUInt(s, 8) - - -def readVarUInt(s): - x = 0 - for i in range(9): - byte = readStrict(s)[0] - x |= (byte & 0x7F) << (7 * i) - - if not byte & 0x80: - return x - - return x - - -def readStringBinary(s): - size = readVarUInt(s) - s = readStrict(s, size) - return s.decode("utf-8") - - -def sendHello(s): - ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary(CLIENT_NAME, ba) - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary(CLICKHOUSE_DATABASE, ba) # database - writeStringBinary("default", ba) # user - writeStringBinary("", ba) # pwd - s.sendall(ba) - - -def receiveHello(s): - p_type = readVarUInt(s) - assert p_type == 0 # Hello - server_name = readStringBinary(s) - # print("Server name: ", server_name) - server_version_major = readVarUInt(s) - # print("Major: ", server_version_major) - server_version_minor = readVarUInt(s) - # print("Minor: ", server_version_minor) - server_revision = readVarUInt(s) - # print("Revision: ", server_revision) - server_timezone = readStringBinary(s) - # print("Timezone: ", server_timezone) - server_display_name = readStringBinary(s) - # print("Display name: ", server_display_name) - server_version_patch = readVarUInt(s) - # print("Version patch: ", server_version_patch) - - -def serializeClientInfo(ba, query_id): - writeStringBinary("default", ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary("127.0.0.1:9000", ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary("os_user", ba) # os_user - writeStringBinary("client_hostname", ba) # client_hostname - writeStringBinary(CLIENT_NAME, ba) # client_name - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary("", ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry - - -def sendQuery(s, query): - ba = bytearray() - query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query - writeStringBinary(query_id, ba) - - ba.append(1) # INITIAL_QUERY - - # client info - serializeClientInfo(ba, query_id) - - writeStringBinary("", ba) # No settings - writeStringBinary("", ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally - s.sendall(ba) - - -def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num - - -def sendEmptyBlock(s): - ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary("", ba) - serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns - s.sendall(ba) - - -def assertPacket(packet, expected): - assert packet == expected, packet +from tcp_client import TCPClient class Progress: - def __init__(self): + def __init__( + self, + read_rows=0, + read_bytes=0, + total_rows_to_read=0, + written_rows=0, + written_bytes=0, + ): # NOTE: this is done in ctor to initialize __dict__ - self.read_rows = 0 - self.read_bytes = 0 - self.total_rows_to_read = 0 - self.written_rows = 0 - self.written_bytes = 0 + self.read_rows = read_rows + self.read_bytes = read_bytes + self.total_rows_to_read = total_rows_to_read + self.written_rows = written_rows + self.written_bytes = written_bytes def __str__(self): return json.dumps(self.__dict__) @@ -195,13 +37,6 @@ class Progress: self.written_bytes += b.written_bytes return self - def readPacket(self, s): - self.read_rows += readVarUInt(s) - self.read_bytes += readVarUInt(s) - self.total_rows_to_read += readVarUInt(s) - self.written_rows += readVarUInt(s) - self.written_bytes += readVarUInt(s) - def __bool__(self): return ( self.read_rows > 0 @@ -212,52 +47,25 @@ class Progress: ) -def readProgress(s): - packet_type = readVarUInt(s) - if packet_type == 2: # Exception - raise RuntimeError(readException(s)) - - if packet_type == 5: # End stream - return None - - assertPacket(packet_type, 3) # Progress - - progress = Progress() - progress.readPacket(s) - return progress - - -def readException(s): - code = readUInt32(s) - name = readStringBinary(s) - text = readStringBinary(s) - readStringBinary(s) # trace - assertPacket(readUInt8(s), 0) # has_nested - return "code {}: {}".format(code, text.replace("DB::Exception:", "")) - - def main(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) + with TCPClient() as client: # For 1 second sleep and 1000ms of interactive_delay we definitelly should have non zero progress packet. # NOTE: interactive_delay=0 cannot be used since in this case CompletedPipelineExecutor will not call cancelled callback. - sendQuery( - s, + client.sendQuery( "insert into function null('_ Int') select sleep(1) from numbers(2) settings max_block_size=1, interactive_delay=1000", ) # external tables - sendEmptyBlock(s) + client.sendEmptyBlock() summary_progress = Progress() non_empty_progress_packets = 0 while True: - progress = readProgress(s) - if progress is None: + progress_info = client.readProgress() + if progress_info is None: break + + progress = Progress(*progress_info) summary_progress += progress if progress: non_empty_progress_packets += 1 @@ -268,8 +76,6 @@ def main(): # - 1 or 2 for each SELECT block assert non_empty_progress_packets in (3, 4), f"{non_empty_progress_packets=:}" - s.close() - if __name__ == "__main__": main() diff --git a/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python b/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python index 48b27d434ec..1736807410f 100644 --- a/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python +++ b/tests/queries/0_stateless/02750_settings_alias_tcp_protocol.python @@ -1,217 +1,23 @@ #!/usr/bin/env python3 -import socket + import os -import uuid -import json +import sys -CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") -CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) -CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") -CLIENT_NAME = "simple native protocol" +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) - -def writeVarUInt(x, ba): - for _ in range(0, 9): - byte = x & 0x7F - if x > 0x7F: - byte |= 0x80 - - ba.append(byte) - - x >>= 7 - if x == 0: - return - - -def writeStringBinary(s, ba): - b = bytes(s, "utf-8") - writeVarUInt(len(s), ba) - ba.extend(b) - - -def readStrict(s, size=1): - res = bytearray() - while size: - cur = s.recv(size) - # if not res: - # raise "Socket is closed" - size -= len(cur) - res.extend(cur) - - return res - - -def readUInt(s, size=1): - res = readStrict(s, size) - val = 0 - for i in range(len(res)): - val += res[i] << (i * 8) - return val - - -def readUInt8(s): - return readUInt(s) - - -def readUInt16(s): - return readUInt(s, 2) - - -def readUInt32(s): - return readUInt(s, 4) - - -def readUInt64(s): - return readUInt(s, 8) - - -def readVarUInt(s): - x = 0 - for i in range(9): - byte = readStrict(s)[0] - x |= (byte & 0x7F) << (7 * i) - - if not byte & 0x80: - return x - - return x - - -def readStringBinary(s): - size = readVarUInt(s) - s = readStrict(s, size) - return s.decode("utf-8") - - -def sendHello(s): - ba = bytearray() - writeVarUInt(0, ba) # Hello - writeStringBinary(CLIENT_NAME, ba) - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary(CLICKHOUSE_DATABASE, ba) # database - writeStringBinary("default", ba) # user - writeStringBinary("", ba) # pwd - s.sendall(ba) - - -def receiveHello(s): - p_type = readVarUInt(s) - assert p_type == 0 # Hello - _server_name = readStringBinary(s) - _server_version_major = readVarUInt(s) - _server_version_minor = readVarUInt(s) - _server_revision = readVarUInt(s) - _server_timezone = readStringBinary(s) - _server_display_name = readStringBinary(s) - _server_version_patch = readVarUInt(s) - - -def serializeClientInfo(ba, query_id): - writeStringBinary("default", ba) # initial_user - writeStringBinary(query_id, ba) # initial_query_id - writeStringBinary("127.0.0.1:9000", ba) # initial_address - ba.extend([0] * 8) # initial_query_start_time_microseconds - ba.append(1) # TCP - writeStringBinary("os_user", ba) # os_user - writeStringBinary("client_hostname", ba) # client_hostname - writeStringBinary(CLIENT_NAME, ba) # client_name - writeVarUInt(21, ba) - writeVarUInt(9, ba) - writeVarUInt(54449, ba) - writeStringBinary("", ba) # quota_key - writeVarUInt(0, ba) # distributed_depth - writeVarUInt(1, ba) # client_version_patch - ba.append(0) # No telemetry - - -def sendQuery(s, query, settings): - ba = bytearray() - query_id = uuid.uuid4().hex - writeVarUInt(1, ba) # query - writeStringBinary(query_id, ba) - - ba.append(1) # INITIAL_QUERY - - # client info - serializeClientInfo(ba, query_id) - - # Settings - for key, value in settings.items(): - writeStringBinary(key, ba) - writeVarUInt(1, ba) # is_important - writeStringBinary(str(value), ba) - writeStringBinary("", ba) # End of settings - - writeStringBinary("", ba) # No interserver secret - writeVarUInt(2, ba) # Stage - Complete - ba.append(0) # No compression - writeStringBinary(query, ba) # query, finally - s.sendall(ba) - - -def serializeBlockInfo(ba): - writeVarUInt(1, ba) # 1 - ba.append(0) # is_overflows - writeVarUInt(2, ba) # 2 - writeVarUInt(0, ba) # 0 - ba.extend([0] * 4) # bucket_num - - -def sendEmptyBlock(s): - ba = bytearray() - writeVarUInt(2, ba) # Data - writeStringBinary("", ba) - serializeBlockInfo(ba) - writeVarUInt(0, ba) # rows - writeVarUInt(0, ba) # columns - s.sendall(ba) - - -def assertPacket(packet, expected): - assert packet == expected, "Got: {}, expected: {}".format(packet, expected) - - -def readResponse(s): - packet_type = readVarUInt(s) - if packet_type == 2: # Exception - raise RuntimeError(readException(s)) - - if packet_type == 1: # Data - return None - if packet_type == 3: # Progress - return None - if packet_type == 5: # End stream - return None - - raise RuntimeError("Unexpected packet: {}".format(packet_type)) - - -def readException(s): - code = readUInt32(s) - _name = readStringBinary(s) - text = readStringBinary(s) - readStringBinary(s) # trace - assertPacket(readUInt8(s), 0) # has_nested - return "code {}: {}".format(code, text.replace("DB::Exception:", "")) +from tcp_client import TCPClient def main(): - with socket.socket(socket.AF_INET, socket.SOCK_STREAM) as s: - s.settimeout(30) - s.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) - sendHello(s) - receiveHello(s) - sendQuery(s, "select 1", {"replication_alter_partitions_sync": 1}) + with TCPClient() as client: + client.sendQuery("select 1", {"replication_alter_partitions_sync": 1}) # external tables - sendEmptyBlock(s) + client.sendEmptyBlock() - while readResponse(s) is not None: + while client.readResponse() is not None: pass - - s.close() print("OK") diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python new file mode 100755 index 00000000000..b3d04d263dc --- /dev/null +++ b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python @@ -0,0 +1,93 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- + + +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from tcp_client import TCPClient + + +def run_query_without_errors(query, support_partial_result): + with TCPClient() as client: + client.sendQuery(query) + + # external tables + client.sendEmptyBlock() + client.readHeader() + + # Partial result + partial_result = client.readDataWithoutProgress()[0] + if support_partial_result: + assert ( + len(partial_result.value) > 0 + ), "Expected at least one block with a non-empty partial result before getting the full result" + + while True: + assert all( + a >= b + for a, b in zip(partial_result.value, partial_result.value[1:]) + ), "Partial result always should be sorted for this test" + + new_partial_result = client.readDataWithoutProgress( + need_print_info=False + )[0] + if len(new_partial_result.value) == 0: + break + + data_size = len(partial_result.value) + assert all( + partial_result.value[i] <= new_partial_result.value[i] + for i in range(data_size) + ), f"New partial result values should always be greater then old one because a new block contains more information about the full data. New result {new_partial_result}. Previous result {partial_result}" + + partial_result = new_partial_result + else: + block_rows = len(partial_result.value) + assert ( + block_rows == 0 + ), f"Expected only empty partial result block before getting the full result, but block has {block_rows} rows" + + # Full result + full_result = client.readDataWithoutProgress()[0] + + data_size = len(partial_result.value) + assert all( + partial_result.value[i] <= full_result.value[i] for i in range(data_size) + ), f"Full result values should always be greater then partial result values. Full result {full_result}. Partial result {partial_result}" + + for result in full_result.value: + print(result) + + +def main(): + # Request with partial result limit less then full limit + run_query_without_errors( + "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 5 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 3", + support_partial_result=True, + ) + + # Request with partial result limit greater then full limit + run_query_without_errors( + "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + support_partial_result=True, + ) + + # Request with OFFSET + run_query_without_errors( + "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 OFFSET 1 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + support_partial_result=True, + ) + + # Request with OFFSET greater then partial result limit (partial result pipeline use blocks with less then OFFSET, so there will be no elements in block after LimitPartialResultTransform) + run_query_without_errors( + "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 OFFSET 15 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + support_partial_result=False, + ) + + +if __name__ == "__main__": + main() diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference new file mode 100644 index 00000000000..211a193940a --- /dev/null +++ b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference @@ -0,0 +1,38 @@ +Rows 0 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +Rows 5 Columns 1 +Column number type UInt64 +5000000 +4999999 +4999998 +4999997 +4999996 +Rows 0 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +5000000 +4999999 +4999998 +Rows 0 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +4999999 +4999998 +4999997 +Rows 0 Columns 1 +Column number type UInt64 +Rows 0 Columns 1 +Column number type UInt64 +Rows 3 Columns 1 +Column number type UInt64 +4999985 +4999984 +4999983 diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh new file mode 100755 index 00000000000..1ed15197dbf --- /dev/null +++ b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02833_partial_sorting_result_during_query_execution.python diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python new file mode 100644 index 00000000000..4d869b05580 --- /dev/null +++ b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python @@ -0,0 +1,125 @@ +#!/usr/bin/env python3 +# -*- coding: utf-8 -*- + + +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from tcp_client import TCPClient + + +def get_keys(results): + return [key for key, _ in results] + + +def check_new_result(new_results, old_results, invariants, rows_limit): + if rows_limit is not None: + assert ( + len(new_results[0].value) <= rows_limit + ), f"Result should have no more then {rows_limit} rows. But it has {len(new_results[0].value)} rows" + + for new_result, old_result in zip(new_results, old_results): + assert ( + new_result.key == old_result.key + ), f"Keys in blocks should be in the same order. Full results keys {get_keys(full_results)}. Partial results keys {get_keys(partial_results)}" + + key = new_result.key + if key in invariants: + new_value = new_result.value + old_value = old_result.value + assert invariants[key]( + old_value, new_value + ), f"Problem with the invariant between new and old result for key: {key}. New value {new_value}. Old value {old_value}" + + +def run_query_without_errors( + query, support_partial_result, invariants=None, rows_limit=None +): + if invariants is None: + invariants = {} + + with TCPClient() as client: + client.sendQuery(query) + + # external tables + client.sendEmptyBlock() + client.readHeader() + + # Partial result + partial_results = client.readDataWithoutProgress() + if support_partial_result: + assert ( + len(partial_results) > 0 and len(partial_results[0].value) > 0 + ), "Expected at least one block with a non-empty partial result before getting the full result" + while True: + new_partial_results = client.readDataWithoutProgress( + need_print_info=False + ) + if len(new_partial_results[0].value) == 0: + break + + check_new_result( + new_partial_results, partial_results, invariants, rows_limit + ) + partial_results = new_partial_results + else: + block_rows = len(partial_results[0].value) + assert ( + block_rows == 0 + ), f"Expected only empty partial result block before getting the full result, but block has {block_rows} rows" + + # Full result + full_results = client.readDataWithoutProgress() + if support_partial_result: + check_new_result(full_results, partial_results, invariants, rows_limit) + + for data in full_results: + if isinstance(data.value[0], int): + print(data.key, data.value) + + +def supported_scenarios_without_key(): + # Simple aggregation query + query = "select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) from numbers_mt(1e7+1) settings max_threads = 1, partial_result_update_duration_ms = 1" + invariants = { + "median(number)": lambda old_value, new_value: old_value <= new_value, + "max(number)": lambda old_value, new_value: old_value <= new_value, + "min(number)": lambda old_value, new_value: old_value >= new_value, + "count(number)": lambda old_value, new_value: old_value <= new_value, + "avg(number)": lambda old_value, new_value: old_value <= new_value, + "sum(number)": lambda old_value, new_value: old_value <= new_value, + } + run_query_without_errors( + query, support_partial_result=True, invariants=invariants, rows_limit=1 + ) + + # Aggregation query with a nested ORDER BY subquery + query = "select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) FROM (SELECT number FROM numbers_mt(1e7) ORDER BY -number LIMIT 3) settings max_threads = 1, partial_result_update_duration_ms=1" + + # Aggregation receives small partial result blocks from ORDER BY which always sends blocks with bigger values + invariants["min(number)"] = lambda old_value, new_value: old_value <= new_value + run_query_without_errors( + query, support_partial_result=True, invariants=invariants, rows_limit=1 + ) + + +def unsupported_scenarios(): + # Currently aggregator for partial result supports only single thread aggregation without key + # Update test when multithreading or aggregation with GROUP BY will be supported for partial result updates + multithread_query = "select sum(number) from numbers_mt(1e7+1) settings max_threads = 2, partial_result_update_duration_ms = 100" + run_query_without_errors(multithread_query, support_partial_result=False) + + group_with_key_query = "select mod2, sum(number) from numbers_mt(1e7+1) group by number % 2 as mod2 settings max_threads = 1, partial_result_update_duration_ms = 100" + run_query_without_errors(group_with_key_query, support_partial_result=False) + + +def main(): + supported_scenarios_without_key() + unsupported_scenarios() + + +if __name__ == "__main__": + main() diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference new file mode 100644 index 00000000000..a813b18f24f --- /dev/null +++ b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference @@ -0,0 +1,88 @@ +Rows 0 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +Rows 1 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +Rows 1 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +max(number) [10000000] +min(number) [0] +any(number) [0] +count(number) [10000001] +sum(number) [50000005000000] +Rows 0 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +Rows 1 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +Rows 1 Columns 9 +Column median(number) type Float64 +Column stddevSamp(number) type Float64 +Column stddevPop(number) type Float64 +Column max(number) type UInt64 +Column min(number) type UInt64 +Column any(number) type UInt64 +Column count(number) type UInt64 +Column avg(number) type Float64 +Column sum(number) type UInt64 +max(number) [9999999] +min(number) [9999997] +any(number) [9999999] +count(number) [3] +sum(number) [29999994] +Rows 0 Columns 1 +Column sum(number) type UInt64 +Rows 0 Columns 1 +Column sum(number) type UInt64 +Rows 1 Columns 1 +Column sum(number) type UInt64 +sum(number) [50000005000000] +Rows 0 Columns 2 +Column mod2 type UInt8 +Column sum(number) type UInt64 +Rows 0 Columns 2 +Column mod2 type UInt8 +Column sum(number) type UInt64 +Rows 2 Columns 2 +Column mod2 type UInt8 +Column sum(number) type UInt64 +mod2 [0, 1] +sum(number) [25000005000000, 25000000000000] diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh new file mode 100755 index 00000000000..e70a3c53ec4 --- /dev/null +++ b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02834_partial_aggregating_result_during_query_execution.python diff --git a/tests/queries/0_stateless/helpers/tcp_client.py b/tests/queries/0_stateless/helpers/tcp_client.py new file mode 100644 index 00000000000..fdc4ab28e04 --- /dev/null +++ b/tests/queries/0_stateless/helpers/tcp_client.py @@ -0,0 +1,313 @@ +import socket +import os +import uuid +import struct + +CLICKHOUSE_HOST = os.environ.get("CLICKHOUSE_HOST", "127.0.0.1") +CLICKHOUSE_PORT = int(os.environ.get("CLICKHOUSE_PORT_TCP", "900000")) +CLICKHOUSE_DATABASE = os.environ.get("CLICKHOUSE_DATABASE", "default") +CLIENT_NAME = "simple native protocol" + + +def writeVarUInt(x, ba): + for _ in range(0, 9): + byte = x & 0x7F + if x > 0x7F: + byte |= 0x80 + + ba.append(byte) + + x >>= 7 + if x == 0: + return + + +def writeStringBinary(s, ba): + b = bytes(s, "utf-8") + writeVarUInt(len(s), ba) + ba.extend(b) + + +def serializeClientInfo(ba, query_id): + writeStringBinary("default", ba) # initial_user + writeStringBinary(query_id, ba) # initial_query_id + writeStringBinary("127.0.0.1:9000", ba) # initial_address + ba.extend([0] * 8) # initial_query_start_time_microseconds + ba.append(1) # TCP + writeStringBinary("os_user", ba) # os_user + writeStringBinary("client_hostname", ba) # client_hostname + writeStringBinary(CLIENT_NAME, ba) # client_name + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary("", ba) # quota_key + writeVarUInt(0, ba) # distributed_depth + writeVarUInt(1, ba) # client_version_patch + ba.append(0) # No telemetry + + +def serializeBlockInfo(ba): + writeVarUInt(1, ba) # 1 + ba.append(0) # is_overflows + writeVarUInt(2, ba) # 2 + writeVarUInt(0, ba) # 0 + ba.extend([0] * 4) # bucket_num + + +def assertPacket(packet, expected): + assert packet == expected, "Got: {}, expected: {}".format(packet, expected) + + +class Data(object): + def __init__(self, key, value): + self.key = key + self.value = value + + +class TCPClient(object): + def __init__(self, timeout=30): + self.timeout = timeout + self.socket = None + + def __enter__(self): + self.socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + self.socket.settimeout(self.timeout) + self.socket.connect((CLICKHOUSE_HOST, CLICKHOUSE_PORT)) + + self.sendHello() + self.receiveHello() + + return self + + def __exit__(self, exc_type, exc_value, traceback): + if self.socket: + self.socket.close() + + def readStrict(self, size=1): + res = bytearray() + while size: + cur = self.socket.recv(size) + # if not res: + # raise "Socket is closed" + size -= len(cur) + res.extend(cur) + + return res + + def readUInt(self, size=1): + res = self.readStrict(size) + val = 0 + for i in range(len(res)): + val += res[i] << (i * 8) + return val + + def readUInt8(self): + return self.readUInt() + + def readUInt16(self): + return self.readUInt(2) + + def readUInt32(self): + return self.readUInt(4) + + def readUInt64(self): + return self.readUInt(8) + + def readFloat16(self): + return struct.unpack("e", self.readStrict(2)) + + def readFloat32(self): + return struct.unpack("f", self.readStrict(4)) + + def readFloat64(self): + return struct.unpack("d", self.readStrict(8)) + + def readVarUInt(self): + x = 0 + for i in range(9): + byte = self.readStrict()[0] + x |= (byte & 0x7F) << (7 * i) + + if not byte & 0x80: + return x + + return x + + def readStringBinary(self): + size = self.readVarUInt() + s = self.readStrict(size) + return s.decode("utf-8") + + def send(self, byte_array): + self.socket.sendall(byte_array) + + def sendHello(self): + ba = bytearray() + writeVarUInt(0, ba) # Hello + writeStringBinary(CLIENT_NAME, ba) + writeVarUInt(21, ba) + writeVarUInt(9, ba) + writeVarUInt(54449, ba) + writeStringBinary(CLICKHOUSE_DATABASE, ba) # database + writeStringBinary("default", ba) # user + writeStringBinary("", ba) # pwd + self.send(ba) + + def receiveHello(self): + p_type = self.readVarUInt() + assert p_type == 0 # Hello + _server_name = self.readStringBinary() + _server_version_major = self.readVarUInt() + _server_version_minor = self.readVarUInt() + _server_revision = self.readVarUInt() + _server_timezone = self.readStringBinary() + _server_display_name = self.readStringBinary() + _server_version_patch = self.readVarUInt() + + def sendQuery(self, query, settings=None): + if settings == None: + settings = {} # No settings + + ba = bytearray() + query_id = uuid.uuid4().hex + writeVarUInt(1, ba) # query + writeStringBinary(query_id, ba) + + ba.append(1) # INITIAL_QUERY + + # client info + serializeClientInfo(ba, query_id) + + # Settings + for key, value in settings.items(): + writeStringBinary(key, ba) + writeVarUInt(1, ba) # is_important + writeStringBinary(str(value), ba) + writeStringBinary("", ba) # End of settings + + writeStringBinary("", ba) # No interserver secret + writeVarUInt(2, ba) # Stage - Complete + ba.append(0) # No compression + writeStringBinary(query, ba) # query, finally + self.send(ba) + + def sendEmptyBlock(self): + ba = bytearray() + writeVarUInt(2, ba) # Data + writeStringBinary("", ba) + serializeBlockInfo(ba) + writeVarUInt(0, ba) # rows + writeVarUInt(0, ba) # columns + self.send(ba) + + def readException(self): + code = self.readUInt32() + _name = self.readStringBinary() + text = self.readStringBinary() + self.readStringBinary() # trace + assertPacket(self.readUInt8(), 0) # has_nested + return "code {}: {}".format(code, text.replace("DB::Exception:", "")) + + def readPacketType(self): + packet_type = self.readVarUInt() + if packet_type == 2: # Exception + raise RuntimeError(self.readException()) + + return packet_type + + def readResponse(self): + packet_type = self.readPacketType() + if packet_type == 1: # Data + return None + if packet_type == 3: # Progress + return None + if packet_type == 5: # End stream + return None + + raise RuntimeError("Unexpected packet: {}".format(packet_type)) + + def readProgressData(self): + read_rows = self.readVarUInt() + read_bytes = self.readVarUInt() + total_rows_to_read = self.readVarUInt() + written_rows = self.readVarUInt() + written_bytes = self.readVarUInt() + + return read_rows, read_bytes, total_rows_to_read, written_rows, written_bytes + + def readProgress(self): + packet_type = self.readPacketType() + if packet_type == 5: # End stream + return None + assertPacket(packet_type, 3) # Progress + return self.readProgressData() + + def readHeaderInfo(self): + self.readStringBinary() # external table name + # BlockInfo + assertPacket(self.readVarUInt(), 1) # field number 1 + assertPacket(self.readUInt8(), 0) # is_overflows + assertPacket(self.readVarUInt(), 2) # field number 2 + assertPacket(self.readUInt32(), 4294967295) # bucket_num + assertPacket(self.readVarUInt(), 0) # 0 + columns = self.readVarUInt() # rows + rows = self.readVarUInt() # columns + + return columns, rows + + def readHeader(self): + packet_type = self.readPacketType() + assertPacket(packet_type, 1) # Data + + columns, rows = self.readHeaderInfo() + print("Rows {} Columns {}".format(rows, columns)) + for _ in range(columns): + col_name = self.readStringBinary() + type_name = self.readStringBinary() + print("Column {} type {}".format(col_name, type_name)) + + def readRow(self, row_type, rows): + supported_row_types = { + "UInt8": self.readUInt8, + "UInt16": self.readUInt16, + "UInt32": self.readUInt32, + "UInt64": self.readUInt64, + "Float16": self.readFloat16, + "Float32": self.readFloat32, + "Float64": self.readFloat64, + } + if row_type in supported_row_types: + read_type = supported_row_types[row_type] + row = [read_type() for _ in range(rows)] + return row + else: + raise RuntimeError( + "Current python version of tcp client doesn't support the following type of row: {}".format( + row_type + ) + ) + + def readDataWithoutProgress(self, need_print_info=True): + packet_type = self.readPacketType() + while packet_type == 3: # Progress + self.readProgressData() + packet_type = self.readPacketType() + + if packet_type == 5: # End stream + return None + assertPacket(packet_type, 1) # Data + + columns, rows = self.readHeaderInfo() + data = [] + if need_print_info: + print("Rows {} Columns {}".format(rows, columns)) + + for _ in range(columns): + col_name = self.readStringBinary() + type_name = self.readStringBinary() + if need_print_info: + print("Column {} type {}".format(col_name, type_name)) + + data.append(Data(col_name, self.readRow(type_name, rows))) + + return data From 7f778692164d8a9169d83a7b30c712e00bb1bbeb Mon Sep 17 00:00:00 2001 From: alexX512 Date: Sat, 9 Sep 2023 10:11:10 +0000 Subject: [PATCH 138/327] Increase amount of rows in requests for partial result tests (cherry picked from commit 54bc3fea9a7240bda99949aefd3fe8b0fa2d15ec) --- src/Processors/LimitTransform.h | 8 +++++- ...rting_result_during_query_execution.python | 10 ++++--- ...ng_result_during_query_execution.reference | 28 +++++++++---------- ...ating_result_during_query_execution.python | 12 +++++--- ...ng_result_during_query_execution.reference | 18 ++++++------ 5 files changed, 44 insertions(+), 32 deletions(-) diff --git a/src/Processors/LimitTransform.h b/src/Processors/LimitTransform.h index eac5f9e8d6d..cfacc9634f9 100644 --- a/src/Processors/LimitTransform.h +++ b/src/Processors/LimitTransform.h @@ -76,7 +76,13 @@ public: void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) override { rows_before_limit_at_least.swap(counter); } void setInputPortHasCounter(size_t pos) { ports_data[pos].input_port_has_counter = true; } - PartialResultStatus getPartialResultProcessorSupportStatus() const override { return PartialResultStatus::FullSupported; } + PartialResultStatus getPartialResultProcessorSupportStatus() const override + { + /// Currently LimitPartialResultTransform support only single-thread work. + bool is_partial_result_supported = inputs.size() == 1 && outputs.size() == 1; + + return is_partial_result_supported ? PartialResultStatus::FullSupported : PartialResultStatus::NotSupported; + } }; } diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python index b3d04d263dc..39d81438c1b 100755 --- a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python +++ b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.python @@ -64,27 +64,29 @@ def run_query_without_errors(query, support_partial_result): def main(): + rows_number = 2e7 + 1 + # Request with partial result limit less then full limit run_query_without_errors( - "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 5 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 3", + f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 5 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 3", support_partial_result=True, ) # Request with partial result limit greater then full limit run_query_without_errors( - "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", support_partial_result=True, ) # Request with OFFSET run_query_without_errors( - "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 OFFSET 1 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 OFFSET 1 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", support_partial_result=True, ) # Request with OFFSET greater then partial result limit (partial result pipeline use blocks with less then OFFSET, so there will be no elements in block after LimitPartialResultTransform) run_query_without_errors( - "SELECT number FROM numbers_mt(5e6+1) ORDER BY -number LIMIT 3 OFFSET 15 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", + f"SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3 OFFSET 15 SETTINGS max_threads = 1, partial_result_update_duration_ms = 1, max_rows_in_partial_result = 5", support_partial_result=False, ) diff --git a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference index 211a193940a..dd3a343560f 100644 --- a/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference +++ b/tests/queries/0_stateless/02833_partial_sorting_result_during_query_execution.reference @@ -4,35 +4,35 @@ Rows 3 Columns 1 Column number type UInt64 Rows 5 Columns 1 Column number type UInt64 -5000000 -4999999 -4999998 -4999997 -4999996 +20000000 +19999999 +19999998 +19999997 +19999996 Rows 0 Columns 1 Column number type UInt64 Rows 3 Columns 1 Column number type UInt64 Rows 3 Columns 1 Column number type UInt64 -5000000 -4999999 -4999998 +20000000 +19999999 +19999998 Rows 0 Columns 1 Column number type UInt64 Rows 3 Columns 1 Column number type UInt64 Rows 3 Columns 1 Column number type UInt64 -4999999 -4999998 -4999997 +19999999 +19999998 +19999997 Rows 0 Columns 1 Column number type UInt64 Rows 0 Columns 1 Column number type UInt64 Rows 3 Columns 1 Column number type UInt64 -4999985 -4999984 -4999983 +19999985 +19999984 +19999983 diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python index 4d869b05580..aa9f80c751f 100644 --- a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python +++ b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.python @@ -82,8 +82,10 @@ def run_query_without_errors( def supported_scenarios_without_key(): + rows_number = 2e7 + 1 + # Simple aggregation query - query = "select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) from numbers_mt(1e7+1) settings max_threads = 1, partial_result_update_duration_ms = 1" + query = f"select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) from numbers_mt({rows_number}) settings max_threads = 1, partial_result_update_duration_ms = 1" invariants = { "median(number)": lambda old_value, new_value: old_value <= new_value, "max(number)": lambda old_value, new_value: old_value <= new_value, @@ -97,7 +99,7 @@ def supported_scenarios_without_key(): ) # Aggregation query with a nested ORDER BY subquery - query = "select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) FROM (SELECT number FROM numbers_mt(1e7) ORDER BY -number LIMIT 3) settings max_threads = 1, partial_result_update_duration_ms=1" + query = f"select median(number), stddevSamp(number), stddevPop(number), max(number), min(number), any(number), count(number), avg(number), sum(number) FROM (SELECT number FROM numbers_mt({rows_number}) ORDER BY -number LIMIT 3) settings max_threads = 1, partial_result_update_duration_ms=1" # Aggregation receives small partial result blocks from ORDER BY which always sends blocks with bigger values invariants["min(number)"] = lambda old_value, new_value: old_value <= new_value @@ -107,12 +109,14 @@ def supported_scenarios_without_key(): def unsupported_scenarios(): + rows_number = 2e7 + 1 + # Currently aggregator for partial result supports only single thread aggregation without key # Update test when multithreading or aggregation with GROUP BY will be supported for partial result updates - multithread_query = "select sum(number) from numbers_mt(1e7+1) settings max_threads = 2, partial_result_update_duration_ms = 100" + multithread_query = f"select sum(number) from numbers_mt({rows_number}) settings max_threads = 2, partial_result_update_duration_ms = 100" run_query_without_errors(multithread_query, support_partial_result=False) - group_with_key_query = "select mod2, sum(number) from numbers_mt(1e7+1) group by number % 2 as mod2 settings max_threads = 1, partial_result_update_duration_ms = 100" + group_with_key_query = f"select mod2, sum(number) from numbers_mt({rows_number}) group by number % 2 as mod2 settings max_threads = 1, partial_result_update_duration_ms = 100" run_query_without_errors(group_with_key_query, support_partial_result=False) diff --git a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference index a813b18f24f..aea61fad42f 100644 --- a/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference +++ b/tests/queries/0_stateless/02834_partial_aggregating_result_during_query_execution.reference @@ -28,11 +28,11 @@ Column any(number) type UInt64 Column count(number) type UInt64 Column avg(number) type Float64 Column sum(number) type UInt64 -max(number) [10000000] +max(number) [20000000] min(number) [0] any(number) [0] -count(number) [10000001] -sum(number) [50000005000000] +count(number) [20000001] +sum(number) [200000010000000] Rows 0 Columns 9 Column median(number) type Float64 Column stddevSamp(number) type Float64 @@ -63,18 +63,18 @@ Column any(number) type UInt64 Column count(number) type UInt64 Column avg(number) type Float64 Column sum(number) type UInt64 -max(number) [9999999] -min(number) [9999997] -any(number) [9999999] +max(number) [20000000] +min(number) [19999998] +any(number) [20000000] count(number) [3] -sum(number) [29999994] +sum(number) [59999997] Rows 0 Columns 1 Column sum(number) type UInt64 Rows 0 Columns 1 Column sum(number) type UInt64 Rows 1 Columns 1 Column sum(number) type UInt64 -sum(number) [50000005000000] +sum(number) [200000010000000] Rows 0 Columns 2 Column mod2 type UInt8 Column sum(number) type UInt64 @@ -85,4 +85,4 @@ Rows 2 Columns 2 Column mod2 type UInt8 Column sum(number) type UInt64 mod2 [0, 1] -sum(number) [25000005000000, 25000000000000] +sum(number) [100000010000000, 100000000000000] From 1298dba0c6bbad1551ce819da043975956218154 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 9 Sep 2023 03:11:14 +0200 Subject: [PATCH 139/327] fix the bug, add useful logs --- .../DiskObjectStorageTransaction.cpp | 68 +++++++++++++++---- .../ObjectStorages/S3/S3ObjectStorage.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++ .../test.py | 51 +++++++++----- 4 files changed, 99 insertions(+), 33 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 0ae577602b1..b42612d19b3 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -7,6 +7,7 @@ #include #include +#include namespace DB { @@ -156,14 +157,13 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperation { - RemoveBatchRequest remove_paths; - bool keep_all_batch_data; - NameSet file_names_remove_metadata_only; + const RemoveBatchRequest remove_paths; + const bool keep_all_batch_data; + const NameSet file_names_remove_metadata_only; + std::vector removed_files; std::vector objects_to_remove; - bool remove_from_cache = false; - RemoveManyObjectStorageOperation( IObjectStorage & object_storage_, IMetadataStorage & metadata_storage_, @@ -203,6 +203,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) { objects_to_remove.emplace_back(ObjectsToRemove{std::move(objects), std::move(unlink_outcome)}); + removed_files.push_back(path); } } catch (const Exception & e) @@ -213,6 +214,12 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati || e.code() == ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == ErrorCodes::CANNOT_OPEN_FILE) { + LOG_DEBUG( + &Poco::Logger::get("RemoveManyObjectStorageOperation"), + "Can't read metadata because of an exception. Just remove it from the filesystem. Path: {}, exception: {}", + metadata_storage.getPath() + path, + e.message()); + tx->unlinkFile(path); } else @@ -238,16 +245,31 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati /// TL;DR Don't pay any attention to 404 status code if (!remove_from_remote.empty()) object_storage.removeObjectsIfExist(remove_from_remote); + + if (!keep_all_batch_data) + { + LOG_DEBUG( + &Poco::Logger::get("RemoveManyObjectStorageOperation"), + "metadata and objects were removed for [{}], " + "only metadata were removed for [{}].", + boost::algorithm::join(removed_files, ", "), + boost::algorithm::join(file_names_remove_metadata_only, ", ")); + } } }; struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOperation { - std::string path; + /// path inside disk with metadata + const std::string path; + const bool keep_all_batch_data; + /// paths inside the 'this->path' + const NameSet file_names_remove_metadata_only; + + /// map from local_path to its remote objects with hardlinks counter + /// local_path is the path inside 'this->path' std::unordered_map objects_to_remove_by_path; - bool keep_all_batch_data; - NameSet file_names_remove_metadata_only; RemoveRecursiveObjectStorageOperation( IObjectStorage & object_storage_, @@ -274,11 +296,16 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp { try { + chassert(path_to_remove.starts_with(path)); + auto rel_path = String(fs::relative(fs::path(path_to_remove), fs::path(path))); + auto objects_paths = metadata_storage.getStorageObjects(path_to_remove); auto unlink_outcome = tx->unlinkMetadata(path_to_remove); - if (unlink_outcome) + + if (unlink_outcome && !file_names_remove_metadata_only.contains(rel_path)) { - objects_to_remove_by_path[path_to_remove] = ObjectsToRemove{std::move(objects_paths), std::move(unlink_outcome)}; + objects_to_remove_by_path[std::move(rel_path)] + = ObjectsToRemove{std::move(objects_paths), std::move(unlink_outcome)}; } } catch (const Exception & e) @@ -320,25 +347,38 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp void undo() override { - } void finalize() override { if (!keep_all_batch_data) { + std::vector total_removed_paths; + total_removed_paths.reserve(objects_to_remove_by_path.size()); + StoredObjects remove_from_remote; for (auto && [local_path, objects_to_remove] : objects_to_remove_by_path) { - if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename())) + chassert(!file_names_remove_metadata_only.contains(local_path)); + if (objects_to_remove.unlink_outcome->num_hardlinks == 0) { - if (objects_to_remove.unlink_outcome->num_hardlinks == 0) - std::move(objects_to_remove.objects.begin(), objects_to_remove.objects.end(), std::back_inserter(remove_from_remote)); + std::move(objects_to_remove.objects.begin(), objects_to_remove.objects.end(), std::back_inserter(remove_from_remote)); + total_removed_paths.push_back(local_path); } } + /// Read comment inside RemoveObjectStorageOperation class /// TL;DR Don't pay any attention to 404 status code object_storage.removeObjectsIfExist(remove_from_remote); + + LOG_DEBUG( + &Poco::Logger::get("RemoveRecursiveObjectStorageOperation"), + "Recursively remove path {}: " + "metadata and objects were removed for [{}], " + "only metadata were removed for [{}].", + path, + boost::algorithm::join(total_removed_paths, ", "), + boost::algorithm::join(file_names_remove_metadata_only, ", ")); } } }; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index bbfc6609079..0d9670efebe 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -320,7 +320,7 @@ void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exis throwIfUnexpectedError(outcome, if_exists); - LOG_TRACE(log, "Object with path {} was removed from S3", object.remote_path); + LOG_DEBUG(log, "Object with path {} was removed from S3", object.remote_path); } void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_exists) @@ -368,7 +368,7 @@ void S3ObjectStorage::removeObjectsImpl(const StoredObjects & objects, bool if_e throwIfUnexpectedError(outcome, if_exists); - LOG_TRACE(log, "Objects with paths [{}] were removed from S3", keys); + LOG_DEBUG(log, "Objects with paths [{}] were removed from S3", keys); } } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cabdf67a315..d1be8bd249d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1593,6 +1593,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd while (true) { LOG_DEBUG(log, "Committing part {} to zookeeper", part->name); + Coordination::Requests ops; NameSet absent_part_paths_on_replicas; @@ -8788,6 +8789,14 @@ void StorageReplicatedMergeTree::getLockSharedDataOps( { String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; + if (!path_to_set_hardlinked_files.empty() && !hardlinks.empty()) + { + LOG_DEBUG(log, "Locking shared node {} with hardlinks from the other shared node {}, " + "hardlinks: [{}]", + zookeeper_node, path_to_set_hardlinked_files, + boost::algorithm::join(hardlinks, ",")); + } + getZeroCopyLockNodeCreateOps( zookeeper, zookeeper_node, requests, zkutil::CreateMode::Persistent, replace_existing_lock, path_to_set_hardlinked_files, hardlinks); diff --git a/tests/integration/test_replicated_zero_copy_projection_mutation/test.py b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py index 3f594158a46..1b68aac08a7 100644 --- a/tests/integration/test_replicated_zero_copy_projection_mutation/test.py +++ b/tests/integration/test_replicated_zero_copy_projection_mutation/test.py @@ -192,6 +192,8 @@ def test_hardlinks_preserved_when_projection_dropped( first_cluster_node.query(create_query + first_node_settings) second_cluster_node.query(create_query + second_node_settings) + objects_empty_table = list_objects(cluster) + first_cluster_node.query("SYSTEM FLUSH LOGS") table_uuid = first_cluster_node.query( """ @@ -239,6 +241,7 @@ def test_hardlinks_preserved_when_projection_dropped( sleep_time=1, ) + # make sure that alter update made hardlinks inside projection hardlinks = ( first_cluster_node.query( f""" @@ -255,8 +258,6 @@ def test_hardlinks_preserved_when_projection_dropped( assert len(hardlinks) > 0, ",".join(hardlinks) assert any(["proj/" in x for x in hardlinks]), ",".join(hardlinks) - logging.info("hardlinks has been planted") - part_path_on_second_node = second_cluster_node.query( """ SELECT path FROM system.parts @@ -284,6 +285,7 @@ def test_hardlinks_preserved_when_projection_dropped( second_cluster_node.query("SYSTEM FLUSH LOGS") + # make sure there is outdated broken-on-start part broken_parts = ( second_cluster_node.query( """ @@ -310,6 +312,7 @@ def test_hardlinks_preserved_when_projection_dropped( ) second_cluster_node.exec_in_container(["bash", "-c", script]) + # when detached part is removed, removeSharedRecursive is called second_cluster_node.query( f""" ALTER TABLE test_hardlinks_preserved_when_projection_dropped @@ -318,21 +321,35 @@ def test_hardlinks_preserved_when_projection_dropped( settings={"allow_drop_detached": "1"}, ) - res = first_cluster_node.query( - """ - CHECK TABLE test_hardlinks_preserved_when_projection_dropped - """ - ).strip() + # it is an easy way to read all data in part + # "0" means corrupted, https://clickhouse.com/docs/en/sql-reference/statements/check-table + assert ( + "1" + == first_cluster_node.query( + """ + CHECK TABLE test_hardlinks_preserved_when_projection_dropped + """ + ).strip() + ) - # corrupted - assert res == "0" + assert ( + "1" + == second_cluster_node.query( + """ + CHECK TABLE test_hardlinks_preserved_when_projection_dropped + """ + ).strip() + ) - data = first_cluster_node.query( - """ - SELECT a, b FROM test_hardlinks_preserved_when_projection_dropped - WHERE b > 104 - ORDER BY b - """ - ).split("\n") + second_cluster_node.query( + f""" + ALTER TABLE test_hardlinks_preserved_when_projection_dropped + DROP PART 'all_0_0_0_1' + """, + settings={"alter_sync": 2}, + ) - assert len(data) == 5, data + wait_for_delete_s3_objects(cluster, len(objects_empty_table)) + + objects_at_the_end = list_objects(cluster) + assert objects_at_the_end == objects_empty_table From 0a05831839d552488cf77b4c6d6eb00a74308f21 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 9 Sep 2023 16:48:17 +0000 Subject: [PATCH 140/327] Cosmetics --- src/Functions/DateTimeTransforms.h | 556 ++++++++++++++--------------- src/Functions/base64Decode.cpp | 2 +- src/Functions/base64Encode.cpp | 2 +- src/Functions/toDayOfMonth.cpp | 2 +- src/Functions/toDayOfWeek.cpp | 2 +- src/Functions/toDayOfYear.cpp | 2 +- src/Functions/toHour.cpp | 2 +- src/Functions/toMinute.cpp | 2 +- src/Functions/toMonth.cpp | 2 +- src/Functions/toQuarter.cpp | 2 +- src/Functions/toSecond.cpp | 2 +- src/Functions/toYear.cpp | 2 +- 12 files changed, 289 insertions(+), 289 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a1c880f6956..0fb4edc1375 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -49,37 +49,37 @@ namespace ErrorCodes /// This factor transformation will say that the function is monotone everywhere. struct ZeroTransform { - static inline UInt16 execute(Int64, const DateLUTImpl &) { return 0; } - static inline UInt16 execute(UInt32, const DateLUTImpl &) { return 0; } - static inline UInt16 execute(Int32, const DateLUTImpl &) { return 0; } - static inline UInt16 execute(UInt16, const DateLUTImpl &) { return 0; } + static UInt16 execute(Int64, const DateLUTImpl &) { return 0; } + static UInt16 execute(UInt32, const DateLUTImpl &) { return 0; } + static UInt16 execute(Int32, const DateLUTImpl &) { return 0; } + static UInt16 execute(UInt16, const DateLUTImpl &) { return 0; } }; struct ToDateImpl { static constexpr auto name = "toDate"; - static inline UInt16 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt16 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toDayNum(t.whole)); } - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return UInt16(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return UInt16(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32, const DateLUTImpl &) + static UInt16 execute(Int32, const DateLUTImpl &) { throwDateIsNotSupported(name); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl &) + static UInt16 execute(UInt16 d, const DateLUTImpl &) { return d; } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toDayNum(t.whole), 0}; } @@ -91,20 +91,20 @@ struct ToDate32Impl { static constexpr auto name = "toDate32"; - static inline Int32 execute(Int64 t, const DateLUTImpl & time_zone) + static Int32 execute(Int64 t, const DateLUTImpl & time_zone) { return Int32(time_zone.toDayNum(t)); } - static inline Int32 execute(UInt32 t, const DateLUTImpl & time_zone) + static Int32 execute(UInt32 t, const DateLUTImpl & time_zone) { /// Don't saturate. return Int32(time_zone.toDayNum(t)); } - static inline Int32 execute(Int32 d, const DateLUTImpl &) + static Int32 execute(Int32 d, const DateLUTImpl &) { return d; } - static inline Int32 execute(UInt16 d, const DateLUTImpl &) + static Int32 execute(UInt16 d, const DateLUTImpl &) { return d; } @@ -116,27 +116,27 @@ struct ToStartOfDayImpl { static constexpr auto name = "toStartOfDay"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toDate(static_cast(t.whole))); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toDate(t)); } - static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.toDate(ExtendedDayNum(d))); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.toDate(DayNum(d))); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toDate(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.fromDayNum(ExtendedDayNum(d)) * DecimalUtils::scaleMultiplier(DataTypeDateTime64::default_scale); } @@ -148,29 +148,29 @@ struct ToMondayImpl { static constexpr auto name = "toMonday"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { //return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfWeek(t); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { //return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t)); return time_zone.toFirstDayNumOfWeek(t); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfWeek(ExtendedDayNum(d)); } @@ -181,27 +181,27 @@ struct ToStartOfMonthImpl { static constexpr auto name = "toStartOfMonth"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfMonth(ExtendedDayNum(d)); } @@ -213,27 +213,27 @@ struct ToLastDayOfMonthImpl { static constexpr auto name = "toLastDayOfMonth"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toLastDayNumOfMonth(ExtendedDayNum(d)); } @@ -244,27 +244,27 @@ struct ToStartOfQuarterImpl { static constexpr auto name = "toStartOfQuarter"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfQuarter(ExtendedDayNum(d)); } @@ -275,27 +275,27 @@ struct ToStartOfYearImpl { static constexpr auto name = "toStartOfYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfYear(ExtendedDayNum(d)); } @@ -313,19 +313,19 @@ struct ToTimeImpl { return static_cast(time_zone.toTime(t.whole) + 86400); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toTime(t) + 86400); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -334,27 +334,27 @@ struct ToStartOfMinuteImpl { static constexpr auto name = "toStartOfMinute"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toStartOfMinute(t.whole)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfMinute(t); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toStartOfMinute(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -369,7 +369,7 @@ struct ToStartOfSecondImpl { static constexpr auto name = "toStartOfSecond"; - static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) + static DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) { auto fractional_with_sign = DecimalUtils::getFractionalPartWithScaleMultiplier(datetime64, scale_multiplier); @@ -385,19 +385,19 @@ struct ToStartOfSecondImpl return datetime64 - fractional_with_sign; } - static inline UInt32 execute(UInt32, const DateLUTImpl &) + static UInt32 execute(UInt32, const DateLUTImpl &) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -406,7 +406,7 @@ struct ToStartOfMillisecondImpl { static constexpr auto name = "toStartOfMillisecond"; - static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) + static DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) { // given that scale is 6, scale_multiplier is 1000000 // for DateTime64 value of 123.456789: @@ -433,19 +433,19 @@ struct ToStartOfMillisecondImpl } } - static inline UInt32 execute(UInt32, const DateLUTImpl &) + static UInt32 execute(UInt32, const DateLUTImpl &) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -454,7 +454,7 @@ struct ToStartOfMicrosecondImpl { static constexpr auto name = "toStartOfMicrosecond"; - static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) + static DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) { // @see ToStartOfMillisecondImpl @@ -477,19 +477,19 @@ struct ToStartOfMicrosecondImpl } } - static inline UInt32 execute(UInt32, const DateLUTImpl &) + static UInt32 execute(UInt32, const DateLUTImpl &) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -498,7 +498,7 @@ struct ToStartOfNanosecondImpl { static constexpr auto name = "toStartOfNanosecond"; - static inline DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) + static DateTime64 execute(const DateTime64 & datetime64, Int64 scale_multiplier, const DateLUTImpl &) { // @see ToStartOfMillisecondImpl if (scale_multiplier == 1000000000) @@ -515,19 +515,19 @@ struct ToStartOfNanosecondImpl } } - static inline UInt32 execute(UInt32, const DateLUTImpl &) + static UInt32 execute(UInt32, const DateLUTImpl &) { throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type DateTime of argument for function {}", name); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -536,27 +536,27 @@ struct ToStartOfFiveMinutesImpl { static constexpr auto name = "toStartOfFiveMinutes"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toStartOfFiveMinutes(t.whole)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfFiveMinutes(t); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toStartOfFiveMinutes(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -568,27 +568,27 @@ struct ToStartOfTenMinutesImpl { static constexpr auto name = "toStartOfTenMinutes"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toStartOfTenMinutes(t.whole)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfTenMinutes(t); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toStartOfTenMinutes(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -600,27 +600,27 @@ struct ToStartOfFifteenMinutesImpl { static constexpr auto name = "toStartOfFifteenMinutes"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toStartOfFifteenMinutes(t.whole)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfFifteenMinutes(t); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toStartOfFifteenMinutes(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -633,34 +633,34 @@ struct TimeSlotImpl { static constexpr auto name = "timeSlot"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl &) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl &) { return static_cast(t.whole / 1800 * 1800); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl &) + static UInt32 execute(UInt32 t, const DateLUTImpl &) { return t / 1800 * 1800; } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl &) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl &) { if (likely(t.whole >= 0)) return {t.whole / 1800 * 1800, 0}; return {(t.whole + 1 - 1800) / 1800 * 1800, 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -672,32 +672,32 @@ struct ToStartOfHourImpl { static constexpr auto name = "toStartOfHour"; - static inline UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static UInt32 execute(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toStartOfHour(t.whole)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toStartOfHour(t); } - static inline UInt32 execute(Int32, const DateLUTImpl &) + static UInt32 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static UInt32 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) + static DecimalUtils::DecimalComponents executeExtendedResult(const DecimalUtils::DecimalComponents & t, const DateLUTImpl & time_zone) { return {time_zone.toStartOfHour(t.whole), 0}; } - static inline Int64 executeExtendedResult(Int32, const DateLUTImpl &) + static Int64 executeExtendedResult(Int32, const DateLUTImpl &) { throwDate32IsNotSupported(name); } @@ -709,26 +709,26 @@ struct ToYearImpl { static constexpr auto name = "toYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(t); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toYear(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } + static constexpr bool hasPreimage() { return true; } - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) + static RangeOrNull getPreimage(const IDataType & type, const Field & point) { if (point.getType() != Field::Types::UInt64) return std::nullopt; @@ -757,19 +757,19 @@ struct ToWeekYearImpl static constexpr Int8 week_mode = 3; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toYearWeek(t, week_mode).first; } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYearWeek(t, week_mode).first; } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toYearWeek(ExtendedDayNum(d), week_mode).first; } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toYearWeek(DayNum(d), week_mode).first; } @@ -781,19 +781,19 @@ struct ToWeekOfWeekYearImpl { static constexpr auto name = "toWeekOfWeekYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(t); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(t); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(DayNum(d)); } @@ -805,23 +805,23 @@ struct ToQuarterImpl { static constexpr auto name = "toQuarter"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toQuarter(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toQuarter(t); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toQuarter(ExtendedDayNum(d)); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toQuarter(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -830,23 +830,23 @@ struct ToMonthImpl { static constexpr auto name = "toMonth"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toMonth(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMonth(t); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toMonth(ExtendedDayNum(d)); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -855,24 +855,24 @@ struct ToDayOfMonthImpl { static constexpr auto name = "toDayOfMonth"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(t); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(ExtendedDayNum(d)); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMonthImpl; }; @@ -880,19 +880,19 @@ struct ToDayOfWeekImpl { static constexpr auto name = "toDayOfWeek"; - static inline UInt8 execute(Int64 t, UInt8 mode, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, UInt8 mode, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t, mode); } - static inline UInt8 execute(UInt32 t, UInt8 mode, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, UInt8 mode, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(t, mode); } - static inline UInt8 execute(Int32 d, UInt8 mode, const DateLUTImpl & time_zone) + static UInt8 execute(Int32 d, UInt8 mode, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(ExtendedDayNum(d), mode); } - static inline UInt8 execute(UInt16 d, UInt8 mode, const DateLUTImpl & time_zone) + static UInt8 execute(UInt16 d, UInt8 mode, const DateLUTImpl & time_zone) { return time_zone.toDayOfWeek(DayNum(d), mode); } @@ -904,23 +904,23 @@ struct ToDayOfYearImpl { static constexpr auto name = "toDayOfYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfYear(t); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayOfYear(t); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfYear(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toDayOfYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -929,23 +929,23 @@ struct ToHourImpl { static constexpr auto name = "toHour"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toHour(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toHour(t); } - static inline UInt8 execute(Int32, const DateLUTImpl &) + static UInt8 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt8 execute(UInt16, const DateLUTImpl &) + static UInt8 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -954,27 +954,27 @@ struct TimezoneOffsetImpl { static constexpr auto name = "timezoneOffset"; - static inline time_t execute(Int64 t, const DateLUTImpl & time_zone) + static time_t execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.timezoneOffset(t); } - static inline time_t execute(UInt32 t, const DateLUTImpl & time_zone) + static time_t execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.timezoneOffset(t); } - static inline time_t execute(Int32, const DateLUTImpl &) + static time_t execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline time_t execute(UInt16, const DateLUTImpl &) + static time_t execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToTimeImpl; }; @@ -982,23 +982,23 @@ struct ToMinuteImpl { static constexpr auto name = "toMinute"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toMinute(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toMinute(t); } - static inline UInt8 execute(Int32, const DateLUTImpl &) + static UInt8 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt8 execute(UInt16, const DateLUTImpl &) + static UInt8 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfHourImpl; }; @@ -1007,23 +1007,23 @@ struct ToSecondImpl { static constexpr auto name = "toSecond"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toSecond(t); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toSecond(t); } - static inline UInt8 execute(Int32, const DateLUTImpl &) + static UInt8 execute(Int32, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline UInt8 execute(UInt16, const DateLUTImpl &) + static UInt8 execute(UInt16, const DateLUTImpl &) { - throwDateTimeIsNotSupported(name); + throwDateIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMinuteImpl; }; @@ -1032,23 +1032,23 @@ struct ToISOYearImpl { static constexpr auto name = "toISOYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toISOYear(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toISOYear(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toISOYear(ExtendedDayNum(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toISOYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1057,27 +1057,27 @@ struct ToStartOfISOYearImpl { static constexpr auto name = "toStartOfISOYear"; - static inline UInt16 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt16 execute(Int64 t, const DateLUTImpl & time_zone) { return t < 0 ? 0 : time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(std::min(Int32(time_zone.toDayNum(t)), Int32(DATE_LUT_MAX_DAY_NUM)))); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t)); } - static inline UInt16 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt16 execute(Int32 d, const DateLUTImpl & time_zone) { return d < 0 ? 0 : time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(std::min(d, Int32(DATE_LUT_MAX_DAY_NUM)))); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfISOYear(DayNum(d)); } - static inline Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) + static Int64 executeExtendedResult(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfISOYear(time_zone.toDayNum(t)); } - static inline Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) + static Int32 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toFirstDayNumOfISOYear(ExtendedDayNum(d)); } @@ -1089,23 +1089,23 @@ struct ToISOWeekImpl { static constexpr auto name = "toISOWeek"; - static inline UInt8 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt8 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(time_zone.toDayNum(t)); } - static inline UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt8 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(time_zone.toDayNum(t)); } - static inline UInt8 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt8 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(ExtendedDayNum(d)); } - static inline UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt8 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toISOWeek(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ToISOYearImpl; }; @@ -1126,29 +1126,29 @@ struct ToRelativeYearNumImpl { static constexpr auto name = "toRelativeYearNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toYear(t)); else return static_cast(time_zone.toYear(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toYear(static_cast(t)); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toYear(ExtendedDayNum(d))); else return static_cast(time_zone.toYear(ExtendedDayNum(d))); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1158,29 +1158,29 @@ struct ToRelativeQuarterNumImpl { static constexpr auto name = "toRelativeQuarterNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeQuarterNum(t)); else return static_cast(time_zone.toRelativeQuarterNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeQuarterNum(static_cast(t)); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeQuarterNum(ExtendedDayNum(d))); else return static_cast(time_zone.toRelativeQuarterNum(ExtendedDayNum(d))); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeQuarterNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1190,29 +1190,29 @@ struct ToRelativeMonthNumImpl { static constexpr auto name = "toRelativeMonthNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeMonthNum(t)); else return static_cast(time_zone.toRelativeMonthNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeMonthNum(static_cast(t)); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeMonthNum(ExtendedDayNum(d))); else return static_cast(time_zone.toRelativeMonthNum(ExtendedDayNum(d))); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeMonthNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1222,29 +1222,29 @@ struct ToRelativeWeekNumImpl { static constexpr auto name = "toRelativeWeekNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeWeekNum(t)); else return static_cast(time_zone.toRelativeWeekNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toRelativeWeekNum(static_cast(t)); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeWeekNum(ExtendedDayNum(d))); else return static_cast(time_zone.toRelativeWeekNum(ExtendedDayNum(d))); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt16 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toRelativeWeekNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1254,29 +1254,29 @@ struct ToRelativeDayNumImpl { static constexpr auto name = "toRelativeDayNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toDayNum(t)); else return static_cast(time_zone.toDayNum(t)); } - static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toDayNum(static_cast(t)); } - static inline auto execute(Int32 d, const DateLUTImpl &) + static auto execute(Int32 d, const DateLUTImpl &) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(static_cast(d)); else return static_cast(static_cast(d)); } - static inline UInt16 execute(UInt16 d, const DateLUTImpl &) + static UInt16 execute(UInt16 d, const DateLUTImpl &) { return static_cast(d); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1286,35 +1286,35 @@ struct ToRelativeHourNumImpl { static constexpr auto name = "toRelativeHourNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toStableRelativeHourNum(t)); else return static_cast(time_zone.toRelativeHourNum(t)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toStableRelativeHourNum(static_cast(t))); else return static_cast(time_zone.toRelativeHourNum(static_cast(t))); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toStableRelativeHourNum(ExtendedDayNum(d))); else return static_cast(time_zone.toRelativeHourNum(ExtendedDayNum(d))); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toStableRelativeHourNum(DayNum(d))); else return static_cast(time_zone.toRelativeHourNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1324,29 +1324,29 @@ struct ToRelativeMinuteNumImpl { static constexpr auto name = "toRelativeMinuteNum"; - static inline auto execute(Int64 t, const DateLUTImpl & time_zone) + static auto execute(Int64 t, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeMinuteNum(t)); else return static_cast(time_zone.toRelativeMinuteNum(t)); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return static_cast(time_zone.toRelativeMinuteNum(static_cast(t))); } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.toRelativeMinuteNum(ExtendedDayNum(d))); else return static_cast(time_zone.toRelativeMinuteNum(ExtendedDayNum(d))); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.toRelativeMinuteNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1356,26 +1356,26 @@ struct ToRelativeSecondNumImpl { static constexpr auto name = "toRelativeSecondNum"; - static inline Int64 execute(Int64 t, const DateLUTImpl &) + static Int64 execute(Int64 t, const DateLUTImpl &) { return t; } - static inline UInt32 execute(UInt32 t, const DateLUTImpl &) + static UInt32 execute(UInt32 t, const DateLUTImpl &) { return t; } - static inline auto execute(Int32 d, const DateLUTImpl & time_zone) + static auto execute(Int32 d, const DateLUTImpl & time_zone) { if constexpr (precision_ == ResultPrecision::Extended) return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); else return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.fromDayNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1385,7 +1385,7 @@ struct ToRelativeSubsecondNumImpl { static constexpr auto name = "toRelativeSubsecondNumImpl"; - static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) + static Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) { static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); if (scale == scale_multiplier) @@ -1394,15 +1394,15 @@ struct ToRelativeSubsecondNumImpl return t.value / (scale / scale_multiplier); return t.value * (scale_multiplier / scale); } - static inline Int64 execute(UInt32 t, const DateLUTImpl &) + static Int64 execute(UInt32 t, const DateLUTImpl &) { return t * scale_multiplier; } - static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) + static Int64 execute(Int32 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; } - static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) + static Int64 execute(UInt16 d, const DateLUTImpl & time_zone) { return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); } @@ -1414,25 +1414,25 @@ struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; - static inline UInt32 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt32 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(t); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(t); } - static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(ExtendedDayNum(d)); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMM(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } + static constexpr bool hasPreimage() { return true; } - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) + static RangeOrNull getPreimage(const IDataType & type, const Field & point) { if (point.getType() != Field::Types::UInt64) return std::nullopt; @@ -1463,23 +1463,23 @@ struct ToYYYYMMDDImpl { static constexpr auto name = "toYYYYMMDD"; - static inline UInt32 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt32 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(t); } - static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(t); } - static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt32 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(ExtendedDayNum(d)); } - static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDD(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1488,23 +1488,23 @@ struct ToYYYYMMDDhhmmssImpl { static constexpr auto name = "toYYYYMMDDhhmmss"; - static inline UInt64 execute(Int64 t, const DateLUTImpl & time_zone) + static UInt64 execute(Int64 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(t); } - static inline UInt64 execute(UInt32 t, const DateLUTImpl & time_zone) + static UInt64 execute(UInt32 t, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(t); } - static inline UInt64 execute(Int32 d, const DateLUTImpl & time_zone) + static UInt64 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(ExtendedDayNum(d))); } - static inline UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) + static UInt64 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } + static constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1519,7 +1519,7 @@ struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) + static DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) { auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); @@ -1539,15 +1539,15 @@ struct ToDateTimeComponentsImpl UInt16 microsecond = static_cast(fractional % divider); return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; } - static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) + static DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) { return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; } - static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) + static DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) { return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; } - static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) + static DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) { return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; } diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index 4060aafe1a3..e1a24756705 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -10,7 +10,7 @@ REGISTER_FUNCTION(Base64Decode) tb64ini(0, 0); factory.registerFunction>(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index 773db7e09d9..4c8674055ac 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -10,7 +10,7 @@ REGISTER_FUNCTION(Base64Encode) tb64ini(0, 0); factory.registerFunction>(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/toDayOfMonth.cpp b/src/Functions/toDayOfMonth.cpp index d7689ef00f2..c20b0b75797 100644 --- a/src/Functions/toDayOfMonth.cpp +++ b/src/Functions/toDayOfMonth.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToDayOfMonth) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("DAY", "toDayOfMonth", FunctionFactory::CaseInsensitive); factory.registerAlias("DAYOFMONTH", "toDayOfMonth", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toDayOfWeek.cpp b/src/Functions/toDayOfWeek.cpp index 06343714b9d..dc508d70814 100644 --- a/src/Functions/toDayOfWeek.cpp +++ b/src/Functions/toDayOfWeek.cpp @@ -12,7 +12,7 @@ REGISTER_FUNCTION(ToDayOfWeek) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("DAYOFWEEK", "toDayOfWeek", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toDayOfYear.cpp b/src/Functions/toDayOfYear.cpp index 8b03f1a4211..0cbafd6275a 100644 --- a/src/Functions/toDayOfYear.cpp +++ b/src/Functions/toDayOfYear.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToDayOfYear) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("DAYOFYEAR", "toDayOfYear", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toHour.cpp b/src/Functions/toHour.cpp index a6a57946e33..fc9ec657adf 100644 --- a/src/Functions/toHour.cpp +++ b/src/Functions/toHour.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToHour) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("HOUR", "toHour", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toMinute.cpp b/src/Functions/toMinute.cpp index 25939870554..162ecb282df 100644 --- a/src/Functions/toMinute.cpp +++ b/src/Functions/toMinute.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToMinute) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("MINUTE", "toMinute", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toMonth.cpp b/src/Functions/toMonth.cpp index 783a1341e23..422f21e7df8 100644 --- a/src/Functions/toMonth.cpp +++ b/src/Functions/toMonth.cpp @@ -12,7 +12,7 @@ using FunctionToMonth = FunctionDateOrDateTimeToSomething(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("MONTH", "toMonth", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toQuarter.cpp b/src/Functions/toQuarter.cpp index 2268b6402c6..3c301095ff2 100644 --- a/src/Functions/toQuarter.cpp +++ b/src/Functions/toQuarter.cpp @@ -12,7 +12,7 @@ using FunctionToQuarter = FunctionDateOrDateTimeToSomething(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("QUARTER", "toQuarter", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toSecond.cpp b/src/Functions/toSecond.cpp index 2fd64912c0f..372097fd488 100644 --- a/src/Functions/toSecond.cpp +++ b/src/Functions/toSecond.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToSecond) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("SECOND", "toSecond", FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/toYear.cpp b/src/Functions/toYear.cpp index 9cf2a260921..75479adb82c 100644 --- a/src/Functions/toYear.cpp +++ b/src/Functions/toYear.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToYear) { factory.registerFunction(); - /// MysQL compatibility alias. + /// MySQL compatibility alias. factory.registerAlias("YEAR", "toYear", FunctionFactory::CaseInsensitive); } From 1ce84774f0e829742827c4a420c310c56637c859 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 8 Sep 2023 13:27:24 +0000 Subject: [PATCH 141/327] Implement toDaysSinceYearZero() / to_days() (MySQL alias) Fixes: #54277 --- .../functions/date-time-functions.md | 36 +++++++++ src/Functions/DateTimeTransforms.h | 25 +++++++ src/Functions/toDaysSinceYearZero.cpp | 73 +++++++++++++++++++ .../02874_daysSinceYearZero.reference | 13 ++++ .../0_stateless/02874_daysSinceYearZero.sql | 22 ++++++ 5 files changed, 169 insertions(+) create mode 100644 src/Functions/toDaysSinceYearZero.cpp create mode 100644 tests/queries/0_stateless/02874_daysSinceYearZero.reference create mode 100644 tests/queries/0_stateless/02874_daysSinceYearZero.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 6bcc9d654c5..ef0ccf84f73 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -720,6 +720,42 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d └────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` +## toDaysSinceYearZero + +Returns for a given date, the number of days passed since the [year 0](https://en.wikipedia.org/wiki/Year_zero). + +**Syntax** + +``` sql +toDaysSinceYearZero(date) +``` + +Aliases: `TO_DAYS` + +**Arguments** + +- `date` — The date to calculate the number of days passed since year zero from. [Date](../../sql-reference/data-types/date.md) or [Date64](../../sql-reference/data-types/date64.md). + +**Returned value** + +The number of days passed since date 0000-01-01. + +Type: [UInt32](../../sql-reference/data-types/int-uint.md). + +**Example** + +``` sql +SELECT toDaysSinceYearZero(toDate('2023-09-08')); +``` + +Result: + +``` text +┌─toDaysSinceYearZero(toDate('2023-09-08')))─┐ +│ 738772 │ +└────────────────────────────────────────────┘ +``` + ## age Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 0fb4edc1375..16cfba69275 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -925,6 +925,31 @@ struct ToDayOfYearImpl using FactorTransform = ToStartOfYearImpl; }; +struct ToDaysSinceYearZeroImpl +{ + static constexpr auto name = "toDaysSinceYearZero"; + + static UInt32 execute(Int64, const DateLUTImpl &) + { + throwDateTimeIsNotSupported(name); + } + static UInt32 execute(UInt32, const DateLUTImpl &) + { + throwDateTimeIsNotSupported(name); + } + static UInt32 execute(Int32 d, const DateLUTImpl &) + { + return /* days between 0000-01-01 and 1970-01-01 */ 719'164 + d; + } + static UInt32 execute(UInt16 d, const DateLUTImpl &) + { + return /* days between 0000-01-01 and 1970-01-01 */ 719'164 + d; + } + static constexpr bool hasPreimage() { return false; } + + using FactorTransform = ZeroTransform; +}; + struct ToHourImpl { static constexpr auto name = "toHour"; diff --git a/src/Functions/toDaysSinceYearZero.cpp b/src/Functions/toDaysSinceYearZero.cpp new file mode 100644 index 00000000000..c4233416195 --- /dev/null +++ b/src/Functions/toDaysSinceYearZero.cpp @@ -0,0 +1,73 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +/** Returns number of days passed since 0001-01-01 */ +class FunctionToDaysSinceYearZero : public IFunction +{ + using ResultType = DataTypeUInt32; +public: + static constexpr auto name = "toDaysSinceYearZero"; + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + explicit FunctionToDaysSinceYearZero(ContextPtr /*context*/) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors mandatory_args{ + {"date", &isDateOrDate32, nullptr, "Date or Date32"} + }; + + validateFunctionArgumentTypes(*this, arguments, mandatory_args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const IDataType * from_type = arguments[0].type.get(); + WhichDataType which(from_type); + + if (which.isDate()) + return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count); + else if (which.isDate32()) + return DateTimeTransformImpl::execute(arguments, result_type, input_rows_count); + + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0].type->getName(), this->getName()); + } +}; + +} + +REGISTER_FUNCTION(ToDaysSinceYearZero) +{ + factory.registerFunction(); + /// MySQL compatibility alias. + factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02874_daysSinceYearZero.reference b/tests/queries/0_stateless/02874_daysSinceYearZero.reference new file mode 100644 index 00000000000..9bfb9b78910 --- /dev/null +++ b/tests/queries/0_stateless/02874_daysSinceYearZero.reference @@ -0,0 +1,13 @@ +Reject invalid parameters +Const argument +719164 +738772 +693597 +738772 +\N +Non-const argument +738772 +738772 +MySQL alias +738772 +738772 diff --git a/tests/queries/0_stateless/02874_daysSinceYearZero.sql b/tests/queries/0_stateless/02874_daysSinceYearZero.sql new file mode 100644 index 00000000000..0db1a305ea9 --- /dev/null +++ b/tests/queries/0_stateless/02874_daysSinceYearZero.sql @@ -0,0 +1,22 @@ +SELECT 'Reject invalid parameters'; +SELECT daysSinceYearZero(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT daysSinceYearZero(toDate('2023-09-08'), toDate('2023-09-08')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT daysSinceYearZero('str'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT daysSinceYearZero(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT daysSinceYearZero(toDateTime('2023-09-08 11:11:11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT daysSinceYearZero(toDateTime64('2023-09-08 11:11:11', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Const argument'; +SELECT daysSinceYearZero(toDate('1970-01-01')); +SELECT daysSinceYearZero(toDate('2023-09-08')); +SELECT daysSinceYearZero(toDate32('1900-01-01')); +SELECT daysSinceYearZero(toDate32('2023-09-08')); +SELECT daysSinceYearZero(NULL); + +SELECT 'Non-const argument'; +SELECT daysSinceYearZero(materialize(toDate('2023-09-08'))); +SELECT daysSinceYearZero(materialize(toDate32('2023-09-08'))); + +SELECT 'MySQL alias'; +SELECT to_days(toDate('2023-09-08')); +SELECT TO_DAYS(toDate('2023-09-08')); From fb0f9ff5650c19ec2027e82ea98bb0ede70b8227 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 10 Sep 2023 02:38:12 +0800 Subject: [PATCH 142/327] Fix aggregate projections with normalized states --- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Columns/ColumnAggregateFunction.h | 2 +- src/Functions/FunctionsConversion.h | 39 ++++++++++++++++--- .../optimizeUseAggregateProjection.cpp | 17 +++++--- ...rojection_with_normalized_states.reference | 1 + ...gate_projection_with_normalized_states.sql | 29 ++++++++++++++ 6 files changed, 76 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 3ebb30df87e..0ec5db6c69d 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -73,7 +73,7 @@ ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & fu } -void ColumnAggregateFunction::set(const AggregateFunctionPtr & func_, size_t version_) +void ColumnAggregateFunction::set(const AggregateFunctionPtr & func_, std::optional version_) { func = func_; version = version_; diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 7c7201e585a..e37a8a5f109 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -103,7 +103,7 @@ private: public: ~ColumnAggregateFunction() override; - void set(const AggregateFunctionPtr & func_, size_t version_); + void set(const AggregateFunctionPtr & func_, std::optional version_ = std::nullopt); AggregateFunctionPtr getAggregateFunction() { return func; } AggregateFunctionPtr getAggregateFunction() const { return func; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 00e2ebcda43..535723bce4e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -3188,14 +3189,40 @@ private: { return &ConvertImplGenericFromString::execute; } - else + else if (const auto * agg_type = checkAndGetDataType(from_type_untyped.get())) { - if (cast_type == CastType::accurateOrNull) - return createToNullableColumnWrapper(); - else - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", - from_type_untyped->getName(), to_type->getName()); + if (agg_type->getFunction()->haveSameStateRepresentation(*to_type->getFunction())) + { + return [agg_type]( + ColumnsWithTypeAndName & arguments, + const DataTypePtr & /* result_type */, + const ColumnNullable * /* nullable_source */, + size_t /*input_rows_count*/) -> ColumnPtr + { + const auto & argument_column = arguments.front(); + const auto * col_agg = checkAndGetColumn(argument_column.column.get()); + if (col_agg) + { + auto new_col_agg = ColumnAggregateFunction::create(*col_agg); + new_col_agg->set(agg_type->getFunction()); + return new_col_agg; + } + else + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Illegal column {} for function CAST AS AggregateFunction", + argument_column.column->getName()); + } + }; + } } + + if (cast_type == CastType::accurateOrNull) + return createToNullableColumnWrapper(); + else + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion from {} to {} is not supported", + from_type_untyped->getName(), to_type->getName()); } WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray & to_type) const diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index eab4d3f5d43..0b24d2adbf9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -143,12 +143,12 @@ std::optional matchAggregateFunctions( argument_types.clear(); const auto & candidate = info.aggregates[idx]; - /// Note: this check is a bit strict. - /// We check that aggregate function names, argument types and parameters are equal. /// In some cases it's possible only to check that states are equal, /// e.g. for quantile(0.3)(...) and quantile(0.5)(...). - /// But also functions sum(...) and sumIf(...) will have equal states, - /// and we can't replace one to another from projection. + /// + /// Note we already checked that aggregate function names are equal, + /// so that functions sum(...) and sumIf(...) with equal states will + /// not match. if (!candidate.function->getStateType()->equals(*aggregate.function->getStateType())) { // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Cannot match agg func {} vs {} by state {} vs {}", @@ -249,12 +249,17 @@ static void appendAggregateFunctions( auto & input = inputs[match.description]; if (!input) - input = &proj_dag.addInput(match.description->column_name, std::move(type)); + input = &proj_dag.addInput(match.description->column_name, type); const auto * node = input; if (node->result_name != aggregate.column_name) - node = &proj_dag.addAlias(*node, aggregate.column_name); + { + /// Always cast to aggregate types specified in query, because input + /// columns from projection might have the same state but different + /// type, which can generate wrong results during finalization. + node = &proj_dag.addCast(*node, type, aggregate.column_name); + } proj_dag_outputs.push_back(node); } diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference new file mode 100644 index 00000000000..9db43c50b71 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference @@ -0,0 +1 @@ +950 990 500 2000 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql new file mode 100644 index 00000000000..f7be9239c11 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS r; + +CREATE TABLE r ( + x String, + a LowCardinality(String), + q AggregateFunction(quantilesTiming(0.5, 0.95, 0.99), Int64), + s Int64, + PROJECTION p + (SELECT a, quantilesTimingMerge(0.5, 0.95, 0.99)(q), sum(s) GROUP BY a) +) Engine=SummingMergeTree order by (x, a); + +insert into r +select number%100 x, + 'x' a, + quantilesTimingState(0.5, 0.95, 0.99)(number::Int64) q, + sum(1) s +from numbers(1000) +group by x,a; + +SELECT + ifNotFinite(quantilesTimingMerge(0.95)(q)[1],0) as d1, + ifNotFinite(quantilesTimingMerge(0.99)(q)[1],0) as d2, + ifNotFinite(quantilesTimingMerge(0.50)(q)[1],0) as d3, + sum(s) +FROM cluster('test_cluster_two_shards', currentDatabase(), r) +WHERE a = 'x' +settings prefer_localhost_replica=0; + +DROP TABLE r; From b610faf685d672517377c70f5efd65d077d6f193 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 29 Aug 2023 18:58:56 +0200 Subject: [PATCH 143/327] Revert "Revert "Fixed wrong python test name pattern"" --- tests/integration/ci-runner.py | 53 ++++++++++------------------------ 1 file changed, 16 insertions(+), 37 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index aa89ccf11b3..5c3a7695119 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -9,6 +9,7 @@ import os import random import re import shutil +import string import subprocess import time import shlex @@ -430,19 +431,12 @@ class ClickhouseIntegrationTestsRunner: def _get_all_tests(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) - out_file = "all_tests.txt" + runner_opts = self._get_runner_opts() out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( - "cd {repo_path}/tests/integration && " - "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} -- --setup-plan " - "| tee '{out_file_full}' | grep -F '::' | sed -r 's/ \(fixtures used:.*//g; s/^ *//g; s/ *$//g' " - "| grep -v -F 'SKIPPED' | sort --unique > {out_file}".format( - repo_path=repo_path, - runner_opts=self._get_runner_opts(), - image_cmd=image_cmd, - out_file=out_file, - out_file_full=out_file_full, - ) + f"cd {repo_path}/tests/integration && " + f"timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} -- --setup-plan " + f"| tee '{out_file_full}'" ) logging.info("Getting all tests with cmd '%s'", cmd) @@ -450,34 +444,19 @@ class ClickhouseIntegrationTestsRunner: cmd, shell=True ) - all_tests_file_path = "{repo_path}/tests/integration/{out_file}".format( - repo_path=repo_path, out_file=out_file - ) - if ( - not os.path.isfile(all_tests_file_path) - or os.path.getsize(all_tests_file_path) == 0 - ): - if os.path.isfile(out_file_full): - # log runner output - logging.info("runner output:") - with open(out_file_full, "r") as all_tests_full_file: - for line in all_tests_full_file: - line = line.rstrip() - if line: - logging.info("runner output: %s", line) - else: - logging.info("runner output '%s' is empty", out_file_full) + all_tests = set() + with open(out_file_full, "r", encoding="utf-8") as all_tests_fd: + for line in all_tests_fd: + if ( + line[0] in string.whitespace # test names at the start of lines + or "::test" not in line # test names contain '::test' + or "SKIPPED" in line # pytest.mark.skip/-if + ): + continue + all_tests.add(line.strip()) - raise Exception( - "There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format( - all_tests_file_path - ) - ) + assert all_tests - all_tests = [] - with open(all_tests_file_path, "r") as all_tests_file: - for line in all_tests_file: - all_tests.append(line.strip()) return list(sorted(all_tests)) def _get_parallel_tests_skip_list(self, repo_path): From 5f0d94d5236ad441a61b69a7db36ba9bd77f61a2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 30 Aug 2023 09:31:26 +0200 Subject: [PATCH 144/327] Disable broken tests to have the bug fixed --- tests/integration/test_concurrent_backups_s3/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_concurrent_backups_s3/test.py b/tests/integration/test_concurrent_backups_s3/test.py index b29058865c0..312ebdba5e3 100644 --- a/tests/integration/test_concurrent_backups_s3/test.py +++ b/tests/integration/test_concurrent_backups_s3/test.py @@ -24,6 +24,7 @@ def start_cluster(): cluster.shutdown() +@pytest.mark.skip(reason="broken test") def test_concurrent_backups(start_cluster): node.query("DROP TABLE IF EXISTS s3_test SYNC") columns = [f"column_{i} UInt64" for i in range(1000)] From b17dc582a548daed6650b484df1224fcf3730684 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sat, 9 Sep 2023 20:13:32 +0200 Subject: [PATCH 145/327] Disable test_disk_over_web_server for analyzer --- tests/integration/test_disk_over_web_server/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index bc60e5c865e..2edc05c4d7e 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -13,6 +13,7 @@ def cluster(): "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True, + allow_analyzer=False, ) cluster.add_instance( "node2", @@ -20,12 +21,14 @@ def cluster(): with_nginx=True, stay_alive=True, with_zookeeper=True, + allow_analyzer=False, ) cluster.add_instance( "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True, with_zookeeper=True, + allow_analyzer=False, ) cluster.add_instance( @@ -144,7 +147,6 @@ def test_usage(cluster, node_name): def test_incorrect_usage(cluster): - node1 = cluster.instances["node1"] node2 = cluster.instances["node3"] global uuids node2.query( @@ -186,7 +188,7 @@ def test_cache(cluster, node_name): (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'cached_web'; """.format( - i, uuids[i], i, i + i, uuids[i] ) ) From c7e058fa6e017a7539e702f87dd7b94d8c53c797 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Sat, 9 Sep 2023 20:16:36 +0200 Subject: [PATCH 146/327] Disable broken tests --- tests/integration/test_concurrent_threads_soft_limit/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_concurrent_threads_soft_limit/test.py b/tests/integration/test_concurrent_threads_soft_limit/test.py index 2f76f44ddc2..d1e233ee12f 100644 --- a/tests/integration/test_concurrent_threads_soft_limit/test.py +++ b/tests/integration/test_concurrent_threads_soft_limit/test.py @@ -51,6 +51,7 @@ def test_concurrent_threads_soft_limit_default(started_cluster): ) +@pytest.mark.skip(reason="broken test") def test_concurrent_threads_soft_limit_defined_50(started_cluster): node2.query( "SELECT count(*) FROM numbers_mt(10000000)", @@ -65,6 +66,7 @@ def test_concurrent_threads_soft_limit_defined_50(started_cluster): ) +@pytest.mark.skip(reason="broken test") def test_concurrent_threads_soft_limit_defined_1(started_cluster): node3.query( "SELECT count(*) FROM numbers_mt(10000000)", @@ -82,6 +84,7 @@ def test_concurrent_threads_soft_limit_defined_1(started_cluster): # In config_limit_reached.xml there is concurrent_threads_soft_limit=10 # Background query starts in a separate thread to reach this limit. # When this limit is reached the foreground query gets less than 5 queries despite the fact that it has settings max_threads=5 +@pytest.mark.skip(reason="broken test") def test_concurrent_threads_soft_limit_limit_reached(started_cluster): def background_query(): try: From 5535c7462dd7ff719f84d3b5e52fa028152338bf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Sep 2023 18:33:43 +0200 Subject: [PATCH 147/327] Support SAMPLE BY for VIEW Signed-off-by: Azat Khuzhin --- src/Storages/StorageView.cpp | 41 +++++++++++++------ .../0_stateless/02881_view_sampling.reference | 7 ++++ .../0_stateless/02881_view_sampling.sql | 18 ++++++++ tests/queries/1_stateful/00056_view.reference | 20 ++++----- 4 files changed, 63 insertions(+), 23 deletions(-) create mode 100644 tests/queries/0_stateless/02881_view_sampling.reference create mode 100644 tests/queries/0_stateless/02881_view_sampling.sql diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index f0f9b9540de..c92e72e7c0d 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -101,6 +101,19 @@ ContextPtr getViewContext(ContextPtr context) return view_context; } +ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) +{ + if (!select_query.tables() || select_query.tables()->children.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no table expression in view select AST"); + + auto * select_element = select_query.tables()->children[0]->as(); + + if (!select_element->table_expression) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression"); + + return select_element->table_expression->as(); +} + } StorageView::StorageView( @@ -148,6 +161,21 @@ void StorageView::read( current_inner_query = query_info.view_query->clone(); } + const auto & select_query = query_info.query->as(); + if (auto sample_size = select_query.sampleSize(), sample_offset = select_query.sampleOffset(); sample_size || sample_offset) + { + for (auto & inner_select_query : current_inner_query->as().list_of_selects->children) + { + if (auto * select = inner_select_query->as(); select) + { + ASTTableExpression * table_expression = getFirstTableExpression(*select); + + table_expression->sample_offset = sample_offset; + table_expression->sample_size = sample_size; + } + } + } + auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done); if (context->getSettingsRef().allow_experimental_analyzer) @@ -196,19 +224,6 @@ void StorageView::read( query_plan.addStep(std::move(converting)); } -static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) -{ - if (!select_query.tables() || select_query.tables()->children.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no table expression in view select AST"); - - auto * select_element = select_query.tables()->children[0]->as(); - - if (!select_element->table_expression) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression"); - - return select_element->table_expression->as(); -} - void StorageView::replaceQueryParametersIfParametrizedView(ASTPtr & outer_query, const NameToNameMap & parameter_values) { ReplaceQueryParameterVisitor visitor(parameter_values); diff --git a/tests/queries/0_stateless/02881_view_sampling.reference b/tests/queries/0_stateless/02881_view_sampling.reference new file mode 100644 index 00000000000..13ad4c0b16f --- /dev/null +++ b/tests/queries/0_stateless/02881_view_sampling.reference @@ -0,0 +1,7 @@ +-- { echo } +select * from data_mt sample 0.1 order by key; +9 +select * from view_mt sample 0.1 order by key; +9 +select * from data_mem sample 0.1 order by key; -- { serverError SAMPLING_NOT_SUPPORTED } +select * from view_mem sample 0.1 order by key; -- { serverError SAMPLING_NOT_SUPPORTED } diff --git a/tests/queries/0_stateless/02881_view_sampling.sql b/tests/queries/0_stateless/02881_view_sampling.sql new file mode 100644 index 00000000000..22fc7c6b923 --- /dev/null +++ b/tests/queries/0_stateless/02881_view_sampling.sql @@ -0,0 +1,18 @@ +drop table if exists data_mt; +drop table if exists view_mt; +drop table if exists data_mem; +drop table if exists view_mem; + +create table data_mt (key Int) engine=MergeTree() order by (key, sipHash64(key)) sample by sipHash64(key); +insert into data_mt select * from numbers(10); +create view view_mt as select * from data_mt; + +create table data_mem (key Int) engine=Memory(); +insert into data_mem select * from numbers(10); +create view view_mem as select * from data_mem; + +-- { echo } +select * from data_mt sample 0.1 order by key; +select * from view_mt sample 0.1 order by key; +select * from data_mem sample 0.1 order by key; -- { serverError SAMPLING_NOT_SUPPORTED } +select * from view_mem sample 0.1 order by key; -- { serverError SAMPLING_NOT_SUPPORTED } diff --git a/tests/queries/1_stateful/00056_view.reference b/tests/queries/1_stateful/00056_view.reference index 30b3a449c1e..20382f13238 100644 --- a/tests/queries/1_stateful/00056_view.reference +++ b/tests/queries/1_stateful/00056_view.reference @@ -19,13 +19,13 @@ 59183 85379 33010362 77807 800784 77492 -1704509 523264 -732797 475698 -598875 337212 -792887 252197 -3807842 196036 -25703952 147211 -716829 90109 -59183 85379 -33010362 77807 -800784 77492 +732797 46144 +1704509 40129 +598875 30482 +792887 22585 +3807842 18891 +11312316 13181 +25703952 11177 +59183 9935 +4379238 8593 +716829 8411 From caadfe393bee6da65fe00a01943925d87ec5f22d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Sep 2023 12:55:15 +0000 Subject: [PATCH 148/327] Fixes --- .../functions/date-time-functions.md | 6 ++--- src/Functions/DateTimeTransforms.h | 10 +++++++-- src/Functions/toDaysSinceYearZero.cpp | 14 ++++++++++-- .../02874_daysSinceYearZero.reference | 13 ----------- .../0_stateless/02874_daysSinceYearZero.sql | 22 ------------------- .../02874_toDaysSinceYearZero.reference | 13 +++++++++++ .../0_stateless/02874_toDaysSinceYearZero.sql | 22 +++++++++++++++++++ 7 files changed, 58 insertions(+), 42 deletions(-) delete mode 100644 tests/queries/0_stateless/02874_daysSinceYearZero.reference delete mode 100644 tests/queries/0_stateless/02874_daysSinceYearZero.sql create mode 100644 tests/queries/0_stateless/02874_toDaysSinceYearZero.reference create mode 100644 tests/queries/0_stateless/02874_toDaysSinceYearZero.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index ef0ccf84f73..898d57c64db 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -722,7 +722,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## toDaysSinceYearZero -Returns for a given date, the number of days passed since the [year 0](https://en.wikipedia.org/wiki/Year_zero). +Returns for a given date, the number of days passed since [1 January 0000](https://en.wikipedia.org/wiki/Year_zero) in the [proleptic Gregorian calendar defined by ISO 8601](https://en.wikipedia.org/wiki/Gregorian_calendar#Proleptic_Gregorian_calendar). The calculation is the same as in MySQL's [`TO_DAYS()`](https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_to-days) function. **Syntax** @@ -734,7 +734,7 @@ Aliases: `TO_DAYS` **Arguments** -- `date` — The date to calculate the number of days passed since year zero from. [Date](../../sql-reference/data-types/date.md) or [Date64](../../sql-reference/data-types/date64.md). +- `date` — The date to calculate the number of days passed since year zero from. [Date](../../sql-reference/data-types/date.md) or [Date32](../../sql-reference/data-types/date32.md). **Returned value** @@ -752,7 +752,7 @@ Result: ``` text ┌─toDaysSinceYearZero(toDate('2023-09-08')))─┐ -│ 738772 │ +│ 713569 │ └────────────────────────────────────────────┘ ``` diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 16cfba69275..a351d7fdf30 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -927,6 +927,12 @@ struct ToDayOfYearImpl struct ToDaysSinceYearZeroImpl { +private: + /// Constant calculated from MySQL's TO_DAYS() implementation. + /// https://github.com/mysql/mysql-server/blob/ea1efa9822d81044b726aab20c857d5e1b7e046a/mysys/my_time.cc#L1042 + static constexpr auto DAYS_BETWEEN_YEARS_0_AND_1900 = 693'961; /// 01 January, each + +public: static constexpr auto name = "toDaysSinceYearZero"; static UInt32 execute(Int64, const DateLUTImpl &) @@ -939,11 +945,11 @@ struct ToDaysSinceYearZeroImpl } static UInt32 execute(Int32 d, const DateLUTImpl &) { - return /* days between 0000-01-01 and 1970-01-01 */ 719'164 + d; + return DAYS_BETWEEN_YEARS_0_AND_1900 + d; } static UInt32 execute(UInt16 d, const DateLUTImpl &) { - return /* days between 0000-01-01 and 1970-01-01 */ 719'164 + d; + return DAYS_BETWEEN_YEARS_0_AND_1900 + d; } static constexpr bool hasPreimage() { return false; } diff --git a/src/Functions/toDaysSinceYearZero.cpp b/src/Functions/toDaysSinceYearZero.cpp index c4233416195..e569c5cb1f3 100644 --- a/src/Functions/toDaysSinceYearZero.cpp +++ b/src/Functions/toDaysSinceYearZero.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes namespace { -/** Returns number of days passed since 0001-01-01 */ +/** Returns number of days passed since 0000-01-01 */ class FunctionToDaysSinceYearZero : public IFunction { using ResultType = DataTypeUInt32; @@ -65,7 +65,17 @@ public: REGISTER_FUNCTION(ToDaysSinceYearZero) { - factory.registerFunction(); + factory.registerFunction( + FunctionDocumentation{ + .description=R"( +Returns for a given date, the number of days passed since 1 January 0000 in the proleptic Gregorian calendar defined by ISO 8601. +The calculation is the same as in MySQL's TO_DAYS() function. +)", + .examples{ + {"typical", "SELECT toDaysSinceYearZero(toDate('2023-09-08'))", "713569"}}, + .categories{"Dates and Times"} + }); + /// MySQL compatibility alias. factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); } diff --git a/tests/queries/0_stateless/02874_daysSinceYearZero.reference b/tests/queries/0_stateless/02874_daysSinceYearZero.reference deleted file mode 100644 index 9bfb9b78910..00000000000 --- a/tests/queries/0_stateless/02874_daysSinceYearZero.reference +++ /dev/null @@ -1,13 +0,0 @@ -Reject invalid parameters -Const argument -719164 -738772 -693597 -738772 -\N -Non-const argument -738772 -738772 -MySQL alias -738772 -738772 diff --git a/tests/queries/0_stateless/02874_daysSinceYearZero.sql b/tests/queries/0_stateless/02874_daysSinceYearZero.sql deleted file mode 100644 index 0db1a305ea9..00000000000 --- a/tests/queries/0_stateless/02874_daysSinceYearZero.sql +++ /dev/null @@ -1,22 +0,0 @@ -SELECT 'Reject invalid parameters'; -SELECT daysSinceYearZero(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT daysSinceYearZero(toDate('2023-09-08'), toDate('2023-09-08')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT daysSinceYearZero('str'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT daysSinceYearZero(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT daysSinceYearZero(toDateTime('2023-09-08 11:11:11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT daysSinceYearZero(toDateTime64('2023-09-08 11:11:11', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } - -SELECT 'Const argument'; -SELECT daysSinceYearZero(toDate('1970-01-01')); -SELECT daysSinceYearZero(toDate('2023-09-08')); -SELECT daysSinceYearZero(toDate32('1900-01-01')); -SELECT daysSinceYearZero(toDate32('2023-09-08')); -SELECT daysSinceYearZero(NULL); - -SELECT 'Non-const argument'; -SELECT daysSinceYearZero(materialize(toDate('2023-09-08'))); -SELECT daysSinceYearZero(materialize(toDate32('2023-09-08'))); - -SELECT 'MySQL alias'; -SELECT to_days(toDate('2023-09-08')); -SELECT TO_DAYS(toDate('2023-09-08')); diff --git a/tests/queries/0_stateless/02874_toDaysSinceYearZero.reference b/tests/queries/0_stateless/02874_toDaysSinceYearZero.reference new file mode 100644 index 00000000000..885332ab835 --- /dev/null +++ b/tests/queries/0_stateless/02874_toDaysSinceYearZero.reference @@ -0,0 +1,13 @@ +Invalid parameters +Const argument +693961 +713569 +668394 +713569 +\N +Non-const argument +713569 +713569 +MySQL alias +713569 +713569 diff --git a/tests/queries/0_stateless/02874_toDaysSinceYearZero.sql b/tests/queries/0_stateless/02874_toDaysSinceYearZero.sql new file mode 100644 index 00000000000..2c35920e569 --- /dev/null +++ b/tests/queries/0_stateless/02874_toDaysSinceYearZero.sql @@ -0,0 +1,22 @@ +SELECT 'Invalid parameters'; +SELECT toDaysSinceYearZero(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT toDaysSinceYearZero(toDate('2023-09-08'), toDate('2023-09-08')); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT toDaysSinceYearZero('str'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toDaysSinceYearZero(42); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toDaysSinceYearZero(toDateTime('2023-09-08 11:11:11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT toDaysSinceYearZero(toDateTime64('2023-09-08 11:11:11', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'Const argument'; +SELECT toDaysSinceYearZero(toDate('1970-01-01')); +SELECT toDaysSinceYearZero(toDate('2023-09-08')); +SELECT toDaysSinceYearZero(toDate32('1900-01-01')); +SELECT toDaysSinceYearZero(toDate32('2023-09-08')); +SELECT toDaysSinceYearZero(NULL); + +SELECT 'Non-const argument'; +SELECT toDaysSinceYearZero(materialize(toDate('2023-09-08'))); +SELECT toDaysSinceYearZero(materialize(toDate32('2023-09-08'))); + +SELECT 'MySQL alias'; +SELECT to_days(toDate('2023-09-08')); +SELECT TO_DAYS(toDate('2023-09-08')); From d326554caaf8c7536463c1d3d73d6ec7c0f64c19 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 10 Sep 2023 14:55:40 +0000 Subject: [PATCH 149/327] fix keeper build --- src/Coordination/Standalone/Context.cpp | 10 ++++++++++ src/Coordination/Standalone/Context.h | 4 ++++ src/Disks/ObjectStorages/DiskObjectStorage.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/Context.h | 2 +- 5 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 84083169df3..f0ba16e50ee 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -268,6 +268,16 @@ ReadSettings Context::getReadSettings() const return ReadSettings{}; } +ResourceManagerPtr Context::getResourceManager() const +{ + return nullptr; +} + +ClassifierPtr Context::getWorkloadClassifier() const +{ + return nullptr; +} + void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const { const auto & config_ref = getConfigRef(); diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index cf0fceab34c..95f63d40560 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -118,6 +118,10 @@ public: ReadSettings getReadSettings() const; + /// Resource management related + ResourceManagerPtr getResourceManager() const; + ClassifierPtr getWorkloadClassifier() const; + std::shared_ptr getKeeperDispatcher() const; std::shared_ptr tryGetKeeperDispatcher() const; void initializeKeeperDispatcher(bool start_async) const; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 5a70b2ca579..466a1d3d5dd 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -491,7 +491,7 @@ static inline Settings updateResourceLink(const Settings & settings, const Strin if (auto query_context = CurrentThread::getQueryContext()) { Settings result(settings); - result.resource_link = query_context->getClassifier()->get(resource_name); + result.resource_link = query_context->getWorkloadClassifier()->get(resource_name); return result; } return settings; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 8074fd8afa8..0b5294b8d00 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1294,7 +1294,7 @@ ResourceManagerPtr Context::getResourceManager() const return shared->resource_manager; } -ClassifierPtr Context::getClassifier() const +ClassifierPtr Context::getWorkloadClassifier() const { auto lock = getLock(); if (!classifier) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index eceed31ae86..a4019d214e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -581,7 +581,7 @@ public: /// Resource management related ResourceManagerPtr getResourceManager() const; - ClassifierPtr getClassifier() const; + ClassifierPtr getWorkloadClassifier() const; /// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once. void setExternalTablesInitializer(ExternalTablesInitializer && initializer); From 6a2534b555ff38f740f4344ee9d4f1381a4a8ab7 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 10 Sep 2023 17:43:04 +0000 Subject: [PATCH 150/327] fix keeper build #2 --- src/Coordination/Standalone/Context.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 95f63d40560..d9bbccea2e9 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -13,6 +13,7 @@ #include #include +#include #include From eed11c36a4b2c95ec162d029c6c78e20efa151d5 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Sun, 10 Sep 2023 23:11:24 +0200 Subject: [PATCH 151/327] Add jq to fasttest container --- docker/test/fasttest/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index ad24e662a6c..fd7a5640964 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -18,6 +18,7 @@ RUN apt-get update \ python3-termcolor \ unixodbc \ pv \ + jq \ zstd \ --yes --no-install-recommends From aac974b38d320ec4a2d827457078062413393a14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Sep 2023 05:35:15 +0200 Subject: [PATCH 152/327] Fix strange message --- cmake/limit_jobs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index acc38b6fa2a..28ccb62e10c 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -47,7 +47,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB DRAM, 'OFF' means the native core count).") +message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB RAM, 'OFF' means the native core count).") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From b72a49f090dcb5b90f6585d9097872ea0d0c865f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Sep 2023 06:32:14 +0000 Subject: [PATCH 153/327] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2e45b885afd..34b1766d8eb 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1997,6 +1997,7 @@ privateKeyPassphraseHandler prlimit procfs profiler +proleptic prometheus proto protobuf @@ -2353,6 +2354,7 @@ toDateTimeOrZero toDayOfMonth toDayOfWeek toDayOfYear +toDaysSinceYearZero toDecimal toDecimalString toFixedString From ea038380b036987714125bc8e6f2df8f3c49ebbb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Sep 2023 07:12:49 +0000 Subject: [PATCH 154/327] Bump re2 to latest main Now supports an alternative form for named capturing groups [0]: (?re) [0] https://github.com/google/re2/commit/6148386f0c8f03da1ab0bf982a4dbe080b4ea7bc --- contrib/re2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/re2 b/contrib/re2 index 03da4fc0857..a807e8a3aac 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit 03da4fc0857c285e3a26782f6bc8931c4c950df4 +Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c From beb9a75a6819ed601471104a309bc05ebb6c73ac Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Sep 2023 12:40:25 +0200 Subject: [PATCH 155/327] A few more --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ---- docs/zh/engines/table-engines/mergetree-family/mergetree.md | 5 ----- programs/diagnostics/testdata/configs/xml/config.xml | 2 -- programs/server/config.xml | 2 -- 4 files changed, 13 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 1b0dab2b3a5..00eb830c9ef 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -805,8 +805,6 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 4 1000 /var/lib/clickhouse/disks/s3/ - true - /var/lib/clickhouse/disks/s3/cache/ false @@ -832,8 +830,6 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' - `single_read_retries` — число попыток выполнения запроса в случае возникновения ошибки в процессе чтения. Значение по умолчанию: `4`. - `min_bytes_for_seek` — минимальное количество байтов, которые используются для операций поиска вместо последовательного чтения. Значение по умолчанию: 1 МБайт. - `metadata_path` — путь к локальному файловому хранилищу для хранения файлов с метаданными для S3. Значение по умолчанию: `/var/lib/clickhouse/disks//`. -- `cache_enabled` — признак, разрешено ли хранение кэша засечек и файлов индекса в локальной файловой системе. Значение по умолчанию: `true`. -- `cache_path` — путь в локальной файловой системе, где будут храниться кэш засечек и файлы индекса. Значение по умолчанию: `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — признак, выполнять ли проверку доступов при запуске диска. Если установлено значение `true`, то проверка не выполняется. Значение по умолчанию: `false`. Диск S3 может быть сконфигурирован как `main` или `cold`: diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index 78b9c678eb9..cec4cb09047 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -745,8 +745,6 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 4 1000 /var/lib/clickhouse/disks/s3/ - true - /var/lib/clickhouse/disks/s3/cache/ false @@ -772,8 +770,6 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' - `single_read_retries` - 读过程中连接丢失后重试次数,默认值为4。 - `min_bytes_for_seek` - 使用查找操作,而不是顺序读操作的最小字节数,默认值为1000。 - `metadata_path` - 本地存放S3元数据文件的路径,默认值为`/var/lib/clickhouse/disks//` -- `cache_enabled` - 是否允许缓存标记和索引文件。默认值为`true`。 -- `cache_path` - 本地缓存标记和索引文件的路径。默认值为`/var/lib/clickhouse/disks//cache/`。 - `skip_access_check` - 如果为`true`,Clickhouse启动时不检查磁盘是否可用。默认为`false`。 - `server_side_encryption_customer_key_base64` - 如果指定该项的值,请求时会加上为了访问SSE-C加密数据而必须的头信息。 @@ -823,4 +819,3 @@ S3磁盘也可以设置冷热存储: - `_part_uuid` - 唯一部分标识符(如果 MergeTree 设置`assign_part_uuids` 已启用)。 - `_partition_value` — `partition by` 表达式的值(元组)。 - `_sample_factor` - 采样因子(来自请求)。 - diff --git a/programs/diagnostics/testdata/configs/xml/config.xml b/programs/diagnostics/testdata/configs/xml/config.xml index c08b0b2970f..c9c0a824ef6 100644 --- a/programs/diagnostics/testdata/configs/xml/config.xml +++ b/programs/diagnostics/testdata/configs/xml/config.xml @@ -1209,8 +1209,6 @@ 4 1000 /var/lib/clickhouse/disks/s3/ - true - /var/lib/clickhouse/disks/s3/cache/ false diff --git a/programs/server/config.xml b/programs/server/config.xml index 117be72d758..39ad1f82eed 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -448,8 +448,6 @@ account pass123 /var/lib/clickhouse/disks/blob_storage_disk/ - true - /var/lib/clickhouse/disks/blob_storage_disk/cache/ false From f9351fdf7be58dcba4b073fcea92e78c5df750b1 Mon Sep 17 00:00:00 2001 From: Alexander van Olst Date: Mon, 11 Sep 2023 13:52:14 +0200 Subject: [PATCH 156/327] Correct default value for parts_to_delay_insert --- docs/en/operations/settings/merge-tree-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 8889e04dba1..e746719b629 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -71,7 +71,7 @@ Possible values: - Any positive integer. -Default value: 150. +Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. From 5061da4e8ace9664e880cb67070e295f76477da5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 12:12:02 +0000 Subject: [PATCH 157/327] clickhouse-local support --database command line argument --- programs/local/LocalServer.cpp | 2 ++ .../02141_clickhouse_local_interactive_table.reference | 1 + .../0_stateless/02141_clickhouse_local_interactive_table.sh | 1 + 3 files changed, 4 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5c522b678ef..3fa2532181a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -881,6 +881,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp config().setBool("no-system-tables", true); if (options.count("only-system-tables")) config().setBool("only-system-tables", true); + if (options.count("database")) + config().setString("default_database", options["database"].as()); if (options.count("input-format")) config().setString("table-data-format", options["input-format"].as()); diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference index e4c93e9e1c5..4c1d5dc829f 100644 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.reference @@ -1 +1,2 @@ CREATE TABLE _local.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') +CREATE TABLE foo.table\n(\n `key` String\n)\nENGINE = File(\'TSVWithNamesAndTypes\', \'/dev/null\') diff --git a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh index fc71f779fa1..934d87616ac 100755 --- a/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh +++ b/tests/queries/0_stateless/02141_clickhouse_local_interactive_table.sh @@ -5,3 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' +$CLICKHOUSE_LOCAL --database foo --file /dev/null --structure "key String" --input-format TSVWithNamesAndTypes --interactive --send_logs_level=trace <<<'show create table table' From 1494eb3a864caa53cf8734e13f97f6e08b24a47a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 11 Sep 2023 12:34:16 +0000 Subject: [PATCH 158/327] Fix version parsing --- tests/integration/helpers/cluster.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7ee86d74d35..dfd964b04d2 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4260,7 +4260,19 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - if self.tag == "latest": + version = None + version_parts = self.tag.split(".") + if version_parts[0].isdigit() and version_parts[1].isdigit(): + version = {"major": int(version_parts[0]), "minor": int(version_parts[1])} + + # async replication is only supported in version 23.9+ + # for tags that don't specify a version we assume it has a version of ClickHouse + # that supports async replication if a test for it is present + if ( + version == None + or version["major"] > 23 + or (version["major"] == 23 and version["minor"] >= 9) + ): write_embedded_config( "0_common_enable_keeper_async_replication.xml", self.config_d_dir ) From 4a8f3bf36397311b6049b6e918211f80a95f36e1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Sep 2023 13:01:55 +0000 Subject: [PATCH 159/327] Add SYSTEM FLUSH LOGS --- tests/queries/0_stateless/01165_lost_part_empty_partition.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql index 8486a6ac2e6..a1db1c27bee 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.sql +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.sql @@ -12,6 +12,7 @@ drop table rmt1; system sync replica rmt2; select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; +SYSTEM FLUSH LOGS; select count() from system.text_log where logger_name like '%' || currentDatabase() || '%' and message ilike '%table with non-zero lost_part_count equal to%'; @@ -25,6 +26,7 @@ drop table rmt1; system sync replica rmt2; select lost_part_count from system.replicas where database = currentDatabase() and table = 'rmt2'; drop table rmt2; +SYSTEM FLUSH LOGS; select count() from system.text_log where logger_name like '%' || currentDatabase() || '%' and message ilike '%table with non-zero lost_part_count equal to%'; From 297306b4b974188ec19d8d83c5f1f965974619ad Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 13:36:02 +0000 Subject: [PATCH 160/327] status emoji in the beginning of title --- tests/ci/commit_status_helper.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 9788e629cb8..ed686dae11c 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -146,9 +146,9 @@ def update_pr_title_icon(pr: PullRequest, status: str) -> None: if not new_status_icon: return new_title = pr.title - if new_title and new_title[-1] != new_status_icon: - if new_title[-1] in set(STATUS_ICON_MAP.values()): - new_title = new_title[:-1] + " " + new_status_icon + if new_title and new_title[0] != new_status_icon: + if new_title[0] in set(STATUS_ICON_MAP.values()): + new_title = new_title[1:] + new_status_icon else: new_title = new_title + " " + new_status_icon pr.edit(title=new_title) From 73ce1993fb524ba87a9097ae6df2f3ac593feb08 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Sep 2023 12:40:49 +0000 Subject: [PATCH 161/327] Continue #45070 --- .../functions/date-time-functions.md | 95 ++++- src/Common/DateLUTImpl.h | 222 ++++++------ src/Functions/makeDate.cpp | 335 +++++++++++++++--- .../0_stateless/02245_make_datetime64.sql | 2 +- ...new_functions_must_be_documented.reference | 4 + .../02876_yyyymmddhhmmsstodatetime.reference | 85 +++++ .../02876_yyyymmddhhmmsstodatetime.sql | 119 +++++++ .../02876_yyyymmddtodate.reference | 84 +++++ .../0_stateless/02876_yyyymmddtodate.sql | 112 ++++++ 9 files changed, 900 insertions(+), 158 deletions(-) create mode 100644 tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference create mode 100644 tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql create mode 100644 tests/queries/0_stateless/02876_yyyymmddtodate.reference create mode 100644 tests/queries/0_stateless/02876_yyyymmddtodate.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index cc8469e82ef..8f128f13de0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1290,6 +1290,8 @@ Rounds the time to the half hour. Converts a date or date with time to a UInt32 number containing the year and month number (YYYY \* 100 + MM). Accepts a second optional timezone argument. If provided, the timezone must be a string constant. +This functions is the opposite of function `YYYYMMDDToDate()`. + **Example** ``` sql @@ -1312,8 +1314,7 @@ Converts a date or date with time to a UInt32 number containing the year and mon **Example** ```sql -SELECT - toYYYYMMDD(now(), 'US/Eastern') +SELECT toYYYYMMDD(now(), 'US/Eastern') ``` Result: @@ -1331,8 +1332,7 @@ Converts a date or date with time to a UInt64 number containing the year and mon **Example** ```sql -SELECT - toYYYYMMDDhhmmss(now(), 'US/Eastern') +SELECT toYYYYMMDDhhmmss(now(), 'US/Eastern') ``` Result: @@ -1343,6 +1343,93 @@ Result: └───────────────────────────────────────┘ ``` +## YYYYMMDDToDate + +Converts a number containing the year, month and day number to a [Date](../../sql-reference/data-types/date.md). + +This functions is the opposite of function `toYYYYMMDD()`. + +The output is undefined if the input does not encode a valid Date value. + +**Syntax** + +```sql +YYYYMMDDToDate(yyyymmdd); +``` + +**Arguments** + +- `yyyymmdd` - A number representing the year, month and day. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). + +**Returned value** + +- a date created from the arguments. + +Type: [Date](../../sql-reference/data-types/date.md). + +**Example** + +```sql +SELECT YYYYMMDDToDate(20230911); +``` + +Result: + +```response +┌─toYYYYMMDD(20230911)─┐ +│ 2023-09-11 │ +└──────────────────────┘ +``` + +## YYYYMMDDToDate32 + +Like function `YYYYMMDDToDate()` but produces a [Date32](../../sql-reference/data-types/date32.md). + +## YYYYMMDDhhmmssToDateTime + +Converts a number containing the year, month and day number to a [DateTime](../../sql-reference/data-types/datetime.md). + +The output is undefined if the input does not encode a valid DateTime value. + +This functions is the opposite of function `toYYYYMMDD()`. + +**Syntax** + +```sql +YYYYMMDDhhmmssToDateTime(yyyymmddhhmmss[, timezone]); +``` + +**Arguments** + +- `yyyymmddhhmmss` - A number representing the year, month and day. [Integer](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). +- `timezone` - [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). + +**Returned value** + +- a date with time created from the arguments. + +Type: [DateTime](../../sql-reference/data-types/datetime.md). + +**Example** + +```sql +SELECT YYYYMMDDToDateTime(20230911131415); +``` + +Result: + +```response +┌────toYYYYMMDDhhmmssToDateTime(20230911131415)─┐ +│ 2023-09-11 13:14:15 │ +└───────────────────────────────────────────────┘ +``` + +## YYYYMMDDhhmmssToDateTime64 + +Like function `YYYYMMDDhhmmssToDate()` but produces a [DateTime64](../../sql-reference/data-types/datetime64.md). + +Accepts an additional, optional `precision` parameter after the `timezone` parameter. + ## addYears, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addQuarters Function adds a Date/DateTime interval to a Date/DateTime and then return the Date/DateTime. For example: diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 6d0ba718057..0e72b489ace 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -71,14 +71,14 @@ private: // Same as above but select different function overloads for zero saturation. STRONG_TYPEDEF(UInt32, LUTIndexWithSaturation) - static inline LUTIndex normalizeLUTIndex(UInt32 index) + static LUTIndex normalizeLUTIndex(UInt32 index) { if (index >= DATE_LUT_SIZE) return LUTIndex(DATE_LUT_SIZE - 1); return LUTIndex{index}; } - static inline LUTIndex normalizeLUTIndex(Int64 index) + static LUTIndex normalizeLUTIndex(Int64 index) { if (unlikely(index < 0)) return LUTIndex(0); @@ -88,59 +88,59 @@ private: } template - friend inline LUTIndex operator+(const LUTIndex & index, const T v) + friend LUTIndex operator+(const LUTIndex & index, const T v) { return normalizeLUTIndex(index.toUnderType() + UInt32(v)); } template - friend inline LUTIndex operator+(const T v, const LUTIndex & index) + friend LUTIndex operator+(const T v, const LUTIndex & index) { return normalizeLUTIndex(static_cast(v + index.toUnderType())); } - friend inline LUTIndex operator+(const LUTIndex & index, const LUTIndex & v) + friend LUTIndex operator+(const LUTIndex & index, const LUTIndex & v) { return normalizeLUTIndex(static_cast(index.toUnderType() + v.toUnderType())); } template - friend inline LUTIndex operator-(const LUTIndex & index, const T v) + friend LUTIndex operator-(const LUTIndex & index, const T v) { return normalizeLUTIndex(static_cast(index.toUnderType() - UInt32(v))); } template - friend inline LUTIndex operator-(const T v, const LUTIndex & index) + friend LUTIndex operator-(const T v, const LUTIndex & index) { return normalizeLUTIndex(static_cast(v - index.toUnderType())); } - friend inline LUTIndex operator-(const LUTIndex & index, const LUTIndex & v) + friend LUTIndex operator-(const LUTIndex & index, const LUTIndex & v) { return normalizeLUTIndex(static_cast(index.toUnderType() - v.toUnderType())); } template - friend inline LUTIndex operator*(const LUTIndex & index, const T v) + friend LUTIndex operator*(const LUTIndex & index, const T v) { return normalizeLUTIndex(index.toUnderType() * UInt32(v)); } template - friend inline LUTIndex operator*(const T v, const LUTIndex & index) + friend LUTIndex operator*(const T v, const LUTIndex & index) { return normalizeLUTIndex(v * index.toUnderType()); } template - friend inline LUTIndex operator/(const LUTIndex & index, const T v) + friend LUTIndex operator/(const LUTIndex & index, const T v) { return normalizeLUTIndex(index.toUnderType() / UInt32(v)); } template - friend inline LUTIndex operator/(const T v, const LUTIndex & index) + friend LUTIndex operator/(const T v, const LUTIndex & index) { return normalizeLUTIndex(UInt32(v) / index.toUnderType()); } @@ -172,12 +172,12 @@ public: Int8 amount_of_offset_change_value; /// Usually -4 or 4, but look at Lord Howe Island. Multiply by OffsetChangeFactor UInt8 time_at_offset_change_value; /// In seconds from beginning of the day. Multiply by OffsetChangeFactor - inline Int32 amount_of_offset_change() const /// NOLINT + Int32 amount_of_offset_change() const /// NOLINT { return static_cast(amount_of_offset_change_value) * OffsetChangeFactor; } - inline UInt32 time_at_offset_change() const /// NOLINT + UInt32 time_at_offset_change() const /// NOLINT { return static_cast(time_at_offset_change_value) * OffsetChangeFactor; } @@ -221,7 +221,7 @@ private: /// Time zone name. std::string time_zone; - inline LUTIndex findIndex(Time t) const + LUTIndex findIndex(Time t) const { /// First guess. Time guess = (t / 86400) + daynum_offset_epoch; @@ -248,34 +248,34 @@ private: return LUTIndex(guess ? static_cast(guess) - 1 : 0); } - static inline LUTIndex toLUTIndex(DayNum d) + static LUTIndex toLUTIndex(DayNum d) { return normalizeLUTIndex(d + daynum_offset_epoch); } - static inline LUTIndex toLUTIndex(ExtendedDayNum d) + static LUTIndex toLUTIndex(ExtendedDayNum d) { return normalizeLUTIndex(static_cast(d + daynum_offset_epoch)); } - inline LUTIndex toLUTIndex(Time t) const + LUTIndex toLUTIndex(Time t) const { return findIndex(t); } - static inline LUTIndex toLUTIndex(LUTIndex i) + static LUTIndex toLUTIndex(LUTIndex i) { return i; } template - inline const Values & find(DateOrTime v) const + const Values & find(DateOrTime v) const { return lut[toLUTIndex(v)]; } template - inline DateOrTime roundDown(DateOrTime x, Divisor divisor) const + DateOrTime roundDown(DateOrTime x, Divisor divisor) const { static_assert(std::is_integral_v && std::is_integral_v); assert(divisor > 0); @@ -336,7 +336,7 @@ public: } template - inline auto toDayNum(DateOrTime v) const + auto toDayNum(DateOrTime v) const { if constexpr (std::is_unsigned_v || std::is_same_v) return DayNum{static_cast(saturateMinus(toLUTIndex(v).toUnderType(), daynum_offset_epoch))}; @@ -346,7 +346,7 @@ public: /// Round down to start of monday. template - inline Time toFirstDayOfWeek(DateOrTime v) const + Time toFirstDayOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -356,7 +356,7 @@ public: } template - inline auto toFirstDayNumOfWeek(DateOrTime v) const + auto toFirstDayNumOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -367,7 +367,7 @@ public: /// Round up to the last day of week. template - inline Time toLastDayOfWeek(DateOrTime v) const + Time toLastDayOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -377,7 +377,7 @@ public: } template - inline auto toLastDayNumOfWeek(DateOrTime v) const + auto toLastDayNumOfWeek(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -388,7 +388,7 @@ public: /// Round down to start of month. template - inline Time toFirstDayOfMonth(DateOrTime v) const + Time toFirstDayOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -398,7 +398,7 @@ public: } template - inline auto toFirstDayNumOfMonth(DateOrTime v) const + auto toFirstDayNumOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -409,7 +409,7 @@ public: /// Round up to last day of month. template - inline Time toLastDayOfMonth(DateOrTime v) const + Time toLastDayOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -419,7 +419,7 @@ public: } template - inline auto toLastDayNumOfMonth(DateOrTime v) const + auto toLastDayNumOfMonth(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); if constexpr (std::is_unsigned_v || std::is_same_v) @@ -430,7 +430,7 @@ public: /// Round down to start of quarter. template - inline auto toFirstDayNumOfQuarter(DateOrTime v) const + auto toFirstDayNumOfQuarter(DateOrTime v) const { if constexpr (std::is_unsigned_v || std::is_same_v) return toDayNum(LUTIndexWithSaturation(toFirstDayOfQuarterIndex(v))); @@ -439,7 +439,7 @@ public: } template - inline LUTIndex toFirstDayOfQuarterIndex(DateOrTime v) const + LUTIndex toFirstDayOfQuarterIndex(DateOrTime v) const { LUTIndex index = toLUTIndex(v); size_t month_inside_quarter = (lut[index].month - 1) % 3; @@ -455,25 +455,25 @@ public: } template - inline Time toFirstDayOfQuarter(DateOrTime v) const + Time toFirstDayOfQuarter(DateOrTime v) const { return toDate(toFirstDayOfQuarterIndex(v)); } /// Round down to start of year. - inline Time toFirstDayOfYear(Time t) const + Time toFirstDayOfYear(Time t) const { return lut[years_lut[lut[findIndex(t)].year - DATE_LUT_MIN_YEAR]].date; } template - inline LUTIndex toFirstDayNumOfYearIndex(DateOrTime v) const + LUTIndex toFirstDayNumOfYearIndex(DateOrTime v) const { return years_lut[lut[toLUTIndex(v)].year - DATE_LUT_MIN_YEAR]; } template - inline auto toFirstDayNumOfYear(DateOrTime v) const + auto toFirstDayNumOfYear(DateOrTime v) const { if constexpr (std::is_unsigned_v || std::is_same_v) return toDayNum(LUTIndexWithSaturation(toFirstDayNumOfYearIndex(v))); @@ -481,14 +481,14 @@ public: return toDayNum(LUTIndex(toFirstDayNumOfYearIndex(v))); } - inline Time toFirstDayOfNextMonth(Time t) const + Time toFirstDayOfNextMonth(Time t) const { LUTIndex index = findIndex(t); index += 32 - lut[index].day_of_month; return lut[index - (lut[index].day_of_month - 1)].date; } - inline Time toFirstDayOfPrevMonth(Time t) const + Time toFirstDayOfPrevMonth(Time t) const { LUTIndex index = findIndex(t); index -= lut[index].day_of_month; @@ -496,13 +496,13 @@ public: } template - inline UInt8 daysInMonth(DateOrTime value) const + UInt8 daysInMonth(DateOrTime value) const { const LUTIndex i = toLUTIndex(value); return lut[i].days_in_month; } - inline UInt8 daysInMonth(Int16 year, UInt8 month) const + UInt8 daysInMonth(Int16 year, UInt8 month) const { UInt16 idx = year - DATE_LUT_MIN_YEAR; if (unlikely(idx >= DATE_LUT_YEARS)) @@ -515,12 +515,12 @@ public: /** Round to start of day, then shift for specified amount of days. */ - inline Time toDateAndShift(Time t, Int32 days) const + Time toDateAndShift(Time t, Int32 days) const { return lut[findIndex(t) + days].date; } - inline Time toTime(Time t) const + Time toTime(Time t) const { const LUTIndex index = findIndex(t); @@ -532,7 +532,7 @@ public: return res - offset_at_start_of_epoch; /// Starting at 1970-01-01 00:00:00 local time. } - inline unsigned toHour(Time t) const + unsigned toHour(Time t) const { const LUTIndex index = findIndex(t); @@ -552,7 +552,7 @@ public: * then subtract the former from the latter to get the offset result. * The boundaries when meets DST(daylight saving time) change should be handled very carefully. */ - inline Time timezoneOffset(Time t) const + Time timezoneOffset(Time t) const { const LUTIndex index = findIndex(t); @@ -574,7 +574,7 @@ public: } - inline unsigned toSecond(Time t) const + unsigned toSecond(Time t) const { if (likely(offset_is_whole_number_of_minutes_during_epoch)) { @@ -593,7 +593,7 @@ public: return time % 60; } - inline unsigned toMinute(Time t) const + unsigned toMinute(Time t) const { if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) return (t / 60) % 60; @@ -630,11 +630,11 @@ public: * because the same calendar day starts/ends at different timestamps in different time zones) */ - inline Time fromDayNum(DayNum d) const { return lut_saturated[toLUTIndex(d)].date; } - inline Time fromDayNum(ExtendedDayNum d) const { return lut[toLUTIndex(d)].date; } + Time fromDayNum(DayNum d) const { return lut_saturated[toLUTIndex(d)].date; } + Time fromDayNum(ExtendedDayNum d) const { return lut[toLUTIndex(d)].date; } template - inline Time toDate(DateOrTime v) const + Time toDate(DateOrTime v) const { if constexpr (std::is_unsigned_v || std::is_same_v) return lut_saturated[toLUTIndex(v)].date; @@ -643,20 +643,20 @@ public: } template - inline UInt8 toMonth(DateOrTime v) const { return lut[toLUTIndex(v)].month; } + UInt8 toMonth(DateOrTime v) const { return lut[toLUTIndex(v)].month; } template - inline UInt8 toQuarter(DateOrTime v) const { return (lut[toLUTIndex(v)].month - 1) / 3 + 1; } + UInt8 toQuarter(DateOrTime v) const { return (lut[toLUTIndex(v)].month - 1) / 3 + 1; } template - inline Int16 toYear(DateOrTime v) const { return lut[toLUTIndex(v)].year; } + Int16 toYear(DateOrTime v) const { return lut[toLUTIndex(v)].year; } /// 1-based, starts on Monday template - inline UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } + UInt8 toDayOfWeek(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_week; } template - inline UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const + UInt8 toDayOfWeek(DateOrTime v, UInt8 week_day_mode) const { WeekDayMode mode = check_week_day_mode(week_day_mode); @@ -674,10 +674,10 @@ public: } template - inline UInt8 toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; } + UInt8 toDayOfMonth(DateOrTime v) const { return lut[toLUTIndex(v)].day_of_month; } template - inline UInt16 toDayOfYear(DateOrTime v) const + UInt16 toDayOfYear(DateOrTime v) const { // TODO: different overload for ExtendedDayNum const LUTIndex i = toLUTIndex(v); @@ -688,7 +688,7 @@ public: /// (round down to monday and divide DayNum by 7; we made an assumption, /// that in domain of the function there was no weeks with any other number of days than 7) template - inline Int32 toRelativeWeekNum(DateOrTime v) const + Int32 toRelativeWeekNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); /// We add 8 to avoid underflow at beginning of unix epoch. @@ -697,7 +697,7 @@ public: /// Get year that contains most of the current week. Week begins at monday. template - inline Int16 toISOYear(DateOrTime v) const + Int16 toISOYear(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); /// That's effectively the year of thursday of current week. @@ -708,7 +708,7 @@ public: /// Example: ISO year 2019 begins at 2018-12-31. And ISO year 2017 begins at 2017-01-02. /// https://en.wikipedia.org/wiki/ISO_week_date template - inline LUTIndex toFirstDayNumOfISOYearIndex(DateOrTime v) const + LUTIndex toFirstDayNumOfISOYearIndex(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); auto iso_year = toISOYear(i); @@ -722,7 +722,7 @@ public: } template - inline auto toFirstDayNumOfISOYear(DateOrTime v) const + auto toFirstDayNumOfISOYear(DateOrTime v) const { if constexpr (std::is_unsigned_v || std::is_same_v) return toDayNum(LUTIndexWithSaturation(toFirstDayNumOfISOYearIndex(v))); @@ -730,7 +730,7 @@ public: return toDayNum(LUTIndex(toFirstDayNumOfISOYearIndex(v))); } - inline Time toFirstDayOfISOYear(Time t) const + Time toFirstDayOfISOYear(Time t) const { return lut[toFirstDayNumOfISOYearIndex(t)].date; } @@ -738,7 +738,7 @@ public: /// ISO 8601 week number. Week begins at monday. /// The week number 1 is the first week in year that contains 4 or more days (that's more than half). template - inline UInt8 toISOWeek(DateOrTime v) const + UInt8 toISOWeek(DateOrTime v) const { return 1 + (toFirstDayNumOfWeek(v) - toDayNum(toFirstDayNumOfISOYearIndex(v))) / 7; } @@ -777,7 +777,7 @@ public: next week is week 1. */ template - inline YearWeek toYearWeek(DateOrTime v, UInt8 week_mode) const + YearWeek toYearWeek(DateOrTime v, UInt8 week_mode) const { const bool newyear_day_mode = week_mode & static_cast(WeekModeFlag::NEWYEAR_DAY); week_mode = check_week_mode(week_mode); @@ -836,7 +836,7 @@ public: /// Calculate week number of WeekModeFlag::NEWYEAR_DAY mode /// The week number 1 is the first week in year that contains January 1, template - inline YearWeek toYearWeekOfNewyearMode(DateOrTime v, bool monday_first_mode) const + YearWeek toYearWeekOfNewyearMode(DateOrTime v, bool monday_first_mode) const { YearWeek yw(0, 0); UInt16 offset_day = monday_first_mode ? 0U : 1U; @@ -870,7 +870,7 @@ public: /// Get first day of week with week_mode, return Sunday or Monday template - inline auto toFirstDayNumOfWeek(DateOrTime v, UInt8 week_mode) const + auto toFirstDayNumOfWeek(DateOrTime v, UInt8 week_mode) const { bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); if (monday_first_mode) @@ -889,7 +889,7 @@ public: /// Get last day of week with week_mode, return Saturday or Sunday template - inline auto toLastDayNumOfWeek(DateOrTime v, UInt8 week_mode) const + auto toLastDayNumOfWeek(DateOrTime v, UInt8 week_mode) const { bool monday_first_mode = week_mode & static_cast(WeekModeFlag::MONDAY_FIRST); if (monday_first_mode) @@ -908,7 +908,7 @@ public: } /// Check and change mode to effective. - inline UInt8 check_week_mode(UInt8 mode) const /// NOLINT + UInt8 check_week_mode(UInt8 mode) const /// NOLINT { UInt8 week_format = (mode & 7); if (!(week_format & static_cast(WeekModeFlag::MONDAY_FIRST))) @@ -917,7 +917,7 @@ public: } /// Check and change mode to effective. - inline WeekDayMode check_week_day_mode(UInt8 mode) const /// NOLINT + WeekDayMode check_week_day_mode(UInt8 mode) const /// NOLINT { return static_cast(mode & 3); } @@ -926,7 +926,7 @@ public: * Returns 0 for monday, 1 for tuesday... */ template - inline UInt8 calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const /// NOLINT + UInt8 calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const /// NOLINT { const LUTIndex i = toLUTIndex(v); if (!sunday_first_day_of_week) @@ -936,28 +936,28 @@ public: } /// Calculate days in one year. - inline UInt16 calc_days_in_year(Int32 year) const /// NOLINT + UInt16 calc_days_in_year(Int32 year) const /// NOLINT { return ((year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)) ? 366 : 365); } /// Number of month from some fixed moment in the past (year * 12 + month) template - inline Int32 toRelativeMonthNum(DateOrTime v) const + Int32 toRelativeMonthNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); return lut[i].year * 12 + lut[i].month; } template - inline Int32 toRelativeQuarterNum(DateOrTime v) const + Int32 toRelativeQuarterNum(DateOrTime v) const { const LUTIndex i = toLUTIndex(v); return lut[i].year * 4 + (lut[i].month - 1) / 3; } /// We count all hour-length intervals, unrelated to offset changes. - inline Time toRelativeHourNum(Time t) const + Time toRelativeHourNum(Time t) const { if (t >= 0 && offset_is_whole_number_of_hours_during_epoch) return t / 3600; @@ -968,37 +968,37 @@ public: } template - inline Time toRelativeHourNum(DateOrTime v) const + Time toRelativeHourNum(DateOrTime v) const { return toRelativeHourNum(lut[toLUTIndex(v)].date); } /// The same formula is used for positive time (after Unix epoch) and negative time (before Unix epoch). /// It’s needed for correct work of dateDiff function. - inline Time toStableRelativeHourNum(Time t) const + Time toStableRelativeHourNum(Time t) const { return (t + DATE_LUT_ADD + 86400 - offset_at_start_of_epoch) / 3600 - (DATE_LUT_ADD / 3600); } template - inline Time toStableRelativeHourNum(DateOrTime v) const + Time toStableRelativeHourNum(DateOrTime v) const { return toStableRelativeHourNum(lut[toLUTIndex(v)].date); } - inline Time toRelativeMinuteNum(Time t) const /// NOLINT + Time toRelativeMinuteNum(Time t) const /// NOLINT { return (t + DATE_LUT_ADD) / 60 - (DATE_LUT_ADD / 60); } template - inline Time toRelativeMinuteNum(DateOrTime v) const + Time toRelativeMinuteNum(DateOrTime v) const { return toRelativeMinuteNum(lut[toLUTIndex(v)].date); } template - inline auto toStartOfYearInterval(DateOrTime v, UInt64 years) const + auto toStartOfYearInterval(DateOrTime v, UInt64 years) const { if (years == 1) return toFirstDayNumOfYear(v); @@ -1019,7 +1019,7 @@ public: template requires std::is_same_v || std::is_same_v - inline auto toStartOfQuarterInterval(Date d, UInt64 quarters) const + auto toStartOfQuarterInterval(Date d, UInt64 quarters) const { if (quarters == 1) return toFirstDayNumOfQuarter(d); @@ -1028,7 +1028,7 @@ public: template requires std::is_same_v || std::is_same_v - inline auto toStartOfMonthInterval(Date d, UInt64 months) const + auto toStartOfMonthInterval(Date d, UInt64 months) const { if (months == 1) return toFirstDayNumOfMonth(d); @@ -1042,7 +1042,7 @@ public: template requires std::is_same_v || std::is_same_v - inline auto toStartOfWeekInterval(Date d, UInt64 weeks) const + auto toStartOfWeekInterval(Date d, UInt64 weeks) const { if (weeks == 1) return toFirstDayNumOfWeek(d); @@ -1056,7 +1056,7 @@ public: template requires std::is_same_v || std::is_same_v - inline Time toStartOfDayInterval(Date d, UInt64 days) const + Time toStartOfDayInterval(Date d, UInt64 days) const { if (days == 1) return toDate(d); @@ -1152,7 +1152,7 @@ public: return static_cast(roundDown(t, seconds)); } - inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const + LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const { if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) return LUTIndex(0); @@ -1167,7 +1167,7 @@ public: } /// Create DayNum from year, month, day of month. - inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const + ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const { if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) return ExtendedDayNum(default_error_day_num); @@ -1175,14 +1175,14 @@ public: return toDayNum(makeLUTIndex(year, month, day_of_month)); } - inline Time makeDate(Int16 year, UInt8 month, UInt8 day_of_month) const + Time makeDate(Int16 year, UInt8 month, UInt8 day_of_month) const { return lut[makeLUTIndex(year, month, day_of_month)].date; } /** Does not accept daylight saving time as argument: in case of ambiguity, it choose greater timestamp. */ - inline Time makeDateTime(Int16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, UInt8 minute, UInt8 second) const + Time makeDateTime(Int16 year, UInt8 month, UInt8 day_of_month, UInt8 hour, UInt8 minute, UInt8 second) const { size_t index = makeLUTIndex(year, month, day_of_month); Time time_offset = hour * 3600 + minute * 60 + second; @@ -1194,28 +1194,28 @@ public: } template - inline const Values & getValues(DateOrTime v) const { return lut[toLUTIndex(v)]; } + const Values & getValues(DateOrTime v) const { return lut[toLUTIndex(v)]; } template - inline UInt32 toNumYYYYMM(DateOrTime v) const + UInt32 toNumYYYYMM(DateOrTime v) const { const Values & values = getValues(v); return values.year * 100 + values.month; } template - inline UInt32 toNumYYYYMMDD(DateOrTime v) const + UInt32 toNumYYYYMMDD(DateOrTime v) const { const Values & values = getValues(v); return values.year * 10000 + values.month * 100 + values.day_of_month; } - inline Time YYYYMMDDToDate(UInt32 num) const /// NOLINT + Time YYYYMMDDToDate(UInt32 num) const /// NOLINT { return makeDate(num / 10000, num / 100 % 100, num % 100); } - inline ExtendedDayNum YYYYMMDDToDayNum(UInt32 num) const /// NOLINT + ExtendedDayNum YYYYMMDDToDayNum(UInt32 num) const /// NOLINT { return makeDayNum(num / 10000, num / 100 % 100, num % 100); } @@ -1241,13 +1241,13 @@ public: TimeComponents time; }; - inline DateComponents toDateComponents(Time t) const + DateComponents toDateComponents(Time t) const { const Values & values = getValues(t); return { values.year, values.month, values.day_of_month }; } - inline DateTimeComponents toDateTimeComponents(Time t) const + DateTimeComponents toDateTimeComponents(Time t) const { const LUTIndex index = findIndex(t); const Values & values = lut[index]; @@ -1283,12 +1283,12 @@ public: } template - inline DateTimeComponents toDateTimeComponents(DateOrTime v) const + DateTimeComponents toDateTimeComponents(DateOrTime v) const { return toDateTimeComponents(lut[toLUTIndex(v)].date); } - inline UInt64 toNumYYYYMMDDhhmmss(Time t) const + UInt64 toNumYYYYMMDDhhmmss(Time t) const { DateTimeComponents components = toDateTimeComponents(t); @@ -1301,7 +1301,7 @@ public: + UInt64(components.date.year) * 10000000000; } - inline Time YYYYMMDDhhmmssToTime(UInt64 num) const /// NOLINT + Time YYYYMMDDhhmmssToTime(UInt64 num) const /// NOLINT { return makeDateTime( num / 10000000000, @@ -1315,7 +1315,7 @@ public: /// Adding calendar intervals. /// Implementation specific behaviour when delta is too big. - inline NO_SANITIZE_UNDEFINED Time addDays(Time t, Int64 delta) const + NO_SANITIZE_UNDEFINED Time addDays(Time t, Int64 delta) const { const LUTIndex index = findIndex(t); const Values & values = lut[index]; @@ -1332,12 +1332,12 @@ public: return lut[new_index].date + time; } - inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const + NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const { return addDays(t, delta * 7); } - inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const + UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const { if (likely(day_of_month <= 28)) return day_of_month; @@ -1351,7 +1351,7 @@ public: } template - inline LUTIndex NO_SANITIZE_UNDEFINED addMonthsIndex(DateOrTime v, Int64 delta) const + LUTIndex NO_SANITIZE_UNDEFINED addMonthsIndex(DateOrTime v, Int64 delta) const { const Values & values = lut[toLUTIndex(v)]; @@ -1375,11 +1375,11 @@ public: } } - /// If resulting month has less deys than source month, then saturation can happen. + /// If resulting month has less days than source month, then saturation can happen. /// Example: 31 Aug + 1 month = 30 Sep. template requires std::is_same_v || std::is_same_v || std::is_same_v - inline Time NO_SANITIZE_UNDEFINED addMonths(DateTime t, Int64 delta) const + Time NO_SANITIZE_UNDEFINED addMonths(DateTime t, Int64 delta) const { const auto result_day = addMonthsIndex(t, delta); @@ -1405,7 +1405,7 @@ public: template requires std::is_same_v || std::is_same_v - inline auto NO_SANITIZE_UNDEFINED addMonths(Date d, Int64 delta) const + auto NO_SANITIZE_UNDEFINED addMonths(Date d, Int64 delta) const { if constexpr (std::is_same_v) return toDayNum(LUTIndexWithSaturation(addMonthsIndex(d, delta))); @@ -1414,13 +1414,13 @@ public: } template - inline auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const + auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const { return addMonths(d, delta * 3); } template - inline LUTIndex NO_SANITIZE_UNDEFINED addYearsIndex(DateOrTime v, Int64 delta) const + LUTIndex NO_SANITIZE_UNDEFINED addYearsIndex(DateOrTime v, Int64 delta) const { const Values & values = lut[toLUTIndex(v)]; @@ -1438,7 +1438,7 @@ public: /// Saturation can occur if 29 Feb is mapped to non-leap year. template requires std::is_same_v || std::is_same_v || std::is_same_v - inline Time addYears(DateTime t, Int64 delta) const + Time addYears(DateTime t, Int64 delta) const { auto result_day = addYearsIndex(t, delta); @@ -1464,7 +1464,7 @@ public: template requires std::is_same_v || std::is_same_v - inline auto addYears(Date d, Int64 delta) const + auto addYears(Date d, Int64 delta) const { if constexpr (std::is_same_v) return toDayNum(LUTIndexWithSaturation(addYearsIndex(d, delta))); @@ -1473,7 +1473,7 @@ public: } - inline std::string timeToString(Time t) const + std::string timeToString(Time t) const { DateTimeComponents components = toDateTimeComponents(t); @@ -1498,7 +1498,7 @@ public: return s; } - inline std::string dateToString(Time t) const + std::string dateToString(Time t) const { const Values & values = getValues(t); @@ -1516,7 +1516,7 @@ public: return s; } - inline std::string dateToString(ExtendedDayNum d) const + std::string dateToString(ExtendedDayNum d) const { const Values & values = getValues(d); diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 1e4f3604c94..bb8c302d139 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -28,7 +28,9 @@ namespace ErrorCodes namespace { -/// Functions common to makeDate, makeDate32, makeDateTime, makeDateTime64 +/// Functionality common to +/// - makeDate, makeDate32, makeDateTime, makeDateTime64, +/// - YYYYMMDDToDate, YYYYMMDDToDate32, YYYYMMDDhhmmssToDateTime, YYYYMMDDhhmmssToDateTime64 class FunctionWithNumericParamsBase : public IFunction { public: @@ -48,11 +50,11 @@ public: size_t getNumberOfArguments() const override { return 0; } protected: - template + template Columns convertMandatoryArguments(const ColumnsWithTypeAndName & arguments, const ArgumentNames & argument_names) const { Columns converted_arguments; - const DataTypePtr converted_argument_type = std::make_shared(); + const DataTypePtr converted_argument_type = std::make_shared(); for (size_t i = 0; i < argument_names.size(); ++i) { ColumnPtr argument_column = castColumn(arguments[i], converted_argument_type); @@ -63,7 +65,7 @@ protected: } }; -/// Common implementation for makeDate, makeDate32 +/// Implementation of makeDate, makeDate32 template class FunctionMakeDate : public FunctionWithNumericParamsBase { @@ -72,7 +74,7 @@ private: static constexpr std::array mandatory_argument_names_year_dayofyear = {"year", "dayofyear"}; public: - static constexpr auto name = Traits::name; + static constexpr auto name = Traits::makeDateName; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -80,9 +82,9 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const bool isYearMonthDayVariant = (arguments.size() == 3); + const bool is_year_month_variant = (arguments.size() == 3); - if (isYearMonthDayVariant) + if (is_year_month_variant) { FunctionArgumentDescriptors args{ {mandatory_argument_names_year_month_day[0], &isNumber, nullptr, "Number"}, @@ -105,10 +107,10 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const bool isYearMonthDayVariant = (arguments.size() == 3); + const bool is_year_month_day_variant = (arguments.size() == 3); Columns converted_arguments; - if (isYearMonthDayVariant) + if (is_year_month_day_variant) converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names_year_month_day); else converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names_year_dayofyear); @@ -119,7 +121,7 @@ public: const auto & date_lut = DateLUT::instance(); const Int32 max_days_since_epoch = date_lut.makeDayNum(Traits::MAX_DATE[0], Traits::MAX_DATE[1], Traits::MAX_DATE[2]); - if (isYearMonthDayVariant) + if (is_year_month_day_variant) { const auto & year_data = typeid_cast(*converted_arguments[0]).getData(); const auto & month_data = typeid_cast(*converted_arguments[1]).getData(); @@ -133,8 +135,7 @@ public: Int32 day_num = 0; - if (year >= Traits::MIN_YEAR && - year <= Traits::MAX_YEAR && + if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR && month >= 1 && month <= 12 && day >= 1 && day <= 31) { @@ -158,8 +159,7 @@ public: Int32 day_num = 0; - if (year >= Traits::MIN_YEAR && - year <= Traits::MAX_YEAR && + if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR && dayofyear >= 1 && dayofyear <= 365) { Int32 days_since_epoch = date_lut.makeDayNum(static_cast(year), 1, 1) + static_cast(dayofyear) - 1; @@ -175,20 +175,87 @@ public: } }; -struct MakeDateTraits +/// Implementation of YYYYMMDDToDate, YYYYMMDDToDate32 +template +class FunctionYYYYYMMDDToDate : public FunctionWithNumericParamsBase { - static constexpr auto name = "makeDate"; +private: + static constexpr std::array mandatory_argument_names = { "YYYYMMDD" }; + +public: + static constexpr auto name = Traits::YYYYMMDDName; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return mandatory_argument_names.size(); } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + Columns converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names); + + auto res_column = Traits::ReturnDataType::ColumnType::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & yyyymmdd_data = typeid_cast(*converted_arguments[0]).getData(); + + const auto & date_lut = DateLUT::instance(); + const Int32 max_days_since_epoch = date_lut.makeDayNum(Traits::MAX_DATE[0], Traits::MAX_DATE[1], Traits::MAX_DATE[2]); + + for (size_t i = 0; i < input_rows_count; ++i) + { + const auto yyyymmdd = static_cast(yyyymmdd_data[i]); + + const auto year = yyyymmdd / 10'000; + const auto month = yyyymmdd / 100 % 100; + const auto day = yyyymmdd % 100; + + Int32 day_num = 0; + + if (year >= Traits::MIN_YEAR && year <= Traits::MAX_YEAR && + month >= 1 && month <= 12 && + day >= 1 && day <= 31) + { + Int32 days_since_epoch = date_lut.makeDayNum(static_cast(year), static_cast(month), static_cast(day)); + if (days_since_epoch <= max_days_since_epoch) + day_num = days_since_epoch; + } + + result_data[i] = day_num; + } + + return res_column; + } +}; + +struct DateTraits +{ + static constexpr auto makeDateName = "makeDate"; + static constexpr auto YYYYMMDDName = "YYYYMMDDToDate"; using ReturnDataType = DataTypeDate; static constexpr auto MIN_YEAR = 1970; static constexpr auto MAX_YEAR = 2149; - /// This date has the maximum day number that fits in 16-bit uint static constexpr std::array MAX_DATE = {MAX_YEAR, 6, 6}; }; -struct MakeDate32Traits +struct Date32Traits { - static constexpr auto name = "makeDate32"; + static constexpr auto makeDateName = "makeDate32"; + static constexpr auto YYYYMMDDName = "YYYYMMDDToDate32"; using ReturnDataType = DataTypeDate32; static constexpr auto MIN_YEAR = 1900; @@ -196,11 +263,11 @@ struct MakeDate32Traits static constexpr std::array MAX_DATE = {MAX_YEAR, 12, 31}; }; -/// Common implementation for makeDateTime, makeDateTime64 -class FunctionMakeDateTimeBase : public FunctionWithNumericParamsBase +/// Functionality common to makeDateTime, makeDateTime64, YYYYMMDDhhmmssToDateTime, YYYYMMDDhhmmssToDateTime64 +class FunctionDateTimeBase : public FunctionWithNumericParamsBase { protected: - static constexpr std::array mandatory_argument_names = {"year", "month", "day", "hour", "minute", "second"}; + static constexpr UInt32 DEFAULT_PRECISION = 3; template static Int64 dateTime(T year, T month, T day_of_month, T hour, T minute, T second, const DateLUTImpl & lut) @@ -233,14 +300,35 @@ protected: std::string extractTimezone(const ColumnWithTypeAndName & timezone_argument) const { - std::string timezone; if (!isStringOrFixedString(timezone_argument.type) || !timezone_argument.column || (timezone_argument.column->size() != 1 && !typeid_cast(timezone_argument.column.get()))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 'timezone' for function {} must be const string", getName()); - timezone = timezone_argument.column->getDataAt(0).toString(); + + String timezone = timezone_argument.column->getDataAt(0).toString(); return timezone; } + + UInt32 extractPrecision(const ColumnWithTypeAndName & precision_argument) const + { + if (!isNumber(precision_argument.type) || !precision_argument.column || (precision_argument.column->size() != 1 && !typeid_cast(precision_argument.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument 'precision' for function {} must be constant number", getName()); + + Int64 precision = precision_argument.column->getInt(0); + + if (precision < 0 || precision > 9) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Argument 'precision' for function {} must be in range [0, 9]", getName()); + + return static_cast(precision); + } +}; + +class FunctionMakeDateTimeBase : public FunctionDateTimeBase +{ +protected: + static constexpr std::array mandatory_argument_names = {"year", "month", "day", "hour", "minute", "second"}; }; /// makeDateTime(year, month, day, hour, minute, second, [timezone]) @@ -268,7 +356,7 @@ public: }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isString, nullptr, "String"} + {optional_argument_names[0], &isString, isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -329,7 +417,6 @@ class FunctionMakeDateTime64 : public FunctionMakeDateTimeBase { private: static constexpr std::array optional_argument_names = {"fraction", "precision", "timezone"}; - static constexpr UInt8 DEFAULT_PRECISION = 3; public: static constexpr auto name = "makeDateTime64"; @@ -350,9 +437,9 @@ public: }; FunctionArgumentDescriptors optional_args{ - {optional_argument_names[0], &isNumber, nullptr, "Number"}, - {optional_argument_names[1], &isNumber, nullptr, "Number"}, - {optional_argument_names[2], &isString, nullptr, "String"} + {optional_argument_names[0], &isNumber, isColumnConst, "const Number"}, + {optional_argument_names[1], &isNumber, isColumnConst, "const Number"}, + {optional_argument_names[2], &isString, isColumnConst, "const String"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -456,20 +543,179 @@ public: return res_column; } +}; +class FunctionYYYYMMDDhhmmssToDateTimeBase : public FunctionDateTimeBase +{ +protected: + static constexpr std::array mandatory_argument_names = { "YYYYMMDDhhmmss" }; +}; + +/// YYYYMMDDhhmmssToDateTime +class FunctionYYYYMMDDhhmmssToDateTime : public FunctionYYYYMMDDhhmmssToDateTimeBase +{ private: - UInt8 extractPrecision(const ColumnWithTypeAndName & precision_argument) const - { - Int64 precision = DEFAULT_PRECISION; - if (!isNumber(precision_argument.type) || !precision_argument.column || (precision_argument.column->size() != 1 && !typeid_cast(precision_argument.column.get()))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument 'precision' for function {} must be constant number", getName()); - precision = precision_argument.column->getInt(0); - if (precision < 0 || precision > 9) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, - "Argument 'precision' for function {} must be in range [0, 9]", getName()); + static constexpr std::array optional_argument_names = { "timezone" }; - return precision; +public: + static constexpr auto name = "YYYYMMDDhhmmssToDateTime"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors mandatory_args{ + {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + }; + + FunctionArgumentDescriptors optional_args{ + {optional_argument_names[0], &isString, isColumnConst, "const String"} + }; + + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == mandatory_argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + std::string timezone; + if (arguments.size() == mandatory_argument_names.size() + 1) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names); + + auto res_column = ColumnDateTime::create(input_rows_count); + auto & result_data = res_column->getData(); + + const auto & yyyymmddhhmmss_data = typeid_cast(*converted_arguments[0]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + for (size_t i = 0; i < input_rows_count; i++) + { + const auto yyyymmddhhmmss = static_cast(yyyymmddhhmmss_data[i]); + + const auto yyyymmdd = yyyymmddhhmmss / 1'000'000; + const auto hhmmss = yyyymmddhhmmss % 1'000'000; + + const auto year = yyyymmdd / 10'000; + const auto month = yyyymmdd / 100 % 100; + const auto day = yyyymmdd % 100; + const auto hour = hhmmss / 10'000; + const auto minute = hhmmss / 100 % 100; + const auto second = hhmmss % 100; + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + + if (date_time < 0) [[unlikely]] + date_time = 0; + else if (date_time > 0x0ffffffffll) [[unlikely]] + date_time = 0x0ffffffffll; + + result_data[i] = static_cast(date_time); + } + + return res_column; + } +}; + +/// YYYYMMDDhhmmssToDateTime64 +class FunctionYYYYMMDDhhmmssToDateTime64 : public FunctionYYYYMMDDhhmmssToDateTimeBase +{ +private: + static constexpr std::array optional_argument_names = { "precision", "timezone" }; + +public: + static constexpr auto name = "YYYYMMDDhhmmssToDateTime64"; + + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors mandatory_args{ + {mandatory_argument_names[0], &isNumber, nullptr, "Number"} + }; + + FunctionArgumentDescriptors optional_args{ + {optional_argument_names[0], &isNumber, isColumnConst, "const Number"}, + {optional_argument_names[0], &isString, isColumnConst, "const String"} + }; + + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); + + /// Optional precision argument + auto precision = DEFAULT_PRECISION; + if (arguments.size() >= mandatory_argument_names.size() + 1) + precision = extractPrecision(arguments[mandatory_argument_names.size()]); + + /// Optional timezone argument + std::string timezone; + if (arguments.size() == mandatory_argument_names.size() + 2) + timezone = extractTimezone(arguments.back()); + + return std::make_shared(precision, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + UInt32 precision = DEFAULT_PRECISION; + if (arguments.size() >= mandatory_argument_names.size() + 1) + precision = extractPrecision(arguments[mandatory_argument_names.size()]); + + std::string timezone; + if (arguments.size() == mandatory_argument_names.size() + 2) + timezone = extractTimezone(arguments.back()); + + Columns converted_arguments = convertMandatoryArguments(arguments, mandatory_argument_names); + + auto res_column = ColumnDateTime64::create(input_rows_count, static_cast(precision)); + auto & result_data = res_column->getData(); + + const auto & yyyymmddhhmmss_data = typeid_cast(*converted_arguments[0]).getData(); + + const auto & date_lut = DateLUT::instance(timezone); + + const auto fraction_pow = common::exp10_i32(precision); + + for (size_t i = 0; i < input_rows_count; i++) + { + const auto float_date = yyyymmddhhmmss_data[i]; + + const auto yyyymmddhhmmss = static_cast(float_date); + + const auto yyyymmdd = yyyymmddhhmmss / 1'000'000; + const auto hhmmss = yyyymmddhhmmss % 1'000'000; + + const auto decimal = float_date - yyyymmddhhmmss; + + const auto year = yyyymmdd / 10'000; + const auto month = yyyymmdd / 100 % 100; + const auto day = yyyymmdd % 100; + const auto hour = hhmmss / 10'000; + const auto minute = hhmmss / 100 % 100; + const auto second = hhmmss % 100; + + auto fraction = static_cast(decimal * fraction_pow); + + auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); + + result_data[i] = DecimalUtils::decimalFromComponents( + date_time, + static_cast(fraction), + static_cast(precision)); + } + + return res_column; } }; @@ -477,10 +723,15 @@ private: REGISTER_FUNCTION(MakeDate) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>(); + factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>(); factory.registerFunction(); factory.registerFunction(); + + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index a7b3a3d23c5..62784cb9b75 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -86,4 +86,4 @@ select makeDateTime64(year, 1, 1, 1, 0, 0, 0, precision, timezone) from ( select 1984 as year, 5 as precision, 'UTC' as timezone union all select 1985 as year, 5 as precision, 'UTC' as timezone -); -- { serverError 43 } +); -- { serverError 44 } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index dd843058281..7928ac23438 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -56,6 +56,10 @@ URLHierarchy URLPathHierarchy UUIDNumToString UUIDStringToNum +YYYYMMDDToDate +YYYYMMDDToDate32 +YYYYMMDDhhmmssToDateTime +YYYYMMDDhhmmssToDateTime64 _CAST __bitBoolMaskAnd __bitBoolMaskOr diff --git a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference new file mode 100644 index 00000000000..9d010535451 --- /dev/null +++ b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference @@ -0,0 +1,85 @@ +--- YYYYMMDDToDateTime +Invalid input types are rejected +Result type is DateTime +DateTime +Nullable(DateTime) +Check correctness, integer arguments +1970-01-02 11:59:59 +1970-01-01 00:00:00 +2020-02-29 11:11:11 +2023-09-11 15:05:05 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +Check correctness, float arguments +1970-01-02 11:59:59 +1970-01-01 00:00:00 +2020-02-29 11:11:11 +2023-09-11 15:05:05 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +Check correctness, decimal arguments +1970-01-02 11:59:59 +1970-01-01 00:00:00 +2020-02-29 11:11:11 +2023-09-11 15:05:05 +2106-02-07 06:28:15 +2106-02-07 06:28:15 +Special cases +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2106-02-07 06:28:15 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +2023-02-28 11:11:11 +1970-01-01 00:00:00 +1970-01-01 00:00:00 +\N +1970-01-01 00:00:00 +1970-01-01 00:00:00 +--- YYYYMMDDToDateTime64 +Invalid input types are rejected +Result type is DateTime +DateTime64(3) +DateTime64(5) +Nullable(DateTime64(3)) +Check correctness, integer arguments +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +2020-02-29 11:11:11.000 +2023-09-11 15:05:05.000 +2299-12-31 23:59:59.000 +2299-12-31 23:59:59.000 +Check correctness, float arguments +1900-01-01 00:00:00.900 +1900-01-01 00:00:00.899 +2020-02-29 11:11:11.101 +2023-09-11 15:05:05.101 +2299-12-31 23:59:59.101 +2299-12-31 23:59:59.101 +Check correctness, decimal arguments +1900-01-01 00:00:00.900 +1900-01-01 00:00:00.899 +2020-02-29 11:11:11.101 +2023-09-11 15:05:05.101 +2299-12-31 23:59:59.101 +2299-12-31 23:59:59.101 +Special cases +1900-01-01 00:00:00.648 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +2299-12-31 23:59:59.000 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +2023-02-28 11:11:11.000 +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 +\N +1900-01-01 00:00:00.000 +1900-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql new file mode 100644 index 00000000000..1b5f6220b50 --- /dev/null +++ b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql @@ -0,0 +1,119 @@ +SET session_timezone = 'UTC'; -- no time zone randomization, please + +----------------------------------------------------------- +SELECT '--- YYYYMMDDToDateTime'; + +SELECT 'Invalid input types are rejected'; +SELECT YYYYMMDDhhmmssToDateTime(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT YYYYMMDDhhmmssToDateTime(toDate('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime(toDate32('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime(toDateTime('2023-09-11 12:18:00')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime(toDateTime64('2023-09-11 12:18:00', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime('2023-09-11'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime(20230911134254, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime(20230911134254, 'invalid tz'); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime(20230911134254, 'Europe/Berlin', 'bad'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Result type is DateTime'; +SELECT toTypeName(YYYYMMDDhhmmssToDateTime(19910824)); +SELECT toTypeName(YYYYMMDDhhmmssToDateTime(cast(19910824 AS Nullable(UInt64)))); +-- +SELECT 'Check correctness, integer arguments'; +SELECT YYYYMMDDhhmmssToDateTime(19691231595959); +SELECT YYYYMMDDhhmmssToDateTime(19700101000000); +SELECT YYYYMMDDhhmmssToDateTime(20200229111111); -- leap day +SELECT YYYYMMDDhhmmssToDateTime(20230911150505); +SELECT YYYYMMDDhhmmssToDateTime(21060207062815); +SELECT YYYYMMDDhhmmssToDateTime(21060207062816); + +SELECT 'Check correctness, float arguments'; +SELECT YYYYMMDDhhmmssToDateTime(19691231595959.1); +SELECT YYYYMMDDhhmmssToDateTime(19700101000000.1); +SELECT YYYYMMDDhhmmssToDateTime(20200229111111.1); -- leap day +SELECT YYYYMMDDhhmmssToDateTime(20230911150505.1); +SELECT YYYYMMDDhhmmssToDateTime(21060207062815.1); +SELECT YYYYMMDDhhmmssToDateTime(21060207062816.1); + +SELECT 'Check correctness, decimal arguments'; +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(19691231595959.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(19700101000000.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(20200229111111.1, 5)); -- leap day +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(20230911150505.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(21060207062815.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(21060207062816.1, 5)); + +SELECT 'Special cases'; +SELECT YYYYMMDDhhmmssToDateTime(-20230911111111); -- negative +SELECT YYYYMMDDhhmmssToDateTime(110); -- invalid everything +SELECT YYYYMMDDhhmmssToDateTime(999999999999999999); -- huge value +SELECT YYYYMMDDhhmmssToDateTime(15001113111111); -- year out of range +SELECT YYYYMMDDhhmmssToDateTime(35001113111111); -- year out of range +SELECT YYYYMMDDhhmmssToDateTime(20231620111111); -- invalid month +SELECT YYYYMMDDhhmmssToDateTime(20230020111111); -- invalid month +SELECT YYYYMMDDhhmmssToDateTime(20230940111111); -- invalid day +SELECT YYYYMMDDhhmmssToDateTime(20230900111111); -- invalid day +SELECT YYYYMMDDhhmmssToDateTime(20230228111111); -- leap day when there is none +SELECT YYYYMMDDhhmmssToDateTime(True); +SELECT YYYYMMDDhhmmssToDateTime(False); +SELECT YYYYMMDDhhmmssToDateTime(NULL); +SELECT YYYYMMDDhhmmssToDateTime(yyyymmdd) FROM (SELECT 19840121 AS yyyymmdd UNION ALL SELECT 20230911 AS yyyymmdd) ORDER BY yyyymmdd; -- non-const + +----------------------------------------------------------- +SELECT '--- YYYYMMDDToDateTime64'; + +SELECT 'Invalid input types are rejected'; +SELECT YYYYMMDDhhmmssToDateTime64(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT YYYYMMDDhhmmssToDateTime64(toDate('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64(toDate32('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64(toDateTime('2023-09-11 12:18:00')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64(toDateTime64('2023-09-11 12:18:00', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64('2023-09-11'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64('2023-09-11', 'invalid precision'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64(20230911134254, 3, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDhhmmssToDateTime64(20230911134254, 3, 'invalid tz'); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime64(20230911134254, 3, 'Europe/Berlin', 'no more args'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Result type is DateTime'; +SELECT toTypeName(YYYYMMDDhhmmssToDateTime64(19910824)); +SELECT toTypeName(YYYYMMDDhhmmssToDateTime64(19910824, 5)); +SELECT toTypeName(YYYYMMDDhhmmssToDateTime64(cast(19910824 AS Nullable(UInt64)))); + +SELECT 'Check correctness, integer arguments'; +SELECT YYYYMMDDhhmmssToDateTime64(189912315959); +SELECT YYYYMMDDhhmmssToDateTime64(19000101000000); +SELECT YYYYMMDDhhmmssToDateTime64(20200229111111); -- leap day +SELECT YYYYMMDDhhmmssToDateTime64(20230911150505); +SELECT YYYYMMDDhhmmssToDateTime64(22991231235959); +SELECT YYYYMMDDhhmmssToDateTime64(23000101000000); + +SELECT 'Check correctness, float arguments'; +SELECT YYYYMMDDhhmmssToDateTime64(189912315959.1); +SELECT YYYYMMDDhhmmssToDateTime64(19000101000000.1); +SELECT YYYYMMDDhhmmssToDateTime64(20200229111111.1); -- leap day +SELECT YYYYMMDDhhmmssToDateTime64(20230911150505.1); +SELECT YYYYMMDDhhmmssToDateTime64(22991231235959.1); +SELECT YYYYMMDDhhmmssToDateTime64(23000101000000.1); + +SELECT 'Check correctness, decimal arguments'; +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(189912315959.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(19000101000000.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(20200229111111.1, 5)); -- leap day +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(20230911150505.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(22991231235959.1, 5)); +SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(23000101000000.1, 5)); + +SELECT 'Special cases'; +SELECT YYYYMMDDhhmmssToDateTime64(-20230911111111); -- negative +SELECT YYYYMMDDhhmmssToDateTime64(110); -- invalid everything +SELECT YYYYMMDDhhmmssToDateTime64(999999999999999999); -- huge value +SELECT YYYYMMDDhhmmssToDateTime64(15001113111111); -- year out of range +SELECT YYYYMMDDhhmmssToDateTime64(35001113111111); -- year out of range +SELECT YYYYMMDDhhmmssToDateTime64(20231620111111); -- invalid month +SELECT YYYYMMDDhhmmssToDateTime64(20230020111111); -- invalid month +SELECT YYYYMMDDhhmmssToDateTime64(20230940111111); -- invalid day +SELECT YYYYMMDDhhmmssToDateTime64(20230900111111); -- invalid day +SELECT YYYYMMDDhhmmssToDateTime64(20230228111111); -- leap day when there is none +SELECT YYYYMMDDhhmmssToDateTime64(True); +SELECT YYYYMMDDhhmmssToDateTime64(False); +SELECT YYYYMMDDhhmmssToDateTime64(NULL); +SELECT YYYYMMDDhhmmssToDateTime64(yyyymmdd) FROM (SELECT 19840121 AS yyyymmdd UNION ALL SELECT 20230911 AS yyyymmdd) ORDER BY yyyymmdd; -- non-const diff --git a/tests/queries/0_stateless/02876_yyyymmddtodate.reference b/tests/queries/0_stateless/02876_yyyymmddtodate.reference new file mode 100644 index 00000000000..e3c6a9e2d7c --- /dev/null +++ b/tests/queries/0_stateless/02876_yyyymmddtodate.reference @@ -0,0 +1,84 @@ +--- YYYYMMDDToDate +Invalid input types are rejected +Result type is Date +Date +Nullable(Date) +Check correctness, integer arguments +1970-01-01 +1970-01-01 +2020-02-29 +2023-09-11 +2149-06-06 +1970-01-01 +Check correctness, float arguments +1970-01-01 +1970-01-01 +2020-02-29 +2023-09-11 +2149-06-06 +1970-01-01 +Check correctness, decimal arguments +1970-01-01 +1970-01-01 +2020-02-29 +2023-09-11 +2149-06-06 +1970-01-01 +Special cases +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +2023-02-28 +1970-01-01 +1970-01-01 +\N +1984-01-21 +2023-09-11 +--- YYYYMMDDToDate32 +Invalid input types are rejected +Result type is Date32 +Date32 +Nullable(Date32) +Check correctness, integer arguments +1970-01-01 +1900-01-01 +2020-02-29 +2023-09-11 +2299-12-31 +1970-01-01 +Check correctness, float arguments +1970-01-01 +1900-01-01 +2020-02-29 +2023-09-11 +2299-12-31 +1970-01-01 +Check correctness, decimal arguments +1970-01-01 +1900-01-01 +2020-02-29 +2023-09-11 +2299-12-31 +1970-01-01 +Special cases +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +1970-01-01 +2023-02-28 +1970-01-01 +1970-01-01 +\N +1984-01-21 +2023-09-11 diff --git a/tests/queries/0_stateless/02876_yyyymmddtodate.sql b/tests/queries/0_stateless/02876_yyyymmddtodate.sql new file mode 100644 index 00000000000..fbacca1e2a2 --- /dev/null +++ b/tests/queries/0_stateless/02876_yyyymmddtodate.sql @@ -0,0 +1,112 @@ +----------------------------------------------------------- +SELECT '--- YYYYMMDDToDate'; + +SELECT 'Invalid input types are rejected'; +SELECT YYYYMMDDToDate(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT YYYYMMDDToDate(toDate('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate(toDate32('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate(toDateTime('2023-09-11 12:18:00')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate(toDateTime64('2023-09-11 12:18:00', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate('2023-09-11'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate(2023, 09, 11); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT YYYYMMDDToDate(2023, 110); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Result type is Date'; +SELECT toTypeName(YYYYMMDDToDate(19910824)); +SELECT toTypeName(YYYYMMDDToDate(cast(19910824 AS Nullable(UInt64)))); + +SELECT 'Check correctness, integer arguments'; +SELECT YYYYMMDDToDate(19691231); +SELECT YYYYMMDDToDate(19700101); +SELECT YYYYMMDDToDate(20200229); -- leap day +SELECT YYYYMMDDToDate(20230911); +SELECT YYYYMMDDToDate(21490606); +SELECT YYYYMMDDToDate(21490607); + +SELECT 'Check correctness, float arguments'; +SELECT YYYYMMDDToDate(19691231.1); +SELECT YYYYMMDDToDate(19700101.1); +SELECT YYYYMMDDToDate(20200229.1); -- leap day +SELECT YYYYMMDDToDate(20230911.1); +SELECT YYYYMMDDToDate(21490606.1); +SELECT YYYYMMDDToDate(21490607.1); + +SELECT 'Check correctness, decimal arguments'; +SELECT YYYYMMDDToDate(toDecimal64(19691231.1, 5)); +SELECT YYYYMMDDToDate(toDecimal64(19700101.1, 5)); +SELECT YYYYMMDDToDate(toDecimal64(20200229.1, 5)); -- leap day +SELECT YYYYMMDDToDate(toDecimal64(20230911.1, 5)); +SELECT YYYYMMDDToDate(toDecimal64(21490606.1, 5)); +SELECT YYYYMMDDToDate(toDecimal64(21490607.1, 5)); + +SELECT 'Special cases'; +SELECT YYYYMMDDToDate(-20230911); -- negative +SELECT YYYYMMDDToDate(110); -- invalid everything +SELECT YYYYMMDDToDate(9999999999999); -- huge value +SELECT YYYYMMDDToDate(15001113); -- year out of range +SELECT YYYYMMDDToDate(35001113); -- year out of range +SELECT YYYYMMDDToDate(20231620); -- invalid month +SELECT YYYYMMDDToDate(20230020); -- invalid month +SELECT YYYYMMDDToDate(20230940); -- invalid day +SELECT YYYYMMDDToDate(20230900); -- invalid day +SELECT YYYYMMDDToDate(20230228); -- leap day when there is none +SELECT YYYYMMDDToDate(True); +SELECT YYYYMMDDToDate(False); +SELECT YYYYMMDDToDate(NULL); +SELECT YYYYMMDDToDate(yyyymmdd) FROM (SELECT 19840121 AS yyyymmdd UNION ALL SELECT 20230911 AS yyyymmdd) ORDER BY yyyymmdd; -- non-const + +----------------------------------------------------------- +SELECT '--- YYYYMMDDToDate32'; + +SELECT 'Invalid input types are rejected'; +SELECT YYYYMMDDToDate32(toDate('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate32(toDate32('2023-09-11')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate32(toDateTime('2023-09-11 12:18:00')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate32(toDateTime64('2023-09-11 12:18:00', 3)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate32('2023-09-11'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT YYYYMMDDToDate32(2023, 09, 11); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT YYYYMMDDToDate32(2023, 110); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT 'Result type is Date32'; +SELECT toTypeName(YYYYMMDDToDate32(19910824)); +SELECT toTypeName(YYYYMMDDToDate32(cast(19910824 AS Nullable(UInt64)))); + +SELECT 'Check correctness, integer arguments'; +SELECT YYYYMMDDToDate32(18991231); +SELECT YYYYMMDDToDate32(19000101); +SELECT YYYYMMDDToDate32(20200229); -- leap day +SELECT YYYYMMDDToDate32(20230911); +SELECT YYYYMMDDToDate32(22991231); +SELECT YYYYMMDDToDate32(23000101); + +SELECT 'Check correctness, float arguments'; +SELECT YYYYMMDDToDate32(18991231.1); +SELECT YYYYMMDDToDate32(19000101.1); +SELECT YYYYMMDDToDate32(20200229.1); -- leap day +SELECT YYYYMMDDToDate32(20230911.1); +SELECT YYYYMMDDToDate32(22991231.1); +SELECT YYYYMMDDToDate32(23000101.1); + +SELECT 'Check correctness, decimal arguments'; +SELECT YYYYMMDDToDate32(toDecimal64(18991231.1, 5)); +SELECT YYYYMMDDToDate32(toDecimal64(19000101.1, 5)); +SELECT YYYYMMDDToDate32(toDecimal64(20200229.1, 5)); -- leap day +SELECT YYYYMMDDToDate32(toDecimal64(20230911.1, 5)); +SELECT YYYYMMDDToDate32(toDecimal64(22991231.1, 5)); +SELECT YYYYMMDDToDate32(toDecimal64(23000101.1, 5)); + +SELECT 'Special cases'; +SELECT YYYYMMDDToDate32(-20230911); -- negative +SELECT YYYYMMDDToDate32(110); -- invalid everything +SELECT YYYYMMDDToDate32(9999999999999); -- huge value +SELECT YYYYMMDDToDate32(15001113); -- year out of range +SELECT YYYYMMDDToDate32(35001113); -- year out of range +SELECT YYYYMMDDToDate32(20231620); -- invalid month +SELECT YYYYMMDDToDate32(20230020); -- invalid month +SELECT YYYYMMDDToDate32(20230940); -- invalid day +SELECT YYYYMMDDToDate32(20230900); -- invalid day +SELECT YYYYMMDDToDate32(20230228); -- leap day when there is none +SELECT YYYYMMDDToDate32(True); +SELECT YYYYMMDDToDate32(False); +SELECT YYYYMMDDToDate32(NULL); +SELECT YYYYMMDDToDate32(yyyymmdd) FROM (SELECT 19840121 AS yyyymmdd UNION ALL SELECT 20230911 AS yyyymmdd) ORDER BY yyyymmdd; -- non-const From 61fdb63aad07084bf918a676f96bf9a17d31acf9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 13:53:20 +0000 Subject: [PATCH 162/327] fix --- tests/ci/commit_status_helper.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index ed686dae11c..c3b1a4d0f95 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -148,9 +148,9 @@ def update_pr_title_icon(pr: PullRequest, status: str) -> None: new_title = pr.title if new_title and new_title[0] != new_status_icon: if new_title[0] in set(STATUS_ICON_MAP.values()): - new_title = new_title[1:] + new_status_icon + new_title = new_status_icon + new_title[1:] else: - new_title = new_title + " " + new_status_icon + new_title = new_status_icon + " " + new_title pr.edit(title=new_title) From ec37e934611f7ad91f28baf2f382f2c30a5529be Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 11 Sep 2023 14:52:59 +0000 Subject: [PATCH 163/327] fix use-after-free in MergeTreePrefetchedReadPool --- src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 465dcd6a7ab..75f2fd26600 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -61,6 +61,12 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetechedReaders::get( { SCOPE_EXIT({ is_valid = false; }); ProfileEventTimeIncrement watch(ProfileEvents::WaitPrefetchTaskMicroseconds); + + /// First wait for completion of all futures. + for (auto & prefetch_future : prefetch_futures) + prefetch_future.wait(); + + /// Then rethrow first exception if any. for (auto & prefetch_future : prefetch_futures) prefetch_future.get(); From 2d8f33bfa2663a174c3d2f932525f1c4ea9cfed9 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Sep 2023 14:55:37 +0000 Subject: [PATCH 164/327] Fix parsing error in WithNames formats while reading subset of columns with disabled input_format_with_names_use_header --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 ++-- src/Formats/FormatFactory.cpp | 17 +++++++++++++---- src/Formats/FormatFactory.h | 11 ++++++++--- src/Formats/registerWithNamesAndTypes.cpp | 5 +++-- src/Interpreters/Context.cpp | 4 ++-- src/Storages/HDFS/StorageHDFS.cpp | 6 +++--- src/Storages/HDFS/StorageHDFS.h | 2 +- src/Storages/Hive/StorageHive.h | 2 +- src/Storages/IStorage.h | 2 -- src/Storages/S3Queue/StorageS3Queue.cpp | 8 ++++---- src/Storages/S3Queue/StorageS3Queue.h | 2 +- src/Storages/StorageAzureBlob.cpp | 6 +++--- src/Storages/StorageAzureBlob.h | 2 +- src/Storages/StorageFile.cpp | 6 +++--- src/Storages/StorageFile.h | 2 +- src/Storages/StorageS3.cpp | 6 +++--- src/Storages/StorageS3.h | 2 +- src/Storages/StorageURL.cpp | 8 ++++---- src/Storages/StorageURL.h | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/StorageXDBC.h | 2 +- src/TableFunctions/ITableFunction.h | 2 +- src/TableFunctions/ITableFunctionFileLike.cpp | 4 ++-- src/TableFunctions/ITableFunctionFileLike.h | 2 +- .../TableFunctionAzureBlobStorage.cpp | 4 ++-- .../TableFunctionAzureBlobStorage.h | 2 +- src/TableFunctions/TableFunctionS3.cpp | 4 ++-- src/TableFunctions/TableFunctionS3.h | 2 +- ...formats_with_names_dont_use_header.reference | 1 + .../02876_formats_with_names_dont_use_header.sh | 10 ++++++++++ ...with_names_dont_use_header_test.csvwithnames | 2 ++ 31 files changed, 80 insertions(+), 54 deletions(-) create mode 100644 tests/queries/0_stateless/02876_formats_with_names_dont_use_header.reference create mode 100755 tests/queries/0_stateless/02876_formats_with_names_dont_use_header.sh create mode 100644 tests/queries/0_stateless/02876_formats_with_names_dont_use_header_test.csvwithnames diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 348189854e8..9bb9308a609 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6341,9 +6341,9 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") - use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(scope.context->getInsertFormat()); + use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(scope.context->getInsertFormat(), scope.context); else - use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(); + use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(scope.context); } if (use_columns_from_insert_query) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index c5a4cf7a5c6..6481cc2d125 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -684,10 +684,18 @@ void FormatFactory::markOutputFormatSupportsParallelFormatting(const String & na void FormatFactory::markFormatSupportsSubsetOfColumns(const String & name) { - auto & target = dict[name].supports_subset_of_columns; + auto & target = dict[name].subset_of_columns_support_checker; if (target) throw Exception(ErrorCodes::LOGICAL_ERROR, "FormatFactory: Format {} is already marked as supporting subset of columns", name); - target = true; + target = [](const FormatSettings &){ return true; }; +} + +void FormatFactory::registerSubsetOfColumnsSupportChecker(const String & name, SubsetOfColumnsSupportChecker subset_of_columns_support_checker) +{ + auto & target = dict[name].subset_of_columns_support_checker; + if (target) + throw Exception(ErrorCodes::LOGICAL_ERROR, "FormatFactory: Format {} is already marked as supporting subset of columns", name); + target = std::move(subset_of_columns_support_checker); } void FormatFactory::markOutputFormatPrefersLargeBlocks(const String & name) @@ -698,10 +706,11 @@ void FormatFactory::markOutputFormatPrefersLargeBlocks(const String & name) target = true; } -bool FormatFactory::checkIfFormatSupportsSubsetOfColumns(const String & name) const +bool FormatFactory::checkIfFormatSupportsSubsetOfColumns(const DB::String & name, const ContextPtr & context, const std::optional & format_settings_) const { const auto & target = getCreators(name); - return target.supports_subset_of_columns; + auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); + return target.subset_of_columns_support_checker && target.subset_of_columns_support_checker(format_settings); } void FormatFactory::registerAdditionalInfoForSchemaCacheGetter( diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 2034e6446fb..9b6eb97456a 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -123,6 +123,10 @@ private: /// and the name of the message. using AdditionalInfoForSchemaCacheGetter = std::function; + /// Some formats can support reading subset of columns depending on settings. + /// The checker should return true if format support append. + using SubsetOfColumnsSupportChecker = std::function; + struct Creators { InputCreator input_creator; @@ -133,11 +137,11 @@ private: ExternalSchemaReaderCreator external_schema_reader_creator; bool supports_parallel_formatting{false}; bool supports_subcolumns{false}; - bool supports_subset_of_columns{false}; bool prefers_large_blocks{false}; NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker; AppendSupportChecker append_support_checker; AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter; + SubsetOfColumnsSupportChecker subset_of_columns_support_checker; }; using FormatsDictionary = std::unordered_map; @@ -225,9 +229,10 @@ public: void markOutputFormatSupportsParallelFormatting(const String & name); void markOutputFormatPrefersLargeBlocks(const String & name); - void markFormatSupportsSubsetOfColumns(const String & name); - bool checkIfFormatSupportsSubsetOfColumns(const String & name) const; + void markFormatSupportsSubsetOfColumns(const String & name); + void registerSubsetOfColumnsSupportChecker(const String & name, SubsetOfColumnsSupportChecker subset_of_columns_support_checker); + bool checkIfFormatSupportsSubsetOfColumns(const String & name, const ContextPtr & context, const std::optional & format_settings_ = std::nullopt) const; bool checkIfFormatHasSchemaReader(const String & name) const; bool checkIfFormatHasExternalSchemaReader(const String & name) const; diff --git a/src/Formats/registerWithNamesAndTypes.cpp b/src/Formats/registerWithNamesAndTypes.cpp index 2dee107844d..674865a3bed 100644 --- a/src/Formats/registerWithNamesAndTypes.cpp +++ b/src/Formats/registerWithNamesAndTypes.cpp @@ -12,8 +12,9 @@ void registerWithNamesAndTypes(const std::string & base_format_name, RegisterWit void markFormatWithNamesAndTypesSupportsSamplingColumns(const std::string & base_format_name, FormatFactory & factory) { - factory.markFormatSupportsSubsetOfColumns(base_format_name + "WithNames"); - factory.markFormatSupportsSubsetOfColumns(base_format_name + "WithNamesAndTypes"); + auto setting_checker = [](const FormatSettings & settings){ return settings.with_names_use_header; }; + factory.registerSubsetOfColumnsSupportChecker(base_format_name + "WithNames", setting_checker); + factory.registerSubsetOfColumnsSupportChecker(base_format_name + "WithNamesAndTypes", setting_checker); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ee7efdfeb1b..8695669a7de 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1684,9 +1684,9 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") - use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat()); + use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat(), shared_from_this()); else - use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(); + use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(shared_from_this()); } if (use_columns_from_insert_query) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index d2830d00748..7d144814803 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -838,9 +838,9 @@ private: }; -bool StorageHDFS::supportsSubsetOfColumns() const +bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_); } Pipe StorageHDFS::read( @@ -878,7 +878,7 @@ Pipe StorageHDFS::read( }); } - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && context_->getSettingsRef().optimize_count_from_files; diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 8d9976a6a35..ffbf4e93ff9 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -76,7 +76,7 @@ public: /// Is is useful because column oriented formats could effectively skip unknown columns /// So we can create a header of only required columns in read method and ask /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr & context_) const; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 604df70f4d0..a3c47d400e2 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -65,7 +65,7 @@ public: NamesAndTypesList getVirtuals() const override; - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns() const; std::optional totalRows(const Settings & settings) const override; std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a6d990a3eff..fcf7675d15d 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -620,8 +620,6 @@ public: /// NOTE: write-once also does not support INSERTs/merges/... for MergeTree virtual bool isStaticStorage() const; - virtual bool supportsSubsetOfColumns() const { return false; } - /// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. /// Used for: /// - Simple count() optimization diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 08cbff96cd0..ee840ca2ba1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -162,9 +162,9 @@ bool StorageS3Queue::supportsSubcolumns() const return true; } -bool StorageS3Queue::supportsSubsetOfColumns() const +bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); } Pipe StorageS3Queue::read( @@ -187,7 +187,7 @@ Pipe StorageS3Queue::read( std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; @@ -363,7 +363,7 @@ void StorageS3Queue::streamToViews() // Create a stream for each consumer and join them in a union stream std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(getContext()), getVirtuals()); const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; auto pipe = Pipe(std::make_shared( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 712fe9e530b..98d46e80ad2 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -125,7 +125,7 @@ private: }; std::shared_ptr task; - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr & context_) const; const UInt32 zk_create_table_retries = 1000; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index fb790c50e52..bdefe781ee6 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -687,7 +687,7 @@ Pipe StorageAzureBlob::read( query_info.query, virtual_columns, local_context, nullptr, local_context->getFileProgressCallback()); } - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; @@ -792,9 +792,9 @@ bool StorageAzureBlob::supportsPartitionBy() const return true; } -bool StorageAzureBlob::supportsSubsetOfColumns() const +bool StorageAzureBlob::supportsSubsetOfColumns(const ContextPtr & context) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context, format_settings); } bool StorageAzureBlob::prefersLargeBlocks() const diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 706817670d2..448d8e20e05 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -99,7 +99,7 @@ public: bool supportsSubcolumns() const override { return true; } - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr & context) const; bool supportsTrivialCountOptimization() const override { return true; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8fd888f571d..0e154803602 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -803,9 +803,9 @@ ColumnsDescription StorageFile::getTableStructureFromFile( return columns; } -bool StorageFile::supportsSubsetOfColumns() const +bool StorageFile::supportsSubsetOfColumns(const ContextPtr & context) const { - return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context, format_settings); } bool StorageFile::prefersLargeBlocks() const @@ -1433,7 +1433,7 @@ Pipe StorageFile::read( if (progress_callback && !archive_info) progress_callback(FileProgress(0, total_bytes_to_read)); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && context->getSettingsRef().optimize_count_from_files; diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 97c3cebabbb..f1464b90ab4 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -74,7 +74,7 @@ public: /// Is is useful because such formats could effectively skip unknown columns /// So we can create a header of only required columns in read method and ask /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr & context) const; bool supportsSubcolumns() const override { return true; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 24b2279bfdc..341d8b3f768 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -983,9 +983,9 @@ std::shared_ptr StorageS3::createFileIterator( } } -bool StorageS3::supportsSubsetOfColumns() const +bool StorageS3::supportsSubsetOfColumns(const ContextPtr & context) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context, format_settings); } bool StorageS3::prefersLargeBlocks() const @@ -1017,7 +1017,7 @@ Pipe StorageS3::read( std::shared_ptr iterator_wrapper = createFileIterator( query_configuration, distributed_processing, local_context, query_info.query, virtual_columns, nullptr, local_context->getFileProgressCallback()); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index a7dba400f37..ee03b9f18c2 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -388,7 +388,7 @@ private: bool supportsSubcolumns() const override { return true; } - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr & context) const; bool prefersLargeBlocks() const override; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fcc9a3bbeb2..a0f5379a1fd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -817,9 +817,9 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( return columns; } -bool IStorageURLBase::supportsSubsetOfColumns() const +bool IStorageURLBase::supportsSubsetOfColumns(const ContextPtr & context) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context, format_settings); } bool IStorageURLBase::prefersLargeBlocks() const @@ -846,7 +846,7 @@ Pipe IStorageURLBase::read( std::shared_ptr iterator_wrapper{nullptr}; bool is_url_with_globs = urlWithGlobs(uri); size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); if (distributed_processing) { @@ -951,7 +951,7 @@ Pipe StorageURLWithFailover::read( return uri_options; }); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); const size_t max_threads = local_context->getSettingsRef().max_threads; const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / num_streams); diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 504b0d5de40..6c5c50af326 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -114,7 +114,7 @@ protected: QueryProcessingStage::Enum & processed_stage, size_t max_block_size) const; - bool supportsSubsetOfColumns() const override; + virtual bool supportsSubsetOfColumns(const ContextPtr & context) const; bool prefersLargeBlocks() const override; diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 1715cde9d1e..0ba8838d4c3 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -146,7 +146,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet compression_method); } -bool StorageXDBC::supportsSubsetOfColumns() const +bool StorageXDBC::supportsSubsetOfColumns(const ContextPtr &) const { return true; } diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index d7a1138c710..1c1651cb333 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -68,7 +68,7 @@ private: Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const override; - bool supportsSubsetOfColumns() const override; + bool supportsSubsetOfColumns(const ContextPtr &) const override; }; } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 028bbb99c15..1946d8e8905 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -76,7 +76,7 @@ public: /// because we cannot determine which column from table correspond to this virtual column. virtual std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const { return {}; } - virtual bool supportsReadingSubsetOfColumns() { return true; } + virtual bool supportsReadingSubsetOfColumns(const ContextPtr &) { return true; } /// Create storage according to the query. StoragePtr diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 487826dc363..b88af855309 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -32,9 +32,9 @@ String ITableFunctionFileLike::getFormatFromFirstArgument() return FormatFactory::instance().getFormatFromFileName(filename, true); } -bool ITableFunctionFileLike::supportsReadingSubsetOfColumns() +bool ITableFunctionFileLike::supportsReadingSubsetOfColumns(const ContextPtr & context) { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format, context); } void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context) diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 902c7b01c4d..5fe86587797 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -27,7 +27,7 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - bool supportsReadingSubsetOfColumns() override; + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; static size_t getMaxNumberOfArguments() { return 4; } diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index a4ece2f6a07..c29bea2c5c7 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -269,9 +269,9 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex return parseColumnsListFromString(configuration.structure, context); } -bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns() +bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns(const ContextPtr & context) { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); } StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index 83c4449fd17..e1759740f7f 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -49,7 +49,7 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - bool supportsReadingSubsetOfColumns() override; + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override { diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 0ae3a19dc7f..f26550d0f70 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -325,9 +325,9 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context, return parseColumnsListFromString(configuration.structure, context); } -bool TableFunctionS3::supportsReadingSubsetOfColumns() +bool TableFunctionS3::supportsReadingSubsetOfColumns(const ContextPtr & context) { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); } StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index ac932260792..a38ea5ba56b 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -47,7 +47,7 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - bool supportsReadingSubsetOfColumns() override; + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override { diff --git a/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.reference b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.sh b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.sh new file mode 100755 index 00000000000..ce06ff530b9 --- /dev/null +++ b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo -e "a,b,c\n1,2,3" > $CLICKHOUSE_TEST_UNIQUE_NAME.csvwithnames + +$CLICKHOUSE_LOCAL -q "select b from file('$CLICKHOUSE_TEST_UNIQUE_NAME.csvwithnames') settings input_format_with_names_use_header=0" + diff --git a/tests/queries/0_stateless/02876_formats_with_names_dont_use_header_test.csvwithnames b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header_test.csvwithnames new file mode 100644 index 00000000000..bfde6bfa0b8 --- /dev/null +++ b/tests/queries/0_stateless/02876_formats_with_names_dont_use_header_test.csvwithnames @@ -0,0 +1,2 @@ +a,b,c +1,2,3 From c270ebfe5e65bf5cc526d813a1a36a99212a2fa4 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Sep 2023 14:56:29 +0000 Subject: [PATCH 165/327] Update reference file --- .../0_stateless/01165_lost_part_empty_partition.reference | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference index 6ed281c757a..1af4bf8965c 100644 --- a/tests/queries/0_stateless/01165_lost_part_empty_partition.reference +++ b/tests/queries/0_stateless/01165_lost_part_empty_partition.reference @@ -1,2 +1,4 @@ 1 1 +1 +2 From b69a90d61f1cefc97a0c45a703b1e837119765b3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Sep 2023 14:57:25 +0000 Subject: [PATCH 166/327] Change log message level --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index aac85b4afc0..0de89dff8ed 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -989,7 +989,7 @@ void StorageReplicatedMergeTree::drop() { UInt64 lost_part_count = lost_part_count_str.empty() ? 0 : parse(lost_part_count_str); if (lost_part_count > 0) - LOG_DEBUG(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); + LOG_INFO(log, "Dropping table with non-zero lost_part_count equal to {}", lost_part_count); } dropReplica(zookeeper, zookeeper_path, replica_name, log, getSettings(), &has_metadata_in_zookeeper); } From b5cccc5f8d12752bdc713429ef2191d52d25f10b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Sep 2023 14:58:02 +0000 Subject: [PATCH 167/327] Remove unused field --- src/Formats/FormatFactory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 9b6eb97456a..48a1869d563 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -136,7 +136,6 @@ private: SchemaReaderCreator schema_reader_creator; ExternalSchemaReaderCreator external_schema_reader_creator; bool supports_parallel_formatting{false}; - bool supports_subcolumns{false}; bool prefers_large_blocks{false}; NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker; AppendSupportChecker append_support_checker; From 27ac4be8e2bc0511bc5fdd2da0ca9d4c1a8d0e42 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 11 Sep 2023 15:30:06 +0000 Subject: [PATCH 168/327] Fix style --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6e26a8c8d17..5d85c86ee79 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -969,6 +969,8 @@ Xeon YAML YAMLRegExpTree YYYY +YYYYMMDDToDate +YYYYMMDDhhmmssToDateTime Yandex Yasm Zabbix From a1ee226969890d2f31015b732ab146d41a38aa18 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 16:13:11 +0000 Subject: [PATCH 169/327] Update PR status icons --- tests/ci/commit_status_helper.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index c3b1a4d0f95..df78366b9ab 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -133,10 +133,10 @@ def post_commit_status( STATUS_ICON_MAP = defaultdict( str, { - ERROR: "🔴", - FAILURE: "🔴", - PENDING: "🟡", - SUCCESS: "🟢", + ERROR: "❌", + FAILURE: "❌", + PENDING: "⏳", + SUCCESS: "✅", }, ) From 960ca5ebdee8fbe3210af923972396fa36d5b22a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 16:13:33 +0000 Subject: [PATCH 170/327] Show PR status in label --- tests/ci/commit_status_helper.py | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index df78366b9ab..eb9247c2dc4 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -141,17 +141,14 @@ STATUS_ICON_MAP = defaultdict( ) -def update_pr_title_icon(pr: PullRequest, status: str) -> None: - new_status_icon = STATUS_ICON_MAP[status] - if not new_status_icon: - return - new_title = pr.title - if new_title and new_title[0] != new_status_icon: - if new_title[0] in set(STATUS_ICON_MAP.values()): - new_title = new_status_icon + new_title[1:] - else: - new_title = new_status_icon + " " + new_title - pr.edit(title=new_title) +def update_pr_status_label(pr: PullRequest, status: str) -> None: + new_label = "pr-status-" + STATUS_ICON_MAP[status] + for label in pr.get_labels(): + if label.name == new_label: + return + if label.name.startswith("pr-status-"): + label.delete() + pr.add_to_labels(new_label) def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: @@ -193,7 +190,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: comment = ic break - update_pr_title_icon(pr, get_worst_state(statuses)) + update_pr_status_label(pr, get_worst_state(statuses)) if comment is None: pr.create_issue_comment(comment_body) From 4a80a0f1c07db495879e94ab1953f171995b1308 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 16:13:52 +0000 Subject: [PATCH 171/327] upd --- tests/ci/commit_status_helper.py | 16 +++++++--------- tests/ci/report.py | 2 +- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index eb9247c2dc4..54f8f7361fc 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -263,12 +263,6 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: result = [comment_body] - if visible_table_rows: - visible_table_rows.sort() - result.append(table_header) - result.extend(visible_table_rows) - result.append(table_footer) - if hidden_table_rows: hidden_table_rows.sort() result.append(details_header) @@ -277,6 +271,12 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: result.append(table_footer) result.append(details_footer) + if visible_table_rows: + visible_table_rows.sort() + result.append(table_header) + result.extend(visible_table_rows) + result.append(table_footer) + return "".join(result) @@ -292,9 +292,7 @@ def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: log_urls = None if status.target_url is not None: log_urls = [status.target_url] - raw_logs = None - if status.description: - raw_logs = status.description + raw_logs = status.description or None test_results.append( TestResult( status.context, status.state, log_urls=log_urls, raw_logs=raw_logs diff --git a/tests/ci/report.py b/tests/ci/report.py index eb6e3bc1206..2c9e38fb1b1 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -578,7 +578,7 @@ def create_test_html_report( colspan += 1 row.append("") - rows_part.append("".join(row)) + rows_part.append("\n".join(row)) if test_result.raw_logs is not None: raw_logs = escape(test_result.raw_logs) row_raw_logs = ( From 8bb0fbfd2c69c206b210983c85701d6477c20e71 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 17:03:28 +0000 Subject: [PATCH 172/327] fix --- tests/ci/commit_status_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 54f8f7361fc..5bcdb11ab6b 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -147,7 +147,7 @@ def update_pr_status_label(pr: PullRequest, status: str) -> None: if label.name == new_label: return if label.name.startswith("pr-status-"): - label.delete() + pr.remove_from_labels(label.name) pr.add_to_labels(new_label) From 790d61971c27d2e78603254d20dec0b3159359ec Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 11 Sep 2023 17:06:16 +0000 Subject: [PATCH 173/327] sucessfull items in report can be expandable --- tests/ci/report.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 2c9e38fb1b1..73a2fe6f880 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -110,8 +110,8 @@ p.links a {{ padding: 5px; margin: 3px; background: var(--menu-background); line p.links a:hover {{ background: var(--menu-hover-background); color: var(--menu-hover-color); }} th {{ cursor: pointer; }} tr:hover {{ filter: var(--tr-hover-filter); }} -.failed {{ cursor: pointer; }} -.failed-content {{ display: none; }} +.expandable {{ cursor: pointer; }} +.expandable-content {{ display: none; }} #fish {{ display: none; float: right; position: relative; top: -20em; right: 2vw; margin-bottom: -20em; width: 30vw; filter: brightness(7%); z-index: -1; }} .themes {{ @@ -164,9 +164,9 @@ FOOTER_HTML_TEMPLATE = """ tr.addEventListener('click', function() {{ + Array.from(document.getElementsByClassName("expandable")).forEach(tr => tr.addEventListener('click', function() {{ var content = this.nextElementSibling; - content.classList.toggle("failed-content"); + content.classList.toggle("expandable-content"); }})); let theme = 'dark'; @@ -546,9 +546,8 @@ def create_test_html_report( has_log_urls = True row = [] - has_error = test_result.status in ("FAIL", "NOT_FAILED") - if has_error and test_result.raw_logs is not None: - row.append('') + if test_result.raw_logs is not None: + row.append('') else: row.append("") row.append(f"{test_result.name}") @@ -557,6 +556,7 @@ def create_test_html_report( # Allow to quickly scroll to the first failure. fail_id = "" + has_error = test_result.status in ("FAIL", "NOT_FAILED") if has_error: num_fails = num_fails + 1 fail_id = f'id="fail{num_fails}" ' @@ -582,7 +582,7 @@ def create_test_html_report( if test_result.raw_logs is not None: raw_logs = escape(test_result.raw_logs) row_raw_logs = ( - '' + '' f'
{raw_logs}
' "" ) From d5268f3fb0e61b143a2560a9434a28970cb40151 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 11 Sep 2023 19:57:16 +0200 Subject: [PATCH 174/327] rename a member --- src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index b42612d19b3..160ed766c7e 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -161,7 +161,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati const bool keep_all_batch_data; const NameSet file_names_remove_metadata_only; - std::vector removed_files; + std::vector paths_removed_with_objects; std::vector objects_to_remove; RemoveManyObjectStorageOperation( @@ -203,7 +203,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) { objects_to_remove.emplace_back(ObjectsToRemove{std::move(objects), std::move(unlink_outcome)}); - removed_files.push_back(path); + paths_removed_with_objects.push_back(path); } } catch (const Exception & e) @@ -252,7 +252,7 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati &Poco::Logger::get("RemoveManyObjectStorageOperation"), "metadata and objects were removed for [{}], " "only metadata were removed for [{}].", - boost::algorithm::join(removed_files, ", "), + boost::algorithm::join(paths_removed_with_objects, ", "), boost::algorithm::join(file_names_remove_metadata_only, ", ")); } } From 803d8dcf85699ec5a8f6f2e9b948e5dfb64898da Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 6 Sep 2023 12:05:08 +0000 Subject: [PATCH 175/327] Support NULL as default for nested types Array/Tuple/Map for input formats --- src/DataTypes/DataTypeNullable.cpp | 3 + .../Serializations/SerializationArray.cpp | 15 +++- .../Serializations/SerializationMap.cpp | 5 +- .../Serializations/SerializationTuple.cpp | 10 ++- src/Formats/insertNullAsDefaultIfNeeded.cpp | 86 ++++++++++++++++++- src/Formats/insertNullAsDefaultIfNeeded.h | 2 +- .../Impl/NativeORCBlockInputFormat.cpp | 8 -- .../02872_null_as_default_nested.reference | 66 ++++++++++++++ .../02872_null_as_default_nested.sh | 15 ++++ 9 files changed, 193 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/02872_null_as_default_nested.reference create mode 100755 tests/queries/0_stateless/02872_null_as_default_nested.sh diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 41a9a1de543..448caa8275a 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -3,6 +3,9 @@ #include #include #include +#include +#include +#include #include #include #include diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index e01c1aea0e9..d6f36e45e64 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -493,7 +493,10 @@ void SerializationArray::deserializeText(IColumn & column, ReadBuffer & istr, co deserializeTextImpl(column, istr, [&](IColumn & nested_column) { - nested->deserializeTextQuoted(nested_column, istr, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextQuotedImpl(nested_column, istr, settings, nested); + else + nested->deserializeTextQuoted(nested_column, istr, settings); }, false); if (whole && !istr.eof()) @@ -604,7 +607,10 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, rb, [&](IColumn & nested_column) { - nested->deserializeTextCSV(nested_column, rb, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextCSVImpl(nested_column, rb, settings, nested); + else + nested->deserializeTextCSV(nested_column, rb, settings); }, true); } else @@ -612,7 +618,10 @@ void SerializationArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, deserializeTextImpl(column, rb, [&](IColumn & nested_column) { - nested->deserializeTextQuoted(nested_column, rb, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextQuotedImpl(nested_column, rb, settings, nested); + else + nested->deserializeTextQuoted(nested_column, rb, settings); }, true); } } diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index af1d96c4ca7..7588e630689 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -192,7 +192,10 @@ void SerializationMap::deserializeText(IColumn & column, ReadBuffer & istr, cons deserializeTextImpl(column, istr, [&settings](ReadBuffer & buf, const SerializationPtr & subcolumn_serialization, IColumn & subcolumn) { - subcolumn_serialization->deserializeTextQuoted(subcolumn, buf, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextQuotedImpl(subcolumn, buf, settings, subcolumn_serialization); + else + subcolumn_serialization->deserializeTextQuoted(subcolumn, buf, settings); }); if (whole && !istr.eof()) diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 7f3e7619b0d..5c9487b97d4 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -135,7 +135,10 @@ void SerializationTuple::deserializeText(IColumn & column, ReadBuffer & istr, co assertChar(',', istr); skipWhitespaceIfAny(istr); } - elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextQuotedImpl(extractElementColumn(column, i), istr, settings, elems[i]); + else + elems[i]->deserializeTextQuoted(extractElementColumn(column, i), istr, settings); } // Special format for one element tuple (1,) @@ -366,7 +369,10 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, assertChar(settings.csv.tuple_delimiter, istr); skipWhitespaceIfAny(istr); } - elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings); + if (settings.null_as_default) + SerializationNullable::deserializeTextCSVImpl(extractElementColumn(column, i), istr, settings, elems[i]); + else + elems[i]->deserializeTextCSV(extractElementColumn(column, i), istr, settings); } }); } diff --git a/src/Formats/insertNullAsDefaultIfNeeded.cpp b/src/Formats/insertNullAsDefaultIfNeeded.cpp index 767892718c5..c42b8c54d73 100644 --- a/src/Formats/insertNullAsDefaultIfNeeded.cpp +++ b/src/Formats/insertNullAsDefaultIfNeeded.cpp @@ -1,16 +1,96 @@ #include #include +#include +#include +#include #include #include +#include +#include +#include #include +#include namespace DB { -void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values) +bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values) { + if (isArray(input_column.type) && isArray(header_column.type)) + { + ColumnWithTypeAndName nested_input_column; + const auto * array_input_column = checkAndGetColumn(input_column.column.get()); + nested_input_column.column = array_input_column->getDataPtr(); + nested_input_column.type = checkAndGetDataType(input_column.type.get())->getNestedType(); + + ColumnWithTypeAndName nested_header_column; + nested_header_column.column = checkAndGetColumn(header_column.column.get())->getDataPtr(); + nested_header_column.type = checkAndGetDataType(header_column.type.get())->getNestedType(); + + if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr)) + return false; + + input_column.column = ColumnArray::create(nested_input_column.column, array_input_column->getOffsetsPtr()); + input_column.type = std::make_shared(std::move(nested_input_column.type)); + return true; + } + + if (isTuple(input_column.type) && isTuple(header_column.type)) + { + const auto * tuple_input_column = checkAndGetColumn(input_column.column.get()); + const auto * tuple_input_type = checkAndGetDataType(input_column.type.get()); + const auto * tuple_header_column = checkAndGetColumn(header_column.column.get()); + const auto * tuple_header_type = checkAndGetDataType(header_column.type.get()); + + if (tuple_input_type->getElements().size() != tuple_header_type->getElements().size()) + return false; + + Columns nested_input_columns; + nested_input_columns.reserve(tuple_input_type->getElements().size()); + DataTypes nested_input_types; + nested_input_types.reserve(tuple_input_type->getElements().size()); + bool changed = false; + for (size_t i = 0; i != tuple_input_type->getElements().size(); ++i) + { + ColumnWithTypeAndName nested_input_column; + nested_input_column.column = tuple_input_column->getColumnPtr(i); + nested_input_column.type = tuple_input_type->getElement(i); + ColumnWithTypeAndName nested_header_column; + nested_header_column.column = tuple_header_column->getColumnPtr(i); + nested_header_column.type = tuple_header_type->getElement(i); + changed |= insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr); + nested_input_columns.push_back(std::move(nested_input_column.column)); + nested_input_types.push_back(std::move(nested_input_column.type)); + } + + if (!changed) + return false; + + input_column.column = ColumnTuple::create(std::move(nested_input_columns)); + input_column.type = std::make_shared(std::move(nested_input_types)); + return true; + } + + if (isMap(input_column.type) && isMap(header_column.type)) + { + ColumnWithTypeAndName nested_input_column; + nested_input_column.column = checkAndGetColumn(input_column.column.get())->getNestedColumnPtr(); + nested_input_column.type = checkAndGetDataType(input_column.type.get())->getNestedType(); + + ColumnWithTypeAndName nested_header_column; + nested_header_column.column = checkAndGetColumn(header_column.column.get())->getNestedColumnPtr(); + nested_header_column.type = checkAndGetDataType(header_column.type.get())->getNestedType(); + + if (!insertNullAsDefaultIfNeeded(nested_input_column, nested_header_column, 0, nullptr)) + return false; + + input_column.column = ColumnMap::create(std::move(nested_input_column.column)); + input_column.type = std::make_shared(std::move(nested_input_column.type)); + return true; + } + if (!isNullableOrLowCardinalityNullable(input_column.type) || isNullableOrLowCardinalityNullable(header_column.type)) - return; + return false; if (block_missing_values) { @@ -32,6 +112,8 @@ void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const Col const auto * lc_type = assert_cast(input_column.type.get()); input_column.type = std::make_shared(removeNullable(lc_type->getDictionaryType())); } + + return true; } } diff --git a/src/Formats/insertNullAsDefaultIfNeeded.h b/src/Formats/insertNullAsDefaultIfNeeded.h index 3e4dcd1e74a..874f803a14c 100644 --- a/src/Formats/insertNullAsDefaultIfNeeded.h +++ b/src/Formats/insertNullAsDefaultIfNeeded.h @@ -5,6 +5,6 @@ namespace DB { -void insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values); +bool insertNullAsDefaultIfNeeded(ColumnWithTypeAndName & input_column, const ColumnWithTypeAndName & header_column, size_t column_i, BlockMissingValues * block_missing_values); } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 18316a6ebb4..1b6cde11be7 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -850,14 +850,6 @@ static ColumnWithTypeAndName readColumnFromORCColumn( if (skipped) return {}; - if (value_type_hint && !value_type_hint->equals(*value_column.type)) - { - /// Cast value column to target type, because it can happen - /// that parsed type cannot be ClickHouse Map value type. - value_column.column = castColumn(value_column, value_type_hint); - value_column.type = value_type_hint; - } - auto offsets_column = readOffsetsFromORCListColumn(orc_map_column); auto map_column = ColumnMap::create(key_column.column, value_column.column, offsets_column); auto map_type = std::make_shared(key_column.type, value_column.type); diff --git a/tests/queries/0_stateless/02872_null_as_default_nested.reference b/tests/queries/0_stateless/02872_null_as_default_nested.reference new file mode 100644 index 00000000000..dfcaf696d38 --- /dev/null +++ b/tests/queries/0_stateless/02872_null_as_default_nested.reference @@ -0,0 +1,66 @@ +Native +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +Parquet +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +ORC +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +Arrow +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +Avro +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +BSONEachRow +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +MsgPack +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +JSONEachRow +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +CSV +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +TSV +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) +Values +0 [0,0] [[[0,0],[0]]] ('hello',0,0) ('Hello',('Hello',(0,0)),0) {0:0} {0:{0:0}} ('Hello',[('Hello',{0:0},[0])],0) +42 [1,0] [[[1,0],[0]]] ('hello',0,1) ('Hello',('Hello',(1,0)),1) {1:0} {1:{1:0}} ('Hello',[('Hello',{1:0},[0])],1) +2 [2,2] [[[2,2],[0]]] ('hello',2,2) ('Hello',('Hello',(2,2)),2) {2:2} {2:{2:2}} ('Hello',[('Hello',{2:2},[2])],2) +42 [3,0] [[[3,0],[0]]] ('hello',0,3) ('Hello',('Hello',(3,0)),3) {3:0} {3:{3:0}} ('Hello',[('Hello',{3:0},[0])],3) +4 [4,4] [[[4,4],[0]]] ('hello',4,4) ('Hello',('Hello',(4,4)),4) {4:4} {4:{4:4}} ('Hello',[('Hello',{4:4},[4])],4) diff --git a/tests/queries/0_stateless/02872_null_as_default_nested.sh b/tests/queries/0_stateless/02872_null_as_default_nested.sh new file mode 100755 index 00000000000..8f91d573b89 --- /dev/null +++ b/tests/queries/0_stateless/02872_null_as_default_nested.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +for format in Native Parquet ORC Arrow Avro BSONEachRow MsgPack JSONEachRow CSV TSV Values +do + echo $format + $CLICKHOUSE_LOCAL -q "select number % 2 ? NULL : number as n, [number, number % 2 ? NULL : number] as arr1, [[[number, number % 2 ? NULL : number], [NULL]]] as arr2, tuple('hello', number % 2 ? NULL : number, number) as tup1, tuple('Hello', tuple('Hello', tuple(number, number % 2 ? NULL : number)), number) as tup2, map(number, number % 2 ? NULL : number) as map1, map(number, map(number, number % 2 ? null : number)) as map2, tuple('Hello', [tuple('Hello', map(number, number % 2 ? NULL : number), [number % 2 ? NULL : number])], number) as nested from numbers(5) format $format" > $CLICKHOUSE_TEST_UNIQUE_NAME.$format + $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TEST_UNIQUE_NAME.$format', auto, 'n UInt64 default 42, arr1 Array(UInt64), arr2 Array(Array(Array(UInt64))), tup1 Tuple(String, UInt64, UInt64), tup2 Tuple(String, Tuple(String, Tuple(UInt64, UInt64)), UInt64), map1 Map(UInt64, UInt64), map2 Map(UInt64, Map(UInt64, UInt64)), nested Tuple(String, Array(Tuple(String, Map(UInt64, UInt64), Array(UInt64))), UInt64)') settings input_format_null_as_default=1" + rm $CLICKHOUSE_TEST_UNIQUE_NAME.$format +done + From c9612de81f1b96e2927efab4bceb50b7c780f8ca Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 6 Sep 2023 12:06:48 +0000 Subject: [PATCH 176/327] Remove unneded includes --- src/DataTypes/DataTypeNullable.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 448caa8275a..41a9a1de543 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -3,9 +3,6 @@ #include #include #include -#include -#include -#include #include #include #include From 8cd6153c8395af520530321132a59c654488a759 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Sep 2023 14:39:35 +0200 Subject: [PATCH 177/327] Add modification_time into system.detached_parts Signed-off-by: Azat Khuzhin --- .../System/StorageSystemDetachedParts.cpp | 17 ++++++++++++++++- tests/integration/test_partition/test.py | 2 +- .../00502_custom_partitioning_local.sql | 2 +- .../02117_show_create_table_system.reference | 1 + ...m_detached_parts_modification_time.reference | 2 ++ ...stem_detached_parts_modification_time.sql.j2 | 16 ++++++++++++++++ 6 files changed, 37 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02881_system_detached_parts_modification_time.reference create mode 100644 tests/queries/0_stateless/02881_system_detached_parts_modification_time.sql.j2 diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index c5d2ba94e09..cec54cef914 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -230,6 +231,19 @@ private: size_t bytes_on_disk = parts_sizes.at(p_id - begin).load(); new_columns[res_index++]->insert(bytes_on_disk); } + if (columns_mask[src_index++]) + { + Poco::Timestamp modification_time; + try + { + modification_time = p.disk->getLastModified(fs::path(current_info.data->getRelativeDataPath()) / MergeTreeData::DETACHED_DIR_NAME / p.dir_name); + } + catch (const fs::filesystem_error &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + new_columns[res_index++]->insert(static_cast(modification_time.epochTime())); + } if (columns_mask[src_index++]) new_columns[res_index++]->insert(p.disk->getName()); if (columns_mask[src_index++]) @@ -260,12 +274,13 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i {"partition_id", std::make_shared(std::make_shared())}, {"name", std::make_shared()}, {"bytes_on_disk", std::make_shared()}, + {"modification_time",std::make_shared()}, {"disk", std::make_shared()}, {"path", std::make_shared()}, {"reason", std::make_shared(std::make_shared())}, {"min_block_number", std::make_shared(std::make_shared())}, {"max_block_number", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared())} + {"level", std::make_shared(std::make_shared())}, }}); setInMemoryMetadata(storage_metadata); } diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 2517b2d1ae6..97d21d14184 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -431,7 +431,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select system.detached_parts.* except (bytes_on_disk, `path`) from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`, modification_time) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index 3d5f71429fe..a116f8bca24 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT (bytes_on_disk, `path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (bytes_on_disk, `path`, disk, modification_time) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index d96a02834b4..9b633314bd3 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -145,6 +145,7 @@ CREATE TABLE system.detached_parts `partition_id` Nullable(String), `name` String, `bytes_on_disk` UInt64, + `modification_time` DateTime, `disk` String, `path` String, `reason` Nullable(String), diff --git a/tests/queries/0_stateless/02881_system_detached_parts_modification_time.reference b/tests/queries/0_stateless/02881_system_detached_parts_modification_time.reference new file mode 100644 index 00000000000..15d2c113fda --- /dev/null +++ b/tests/queries/0_stateless/02881_system_detached_parts_modification_time.reference @@ -0,0 +1,2 @@ +after detach 1 +after detach 1 diff --git a/tests/queries/0_stateless/02881_system_detached_parts_modification_time.sql.j2 b/tests/queries/0_stateless/02881_system_detached_parts_modification_time.sql.j2 new file mode 100644 index 00000000000..ded17b4e328 --- /dev/null +++ b/tests/queries/0_stateless/02881_system_detached_parts_modification_time.sql.j2 @@ -0,0 +1,16 @@ +set mutations_sync=1; + +{% for id, settings in [ + ("wide", "min_bytes_for_wide_part=0, min_rows_for_wide_part=0"), + ("compact", "min_bytes_for_wide_part=1000, min_rows_for_wide_part=100"), +] +%} + +drop table if exists data_{{ id }}; +create table data_{{ id }} (key Int) engine=MergeTree() order by tuple() settings {{ settings }}; +insert into data_{{ id }} values (1); +select 'before detach', now()-modification_time < 10 from system.detached_parts where database = currentDatabase() and table = 'data_{{ id }}'; +alter table data_{{ id }} detach partition all; +select 'after detach', now()-modification_time < 10 from system.detached_parts where database = currentDatabase() and table = 'data_{{ id }}'; + +{% endfor %} From 4f5d132019c011cdee07faf90a329536c11a7f6f Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 11 Sep 2023 14:46:40 -0400 Subject: [PATCH 178/327] Fix timeout keeper-client --- tests/integration/helpers/keeper_utils.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 767416a3dbc..a1d20e0392b 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -78,6 +78,8 @@ class KeeperClient(object): self.proc.stdin.flush() events = self.poller.poll(timeout) + if not events: + raise TimeoutError(f"Keeper client returned no output") for fd_num, event in events: if event & (select.EPOLLIN | select.EPOLLPRI): From 9e56cff58889395d1375ee0cb2875acba5dde264 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 12 Sep 2023 02:49:27 +0800 Subject: [PATCH 179/327] Better lambda capture --- src/Functions/FunctionsConversion.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 535723bce4e..8d28c9f0a31 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3193,7 +3193,7 @@ private: { if (agg_type->getFunction()->haveSameStateRepresentation(*to_type->getFunction())) { - return [agg_type]( + return [function = agg_type->getFunction()]( ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, const ColumnNullable * /* nullable_source */, @@ -3204,7 +3204,7 @@ private: if (col_agg) { auto new_col_agg = ColumnAggregateFunction::create(*col_agg); - new_col_agg->set(agg_type->getFunction()); + new_col_agg->set(function); return new_col_agg; } else From dd1567f473adc72ba786a5e0c194c4a30dc7d0d3 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Sep 2023 18:54:27 +0000 Subject: [PATCH 180/327] Fix fasttest --- tests/queries/0_stateless/00748_insert_array_with_null.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00748_insert_array_with_null.sql b/tests/queries/0_stateless/00748_insert_array_with_null.sql index ca36352c2cf..ac55d4e9d8c 100644 --- a/tests/queries/0_stateless/00748_insert_array_with_null.sql +++ b/tests/queries/0_stateless/00748_insert_array_with_null.sql @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS arraytest; set allow_deprecated_syntax_for_merge_tree=1; +set input_format_null_as_default=0; CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192); INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']); From dbd2c3f7d986a44b9107623eb077f6d416501923 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Sep 2023 20:57:49 +0200 Subject: [PATCH 181/327] Fix tests --- .../0_stateless/02842_capn_proto_outfile_without_schema.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh index ae57cdd7543..f74e662520b 100755 --- a/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh +++ b/tests/queries/0_stateless/02842_capn_proto_outfile_without_schema.sh @@ -6,6 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "select * from numbers(10) into outfile '$CLICKHOUSE_TEST_UNIQUE_NAME.capnp'" 2>&1 | grep "The format CapnProto requires a schema" -c +$CLICKHOUSE_LOCAL -q "select * from numbers(10) into outfile '$CLICKHOUSE_TEST_UNIQUE_NAME.capnp' settings format_capn_proto_use_autogenerated_schema=0" 2>&1 | grep "The format CapnProto requires a schema" -c rm $CLICKHOUSE_TEST_UNIQUE_NAME.capnp From 0a8138935908ef9aaa6de59a70fb1b6a2ca12b9f Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 11 Sep 2023 12:14:01 -0700 Subject: [PATCH 182/327] Fix endian issue in jemalloc_bins for s390x --- src/Storages/System/StorageSystemJemalloc.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/System/StorageSystemJemalloc.cpp b/src/Storages/System/StorageSystemJemalloc.cpp index 4348349ebbc..d874f657859 100644 --- a/src/Storages/System/StorageSystemJemalloc.cpp +++ b/src/Storages/System/StorageSystemJemalloc.cpp @@ -25,6 +25,12 @@ UInt64 getJeMallocValue(const char * name) UInt64 value{}; size_t size = sizeof(value); mallctl(name, &value, &size, nullptr, 0); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + if (size == 4) + { + value >>= 32; + } +#endif return value; } From 667426f1f222cb6264a61be92b95ad8d9cb9b359 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 12 Sep 2023 03:54:19 +0800 Subject: [PATCH 183/327] DataTypeAggregateFunction::strictEquals --- src/DataTypes/DataTypeAggregateFunction.cpp | 56 +++++++++---------- src/DataTypes/DataTypeAggregateFunction.h | 1 + src/Functions/FunctionsConversion.h | 13 ++++- .../optimizeUseAggregateProjection.cpp | 15 +++-- ...rojection_with_normalized_states.reference | 1 + ...gate_projection_with_normalized_states.sql | 2 + 6 files changed, 54 insertions(+), 34 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index b580ecc9592..be60886d74b 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -117,6 +117,33 @@ Field DataTypeAggregateFunction::getDefault() const return field; } +bool DataTypeAggregateFunction::strictEquals(const DataTypePtr & lhs_state_type, const DataTypePtr & rhs_state_type) +{ + const auto * lhs_state = typeid_cast(lhs_state_type.get()); + const auto * rhs_state = typeid_cast(rhs_state_type.get()); + + if (!lhs_state || !rhs_state) + return false; + + if (lhs_state->function->getName() != rhs_state->function->getName()) + return false; + + if (lhs_state->parameters.size() != rhs_state->parameters.size()) + return false; + + for (size_t i = 0; i < lhs_state->parameters.size(); ++i) + if (lhs_state->parameters[i] != rhs_state->parameters[i]) + return false; + + if (lhs_state->argument_types.size() != rhs_state->argument_types.size()) + return false; + + for (size_t i = 0; i < lhs_state->argument_types.size(); ++i) + if (!lhs_state->argument_types[i]->equals(*rhs_state->argument_types[i])) + return false; + + return true; +} bool DataTypeAggregateFunction::equals(const IDataType & rhs) const { @@ -126,34 +153,7 @@ bool DataTypeAggregateFunction::equals(const IDataType & rhs) const auto lhs_state_type = function->getNormalizedStateType(); auto rhs_state_type = typeid_cast(rhs).function->getNormalizedStateType(); - if (typeid(lhs_state_type.get()) != typeid(rhs_state_type.get())) - return false; - - if (const auto * lhs_state = typeid_cast(lhs_state_type.get())) - { - const auto & rhs_state = typeid_cast(*rhs_state_type); - - if (lhs_state->function->getName() != rhs_state.function->getName()) - return false; - - if (lhs_state->parameters.size() != rhs_state.parameters.size()) - return false; - - for (size_t i = 0; i < lhs_state->parameters.size(); ++i) - if (lhs_state->parameters[i] != rhs_state.parameters[i]) - return false; - - if (lhs_state->argument_types.size() != rhs_state.argument_types.size()) - return false; - - for (size_t i = 0; i < lhs_state->argument_types.size(); ++i) - if (!lhs_state->argument_types[i]->equals(*rhs_state.argument_types[i])) - return false; - - return true; - } - - return lhs_state_type->equals(*rhs_state_type); + return strictEquals(lhs_state_type, rhs_state_type); } diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index 83c9f10f407..6331c23222f 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -60,6 +60,7 @@ public: Field getDefault() const override; + static bool strictEquals(const DataTypePtr & lhs_state_type, const DataTypePtr & rhs_state_type); bool equals(const IDataType & rhs) const override; bool isParametric() const override { return true; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 8d28c9f0a31..ae5b3305baf 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3193,7 +3193,7 @@ private: { if (agg_type->getFunction()->haveSameStateRepresentation(*to_type->getFunction())) { - return [function = agg_type->getFunction()]( + return [function = to_type->getFunction()]( ColumnsWithTypeAndName & arguments, const DataTypePtr & /* result_type */, const ColumnNullable * /* nullable_source */, @@ -4003,7 +4003,16 @@ private: safe_convert_custom_types = to_type->getCustomName() && from_type_custom_name->getName() == to_type->getCustomName()->getName(); if (from_type->equals(*to_type) && safe_convert_custom_types) - return createIdentityWrapper(from_type); + { + /// We can only use identity conversion for DataTypeAggregateFunction when they are strictly equivalent. + if (typeid_cast(from_type.get())) + { + if (DataTypeAggregateFunction::strictEquals(from_type, to_type)) + return createIdentityWrapper(from_type); + } + else + return createIdentityWrapper(from_type); + } else if (WhichDataType(from_type).isNothing()) return createNothingWrapper(to_type.get()); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 0b24d2adbf9..0599a0fa369 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -255,10 +255,17 @@ static void appendAggregateFunctions( if (node->result_name != aggregate.column_name) { - /// Always cast to aggregate types specified in query, because input - /// columns from projection might have the same state but different - /// type, which can generate wrong results during finalization. - node = &proj_dag.addCast(*node, type, aggregate.column_name); + if (DataTypeAggregateFunction::strictEquals(type, node->result_type)) + { + node = &proj_dag.addAlias(*node, aggregate.column_name); + } + else + { + /// Cast to aggregate types specified in query if it's not + /// strictly the same as the one specified in projection. This + /// is required to generate correct results during finalization. + node = &proj_dag.addCast(*node, type, aggregate.column_name); + } } proj_dag_outputs.push_back(node); diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference index 9db43c50b71..25aa9dc5dec 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.reference @@ -1 +1,2 @@ +3 950 990 500 2000 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql index f7be9239c11..e023c0991b3 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_normalized_states.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS r; +select finalizeAggregation(cast(quantileState(0)(arrayJoin([1,2,3])) as AggregateFunction(quantile(1), UInt8))); + CREATE TABLE r ( x String, a LowCardinality(String), From 26b17f88d5e98df38f67c96c69d60824eccecf33 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 11 Sep 2023 13:52:58 -0700 Subject: [PATCH 184/327] Add comment for the fix --- src/Storages/System/StorageSystemJemalloc.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/System/StorageSystemJemalloc.cpp b/src/Storages/System/StorageSystemJemalloc.cpp index d874f657859..207b378c240 100644 --- a/src/Storages/System/StorageSystemJemalloc.cpp +++ b/src/Storages/System/StorageSystemJemalloc.cpp @@ -25,6 +25,9 @@ UInt64 getJeMallocValue(const char * name) UInt64 value{}; size_t size = sizeof(value); mallctl(name, &value, &size, nullptr, 0); + /// mallctl() fills the value with 32 bit integer for some queries("arenas.nbins" for example). + /// In this case variable 'size' will be changed from 8 to 4 and the 64 bit variable 'value' will hold the 32 bit actual value times 2^32 on big-endian machines. + /// We should right shift the value by 32 on big-endian machines(which is unnecessary on little-endian machines). #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ if (size == 4) { From df71dcd94d893f124040105ad35755b750abbe9f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 26 Jun 2023 21:49:44 +0000 Subject: [PATCH 185/327] Fix data race in copyFromIStreamWithProgressCallback --- src/IO/HTTPCommon.h | 1 + src/IO/ReadBufferFromS3.cpp | 21 ++++++++++-------- src/IO/ReadWriteBufferFromHTTP.cpp | 26 ++++++++++++----------- src/IO/SeekableReadBuffer.cpp | 5 +++-- src/IO/SeekableReadBuffer.h | 3 ++- tests/integration/test_storage_s3/test.py | 6 +++--- 6 files changed, 35 insertions(+), 27 deletions(-) diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index caf2fa361d9..04ca85925af 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -62,6 +62,7 @@ struct HTTPSessionReuseTag { }; +void markSessionForReuse(Poco::Net::HTTPSession & session); void markSessionForReuse(HTTPSessionPtr session); void markSessionForReuse(PooledHTTPSessionPtr session); diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index de38487642c..c038523bdaa 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -221,13 +221,12 @@ bool ReadBufferFromS3::nextImpl() size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) { - if (n == 0) - return 0; - + size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0;; ++attempt) + for (size_t attempt = 0; n > 0; ++attempt) { bool last_attempt = attempt + 1 >= request_settings.max_single_read_retries; + size_t bytes_copied = 0; ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3Microseconds); @@ -236,14 +235,12 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons auto result = sendRequest(range_begin, range_begin + n - 1); std::istream & istr = result.GetBody(); - size_t bytes = copyFromIStreamWithProgressCallback(istr, to, n, progress_callback); + copyFromIStreamWithProgressCallback(istr, to, n, progress_callback, &bytes_copied); - ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes); + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3Bytes, bytes_copied); if (read_settings.remote_throttler) - read_settings.remote_throttler->add(bytes, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - - return bytes; + read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); } catch (Poco::Exception & e) { @@ -253,7 +250,13 @@ size_t ReadBufferFromS3::readBigAt(char * to, size_t n, size_t range_begin, cons sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; } + + range_begin += bytes_copied; + to += bytes_copied; + n -= bytes_copied; } + + return initial_n; } bool ReadBufferFromS3::processException(Poco::Exception & e, size_t read_offset, size_t attempt) const diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 7e5c0d37c8e..d0384cfa524 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -594,16 +594,14 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si /// This ensures we've sent at least one HTTP request and populated saved_uri_redirect. chassert(file_info && file_info->seekable); - if (n == 0) - return 0; - Poco::URI uri_ = saved_uri_redirect.value_or(uri); if (uri_.getPath().empty()) uri_.setPath("/"); + size_t initial_n = n; size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; - for (size_t attempt = 0;; ++attempt) + for (size_t attempt = 0; n > 0; ++attempt) { bool last_attempt = attempt + 1 >= settings.http_max_tries; @@ -616,6 +614,7 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si Poco::Net::HTTPResponse response; std::istream * result_istr; + size_t bytes_copied = 0; try { @@ -629,17 +628,14 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si "Expected 206 Partial Content, got {} when reading {} range [{}, {})", toString(response.getStatus()), uri_.toString(), offset, offset + n); - bool cancelled; - size_t r = copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &cancelled); - - if (!cancelled) + copyFromIStreamWithProgressCallback(*result_istr, to, n, progress_callback, &bytes_copied); + if (bytes_copied == n) { + result_istr->ignore(UINT64_MAX); /// Response was fully read. - markSessionForReuse(sess); + markSessionForReuse(*sess); ProfileEvents::increment(ProfileEvents::ReadWriteBufferFromHTTPPreservedSessions); } - - return r; } catch (const Poco::Exception & e) { @@ -664,9 +660,15 @@ size_t ReadWriteBufferFromHTTPBase::readBigAt(char * to, si sleepForMilliseconds(milliseconds_to_wait); milliseconds_to_wait = std::min(milliseconds_to_wait * 2, settings.http_retry_max_backoff_ms); - continue; } + + /// Make sure retries don't re-read the bytes that we've already reported to progress_callback. + offset += bytes_copied; + to += bytes_copied; + n -= bytes_copied; } + + return initial_n; } template diff --git a/src/IO/SeekableReadBuffer.cpp b/src/IO/SeekableReadBuffer.cpp index b83e382db01..5d83f4e1b4a 100644 --- a/src/IO/SeekableReadBuffer.cpp +++ b/src/IO/SeekableReadBuffer.cpp @@ -64,7 +64,7 @@ std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBu return std::make_unique>(*ptr, SeekableReadBufferPtr{ptr}); } -size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled) +void copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, size_t * out_bytes_copied, bool * out_cancelled) { const size_t chunk = DBMS_DEFAULT_BUFFER_SIZE; if (out_cancelled) @@ -82,6 +82,7 @@ size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_ bool cancelled = false; if (gcount && progress_callback) cancelled = progress_callback(copied); + *out_bytes_copied = copied; if (gcount != to_copy) { @@ -103,7 +104,7 @@ size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_ } } - return copied; + *out_bytes_copied = copied; } } diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 5770948be20..e21513e0ea2 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -98,6 +98,7 @@ std::unique_ptr wrapSeekableReadBufferReference(SeekableRead std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBufferPtr ptr); /// Helper for implementing readBigAt(). -size_t copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, bool * out_cancelled = nullptr); +/// Updates *out_bytes_copied after each call to the callback, as well as at the end. +void copyFromIStreamWithProgressCallback(std::istream & istr, char * to, size_t n, const std::function & progress_callback, size_t * out_bytes_copied, bool * out_cancelled = nullptr); } diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ba0af711151..6b05379c712 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1018,11 +1018,11 @@ def test_url_reconnect_in_the_middle(started_cluster): def select(): global result result = instance.query( - f"""select sum(cityHash64(x)) from (select toUInt64(id) + sleep(0.1) as x from + f"""select count(), sum(cityHash64(x)) from (select toUInt64(id) + sleep(0.1) as x from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'TSV', '{table_format}') settings http_max_tries = 10, http_retry_max_backoff_ms=2000, http_send_timeout=1, http_receive_timeout=1)""" ) - assert int(result) == 3914219105369203805 + assert result == "1000000\t3914219105369203805\n" thread = threading.Thread(target=select) thread.start() @@ -1035,7 +1035,7 @@ def test_url_reconnect_in_the_middle(started_cluster): thread.join() - assert int(result) == 3914219105369203805 + assert result == "1000000\t3914219105369203805\n" def test_seekable_formats(started_cluster): From d5ea047ab8cc11fd60ea91fb876fc98be1aaec99 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 11 Sep 2023 21:52:40 +0000 Subject: [PATCH 186/327] Parallel replicas: cleanup unused params --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 -- src/Interpreters/ClusterProxy/executeQuery.h | 1 - src/Processors/QueryPlan/ReadFromRemote.cpp | 2 -- src/Processors/QueryPlan/ReadFromRemote.h | 2 -- src/Storages/StorageMergeTree.cpp | 10 +++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 11 +++++++---- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 34be2636917..0890801062e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -266,7 +266,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, const StorageID & main_table, - const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, @@ -328,7 +327,6 @@ void executeQueryWithParallelReplicas( stream_factory.header, stream_factory.processed_stage, main_table, - table_func_ptr, new_context, getThrottler(new_context), std::move(scalars), diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index b663dffb7fa..5a88f5a5cb1 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -65,7 +65,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, const StorageID & main_table, - const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 78da19f48a0..f389e5a9e1e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -305,7 +305,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, - ASTPtr table_func_ptr_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -318,7 +317,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) , main_table(std::move(main_table_)) - , table_func_ptr(table_func_ptr_) , context(context_) , throttler(throttler_) , scalars(scalars_) diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 43e1904c443..a2486e1eaa1 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -74,7 +74,6 @@ public: Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, - ASTPtr table_func_ptr_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -98,7 +97,6 @@ private: ParallelReplicasReadingCoordinatorPtr coordinator; QueryProcessingStage::Enum stage; StorageID main_table; - ASTPtr table_func_ptr; ContextMutablePtr context; ThrottlerPtr throttler; Scalars scalars; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ac2c7baf857..bae91ec8bb1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -237,9 +237,13 @@ void StorageMergeTree::read( processed_stage); ClusterProxy::executeQueryWithParallelReplicas( - query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, - select_stream_factory, modified_query_ast, - local_context, query_info.storage_limits, cluster); + query_plan, + getStorageID(), + select_stream_factory, + modified_query_ast, + local_context, + query_info.storage_limits, + cluster); } else { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f9538f60760..fd2ff0ca4b2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5187,10 +5187,13 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( processed_stage); ClusterProxy::executeQueryWithParallelReplicas( - query_plan, getStorageID(), - /* table_func_ptr= */ nullptr, - select_stream_factory, modified_query_ast, - local_context, query_info.storage_limits, parallel_replicas_cluster); + query_plan, + getStorageID(), + select_stream_factory, + modified_query_ast, + local_context, + query_info.storage_limits, + parallel_replicas_cluster); } void StorageReplicatedMergeTree::readLocalImpl( From 60c9dd226dfd811069bc015c99202d6e9363fb10 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 12 Sep 2023 10:10:05 +1000 Subject: [PATCH 187/327] Update src/Interpreters/QueryLog.h Co-authored-by: Dmitry Novik --- src/Interpreters/QueryLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 733be5840e5..fe9b7cbdbc8 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -91,7 +91,7 @@ struct QueryLogElement String log_comment; std::vector thread_ids; - UInt64 peak_threads_usage{}; + UInt64 peak_threads_usage = 0; std::shared_ptr profile_counters; std::shared_ptr async_read_counters; std::shared_ptr query_settings; From 30c213a65eca96cddfc0aa2a4acde2d466945168 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 12 Sep 2023 10:10:20 +1000 Subject: [PATCH 188/327] Update src/Common/ConcurrencyControl.cpp Co-authored-by: Dmitry Novik --- src/Common/ConcurrencyControl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ConcurrencyControl.cpp b/src/Common/ConcurrencyControl.cpp index ebc1c3f9f42..c9fe51550dc 100644 --- a/src/Common/ConcurrencyControl.cpp +++ b/src/Common/ConcurrencyControl.cpp @@ -57,7 +57,7 @@ ConcurrencyControl::Allocation::Allocation(ConcurrencyControl & parent_, SlotCou *waiter = this; } -// Grant single slot to allocation, returns true if more slot(s) are required +// Grant single slot to allocation returns true iff more slot(s) are required bool ConcurrencyControl::Allocation::grant() { std::unique_lock lock{mutex}; From 4890a516b422701907b59674ce90074e7f43a17e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Sep 2023 04:50:23 +0300 Subject: [PATCH 189/327] Revert "add runOptimize call in bitmap write method" --- src/AggregateFunctions/AggregateFunctionGroupBitmapData.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index f92f8c1b5e5..7ea1ebe7749 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -151,7 +151,6 @@ public: } else if (BitmapKind::Bitmap == kind) { - roaring_bitmap->runOptimize(); auto size = roaring_bitmap->getSizeInBytes(); writeVarUInt(size, out); std::unique_ptr buf(new char[size]); From 44a9dbd2b701ec299161f32974ec3d502f7eaf83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Sep 2023 04:53:51 +0300 Subject: [PATCH 190/327] Update StorageSystemDetachedParts.cpp --- src/Storages/System/StorageSystemDetachedParts.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index cec54cef914..a9cd5f2610a 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -233,7 +233,7 @@ private: } if (columns_mask[src_index++]) { - Poco::Timestamp modification_time; + Poco::Timestamp modification_time{}; try { modification_time = p.disk->getLastModified(fs::path(current_info.data->getRelativeDataPath()) / MergeTreeData::DETACHED_DIR_NAME / p.dir_name); From ed7b687a696b53d7443432122e954b88bc30883b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 12 Sep 2023 08:58:43 +0000 Subject: [PATCH 191/327] parallelize infile --- src/QueryPipeline/QueryPipeline.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp index 9a836f68da1..4250e9c21a3 100644 --- a/src/QueryPipeline/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -353,7 +353,8 @@ QueryPipeline::QueryPipeline(Pipe pipe) totals = pipe.getTotalsPort(); extremes = pipe.getExtremesPort(); partial_result = pipe.getPartialResultPort(0); - + num_threads = pipe.max_parallel_streams; + processors = std::move(pipe.processors); checkPulling(*processors, output, totals, extremes, partial_result); } From dd6ee83ad438893da464814e308d675964823a93 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Sep 2023 11:29:24 +0200 Subject: [PATCH 192/327] Ping CI From 3e16bdbcc7a437f7186b9de05703a71c2278ca5a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Sep 2023 11:10:17 +0000 Subject: [PATCH 193/327] Fix UB --- src/Functions/makeDate.cpp | 14 +++++------- .../02876_yyyymmddhhmmsstodatetime.reference | 22 +++++++++---------- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index bb8c302d139..735fcdfd657 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -16,6 +16,7 @@ #include #include +#include namespace DB { @@ -217,7 +218,7 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { - const auto yyyymmdd = static_cast(yyyymmdd_data[i]); + const auto yyyymmdd = static_cast(static_cast(yyyymmdd_data[i])); /// Float64-to-UInt64 is UB, double-cast avoids it const auto year = yyyymmdd / 10'000; const auto month = yyyymmdd / 100 % 100; @@ -691,7 +692,7 @@ public: { const auto float_date = yyyymmddhhmmss_data[i]; - const auto yyyymmddhhmmss = static_cast(float_date); + const auto yyyymmddhhmmss = static_cast(static_cast(float_date)); /// Float64-to-UInt64 is UB, double-cast avoids it const auto yyyymmdd = yyyymmddhhmmss / 1'000'000; const auto hhmmss = yyyymmddhhmmss % 1'000'000; @@ -705,14 +706,11 @@ public: const auto minute = hhmmss / 100 % 100; const auto second = hhmmss % 100; - auto fraction = static_cast(decimal * fraction_pow); - auto date_time = dateTime(year, month, day, hour, minute, second, date_lut); - result_data[i] = DecimalUtils::decimalFromComponents( - date_time, - static_cast(fraction), - static_cast(precision)); + auto fraction = std::llround(decimal * fraction_pow); + + result_data[i] = DecimalUtils::decimalFromComponents(date_time, fraction, precision); } return res_column; diff --git a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference index 9d010535451..ffef2f3e360 100644 --- a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference +++ b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.reference @@ -55,20 +55,20 @@ Check correctness, integer arguments 2299-12-31 23:59:59.000 Check correctness, float arguments 1900-01-01 00:00:00.900 -1900-01-01 00:00:00.899 -2020-02-29 11:11:11.101 -2023-09-11 15:05:05.101 -2299-12-31 23:59:59.101 -2299-12-31 23:59:59.101 +1900-01-01 00:00:00.898 +2020-02-29 11:11:11.102 +2023-09-11 15:05:05.102 +2299-12-31 23:59:59.102 +2299-12-31 23:59:59.102 Check correctness, decimal arguments 1900-01-01 00:00:00.900 -1900-01-01 00:00:00.899 -2020-02-29 11:11:11.101 -2023-09-11 15:05:05.101 -2299-12-31 23:59:59.101 -2299-12-31 23:59:59.101 +1900-01-01 00:00:00.898 +2020-02-29 11:11:11.102 +2023-09-11 15:05:05.102 +2299-12-31 23:59:59.102 +2299-12-31 23:59:59.102 Special cases -1900-01-01 00:00:00.648 +1900-01-01 00:00:00.808 1900-01-01 00:00:00.000 1900-01-01 00:00:00.000 1900-01-01 00:00:00.000 From 72d5a5d094772d390115a67419b0400c04feb5e2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Sep 2023 11:54:35 +0000 Subject: [PATCH 194/327] fix report.py --- tests/ci/report.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 73a2fe6f880..dc31314d031 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -148,7 +148,7 @@ FOOTER_HTML_TEMPLATE = """ Date: Tue, 12 Sep 2023 11:55:55 +0000 Subject: [PATCH 195/327] Revert "do not merge this commit" This reverts commit 6a51d9fd7d2a373741f79ac7f9100225d79ef8bd. --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 840a77868d6..e6edb99e301 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7656,7 +7656,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom( entry_replace.columns_version = -1; } - if (replace) { + if (replace) + { /// Cancel concurrent inserts in range clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); /// Remove deduplication block_ids of replacing parts From 8db991faa0f781dafb11cd81f8cce4d26a73a5e8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 12 Sep 2023 13:41:00 +0000 Subject: [PATCH 196/327] Fix style --- tests/ci/fast_test_check.py | 2 +- tests/ci/stress_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 3d6d43a2d54..13dd6dcb5e2 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: - logging.info(f"Timeout expired for command: {run_cmd}") + logging.info("Timeout expired for command: %s", run_cmd) timeout_expired = True elif retcode == 0: logging.info("Run successfully") diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 9e607f7da82..4e79bdc769d 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -179,7 +179,7 @@ def run_stress_test(docker_image_name): with TeePopen(run_command, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: - logging.info(f"Timeout expired for command: {run_command}") + logging.info("Timeout expired for command: %s", run_command) timeout_expired = True elif retcode == 0: logging.info("Run successfully") From acb559ce8bd385d311c27aabb3e88fbf4a1cfa22 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 12 Sep 2023 17:34:52 +0200 Subject: [PATCH 197/327] Update tests/ci/commit_status_helper.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/commit_status_helper.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 5bcdb11ab6b..a5fd27efb6b 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -289,9 +289,9 @@ def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: to S3 tests bucket. Then it returns the URL""" test_results = [] # type: TestResults for status in statuses: - log_urls = None + log_urls = [] if status.target_url is not None: - log_urls = [status.target_url] + log_urls.append(status.target_url) raw_logs = status.description or None test_results.append( TestResult( From e406467346471cad47507bfdb0b5596df238b0b0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Sep 2023 16:28:08 +0000 Subject: [PATCH 198/327] More UB safety --- src/Functions/makeDate.cpp | 12 +++++++++++- .../0_stateless/02876_yyyymmddhhmmsstodatetime.sql | 6 ++++++ tests/queries/0_stateless/02876_yyyymmddtodate.sql | 6 ++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 735fcdfd657..5090b39977f 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -22,8 +22,9 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -218,6 +219,9 @@ public: for (size_t i = 0; i < input_rows_count; ++i) { + if (std::isinf(yyyymmdd_data[i]) || std::isnan(yyyymmdd_data[i])) [[unlikely]] + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName()); + const auto yyyymmdd = static_cast(static_cast(yyyymmdd_data[i])); /// Float64-to-UInt64 is UB, double-cast avoids it const auto year = yyyymmdd / 10'000; @@ -602,6 +606,9 @@ public: for (size_t i = 0; i < input_rows_count; i++) { + if (std::isinf(yyyymmddhhmmss_data[i]) || std::isnan(yyyymmddhhmmss_data[i])) [[unlikely]] + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName()); + const auto yyyymmddhhmmss = static_cast(yyyymmddhhmmss_data[i]); const auto yyyymmdd = yyyymmddhhmmss / 1'000'000; @@ -692,6 +699,9 @@ public: { const auto float_date = yyyymmddhhmmss_data[i]; + if (std::isinf(float_date) || std::isnan(float_date)) [[unlikely]] + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument for function {} must be finite", getName()); + const auto yyyymmddhhmmss = static_cast(static_cast(float_date)); /// Float64-to-UInt64 is UB, double-cast avoids it const auto yyyymmdd = yyyymmddhhmmss / 1'000'000; diff --git a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql index 1b5f6220b50..84b3bcc4491 100644 --- a/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql +++ b/tests/queries/0_stateless/02876_yyyymmddhhmmsstodatetime.sql @@ -33,6 +33,9 @@ SELECT YYYYMMDDhhmmssToDateTime(20200229111111.1); -- leap day SELECT YYYYMMDDhhmmssToDateTime(20230911150505.1); SELECT YYYYMMDDhhmmssToDateTime(21060207062815.1); SELECT YYYYMMDDhhmmssToDateTime(21060207062816.1); +SELECT YYYYMMDDhhmmssToDateTime(NaN); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime(Inf); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime(-Inf); -- { serverError BAD_ARGUMENTS } SELECT 'Check correctness, decimal arguments'; SELECT YYYYMMDDhhmmssToDateTime(toDecimal64(19691231595959.1, 5)); @@ -93,6 +96,9 @@ SELECT YYYYMMDDhhmmssToDateTime64(20200229111111.1); -- leap day SELECT YYYYMMDDhhmmssToDateTime64(20230911150505.1); SELECT YYYYMMDDhhmmssToDateTime64(22991231235959.1); SELECT YYYYMMDDhhmmssToDateTime64(23000101000000.1); +SELECT YYYYMMDDhhmmssToDateTime64(NaN); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime64(Inf); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDhhmmssToDateTime64(-Inf); -- { serverError BAD_ARGUMENTS } SELECT 'Check correctness, decimal arguments'; SELECT YYYYMMDDhhmmssToDateTime64(toDecimal64(189912315959.1, 5)); diff --git a/tests/queries/0_stateless/02876_yyyymmddtodate.sql b/tests/queries/0_stateless/02876_yyyymmddtodate.sql index fbacca1e2a2..2d4ab46f9ea 100644 --- a/tests/queries/0_stateless/02876_yyyymmddtodate.sql +++ b/tests/queries/0_stateless/02876_yyyymmddtodate.sql @@ -30,6 +30,9 @@ SELECT YYYYMMDDToDate(20200229.1); -- leap day SELECT YYYYMMDDToDate(20230911.1); SELECT YYYYMMDDToDate(21490606.1); SELECT YYYYMMDDToDate(21490607.1); +SELECT YYYYMMDDToDate(NaN); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDToDate(Inf); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDToDate(-Inf); -- { serverError BAD_ARGUMENTS } SELECT 'Check correctness, decimal arguments'; SELECT YYYYMMDDToDate(toDecimal64(19691231.1, 5)); @@ -86,6 +89,9 @@ SELECT YYYYMMDDToDate32(20200229.1); -- leap day SELECT YYYYMMDDToDate32(20230911.1); SELECT YYYYMMDDToDate32(22991231.1); SELECT YYYYMMDDToDate32(23000101.1); +SELECT YYYYMMDDToDate32(NaN); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDToDate32(Inf); -- { serverError BAD_ARGUMENTS } +SELECT YYYYMMDDToDate32(-Inf); -- { serverError BAD_ARGUMENTS } SELECT 'Check correctness, decimal arguments'; SELECT YYYYMMDDToDate32(toDecimal64(18991231.1, 5)); From dddea9219a1327827d01bdabacd4f879c28aea0a Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 12 Sep 2023 18:39:03 +0200 Subject: [PATCH 199/327] Address the review comments --- .../integration/mysql_java_client/Dockerfile | 11 +- .../PreparedStatementsTest.java | 5 +- .../docker_compose_mysql_java_client.yml | 2 +- docs/en/sql-reference/statements/show.md | 2 +- src/Core/MySQL/PacketsPreparedStatements.cpp | 52 +- src/Core/MySQL/PacketsPreparedStatements.h | 38 +- src/Core/MySQL/PacketsProtocolBinary.cpp | 651 +++++++++--------- src/Core/MySQL/PacketsProtocolBinary.h | 47 +- src/Core/MySQL/PacketsProtocolText.cpp | 5 +- src/Formats/FormatSettings.h | 2 +- .../InterpreterShowColumnsQuery.cpp | 2 +- .../Formats/Impl/MySQLOutputFormat.cpp | 2 +- src/Server/MySQLHandler.cpp | 92 +-- src/Server/MySQLHandler.h | 14 +- .../prepared_statements.reference | 5 - .../prepared_statements_test.sql | 25 +- 16 files changed, 477 insertions(+), 478 deletions(-) diff --git a/docker/test/integration/mysql_java_client/Dockerfile b/docker/test/integration/mysql_java_client/Dockerfile index 75e3c9de138..83c17551481 100644 --- a/docker/test/integration/mysql_java_client/Dockerfile +++ b/docker/test/integration/mysql_java_client/Dockerfile @@ -1,16 +1,9 @@ # docker build -t clickhouse/mysql-java-client . # MySQL Java client docker container -FROM ubuntu:22.04 +FROM openjdk:8-jdk-alpine -RUN apt-get update && \ - apt-get install -y software-properties-common build-essential openjdk-8-jdk curl - -RUN rm -rf \ - /var/lib/apt/lists/* \ - /var/cache/debconf \ - /tmp/* \ -RUN apt-get clean +RUN apk --no-cache add curl ARG ver=8.1.0 RUN curl -L -o /mysql-connector-j-${ver}.jar https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/${ver}/mysql-connector-j-${ver}.jar diff --git a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java index 9efb3330fd5..dc72f8d3526 100644 --- a/docker/test/integration/mysql_java_client/PreparedStatementsTest.java +++ b/docker/test/integration/mysql_java_client/PreparedStatementsTest.java @@ -33,8 +33,8 @@ public class PreparedStatementsTest { } } - // useServerPrepStmts allows us to send COM_STMT_PREPARE and COM_STMT_EXECUTE - // to test the binary protocol implementation + // useServerPrepStmts uses COM_STMT_PREPARE and COM_STMT_EXECUTE + // instead of COM_QUERY which allows us to test the binary protocol String jdbcUrl = String.format("jdbc:mysql://%s:%s/%s?useSSL=false&useServerPrepStmts=true", host, port, database); @@ -174,4 +174,5 @@ public class PreparedStatementsTest { return String.format("%s type is %s", columnLabel, MysqlType.getByJdbcType(meta.getColumnType(rs.findColumn(columnLabel)))); } + } diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml index eb5ffb01baa..529974dd4bf 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_java_client.yml @@ -3,4 +3,4 @@ services: java1: image: clickhouse/mysql-java-client:${DOCKER_MYSQL_JAVA_CLIENT_TAG:-latest} # to keep container running - command: sleep infinity + command: sleep 1d diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index e94718394de..4341cf2998e 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -208,7 +208,7 @@ The optional keyword `FULL` causes the output to include the collation, comment The statement produces a result table with the following structure: - field - The name of the column (String) - type - The column data type (String) -- null - If the column data type is Nullable (UInt8) +- null - If the column data type is Nullable (String) - key - `PRI` if the column is part of the primary key, `SOR` if the column is part of the sorting key, empty otherwise (String) - default - Default expression of the column if it is of type `ALIAS`, `DEFAULT`, or `MATERIALIZED`, otherwise `NULL`. (Nullable(String)) - extra - Additional information, currently unused (String) diff --git a/src/Core/MySQL/PacketsPreparedStatements.cpp b/src/Core/MySQL/PacketsPreparedStatements.cpp index dcff294ebad..654c28379d9 100644 --- a/src/Core/MySQL/PacketsPreparedStatements.cpp +++ b/src/Core/MySQL/PacketsPreparedStatements.cpp @@ -2,33 +2,41 @@ #include #include + namespace DB { namespace MySQLProtocol { - namespace PreparedStatements - { - size_t PrepareStatementResponseOK::getPayloadSize() const - { - return 13; - } +namespace PreparedStatements +{ +size_t PreparedStatementResponseOK::getPayloadSize() const +{ + // total = 13 + return 1 // status + + 4 // statement_id + + 2 // num_columns + + 2 // num_params + + 1 // reserved_1 (filler) + + 2 // warnings_count + + 1; // metadata_follows +} - void PrepareStatementResponseOK::writePayloadImpl(WriteBuffer & buffer) const - { - buffer.write(reinterpret_cast(&status), 1); - buffer.write(reinterpret_cast(&statement_id), 4); - buffer.write(reinterpret_cast(&num_columns), 2); - buffer.write(reinterpret_cast(&num_params), 2); - buffer.write(reinterpret_cast(&reserved_1), 1); - buffer.write(reinterpret_cast(&warnings_count), 2); - buffer.write(0x0); // RESULTSET_METADATA_NONE - } +void PreparedStatementResponseOK::writePayloadImpl(WriteBuffer & buffer) const +{ + buffer.write(reinterpret_cast(&status), 1); + buffer.write(reinterpret_cast(&statement_id), 4); + buffer.write(reinterpret_cast(&num_columns), 2); + buffer.write(reinterpret_cast(&num_params), 2); + buffer.write(reinterpret_cast(&reserved_1), 1); + buffer.write(reinterpret_cast(&warnings_count), 2); + buffer.write(0x0); // RESULTSET_METADATA_NONE +} - PrepareStatementResponseOK::PrepareStatementResponseOK( - uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_) - : statement_id(statement_id_), num_columns(num_columns_), num_params(num_params_), warnings_count(warnings_count_) - { - } - } +PreparedStatementResponseOK::PreparedStatementResponseOK( + uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_) + : statement_id(statement_id_), num_columns(num_columns_), num_params(num_params_), warnings_count(warnings_count_) +{ +} +} } } diff --git a/src/Core/MySQL/PacketsPreparedStatements.h b/src/Core/MySQL/PacketsPreparedStatements.h index 5aca459fc3a..2c5ab8b5596 100644 --- a/src/Core/MySQL/PacketsPreparedStatements.h +++ b/src/Core/MySQL/PacketsPreparedStatements.h @@ -6,27 +6,27 @@ namespace DB { namespace MySQLProtocol { - namespace PreparedStatements - { - // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_prepare.html#sect_protocol_com_stmt_prepare_response_ok - class PrepareStatementResponseOK : public IMySQLWritePacket - { - public: - uint8_t status = 0x00; - uint32_t statement_id; - uint16_t num_columns; - uint16_t num_params; - uint8_t reserved_1 = 0; - uint16_t warnings_count; +namespace PreparedStatements +{ +// https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_prepare.html#sect_protocol_com_stmt_prepare_response_ok +class PreparedStatementResponseOK : public IMySQLWritePacket +{ +public: + const uint8_t status = 0x00; + uint32_t statement_id; + uint16_t num_columns; + uint16_t num_params; + const uint8_t reserved_1 = 0; + uint16_t warnings_count; - protected: - size_t getPayloadSize() const override; +protected: + size_t getPayloadSize() const override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; - public: - PrepareStatementResponseOK(uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_); - }; - } +public: + PreparedStatementResponseOK(uint32_t statement_id_, uint16_t num_columns_, uint16_t num_params_, uint16_t warnings_count_); +}; +} } } diff --git a/src/Core/MySQL/PacketsProtocolBinary.cpp b/src/Core/MySQL/PacketsProtocolBinary.cpp index 9e5e2993595..13014e5a46b 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.cpp +++ b/src/Core/MySQL/PacketsProtocolBinary.cpp @@ -20,344 +20,337 @@ namespace DB { - namespace MySQLProtocol { - - namespace ProtocolBinary +namespace ProtocolBinary +{ +ResultSetRow::ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_) + : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) +{ + payload_size = 1 + null_bitmap_size; + FormatSettings format_settings; + for (size_t i = 0; i < columns.size(); ++i) { - ResultSetRow::ResultSetRow( - const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_) - : row_num(row_num_), columns(columns_), data_types(data_types_), serializations(serializations_) + ColumnPtr col = columns[i]; + if (col->isNullAt(row_num)) { - /// See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row - payload_size = 1 + null_bitmap_size; - FormatSettings format_settings; - for (size_t i = 0; i < columns.size(); ++i) - { - ColumnPtr col = columns[i]; - if (col->isNullAt(row_num)) - { - size_t byte = (i + 2) / 8; - int bit = 1 << ((i + 2) % 8); - null_bitmap[byte] |= bit; - continue; // NULLs are stored in the null bitmap only - } - - DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); - TypeIndex type_index = data_type->getTypeId(); - switch (type_index) - { - case TypeIndex::Int8: - payload_size += 1; - break; - case TypeIndex::UInt8: - if (data_type->getName() == "Bool") - { - payload_size += 2; // BIT MySQL type is string in binary - } - else - { - payload_size += 1; - } - break; - case TypeIndex::Int16: - case TypeIndex::UInt16: - payload_size += 2; - break; - case TypeIndex::Int32: - case TypeIndex::UInt32: - case TypeIndex::Float32: - payload_size += 4; - break; - case TypeIndex::Int64: - case TypeIndex::UInt64: - case TypeIndex::Float64: - payload_size += 8; - break; - case TypeIndex::Date: { - payload_size += 5; - break; - } - case TypeIndex::Date32: { - payload_size += 5; - break; - } - case TypeIndex::DateTime: { - UInt32 value = assert_cast &>(*col).getData()[row_num]; - LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); - - bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); - if (has_time) - { - payload_size += 8; - } - else - { - payload_size += 5; - } - break; - } - case TypeIndex::DateTime64: { - const auto * date_time_type = typeid_cast(data_type.get()); - UInt32 scale = date_time_type->getScale(); - - static constexpr UInt32 MaxScale = DecimalUtils::max_precision; - scale = scale > MaxScale ? MaxScale : scale; - - const auto dt64 = assert_cast(*col).getData()[row_num]; - auto components = DecimalUtils::split(dt64, scale); - - using T = typename DateTime64::NativeType; - if (dt64.value < 0 && components.fractional) - { - components.fractional - = DecimalUtils::scaleMultiplier(scale) + (components.whole ? T(-1) : T(1)) * components.fractional; - --components.whole; - } - if (scale > 6) - { - // MySQL Timestamp has max scale of 6 - components.fractional /= static_cast(pow(10, scale - 6)); - } - - LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))); - - bool has_microseconds = components.fractional != 0; - bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); - if (has_microseconds) - { - payload_size += 12; - } - else if (has_time) - { - payload_size += 8; - } - else - { - payload_size += 5; - } - break; - } - default: - WriteBufferFromOwnString ostr; - serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); - payload_size += getLengthEncodedStringSize(ostr.str()); - serialized[i] = std::move(ostr.str()); - break; - } - } + // See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row + size_t byte = (i + 2) / 8; + int bit = 1 << ((i + 2) % 8); + null_bitmap[byte] |= bit; + continue; // NULLs are stored in the null bitmap only } - void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const + DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); + TypeIndex type_index = data_type->getTypeId(); + switch (type_index) { - buffer.write(static_cast(0x00)); - buffer.write(null_bitmap.data(), null_bitmap_size); - for (size_t i = 0; i < columns.size(); ++i) - { - ColumnPtr col = columns[i]; - if (col->isNullAt(row_num)) - { - continue; - } - - DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); - TypeIndex type_index = data_type->getTypeId(); - switch (type_index) - { - case TypeIndex::UInt8: { - UInt8 value = assert_cast &>(*col).getData()[row_num]; - if (data_type->getName() == "Bool") - { - buffer.write(static_cast(1)); - } - buffer.write(reinterpret_cast(&value), 1); - break; - } - case TypeIndex::UInt16: { - UInt16 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 2); - break; - } - case TypeIndex::UInt32: { - UInt32 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 4); - break; - } - case TypeIndex::UInt64: { - UInt64 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 8); - break; - } - case TypeIndex::Int8: { - Int8 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 1); - break; - } - case TypeIndex::Int16: { - Int16 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 2); - break; - } - case TypeIndex::Int32: { - Int32 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 4); - break; - } - case TypeIndex::Int64: { - Int64 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 8); - break; - } - case TypeIndex::Float32: { - Float32 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 4); - break; - } - case TypeIndex::Float64: { - Float64 value = assert_cast &>(*col).getData()[row_num]; - buffer.write(reinterpret_cast(&value), 8); - break; - } - case TypeIndex::Date: { - UInt16 value = assert_cast &>(*col).getData()[row_num]; - LocalDate ld = LocalDate(DayNum(value)); - buffer.write(static_cast(4)); // bytes_following - auto year = ld.year(); - auto month = ld.month(); - auto day = ld.day(); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - break; - } - case TypeIndex::Date32: { - Int32 value = assert_cast &>(*col).getData()[row_num]; - LocalDate ld = LocalDate(ExtendedDayNum(value)); - buffer.write(static_cast(4)); // bytes_following - auto year = ld.year(); - auto month = ld.month(); - auto day = ld.day(); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - break; - } - case TypeIndex::DateTime: { - UInt32 value = assert_cast &>(*col).getData()[row_num]; - String timezone = getDateTimeTimezone(*data_type); - LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(timezone)); - int year = ldt.year(); - int month = ldt.month(); - int day = ldt.day(); - int hour = ldt.hour(); - int minute = ldt.minute(); - int second = ldt.second(); - bool has_time = !(hour == 0 && minute == 0 && second == 0); - size_t bytes_following = has_time ? 7 : 4; - buffer.write(reinterpret_cast(&bytes_following), 1); - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - if (has_time) - { - buffer.write(reinterpret_cast(&hour), 1); - buffer.write(reinterpret_cast(&minute), 1); - buffer.write(reinterpret_cast(&second), 1); - } - break; - } - case TypeIndex::DateTime64: { - const auto * date_time_type = typeid_cast(data_type.get()); - UInt32 scale = date_time_type->getScale(); - - static constexpr UInt32 MaxScale = DecimalUtils::max_precision; - scale = scale > MaxScale ? MaxScale : scale; - - const auto dt64 = assert_cast(*col).getData()[row_num]; - auto components = DecimalUtils::split(dt64, scale); - - using T = typename DateTime64::NativeType; - if (dt64.value < 0 && components.fractional) - { - components.fractional - = DecimalUtils::scaleMultiplier(scale) + (components.whole ? T(-1) : T(1)) * components.fractional; - --components.whole; - } - - if (components.fractional != 0) - { - if (scale > 6) - { - // MySQL Timestamp has max scale of 6 - components.fractional /= static_cast(pow(10, scale - 6)); - } - else - { - // fractional == 1 is a different microsecond value depending on the scale - // Scale 1 = 100 000 - // Scale 2 = 010 000 - // Scale 3 = 001 000 - // Scale 4 = 000 100 - // Scale 5 = 000 010 - // Scale 6 = 000 001 - components.fractional *= static_cast(pow(10, 6 - scale)); - } - } - - String timezone = getDateTimeTimezone(*data_type); - LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone)); - auto year = ldt.year(); - auto month = ldt.month(); - auto day = ldt.day(); - auto hour = ldt.hour(); - auto minute = ldt.minute(); - auto second = ldt.second(); - - bool has_time = !(hour == 0 && minute == 0 && second == 0); - bool has_microseconds = components.fractional != 0; - - if (has_microseconds) - { - buffer.write(static_cast(11)); // bytes_following - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - buffer.write(reinterpret_cast(&hour), 1); - buffer.write(reinterpret_cast(&minute), 1); - buffer.write(reinterpret_cast(&second), 1); - buffer.write(reinterpret_cast(&components.fractional), 4); - } - else if (has_time) - { - buffer.write(static_cast(7)); // bytes_following - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - buffer.write(reinterpret_cast(&hour), 1); - buffer.write(reinterpret_cast(&minute), 1); - buffer.write(reinterpret_cast(&second), 1); - } - else - { - buffer.write(static_cast(4)); // bytes_following - buffer.write(reinterpret_cast(&year), 2); - buffer.write(reinterpret_cast(&month), 1); - buffer.write(reinterpret_cast(&day), 1); - } - break; - } - default: - writeLengthEncodedString(serialized[i], buffer); - break; - } + case TypeIndex::Int8: + payload_size += 1; + break; + case TypeIndex::UInt8: + if (data_type->getName() == "Bool") + payload_size += 2; // BIT MySQL type is string in binary + else + payload_size += 1; + break; + case TypeIndex::Int16: + case TypeIndex::UInt16: + payload_size += 2; + break; + case TypeIndex::Int32: + case TypeIndex::UInt32: + case TypeIndex::Float32: + payload_size += 4; + break; + case TypeIndex::Int64: + case TypeIndex::UInt64: + case TypeIndex::Float64: + payload_size += 8; + break; + case TypeIndex::Date: + case TypeIndex::Date32: { + size_t size = 1 // number of bytes following + + 2 // year + + 1 // month + + 1; // day + payload_size += size; + break; } - } + case TypeIndex::DateTime: { + UInt32 value = assert_cast &>(*col).getData()[row_num]; + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(getDateTimeTimezone(*data_type))); - size_t ResultSetRow::getPayloadSize() const - { - return payload_size; - }; + bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); + size_t size = 1 // number of bytes following + + 2 // year + + 1 // month + + 1; // day + payload_size += size; + if (has_time) + { + size_t additional_size = 1 // hour + + 1 // minute + + 1; // second + payload_size += additional_size; + } + break; + } + case TypeIndex::DateTime64: { + auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col); + if (scale > 6) + { + // MySQL Timestamp has max scale of 6 + components.fractional /= static_cast(pow(10, scale - 6)); + } + + LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(getDateTimeTimezone(*data_type))); + + bool has_microseconds = components.fractional != 0; + bool has_time = !(ldt.hour() == 0 && ldt.minute() == 0 && ldt.second() == 0); + size_t size = 1 // number of bytes following + + 2 // year + + 1 // month + + 1; // day + payload_size += size; + if (has_microseconds) + { + size_t additional_size = 1 // hour + + 1 // minute + + 1 // second + + 4; // microsecond; + payload_size += additional_size; + } + else if (has_time) + { + size_t additional_size = 1 // hour + + 1 // minute + + 1; // second + payload_size += additional_size; + } + break; + } + // All other types including all Decimal types are string in binary + default: + WriteBufferFromOwnString ostr; + serializations[i]->serializeText(*columns[i], row_num, ostr, format_settings); + payload_size += getLengthEncodedStringSize(ostr.str()); + serialized[i] = std::move(ostr.str()); + break; + } } } + +size_t ResultSetRow::getPayloadSize() const +{ + return payload_size; +} + +void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const +{ + buffer.write(static_cast(0x00)); + buffer.write(null_bitmap.data(), null_bitmap_size); + for (size_t i = 0; i < columns.size(); ++i) + { + ColumnPtr col = columns[i]; + if (col->isNullAt(row_num)) + continue; + + DataTypePtr data_type = removeLowCardinality(removeNullable((data_types[i]))); + TypeIndex type_index = data_type->getTypeId(); + switch (type_index) + { + case TypeIndex::Int8: { + Int8 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 1); + break; + } + case TypeIndex::UInt8: { + UInt8 value = assert_cast &>(*col).getData()[row_num]; + if (data_type->getName() == "Bool") + buffer.write(static_cast(1)); + buffer.write(reinterpret_cast(&value), 1); + break; + } + case TypeIndex::Int16: { + Int16 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 2); + break; + } + case TypeIndex::UInt16: { + UInt16 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 2); + break; + } + case TypeIndex::Int32: { + Int32 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::UInt32: { + UInt32 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::Float32: { + Float32 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 4); + break; + } + case TypeIndex::Int64: { + Int64 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::UInt64: { + UInt64 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::Float64: { + Float64 value = assert_cast &>(*col).getData()[row_num]; + buffer.write(reinterpret_cast(&value), 8); + break; + } + case TypeIndex::Date: { + UInt16 value = assert_cast &>(*col).getData()[row_num]; + LocalDate ld = LocalDate(DayNum(value)); + buffer.write(static_cast(4)); // bytes_following + UInt16 year = ld.year(); + UInt8 month = ld.month(); + UInt8 day = ld.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + break; + } + case TypeIndex::Date32: { + Int32 value = assert_cast &>(*col).getData()[row_num]; + LocalDate ld = LocalDate(ExtendedDayNum(value)); + buffer.write(static_cast(4)); // bytes_following + UInt16 year = ld.year(); + UInt8 month = ld.month(); + UInt8 day = ld.day(); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + break; + } + case TypeIndex::DateTime: { + UInt32 value = assert_cast &>(*col).getData()[row_num]; + String timezone = getDateTimeTimezone(*data_type); + LocalDateTime ldt = LocalDateTime(value, DateLUT::instance(timezone)); + UInt16 year = ldt.year(); + UInt8 month = ldt.month(); + UInt8 day = ldt.day(); + UInt8 hour = ldt.hour(); + UInt8 minute = ldt.minute(); + UInt8 second = ldt.second(); + bool has_time = !(hour == 0 && minute == 0 && second == 0); + size_t bytes_following = has_time ? 7 : 4; + buffer.write(reinterpret_cast(&bytes_following), 1); + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + if (has_time) + { + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + } + break; + } + case TypeIndex::DateTime64: { + auto [components, scale] = getDateTime64ComponentsWithScale(data_type, col); + if (components.fractional != 0) + { + if (scale > 6) + { + // MySQL Timestamp has max scale of 6 + components.fractional /= static_cast(pow(10, scale - 6)); + } + else + { + // fractional == 1 is a different microsecond value depending on the scale + // Scale 1 = 100000 + // Scale 2 = 010000 + // Scale 3 = 001000 + // Scale 4 = 000100 + // Scale 5 = 000010 + // Scale 6 = 000001 + components.fractional *= static_cast(pow(10, 6 - scale)); + } + } + + String timezone = getDateTimeTimezone(*data_type); + LocalDateTime ldt = LocalDateTime(components.whole, DateLUT::instance(timezone)); + UInt16 year = ldt.year(); + UInt8 month = ldt.month(); + UInt8 day = ldt.day(); + UInt8 hour = ldt.hour(); + UInt8 minute = ldt.minute(); + UInt8 second = ldt.second(); + + bool has_time = !(hour == 0 && minute == 0 && second == 0); + bool has_microseconds = components.fractional != 0; + + if (has_microseconds) + { + buffer.write(static_cast(11)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + buffer.write(reinterpret_cast(&components.fractional), 4); + } + else if (has_time) + { + buffer.write(static_cast(7)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + buffer.write(reinterpret_cast(&hour), 1); + buffer.write(reinterpret_cast(&minute), 1); + buffer.write(reinterpret_cast(&second), 1); + } + else + { + buffer.write(static_cast(4)); // bytes_following + buffer.write(reinterpret_cast(&year), 2); + buffer.write(reinterpret_cast(&month), 1); + buffer.write(reinterpret_cast(&day), 1); + } + break; + } + // All other types including all Decimal types are string in binary + default: + writeLengthEncodedString(serialized[i], buffer); + break; + } + } +} + +ResultSetRow::DateTime64ComponentsWithScale ResultSetRow::getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const +{ + const auto * date_time_type = typeid_cast(data_type.get()); + + static constexpr UInt32 MaxScale = DecimalUtils::max_precision; + UInt32 scale = std::min(MaxScale, date_time_type->getScale()); + + const auto value = assert_cast(*col).getData()[row_num]; + auto components = DecimalUtils::split(value, scale); + + using T = typename DateTime64::NativeType; + if (value.value < 0 && components.fractional) + { + components.fractional = DecimalUtils::scaleMultiplier(scale) + (components.whole ? T(-1) : T(1)) * components.fractional; + --components.whole; + } + + return std::make_pair(components, scale); +} +} +} } diff --git a/src/Core/MySQL/PacketsProtocolBinary.h b/src/Core/MySQL/PacketsProtocolBinary.h index 1a201ce3a4a..d2fcc783825 100644 --- a/src/Core/MySQL/PacketsProtocolBinary.h +++ b/src/Core/MySQL/PacketsProtocolBinary.h @@ -4,42 +4,45 @@ #include #include #include +#include "Common/LocalDateTime.h" +#include "Core/DecimalFunctions.h" #include "DataTypes/IDataType.h" #include "DataTypes/Serializations/ISerialization.h" namespace DB { - namespace MySQLProtocol { +namespace ProtocolBinary +{ +class ResultSetRow : public IMySQLWritePacket +{ + using DateTime64ComponentsWithScale = std::pair, UInt32>; - namespace ProtocolBinary - { - class ResultSetRow : public IMySQLWritePacket - { - private: - TypeIndex getTypeIndex(DataTypePtr data_type, const ColumnPtr & col) const; +private: + DateTime64ComponentsWithScale getDateTime64ComponentsWithScale(DataTypePtr data_type, ColumnPtr col) const; - protected: - int row_num; - const Columns & columns; - const DataTypes & data_types; - const Serializations & serializations; +protected: + int row_num; + const Columns & columns; + const DataTypes & data_types; + const Serializations & serializations; - std::vector serialized = std::vector(columns.size()); + std::vector serialized = std::vector(columns.size()); - size_t null_bitmap_size = (columns.size() + 7 + 2) / 8; - std::vector null_bitmap = std::vector(null_bitmap_size, static_cast(0)); + // See https://dev.mysql.com/doc/dev/mysql-server/8.1.0/page_protocol_binary_resultset.html#sect_protocol_binary_resultset_row + size_t null_bitmap_size = (columns.size() + 7 + 2) / 8; + std::vector null_bitmap = std::vector(null_bitmap_size, static_cast(0)); - size_t payload_size = 0; + size_t payload_size = 0; - size_t getPayloadSize() const override; + size_t getPayloadSize() const override; - void writePayloadImpl(WriteBuffer & buffer) const override; + void writePayloadImpl(WriteBuffer & buffer) const override; - public: - ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_); - }; - } +public: + ResultSetRow(const Serializations & serializations_, const DataTypes & data_types_, const Columns & columns_, int row_num_); +}; +} } } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index ebfff1c2370..b26346a9049 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -198,9 +198,6 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP flags = ColumnDefinitionFlags::BINARY_FLAG; break; case TypeIndex::DateTime: - column_type = ColumnType::MYSQL_TYPE_DATETIME; - flags = ColumnDefinitionFlags::BINARY_FLAG; - break; case TypeIndex::DateTime64: column_type = ColumnType::MYSQL_TYPE_DATETIME; flags = ColumnDefinitionFlags::BINARY_FLAG; @@ -212,6 +209,8 @@ ColumnDefinition getColumnDefinition(const String & column_name, const DataTypeP break; case TypeIndex::Decimal128: { // MySQL Decimal has max 65 precision and 30 scale + // Decimal256 (min scale is 39) is higher than the MySQL supported range and handled in the default case + // See https://dev.mysql.com/doc/refman/8.0/en/precision-math-decimal-characteristics.html const auto & type = assert_cast(*data_type); if (type.getPrecision() > 65 || type.getScale() > 30) { diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 12f24d0ebe6..db35f8925a5 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -295,7 +295,7 @@ struct FormatSettings * https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_execute_response.html * By default, use Text ResultSet. */ - bool use_binary_result_set = false; + bool binary_protocol = false; } mysql_wire; struct diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 6c55ef0e6e9..54039a15807 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -37,7 +37,7 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() SELECT name AS field, type AS type, - if(startsWith(type, 'Nullable'), 'YES', 'NO') AS `null`, + if (startsWith(type, 'Nullable'), 'YES', 'NO') AS `null`, trim(concatWithSeparator(' ', if (is_in_primary_key, 'PRI', ''), if (is_in_sorting_key, 'SOR', ''))) AS key, if (default_kind IN ('ALIAS', 'DEFAULT', 'MATERIALIZED'), default_expression, NULL) AS default, '' AS extra )"; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 2f056daf8e4..87c8c742f8d 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -25,7 +25,7 @@ MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, /// There is no `sequence_id` stored in `settings_.mysql_wire` in this case, so we create a dummy one. sequence_id = settings_.mysql_wire.sequence_id ? settings_.mysql_wire.sequence_id : &dummy_sequence_id; /// Switch between Text (COM_QUERY) and Binary (COM_EXECUTE_STMT) ResultSet - use_binary_result_set = settings_.mysql_wire.use_binary_result_set; + use_binary_result_set = settings_.mysql_wire.binary_protocol; const auto & header = getPort(PortKind::Main).getHeader(); data_types = header.getDataTypes(); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 219e9c15ac2..6297af18817 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -318,7 +318,7 @@ void MySQLHandler::comPing() static bool isFederatedServerSetupSetCommand(const String & query); -void MySQLHandler::comQuery(ReadBuffer & payload, bool use_binary_protocol_result_set) +void MySQLHandler::comQuery(ReadBuffer & payload, bool binary_protocol) { String query = String(payload.position(), payload.buffer().end()); @@ -364,7 +364,7 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool use_binary_protocol_resul format_settings.mysql_wire.client_capabilities = client_capabilities; format_settings.mysql_wire.max_packet_size = max_packet_size; format_settings.mysql_wire.sequence_id = &sequence_id; - format_settings.mysql_wire.use_binary_result_set = use_binary_protocol_result_set; + format_settings.mysql_wire.binary_protocol = binary_protocol; auto set_result_details = [&with_output](const QueryResultDetails & details) { @@ -386,44 +386,13 @@ void MySQLHandler::comQuery(ReadBuffer & payload, bool use_binary_protocol_resul void MySQLHandler::comStmtPrepare(DB::ReadBuffer & payload) { - if (prepared_statements_map.size() > 10000) /// Shouldn't happen in reality as COM_STMT_CLOSE cleans up the elements - { - LOG_ERROR(log, "Too many prepared statements"); - current_prepared_statement_id = 0; - prepared_statements_map.clear(); - packet_endpoint->sendPacket(ERRPacket(), true); - return; - } - - String query; - readStringUntilEOF(query, payload); - - uint32_t statement_id = current_prepared_statement_id; - if (current_prepared_statement_id == std::numeric_limits::max()) - { - current_prepared_statement_id = 0; - } + String statement; + readStringUntilEOF(statement, payload); + auto [is_success, statement_id] = emplacePreparedStatement(std::move(statement)); + if (is_success) + packet_endpoint->sendPacket(PreparedStatementResponseOK(statement_id, 0, 0, 0), true); else - { - current_prepared_statement_id++; - } - - // Key collisions should not happen here, as we remove the elements from the map with COM_STMT_CLOSE, - // and we have quite a big range of available identifiers with 32-bit unsigned integer - if (prepared_statements_map.contains(statement_id)) - { - LOG_ERROR( - log, - "Failed to store a new statement `{}` with id {}; it is already taken by `{}`", - query, - statement_id, - prepared_statements_map.at(statement_id)); packet_endpoint->sendPacket(ERRPacket(), true); - return; - } - - prepared_statements_map.emplace(statement_id, query); - packet_endpoint->sendPacket(PrepareStatementResponseOK(statement_id, 0, 0, 0), true); } void MySQLHandler::comStmtExecute(ReadBuffer & payload) @@ -431,7 +400,7 @@ void MySQLHandler::comStmtExecute(ReadBuffer & payload) uint32_t statement_id; payload.readStrict(reinterpret_cast(&statement_id), 4); - if (!prepared_statements_map.contains(statement_id)) + if (!prepared_statements.contains(statement_id)) { LOG_ERROR(log, "Could not find prepared statement with id {}", statement_id); packet_endpoint->sendPacket(ERRPacket(), true); @@ -439,7 +408,7 @@ void MySQLHandler::comStmtExecute(ReadBuffer & payload) } // Temporary workaround as we work only with queries that do not bind any parameters atm - ReadBufferFromString com_query_payload(prepared_statements_map.at(statement_id)); + ReadBufferFromString com_query_payload(prepared_statements.at(statement_id)); MySQLHandler::comQuery(com_query_payload, true); }; @@ -447,16 +416,49 @@ void MySQLHandler::comStmtClose(ReadBuffer & payload) { uint32_t statement_id; payload.readStrict(reinterpret_cast(&statement_id), 4); - - if (prepared_statements_map.contains(statement_id)) - { - prepared_statements_map.erase(statement_id); - } + erasePreparedStatement(statement_id); // https://dev.mysql.com/doc/dev/mysql-server/latest/page_protocol_com_stmt_close.html // No response packet is sent back to the client. }; +MySQLHandler::EmplacePreparedStatementResult MySQLHandler::emplacePreparedStatement(String statement) +{ + static constexpr size_t MAX_PREPARED_STATEMENTS = 10'000; + std::lock_guard lock(prepared_statements_mutex); + if (prepared_statements.size() > MAX_PREPARED_STATEMENTS) /// Shouldn't happen in reality as COM_STMT_CLOSE cleans up the elements + { + LOG_ERROR(log, "Too many prepared statements"); + current_prepared_statement_id = 0; + prepared_statements.clear(); + return std::make_pair(false, 0); + } + uint32_t statement_id = current_prepared_statement_id++; + + // Key collisions should not happen here, as we remove the elements from the map with COM_STMT_CLOSE, + // and we have quite a big range of available identifiers with 32-bit unsigned integer + if (prepared_statements.contains(statement_id)) + { + LOG_ERROR( + log, + "Failed to store a new statement `{}` with id {}; it is already taken by `{}`", + statement, + statement_id, + prepared_statements.at(statement_id)); + return std::make_pair(false, 0); + } + + prepared_statements.emplace(statement_id, statement); + return std::make_pair(true, statement_id); +}; + +void MySQLHandler::erasePreparedStatement(UInt32 statement_id) +{ + std::lock_guard lock(prepared_statements_mutex); + if (prepared_statements.contains(statement_id)) + prepared_statements.erase(statement_id); +} + void MySQLHandler::authPluginSSL() { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 240057e8ba7..94f86f5e5c6 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -30,6 +30,9 @@ class TCPServer; /// Handler for MySQL wire protocol connections. Allows to connect to ClickHouse using MySQL client. class MySQLHandler : public Poco::Net::TCPServerConnection { + using PreparedStatements = std::unordered_map; /// statement_id -> statement + using EmplacePreparedStatementResult = std::pair; /// is_success? -> statement_id + public: MySQLHandler( IServer & server_, @@ -46,7 +49,7 @@ protected: /// Enables SSL, if client requested. void finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResponse &); - void comQuery(ReadBuffer & payload, bool use_binary_protocol_result_set); + void comQuery(ReadBuffer & payload, bool binary_protocol); void comFieldList(ReadBuffer & payload); @@ -62,6 +65,9 @@ protected: void comStmtClose(ReadBuffer & payload); + EmplacePreparedStatementResult emplacePreparedStatement(String statement); + void erasePreparedStatement(UInt32 statement_id); + virtual void authPluginSSL(); virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet); @@ -82,9 +88,9 @@ protected: using Replacements = std::unordered_map; Replacements replacements; - uint32_t current_prepared_statement_id = 0; - using PreparedStatementsMap = std::unordered_map; - PreparedStatementsMap prepared_statements_map; + std::mutex prepared_statements_mutex; + UInt32 current_prepared_statement_id = 0; + PreparedStatements prepared_statements; std::unique_ptr auth_plugin; std::shared_ptr in; diff --git a/tests/integration/test_mysql_protocol/prepared_statements.reference b/tests/integration/test_mysql_protocol/prepared_statements.reference index 6ea4b218f56..1cea2dc1b1e 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements.reference +++ b/tests/integration/test_mysql_protocol/prepared_statements.reference @@ -60,11 +60,6 @@ d256 type is CHAR, value: 123456789123456789123456789112345678912345678912345678 ### testMiscTypes Row #1 -a type is CHAR, value: [] -u type is CHAR, value: 9a0ccc06-2578-4861-8534-631c9d40f3f7 -t type is CHAR, value: (0,'') -m type is CHAR, value: {} -Row #2 a type is CHAR, value: ['foo','bar'] u type is CHAR, value: 5da5038d-788f-48c6-b510-babb41c538d3 t type is CHAR, value: (42,'qaz') diff --git a/tests/integration/test_mysql_protocol/prepared_statements_test.sql b/tests/integration/test_mysql_protocol/prepared_statements_test.sql index 0302fe5aca7..0cca2b354ba 100644 --- a/tests/integration/test_mysql_protocol/prepared_statements_test.sql +++ b/tests/integration/test_mysql_protocol/prepared_statements_test.sql @@ -1,4 +1,4 @@ -CREATE TABLE ps_simple_data_types +CREATE OR REPLACE TABLE ps_simple_data_types ( i8 Int8, i16 Int16, @@ -28,11 +28,11 @@ VALUES (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303 120, 1234, 51234, 421342, 15324355, 41345135123432, -0.7968956, -0.113259, TRUE); -CREATE TABLE ps_string_types +CREATE OR REPLACE TABLE ps_string_types ( - s String, - sn Nullable(String), - lc LowCardinality(String), + s String, + sn Nullable(String), + lc LowCardinality(String), nlc LowCardinality(Nullable(String)) ) ENGINE MergeTree ORDER BY s; @@ -40,7 +40,7 @@ INSERT INTO ps_string_types VALUES ('foo', 'bar', 'qaz', 'qux'), ('42', NULL, 'test', NULL); -CREATE TABLE ps_decimal_types +CREATE OR REPLACE TABLE ps_decimal_types ( d32 Decimal(9, 2), d64 Decimal(18, 3), @@ -57,7 +57,7 @@ VALUES (1234567.89, 12345678912345678912345678911234567891234567891234567891.12345678911234567891), (-1.55, 6.03, 5, -1224124.23423, -54342.3); -CREATE TABLE ps_misc_types +CREATE OR REPLACE TABLE ps_misc_types ( a Array(String), u UUID, @@ -66,10 +66,9 @@ CREATE TABLE ps_misc_types ) ENGINE MergeTree ORDER BY u; INSERT INTO ps_misc_types -VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255}), - ([], '9a0ccc06-2578-4861-8534-631c9d40f3f7', (0, ''), {}); +VALUES (['foo', 'bar'], '5da5038d-788f-48c6-b510-babb41c538d3', (42, 'qaz'), {'qux': 144, 'text': 255}); -CREATE TABLE ps_date_types +CREATE OR REPLACE TABLE ps_date_types ( d Date, d32 Date32, @@ -89,7 +88,7 @@ VALUES ('2149-06-06', '2178-04-16', '2106-02-07 06:28:15', '1900-01-01 00:00:00.000001', '1900-01-01 00:00:00.000000001');; -CREATE TABLE ps_unusual_datetime64_scales +CREATE OR REPLACE TABLE ps_unusual_datetime64_scales ( dt64_0 DateTime64(0, 'UTC'), dt64_1 DateTime64(1, 'UTC'), @@ -116,7 +115,7 @@ VALUES ('2022-04-13 03:17:45', '2022-04-13 03:17:45.0000001', '2022-04-13 03:17:45.00000001'); -CREATE TABLE ps_datetime_timezones +CREATE OR REPLACE TABLE ps_datetime_timezones ( dt DateTime('Europe/Amsterdam'), dt64_3 DateTime64(3, 'Asia/Shanghai') @@ -124,4 +123,4 @@ CREATE TABLE ps_datetime_timezones INSERT INTO ps_datetime_timezones VALUES ('2022-09-04 20:31:05', '2022-09-04 20:31:05.022'), - ('1970-01-01 00:00:00', '1969-12-31 16:00:00'); + ('1970-01-01 01:00:00', '1969-12-31 16:00:00'); From 0a1495fd0028ad9ad859af9d85f12a96b58e527c Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 12 Sep 2023 18:46:10 +0200 Subject: [PATCH 200/327] Revert information_schema changes (new PR) --- .../InformationSchema/key_column_usage.sql | 38 ------------- .../referential_constraints.sql | 25 --------- .../System/InformationSchema/schemata.sql | 55 ++++++++----------- .../System/InformationSchema/tables.sql | 50 ++++++----------- .../System/attachInformationSchemaTables.cpp | 5 +- 5 files changed, 41 insertions(+), 132 deletions(-) delete mode 100644 src/Storages/System/InformationSchema/key_column_usage.sql delete mode 100644 src/Storages/System/InformationSchema/referential_constraints.sql diff --git a/src/Storages/System/InformationSchema/key_column_usage.sql b/src/Storages/System/InformationSchema/key_column_usage.sql deleted file mode 100644 index 43630b8c8b9..00000000000 --- a/src/Storages/System/InformationSchema/key_column_usage.sql +++ /dev/null @@ -1,38 +0,0 @@ -ATTACH VIEW key_column_usage - ( - `referenced_table_schema` Nullable(String), - `referenced_table_name` Nullable(String), - `referenced_column_name` Nullable(String), - `table_schema` String, - `table_name` String, - `column_name` Nullable(String), - `ordinal_position` UInt32, - `constraint_name` Nullable(String), - `REFERENCED_TABLE_SCHEMA` Nullable(String), - `REFERENCED_TABLE_NAME` Nullable(String), - `REFERENCED_COLUMN_NAME` Nullable(String), - `TABLE_SCHEMA` String, - `TABLE_NAME` String, - `COLUMN_NAME` Nullable(String), - `ORDINAL_POSITION` UInt32, - `CONSTRAINT_NAME` Nullable(String) - ) AS -SELECT NULL AS `referenced_table_schema`, - NULL AS `referenced_table_name`, - NULL AS `referenced_column_name`, - database AS `table_schema`, - table AS `table_name`, - name AS `column_name`, - position AS `ordinal_position`, - 'PRIMARY' AS `constraint_name`, - - `referenced_table_schema` AS `REFERENCED_TABLE_SCHEMA`, - `referenced_table_name` AS `REFERENCED_TABLE_NAME`, - `referenced_column_name` AS `REFERENCED_COLUMN_NAME`, - `table_schema` AS `TABLE_SCHEMA`, - `table_name` AS `TABLE_NAME`, - `column_name` AS `COLUMN_NAME`, - `ordinal_position` AS `ORDINAL_POSITION`, - `constraint_name` AS `CONSTRAINT_NAME` -FROM system.columns -WHERE is_in_primary_key; \ No newline at end of file diff --git a/src/Storages/System/InformationSchema/referential_constraints.sql b/src/Storages/System/InformationSchema/referential_constraints.sql deleted file mode 100644 index 8216b8fff83..00000000000 --- a/src/Storages/System/InformationSchema/referential_constraints.sql +++ /dev/null @@ -1,25 +0,0 @@ -ATTACH VIEW referential_constraints - ( - `constraint_name` Nullable(String), - `constraint_schema` String, - `table_name` String, - `update_rule` String, - `delete_rule` String, - `CONSTRAINT_NAME` Nullable(String), - `CONSTRAINT_SCHEMA` String, - `TABLE_NAME` String, - `UPDATE_RULE` String, - `DELETE_RULE` String - ) AS -SELECT NULL AS `constraint_name`, - '' AS `constraint_schema`, - '' AS `table_name`, - '' AS `update_rule`, - '' AS `delete_rule`, - - NULL AS `CONSTRAINT_NAME`, - '' AS `CONSTRAINT_SCHEMA`, - '' AS `TABLE_NAME`, - '' AS `UPDATE_RULE`, - '' AS `DELETE_RULE` -WHERE false; \ No newline at end of file diff --git a/src/Storages/System/InformationSchema/schemata.sql b/src/Storages/System/InformationSchema/schemata.sql index ca4ad4f7310..9686fcbf4fa 100644 --- a/src/Storages/System/InformationSchema/schemata.sql +++ b/src/Storages/System/InformationSchema/schemata.sql @@ -1,33 +1,26 @@ ATTACH VIEW schemata - ( - `catalog_name` String, - `schema_name` String, - `schema_owner` String, - `default_character_set_catalog` Nullable(String), - `default_character_set_schema` Nullable(String), - `default_character_set_name` Nullable(String), - `sql_path` Nullable(String), - `CATALOG_NAME` String, - `SCHEMA_NAME` String, - `SCHEMA_OWNER` String, - `DEFAULT_CHARACTER_SET_CATALOG` Nullable(String), - `DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String), - `DEFAULT_CHARACTER_SET_NAME` Nullable(String), - `SQL_PATH` Nullable(String) - ) AS -SELECT name AS `catalog_name`, - name AS `schema_name`, - 'default' AS `schema_owner`, - NULL AS `default_character_set_catalog`, - NULL AS `default_character_set_schema`, - NULL AS `default_character_set_name`, - NULL AS `sql_path`, - - catalog_name AS `CATALOG_NAME`, - schema_name AS `SCHEMA_NAME`, - schema_owner AS `SCHEMA_OWNER`, - NULL AS `DEFAULT_CHARACTER_SET_CATALOG`, - NULL AS `DEFAULT_CHARACTER_SET_SCHEMA`, - NULL AS `DEFAULT_CHARACTER_SET_NAME`, - NULL AS `SQL_PATH` +( + `catalog_name` String, + `schema_name` String, + `schema_owner` String, + `default_character_set_catalog` Nullable(String), + `default_character_set_schema` Nullable(String), + `default_character_set_name` Nullable(String), + `sql_path` Nullable(String), + `CATALOG_NAME` String ALIAS catalog_name, + `SCHEMA_NAME` String ALIAS schema_name, + `SCHEMA_OWNER` String ALIAS schema_owner, + `DEFAULT_CHARACTER_SET_CATALOG` Nullable(String) ALIAS default_character_set_catalog, + `DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String) ALIAS default_character_set_schema, + `DEFAULT_CHARACTER_SET_NAME` Nullable(String) ALIAS default_character_set_name, + `SQL_PATH` Nullable(String) ALIAS sql_path +) AS +SELECT + name AS catalog_name, + name AS schema_name, + 'default' AS schema_owner, + NULL AS default_character_set_catalog, + NULL AS default_character_set_schema, + NULL AS default_character_set_name, + NULL AS sql_path FROM system.databases diff --git a/src/Storages/System/InformationSchema/tables.sql b/src/Storages/System/InformationSchema/tables.sql index b3bbfa72517..8eea3713923 100644 --- a/src/Storages/System/InformationSchema/tables.sql +++ b/src/Storages/System/InformationSchema/tables.sql @@ -1,35 +1,17 @@ ATTACH VIEW tables - ( - `table_catalog` String, - `table_schema` String, - `table_name` String, - `table_type` String, - `table_comment` String, - `table_collation` String, - `TABLE_CATALOG` String, - `TABLE_SCHEMA` String, - `TABLE_NAME` String, - `TABLE_TYPE` String, - `TABLE_COMMENT` String, - `TABLE_COLLATION` String - ) AS -SELECT database AS `table_catalog`, - database AS `table_schema`, - name AS `table_name`, - comment AS `table_comment`, - multiIf( - is_temporary, 'LOCAL TEMPORARY', - engine LIKE '%View', 'VIEW', - engine LIKE 'System%', 'SYSTEM VIEW', - has_own_data = 0, 'FOREIGN TABLE', - 'BASE TABLE' - ) AS `table_type`, - 'utf8mb4_0900_ai_ci' AS `table_collation`, - - table_catalog AS `TABLE_CATALOG`, - table_schema AS `TABLE_SCHEMA`, - table_name AS `TABLE_NAME`, - table_comment AS `TABLE_COMMENT`, - table_type AS `TABLE_TYPE`, - table_collation AS `TABLE_COLLATION` -FROM system.tables \ No newline at end of file +( + `table_catalog` String, + `table_schema` String, + `table_name` String, + `table_type` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5), + `TABLE_CATALOG` String ALIAS table_catalog, + `TABLE_SCHEMA` String ALIAS table_schema, + `TABLE_NAME` String ALIAS table_name, + `TABLE_TYPE` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5) ALIAS table_type +) AS +SELECT + database AS table_catalog, + database AS table_schema, + name AS table_name, + multiIf(is_temporary, 4, engine like '%View', 2, engine LIKE 'System%', 5, has_own_data = 0, 3, 1) AS table_type +FROM system.tables diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index d4775bf0d4a..074a648d235 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -12,8 +12,7 @@ INCBIN(resource_schemata_sql, SOURCE_DIR "/src/Storages/System/InformationSchema INCBIN(resource_tables_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/tables.sql"); INCBIN(resource_views_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/views.sql"); INCBIN(resource_columns_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/columns.sql"); -INCBIN(resource_key_column_usage_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/key_column_usage.sql"); -INCBIN(resource_referential_constraints_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/referential_constraints.sql"); + namespace DB { @@ -67,8 +66,6 @@ void attachInformationSchema(ContextMutablePtr context, IDatabase & information_ createInformationSchemaView(context, information_schema_database, "tables", std::string_view(reinterpret_cast(gresource_tables_sqlData), gresource_tables_sqlSize)); createInformationSchemaView(context, information_schema_database, "views", std::string_view(reinterpret_cast(gresource_views_sqlData), gresource_views_sqlSize)); createInformationSchemaView(context, information_schema_database, "columns", std::string_view(reinterpret_cast(gresource_columns_sqlData), gresource_columns_sqlSize)); - createInformationSchemaView(context, information_schema_database, "key_column_usage", std::string_view(reinterpret_cast(gresource_key_column_usage_sqlData), gresource_key_column_usage_sqlSize)); - createInformationSchemaView(context, information_schema_database, "referential_constraints", std::string_view(reinterpret_cast(gresource_referential_constraints_sqlData), gresource_referential_constraints_sqlSize)); } } From cf662712ea9eb9c61c7497175c7609e3c5c9ea58 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 12 Sep 2023 16:45:02 +0000 Subject: [PATCH 201/327] Add new subDate function, apply review suggestions --- .../functions/date-time-functions.md | 45 ++++++++++++++++- .../{addDate.cpp => FunctionsOpDate.cpp} | 48 ++++++++++++------- ...new_functions_must_be_documented.reference | 1 + .../02834_add_date_function.reference | 4 -- .../0_stateless/02834_add_date_function.sql | 9 ---- .../02834_add_sub_date_functions.reference | 17 +++++++ .../02834_add_sub_date_functions.sql | 31 ++++++++++++ 7 files changed, 124 insertions(+), 31 deletions(-) rename src/Functions/{addDate.cpp => FunctionsOpDate.cpp} (65%) delete mode 100644 tests/queries/0_stateless/02834_add_date_function.reference delete mode 100644 tests/queries/0_stateless/02834_add_date_function.sql create mode 100644 tests/queries/0_stateless/02834_add_sub_date_functions.reference create mode 100644 tests/queries/0_stateless/02834_add_sub_date_functions.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3940aef8fbf..dafedfb3adc 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1150,12 +1150,12 @@ addDate(date, interval) **Arguments** -- `date` — The date or date with time to which `value` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +- `date` — The date or date with time to which `interval` is added. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). - `interval` — Interval to add. [Interval](../../sql-reference/data-types/special-data-types/interval.md). **Returned value** -Date or date with time obtained by adding `value`, expressed in `unit`, to `date`. +Date or date with time obtained by adding `interval` to `date`. Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). @@ -1173,6 +1173,47 @@ Result: └──────────────────────────────────────────────────┘ ``` +**See Also** +- [date_add](#date_add) + +## subDate + +Subtracts the time interval or date interval from the provided date or date with time. + +**Syntax** + +``` sql +subDate(date, interval) +``` + +**Arguments** + +- `date` — The date or date with time from which `interval` is subtracted. [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). +- `interval` — Interval to subtract. [Interval](../../sql-reference/data-types/special-data-types/interval.md). + +**Returned value** + +Date or date with time obtained by subtracting `interval` from `date`. + +Type: [Date](../../sql-reference/data-types/date.md) or [DateTime](../../sql-reference/data-types/datetime.md). + +**Example** + +```sql +SELECT subDate(toDate('2018-01-01'), INTERVAL 3 YEAR); +``` + +Result: + +```text +┌─subDate(toDate('2018-01-01'), toIntervalYear(3))─┐ +│ 2015-01-01 │ +└──────────────────────────────────────────────────┘ +``` + +**See Also** +- [date_sub](#date_sub) + ## now Returns the current date and time at the moment of query analysis. The function is a constant expression. diff --git a/src/Functions/addDate.cpp b/src/Functions/FunctionsOpDate.cpp similarity index 65% rename from src/Functions/addDate.cpp rename to src/Functions/FunctionsOpDate.cpp index a34ded341ac..b692fbf97aa 100644 --- a/src/Functions/addDate.cpp +++ b/src/Functions/FunctionsOpDate.cpp @@ -10,15 +10,16 @@ namespace ErrorCodes namespace { - -class FunctionAddDate : public IFunction +template +class FunctionOpDate : public IFunction { public: - static constexpr auto name = "addDate"; + static constexpr auto name = Op::name; - explicit FunctionAddDate(ContextPtr context_) : context(context_) {} + explicit FunctionOpDate(ContextPtr context_) : context(context_) {} - static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + + static FunctionPtr create(ContextPtr context) { return std::make_shared>(context); } String getName() const override { return name; } @@ -37,14 +38,14 @@ public: if (!isInterval(arguments[1].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1st argument of function {}. Should be an interval", - arguments[0].type->getName(), + "Illegal type {} of 2nd argument of function {}. Should be an interval", + arguments[1].type->getName(), getName()); - auto plus = FunctionFactory::instance().get("plus", context); - auto plus_build = plus->build(arguments); + auto op = FunctionFactory::instance().get(Op::internal_name, context); + auto op_build = op->build(arguments); - return plus_build->getResultType(); + return op_build->getResultType(); } bool useDefaultImplementationForConstants() const override { return true; } @@ -62,15 +63,15 @@ public: if (!isInterval(arguments[1].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1st argument of function {}. Should be an interval", - arguments[0].type->getName(), + "Illegal type {} of 2nd argument of function {}. Should be an interval", + arguments[1].type->getName(), getName()); - auto plus = FunctionFactory::instance().get("plus", context); - auto plus_build = plus->build(arguments); + auto op = FunctionFactory::instance().get(Op::internal_name, context); + auto op_build = op->build(arguments); - auto res_type = plus_build->getResultType(); - return plus_build->execute(arguments, res_type, input_rows_count); + auto res_type = op_build->getResultType(); + return op_build->execute(arguments, res_type, input_rows_count); } private: @@ -79,10 +80,25 @@ private: } +struct AddDate +{ + static constexpr auto name = "addDate"; + static constexpr auto internal_name = "plus"; +}; + +struct SubDate +{ + static constexpr auto name = "subDate"; + static constexpr auto internal_name = "minus"; +}; + +using FunctionAddDate = FunctionOpDate; +using FunctionSubDate = FunctionOpDate; REGISTER_FUNCTION(AddInterval) { factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 1a7ffdb1fa1..82f65df4808 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -667,6 +667,7 @@ splitByWhitespace sqrt startsWith subBitmap +subDate substring substringIndex substringIndexUTF8 diff --git a/tests/queries/0_stateless/02834_add_date_function.reference b/tests/queries/0_stateless/02834_add_date_function.reference deleted file mode 100644 index 8912c8c8f63..00000000000 --- a/tests/queries/0_stateless/02834_add_date_function.reference +++ /dev/null @@ -1,4 +0,0 @@ -2022-05-07 00:05:00 -2022-05-07 00:05:00.000 -2022-05-07 00:05:00 -2022-05-07 00:05:00.000 diff --git a/tests/queries/0_stateless/02834_add_date_function.sql b/tests/queries/0_stateless/02834_add_date_function.sql deleted file mode 100644 index 1bf4cb0976e..00000000000 --- a/tests/queries/0_stateless/02834_add_date_function.sql +++ /dev/null @@ -1,9 +0,0 @@ -SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE); -SELECT addDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); -SELECT addDate('2022-05-07'::DateTime, INTERVAL 5 MINUTE); -SELECT addDate('2022-05-07'::DateTime64, INTERVAL 5 MINUTE); - -SELECT addDate('2022-05-07'::Date); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE, 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT addDate('2022-05-07'::Date, 10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT addDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/02834_add_sub_date_functions.reference b/tests/queries/0_stateless/02834_add_sub_date_functions.reference new file mode 100644 index 00000000000..26e4b281526 --- /dev/null +++ b/tests/queries/0_stateless/02834_add_sub_date_functions.reference @@ -0,0 +1,17 @@ +2022-05-07 00:05:00 +2022-05-07 00:05:00.000 +2022-05-07 00:05:00 +2022-05-07 00:05:00.000 +2120-07-26 +2299-12-31 +2027-10-17 11:03:28 +2299-12-31 00:00:00.000 +--- +2022-05-06 23:55:00 +2022-05-06 23:55:00.000 +2022-05-06 23:55:00 +2022-05-06 23:55:00.000 +1970-01-01 +1900-01-01 +1969-12-31 16:00:00 +1900-01-01 00:00:00.000 diff --git a/tests/queries/0_stateless/02834_add_sub_date_functions.sql b/tests/queries/0_stateless/02834_add_sub_date_functions.sql new file mode 100644 index 00000000000..a819c40af85 --- /dev/null +++ b/tests/queries/0_stateless/02834_add_sub_date_functions.sql @@ -0,0 +1,31 @@ +SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::DateTime, INTERVAL 5 MINUTE); +SELECT addDate('2022-05-07'::DateTime64, INTERVAL 5 MINUTE); + +SELECT addDate('2022-05-07'::Date, INTERVAL 400000 YEAR); +SELECT addDate('2022-05-07'::Date32, INTERVAL 400000 YEAR); +SELECT addDate('2022-05-07'::DateTime, INTERVAL 400000 YEAR); +SELECT addDate('2022-05-07'::DateTime64, INTERVAL 400000 YEAR); + +SELECT addDate('2022-05-07'::Date); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT addDate('2022-05-07'::Date, INTERVAL 5 MINUTE, 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT addDate('2022-05-07'::Date, 10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT addDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT '---'; + +SELECT subDate('2022-05-07'::Date, INTERVAL 5 MINUTE); +SELECT subDate('2022-05-07'::Date32, INTERVAL 5 MINUTE); +SELECT subDate('2022-05-07'::DateTime, INTERVAL 5 MINUTE); +SELECT subDate('2022-05-07'::DateTime64, INTERVAL 5 MINUTE); + +SELECT subDate('2022-05-07'::Date, INTERVAL 400000 YEAR); +SELECT subDate('2022-05-07'::Date32, INTERVAL 400000 YEAR); +SELECT subDate('2022-05-07'::DateTime, INTERVAL 400000 YEAR); +SELECT subDate('2022-05-07'::DateTime64, INTERVAL 400000 YEAR); + +SELECT subDate('2022-05-07'::Date); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT subDate('2022-05-07'::Date, INTERVAL 5 MINUTE, 5); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT subDate('2022-05-07'::Date, 10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT subDate('1234', INTERVAL 5 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From f3729da64ba4c1ece9a14318c264a9958ae6805a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Sep 2023 19:55:56 +0200 Subject: [PATCH 202/327] Fix rare case of CHECKSUM_DOESNT_MATCH error --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 43 ++++++++++++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++-- src/Storages/StorageReplicatedMergeTree.h | 8 +++- ...ted_fetch_checksums_doesnt_match.reference | 6 +++ ...eplicated_fetch_checksums_doesnt_match.sql | 42 ++++++++++++++++++ 7 files changed, 106 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.reference create mode 100644 tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3337e136c16..4b6d2ea41ed 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4245,7 +4245,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT } -void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) +bool MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) { DataPartPtr part_to_delete; { @@ -4271,7 +4271,7 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) if (!it->unique()) LOG_WARNING(log, "Cannot immediately remove part {} because someone using it right now " "usage counter {}", part_name_with_state, it->use_count()); - return; + return false; } modifyPartState(it, DataPartState::Deleting); @@ -4296,6 +4296,7 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) removePartsFinally({part_to_delete}); LOG_TRACE(log, "Removed part {}", part_to_delete->name); + return true; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 95d8e74f32c..6f9779bde00 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -671,7 +671,7 @@ public: void outdateUnexpectedPartAndCloneToDetached(const DataPartPtr & part); /// If the part is Obsolete and not used by anybody else, immediately delete it from filesystem and remove from memory. - void tryRemovePartImmediately(DataPartPtr && part); + bool tryRemovePartImmediately(DataPartPtr && part); /// Returns old inactive parts that can be deleted. At the same time removes them from the list of parts but not from the disk. /// If 'force' - don't wait for old_parts_lifetime. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 5b235322394..8e25510b3d9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TABLE_IS_READ_ONLY; extern const int QUERY_WAS_CANCELLED; + extern const int CHECKSUM_DOESNT_MATCH; } template @@ -801,8 +802,46 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Conflict block ids and block number lock should not " "be empty at the same time for async inserts"); - /// Information about the part. - storage.getCommitPartOps(ops, part, block_id_path); + if constexpr (!async_insert) + { + if (!existing_part_name.empty()) + { + LOG_DEBUG(log, "Will check part {} checksums", existing_part_name); + try + { + storage.checkPartChecksumsAndAddCommitOps(zookeeper, part, ops, existing_part_name); + } + catch (const zkutil::KeeperException &) + { + throw; + } + catch (const Exception & ex) + { + if (ex.code() == ErrorCodes::CHECKSUM_DOESNT_MATCH) + { + LOG_INFO( + log, + "Block with ID {} has the same deduplication hash as other part {} on other replica, but checksums (which " + "include metadata files like columns.txt) doesn't match, will not write it locally", + block_id, + existing_part_name); + return; + } + throw; + } + } + else + { + /// Information about the part. + storage.getCommitPartOps(ops, part, block_id_path); + } + } + else + { + chassert(existing_part_name.empty()); + storage.getCommitPartOps(ops, part, block_id_path); + } + /// It's important to create it outside of lock scope because /// otherwise it can lock parts in destructor and deadlock is possible. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2316f9b0225..623d1ea403a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1476,8 +1476,12 @@ void StorageReplicatedMergeTree::syncPinnedPartUUIDs() } } -void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, - const DataPartPtr & part, Coordination::Requests & ops, String part_name, NameSet * absent_replicas_paths) +void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps( + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const DataPartPtr & part, + Coordination::Requests & ops, + String part_name, + NameSet * absent_replicas_paths) { if (part_name.empty()) part_name = part->name; @@ -1596,7 +1600,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd size_t zero_copy_lock_ops_size = ops.size(); /// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`. - checkPartChecksumsAndAddCommitOps(zookeeper, part, ops, part->name, &absent_part_paths_on_replicas); + checkPartChecksumsAndAddCommitOps(std::make_shared(zookeeper), part, ops, part->name, &absent_part_paths_on_replicas); /// Do not commit if the part is obsolete, we have just briefly checked its checksums if (transaction.isEmpty()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1f37416f881..f8ca81c71e3 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -631,8 +631,12 @@ private: * Adds actions to `ops` that add data about the part into ZooKeeper. * Call under lockForShare. */ - void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const DataPartPtr & part, - Coordination::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr); + void checkPartChecksumsAndAddCommitOps( + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const DataPartPtr & part, + Coordination::Requests & ops, + String part_name = "", + NameSet * absent_replicas_paths = nullptr); String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const; diff --git a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.reference b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.reference new file mode 100644 index 00000000000..23dfd352361 --- /dev/null +++ b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.reference @@ -0,0 +1,6 @@ +1 +1 +0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql new file mode 100644 index 00000000000..8ee9d672659 --- /dev/null +++ b/tests/queries/0_stateless/02882_replicated_fetch_checksums_doesnt_match.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS r1; +DROP TABLE IF EXISTS r2; +DROP TABLE IF EXISTS r3; + +CREATE TABLE checksums_r1 (column1 UInt32, column2 String) Engine = ReplicatedMergeTree('/tables/{database}/checksums_table', 'r1') ORDER BY tuple(); + +CREATE TABLE checksums_r2 (column1 UInt32, column2 String) Engine = ReplicatedMergeTree('/tables/{database}/checksums_table', 'r2') ORDER BY tuple(); + +CREATE TABLE checksums_r3 (column1 UInt32, column2 String) Engine = ReplicatedMergeTree('/tables/{database}/checksums_table', 'r3') ORDER BY tuple(); + +SYSTEM STOP REPLICATION QUEUES checksums_r2; +SYSTEM STOP REPLICATION QUEUES checksums_r3; + +ALTER TABLE checksums_r1 MODIFY COLUMN column1 Int32 SETTINGS alter_sync=1; + +INSERT INTO checksums_r1 VALUES (1, 'hello'); + +INSERT INTO checksums_r3 VALUES (1, 'hello'); + +SYSTEM START REPLICATION QUEUES checksums_r2; + +SYSTEM SYNC REPLICA checksums_r2; + +SELECT count() FROM checksums_r1; +SELECT count() FROM checksums_r2; +SELECT count() FROM checksums_r3; + +SYSTEM START REPLICATION QUEUES checksums_r3; +SYSTEM SYNC REPLICA checksums_r3; + +SELECT count() FROM checksums_r1; +SELECT count() FROM checksums_r2; +SELECT count() FROM checksums_r3; + +SYSTEM FLUSH LOGS; + +SELECT * FROM system.text_log WHERE event_time >= now() - 30 and level == 'Error' and message like '%CHECKSUM_DOESNT_MATCH%'and message like '%checksums_r%'; + +DROP TABLE IF EXISTS checksums_r3; +DROP TABLE IF EXISTS checksums_r2; +DROP TABLE IF EXISTS checksums_r1; + From 46fae52c292e3b0d1c43a6a084812889ba6dee94 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Sep 2023 19:57:16 +0200 Subject: [PATCH 203/327] fix zero copy garbage --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../02446_parent_zero_copy_locks.reference | 12 +++++ .../02446_parent_zero_copy_locks.sql | 49 +++++++++++++++++++ 3 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02446_parent_zero_copy_locks.reference create mode 100644 tests/queries/0_stateless/02446_parent_zero_copy_locks.sql diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e6edb99e301..89b6493e81d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9113,7 +9113,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_info, data_format_version, logger); // parent_not_to_remove == std::nullopt means that we were unable to retrieve parts set - if (has_parent || parent_not_to_remove == std::nullopt) + if (has_parent && parent_not_to_remove == std::nullopt) { LOG_TRACE(logger, "Failed to get mutation parent on {} for part {}, refusing to remove blobs", zookeeper_part_replica_node, part_name); return {false, {}}; diff --git a/tests/queries/0_stateless/02446_parent_zero_copy_locks.reference b/tests/queries/0_stateless/02446_parent_zero_copy_locks.reference new file mode 100644 index 00000000000..c3c7c53f625 --- /dev/null +++ b/tests/queries/0_stateless/02446_parent_zero_copy_locks.reference @@ -0,0 +1,12 @@ +0 +1 1 10 1 +1 2 20 2 +1 3 30 3 +1 4 40 4 +1 5 50 5 +2 1 10 1 +2 2 20 2 +2 3 30 3 +2 4 40 4 +2 5 50 5 +3 0 diff --git a/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql new file mode 100644 index 00000000000..86eda526c72 --- /dev/null +++ b/tests/queries/0_stateless/02446_parent_zero_copy_locks.sql @@ -0,0 +1,49 @@ +-- Tags: no-replicated-database, no-fasttest +-- Tag no-replicated-database: different number of replicas + +create table rmt1 (n int, m int, k int) engine=ReplicatedMergeTree('/test/02446/{database}/rmt', '1') order by n + settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, old_parts_lifetime=0, cleanup_delay_period=0, max_cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0; +create table rmt2 (n int, m int, k int) engine=ReplicatedMergeTree('/test/02446/{database}/rmt', '2') order by n + settings storage_policy='s3_cache', allow_remote_fs_zero_copy_replication=1, old_parts_lifetime=0, cleanup_delay_period=0, max_cleanup_delay_period=1, cleanup_delay_period_random_add=1, min_bytes_for_wide_part=0; + +-- FIXME zero-copy locks may remain in ZooKeeper forever if we failed to insert a part. +-- Probably that's why we have to replace repsistent lock with ephemeral sometimes. +-- See also "Replacing persistent lock with ephemeral for path {}. It can happen only in case of local part loss" +-- in StorageReplicatedMergeTree::createZeroCopyLockNode +set insert_keeper_fault_injection_probability=0; + +insert into rmt1 values(1, 1, 1); +insert into rmt2 values(2, 2, 2); + +alter table rmt1 update m = 0 where n=0; +insert into rmt1 values(3, 3, 3); +insert into rmt2 values(4, 4, 4); +select sleepEachRow(0.5) as test_does_not_rely_on_this; + +insert into rmt1 values(5, 5, 5); +alter table rmt2 update m = m * 10 where 1 settings mutations_sync=2; + +system sync replica rmt2; +set optimize_throw_if_noop=1; +optimize table rmt2 final; + +select 1, * from rmt1 order by n; + +system sync replica rmt1; +select 2, * from rmt2 order by n; + +-- a funny way to wait for outdated parts to be removed +select sleep(1), sleepEachRow(0.1) from url('http://localhost:8123/?param_tries={1..10}&query=' || encodeURLComponent( + 'select *, _state from system.parts where database=''' || currentDatabase() || ''' and table like ''rmt%'' and active=0' + ), 'LineAsString', 's String') settings max_threads=1 format Null; + +select *, _state from system.parts where database=currentDatabase() and table like 'rmt%' and active=0; + +-- ensure that old zero copy locks were removed +set allow_unrestricted_reads_from_keeper=1; +select count(), sum(ephemeralOwner) from system.zookeeper where path like '/clickhouse/zero_copy/zero_copy_s3/' || + (select value from system.zookeeper where path='/test/02446/'||currentDatabase()||'/rmt' and name='table_shared_id') || '/%'; + +select * from system.zookeeper where path like '/clickhouse/zero_copy/zero_copy_s3/' || + (select value from system.zookeeper where path='/test/02446/'||currentDatabase()||'/rmt' and name='table_shared_id') || '/%' + and path not like '%/all_0_5_2_6%'; From 8a4654c39eabd8de3e700c32c7e2b3c2fd613507 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 12 Sep 2023 18:06:00 +0000 Subject: [PATCH 204/327] Address review comment --- tests/ci/fast_test_check.py | 7 +++---- tests/ci/report.py | 4 ++++ tests/ci/stress_check.py | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 13dd6dcb5e2..7184a92d36f 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -146,7 +146,7 @@ def main(): run_log_path = logs_path / "run.log" timeout_expired = False - timeout = 90 * 60 + timeout = 1 * 60 with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: @@ -186,10 +186,9 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) if timeout_expired: - test_result_name = "Check timeout expired" - test_results.append(TestResult(test_result_name, "FAIL", timeout)) + test_results.append(TestResult.create_check_timeout_expired(timeout)) state = "failure" - description = format_description(test_result_name) + description = format_description(test_results[-1].name) ch_helper = ClickHouseHelper() s3_path_prefix = os.path.join( diff --git a/tests/ci/report.py b/tests/ci/report.py index eb6e3bc1206..b93128262d9 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -249,6 +249,10 @@ class TestResult: assert file.exists(), file self.log_files.append(file) + @staticmethod + def create_check_timeout_expired(timeout : float) -> "TestResult": + return TestResult("Check timeout expired", "FAIL", timeout) + TestResults = List[TestResult] diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 4e79bdc769d..58453220b6c 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -175,7 +175,7 @@ def run_stress_test(docker_image_name): logging.info("Going to run stress test: %s", run_command) timeout_expired = False - timeout = 60 * 1 + timeout = 60 * 150 with TeePopen(run_command, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: From 85b23e94034b76ecafcdc82807c515b0b23837a2 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 12 Sep 2023 14:06:57 -0400 Subject: [PATCH 205/327] Increase default timeout in tests for keeper-client --- tests/integration/helpers/keeper_utils.py | 24 +++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index a1d20e0392b..19b128abe2c 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -71,7 +71,7 @@ class KeeperClient(object): self.stopped = False - def execute_query(self, query: str, timeout: float = 10.0) -> str: + def execute_query(self, query: str, timeout: float = 60.0) -> str: output = io.BytesIO() self.proc.stdin.write(query.encode() + b"\n") @@ -103,19 +103,19 @@ class KeeperClient(object): data = output.getvalue().strip().decode() return data - def cd(self, path: str, timeout: float = 10.0): + def cd(self, path: str, timeout: float = 60.0): self.execute_query(f"cd {path}", timeout) - def ls(self, path: str, timeout: float = 10.0) -> list[str]: + def ls(self, path: str, timeout: float = 60.0) -> list[str]: return self.execute_query(f"ls {path}", timeout).split(" ") - def create(self, path: str, value: str, timeout: float = 10.0): + def create(self, path: str, value: str, timeout: float = 60.0): self.execute_query(f"create {path} {value}", timeout) - def get(self, path: str, timeout: float = 10.0) -> str: + def get(self, path: str, timeout: float = 60.0) -> str: return self.execute_query(f"get {path}", timeout) - def exists(self, path: str, timeout: float = 10.0) -> bool: + def exists(self, path: str, timeout: float = 60.0) -> bool: return bool(int(self.execute_query(f"exists {path}", timeout))) def stop(self): @@ -123,19 +123,19 @@ class KeeperClient(object): self.stopped = True self.proc.communicate(b"exit\n", timeout=10.0) - def sync(self, path: str, timeout: float = 10.0): + def sync(self, path: str, timeout: float = 60.0): self.execute_query(f"sync {path}", timeout) - def touch(self, path: str, timeout: float = 10.0): + def touch(self, path: str, timeout: float = 60.0): self.execute_query(f"touch {path}", timeout) - def find_big_family(self, path: str, n: int = 10, timeout: float = 10.0) -> str: + def find_big_family(self, path: str, n: int = 10, timeout: float = 60.0) -> str: return self.execute_query(f"find_big_family {path} {n}", timeout) - def find_super_nodes(self, threshold: int, timeout: float = 10.0) -> str: + def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: return self.execute_query(f"find_super_nodes {threshold}", timeout) - def delete_stale_backups(self, timeout: float = 10.0) -> str: + def delete_stale_backups(self, timeout: float = 60.0) -> str: return self.execute_query("delete_stale_backups", timeout) def reconfig( @@ -143,7 +143,7 @@ class KeeperClient(object): joining: tp.Optional[str], leaving: tp.Optional[str], new_members: tp.Optional[str], - timeout: float = 10.0, + timeout: float = 60.0, ) -> str: if bool(joining) + bool(leaving) + bool(new_members) != 1: raise ValueError( From e587eae303cab752f378439541434f6b0e64665e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Sep 2023 18:07:46 +0000 Subject: [PATCH 206/327] Docs: Update information_schema with case-insensitity --- .../operations/system-tables/information_schema.md | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 35fd3a753b5..bdaef8372d6 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -3,12 +3,13 @@ slug: /en/operations/system-tables/information_schema --- # INFORMATION_SCHEMA -`INFORMATION_SCHEMA` (`information_schema`) is a system database that contains views. Using these views, you can get information about the metadata of database objects. These views read data from the columns of the [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md) system tables. - -The structure and composition of system tables may change in different versions of the product, but the support of the `information_schema` makes it possible to change the structure of system tables without changing the method of access to metadata. Metadata requests do not depend on the DBMS used. +`INFORMATION_SCHEMA` (or: `information_schema`) is a system database which provides a (somewhat) standardized, [DBMS-agnostic view](https://en.wikipedia.org/wiki/Information_schema) on metadata of database objects. The views in `INFORMATION_SCHEMA` are generally inferior to normal system tables but tools can use them to obtain basic information in a cross-DBMS manner. The structure and content of views in `INFORMATION_SCHEMA` is supposed to evolves in a backwards-compatible way, i.e. only new functionality is added but existing functionality is not changed or removed. In terms of internal implementation, views in `INFORMATION_SCHEMA` usually map to to normal systme tables like [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md). ``` sql SHOW TABLES FROM INFORMATION_SCHEMA; + +-- or: +SHOW TABLES FROM information_schema; ``` ``` text @@ -17,6 +18,10 @@ SHOW TABLES FROM INFORMATION_SCHEMA; │ SCHEMATA │ │ TABLES │ │ VIEWS │ +│ columns │ +│ schemata │ +│ tables │ +│ views │ └──────────┘ ``` @@ -27,6 +32,8 @@ SHOW TABLES FROM INFORMATION_SCHEMA; - [TABLES](#tables) - [VIEWS](#views) +Case-insensitive equivalent views, e.g. `INFORMATION_SCHEMA.columns` are provided for reasons of compatibility with other databases. + ## COLUMNS {#columns} Contains columns read from the [system.columns](../../operations/system-tables/columns.md) system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. From 4671901313dbb6c247e2312b49fc59af533d2a54 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 12 Sep 2023 18:17:55 +0000 Subject: [PATCH 207/327] Update stress_check.py --- tests/ci/stress_check.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 58453220b6c..4d21791c904 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -195,10 +195,9 @@ def run_stress_test(docker_image_name): ) if timeout_expired: - test_result_name = "Check timeout expired" - test_results.append(TestResult(test_result_name, "FAIL", timeout)) + test_results.append(TestResult.create_check_timeout_expired(timeout)) state = "failure" - description = format_description(test_result_name) + description = format_description(test_results[-1].name) ch_helper = ClickHouseHelper() From 63f71b719d7a50d9493edbc054cb6ed64e8ef4b1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Sep 2023 18:31:34 +0000 Subject: [PATCH 208/327] Automatic style fix --- tests/ci/report.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index b93128262d9..d29f274cd2f 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -250,7 +250,7 @@ class TestResult: self.log_files.append(file) @staticmethod - def create_check_timeout_expired(timeout : float) -> "TestResult": + def create_check_timeout_expired(timeout: float) -> "TestResult": return TestResult("Check timeout expired", "FAIL", timeout) From 8eb12c33a89c05289e41589b793e843b9abafa7c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 12 Sep 2023 20:33:13 +0200 Subject: [PATCH 209/327] Update information_schema.md --- docs/en/operations/system-tables/information_schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index bdaef8372d6..ee03441b9c1 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/information_schema --- # INFORMATION_SCHEMA -`INFORMATION_SCHEMA` (or: `information_schema`) is a system database which provides a (somewhat) standardized, [DBMS-agnostic view](https://en.wikipedia.org/wiki/Information_schema) on metadata of database objects. The views in `INFORMATION_SCHEMA` are generally inferior to normal system tables but tools can use them to obtain basic information in a cross-DBMS manner. The structure and content of views in `INFORMATION_SCHEMA` is supposed to evolves in a backwards-compatible way, i.e. only new functionality is added but existing functionality is not changed or removed. In terms of internal implementation, views in `INFORMATION_SCHEMA` usually map to to normal systme tables like [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md). +`INFORMATION_SCHEMA` (or: `information_schema`) is a system database which provides a (somewhat) standardized, [DBMS-agnostic view](https://en.wikipedia.org/wiki/Information_schema) on metadata of database objects. The views in `INFORMATION_SCHEMA` are generally inferior to normal system tables but tools can use them to obtain basic information in a cross-DBMS manner. The structure and content of views in `INFORMATION_SCHEMA` is supposed to evolves in a backwards-compatible way, i.e. only new functionality is added but existing functionality is not changed or removed. In terms of internal implementation, views in `INFORMATION_SCHEMA` usually map to to normal system tables like [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md). ``` sql SHOW TABLES FROM INFORMATION_SCHEMA; From 031da054f3fa8ab015315f61d0b2339e18108842 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 12 Sep 2023 19:04:59 +0000 Subject: [PATCH 210/327] Return initial timeout --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7184a92d36f..f1bbf356fbb 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -146,7 +146,7 @@ def main(): run_log_path = logs_path / "run.log" timeout_expired = False - timeout = 1 * 60 + timeout = 90 * 60 with TeePopen(run_cmd, run_log_path, timeout=timeout) as process: retcode = process.wait() if process.timeout_exceeded: From 83ac2bf508450cc69093e2569660bace79f0dbb3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 12 Sep 2023 16:14:28 -0300 Subject: [PATCH 211/327] add test --- ...tches_array_inconsistent_offsets.reference | 3 + ...iple_batches_array_inconsistent_offsets.sh | 129 ++++++++++++++++++ ...consistent_offset_multiple_batches.parquet | Bin 0 -> 6953397 bytes 3 files changed, 132 insertions(+) create mode 100644 tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference create mode 100755 tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh create mode 100644 tests/queries/0_stateless/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference new file mode 100644 index 00000000000..ba63f2f7e9c --- /dev/null +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference @@ -0,0 +1,3 @@ +Parquet +e76a749f346078a6a43e0cbd25f0d18a - +400 diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh new file mode 100755 index 00000000000..1d11a2705b3 --- /dev/null +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh @@ -0,0 +1,129 @@ +#!/usr/bin/env bash +# Tags: no-ubsan, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "Parquet" + +# More info on: https://github.com/ClickHouse/ClickHouse/pull/54370 + +# File generated with the below code + +#std::string random_string(size_t length) { +# static const std::string characters = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz"; +# +# std::random_device random_device; +# std::mt19937 generator(random_device()); +# std::uniform_int_distribution<> distribution(0, characters.size() - 1); +# +# std::string random_string; +# random_string.reserve(length); +# +# std::generate_n(std::back_inserter(random_string), length, [&]() { +# return characters[distribution(generator)]; +# }); +# +# return random_string; +#} +# +#static const std::string the_string = random_string(9247124u); +# +#std::shared_ptr CreateIntArray(std::size_t length) { +# arrow::MemoryPool* pool = arrow::default_memory_pool(); +# +# auto int_builder_ptr = std::make_shared(pool); +# auto & int_builder = *int_builder_ptr; +# arrow::ListBuilder list_builder(pool, int_builder_ptr); +# +# for (auto i = 0u; i < length; i++) +# { +# if (i % 10 == 0) +# { +# ARROW_CHECK_OK(list_builder.Append()); +# } +# else +# { +# ARROW_CHECK_OK(int_builder.Append(i)); +# } +# } +# +# std::shared_ptr int_list_array; +# ARROW_CHECK_OK(list_builder.Finish(&int_list_array)); +# return int_list_array; +#} +# +#std::shared_ptr CreateStringArray(std::size_t length) { +# arrow::MemoryPool* pool = arrow::default_memory_pool(); +# +# auto str_builder = std::make_shared(arrow::large_utf8(), pool); +# +# for (auto i = 0u; i < length; i++) +# { +# if (i % 10 == 0) +# { +# ARROW_CHECK_OK(str_builder->AppendNull()); +# } +# else +# { +# ARROW_CHECK_OK(str_builder->Append(the_string)); +# } +# } +# +# std::shared_ptr str_array; +# ARROW_CHECK_OK(str_builder->Finish(&str_array)); +# return str_array; +#} +# +#void run() +#{ +# auto schema = arrow::schema({ +# arrow::field("ints", arrow::list(arrow::int64())), +# arrow::field("strings", arrow::utf8()) +# }); +# +# auto l1_length = 2000; +# auto l2_length = 2000; +# +# std::vector> batches; +# +# auto int_array1 = CreateIntArray(l1_length); +# +# auto int_array2 = CreateIntArray(l1_length); +# +# auto str_array1 = CreateStringArray(l2_length); +# +# auto str_array2 = CreateStringArray(l2_length); +# +# batches.push_back(arrow::RecordBatch::Make(schema, int_array1->length(), {int_array1, str_array1})); +# +# batches.push_back(arrow::RecordBatch::Make(schema, int_array2->length(), {int_array2, str_array2})); +# +# std::shared_ptr outfile; +# PARQUET_ASSIGN_OR_THROW(outfile, arrow::io::FileOutputStream::Open("generated.parquet")); +# +# parquet::WriterProperties::Builder builder; +# builder.compression(arrow::Compression::GZIP); +# builder.dictionary_pagesize_limit(10*1024*1024); // Given your string size is roughly 9.2MB +# builder.data_pagesize(20*1024*1024); // Given your dataset's repetition, let's try with 20MB +# +# std::shared_ptr props = builder.build(); +# +# std::unique_ptr file_writer; +# PARQUET_ASSIGN_OR_THROW(file_writer, parquet::arrow::FileWriter::Open(*schema, ::arrow::default_memory_pool(), outfile, props)); +# +# for (const auto& batch : batches) { +# PARQUET_THROW_NOT_OK(file_writer->WriteRecordBatch(*batch)); +# } +# +# PARQUET_THROW_NOT_OK(file_writer->Close()); +#} + +DATA_FILE=$CUR_DIR/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (ints Array(Int64), strings Nullable(String)) ENGINE = MergeTree ORDER BY tuple()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load SETTINGS max_threads=1" | md5sum +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" +${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet b/tests/queries/0_stateless/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ca0e2cf67627b8438ad9f38449f2321bab1e1a1a GIT binary patch literal 6953397 zcmb5#V{n{*!0z$bwry{0JB@9t@y51oCv9WfYV4#j8{2l$=ifXpPUp;db9Q!ScXr=h zv-6qn-1l#lL{tEn5SRhmpqSt2zhEX1V1l55fDlK+fP6l{IoA5YNTT8(el0cIt^NCa zIM<#yww>ZCRDr$f`I8@jFz^$h?;FC<7DBHQ!XyjAxFfEXT5o6gBxf4-Dkre1V2RxCP)d!j0vVt2}YF(`s)aW zuXBDz<_wJH^jYUj;pU8L=S=eCjC<<+4AmPL)axtK8`{_FRo9#3&>MHvn?_t8kzent zS|2)J@6BBw{Jq}qyFLxhKO)9ILC!yB%HNyJKRCzV-@re7+xb(jbArBe%%*b+zjIW# zv%jNr_|@lUxX-|_PoJgF6t>T(rq3jo&$#>B&-k~2>9@X`x1r0oUcRtanXWs(bSjINQ~3Ol+#$1(-gkhjK0|dzuBCx*%I2##L~@J z)6LZV!ASAJg!sW+=;06qX^#Zy00n8+F>XsbZpSKaTPtoCLSc_W;Q&KnpIzY)VQR;A zYFlq=*N16KlxYWzX`hwp5UzTUuKED4de_T-OVWPF!hT!Bei!LtkLluo=we^+;t-T* z$CzkanP}H3Z%Za`2RCn@C+`r_V2{$^0MlUCbz@6@W5;e|TYqC0TyT$E@Bm$KpH1)( zzI(@}dt0}A*V}JP%5MkTZ=cKW5cy+|`Qw22W4~vR6$G3W1)L2gm=#Big+dI#-p|6& z58x+fK_h2_D`3SdV1qJc#oA?I+64%rvyh?#0Ld)4$!w6SteC27h;yujY%Jt#0Ja7e z`UU`>GYhse8}cnH@ofVLd_78ZElqSCw_GiSTpjy(Ex~vL7=1k&eFI!YJ)TW1l}#Pz zX)V)fogjWKDt-fOc0F!(1Eg*}=2|V|TAcu2EjeEuTSqN^M+3BXJ+^lP@@qXYv?B?$ z6HB-wO}G=clp}_eGr};CVAvUq8i-cnNLS*-Yw1X3>BM>HNOb57iVZ}?c7{y{;%Yil zYC3T&IWjIe32-^0b2-De0P$O#q1}PlkB-cbPD036B*@n+@mDzUw-Aaq7>c(D(>Dan zS7glBtTk73HP^fjS6B|WNS8N6m$#tAH>mknwE5ROhF6q^*BqNygqyeELO19_x9~kT z`2JVa{?}Z8u7v*EmV^@o9KaIHLK6(&{@fO!pOv7W4UC-ib6WtfDGQY;fb(-(P;^#Q zbT-&z*3WGL9CIv;a{z(QZQ&bO@f+BnomoG(1qi{{lEBxod~ORNSC1jrfG}SFxosV5 zMJ-)L9q;G1NT>BgrwyR^^`G0;@#xl4>eg|5ZVS#=kIvTs-%ee(q;_}W`n&Bl2=WaI@-0mK&F8k)?9*2a)7Si;+rrh{;MLqhIoy11do4(OMM``P z_}muK@CMWH7IE|DbK7gSo-6vEYrfBIk^k%(e(-5Q{RP1Y%J?4;%(?#o!DYzk9}vvQ z{sG}@vGG43m~;LEf*+RAKOmUV{sV%!hVefjm<#*^f^)OcKOmS<{{w= zf``J^KOpQe`~!js)Am0g?DPHug4@N`KOpRp{sV$_-u6Er>~s7Bg0JA#KOpSU{{w=h z-}XNs4EG>>+d@wW#K{oBN$JJOA;pRMjvHo-8@G)ch>9DjjFUQplMBE}uEWW8!im4m z41mpyh|CO;$&8`RNGi+7vd&05&d9^fh|J6k)y|Av$qeSnjP}e(f6mB<)Jh1|%8<~C z8PpD<)Q&3B4l~z|!_-Pl)yh=YN?p*(<^p>K_SZjFES41o5G2=@$;@{FPONGkEjvh+wh^vJ{Zh)nkk)%1*A@(kwkjCS`(fAq*l zeoBac%20fYnSKspevYbn4s&>pBYsNEf66p`O5J?Q6?%&9c@F<`&HM+`7@UO!9ProL zU^WObRtzyVgimY9$N{VcEOZ5czt)mf?XrP>T1%S@;8A6vR0aICc8(34jTN1Z z4gS+wYG(k~EerE)-Ct|r8(^aAaiSX_KCNXKuj8k$C8MwVYi&gXlubRBO#{-WwWRoU zfb3e@?7F|!>NX&*)f28YfPY#`-%-csT}$m<_t)B2XAo#03bZrKr?nJPPVB>u48u-; zt)+H`D*@t_I759}%XH`@i0w#;-|IbTuy8)j`S@~f30Jvp?Lk*+UZ*`<{LETTeweasT{63FRz#`um4(0d<&a@gPVT~`DrcV z=Cy#(6}iy$Uu%1Aq5W^L{cn-~+-&wsgFu;)Kw124gfJuNFcYgVW3A6dppdk{kThqP zwES!YW4$3$A1Wi!&qiRSvV=upnKO2FZ(*m8-oQ>1+vk{DSn@zpljHEsr zfy>Pj`N53&!QyWtAZ-aF?HD0#D}6QsP22%a+&*{Q;b$Xkt10ZdOl^JnYy|$PLolX2 zGNyySjZnQMTfOtOdRynS5vc4Bud$f56e;eV0 zSQ{kMOqd%41QP`Fn+9eM4Fl#kZcG$RC`>2<9GGe7|7~0x7|{QH1A+xY^Z)qtzc2q! z51%id>!L+~f;dif(Eb1#xRV#^I-g5c|IV`E5^d)Y-zPj+jZLthY{GAIv#I_KbDU-t z)+uEHV2Ii_9C|h^>BS=6o|6v`v=lvsum>&|vV|UP> z(?o8XSi`cXepLPgxf>1yWPU`GtsGRCi3D>}J`f5-n4b&MWT%)6yvFRdt)_RzQut07 zZhlB!B4a9GB~a5j&@w>FWfDtbtBL;&7iCfn901XsIxUnuDxM$u3|1z+hRZVU;^xht z5C*kqcKfS|@7u00f44>Sx*EJxTuJ(k&BpAv>GL1)!AH$_5gRV{T;BWz1RhvlMQ$Hx zhS=z@ER7A&k&SVt%lGnq`68;to`-sE*H+GzReygxJ)PAGC$l_#t2d=rUO?*jgjmSe(g$g8IWS0@um=B_~=)A#Xro|CSYH&Moh@<&nx(>FU(7v z8vsnK=IgmP=nRPp6PYEcn?c=n2M5$yfx+pAJ1jC|u{Zw|A}KyoLf+31#kf{3t^q1LZ!p+f{&-f_K=~GeJ){w} zka{n8QpBv(NSx$dT-C)hjY#zQOeM|l>V}N3kvYLH&yM!1wsOb@GWwKrXE39(*bz2r z@3UgCgl1CruaA?~_qXGttyJvmU(BE+xLi$Ixk>1f;F0xsBMH1sBzhT)DGPnkH9=#2 zIHh7kVhH+>nxWA^5$vbsokhkmM<=4|pajl&xMTSAJ$UAb%)uoiQ)7l%Cy_C{_GSUp zq3gCe1CY4%rtZoK|Gm9W2CZdPEoYA3PrG~ zgKX$YgT-InqU4G&Z-B;#gF-ZUf@=Eu!1eVPC1Z2pZUFEBobg%(+O9Qtk?U1A_ezDj zVUNnxHqF9O@5BUglRqazzzS^CwKHjooX6P(UwMO(_`(cpA~hCfUmz<-er0#trbI|Aau8(;kEwI`Zrsqv(bf4}WZNb< zD@L(eWi^#$H-WTD1}PR{jnN143fUkEfwzZRUMkc|mxM%__a8)}Z;lN1eNu?$92MsH{Awx~`$R z&|O>0xrWlB7NNQ;E_+F55=={$Lk5i-nMrh%ie8o4C-=-!Pq?^K=M=^~CGPBC7@Pc( zuh+yUG2_MzQ%}-uPmyW={oX7%yk4>Xl%ebAS?*lC4r6IU9qvn{-G4L;?8vA}%7uh` zvm`bQG$-n4>@7m~1{IZ~DvxJDJ+9cUpba0|MxIpVGiosA!kOts>N>vDj}cl2!%0U$ZGA6;WXG>F{yhDtQ+ zis>`U*(!irCT7`ev9ulT_A~9-19+MyxrRc}#6W}D(!CHzyfLVSHLG8hV6b*&KUH7K ze~ULC)pDlH7)?L2WK_Tj-`C1ehGFQ<7t-?MZCs`KboC))vR$anCog*?EnBkI*odB0 zu+45*qXRKor;P-?rK=&?h?-U71hbm+$)F>5OuMagsxvQ~oExcB1Fv!Kv;%3CkbQTI zGWY4FLkd&+=lHF>RiXl&)moRvF3xnc~p?x4#p=oQCQ zec#y{rOWNnY3i#gNduuZrH4YPMt1+6+kENoe%5M(xQMk)WK_Y!Cr#a*>W6p0>tRW9 zg(ir$jtibsAyjiQluBbpRU9hO*YmH<;<}y5PoW8Qi*T+sb>4)A(mXaxzy|v*rva(^ z@}_3~F~l8_Ve&SobFc0_sArmMxm17oV3*Brc`iOHNqRO?maZv_sKhOOi%lM~!-0~+ zrifqPCCdtTgWYyC31jY&+B+BTd3~|e?&7}8KF=u3W(1?2YqauZ zdgq!FDcn`_o@zkG?fDl*Jy_AU#1Xo@bPXK|PA*Lmd{G6PK>!L?gfG-ewx)z3s+ zGl1D_Cq7slLxR!L_WpQe0*WuD?@o{BbAZEJ?9pX05UpW(a%h$2L3!cwYKsc`gmz}Z z1jdDh3ySz)q`^t0__&rO4kSTL;t9yj?cEV{-K-rk&w0MS;A+^B_b12hySOJBPqYUL zdl9b-X~lM`7Kecb&;+LJXc86NAwHXIc&1gdm@8=rOWVY6B*t|^QmTr1rq_RSTnoaTMcll3cu+ML^=X@Jg3if@|tCWQXo z(D*u9<1*`=D@E%ZLoHL&>s)hM=zVb|37m&mJUyYgm1?$ovQ_hSFYwN5b1=0PFTb=d zVIbgZ+?l1luwQPObO&VZCdev1&u$%3Ocp;6s@BAPA$J#;wSiHz?5c z4tShBj*UGGSVAmj|5|9V)&ric>pfbf?%;NwJ=Zi@)wQ=+&eicSdAxeCXaKZX$ku;7 zaItf?VQy>U`h%?h%3QUdeV2d<0z!49Mt}){%K-v~g3AqIZSUd?VQcN|0&Q+vA6%1=jjcS6P3g-W(HE|jKyBZ6E3HCQxQT}`RuU#Q9YY&cKFlQrvL4Cml z`@gQw668t&Po=RWyz#r>J=h@7HoUolA4xjb#3sFZ5f{$wF|z1Rs1(L5&HCH0@o*n#{L)4_71b{eX}lX+ z)DM3axrz(#zo!4mG))dx*D4zAF_pM4u%o;Xckayfe6?XFat>SY^tB@|Yq4}Sx1F$D zgt57PnRy%Aw*RB@-6sr)*+F^#P5DUETLj~6Ve_cri~i--ZyMokNs)(Wt|lNwlhCPe zkwSc6d`j3}YB4RjF3b>tHfCId9jvCk5`tY4n0uL3Yi?;$wD&jR%TjXG6lv1i@&k+Ir0oXzH z8_8<_Dd~{QdcgL5O(BCf5|^Yba%*lUhIPGx7%IO( zR*AC$ECJ)=kwOIxt}4>ebg8-+?*~NF)2L=BwO$c5-OtLVck6mgnAgWz)BMC`b3ML1 zhC`hNXfcD|dMu6X9sAoTO~LeZ8KSDtI!H4ie5^qEVAiTq?fGpkkv_D!cDv@*KOVks zj#aQp_vKlB5omSp}h%ozxEZ^H;fX!CmMYnC(^MZ^yYLi@2xL(1q_@kLbMeH5yC1efK&T z7EFGOzuVYg*NQjo{OXxw*kuD*r~y(LJ*HJ5Y8bbWn4#=$k7oC{7_qLQkw4ZTN^rQVxUj?z=9T1Dh~yccScDp zhVsp&GW)r46e`KO#{9sm5pbVRTJ2%Bx9o$@Pm|kalMn5H)>py$rMh(9Au!zoh)8*P zvz}kUwga|EstU1m&#st*dGfNX5xO)16!Fu?TKzB`{y4R*2-Ee9;@_X214rgz)1 zrbMYQoHKebs0&c-tIXVElU^)63#2LoO>ikpnRSe7(8qEvJoJ}p>tF3MqC_+w?<#Vh z+NOJ>oPdOV9fAwu0ynUxY5DyHIT=lP_cYO)Nc+~UhB{`uTI!V(O-WLQ}onCcF z+1dAWUr4;qzM3FjaOJnro|f;8npv#;TDlmmA&d+nW?8*F`xYwESw!gmtesH__5Mpi zMamw*3YEtbEEONS#i4+u@Nmz|#*QYk{+NNkdOTJq^Yq7XcMcBsu8jSU${dc`9oZ_i zcX93pj=eyT2UJs}oqlzh#)v|?jb!r87V~=Uhfv2z#7TU9s-1TrGGOaGNM7gM!YCqH zpf`ziWfcotBI&o;?4HDvdlYDRv+L#bK64;fPq7%O%f?Go$f^rVAA;5NkwjT)Vnx~M z5+4W!=Pg8|<}K-}$@f}j&A2SE?s*%=eeWU}9Xg#e3wBX(3v`Y)1_x};0q%pZr2eKy zJYIjQ3|>*D7)+wV9_l@{tqyk5QA<#gP|qHA)q`xHGg3L;ze#Gdd+cd;2^Gq=lPkT< zaQeSvxvR#!n$s?mokKUesiOuhcvG7OHn9W_(fSqK>Ps(+L20{NAc^Rz6c? zB+`&eaw$;=@{((qi|QKf{yNZF{?%bC8+%+Q3Xmq4>y%i@9+4gy6!PX#Roz8-K8i&( z$%}T1!LI1LJZ%UCXz^GI5yrCD+PYX;OXPp>&()DV9c`;o9)l;?XBN5I)WJQZzleWi9JDj`CX0C;p!RyIEQSrRM(K>Jl<%dCydDnYt(v7CuYZQv_w6G~+w@9qJ z)Kyv}Aa#6kw6_JGX)Yfp_n(4NG(8|QaiL$!YP8*Bw*!{@CpDBj6H}!j!@L8&&jm#_ z1c$Dpk=|;q1&)u&p2T)P-ltaqt7Er zT-K!ZlrJM|(YGW)3q=tsykXKTOCt8I7aVKW;&l9gW8;#=6xt))!&#LrQdeezbj)wX z!6fCc%_VQ7`DENM+2yTHjb;B1@pdg9jM#wp&~l+JCEukgMLvJJl)T?Q~0h zXke-AwlAx=1yCHRqPa8=LvwPaq|}AK@eFgFCU;p(75SxefE`CqwfEZwRUq21u9m~f zK1h(Atg&^G){6akB2g0lnh}OQ2rZ!?Iq-(gW1B$D*E%^Hlnk!^AXsPF> zmcONOs+=nmU21Ny^@^=|_|sER2Uk_N*!)r)aIHZlO(<2Wl^ze_8Z`CwhA8XEj*z&! z3}L^Rf36w!y4#h8_J*i-f{1=WN#{*1>Ia`4te1hExs~+3Ooi0_ozzK@Ai@}~;jrvH zm{3y&0}W>7Uw`tsG{SVQRato8~*0wYhOLk#AEccYTbj zmog4}a>>c@9sT2A^bla|d5K~wE=~QQE@L8qWE8SLXP36On~~91&Utru$32V1&#V#+ zS_)TF6$I-+P)oPa1F=b^PmC$d=kdDf7fEpwGsy>C3#b)obfdeY1JJL6o*PLeg@s(m zxkd|vsv2VBu@l7z&(1%h#rAcua{ZRJZUTn-NZBXgLukn!(nTPnCKc2(l*I$VXx^@0 z8-Lgc=#6=i6t|p{xt0|HpsDtgQp1Qa6gro^dQ}QP)Dou#2#&BEB%`j-$xS$s5ziO> z9$FW&?bK9S)Ogqz)kEXvF=c_8tjcND<|rKoqSAJ0O@y4^l~zqARn1%8axST~ejutG zB2(?je@T%p!6D77m&dFY5%L+itibp5S;@QThPp)-Gla`mL#o_}JcxJlKk#)>lZN*^O|BUf@4tp9eOvP<`Ys;X}FM8+~;%ZgS zF&mfk;l8;_<&zjwfUjp#p^@U^XaB9WD3p?Z#0(m9!SjpU+$V^GBb54QG2L!gtIN!I zgz}EuYK<}`YtgH77(51iBB-P+XJJv~#hu{9sE$=az6=31T7fTD=O*65Bq~6VM_Z)H zu?yU>*mcgwT@vx<&3gTR7vaJwd)#qh>Nt0;9UqCS%C$;}!}5O~x1gm<{dB<2ePecO zcLL-yZ~uxe1taWtOWdc*#p`3f?Ceq+>NX_$S~8HP!_8(fw@I;RBMw+$NPwFbaY{}W zc)5-H*oNar)r)yps;#b)8!`EY7Nu?9C46}V4JE$(wI&7t0A^L zBtq!0?hHVyYt3wV(&r?)UOyrGY`qrsn00ZrbAjcrS;B`!p=c0D(XtXL{#`k12|6Nk zEo2AfvG!~GDGEz?wk^E+?Zh`)f~$#N)dCC$01UKenUs~>rWyy+bNnEuLb^MVjq}b& zgcxcJ+b>DP5qlxSL;`mh4?XFGs6KiUZ3lBDk0J9{Y@WJewZ^IW$-@PHfnVFADvnhz zgBLh~5VJY9ZUR5!LV&O!RpQD*S|Exs;G$%Pz z7ytuB4%i%)6yu!YIbDxv>39X>R>8=aW$&wTjSK24^wvUZTi{o(smndG&kaKAl@OSW z)6{`u+Z(@Z!LvD>94b&87hY<`o3iBDQ??p39iE!B| z63c9Bn5)A^6Ir3im16&SIt~=NF zB8TOzfHpAr(a9+SYUebnqN3&geY$Xj%V@5(cYPPWw(E6Wv<{+G{Fp$rWY;q=wp;C7NOUtpl&jdLR4PG{!i$R*kTmI$ZTV zWKQ8$$Yz|#1N*@c?7N9s3-%OdwujXbV}ywBYIG6?VkkT>IC9-)ez<*M71zmr0}(kD ze+jwI9ffgoruOMoTKVseLouzJ1?|w?d5IQP6CRyWy$aOq^@*l6=4GF#Fu&++k#&z{ zReiT*J4#M*Nt8KuicgH5O5v<#*b0Ii*73~wp1bpLBc?z#wMf$_S_3P)f5yp3$Jma-e(0oQBbTn!i_Gs zeu`a?lf7D-JkCxI+|@ti)(Ca&TV>AM;x1F`yVv!0#;eIKsE@_>v>it-!1JKCAfSc? zwG4TrU>SSM{k;M$l^by_v7plmGo_Z8*#)<&LU_yqVE2r4CBfecW_ZD%Ps7RE+fM8S znMecZ&TRArx5v42nJuc9kY}JYec#Xjt%#deHKD_WmcOoU zTV6_QLqCj!3{WgEr}3v@8isCO@kv}@O&lG0X4aVl(~b$Q<$VYYr5h&yS?a!XF?uRXBC;$x+7w5HKP&I#3R>fxggU#-+Dt= zAOVQI;oDzg<>m+Pt<`pya5R*vz1-FW8O{+sLnL!lS(U*(10-|P3?hGp zSxf(-mIxC`^wQvrbF5R1?fXkQVFpGL`KldF<7}7aLQ>2d*S9TKC6%?SMl|LX7@0&Y zuP^vb4>5ym7b%De+!P#c9Zjr|+blA0NAvq}h{MBAi0A8Q$A#}TH>p1k7zy1+icula zN^))-h=u|>M)3;rf;x-(ByPzHAjCJmM-4N{GS+;hGx>qI9)Q0md|ytk$|g(Tm#}%! zP%uu~rMUen2cU{& zQB!8`FnBnnV2hk?xxXRZIP}N6BPpI;rG_sV&5+@aImZ4jX5PFcMjbNh*l9S_2{~?6?yj7djwr8{gkU*OOVY`&dp$xn zv>SwYhruvvQzKFDyQ=VBgGzuv#n*F_J{)x_iRCIYv3Bg(-+OlH=oHsx-9lrMHqzDA z^v7tyX#Q#_U-H0l8)YL*qHPn+y4sB39nk1J7DYt4neJ0H{B{_qaY~w|c%?ZGzsHs%bG#7+8cR?-fbnO&MW-E1;;zbA zaKOHmW<=l3WI2!ug9lN}viKXTCeP2-o1RK8fE=8nK5D*F9~Q>dJJ^q@pU5bgY)3*$ zYhbVna}g0%l`_pBV2_BmAS2(0Qi4V^Co3XL2aMA6O+?~woc$hic7ihG_eE85{_fGG z)HOJ=5=6)JiaY6b2{(i2I=3lCaq?vmcf08+?g5=$dUtPr0@5wMt{S2Fi7#`4aM-QH za!(eo>FpIxB9PJ|r~fpu0Bbz5QFF$SRo*6RG$(EioWi~jTc3HF)jiq$p+kH0haMaA zlC_A)`<(SR_VK#vQFEIPwL^IJn|0CKAU+SdL65V=1it}?siV^AS&NZ9C(keXl#yCE z#2-QkiR#~%`nl8_@~C2U)!mn5~mA+Mr=?umkw!iQ(f65S%g~Vq~CZS3)#6D zqK+Dl_*(tMn6v@X4Uq$b?yN$#4cp0}R2*0aK_V>hSY`Y%6bFB1;rBl{}@!u|{&* z5G%l|^{obYr%vag<8-ebd`GK1@93zD!DuoLO6FQld{t2tkh{F3N!pv44$#H($I75a z*x?Y zE~{{{aJy|#ggx%#_v()zU(D~N7RxSC}bupw#hF&F{J&wV&ClZmq}k3^CaFV zeKx0}KR9L7f%bTM(VYuukTJSC9m#yybE<#j^?SG1d z^-}cY7PNl)z%I7zjbo{(4kFnN$QL{uR)lcMD~Z@96MVoDRHZ0|2$24!n1tbqmyqWG z=X#s>U_osxlljoge%+_Z3!&SgH`qfoetk04b~fkd4z59z(b*I~st|zAc`{Zw5!9`3 znAmf=LpVfUY(+Fzs90F9yw&2?>5>_<kyIBqzWYxvqpP44QtucE6w?(QfK@CidEv}2hd8uVQYy4H|r1|d*MM+m%U3;#omq6eGv#!QQ{KmHu z`yl>D?o{hvi!>y3+%jzgJAB+p!|@aa%=hw;{Zt)mbzexb=$7&7Y&L8Ml~U->5$wOk ztgfNxI}&EAjrdH*=r6FL8oOZ2Yo{s-rrw7Zm$Kv8hYXayGe4k^8nq^v5AP=%<+pwF zK-EIe6Fc_msdib=NAADlPLK$C6*f=RDSQ(})uQaSE^0m}u{<9z z0=r~~MBk;2_1qA#xKxO7d2cFtOIxZ!B*>7N;zeb=S1Hn&qPDw0xOh`|nQBx@#K|WD zzD9?*njHs0S!SoAo{XGUa^_}da!MFjQgyi20~W}t$DL&%j%m?8zJQznwpE#!V*X2; zOMBv~6PhPb&1hJ>@UxLZdl&fAn;)-4lNcgX6XRZqyfx*vKRSqE-lnoac}2`66Cmec z&gDiJtenL$so2_vIrq*pO=V*QgxGLh-oN3t+MhhGKRbwO(mPHe+xlNRz)6Ko8}wyP z8IDiJi9UNaOBzR@pVvF+VZzDhinmJtyq%s|yLY5*NrvOh5bilw|MO~fxj1}?vWEyJ zM5P8Kg)86QHe4vsd@nr|-Fyok&Sq-(euC{oPDH!lK$5k6ro5>yy{$b{=h+ZwZ#gN| zuKmsI_YrIfs%Et%S1c7BD{-6O$>S{c;v%yCf;5?3x*!GuoW=#oTlaW9dSNNqECNF#;kRm%VKELm%5oVN$Rk~&HxSs-5Gae!=*@aS`0ex-DAen)GYXPGreKTWu(kU?u7x8r!kZ7 zhCn!#6EN5G9K<3~I(q0)%{vlqa3bJ?HDaC&G?JTuLg zm~q_-ac^K_PUwZ0zf@_vDME$eGo~LK8mR_)p_jzfj?x} zr<=Td?Y(c7wTsb(Oq}@}BWxBnapur^!XfK+0lb%Ap_w3(F2Zm6SbTmZ=z2A~%}@~< zvHpnGcSkpT#fPPp=4#upX=v7FViRV~U^f z%k)4K)0t<|3i)k>>dj|6Z7k%}fhIrZ5bLWC=RD_|*vX;FQ;;Ml9xRksz_R& z?IH?FYXodZLA^f8f(wYbNOu2-w$5GD091dO7ej%7TX++6#IIaDwBj{gK2ZqS$7Q$jiw&a=$qR1O}0Duj9_&N8J>lA1>u5G5XW?4(Lg zvlf}IyFKA#O?(8#F=02lf|RO`ZR{w+6PSL`mp=R$YOFq|cK3$P5hGzoZ=d;u9+$25@_=p$B#UMmRRJ=0_zC3?Gh+li0nOeOvT=FQp2KVZ84v~rF4D$R`1@NK)c_OwmpHq!Dg#d34(op00YzpG5Z{$Rzr<%Oyj zLYd(!N2LrGhAnpBM+#HLD}8{R9V6+xH_!LdXB&8}L%3S6iGiWv-Hw;d95`I~Xcs^i zDoWy1jC0y&u8ljbSH(%uGo*-H$>3THdlE+9aq=nPKWG6{z5v+X#(yi(o$HAsR4|

+_4@hS@&N&|_FG{DSDTp>GV-hnGG+|%idvYZG6vTSlD z^-7Lm-BdfsXaV5=9qEuwyhwWnv#4HVtBgNgzDgWXa>bV?5ZA>NZ^P(MsqrR)%Ak*= z!_vi?IsNC|JeX3+lH28`3Q6wX^O8sc;wc{N>3etZIFv_ZI`aB1M%j4ryA3mveoW_f za0pp|m4@^sae;V$-9PYkdt6YUPRwi~uA1XxYCXkXrT>gAm%12IE7HKc*ql$^KJ1rx zMG-Tr43^hq-Ox#ePREP7DrQJ-kTUH-D|#|^$k5g$z}gWkS`SSfW9CGJu`p8&_1)NV zW~W*JGlB!Eeb0~SRvfRz*cIx&fohB*6ALxCD^TYI~nUOTO z0NEVW09A8kk2%9RZv#s_TZXJEjdyZ|S?DPkJSnqZ{9+Ybxs2$EB-!eACU{*!IX!I{ z0MhmLC*Nr*>souA8B^dF z3g&n*G1tmrj8hY0o%8U@d)ry%uqlv}OPVYEq0OSxN^p0kJTLgwAkZYcEs6Zll14>M zhSRf2ngg_gtXdC`NADmh%rwrLO3$OS>NcJa{sLtO!|InV%B1$^2!E)zMZhXzn8l?z z#PLhte{H8P#6|VmJXQ!gWqkR(&z&%^96So!!kQfkkJ*bDqvGo5hjrXRLga2=aX5IY zz6cAs5)7)-8?iAGYOXf-v%=F!$Q5(@8WvjT@&1OKl#z#nAGyuqQBQVVW*D2|w@6yb zaODa!^HDYq_=#1Ap*bA(_q!&b|ssQngtCD##__DdSa zA%k1^K{=5(yMld5uJP`@DzBYFbqzCw{iDwh#OUE`9IfK94?(8)6T>L~#^uS+d8@{$5ZJfYYR&Glube9Nt)p5c+O_+N>#|i7uBR+dbzj8*xX=qv7Dq@&hebfAt z(>!H3ZGeC6cR4!ro?W(FgqNdzf4!%?O#X~=Gg9e_WB+H3#_i$wWal0c`^$ku(JJE} zB0@Keb|}M9erUs8@k{vh&8%Pg0CslJh(}`iHle?uBLoz)qk&DmL)3+Wgv=ByIm>LP zDyhuu*9O&9iIcqw7V87Jmomdg-l5x0RN zz-Xen=yhZN>u@l-ZU|E9=H{87DkMZQuVj0U$Pno`Ncfpz!l2g>V+nBAPVU=*MOedx zTbJ|w8J`LI%D7w}c3NXn=Jw)2!4FOsO$9|Lkf%<|B+TgC8Dy-smN>c)Ff^X`(${=USE`;Ob}@Au zp+Uu{QJbU+aJgP37#&?<-=m*n^Q$oB zCirFXo31kx{2MmY3lih@eKK!8c0ocZM+TE!n=RnD>0#*D_;MCc(jfOs;=aFw!sn(a z@jkC|J+?;VXmBE7v*@CCGd#OblKvcH#{%1`XVxcjm|(}?md-htTDs^_lB;>Fmfk&z z5l9P2JC)5F=X@KdUE_{uN5!1rw?+D$n!WZ*e=~({_AaW4kl+jZN7newvV$T> ztu221`^LmYoo;2iS`=$RiOktrN?NO5`QydR{P@tH3k#`{HE#0@4?6{0cvLAJ-8!p~ zag$`EZ0+DKc@_8H+RD%UcYbpGQk@)#DhU!Uj*Avetb&Xc0`-)cReo2%kghQBQ!s9R zt!!^Rj6crHrsh>Xc>C2{cwda?$fDuKgDY6?Lq-JRW+%C=pz$TV?zOLsC*Wu#s)r`c zrJ_yh;|S^wl3DRB$Zcyqu*GM)5-xBaMtSi8X>bVZz zzJ|^It#i=|vYV&V+9M$qwo7sRdu=V3)oo!89x?UN6vy^>Z?8U^QO}VQj$e{R&Mu*) z@QOIDvHPMwq}##wwlp#@1zKhal_6c$K4m;2Z2e#ZM95S-8c{RZ(-%4W>1~oj;VG%b zMs>0&3?FH5K*`nUHXB6rRYSIK!#)@=-S1 zFL*}`zw=iv#{#HnZ1GaMWk{+L z71blRGW4JniT=4xJNn5GF+m0N6d{zCd?4n7nj&hO71qt$lQ3J;KsQaDnOxExt7Grs~7`57ea^j6j zP1SgBj`R=O%Yefc0~jvse18K0P#WeV4~QJJMi^Pmwcp#4jbQrRCgt4hvdd7KS=^tn zxj_yArxBi{dQZpz{srIi~`e?wo$NzYW?#3t5+*LDxq@O+jF8bG23=q5vnD-JGz`m625M> zw7MPX1P`$A36i!@<@QpnLGkB?&+Crhn>#68N`9znjV_Xt!3AIm1Xc#L4Rsianbzu#;g#IpQf?JEf34>#Vhom8{oPeOesIB_ zRQ#~Ytq{vd?whPJD@zmN6ODSrwqo~%IY1#-u%hoXi3$-%J&-HPH`fmn5Rsz}!;Z0z z8BZ_(%z4whzRe-K=urF*03AT$ziaAHajyVB(8vR9xO336QQ8$!UaDzv>MBuLRd}n;ezS0#3Dw%gO zG@Y0vwBEe~J;rdiUJ*a5JSHA&Tv?IBb$hn4~h;5o_=jQ1QYQucK)Mv!9EkB~I7dx&P1k95b!T91%s4VTQ5O*6)2Thbs( zwBK|6__D{AGJV9KIwUr40%MvPXk>RBKk3|eFe~NOI&qp+*S*tT>`oHg5?kDL!qI)? zC#yW7cy1&t_rm;RZ>R}u=UaIHVN^&PKW7)jeM^laAu(lbt6glKec1s`Z79A#LhH2& ze4F}ng%p$rr=ezPQl}Cckcs0jde?ca{-Ay<>Ud>EY3MrmZgTzw*SPoLJ6F%ROf ztg3+9I~0^#20#7a64oTRb9Y-T@6qm^aH0+8fBNJ=jv>Q!4=^ZDD( zBFS`f3$re(hr%^Q7M+bQ?^NvW!sbYW=}6}66I(d}h*6M^uiD&TY34{Bgqw{}1j84K zP;@6bhR7ou=J%Y|h2^|gt@;eLrML1JPbtBS4}J!I34QX-J|=Ey*dGY8j)!iB5aJic zv+|2{VGf2v4Y^D^=J-@Z+bNrYJ9Mb*vAab=d=OnB_PINo(DvEMw;^}jY|$#P4=uBr zuRx<%A@NYf5;~-zpWTQlJhDoM?vtCua6u;q%k~C*CS)iPP6Nk;S_%FzmU#D05*XD? z6sXK39;FS*cus!gWX7#bbVUI*jB7lX0G6|k{4@m!!D(VvF!58;0>v8fvSohpb)+fz zlT5?b(g8E_ZWo8GJ9#YoZ9^Ti^K`J!@wXi3->YmFLFJJtb!?V3eQ3u3Kium0Zm;ZD znmf$gQbh2p4O3xS4=qEV2!=nY%Jn(aE&TehO}Yh=ps^iZwJJxFqMtI@5P!0yx_*}T zXQ@ZR-|ap4W6-;OjeMM}4H$1h5Dw|?VW|H81a$nq02uI8+OyqUqpAN_>f&Kk0UvSBTKua7>-Qo!_gn z-)@kECBEbT=x!m<2ieOfvteF(3q z{T@|tblT6fgq0L1KBya#0>A0LXFT_Gd@XA`Zl~xc<8;kR)KOy?P}!5+7pVX~?o*2& zk|ug7br-c0IuON>^Q=yl7F1x*E^II~;tR29Jo|{xNhz@Ti9* zKk@c{GwyK&%;m@3aGl4k;#UUU_AvlIx8X0TFItaGcNrjvJTLv%XkbmKkwUzBzjfxm zOV?QAz6w4Mk?-TCju{_|XBz&$~x@Kr5Ts%%7UP{~H!8BQBPTq)!#E_;(VdnY;8n08=!F>T)`bZwe zu#186EUHx6h$AOnHzuF)d`=Y3f2pEY0HG$b)EE=RSfxT7f{$im9ZozQ8!j&yoAYe> zt1OAzQ$sjqe8tuVtwv-IR~!>N`WV!s0{sTWid9Y>BeUb)$A#ldMI`3#wsxWrhK*Tp z;!_GAclEzJ7vURu8=~I2JQay>(#b`uKYSmaZtU_{4c8uHXX!pPX|Ny}TSmikI#W__ z$ZU?-V&)2;ogYP|<;+2|nVfmH}Y3YaD)U=Q(&9MFMJFiVQ z5g)+=kGaaC5gZsg%>3G@{%j=D*o9JWhl44Z-dMdP^%R2)bI`nrXXquduB2Hu6|lLF zQvU5dNA3PODV3B;9l`b@oPIkpI1+1QGThrWTC9)YM}BZZv)|*$`^Ry+z>oF9WAPhU zsr=TKkBq)ezan*=(LGe=JniRYt%K=T^b9rCd%AWWt$c8>ahg`B-g>?@7ZWPftpzE~ zE+=23(#q@i*UM?)s; zf^BYeUkJG=NYncH;$5L)JZ=*=X^5a^)qRIzVesWfsY=T0WudY@_TF2`(I-sazaknO zw<&+j^(pdIF1IewGylZrXH$sHzMuO9C=vZQK@?{0P4pjIlm`!O;TY(dC3)(y5wLkW zPk%rd0uC9*TG32(r$4-WG5T0dJEMs6xXJu@r!3?db)ZH|V=0X>AS0?YKOCQjNNQIV zaV`t)VajTy^F9qRO?f-{ZOI6~;+rr%>f4hU=?pzAG-%1aUwBUR^o^Dl^S;(=!ukbk zrVI4+He05R`|||kG#sd-lG<4j=og`DBr&&;oda`(V-Fy+9_4s?SHe%SmzbS)os8f; zQ`YXntJ3!aXU){s4Jw04i9Gb7l&}Zwpd`$> zV)h#nRk!?@hQw4yHt>ZEpr7%|pa^}fj|DceHCY5!!L>(OLL`d2dC+?;VF6)*e(WK{JW2EL6Ti0}DE1`FUGCcn!YPt8F$>#SL&XLZf0 z9=mH*0fsqsR0iIX2WfM$e*i_p^tc za%9H}S{F8Qr%Z3(jBv>T@8px#x^?18)>lIHlwO$!QC`IbPrH2|_ zoQJTnCYPJn*y$#22|--XTuRc3dQ5%!EEi_%ch>LrQwjA$boyHa@7mqa9O9mw48%< zGiKsw!6A=RuT^?^wIoCOY@|PYua>)zN|xCyZ11Ss64Cq6Z=1eK_X2gSM$mAXy0Z*$ zAhQHz=rBnSwHa*9MnkRtV^+mx0`o+rC;I40i<=-^R3h?YQK2N6b~_rLWBDalm+m|w z?eqk&%X6s?$0BK<_!bLQ+87~J=$^Uo|6QU(5^g68k7*^bgFbVIr z7gG~-L9Gq0Qc$UEA8F3}qdqyh!#sLn_J>wi4N8T|jo`=fNV%2G`zGn2br8l>g|sX7 zMsJv40N(Hj-;@!n$ngxlL)=HEtoxDd-;)O(Wk$EqBk>5_&&M}HFQA_j&pd&C;x-5~rGykaJ)ubw7lHdJR+WHbG0y`hP2ZXd=EPVQkTRZ@F4w*fs4-l~Cg48KY96Va? zd^!zwHeT&=?gTw=CY`dz>)s9y6j_1`C|a_fWI0)3`SACY&!B0 z>d(BccW;_SFil;iSuM;~gcAH15y{yPaa^w_y_d5|WYnr4P1+r~}>b#n9b#r;)jAW5TV0T5^$gq;XTl-BZhq zIrqFNIwMfYyt6!VHUOP-3qKH}!KS^hkSsgd>H4q&1b)h4cPqh5!=|}`=tqyoGK=zK zgsp>^;NpXvIW~H9AYaP%Wje_&Q)Zcj@r1EUGAXPF>GB_}$8m2bEQ2&UkGiMWSM?!Z zV!m+}nwuutj-tKUU!avqU>nM=t;-%woGnWzCp5$WzVGkRPwOXfg}j87 zN`3uJQ3*-<4-ShynCs@<&9cQSC5?mTgc3GzZ3Gs%sT&@l7 zSbv3;H+sFwvw6k@j3z#E(%CZ^V5od&H@iEc8{@{K7u#iGZTQD=%Eo@qcgMP!QmtAD zjc>zucv9fV37%Q?b#^F34`XqQcOw@0XtHPoc}Lt=__4!7pRX4t-Q*eq{jds&GkQVO z!wX4}n1np%)R}^G68No`JyUe;67KL$wI`RMF}#Vp6Pv=KPtlE zwKR2`AaI;UR>Kvg;nz1m9~69KVgApd-Zk{Ng00pzP7v4)>Glg7hhabt3ltK6*WaFg zDduIwC)?F5rX43fyK1wDC=11EK8kG|nE4m?(k3L3(+Q3<#A6#3<(5;5cePt1A3EzL zI|Z8fb^4ZX{jrd|je!?Psw;5Tj!h8=vP+w>POrYq2UC4v`^_}V31Rni3gCG0E>gv|oS*J1 z0~of#&fW+=o#?R_Wn6}0GHd9GwZ`}NlDvd;twW;_Fv#RE5=R-8y~M@Nqbcu!+ZE~T z*1yS$F{Cqh`N4o4$4)(Adq}L~)>`vP8CByVi&A9+Fx<**#_A@>0Je~ZX)E<4katjT z$4SG=i>co+kPfej5Usw2)VSm30EPu^6LJ}#sUi2%vleU_Bi?e&apZ;ocO>7uK@vDZ z16H|q>6Q84jh`s$c#{`Ps4v#?=&-IhXg?0T%fg+lOo%Niw8NEDXvd`Zl1{$B!O-s2 z<};RijlbsK^T*+mYFVmJG+XO*ew2zqaTLPS$tJ!QkJy=0amFJb4*G%L&XwKM{k_!E zx7Ib_{MX0E&6$`Y7|pCd{p5wK?@Bhyj7RB~dlTA-i1ScEcxQO3#F^tkYwi?C)!^4l zIeQgVpnLk|AK2`Gky`O-uaMcq>#!fQXOF*Ai&Y~fs&tu0CtTQvq*PSV3J zC5JMI&9-@F+?EUaPGc1su6jB(W`V?hU?C~|Y0HvdUxOgDSL|u$D>~24+5U+;Jl*)7 zG)ey{aiRV>SENm*r3#8lbucuPQMZw3iq7lw*^!PE(e%k}={p!cM^j!sc7HbX1tw6}%7PltHl$MOR+!Vtzy{{}$%%=Ug7Q2)(8<)lhcjt)lGb^0EB-fuy3l!D+&g$xnUzhd^RMT`Az54AgQc^v zdPk>RHABj?Z}Jigu|^N0$rbSXE?7XH4eV14NpXAy9NKw%M;+g5lO>8OO4UUC2)Rh`3Bfbur_^__C z=S37;==#ZXC64w!QUgEawKdCuG%`4B3mV$U`dxa_g!Y0FapUqW!Hqyr$5f=IRH5x9 z=~hjh9Ryln0t778$4JL8e9q1@gpXB$^Gi+U}e#A4*DaA&^EIK$XS)ed;w@2fFI zUDrtscaaUg6hBDMoKDt0K;?&b>X2EvqWY5tO^JJ*-7Zuf6O-;UNG5bcppJxE>h@;} zihy?AaW21YJS}Z8h~02^6vKM8m~s;9Igfed`Cy*laRcbtfaRO_%N~;2)xEwi8kG8o z28F>;>nJz{k64%PqY|!+DtXo7=W`0}(pp0uAwx8uw^QLon8+}YUiG)`9D=PjLb2~t zjXBfdr_F4i(BFJOH^NDmW-dvt*j~lQh_cJVMT%@lg@sh;T`hYOsBh)$-Q;~}K>RGB zD1(v!lnjm#*}{Km+!nf9_oUIZ07fBXi~Gi^OU0=&rl+tGK}t^-l=xcto_phqI5V`o zuPQ6!iuFKhOfFs>KTl6uRiFS<3PJbuN_Nc40&@vJwo4!8$?ER6b5{+;q;tENW;=4l z0yCgL4fmc{9yjl|Yu@EeH&3)Fwn}ry;g$ug{_x4foBh+5< zT(fN}rldRnnE8N*;Chgf zE3J=Md23;%v?-o`Qlg*j_=Th^V!^~e96;e}bEn52mZor$S|d}BD6RMv7f+Zh z(t_%_FaXxE+Hb@~5J{84$P`R@)$68B0qJk)2KPc`Xl0|#ZU*Rz<#zSFANHfj?3>ka zTIglNuQf76}1uwH5 zaYtF`VFd4hLfST*Pq1ju+D2ce8e5Ry2+$T32(79CZc%EX;8k*1?+(O!x^sQnE6J^; zFc+_-)pyT6vK?H12Niof(ZrmKbJ;sTFQAqHOZB#?tGvaR)y1Z4TAp?F-U?Xar}l@Dby(nuU`T`OEY&FmPp9WyEEaZ zk`~rscF#dQWa7THMX6vKexQDWQR5qgTT2XgE5{+M2XAY#kh2@koJnd~}RR*k_ z=khVSFwtXt@SrGad zauUOy8N~uTO_ccFIBs~dB_o+J;VrOO@n&df1uhwAc!Tk?6`P zw-;VV6m)s=;AFDhc$52bvY;7Bmis*`y^UUJYVw#Y99u-y_wKU94 zgYPZ~V`#%`b|||ztKjyN6#bTs-^cUpt?G56l}R?<{c0&*H1iAd6Qkcy&~pYM-QM`T zZ0PXZK;j&~YYcJ;8-;T6q`7L;^#i?aLd|LCAecjmzuOYiRBd%89rR$OK%SNOOUSs+ zm7erq1I}zK;p6zxi}Bt^q9CIgn9?U7-8lEt7NcYB*mZnydE4TM^A(qvOTwiy*apLB z&SU9CKuyBLtvAV7EHd$`yp5Wu+u~%hLOnQJQ1$~4S^j+&6eh&#sFWZUCufsZ=OOm> zIR*Xud&0Gz+vkvK`a6=|MSt(jp0y*ql0?y?g2cliSd>jQ3?$g(MDJg<{KVccJ~in6 zC><50pOG>%+D3+-ZFW;JHr>MSq;eWo&JjB}O@kiwi-Ovei-+K8hIQIIQI~E@PoJdE zSmsj0j75r=V9VJ@zLBBVmZl|+7}E$t7#>zJ4j{mMNXI}E_$s%{1AU!0;%(vZ8Q^?8 zw|X_mKo$i(jDPx!O(_{W&UTwGuBVlx$5Y$rV6dRP;1$E`P2^!|wY211HyuGH%ola= z@LRL|OAA=&qf>D~@L>e|3*fYtrJ0aeSOxOeN;HTe;KF@4dcuy`CTqkiEW)Y>D8$ zmGoakLskutIWjORYA3A4gP*gr1C7cdMmBRC9&k+Q44UWMT?>(Kj4|9Rv*Fc>Ge-)9aY>v>(VEC2YZ5E$oB`p>^Pm z3k+Au+f~?iv=v{x5S5tRWYtO{tKzH=As#+r^GhFFfz||!o|G!cxzq(Xk9trHACj<> zqcocu>UJ{PM56DU!V*2tz0V(1U*&|we|V^o@4gdHNj$lCRFm?;67E>D_Ah@+ zAGsyQmIk^Ty~xOoNN&szJTB)k?36~|ul_b1|J6NOC7JOfWef;3Dkg{2=df|%VwS?L zQlgC$zt^#@Wj-Dbk@_EVCNur&m-?~l#&khLeYU7B5m#3BZ#{GfDQCXZ{ftBWojg&V z9)+S%Y8<|7y9viMjrdXCtTmPw>SKKM=6w;8FG?zj>D>>?IBajhRerTD8nf7yyQzno zQ<8OO{1TQ;FK`x(ySpJyrKUB33^e4IBeF0IL&d(>LR(av(o_>slPYtL@MxWxMrmwm ziu&BQxMxjR)ih+kVse_&I<;l9J*`K({W}&9?bEJu~=*;m~e)dcWW5C-Mox zkNWr;xi+>8K9u+`f6U(qx}<1WAwzW3%w@(|ncqCSsW-Pg>}!rLq0kUO{D8ULidlyH zla3cgug;KK!9!b2Ii%!;<~Z}0osO9i*3rR^&eqDpQ~e1c({sY7BQ*RrW6ga5{2RD| z+|Ipjv0Jwt(n9%9q%}EYHjK93I74+~bdzE`lSL+$l zCCK1>e}jT_R=}tlFULgoTXsAt*jX81`zsjrO?BE$n5|0}43scFx}Til>`J@nJD!44 zb{w8gGfb!xe#KQ!urSB|$Y#2O0OO(|1{jWPhx4u|DAH9HjGW!gyD(jw_xtch5JPd= zGVp>vtYgtoZ%@OZG;%RT`se8t@_NRKq(^843m{m7W`u@crj{QAVnQ5 zVJ^m?XPiWpWI3m-5gRvVg^?#xuK(TU#1%m)=r|7?uk(Ma6BB3r(l$-20RJ$>2&>Al zBgXYDFpU`wLD!YEB4MIDGA!S^@uqBu2YRwLVOqpx#dV8Q^?0cRdilY=hv_61D5EfT zR4#VPl&Onr5e|1^EeAJZ@z&@-EpX?rq+{S5!jK-p(CO^#G>YjC^XvC&HG7_$jI>J<;Ebb%{@ z)qdjQm3=CIbfyI)_h~P5_NQtN+CIdrd4TB5vPf@=(k*}KtV_Zi(AnW82dC5pmgCxx zJmVPit3p5?3^tQhl8V2KJ92WzX2YX!e|O8#zf1P;GV+WDul}88>go8oh=HdrbwNu{1U+}Tc{NC zPXds(mtkc^VvA~uw^{T_)2i9HQX`q2Wp0!D;Fp_JEt32Q?>*4CeskAc0qQ&JIoEm1 zaCH9IK_WJ=Tvu}&q5eiT60K)7VN-KaC%*@y>%z81{UD0`4YO+M=^$-S@ndjP(KG9j zyu4Y4MkJiLu7(VaRy^HuUT`iOs-YMpgkjH?MYgXTqrsP)5UL7=93;RyZ?xeQL^NB( zDvqCPz&-GXB5JM8zM)1x2YDu(dqk7;-QZ1I;OVqH8xDy4HGIM=BY`Ln6QN&z(x_CG z1$QZtF6c>Z+_Ao51Q1e@8j&QcQ0O*~L^L&wntzJAdcq?ZL%pV{4Xttp1l-mrpO#-C zB+|DusJ7T6zOABhTt&;5&`B*c4@HV!r5A)QFg$HJbrDe6&|zz@IzH%^+ddo2-XgEl z=I>G1yWoYkaE7tH0o2w5AFx1kty>iY?S(X~ONfdZWGqX&nrU_<&z4F7br$lzOqB|0 zF|_;ZpMAK}j7$ICplpK)SM{9iX!ZgfLdbgvQeY9UAw0Ua61r9P-(zXtF+JocaRl1D zEyl3Ss@`VE>L%=KUpAlcjx$ZFsm}I&4b@;kSqJ5v?U1-EDyS3IZ?wSnU3R)0Ef9j$ z<)HiiNee%)MH+H3D(K-l=SY?&yb9rO$_tSTx8%Bi5ui0yJLM9|avS`-(_j1k8 zvoU|PhmA+n+42T#a_UPpX$l^@Th{pNM~Y6t)2-Wa6B1CB&nEQw7^3gvkoXb2oFUES z37i&NtSn*|c3RyV<+nOYYP&a*fg-=e4Zf7^&z^84Y|Or$5ud@{S>1`d`U>G%I}f+h#+bX6^%BWNgGEP5 zw@Q;amAm(wJ>ztYUec_AdND_4`exM4|Ad$DBK@)8)5S4Kv*)Vyl0aMkL_&E-a>?U9 zm{-;w*~AzHiyu&dE;EmACQeofF|CL4ql7NF?Mkt7KWKP*kKJ<&63v00>o-@jBK(lu zoVC{X#sc8_=01~vs+Ar&>dV0oP==efPWymn%=dnIxO3g3Gd!}DEIb)~Rl}KCF;U!SS3H|* zFR6(w?__mhjsXg^tJB)sJW@4>W`Zo8rL^N8o4(gj@)EF zPn(rSzf~NSxjRAxr}m^aV!zX(fD?|3SZ2+~e!3e_krwgojrdaNg2H4p<*4dV=`_25ePN#DRhWvbw?sH9Ef*9u|nGcKRFmr^6Y z;MSTlMq;8;9#M-Q_xX*5MbADp3U%6cedyTI;exp1q20fNEksRXuF=A(-#Gb6ih{=;q&|)q9wsT-qgGy(yY5TTk z)L*G)%t9110(W(9;jSOc_@cFdAYHyU@FlR;vavty6|0MuneEY|S;W`s#@~M9`%L3C zIgyzn%Xho=8%NiL(rX;-yg&P_kj4vYwh&`T1bFC@*L-YQiv$n`AJaH;r0eY%ePc4Y z1x+~n1zlZ41);`yyZVG6B{k+B1k^Wj*<2%M<+oE2cJdpM-u`9dxoC;hWEXm@(xhkS zVsI5)D1IIA?kwmujkT}oaed*Jos!LR%sbZKr8^(-kn2K~=d!1A zQ8wydh+>KBCeGpMcTdoUU#tt?BP=16^7}W?^5xhDF#ZbIqG8h2N*8J>-1k2lWYK26 zDM)tVx8?fN;HB;@n>)?l)CSTnnXsaJDeEioBkA1DNqnv%*V0h3PXVl;-3DnJ(aS8g zC3-&L=FT$U-B*5gEjlr#celTaqVCOVkSK-_8hc@$UATH9d8-4~RiZAi+pxYUMej7l z4$^R@d`9SUo_g1)WCRz3At-MZF@}Gm%W~Ojx3b^XA(?8qJISZ19E=$G^3pJ<@8>su3MusNP55{^|!$8Vg?VZ{wc5p@#|_jk=hJyy!RL zOIci?Edk5MEn)HKFF(SYisU%O#^A|sOYt9q!ZF0yf=^=j0V@LqbIl-9sZEZrKoaIf zVN{e8>3sg4%S5{Vc&9v|TkziG@OO_>LkSITt%JarrNEEmrqMklvJxT^;|B5dY-S#} zt@=CZd_97Z{!0Phw9-+lyk<0I7_8cWlcSINawk*I*;nh3DY$qTEZi* z^hU9NY2-JA2T?yQ}BA z50;s^vSUZ6sgR$jO-Ie|tY%fu_gzE7l9W*!i}bhcZ&^}OKOD%|5ouf4S}bMKjig)E z%vmj%+7(W1@0hhG^d8JV;aA4BL*C8gk!^8y2GJYnXzoLqWy-0=?d{yCQ_|g&5WF2q)8=mp@LnC%w~6f-c@PP#!jFFzb|bK4sg*@~%^~`+V|13~LHNw+E^| zJ%5Lo=fJ_n2DOx18#vjAIK1o(fM>=0FtXKqH+Lisj zStS=HG!0tV0f#?T^-=s(S%#|f1yuFj-y1cgY#$ILjK@oL z)wj8y6T;fW%{z#6Hy?}%Q&(@7E{1UJq5z;dF-0(?Hr5dh59GjoqMT@6Iu)?oZ#L#T z$=0%@Y&o{*NxX~t#o8^GV&Oyw(_kesu!*m=wWXO-Zm-ro5*E$OY52AJ zBJ&trYbxpPlCAJ|PB?s7b0#tFOCwL+Yyzy<4cbmdd8wR}p#6 zy7q$lCQ7Fj&9NgpW@j5pnIRlX1f{R|)VL(LP_5;M@DVZfUbc;`8I1X#Y zB4Hb|E8hazY?6)@W1KiVW!bDNS3lPz6iU*&2ezcNtuvgroBikJmfXBI!w4MKkwr1T zGQd_wIZSS=jI}^uLzOp64q#4n{ZhQkyy3`|*1}suGIcNIU`2!qX(FgGrSp^+VH&d?pvuF`(Q!A6B7I~?=D0iah?Q4B>ff`?u1}LRDg(*AzAEw0@j!pX_H0TeN9yJqEyIE@ZNsKJcjg&*js^c>@H z94;RnAGYcD@@qWoL^M`=wNf~<0c~8NX7x1yY(Ao@H^95D#(i!lG1oF>paY<+QQJw{ zzLjb}8q*6@`!}GR1Rz4|g%R-dPwou|5vS-#+X0L8G%#&jb5&>o44Y;qFKVQ{jXmwD z`tCNt{jp{2WOlF-2kRwasP<5ai^+W>=Wac#jdw2|`?n@^zKZT{BxkPVCRhMFhZ@-1P*7M>BoC}A9%HaaF@-s*w$8%;kHi^1u4Nhxm)QtrtKQdM> z8w69kyVn>yLE)RN8tyuIVp8|tB%?BMT*jL6lTb1$oR^X8JWZ(%rY&)0#GkzD-7y8ejOq$8G0`^%X|J{%_(QfQC@H!2*;*+VuG&BOk z$o71t7k7h8a+7HEA6h8U8P!Gq@DWCm%vp-IgJ!DEjF?(dHS?_S@ib;EsC5P0B zzxOI6`R)D|Omh%BoYLFsR5U;v)IV_4NFJ4sOid|F+dbR3e!K3s`!1Z7^J^1Xtc68$D`&%khuEwFgHdX@%n5?A z=a;9;)JF(aw-eI)#|HW^H}}Jd<~%; z$WN~W``7-a@N#_L;7cfSIXVjEU#a_e9#%%GbVt{LvJi;<90Sv94{`6C5VC${j&u$% zV04+(Wt;7F;z{^!QG_u^Bb5LWD#OYDeY~njC?%m&P*g?)oTJc_s5WxNEcu3|6u+zG zUe3!4b$x6v^YM9*nsp`( zluqNWoqd)&x9;1q+{wz^F*)=joVRsnOh=jRfnm<)AMW}2qMg>^*M74=!_t-fQFPXS zgrAWDgg~Kxzia~+xuegb3M#8NhV^WN)V6QdIe&CG?MxZQlVR?m_JqatiTvp2&0&7P z8=ZM}4H>2(pQeN8ynoKk!e0BjC9h#2#7fkrC%%pX@MCQfapG_}Jfqtt``DdypM-@< zQb{ouclr7Q^tBmD=Q>E)li%SITAj^wbq@Fz!z)W=0tEZHbe#2|X!+IaNaWiItdx=t zG!UfPb4@&rqy+gHEsVulq{F$3|6^u6MtH#rcIj>_$S2;=)@RqIKn}Yw4vlJQ4NCwW zE=@tdGF6E@5ESM;smOiW7U!QxXT|GM38SrkU=782{gPwC+p8r}{%B6@+&Kw#0X|2J zdFH2nN&VRI0hUOu+7sQyDXXOnKO9icN|b9z8`|JI!3|jSmG*x>o)|(r>Kf*;{B>-t z#ky^0G(sQ9>mYBAFVd9SZaw3d`sJ^T)))>5d?}zI+_uEOktK@+ER8MqQ=C+pL}>!) zI)81W;@m!+t9+E@0S?t&jYlDvD1p&_6i`mnuoQngd=gp~H@vO(^6p1Ktu$?x;JCAD zF?QL!Cn;op!w}mf2TE7KNQEjHMU90%v_9hYfsiu~(-RZZp{#_9p-#NA{ffZ}cJPvj zilY)mj#HSlwAvk&>RfxN&%ff)Am{#;)&wKMxWNkp4Byu?ycmZ2KS$@W94Qb)!C%5N z92@Vwzu~?2eEp)mC{~JC4bxP;${YdReUQAPE+^ZM*8L)CVQd@X^rK4+l->@r*Qu4I zjH9Ts-g6pC^fFLIwc(c~O8&&Y}o^zyafT zl(1W(DJ~j_;G7@7{jFUb7IA&2x+jzV9F7*mVzN=P*^>;;7g{z&;gWEKgWv;rOur7z zsWV|s^>sAAtX`MDoGg)seZAwLFFq_neiXYA4*oluAv*;Lh6fRTFze8P&xZaQglA`} zS{%D;hRdC>(u)BkHO4-_PK8w{PIM0nnz$Mz#u}Rrpm}Vi9pe0&VoH2?5H~7KI zwgKhAO51bg49fV6cUunv`eacel*vB95NrF9ied5VzELhxbBF{B%o_198RS@}o~dgY z*7B8pXmW(q%*Om{eRO^Vw}ok`)M8FCiK+p#`mFZ9hZRd(*mnmk-mp(=fM z6d(E?SIUDm1m&K2l& zuPc%a$aMOt>=WzIQ?#=NV*AF9X$DVT1&JXBVMz@BX}luVYL+L>w88;Jc?!g}1>xf~ z|KiEGh#Om~EBlPXfNm8wgJWf@Kc4iH!^7y|z)jZtYO97@vB4f5>fWRd5z09zi)u*z zR_(zbf)nNzy#Jt8^mHIsB;x-)`qrTXxwIMk{T*y;BK8-;TC4yA+kcDTuL@GXO;>xJ z=^n+gUzp%?g}y8CG0J8)gF}LC(cfolwAdxTdkXNM;Y0Q#?58C?5!y~t%K4u%cnx6 zz`4QDi?eCUM4`JUfZv8+9pfOt!saU9%24X2W zs_7|@9WjBTAnxj^N{}pf`Ju4_eo9!0dEF$!2JQt&cv-KywOxbYdl02|aRQ@H&~$;* z8ki!`XXx)J!ruh z{{Isi!9r_sO%(R zGaUrY1H-QG{2FMj!Q|=ERabO@%PGM3*qQGsPaz1Jr|@xHZ8`FEc-oaBC|jo0z}FQs zt<;SA9aII_QxzP0&4T1DuKr8F?S|*9b=dEBKSE{JD>WgLi5m#!o@{Lw#Rc0fR0?+9 z4U6CG^Em<>5^X~>|iBJgk(J;_S zi4>_R^>@W@#JH$gC!LDQD4Us@Rdh?M{v<2oz%=tTml)lBNuYMF6pMBo^Wj7T`8S6* z15szk&bifg`8}Fxss=bMVvHYYR|V&WhOP*N19G%0-@E;P7{Fz(v9b@ zW|!;2{V@JA#lKPU?m&gyHN?hI!VD@u4CVO-Qo?Q7ldex8`erTQFcCaIP2b3ZjEZf_ z(Z|U?^3k0BsdSI}=GmIuz6Lo7&~vXxZq3Rpw02s?HMy!{w3nDy7ma1HR2{|=gDBoC z97T+hOB1e1^d{EZnwC^k2PbG`UOh=JCcTi+sy^-}V(uQjJ;G3m!E8bD{(DPO(OmYd zRwZEa`jlKyom3v(9r;6>U>BRlQXMf6hKac%eT3S3&+oJ-?|pf$U!$eJJzt0`rZ(J` zZ`(PwdzB5_&+5=lcsqK-DsA+S1NK}$N7;3A#U~kJ-SV0(^;yV-B@T?=5zAE`US;Jo zTVGPqxT)(|#9sX}7!7u1JUu#d?F*6!_?sF+Qa_fX z>6SA;;Q~yh{~5=HQY|J~Kps8$#~#My!wf8is{UbPcm5RP_v1GEsKBG8J)}E$UO#?; z>+KbtA=*77mN#u%Gh5)MsIZ+(&XG|fa8O8u2$Cuuj@iP4X=R|*fnqP z>r%>F3qgEPs;p&kU6?G!OpFxY%vG4fuMBJ(jC3K$W&L+u z9fdUdC)@ax_Q1#a(N#1?azwU`-X zF6O|EG|Cy*UDF=g>Q)q&Y1ihfV%n(ARF1Fjom*{kf;cbl(q zA#DIfK)S!NVTcy8)9(TD1x63?lN+fp*omqwxEfrRup-h#i@_XwVs{RCxzw(K+nu^a z2kL^$rut>#(1i!Z_Of47H8dFqiZ1!5g)X`fXBkhH!#?{-%%^YU*M5Y!>)t|El;S|PR@C5tQWh{PrXwj~A=$$3 zx1M;yyablc5F$SbpwVGMuX+TUD?`rF!2TW=fVU+9i3dUjAKY@fms!$}!7(8_cA{0d zLXlf{pC9-j6=my62I%8nxuV}4W)jW!AGmLm@;4LL-v8~tWlp;lV;Cd% zH^LdYfxWL+X3Na)S|Bjpi$BL+ejuh}w5Xb@Lg%^Uy71y~j74vVg=mWoWnbDy<0)*L zjNifIc^$rUGYG-Qu$su8DG#+xLx}9L7_OmI*6fp%5!I(GS31RFf#TB&N%k%@yeaMF zJ)cddthD3JF!J4S6on;YtPI#ddzF!%=1_J@`8#^*^j*s9n)-S!)=C;_Q?bz(1CZl|iw4HO zp>i^-;1Oe9bl8#kkh115EI}ysdNEm{i-aV^gfb!Vz0_YRr{&y2#UPrx z4SyOoNf)8<2b7r-c_+ioa>lO0gbOJMbw-Kz{Beg$~q$rBf;c z7pl1ED1w!(xp82tRj}q!pr?q^-q9(DlDxtW%YNSDw_GM?!SA)?yhqtcz|X^Zi#42e z!zF-Eka4x&?)j6W{2T40heA}t#xmLroEuO==8cN7>||0=E^#mi=aQlaZ4MOvz=?1s zbIw>zr^q4GUA>Q(PxEu{^E*CaE2I;K=%PWK*?Ys;_L3DkwlI+~G4FugP?(thC+&X^ zl0;0_s7>#h@}~dwVBwFSj@u;V=G_N4tvM$l0>*IRHYVCXg3l^;Lq{Z1Vh|aRWV3_S zy!|*I18Wv_M2JM}IHHL@uXr&!c~pvVy&=QD4-)76b0>#qJVZ81Mp!lJRx$dIr#AOD z0O#|H3hRx_Z%{HahK{^qGvGII->QM8zDhU|yGIL$>Zyo>?51QFg^Mfo48qvzFx}{? zCGaOvlHZ+y#ihCx*fe2tJndh=M_c@m*BS%%r*CWp3!^^KUk9!N%2h7 zHCzS8_cq;o>*jYwe(}~F%c+XquK!>rSh3D|YARGDJhJB4&DK{5JvijiGvgV7k#J`7 zQiurgfn~Vrll|pXjrsN;pJcInJ65G{F0%@oi^uQiblW$=kSG%Ti#ga*O= zecfC6zsy6eZ-=l>5_*!YGZ^qT)}6lGf&202_On`x+lky-?j;Ez4i)na z617Jqd55}y))S^`0ZjAIO?t;$9e>H6Kk7 z!mAT2VO&0Alg)o4x-b~zTj)4yM$31AqVEaCO!zpoWz|MLf_K+KeJ*dq*FjwUsA=jm zo$ZJWcB)|^h4$1I9_vZs-gqA_^DRt+qenQP*e@`=I>DOZkpkvirDWx7qQP;=7JiMV zx~FnzX5DN7R!<|Qv$vZ-ei8`N#lvJ_gdTNhC$dnch7$%@3w#H#sB}Txt@klAudFur z5!8t57UUVKAG|ikQG}a*#PvQym&>*HaK2<+?@G=Ll8OKf^ zU5lu=$22xO;}X^AQl}0TeKCg>hc%d_utx$w00}Jvay(7VDdhmWo{|#t(i4#V-tl)Z zr=DT&y9m%duwd;T%JF$qu1jZw`5|uq8RlgW)?MdAGqBGdSK!8#R>5w2S+*>S%tsOC zuH7LK(naHHR^^}$tlkJ{^&>`7KR=%*NacEme`}S0l7JRdQoo8HTEY%mJ=xr*Pr(Ge z>KLYBh=0$Jy2P&aKV_JS2a@s^M^X*eXv4g4Y>gP! zK`PLn!)iipBKNKo?x#MM6xez?maB_XD)UDH{-7~DNur7TXBa8KajD%1LH01gH#>^_8wB06A&;4;1A6wxmBq8dy7Ni zq?Y$pO=90KCzE46jK4lIV(zIi+gf`5yfi_`?(@_kq6$SSD_cA$J@}G(oalLjnJI+k z>2df*2UVD-{TBkfv3sRY&4dSOkA9#Ku_V$W>4w++_TN#Met%HHO64gsK)bfA+CY*{ zwpU&sd0yZ%KuZ?kqOv1cd4cBm(U)%T8@z>bK~UVy3@&dCklY7D+8cX=I;3Qm%+WYk z!3MWc>sVO#QyJx0dHe0nesY`mW%mLAduk!Wmvg6lA`s2?uFt6uw6oA)5_%)~?U=T^ zLQ{l@b5<=WT(B2+FlK5P+}YL{tf@La6m`6jI5r z^e7@c3bv{*22IJ*Y%RQyEWQ`x7&}|{!>Yfla;9yLeX$GNR~Zp3S><1wT20Aq3({}i zOmI?r()&Dkrue~PPL+V027AxA4$7dHWRWGvmK)X5OM2UKR<8{eepIpP=@jo>pZFn3 zpjvIgydz5z+)+zc?j7y20V(!1pU^&qkKAuOm|xd?X|^tB2#7i@XAlJTt|l}gF~WI&j)rA#evbS)?LxOPw&Z1^KY$s{>qa(_}&FoMDbDZoz&~{p_d~|eoiJ9 z(`M_w2#$)>(G2cTf-2Sqz@X97aeUm^XR4`a&j3g8eWIYd6?Lt0mboyram;VuY7GP0 zCuY=^Mw?xcxP4DF=8aTO8;VQID+L2#yerK0HX@KL780#{Y}9#x4x=3crGb$B&U$yz zuwiXWtyrfYGPgy-)2BDRQ$=oS*kDr}|Mxyax90qs;EA{>)q?$snALUfHE?n7TJTbX zC3w|n_t3L8<^(qSsN|h)O^j9C%2DrQp|_9kv9~lJdh2^EN~q^#!ij$gIuqM3({!Y8 z)Ro?|K&Vm?y$^Q#;>=?xLu@Qc*4H?+2m1GvjvhJ z3?ka^rD{mpAX6ftq*yfXE2Y0zh#)8!A->p#AAK*cWDhEq&S|g?2O{u+B306=w&kKd zo9QeB-g`L6L1vtkAeY~7ah;|Y)>6lmCq`#yVk%|C(96HGx{N#AH&id{@WC1_8j~Cg zP;$kH`zhFqelG-4&C7@ufL<~aHC*P%Ei>DWE+z0PE3SKGdOH)!Tr#E?t8iUypE?|V zUei3u9hzx8SYMcX+^X4;8i(~A4Bo-4M1$lV^;2ox_rJ*7fW4?`Cnap$pD=(+*9}bJ z2A&BE7R|^;qvp+tl;Iw#!lOarOxuEI6n%tbTfSXA^mC7hZW6?7CX@!}&aK$m0o9fH zu2Cixuhk8Pl%z809|LWXrP)fhcZ%^WeU2Ox6BaFb+(g+_Q{>O_ASS-CEsO}?xZERL z`!#z`A?g-e83K)mn%_S>l$ZF1X2qq>(k4nXN!>;desk`L;Q|z?*j9ZPwEUg4VOfex zAzgc#{8M7oT>6>5Z4J5<7l$7I9k9Iw@8t_~&N!7sC}^^oiR@Osz~89q{~WDW$>$3Y z%uU&o5N?Dnp0TV}`|)T~%5DxJ6I~E~KHF`?+=6j1&epi&%#8C0g=q-US=`+xW$ab6 z5h^yv>R%7>2p$Cb;ZPy8xP3~`S`z+kr5e~Hs@%a-jf`C-OR4J<4AKrLE*C2bGg2o zhonx>FGA$eU01Z$BRZoaD*^|MSTN|I5RZ~S{O)L1Ch83(6>0zk_XAu@kuuVW`Yw&A zPUTwtG76*=vYC+fo}gHzQ8?DA(wTlSCFxoAtrlW{t=o6r)=R2j+yop+=cYukQxyL0 z0KN~0#z~qW41%c~sMh^6N;RrXSZTDTl~#N!-M1Nqi(4hlV^2!cK7{QIND7WMDO-cO zDgSn*sBt9pxjnk4^sM6kHY@cfVq4t7j$<~LK)?mB3=eaUok?}pT4{bmrMriAj{xvR zFH_^wnxf_N_O~b4^V^UgCx#lG#fF|@T;nY@CsATp`e?m8r)d53keFbPTD>&#to9fu zQjZ7fIJUK)tFnE&tr+`RyBU=-YRnkK%q6uHQ$HOiL?v*v4VldRx3f423G76s%5~Ec z-yDxnG46<2sPe5K$!s`RuAjr6bpAlUvQTx|!lNGSb38)wlts6z-}=)!479Y7U67y& z_gQhu&#`J#3|(jWs+LW;g(vd)X|1;~WU+Ft=h(v^Vwa+!T(+U)3lOH5Fj1r-UG6)~ zTQYpU>{?y)&8)s#NKE1prbn6QqWjA^q!BRK)kAWm;REXAF>6n4AL^CM30?G;dAT+!!Etz@cGwWl)k7)&L>+=qy$}tU3(m9S?1YK!p?0&g zaW~u_J@0jIv6(Tw2^z`8jo+3HuvT$H`7v)@!fUf-*kh>I2!CU?5JUZ%M;$7KXgX4= zfB37;IK*int#I5*q2>yv;!q`cy-qCmsP6I0FGw5l2}f*3uQK`TT<$;v1r}O$2nYlX zd70sEKK2z2_dV1to@Z&gi?15L%L-{GOimkULRO zWQUSlXGc2SB2~2hh zHe|8h5H5w7FHeuFYm6io<{v?lwoVI^pH)0Tp>G`*ri&YQc{a{m3XoYvMjs00H1lqs zi1{k^jce8maWgGahkRt#Um|-+_IK&Z{@dYP8VoM-VC|3 zrj+NFd9@@$0&#A@6*Z!(R2Ok|6;_axc)u)TQ9eTuuG-L}I}4C!L;H0BTF8RwIKf$& zsh$engCufSvF16}3b)h}o8z#2@B@oC9m~wyaj43&@nR%?r6i?oP`6dV zJN+`^*SAwaT2vYET_0VX%H0d?EsGp%$Xu(y^7Rd>YX}4kli_{s;Lk%!zmW4qmfjDo zPo3?Ag!>uC+NypLqoi%oV|<)fo99l1)JIFmY9X7*CJsG(?{%YmuQX+PpVaIrf}C3r zso~v_my%7nut9p7a)hak?ZwHb+LC$<+nFKaV)DV`ok z3|m1$HKM{TGIQVZ49c<14>nBDv5k`kS{lBw;gZtlnAH)fh4l!^q*f#5Nib!hZ7yTBrevL&G%}v544QwhF^$f`XK2W zsDYMmtY8sY9x|UU`FDAuWY({_qpUz@w2jSphoP^`M|v)~@wO4UrNudGwSxC({7tmN zM5d?Oi8R*INvpc>OC#TW2*09KZmFUuDa`De$dA=TNgn6yU!fVMp}~eWi`Z%CA32Nn z?SF6|6i_uH89Rg7chLM9r10PngXSYPxbei;50VrhG!H)=@a_VKp`1#vyt9|q5^A(7 zlXJGyrDxn4eA6*jX@_TLw_DD{V(F|Il~WBJjtNv_@y4=FSr5AYHob?)P23dRBUB|x zXx*;+uxIA#IRib-TsM)UIK3KmwKQ-tg*xv1Ip170wv;TEnylP+O5Sud!OvZw*0z>Z z>o^dn-cQ?*EFq~EE*fdhM`L<`4#Hm1P%EJx6@i#|16V>)tt9NN5dkNg@s)!nncNfGEih?2(r83OYgaI@EFM8H>cZ9B}tk;rSJg z_OOuW9?BQFLNtxjxz!P?x#fMB7C&P`!{VCk7T$@$gKM4E&P9=p$eEXQat3i^rB6}E zdAcjb033U>FWtT05OXoTJRvJLQtU!xdL_x=kUmAlffMvSBY$zv#A4Ck?SPVWYCKnAs3>D*#z00 z<`|K?^u!T)k+4sd<)u<8t+I0_lGAq@B$kTjIrNIC2{yxhPa!gvI6s_(_7}r=Z6D} zk3gko>ZCXt7iX|F=NbB`!=YdzT8|P8_d8!nwD7b9M?{)>P8_;xe}jbhpv$)={t?lBVvAXiceI<_9y#AYoyT(TKBAN= zua~%^{Mw7|R1n!fs_x2ud^=A%+$**=KB5Qd;eHBA;X%ExSDMctc;!nmrI=Nfw;(X) z3WSDaF-5(22A1FRLdn!sS6+SQAnf`q7;>z!Gd2|hCc*o;A^ZWN7swTaxq}D=p*537 zXHzDgg+46*?X>|A)%$%}B00dO4Q>3gKYyYu{$fa$J_}sD#F@%@r`mhZ3 znSVjO(SYvrl*dr%LM1;pLL3+cjW|()JAq{UNSc(*>XROIRH!sZ2=j+1^bw=yrHEoQ zon+hxIr*)T!jL#4#;9+2oa#iM=Knp&wCLY`i0_KFObY9VTb1k2SQEqzuaGa2^7B(R za4X#DjO<*TvWNU-z8*PhJG2SE#1cwWGibdhuFW}@pCqxYyDZ0`|u18e$BKj?d zA&O(C*G0jTrii&Di9;6S3x_!^#YLE*`wIpVLUFtQ+k1D7Q%jFpc<|#q8_4m^!|ESlwaH(DmEN#VkhL!h2g`<& zlGd#mHdfmnXXx_VLMhs-sr5+IcCo0mA|~1yEK|T6l(iMUmss^+=1z@Q!(zsW;$bOu zIOD+D8QWx>V`Z@16pE7lsAcc5UrLnh-kK8X?(?73u{B3fj zcINFl8H4F-Y~PB|tJ+?9&(xsHcT1Ymjec_kHT0Z$u4MLTYc=_Gg4T~`TmyPl+JXiW z7hfHq79{ah!`KhA!bajPlg1xk$U5QbYY-8HxZB~}*sg|JC^UCl6#uxGlC8hsqCN*d zt_GgHjV9eWwj#-{P>kM5KwVYln+%t&OHv62QP=j)Q905FUG%0jQ1)mjRj0w2a3?Gj z^V%>DY!8MYXkD4#+gE9864&V0d))K z`w1$fUVUoOy0;S;Bwx;nocFt&f>C?muX!!S=x`JpNmNq_{Rlqt*9<5#hB~XeaJ}kV zAXHtKAHT+?udx`TSuw#Rwve`6W6g|$JCKQ@vie2z8S!R{xCM)HwmVP8pWFnf;Zu%y zj@2hHg!OX|IEkjAiFEwjlB+5ft(E)9P<>Ur-yV_kZ@yHOG5UgPi>0Z4;r50?1!n*` zr)N+#zi#d3(p9PXyCQIbb905+S{&)Yg(#-5vq={37L9JtQ_sXI^P$+#j0{H`QkF35H3Sv4b=Syap~jCkCO&eg9?3CJ z&A`6r_isCMCWT2C*6K7yiB6+uJ)e;uyI4r{#K6((G^qmlo+TnnsA78S;-HgdS%le61Q2spEJG6w# z&&GP`bGHf#YT88tn@;dd#4LRGL)jp_!n~1pZ8H zp`w!vpg{E>wJ_KCHr`g>bd8PoVI5Jb*#<c0m9cJc!;!}Y zfz(9WY-o4oz=LHOIie;+tw@2}_x#F`WCB!Lpntlz~~l-x@reL>Z2JH zsf$Tu8Vi1DR?m8}@F7(&(8gs7Dmlw}5D4bU0`?17Bz;q&l}>ud?HefOJ00jPOwrW9 zEMc+XxUPddQyM0u;XNo<6svf3{adj#Sp!N5a##B(p(|+GRS#Jk&F(n${SJTD z;|3m(k1G%GWD0~&=y-0ol2%~|7ddn~L?L%aIDMt*$+W=Sn?|u77n)0q!%<@?X6{>_ ztV7U2-`4FZd$(y%M5%49(()OP$m4@?H0tQj=X=odw8jR;_p*HZ@73iY)UMY^ey3_m zzXV^Zm0lLsS2edLE95v$$(mA==Tt-R?5Nr;&WXT7t(e$1igcyNr|oRbqP(-hr?TV@ zEEAmwxuzQhzN08N2SE~J)03RB5mO>Zz1vD@fh=wHqGA^}VqaG*^lREq*BAV3Fu6oE zY`IJgqt--=thpbmShMDd6Y4EEjdXDBWih+O78^n0NGNJ)M94|-UxcK>*~31(C_U*_ zP6TVeXCxQ*3f1))nieJOkMfoV8xo*EhLku&mVf+Ms29VB47i3qioTV}yi0}}Ni?V! z&&!LaLEVSx?10=tp$!wIMM#`kGTqD9r;P({1lTdX<9^4UR%9U2%TEn#S_hp@xA)J36GJL7gR%z1>U#dcWm z3+2mr&M1|n{9WPSH)5{dX)JZVrt4j6q2CtWwpacz!@+hAV0NvapZ~9Pkx$I_6@SO0tir~BOZ{|d z%aO^*;B4b=9zO;lDE!h_3GjE9C2>e}YzV&e+9tVVz;X{%JAN;iFz0kMeGZ@YXOPbr zeo>&iQ@$EiS`*o)sR|dZFTBEQ5(Vm4gR#6_+>a`?I?BWWfzb&>;Ry`UOjR063H~S3 zs#2fV3VAAj8ENd=f%?^Hb+=caZlJ16LV7d=Q0%XM8+w+|#7!2@-brhS3#~w1qIyMp z@x>d(XGdq3xwZ^!SD! zF-gi5!qZDc71mF6bAdClTt8J_>X+*NgoaXQwzqPzi{aQ6NdK4F!$UALMszzB~k_bIYAde4IO-qlBM%R(2_^{)&%0of`{J za6ERnA$i712(J;81E*+eMpM~EN(h(!3dUKOlEN^KSe#QTEcs()QN#t%VCe)Tk`ZLL z<*06f7OD}5L7*H*viyBnKdz(u36o8_@^CLr-Qeyyj=Sv364Z$hoX1kdNTP zC8?hF3bBiXJ((8%A-7Dw;lM~oOIgO@zhxx1$6{^sryt?ajUGkO8@n-=M`z`i{E-8f ze#MsroU%^kI?^t0s&R`tcx(ho>L^HA(dy$Z=T&d$A~j9DEQNl*V3W3?)ZgdFSBrLE z-C^^MWKsr+1XB-@6Q=PISDv=iv^%kB%F}MIo%m8$=v-ZR2i*o;S%g03zU(oNPBE60 zx19Tc(q;KL)P1laksto*`Y^kt=`(na3{xu8kEJY_0CU!6F}ON|cbtnl&_rb6jM@~5 zmE^&7wmbaqY2Vnq-7O}>1d8kBoBYLXq7yDZ3wbl9*?!fqwARc^Sd^$eFYfY68rOnA zQv0jU4ezNjW5+OVU48i0PCk<71;P2LEwK*w^2X}0IpmM(c{ViqP#PYgz)Sgq3hKP~ zD1KsTu~tCorvx<_&l%i}Ge4Wh*Jl^a#4K&rZD21##ADL5#QvS4pPLqGv&MS>OD9zO z787?H03;fDaPM-i$pQ3JaW9;!yHBe#*fp*`LgsVOx>gZONy$HTXO#y+}b z!8JA`l zi#>?1UT?0)OC=CA^)*87SZ^mFWip^b@f32Zcy3u~6AL ziS44F?ZX;(lSg;mb9-UY+R$anFaLfGsllv1<^wjtJzCcxdTn2Sy43o^tzkn`D$ba1 zR_dMdz9_-xe%pg*O$@QEY{#-FGOnd{xRi@fWhDsApXhbDaMc*BUAvP>-+MdbdmjUp zX%dGPrfEzi*n0#~U%sw?AY6P_>$cm{)Ui7H#f+g^6DH(N zcg>abs7e|Pb0=<%*JK|05T0#(xVqm-UR3q^&UsFDdNd=CcD$Mel5MmZ&D(}nN99&Z z4!Lopsfn>{bk1B+m`w2slV)=}ef8+G3KGIqBisT~#rf4XAYgX##ffYVIw*0I?&Fld zy2zQIAhyao2Im^a-MgGDr@bp>tBo^FNCQXraZ8NHr75^MEx~O!B1fb%C$$9_Fm1+9 zfVk*Sl$N+qX&Y45t1#L!tHfi$G=gG&p!(BP;m;t6!A-Vfpv8+_bmfXm-WQQgRyDyrdoUyUde|n%SmVQtz_)QZFy@%cQOE zkTR)ET6uS^68tA`5;Df@Gas*Bc~#=#X3iGBj6=S-<#otVZm4b2YH_mf4Uq!!b!0U@ zD0@esc(+8AnSM2XU`<=kx+c`DHAz@W{B_2kJh=DMSy;1UG?ErjpQpv|qZGGi3!K)$ zwR5RO&K2Ty;|je%0GX&_{G|EJ=1V51O#H@Gvy^p0$g%RcwXmHMI=fP_+x_!d(G^NHqXk*3}HBl}sE~rqoWGV2E&Fs@fh{lFSUp*sCX? z4D$=ePo4D8E3(KqP!gJ6-$9O7LqkY^HHe~(niRBv)MhrVBbBTVRVs0xo!1I&nvCrV z!O5>lePlisD~2`X{LxBPe@M_YajmWj4{6q8pg$EM;Ye_Qv0h39YV5Io355SzI`2Oh z{wQJh6+lS9O3xv0THn`Uh&M+6%205~0<}*Z9#ZYW_BfQsLF}bmTkak~-B8NQz^Q}p z+%Jr8YplAQwdM*i$mYQsC(~5jQLGb=#sd_eJs4;-9Pt$;>_SEv`sR*&E*sh`N8dO4 z3AI~4O;eS=Rx{58^)f9#+Y9>T=UTnGeuoo#|8TawSN@!QgJeTJvicc*=1(FM7VLCvn@YV2-m*)c2PIF(N}LmM-(7`P?ku8|MDawe)-O^;sd=Q-x!_6ej;=k5q6r>&)n5jV$A? z{-TDBrXaE`S57rOFuTa(TOgAvh=F&q=quawoeR)tN;LtB-CLL6T6Sc!u}3M(PJ=+?O)X%Y5b$bvtJQfm_aCPs#AvIN(7?X>)j99+oMn5 zUPzZP4P;E%7o{uKxzzweqFGVB=#d9M8R%=bV|mK$Jsi&kcHO^xvmr*v!l!?$PzAb* zGxt8bc5)q|66I@4VX;BJPuMlTWjbp+oCM1I;BM8Oo&w!tTWL=gpxyw6P>Y!EFRA^T@1$VcqF);t>c-PDq}8QNr04Zda9KF*m(^yL!^`--#6G zs*iSKbn2H;7t?pNpNwhWAdnnHp1l#iU8AisKNyq_@xGx) z4gJj-goD-HUm}Ylu$)HOL_;h&Ws{5=1vHDW27#^-F6}2Ai1grxl3=X@N?+I)6lh}X z14X~(5AtODM?}f`z`Sm!KAF_kVO5#I(u!q=IgBB?Lh@8Z?wM#8-mPm0dtw^YJ;6+6 z9L*J;AX&S^>2Ldp3^*j|{@5V#HCir{0 z)zH4d+C&iQKVRR5M{Z5{FU+*b2~Z(^`#L{O!rLIdI=Rt!K`#2Pcea*)qgfz6@Md(G zo46j#ADZy@=q2SVrwNnq?btR$fW?@j%leq*Eydd7XQ1T7W@6L_rTpvT@<<|lZVNH8 z@4{3vqHu!pVw6H?zr~#qz3Iz9Q92U2f8zUEke<4Ou|CL}%;|QJq{p2zHTz>wWNh@y zu3`M9beJe#-vvNT%!X!uDTJo9S*2>aSU&?vf}i3{8ALzW;U)$!OM;_bRi4w5Ez2rX z!c0K;wM3sFM+bQg(C7H2+uL+W*UMCxX~?a`dPbFH($rz0U41_uu248 zM@cXYaXnL!;-zRWwsu`FLvJ+>+3^zt=!natt{!-#{1rnB>-8*>C+>_kxiRO!iJdVr z>PJt#vLW50PEO%EW&q{9Szy>NkeSm~|L$R&2ULsE-Aw2z4syxn#@`GNb~M0GBjL!x zoFpKuuPY;eX zL;Ecxw#7!*I(Qz!8r*@n8iDV=W!703dVsAAZfP6?1J!*-ZBl4J;DyoVfr$rgD{8nf zI*wg(lQba?R1l&j*0{mvE5a|4g)D`_Equ(r!+zq@u4BfB%dekt^cR~qa^qn)aJ6vf zYq%j$GVc|hlL@}YFH}-H=YFXr@IMk!S$!+r?V))74y9fG!OTk)_`B38u}*<2dTfSa z*{~b_pp!^L>QZ-5WOmJfF{4i##tkF(a8P7Yfum8&!|SZamN>6ntV8J0MiKK@3vz@# zWffC;((+mLc35|cqc57VBmXkEvTW8}0)yftS#*$_0jKv>aW3d1LkVsQIa&)Fp^HZI z3n)~~Xe&G%cJ^g%ewB91R^^P$_zTJ57|4aZG-&Y}p47RJD{+Ir!VsfY4V0R^<*m7T zwAZ^h#YUnXkcC9GOrkTw8L?8}R7GAr#?FUY@|1R_N{Bx;sIdF-kU5&s*;cf65RChw zz7fY@LJ&KpaNMe^Q?73_ThaGE`-KXuR*7HtPMV4t>}zp!@|s@>I|Vqw7E&36oq-9Z z`n$J*d+w;7qMAxY+(rxX#@^cZn_Ru=ly%;?H0yJ{BVKAyUeHpYdr-fs?O$(U7=73Z zq}aNEpJk6OeyBz$Gw_Gr-$w1>yB^R1Vz8_aJ(!_m5;w6LR0g~!^%P|2x6g-3~BoV58NjNW9~{oyNfQjgZu!^br~alNm;HW*}KV5+Y6a{5Tqq2#dx#jO@J2 zsV>P;u45K|d>!@riqhq|LN{rm)s%GYnIWjcepkm1x5oAEAz`|&kA8r1>MW_iw&M7m z=LQp7YIeBo0q@bSpli|{(zRq(hS$GaYQOe(gZr?Ub9ORyKwkq*&0q+#^P4{gWdIHq z&GEimer09Bxv6d$i!-JxD97F$)Uh(d7D+++ zr~dBx@1dEOtoFy+?b@Feq6|;l$F4=ZqtspIsOEE-1^R9-m(d zm+u@D)$NC+4BgfNdsBZdjEnObvLv4tzW*~-T-<#&rHd6$RjUMN?LW zvJC6`nr3nSEb}Dpc>2dT^Y<%>%nE_Q_61fLD3hHzdy#SYOFsglt+NiV8{xe%S|u1G zTv``jT&iU(r9wp|{WY%Yr?-*6+t2i{`)x|DkHRJM#9(JEGIcLj&rH*t(ZG3!y?-jP zYyMbFiSy%0F}aqdSPq`#Gd{_j3iyrnWEODy6&MTifuBI#sfofo1T!7pEh{gE}pH^oNd_hGugp?;4hp;{p7#OWAeM2=ays^{ z!;m41t^-2$J8oz3uH;RuFZDG|sSY7^OZnT4`ToJ~&ty7c#B(1B!e0?8X}*=MB1|5{ zrpJgAM`xcwQ z#0ndhl47YHJB@4&hp0B;7QqJe>W1!>bpyEP`nfh1#^`{HQ1d>CRG+NE@1rhOiDV9HPjHW(R7onY?fp##mi6Htj62pVmKSc=t( ztx+5MDEceL<`B_Ua_1p`Q6}Ld^uXr^WI!f3r^k`GiC5_2*M?7{x9tL=kwm|Au&4d| z$nXp5D4|*07Lddr3G1>~vJ-a{_lPnB&Ydquu-ibd$Y#6vg^=(p&qKww9b%)?7Kf(+RnD@v1Gfh?pii;i^pRA- z#2gAoMBKicYSskmst}S$IH*?_l{Mg%%C0$M*=+W{T`uy{(D$!JyjE3*j|spM5mWgN+A-%6 zIoe9me2J2shN0z;Ozc$bD8C{y+$iTor!~kBq%(na^b_(4o5`vD_*F~al;<`}E_iA> zJtZddua4_X=SOEy;kz!W!CGl(pCskrq~+!ah{O^yw-H+PQLY7e~0Q;tvR}ssFpcI$j)3+a8d@EG{Bg z6EnSA&REl*mde66uR$BR!z4iSXidMb``#GChZ7TGwG)pxQRckfMpU`F|gaS-{uG+uQcsiJSae >hLg%BH(0J z@Cx_5KT#VhK39@Tz`D5$B#c4Al$I#b8$LeuqYBAN*Hu|$3CS&3tf)BNj=tc=8|bc^ zE+~MqjSC#>ciGiQPJ#o4qz+e0>Qz%m{E`JlIoN$p5ylMi0Sn1QK?h>%Ck*$!tQ*{B z5Eh?-<>MbZ|B z;u_mG@QF!mye`cX2qN7rsx)*fv)<>gaUeuH<2Cc!<=`O=q zTYhIHBGj(>w*}hgCjC%k54$fd#^=2sZQdUHxldKmG2C#yQA@AHtji=E7(`+%P)FVi zj2bWLbfd_`_iCeXdUYm&he*}%l8P0kU#vQvX<>GtlTC4vvnO7;9>}k?Ihu)}#1I6t zjo{UbnODdP+y%!yc|DN|?*2?ph_$Tw>!z+JwuB1#r=3bjG~QwM`I5Y}@G?;ku;HTk z>j+1=Am3rb(7#}<2MD_mSHbn;)o_)?Aa;a(pEqiZVO1K5No0i)ag13#@zd)N^UMn< zsF$~Hs34-C0{K#d=4e-t;ySzKtiC^Ck90psJjxzB6e$=@dMFP?cwIu`>hbU$Gy6~( zFRep|j*5h9xs_>O&4Y4qjuj}2VKK-i7I}vh@lUyjRlNs#oEOB=Z(}L_$6rAZQ{{nn zytxW1HK43f6k8?=pvxKs(l*Tv`r~<}*kRjSD`jCyida`i0Ym7^Cy`xVwo$fKYpIHm z4D>5F!1O-6r>~n9x}wb_X!EOJLFYeFng^TyeZ*EyqTG)=+W18Z+O*NHVUOAQsmH?O z10JO)&YH_!Q{2D|RTr?`+;G}MX}s6}MmSjnBNO8n4J#z-I80< zo5^y6scbZd6kJ&g65+pcex5MG3=}TEyvp9dP0E#Rl_dJ)8ZW!XSm=Y2jpEQX0qr}h z(}Do$*SV?v@S0Leg`bvNE7h<-vmXPCtegkA zj@{%6?UGyS4b$E1&iE4a!I6y3W!6p-;db4MAz%0N^84-dD|X~f3VyIas?C%uRwpO; zB>!Z%q^z>aaDO)N&Qh^lvW(6k$UY%3OTiZ%Q_9Qc%+cC~$}rZsT5 z;4*Rw{unU&4#Qs~V6_&#Y3%b7$Wi^|Rm+-=k=0S$j8HaEZ!ZRc?JwR_MLk3;E4+dl3=%{ufun7>W!^@ldLcAo)2kuLE~)jYvP{b%Y3sh209?DPu!E zUa0qJ;^1$#6SlfsM-=D$q!bNW)2vjrlBfJ;PL@sW(aWTKDN8(?rm}E3|DOji8L<{%OMF{h?bEz=9&u6L4&iwgpWUMhfK z`neQsqlu8@Rd}FEcFKQwbut`;Rbxqa54;ExnY6rjhOh`?`a~jygoLD+5u?MsT#x5+ z$JQfAuHcSwV}8lsLAg&YE4kK)?{XAlAmtTy8-dVZGG{Y={wnRzCThP>TClliwE}E} zu#DfodL)}sgFc%&UFt=v&Y0b>FXP?U(qZNmGI+j&h1Q_yn{8DfI(jSjttVlcU=hwl zbIMdpqTdoHD=B*uyxrL2fTzo-5T|+aE7pp$0?zRFQ8bhf;lK=ZftO~X?8^r-dt6`# zDqb(HSFstuG{QuZD@53u`t%69LG2!Hl+22S(gBU%Q~s26aj_w8z+`mxlnCEdzOKm@ z5=;ysh$x^4P$ail!h5tyGA+n!PAR^H6lEZ=(f<`}Vf7)eeW#q1-!L-068BYOP^|P2 z^r~rLhV63E9{56H&GEK()w%nfzLVj_fYAG&BRltZHiz}fMO;z>>Dj2AFql$iKYeaZ zsk^ceHe?IAk5RyLyUykh(S+pT%YLjSDIwHJFuoG!@Nqh0yrUv6sPZWV!kit;OI|Q_ zk?#p6JRU1oac|`$%-EoFTbZN-i=vN?aWP#+Y^&m*bh6;Ei}WcmbcH=JZ&vY1CemsS zjVaeIUaDalBU1DnAHU0{RWAlGm4=aNFcH#Ocn`vqZ=tP{1utelbcWxDg!S_-kml%y zC-qLxqxAJhd#HX?iuh^gQ?}<9@N1Xa*Do^1L>aI)7O%TD4tIq6NH`UmLC;K2R54df z5f4_rgc+%9`$+wT@)qN*b{=|b4?FZs=FSDdv1kPqfA%6_FEPu-&U0#v?7Rq6HO<`z z&Z_3MOSQDr5F_Mrhcn-8@7@XsPo?YVln6rvz0d7Idsx zD;nH{cgRw?xd-IZuX8bt;7%Q9Zst61@MBT#a>iYQ(pA6p2O{#KqlRU^ab`}xG5ZQ- zjgTtMoWkQ$F|E192&_I*=hw z<@&KQep5Z=!-sm#I~Ow2{9GyjneEM{m5jsl*=k&}?`~%yX-|;xoxpVOujMnITONPY zs!+vP7IiyE3D@Di_X#^{@=M51ZeoFJeD5ym`r}K@`7JA~+gAWWJUJa7M5*06En6#pPGeDnS~K5Lx*cv6wfCMkBcV_EW`C-cA5Gw!g-~q7 zlVH^iV9aT0&gG9(O1h?RQ5$8H1D)OuLl(zK@yBz?u5jG@6b*>nP2SB`by$h$fd=JU z%eORW8Pkf-4l8^U8(pNwY;{7}{|7da;~M{bnTc)!x1M_yR82171|{kvCsO z@v$&TOPS7lQBA_H2M(4X4;kxA3S)J|(X=0Fue+j&xw;FBLRW&ne_!l%q9R>Ox83ES z(Z+-N*YO<~;=O!(RM`0uxZ=p3_tn_dfGLiwNUsoZFK-F`V#6ganY2d7Y}E|A##K&z zD{}MqIsXk+H`HHx>r~R)KzK69VE6Uet_ejB(@Wr=0(Jt~xT(|zotjR(XFkm;L^f zn!%P;kZ7y4K3!cmHo~l(RS-rfoQ}t>Gj>#wNT-!SpYb|8g~H?I3Qr3`me*h-vQFO! zsyFeNN#bFez6v9{V%7e+$i@avWLZIbqW0*qmQg*U8OexuMH-9q>!$AVc}2)!Y9vtV zi>|SKQXCgOs2mH7!$Qbg%2QbJJ#fZN^Cv>kkDoTzCl0 zkX}oi@+%|JnuT`PpO^0G^d08}^`L8%myUQ3PXW>^qkT5C(<7+c!Acl>o zn2#@ETB0W;pwz#MxV95>f<7MLNunFSn}>O#myL6bVm?t+wD!pEh?uztw^x@A2gXk# z->FOYIejN+upy;7_^l;KTTFIrlIEA5JEfuHi{X_Xk#)u!?fELO?Kds*z=cYFE{hTg zWpNHE>+!WzT`lg7Z>H=r`%BZSLIHTkUvu&fMWLn~BI!npA0vA+W@p7_eS*E~c#b>r ziy{6#$?A@(kWVhNwM659;&K_MTbk#`z-(1>_sZdX+If1(4^|O&fNsty>euY(Ue51_ zm0Cyi5HPE=aHD0iZjJhBYch*!?V*SjMI#j{YR{YjU7Tn`kNstvc@sJM1zByKlw;g6 z_F;opr$SF-wC%N8Q~7wQUYs2wKblUydbM%1RKPjunHA2iPrVAw3&2{`un-!;%?t?lMLR0MzkvaV z77wY`hrnYB=(RCgSIh#d9j5<6jYr{3^JuON;MWQ!gj|7F+1@0g)$iB;W}o4*zZ+f+ zl{5$B4DT06GaA(Ze$UPL){D6Iir5_~!?lK!gr?6FUBt@oT+*IGpUp)82r8*%R-j6I02)M$637@PfNLNukVH2a1Ycue2uvo`ORv2;ou_fl#kFGlp#EKu&G<-Gou z;_LT%uV*J~BQ=#0JbaT8k_rs*q{q;0udv2WnDLkHmnCrK-13|wDh_NVIthJ_y)Hgq z03(@~UaM|+YVv^^w3M3I+0r$sO?yXVW$|^KF;7?B(v#o`i-r4AenC?z z8!cX-qSy0e*?9Vj){_OFBGu(rD&22czgW7wIF^+{rI7@Et3Fd=u3-zoze*5*if5V-oPsz zs3$b|`g~I6>6}R^t!g7!`t{Gi?yfn=2dha-E4Q88*KzR6{IWcQYgrPe|AE!r}xuk5${jrjl23@9y^} z2Q5)6G0a>}9QV@69n;|%rQ0vwZkN&Q3%GDRUm_*z`f*pKXsyc;Wo5}ihVr223e48#FLnRMAew7S?rjV&`J@|AGCV82{&WXl{mSORB1$OWGLElF}5UziTqbGy`(+UVFP z{#>p`9c??!msnF?RMIT~o$L5e${%$I0JU*!%&x0WRb6VS!rh{|@7!%>6A7{|=L`oY z-U9)MbN99Kf;zhvlsYY~;*a$Ne0aq)JD(o2;vdH^qPRDgPF$oFm8kN)wBHzaE>0DF zJvb%k0p)e0H+{%Rcz3hhCAwQuq>!^m&^XDsGNZ|)K@y*dO&V?`fJK)e`waydwPiTs z2%csc&VfahPpo>31@`*r%g13n7#(dHLCcXBdxME>ZVYqLulc0QlA$+ntv1y3B-<8!}@6U;`ChlBYfA(vspO z#lJ-Y)4;G?Z@6-Exvn5O+Fbo@<%e57nsiI`M2o)n{WNE5=%%y5C#i@qcVM3JTI)2X z@54l=qkJmg$&*Mfs{btn)y8Zm7b}U0Zz@Bx*z|k4O zTN!UY6X|7e)m7FI&CH~CAyF%C?7DU<11(W) z3MV?8y`svhlqE&<$G57qyI$t1Y*Zn~lo1a|1KUsmX!MFU=|z~fd*2xERaE^A_4T%Q z1Ga`Ore)Pai3%|6akZa6XL~pC$A@eB_0!p@4Wfiibp4#GUwL7Lr>`!7{_Rl0aDH9+ zQ^Y)LD~}6BCp+IXrZgPEkJBZ~N>%UxCf4Oa#ngu!+cY6(UF^spSvoN^y1@Mnk1Q;# zNod3Pv;JiO{0yWIAJzWIUJD$w(r7$#~fe)`!! z{=0&$Lb%@l6nr85PfM4$4*TD@F#4|`x0hj7HD^yp*ChzC@nQPg3d|ER0JwfaKZy zM|M>aUHbqlV^-7nTiO*Xo|02GUZX(VVb&IDe29yW&Rx%6mHRL8xG-m_s;s*2RL`fn zmQeYK1hOmbPNa$nP7E-JOJ84;*CRTP@E8cmeC3h}Qylv-Z@R~cJMS46sXBy_O?NwH zOhU7t$#re+u%|6hR;`fzI6p3(bR`$Mb$&RF* zwqf!siOGqODj1n+dA3P&=2H6D{fNKssTxUBFGV3=kNf1jytl4q>PUOz4Z+L-CM8RX*yo|b5o{wUxMqV?O16g+I zw>alCBo=20wS~>la?zJK=DA?m>6**Q;}Eem{Wy#>p5<~3A7wMvqj-d+rwh@+!a24? zhV8gO!2TtQ{j^I`ywe>@q!Jau;{LuF(J*QII-kD?oQLsN^+h{U5^82s?dHnHtNjAw z@m04y$!tj)Ba$1v`Z1voxC+rLZ}+RU_N^`tcGv_V)vblpQEVj-TLf{qlH;5s0c5g!5yfoSc zzKp$MtRt$%abKFOToP%YxoQ^cjPjYh)Y!U#GNU0VM;kqt5FWL6j;xtR-y!!81X9MD zV3{jV;nRy_B0S*00mpQ@yw5GqxYA5EJHKexmb=M+sV|iaxqv2fksnt89#df*l483R z`q{j^f`q2bze&z{sB^eC2Pbzn0CfT=S@8eVlrM$nZS~slvqSWKro@d2A|tq_+$WKK zAsn!p^4x`)Px>J$W;G%g+XT40+7mB63IEJywG<_NDu!7Mzc-m;GrOy2W}Hx4*E>FY zLCMfF<=+ayug5v8S780_%C}ph)JlpW{4}IeVhgvpN_xqMBX&Iecgoc)RxX8`;b2^w zQhoBl67Anp0t1%otfaB3$Ta zZgxi4)Ez(52&JxdkAOFc(17X@krH--St>Jc{4<7J-II3*%LH|{OGbWZc^rLn9enwl z6(Tw%{H{d7H#jCFRDyhQ;u&Hcw5uRC86yui^@3o z{3+Rr)K-+mmT!KNYh+%K)v`YOP>$xzIFf(tcDgM|>{E z6&8qmY&Y2XvEb+TPN=JT>OhL-d*`nx;4i|S9R46x8YYvp{`?N(IA=*V2F_BuviqtI z>?SRiSi||C5Y2;Ta`K56I$WQe?L&JMSu`u95ZP{wNQ;2Ox>Ni`Q5eohU`_>2-@?s4T4Wqhf)3Hh%Oj3f5e(f2X zHS$S&%fKS}arZL1{X<8*kpG;;!L0S_E$GvRtPz!Uw}JTMf8YG=*bpavMmaX--MyE! zzmo>edIV>3vKbHk1`FKgyM8H0@}gm9Ke7V2XWGR`MXspoD`5>j@i56sEo6C%*T}lgh}$x~9HcwErXd5Wj}>dxmMGW#E%q98MOkjrORUS;I+`uQX>W?J|z_lMsKv zT@Yzs-IK^RE?aJKh)#doZ4}iW7tmwEb7TZ9b8^TVD)x>g^BG6c_ylWP`4Qg*N~kLa zlD6z4N6jjP)q@WmOfrkuc@q>+#4tC_EVNL`Sq9m;PvMX>mdVHV>#|*);VeEfISBIM z$4H=Gykh~kQGdA~;)zNrPVR08Su(W`!?&=OVlR|buF79+XjMNe_RFw8QH*$ak*SPK z5{2k-L`)jVvBWrvxJm>RHb!{}3zC zdw&T#doJ1ys{tYOIy(I0=sdRL1cE5~OF$Esy5my7xvKznyPyDjx^nY z{9~vh&?n)8S3+z~IB}zHQJCyxil-cl`iA+YQPgC@o9Ty)ltUukqNl8!eqyzljK4vY zsG8t;28d)|9wuy>dW^3QkWpe1WxU1EZcA8%p@>(~PcsE$gx?SO)O*B!`KX=s zgG(fdZeg@gy22Y7@>bj9X18vUv$l_Wqxc7BXBjl~S6B30LWoN!F1*3q!`mvJlc$SC zet!PWA8c{4jp8h-r<*7K!uEJwC-&>7vJ>4TxPgg<`CYba-eTIc&%?Ww&S7yjT~6;_ z7Ohy{)7Ze4+rPD9n@Lzygh5uN?FNizDeYjyv;2_jHQn>tGee1x-Z0Jm-Id1iSx>FBO@U7PDc6}zS!ffRz(Kn|1gW>tTw@be&?nDivk8qgw z6m>z6ZH9!MWQ?qn*SiQq2~|i$xtH5`hBNOS=)2&3`EXhCiQWi8bQh-pbLE>0tke>! zGCgTSFyOg~md1O&!NxMbb=(puvUhTfV7`_6tIMSJD>fQ=&qZcF?49FZmnT>@>=LqD za`WTmrPDNm+|QBneiY?nFkbs zUEV&r8k?I&>E@Kck{*1Ft_hy%660l&u-~UxvD%*8{~T>yr_4b}wy2(=q-Y?2RDqc? zz${CXd1npY+uDZvQw14J$JfQay@ic^e;g3(IB@uf4%Ig1w-cUhFONqcW`=$k?xz@C z&TjDw67x>SqUI7Z&Cvj{(iMQhTNk^84AT*f=Wq6+hp|&6pjDo zJ(W9Yj51jt11y?Xhi-AJgc3q6-EnXHzM6Hf-C2nAIJ+R0oRZZ+gMZ38XbVDyT^JG5 z(4D~ho+wFP(sFoVenW<;4uSyDDqQtQ%Wa{aD|0v!v3?&;>e5yjvol5(slkM*?@z7| zwbJ(twq_h1a2v^wvS7(E>yKQ$vJq{gm)QY!7-B>!nRI>$`lTLh23vX+yk9e1#SAlV zY%B3NHeA4uFp9fE>a=q*^h}DC4fUveDD;;%E1qLc=@o-0ue!HM$-P+6QotlKVkq%l z11p&dtLzr+mV6G*k`2v#$oVrCLZ;x_W-x1cg7elbf9_8b3VyX{&32^uR8k#9m43tZ z=HLm=uu|=O4B(h2mh7`BI821PC``ZOv{ioPnGWF)8+%#Ejf;A9zKNEdJ;l<-H&C~s zG4T$DkWwA)p+=&D1kF5~d@P^`&OJzoTmm5Q=@`@bGQVxd9VU3b+Dl;5&3eDR#r6>lr`%AzRL0tF^*%Nq)p8Q%?|R*Jyn~! z>0Zw#D&px^LCI#Q!u2RStcQzTli1qxtzqnAB3Aa1tNeiug zFgR&TOgRa<;Q>q#ALbp1SY#CBoIls$9HnQ9{1ki6;v{Jx^KJvV%be0;Xz6Dg3iY#J zi_N^*LO%}KU>oTJ+pWRuLao8P<@kBC5unCYZoVn>`LewNTx?f%R^b}s2PF-h@E(eb zIJq;mA)91>M`ie%0?~;i&h6?7B(jexUfh|&*O`}D`i`)wf#;#r_gQ6v{uj>#Y$9j8 zwNR$OI8_VJE{2`ofh_d{+w84lS9*&2ro>MWZdX?s~2WAqWB!{q#K5&n;9fL1{*&AJ!fvf@n13C!~xVy1fVum^Uv<=mcKJNH;1&LtS1hr%oir-!t59Aha=$`rKlL;c!h~# zQGmfl%fzgu?0~CzPLSHD=k)F>l{cGmckL*}pBCfFtD)JF{tnRds*Ril1Zy%pES090 zL&G3>$b)fTUkRESHp3~(ZWW2Z#S-u9=+z2Sn$%Ety2#8~Vt7wD-fr)r5JukGhZn7m z)nU<@A{AvCpR1>#HVP?Hj~p}P&4)zVa-D~$mFnDEXwo-ml6YoUTRM+bW<+3NDCU&r zWu=G8-*KRS`{ajr4S|EJqC|U@>*jv^k3A*3byj^ER7Rj*cB^0F)gWH42utac{e!B6 zMCq>*{T*^L+7Uns^qYI(2hiKTY5Zb-{;OSvIl-gCIQO>hHSsz9%I2~meMhF*D z+t?5YRu|p9+79O9XQ}RJ%8{^dY#_i6$4x4iPi2w9Nkk`tf!jM%*GS$p8ytk#h}*bY zW(RWseda0|Tgr(ue_<~zEks`}-?Q2_zmRk}A`Gi=A;=@4!kAsAuArl10i_VK!R(6) zh4)jv@1<`CQqNqRbD2ohcW72~QceCTllH_6qk5ABNKUZ3RDbWsArS?96#Z$GUV9aT zQIGzBx6uzSUUw!1J0e=@Bf$BofAUTc1XyeJ{>(vnn4!X!zI%4Y95F*avA<1ZhFKFU z!*tybdGuoFE7V)|;<>O538GLhJ}y3=PSio_mf>j2J9e`_vK`D)zJgvz?yp+qa$ZAx z7>%iA-g#)`Fjz;I`X!pVQUc`8e%zaffeiz8s7Ie7l%_mq@o|MP9R;c{MXGmDRIA}} ztNGuQdMmXeZ(LRLo&?K#s23F&Gyu!!$TP$e8hrTl1#_+AARCnQb#_=+WV>|WB;xh9 zLKUB1EESM zyf+G+wy(kw;vD_7G$sAbE`D5QiFlZ#IV|{?9LrGj;r(QUpK_;=;X;kyOxOD#Uq7ZUOnL%Z3;*kq=O?YBBn zglOpB=jU|joD^~B!_p2tJx-XQ#-5jMpgd#u^-UJo+x=yd(zRdm{Bs@|9esZXOpg5> zdE=I=X=m57eLlTREqFzH38gPW%)Lv5!SeH#_~}R0z9+JO7YekKK z1Gf>$T?Uq~-1*Y&h!DujHZxozd7_tC>}Nd#F=gWN@N##*a}@0_-WY>ALWcQ^Jyw4x zbw%|ozlMypPq(VnqT7`m_ZoJTmnfnA=nd`=Q)YEpn)zDgC_&D8Sb#H?? ziEuj9K!npxS4vhEzmH8R3qJ~`>i2gp}F z2RNG2>}|AG83+GufiG@UkcZhzr=3&MGcsqBl&%XcAgrRPnNq^3p<0`0G9&3U8{XfKX?B$4tIk!~mbvc% z#m)Qmg%kuQ=M0JL#|7ENPgeO;vI|H?ig-m)jZa8W7p*UOK1-v$!j0q|7X%+5`?t%^ zdd?q7&8WmA$MFo+*J=1;+h(qvfrOqj^(a;9>dD4}53e#{)%#|JfC|gAP@9#Zv#NQ; zR1}OyZJZG-nQA-VjG+{<%4wgne!VXlx`Q0t2{BQT4oLl~=6JD_9CjUusTM!SquPk) z3awVUWIaM&#O;_P)@iL%Ll8m406&%C7D!Kj-@f0KcuuZcdv0{0E>71+V{1r~4!Rdb zro5)$@6JBJ>+!wdF~@T_sV4NZGx}-{%bvj0c~@)M=O-+KS!eYmSS9^AU&c|R<4nL- zhF*^Lc`g`(yrd+gh$ivkG}G1$xY#{~w+GincE*xQ3WR0KLTGY6pA4E~Mf0QYcap1c zcl)&I(uNV0s~Naxv3Vvvr_Qoh(gi0HGK83Cz#T0g8yyBi-?D0bXnXkkDnjzA26wu% zZTaz`k&xv&SsSk0jyd=79!!t5Qxfdh{0!UDXPBWbvI(iC4bS}go?(CV_31sF5*7B_ zyYOtY?;fc!LnCQ783#SP{pLIl@yn~pSWR-z;y|3j+@$3VjT~xB#O{8PPLg&X5_3=h z4@h6OE@gKqS1a{Ui*)hbX!^Q51j{54eUp#W(Q#}xzdDC%lTEnJ|=s90U2&%u=3 z_P%w79HEYSmYAOk!5z;Iie_U}`z51@1 zqHzG4>d}ri!6;O{dzA#$mpxCpln~*vr2mzWBmVwA21fXAM%r9WE2}p06g=4Pg)=|+ zb>?u^aP@aWa5GjbE9=`OGVZLgJH%RxBlKA&!JnxM*3GS2{aV4){JeK zAn)f+jltpu8=+8aSMYi8N%~WaJ60lJ3_tAcFTTV@=n!)Smf`-Xp9r%fbLQKi)d@$= z_k2w_#}P$C&qkGAa%9m{R02eEz-*=2<$Bu9I#?fQ;xAr}i=dNIsKeHf2tU&G2C61Y z1%8B2`qvQ}UJ4UnmtTLKC#wFeBk7s%cd#Nu-FUls6DODQX49g`~oUF?Y;Vje4-H;!H%0%827bb#xX$KU4)O zwlNLWzf9KU$wiqM2qQc6U^SXtcwM2l3N-lqvR}P~wzK4LpB32`GT$IR~^A_krp@s_n0JUn#H*HKEK+76iAm1R}1IYVVqYswU(v~ z<&|G?HxQ~Q_fMAB>%b4ukeV@Fh*9%CR4c5d#W&H7+UCj1V>UM?mJXWd5dGSkpI??^ zQM{009r|RvlXcPE{c38PbA^cuH@_GRUKbO zM`XJ!^4j!1IplJ5PcXYt1LpcGPjA8@jnruKbF$GN<2+oH?gqIU)-atJtWO{c)41|* z3UFx&QP@{F1Q8$F4g2`~#E*_~cRIyY#Aen|smR69svPwnLlQ`2iX_K_CZp4vAerr` zPsioYnJYj-y6G>P6&`|@%at(AWVwvD+1rV6T9EZ(jvN1;yS3P`L=$%{&=Wr&eDz3w z;O5hPj<`+i@}ySHsOXt&--jVgc>?pec}yQBv_>5#ZQb<@4z>B*04bq8C1+xm6HDz| zSRv54RQ;Xw%++!*5S%}1eD0_;(nGq2HM~2(;0O|~4YfUB1E;u734<*rS)yuIJCASX zspOw5IvW|XEY`aAL+Bp{?LHFxynor5SYN*A2&`K#SIZJ7q+h~|ma(XDhOu!=3KAl` zG_JJ#&`hvO?h`{X56g`CM#sQ8KSfLMz$fIXb}2Dd*?LzOBT8Wbb3a7|Na+Uc`I&xP zY9CEqs3onm2otSY5!L~CtC##sJuKDr#pu_*4SndgMcDbF{P@U3wZ8dv3m6p~ZYQSl zcSSEbr$-W`IRMK`Hyxhe6CRd{L=xiz-@bQ{({6%>WB^_ifz| zufla&+{@F4HjM9H7ENhEJnf$MN{^dD->TDIP=gGC5mO9~X|8@d&lu9(XPty&l4&yiG{VfhF30J;wQ<~!Aiw${Z z9$ll?vX)ExS1jQ(mMN7nY>R&E9!|U)bJLkpTW+pi3N?yAY%oWr)_%XP zMH{_4*A{@2D`H76!ZoYPG?f=$gk~Uj{9OV^X9Rp%KG>+Pl+dw&he3metZ7T~r#2WI zVF1}S(3EwtK}8_%8V_5@v|U?$k?GlMKWStLKwm%1ZW6)#Fxe5{6g>R z{q}6pYYXp18c#WC=Jn;HpOuWfOcet4%_XkIE(^bS& z&3Z7g|u8h z!zSfP{^&&_aLD57lR?XI*qfiDz+RneBX*)`90rTB%AC`98Cmk0uzhXs?sxp@?W&$O zZzzj}X@CL$?WTx6RH5xN_gG)TZ&GfmLWL&G zrBcxGs|9^KHjV()!pyZFa{1j}-4L-b&3oiUN0e=BXvFKg*pV^O3rP8E9V8!rvE(J= zY&};brPDLFH;ZSmG@}k4G2!pBnC|{&_!*_X*6I3ip7L@&$P!(ryTDUGvi${<&F@%f z@e`PzgYZ8W93X}X0P>ysJ2VIySPUDs-$wa774GrCd*%1&rkhASMrRY&RZ@39yp+?f z_Ur4XpE<9jHOOW;uu2 z;u*i_fU{v#xIE!;N+QGDY%B})9=ZdXgHhZh`osf!(THCR52aY-=_%nmSPx@X?6+aI*ve}vb;ob_{>lg zU)M#9`(5TY)D4Q}ZS7Kiq=t{Sg4F5bx`isivT!TYh?4#4(il_3!z%rjEb70R>ENYX z@6|}SADopTIS3XiAFt&%h_koS8uBdANyS-6MxAMflLN{KA}!bB9;QX-@Y^KFqHfk9`t5|jquYG!F|T46mA_Uy7NBZ$Clx3;JNdRInq7T9t`LyEos|aoqHz@WpaQK;BVk`j&MI zzZSh$0Bu4iHe}#+Y2}ZpaZ?P7!6{%D1CG8H9})5!F*()D1V5SOo{Z1<&EYt<(}m|z z?it_V;!|mzyEHPvAKrtuFiD~m#& zB>aiS&8qJ4YMQsUW#}0X?UP(R$@;XYe|z|A1922-Rih%Quzt4(G|^uV{)sIE=~YWf z9rpQ1MiKF%?mdI`Zth6sI;Ep++{&T3y!k?wO;Ln4f)(uKI}^MVw3+qTwR+@w`qq6SsQ2gYdb)xXq<1Y7dX#Oze@|OSEPs9 z!u&xbk1Y6{sL_B51&T>JdU|+jd@7||IbFBRT}r!;kj9v_f1?%Ahe#5(DtDhw6HDrp z7EvDvG_^w`1I&WdCBJG;-csV|jcRG6f(zF*igg$@{}x7!J(EYR%z>Rid=fSY>Ls}? zvYDUr^;mSr&M(m;7A+pPCqdt@JW8OOk?NW)Vbc@LkoKaG{N3m`Bqn(&xJP~Ro-7Mj zpPiGk&l<-n|Axwu*|xA8;+I6Xh8r-ba{r zHK;RNW_V{O6sV$gt;1ENHFHzob|;~5FXPSYnT<*MOhVS(_vwW7ttQ4 zw<_z5+YjnJ&Pfw$x0N5`7DDig8MXItwSBX0J~bfdqMnb1x_xSoIF2hID zyYucn{DkVvkL_1MsJp^4-Hv{fcLMp8!DtPyN0@p%XvE+7cl!0ST^sLU`ov;8TR4Ua zn5pM~n}%oIF^{p3QDi=02brc{?YI{<{+u-%^XLj8LFHeF9IeTsg=T#!?Sxip+#5le z&=C|P-e-i7Ur81X@I0Vx<+xU{`g>}f=8w0`3LB!`6d;D~2;W0uu50v)-Ge}ipogAj z+0ke5<#W>r95o_R1P*0SqN12(B8Tqs7~&*(YoB?0x^YX8gZZuk%`Cw!LZViBVZ?+u zemWCk6ahXy>~$_FLO$prgjyDu5)3TTAVFmf0#39k5mH-<(n+GdPQ1Q{z0-j>mggec z`6bks5~=(?K_F^&auz&2QY(l>7}T?Go0<<|d_?M6(%{YKelF&&`*pNLd9@e%&(qGU zwpzYenkL1S?_;OCS!H4PVE-%jREjt@M427L}hL+aA?n$4JSr8;( z9>I4a`3=^4sR}$Q7r6VXU@$9Ou{CRNz;^KB;-)q70$QV`UY2 z>=O>`PU;UHEF_0LoRZu$Z=bLa;amPy!(=pecaqGtglD{$IYIbUF>h{k76DZF2p*b? zh_zM!9_Hh#6;GxIU;N+bsTP(9&Ud$vYhVUhdb4OJ%ycb%MKI~RMCZvq0wOkvbQMXX zmE^X7wKFDLujH)jyqE2}uB)-I^&*|(Mi0yPRiJ;kGe&8v{t>%|WIKYr-#K z&hH}ZGAj5;rejzo;lV7wkMYx4;;>-s&8?XtvpgLRGoi^nW_R1>FsoepK2Xk&^L!S5 zpVd|}YA+s6Jr}*}RwS#`yvgbrf(EnXE3B+UI5;@W~T@3LXgT`d|c&VZL zxR_MLm$;5Q3(gc@`?`(?jsG}(=%gE_61ybysA;ifLfOU0emG|2I-#`$jGlLK<-b%r zpfXrdq-=9>^|iA}vTuS=e@C4X+2kfnx#+C@<`Zx#nj&wr$UL~IxqFvD+Jc;2Ay%on z@dQMyg%jDt^d5)9O*75g zaA2!NxWi}iq$lQC^NmB98cQMIZ0>lk`c3Cc3+fLzZ=vOUVL5xz$n!D%J)9Hn9D2^D zX~OZp^JdeN)>waf**7x9SVkk$J4{xc48~%>2JKUNmx9dfd!`fF;BK27iW@1bL<9er z<0p`Ed^h)*RNzowMP-iOm0qA=Xd}7569v$l{@9uwS*ZPWhAFgo%J}f`Wz4ma>`!z2 z(yN%)mW#x{0tI&oIOlY87b}TmwR?~n-6}S<(|yOR6(b3m)Rp;=wx8j3V;0TJ@$$oz zWEKJw6TaD8FvA$?gcEmh7=E|V0wD~qmqGi+J)Y#KT||9Nur(%1A)IlLnnYbuD9^K3 zRQrg-3Z+yrk>Hx_j@vM5fChzEGYY;$Xdm9!_DQ z07O8$zvH)w)uC+$$j`6+WBP@{vXMej>Qfo_YFORp&+B}f0EWXl)o7~8Y-~QEkd^F? z`|irR!Q0K2qtS0g3<{eS9XR0+^k)*t#Z!1TYne#=dDJjlkgdgVpP`oPRZrKI7j?&@sa zMWLU#D3L@0g?9U;9%&fV7E<(DY?UdFmLOyEhR_Ao#fPc3M9=;3G6KtbE$i!?dBxoP zsP4JVUy}_le$R>;;VZMzS%Nc@zk%J164CeXPNT)TOnn=C38Oht0Xei(q5**pV^HOX0dQ64EeRFr% z6gV0=TiF{mDWQ5m2tHMv|s*m;jzFxkN`gqQ}c4-Sr|M*P9en&3O zyR{}RJ+=-%Dfg{%U4JJS$m0b;5jrB@K1v5}YSg}>hSbuGYe`ml>+pIsT|NjOx~rin z!qzvzzlk+rq~?zP-$yF^!ST(sUihfhDQJJo1Eye>1wRkfDrn_~q$8flUTwRh(W?`+ zuVgkgY>}C@ZJL^kfh%4ZC2BqTy(d0fo!bb=0vFVSnR%2eWqZ%Tf}tU(ZG9%8^0W+l67^*6%iPA9Fb?s+{U) zI*23gH5uOgyN&qY-mprG$w1dx`Eqc^mSZS&hk=hL0?WvdM@HunoiRfzc^dMqI8B(S z@MfbDzXfqjGh*sLpY*67n@%=E3H(i@T^3eL&F4JxO=-OHHrS4(@cbl`0zmT0QCJX+ zV@mYpQitH@B@$26|2h_oT>cpL^qfgDp%2$SMxTF#-pfU6t);jwc zZE=KHqhht0QK=&~YGR3*==wW383}ud<=2UB^{nE71NP;5h|eEgtVY=(yg$1~4?kEpf!Zp-$y^kpaMx~5g)M+scyB{CEj9x^Y0i(g z)pluiRoqFDkqn%Sc&>lSnj+2ZV+ES=KIAq9d)bS^iqO;|>>+<3WiAJNUr(%sry!-O zvzPirw2m^e2INLzdf)E@NeSmjT(= z+RwnuH2=we5W0w!+!|!z3$-#!zJXZ`lQLfQsH(T_d;R?(!bEEjlRz(f8~T<%2%qLT z;mOLer`s?caP=05)jaoxJaF}-2dA*F2qTgz2yUzwYscp-c5ye5=oC{=HQa{1+f>l) z2mt8{1%tn5iGC}Mo38o<$=tp?8P3=wk!aJ?X%ILdOL`{k_@>*Hc2&lq30{I#YPN9F zQ-C;g7(W4CoNKt5{HIjBy~za3{17jT#Y@8&(|!<;du9lzZO8OQaxWus8E{8O0KdNJ4J;V=1*YwlBBzB$-!Ujik`rm{x^h z`-1O!z!E`O@}xc|Dr`)2Mq?sUuJ@;jvVutJevGW~F)n&K_&Ciax5j=og_gqjo~K&1 zuOiP7Y)5qp;yMJ_<9(ANy_KTPub;7cemv3Ro8fP+K%elps5HM{i*=Qo(fSGXbv>~s z21DENCQySc-#~ZoZM$}P)ocB*XS?PoOC={3GYB&>t#5>6QY2eRBvaqD9L@aLkBVmB zH46pCUXyWeJY?%NNoS_o!=c&Z@iA?ZcxEgo-$$??H{vK9RinrXS;>@Fl}6*RbG}6_ zX$Kg_cokzf3y~0c3S1?beDJT?L22^fehT_~3K2nMDA9n+>-V(QtC|Ls_bdwOOuI`? z!%P&0@&I)|y>4ltyLeleWH1$ooY7lLz`U7Lj)y2d4z(> z#3q{dl(x*J@U-u=Sj1tuZb<#Tf%dhWXCRiUOv+vjD3W=4J`0}<<_cDK3TNmR2u-rM zDr5;M&HdqTl2Y9kVIz}A^Ut{AQ68nCw}r8TpdAnBogOPejB(Y~W~dcOx)&#vj#M9Q zCwYy(QJVYK%s}fkyF0+b)$kg7NqqAr$o|a&0M9LFHw%$H zej-JiZ)N?U5ELaGT^38tW<0wlmaDN?GtP)5FQPYtjBbig22@aErQA;}M43A-%L}Vp zY>v($T>7IV!XYT0G?!cN65E%a(u%wSatyzyEw(DdjNnwBYKPH-9!A>%uFvOU{4>LZ z3l%reevzN8H}AsOp=H0#E>Tz`8QHInO3ZezOfa<1Sl;PTS{OQ1ZsLA(?4-i|GM72a zqB(!}H)VM6&n;_TEbqEgji1wX?1Qxd1PP4M@`P`M^!!JTY(_F5ixT1huFL7{ zuRU%CCN^_wXB=Hteo@$(KVX^Df0o)--|V%l%>r^(pg;?PYImL~?u*KYE1Xq4YGu#J zgd9GWU@E^SjQ)m;rnKoZc71oqfQORN`Tgj@mH32GBZf3%J+*DaZq(Kh3Hj*Tn~s)l zU7>9dL-+@h8qp>;b6iAk#_3tdRJgQ5=oBGF<^f7Dk$09TR)(WVpV_ z_;atwr&)NdlrUAH_&1J`chQ(2+g7gbN{x+tQ(2n7(mfW=R!bb-;#&K&ccO8ffh3`y z#+Xm8ij{>WJF`?wzs3TWVxo>gBi7=r6MJl1g_t^ZsRfx?e5C4W= z?71j(NS6)h%Y>T|{qi<9HSLR_w;qe1kD%5qgb6^qW4n7HzmF%{$}}!!7nQUnyq-rv z61+H_HNk`_yJE-p;-?dgWh4qHLry7b=dzGx$tRt!<}Q$j&4hA$!T-qEl3uJED&$8C zY$CtZY!r#8|DcB*iOEnLeyVg0R!|s7(RqAWo>9keP|!rf9K% zHpo=ecLMb+`@kITtb4m}f{J)QCg9NB!@@mT;h;tZm@FHY#OLr0a1?J7rUT~u1_aax zL{OP?b=q+<{EEJ}HrKm%7?+v|ud?Xf*)dMh{xa=2Wc;^TCG+o{#(_X2TQxOS>^m;C zdn>oI6P22y#w~2mEZon<7Vg+VM;}B`XAEiZ)s=03*aP#<-OiygpA4eBue^DrXrW;m zr)k2PPE{YOhh00o+(U$iRieDg-`=^?Ez2k^r1Q93er_JC^rda-Z#0x+g7I1<4jtvM zf*6(&9?KqE0}hS9HBD-j)W{XK6jPF~h^@%*!-!>V1_?S(t|J65cU0l()l~!m1ax9avo%+ zy2VI`-l1KJOnQVIAwCFxlzFARAG7z<2d1bcPTL!-cT6+BRzOch8rMbvZ-+dvk#cFj z3}yCkRD7+q$&fOxRfTz=3&%w*VP#{lnky_(S(Quk?KZzdK~c0z+i~#>i{vL0ugpW= zBbOR$VU^ZV(Jn`WeOL&Ufj7U+BC{TnrD(>8@A1;-eN;;NOOAk*e3u%eCCn&B{Bi9U zYyrf1Cmjuu@?~Z(zDq*|orVX=dn8aNd@zkj214_hhg6Ld;whAa54`Pr37&IQ7%Ds% z7}vRqZL`^17raM(nD%b9*D0#i<->4I)m-(f+_==GNk3!UBXkc*^82TZDTW-;7 z6wQV!5Jes~!^U=gR%SI-{F%WmTWf@Y#2#1A2f z@4$tZ)NxPFI$=Lx*{Xex*_5(>e>3CLJI}5N z_4}^U_WRrWG-!R-*RK@~7GS&ktmGSe;j%4U>ve{=J6KvJY8b>XSOTn%4mLJqwi%~f z|2np<#2!|{oeXACR}L589A6VaYqRZuVkG~D|!udEuS!J!2H{6LGS+Yo5w zxk0m0_AW9F-#hwU&$%d9cLS76uL#WbJYTBFSU(0!8l{l{H4hTCAX|#Xu+#d1-JA2O z1o6bh$ILWS{vLQH{LauD#9|ZK>bL>S879;nATZ&}J)K5j7GOeKb5RNdf2e>a-{8`Z z8Nue^)>H6%g5JpwMn{GkSXYrAZHrs@7TmQaXQy=O za3aY|fHBfa2PhEI!r09d3wiZn;wCCI3>Yxd`+9&`{CCA&WIT_RrH?AroJE^+dadc& zG|h>Bz%bPFgW~L%p6f7j_(JKz)8HY|@DUrFJP>$$*2YjVD{>XY8op$3hN2`GePLne z-QeHMh^UXP@Sp5u*rJQzHktBt_*Ub*Wk{_FnTb&-ho-7UR&E(j6@f<{{?S%Dz7J84 zyl=iDfHj$S=s`TiM!!p{iH7&*;FE;b7dO$)eJy}h0#SiW`kDp%I(y-a{9t2`HfHlG zlSOsF=9LZobT9Y)tFyIkkyf#mU-Zy<`(v5({?Q}40p$F37vs06yBkDSh^`kN`4_&Z z{kr!>bLM8g{GJ^i>`h+Dv0l-j4#l<=*Smq--_*-f)gpgx#@f?vBQ)zHx9#P<$A;KN zK@vE;Uu!&Dogp?9iU3sfY9cL1V!u*y-?kCl)!QAZ$yIS8*`38MWrAKW@$t#0Rczm| zdsELgzTINnVY`86C)Ns~lddPP#+fZW8*R<=`{svyB~XqV7~=dGvg9JLUDtdBFa6w2 zT#6`22nOnzcb%kok_H52`uLDuq8;J8L30^cpV%Zyr=djbF3!@Uj3o;szv|LG()))| z1P=KqiHwVbiP2PnGCV%)CoL&jk&|LHKsESpLH5ODsNchMR**Uorfzk5^Ax!D^K4|x z@6NU2^x8Prxl`JWW#P_kq)7}9Od8C9WmlPFS>B7sxox_EKl2d~4BmVE|*rPu-8N9e7x+UXXRR|lkYkBZ!t+lTJ4 zz{zEkR4jw&W^-mPPQc<;47$uB2=rOv7%{3;ld?*DW?MzhkaC9mvXWs4tl=3MBx*Vb z1~ezO%BcC3Brr;Z@}D$0)!ErNkLSOtW*Tv+;gqMTL}wEeKtPzq(og#}5I1+I%?8JUnkPOssFK87$Rg4)+vdM(7)6Xd}?`fqqg;_1F1$#**rrgG+NaT2Md=NSg$v^qY=O_YCZ0mIBbek|r- zf%%XS<{;PO`0N4DTK2C3@u8m{(md;f&(%g*P{2x5<@ECDpjC)6`7f-)zlb&jZlUR| zg^6eZq)NO%=?aU3qEochjYB$6lP1~gvRACG=l{%kK!N1o8~NM&&~@0D;zkvvrI_~&`ZbbJ3gd4e zC@8Xh0|y-LY;uE}5uwoW$3oc?2CzD*J^k7Bm@FK583Q|iO*4ogQ3`~ARQ#@Z)s5Sc zTc5;4eHdF>e85G$bj`Ev_F1MGnNOhPg^xJnN#hhzM89U26Whut+~3qRBEiKd8Pb?RJj%UKTR@5#9$QAChe^9>_3? zxkfm8x%ySwNv~~)kw8=G!SAY=bOvejh|whKXsGmtWep}{re+JZ27(-Ouw!G@VTlx_ zWBLier}c&~G6Mla%%dmaJOHm1ZTfJn;l-ZyBk7qo9s_+~{p^t?haeey9)0#KyM|q8 zcq^JxFH*C{+l&{7Yg)_WZnV-b93M%p>-1(orlAH+B} z6K{p~7OU0pKEt!VvbjFG1S=xxM9q+sT|}YDu9-YbN56`)&ICD?TpZpY#+QPU@x-)t z7B3pMK_#Dyu$zrn<>|QN*yaa7k^3GC9CLK%ZU_se9mru)ug#ouzg_cI8ao6c#Wjd1 z)LMmq&l}>qTuxt2by*4A=x$6m*rL1q-3y48I^MM`n8v>N0;|iVgJ#*LhmfWsseWx8 z?KZognNklk^0aIiaR^;`_!B&xNnyt`t@lKm(af1&_&y)Z*_rd?K2&0FstNGx;-Q&4 z9h*jMYl;Rce(OHhYq%ZGF{x8lp_MD8;nEGf>t$|n5*9D**vPU~aG2-nNK{EC7*&f< zJaV}ljiQ~x66LEX-SDay{K`Cm6l9ViDLWqtM~^wO8T;3KB~JgIL^Lx)6reMI!A4s) z*(R@t2etEt=!knii|HJ`5a*tZ?%{h;SG(X*4|0_ctq3k}2%(g%M{1yL_|%q;XwEbr zv+ZKZ@CAPexYect1X5;KsF*#C3OzJRTa@Z#>b98+92@8r(fY8>QlU%@rbc?B=-B{% zxaHL9n;9MV_YhCR5GD|YF1b53ltD?{fKtz2LkqvQ)tv=Hy6c5PpU!EH)CWKeXC8c| zy+T);7ZJaBCf%M#J`tS82Cj{{ZB01Af|zC%j{&RCa}Ai;iff}k$LCHQXI6j?`3U?e zvT>VyC};+=(MP#T`lB=ouB@*WqUMz0oHsLT+`z^p;bL}@H}a*;kFk|fQ64|TlK8eU zdi+BC$U|Mz-OYNlg6GkNa%hW{xI+Olm!QPP7E$M6qfh_rAsn>d>CC-{<42A(`<7Q?)PWlSYaUWL-ExBN~ADEyM=bjFi2SgwxlsOc}ELf>mO4y zB7vbIJhMqD-=zI?`KQ{c5zcE4J}~Mx|_yO%6LKB>8vh;D=IA z>mO;MEH%3RZJGBtzcr59briw|Yo2eq&uvP@yd-Q+{xhs9qojegY0-b9;IfTxSm@0; z1$1_ISIgWleJhHVx@AM4qMwqcE3XF)}{*E)df>Zp2GGe9VDhERpk|!Tt`p3`0Nrp z^PUrLdMoi{c0W5fIsG2x_;8V=yEV97WvwE8$Qt=dm!Rnvn=Mp`dlQhg7FXTZCBv*% z0lQZ#uSa7>7E6Fxx56#fdZmn!onUWXRedws>(x%Dd-Rdkc8FEWtD4Lc4wNAHM17!N zAeONjn%_wK4IXzr{_jgy%Cl3#{j*iQ0Ad1_O%5$ zOfrZxQ&$zu3DSt04nZA#ztuqUBZh2WiPNjW71q^-mNWN4A)I&SEcFx_DW`6 zeL`1g3mU5bIW=r*>tdWM(P}yu7UGJpm1YrRu1dx5`7Kc5JaX9mRDWE=4Gybbo?UW19iENa)xr9uj2ZYKCH=5N%v-s^RfqwG9LsXQ-953$&$; zPs*3D(%(iw(=L;rh1S(bl=JGOFqxyp_6%4xF-Q8xtp&ZnhZAv_6voPS8bSQ&^{!K2 zO%l#eYn7UB3{Pqo+Lu@h-)67(x? z{?IDlgS8POKBe}6u}injW)tG&p{TwxIvoM*DHl3n5Oml1d60PC&jiy6T1&K|*Dx~s zlUfogJKT~NPSAG)>G^3Dz)43RvHEq?3XayB%^@g-Devnj)b%}8O0>k&*lyLIK896qqxw0h!;t=cJJtI^W)$QDP0fv#< z=)fn=q`2*h%qqHKe;1@g;d{?{)kTS3xR@N44lML21F)mx8KYBxAlwv<7H~d&Ar}!bN#P@AD5$A~$No7N}fYWH+RT?W=HF-d* zGc(3gb2Dy65N%)L(l6YI72^aPa~i2uvTlMa_v;-WjTC$~N+~p|;tsqMp!ytC!3@27 z<`R**Uo%_5NhZMt)+Yf%3ZpEm>b`utiebornQ@9_*)&fTp596& zu>6%mXN*+kk*`oq=_=r3oV~ohW>XLq!ByNh>e14dZVi3x4H=uB?dAyCK)WA(R)ZMV z6t(G_m52t-U0a_BL#LgXnq~p=zR$NqfY-b0scs{HrDv%~2u`~XxESmo<4~#W!T_N= zN$Ci-iUKrK(8{b}L9v6zR4{p-=Lx@L_8Y~IcpHz8K?WIil7!?S5rfLWUyhGa;^TXs zsc$%o93oT#Hl;cDn(FPYI#e+>^)!(e!*`2ygtt1T!ii0!R|5s~9=v8n^5Q|vFc+N< zOY0!-V~N~uNIY~C!pC5JO>@WhB$Zix&;Ss|&B)!{O=8vsU2DcPE!78^Hzb}Vky_u2 zzDb1kR&evMCk=FixJuF!5)L4##FW7q%p3h9RFkfPT6JBb@8swmO9HoXw5r5a^-1Iu&FTp@?Nn7#AS%Bp8JI33k-o?@6d(TQTS;glns)J_>K&r#wLr<2?4lA|m{5vcsq zZZZ#I6&x!f__1-EP{5H}9DcSA=~LQ!&b($#4Xg}RC>XA_ANLhCT1^(_jBx2uQ?{vr zTJ5o3L=kBpiA~Px@6ttnB0lS@6wynPa!^Z@DYv#JA~&MUh>Cu@&wMec}h6UH%xk{^a8{eZ6i z6rT4_S5uWF7!%Rl2iHTw4vo4m3)h#EMS$QnEjRRJns%I!S#ic9Bd8h?`8&=r;unOk z+yG7ZV(uLS$%0?iv9q-b&A-o!E3s5CP1!8Lr+;!_*azhC#Og|zVQMMlbI|EW>CIA+ z)K3!Rq%xsrOy<8q$K#yZ4&Qt+j?ep|vCX@YC2S+h5Y)udo*u_Qm*eL>!TPszDoW7f zhb0N2N#URRvtal3vIG3 zgUAb++sZnzl`9aO8w=gd;Ld&Bav{uQb{M_+MYt9CBh4h$r9fg51EN4xA?wVKKKo)! z9a9p#=)4^S8P*s(l`;M3@@m%)lo8K=&@d1iOUXJfM8xaZ^q29=UhBcB^1 zK6QWwVB^D^HQV@R*%#2?S0eV~9^ zXEe0&P|N#oXOL*DZWLie|1(>j%jWg!s(ZJP(E8EY1)G<=i6U33%Dj==D9U034G61< ziXl3xoAlLc(mdUpj#mBLpJyXz*-Fo|%fl^frq1pu6|7%aGHi-RuF*vLnsNNLn=aUG zM|UTcD;nemKqz6P@v{1GRZ&F6t<%6FnOQ7q-mm@wC?D;n5fgJ?8Y{=GJ=Cw7v}o}v zB#~H?(0FomR1}c9Kgri^!k}`|>`3*(nW>SSA&NL*EPENIBcCnmbMjJUOs>UhYj(^$ zvT#h#U)e<21vnyon0zKt#K)lSnI77*JrFM4K{`4VP5ac`Iu(90#NS6-Il3{dm@$T5 zsSWGHRjWYH<-$2}d=<5?;Nh#a-FA4)Q@ zo?|o*@BNK-O@QbyakF!ePBP_)EA49(gV;{nc2!4IL{k-S#366&L*spX9YDL~So6mS z?z~`XKwSxo%*}ec zg1Ae@j+mI9Pi`0Ygm~WK+KRm{Aji=Mf+gycG3ONUh1;pyAdz?71`q0B;%4Wn{^ag9 zjE_KYPTlS8zH>@_-=Zve8my`e-Oq+@!E#4RWxti~cP+(>M7~IAI8t8R;F*T+Ifr0PlRzQt4NC}rhc#f%dcRI&iL{5AGI9nuFj*MR^_6 z7>UtbrOb7r3sVJyNeU3lt6b}Yb4OJCwvHc4tT)6aFA$ywydYJ+-dMj#SFbdA9W4x_{-R^da?dFH>Cua@+)t( z_mFFy&Vi*c$9?EuNQyy|+S-rQll8h7Qt)8dlav>;uZ{$nmzANqIMs>E*v-l|XO0^V zJTjBV-ww^mTzX%e^x~AQOt&iO`o4?uk-9?I=bCKACja{2v-+=QKH7a)Po6o{P13v9 zf9#PcxFQJV%fm=v!6AyH1F48f)Nk;Q;;BqA&8Hm=?~~!kGoWi>XuV-~EUA-NWa&7{ z2RAgDU7l}jsSY)btFMhj`Y^Id+X@Aa*UD$8Bk0G-{&k>}A)7jg;tY%=@A$t^uJe~V zx7kgB^m~YfA8AzCMhcqUcg|LLsMUUBj$|j*peFfMie<>94OJGySh(<$ta<@bFRMNS zix4;?A(O&7SQEys%DQJ{Q2M0ZMHQz8KXbe(>+-;ZQ3ez(s%NA2_o;9ek+u0TWqg0X z{@em8bY9uRjKWLxr{ZC)9tQHFEOz(ldz#k5j|8WY4724FbWdJS@pa4P1<5O#&ob>% z4J3a7&Qodq!1uwg*sicnN6u$@Zgs6IB&w)%jPt6*Q}*)x#ZLw>O?YG>WQpPRJI^0U zD)mbznmxAcV)HAO{>&jaiT6!VwxhhJY&TrhiV&8LqlGz>kwF_!oRJpDT>Cxw;6m~8 zt`8TnGGQpTp?AGVpZvIrUK(g4rm!zp6lv&vpP@+Ebv-u)JGMcghX0H)6Rt{cdD>8b zq$|{2wC`$4=*$OlGy<s{}4oB>0b|JStq0cE%4(` zQ&{k_!|6_sJ30?TnZ{$@N8LOU!N$0*f z7E`nf9P`c3dx1%LJ(>-VL0N^~4g{R-@- zM%tv!XXMy;nNf{BshuJ`t^;o2xbIH8&x3EW1z(AmL0iW;$2qaz{>gyv&>PZuRkd3MO zJDi6fD!sEAE!wd%hL!W=S2;tgh~_!;@`w11%(!Bf4u}1rM_eU&d47y=qSLZF>6|9h zA#lDv;ZBly+Xswu9CmTP^~k1%4al)He*~N76c1E7T!2;!1K>YQCDE9^Dld#XM+ouF zcHMxjf2yc^&kf59tLf=W<*3#sp=dyAQT{k9+I{^fX2{E>Z9uJ96JTVRC?+W5JZjqu z)$CM*ax?Io94&C{ePfCf^?0|kh5+?u=KQKan{*+JeMHOhf_{yE9ExbRhH{s5lCB>a zCgiaVSM;LGDwmcf_cupnXFZ{lXgRO5dxmxZLUc5A6&ITvu+|-hg92aogdfvcci8k zRW4o_YgHrh9h*v`*S?Z`W(cuLoo)PhZb=|zvQ&Z^#%;k=v&)v4yH>oIlf4C3y*4HA z8=@b4`Kij0r?`(U<8Y?QR%PC>$?di&qN}Rs&j*ZD;#wb-j8Y)HXQ;%m_p?+nUcF zD9l}8SGE1A6W4--uJNuh_rVJ)oVu$w%Vn*Pg=8bcVpKh^*pv$!M9(r+Qpw**ZBT?! z?KC#q(I$3|P4lqxZbz0BoVKYhD!J$$1+csDeP|JNYH|^d2t(Corz#6b#8&5l<^3pH$E_wNdju8;Okr`X zD)n^nJJVFgHt&b%(^^_Qsae0>?)Q214f9&wt``?p;AgE!Y~WqDL)&v?2t_r_yxmcd z6Md<#&mj;AL_U=j+H$1$jw~TlrV19a5G=YIJ`KZHL}S=i*J-o;qV?@rabPa2-ft(f z&Imi|;d7o@cAf*%cnJ*;l%C*9LlgHIXY|@I-V#)f=<<6FYfEz|KkZu;=X`T0MDd&Q zQ|XW2@|*I`VmD4Vp*q~2tFFz~@~Vf|=!*L)AJiJlk8!kTNuA@BMd8pTljg5@N%7a# z>P$!GG^-b*Ay3%%Zn@1kRLm+*MR42Cm1T7-Pw#7h@Y)jkTzd$Q;G1XJ(-(wJkXP7Wpmq8N+hUQPRw)v~<& zilBEWN3qw(O4aEsgQ-e%8R+I7d65SaJL6tpU4&}&iFeyXup!gN9yMa)?oy4LPDIBx zf~hfBP7pg})@iTg^CyiLs%Sx!>IlUVS+lNINeWwl=cH{Z*`cc}Qjo@n8YTQ2_EmA7 zW&vMdx*pOLT8w4W^qPk>#QkMEPDttJ8NH_N(VSjxaX8DZf7nEuvZQ!W!()@H^7B;T zB?_57ea7Q45z_t0-XsqSCUQJql2O0cN-jpa-o_u56&ms+>ZI<m}v}wNxQ}Yx~6#{jKQu`pq`FmvawWUrr+B9T{<(6{v}8f0Sm&UViip zWaWqcHBMkdj#khP){UMe>V!3b$YAF(elENrW0^pt~lLn9FoQU2)T zU{@qZ+Ej$6TdO2ChB@E%)AllXGVb1F)~l;7iY`3CG3C^=Y5d-s8GETg?2L{#G<7*k+)03suiY{ ziJ81&99n>RyNrW%)FPU4UCbMi5zTh*>4qg}tOJFF z8U%(Ud0RbPMx}>e1T~UJWJ`~U)8eIdqAxx%iADSEk=!azi zH!Jk!PzS%P3k3!#^o^<;94Q1JTIxhjTxB28GFgPvw=T361sv>I_^Y50M5ppKRe@3d z4l#0RXG2$rq>NPI%4}yU>sLj{kWyu%468CVRz55$Z%0 z+>H1L$dSM(oEcTW#r>5|@ORzWt{fWMzx_;W$HX;!{5)Z}P;we{h2gWW)I!eO(7nvlvz;-36|imF09*JII#z2Hy; zhKAbIfy9x`R11AgL3u(7B17vqwnnuvLv}67KOI~*k#g^KIMWampj8cuE$V2%LQmT`~wFwKEE9FKgOAW3@q{%EU2zno<_p)J%C>2oVW`4ZAU&#sDpagkuX+3wrfs+!b;z*f zmuPX7Tm#}EvC38DrzwK{QUz2%OQISk`E;NZ-Zg(`=Cfk`(cLK^GLHqHUkr9Rj%puj zG8co0`M}`S1SAahAv3;x5kbLN-xZ$grQ7U36X6g4hC)n7MzkLcop`N8MYNX<^E18O zE8f0YQ(~mUMi2K?#*jsQczI6!v?7Fmvv!tkVU3PUeP{)n9@|ZQ)5bY1U6o`qItpJ z)bIS3e!Z-$(oXiU?GetY(_M?;^C^t_PnnJiRV29I53{kA%o%%a!p&O2W*sl(;hU&c zW@9+=t|f^x_owaH+=b)UtP*7MuIq3}i6^)5N)0IVIyQ|!6G@U?i^qvr#+!*Msvi*e z?zMU!F(Q2BEUyHDrb2H=FUlm9aFfGKviUGXe|6h$1sN0d(*La9qaC91YCH|c zC3x6M6;Eff!Y4T%(ZJ8%rGBF|pmU6%qp9n`WbJj^q@0pO3*@CA#H<3TvDx4YYiTMG zdbfy)Afi!WgC;8VWGW$2_qFFOF}9wJFi0cu-}Ir! zMyj~UESH<2VepUEPtmim}L%?5!>DjOTIa~4hhzY z&quOz&OL$opN;k7`@SvhOiy9O7^#x9=*iXm>X_wjHa%MYieVyZa4fPi5)vAez9K_E zCTcjy(HPECSesPI8|VJ>3rX8AmQ>}h;bfNbk{nnhRGQZ`BMlPbXeQu-f{ZHKsB+Ke z{4bb6!1Rs%V)nf(7p1mi8P38_6I_vIRp!GekNw-APg@HkcadIi=nC6CA}*K(di|5R zsM}b-2gw+?(sC(H(eGvy^>_dLw!vvJ+cGzX-o(zA7dg~!-9B>+KH3FofCTPIr$Mem z*v!kL2XlCLGApSMvq#!C&J)9fEP5NCT&~jeH`z^PYNm_>wGpNS_w&f_`XPw3rQe(L zzF3Q1R95UhtP!d&5<}Uue6z7~sSOLpwTl%m*bW=ud;U?FT_xB8J*ShCDGj63)A z^B`~B&&m$u#%Ow^x5j;CBHWy+FA*TK(KFng>ptW#i(PCpGYUZ^nL8-C^6wa5_e<;< z%DBdBI-MVm)mSbqON^l0uPTMBm&5rn4P{YY*@FY0Ynti1iuBbh}fHrziZ7|KsSa zwi{Q1DEdoccoZ{(CB2E6SzkY$S@$I?6W`eCsycf+R`-sN`OtYI;v4OmLw^~i_H~HU z+sK&@D8*l9HV8yirhnUeX>SH61SW!1&}J(x{aN0X|81bROF~Cfvq#@f%H3Nwp0_v? zf!?Rk-c%Ew>on?xg32zvUU6!#`YZy_5E0Q~Ut9Ei(vM32gbT zNH@~bQl4++(c+N^`-BxHmXu7yv>N^IJY^90wCOE8_CuRhF`^HU!=A~9xq2y~ zo>D7`1Z{fbtcc}E8{)mSsGpj6%-YJA{SEYdKFROV$qy2|U7dlZcMXvKLXGL-;dtuF>g0Yb_FZU8+%!oPzt1vwjMAAElXMmQHMWdaN~Q?0tBBGXVt&)P;2^LtgpsmUo}YDOuhp(fsKV50 z8f{#n1hT&gb-Q}fV8>s3ct%jtF{uc4JFYpSU|c8K+0%i1y7Z(*nmpazrQ+aKnpqbY z8kN1D91<>_*Q>_vKf?;JF(uVI(0Dbb>|fR%%I4j4Zev#5oLdr%*TRg^IS4xcaV}c! zrsO71q2Y2eFNW-G4BG;5Co>J$vK#Qp;XQ_q1Cr*K-^mfDy*UV;veI+bGxNN_81D$h zs2!p+(y+|e6I(v=%hO+B(-P_J}T-jko-pPo&iCfazsve+9(hOyx4#!P%Nm&f4SaOwECVbC0 z3|t0+peB?xrrnx(ogqGAA1;PoNY3XMLxYDILIAb7Q6!l7sqaqTQt?s<6Gsgzkqx1> zWmGpMt#Q87@yo;zJn^siZ7e>w(YmNhB-xN=KO^(Kztp4UiB1=r!O_7(EYve9@q>>w zAMUdr@bG;;#?p9sv?!~-iUW}L(Y~$}j&dTXTq%})(2Ml3*Zk4V^<_DOtagc7Mh2#+ zL*VA@UVvNv(lRxnFH3m7o_Iu|zBf z8F&e0(P3UAM3ldp&=1O^uSw6D_M2RUrTxa*qk}%JJlSX5SK3w`FmcQyp1B>}3&Jjv zBstTo9m9##HNywdvDxZgX~# z+P`Ia&W!p69uw624Nf8(=gvLh1PX7t zRHZ0+xbM5kj$ltviENsOi#3ysMm7uXpY){@vi4V)(km!vY2zy`uTzL~l-~6W$B&NL z`;vB2epbaNdh-&dhG4H9NdVI}JF=6$BIfArwd8@e*iVv-B1W%{Lx}^JwofIwPM*>m z9PRltk0mvgpWTqb9_x%7;|iw7yYHlNK)IoQpZGcKEN+t$>_yeIs865sz)P#4S~(FV zRUsBENAJyEXJ==m6K$O<{X4fR2`;%f%w7;T9U|YLE5Ea=v4{?$R?t1-k+Cp~FdMX* zh4e}Y(E~871Sw4t8m^|I?t>aj$N8jo6IF#pfCr|>`IY%99Ck7Sw zZ37zEH!@uYhQx-RWBJ)?Kgn|@n_+d+%l}RUpekOsdM%}UHGeB(eWc>Z>Ud(o(HW20q``W(ceKKDf_^(-&N~-7WQl9q0Rq zO*W1?eX$1y+;%MM02$lP+vvB8n3KH+i}>YdVeyf;CE4=EwBMa2)!V-|V@)iGry-rH z@TepM>X&U4I(jyHqx05Ub7rHJpl_+}g6+UqK%oO4e*{f(JU=wr&e-~isiYpox5Efa z2=dv7ZAP=#=78yYleHy(zS#(7`o|nemWP#zMPQ=*XcC&t-BVQc@~aj)l2qaJNigHq zX}`Ct%u>sT1}6FqOoCx0_ldSm>dUnfb#=)gYPXmHcFPd&QIB<&U#_Tt27dy=-c+Of z_B(<05+N85t@=1Ne2uq)+Kn*Sra}$c36#bWdgll}FQo8$j^4z(S3*{}K&=Hc|30`X zg!XTv(S`V}u8;nz0wP;zEU?kzAe*}$AFBo1NQ0++fd1Zpm$q5WOe zhZ>5BROE)GO>!l<779X|Nr0?y$3D%!^8-dB+Ow5{EvYVPT(2&dHU5mM*gLZa#AABz z@36R&QF#wPiBODm;jnlJr#k_F{`^e-bKITzEI+=#wwL zkv{xn4VrepMV%}u*i5oQOUi@e9Af-)h+js8&sq;}KD_9`d3D+m6QCC%1MAwbk7?p* zY`dfm)A%{d05^$OX3B~-w8kIMTPz@B+FBFmry}Av^$QT-Hq|pg@?zK0zFw2hZgC8<`|2VZoA|4n!ouB^Ux>LSS@%vbc!lNz}8Q%OuGAj^4BW-Iq zC&d`SvyV(cAYm*%h+zQl5sHi&7(l+Rd_Q8^VTeCk;4{97gE@i(<{?^1&0Nr)^8upq zbxI)V{Ub5W2kc6oxYGWrhX3Syl`6_s@8{Skj8UG_>ayP zsD*1MqVVLqk(f7f&e{UpD~&PVr@oG?VG0khpkqW-yvxb=Dy@>I&smX`J^k8o4pl)t6|Xy z`CS>lD;(6bLM=_&UST3{7*roW{e8k8>^fX-YU0*D_gyIgD6!&+2)jb5hW#xO?<;nS zR%F#SiH;1Am)^ZrFc3n_EyeIcLa*{vHv`+Pyo>BL*DCHz1ssEf_5@o#7MrIsOvA-_ ziE4WDJ+KGp)klX~E(iZSR z(cPIA(7p>QtID?Xd$eq@+4em^KVqlT_j$_7`Bgr=YLkk}Dr_yuM|IJywLu(XUt+a{ zrZ`oj66in1S!6_&ktJ#h;(qO16D z%RO_TK7oqUqdRHOEKa?9n&Vol*EcDq-I8SYLSE7n541MiXEJ*u^N3dji7iRN0r6A$rQgns(`_(CAOX2e~FQj5# z{a}L@5MAFBF3Oj4=5*8C`b2lkR){#YVX?<}s@T;mLq=^0hV1UvTyy3#SVI&evZ^#`lMK0LhBDAB8$nS)O0Gx(sI-hMV2QjUi>NQ4v zL)RfHJ_;B=E24vLM^c)qpsm@NKA3lXV4YAS<+V_*ONR7fI9sudt0_XGOgXgEn;`T~ zWS1XibyOW33NV>W%a$?_bOKPar|(x4Qltu{w1wH4;eaQIyF zhg&!1*b3eZ#(H~RvEGZjjRrh7{$73_`W(0?W-qeH9|AnCL75F=+N;x_x$MPAUrt&% zk#yUv{iThcIlWhR_0m#vM=_w=$wVdcr0Lz5n7nr6TXQyalC^?;+veP>GdLNpPWdY& zqv>8B?`maAK-o^X_WTjpjp1}=j;hTwC(ifgCLqh@hMvq=7}2Y^SujkRFZ7RoxA8)X z3K44Ic-QQDYZ_#ayoWvf3AEIHDIIUUF{h%s2w`=Txf;QR{Eh_=DKr^>f&Rcvr3wgP zT@llARpl#%mW!@U7qu$w?kXY_8a%{h4@am<=nVaxzNA$#NZtjKdr#+}RnG92f_4QT ztv7T@`#pVQU?OzF=4DHMqgJCluj38;O1+xA#^6~rLSBLDv1nnELc0qfiG%UP+5DOk zHq&sT(hCX&UcZRD_-_P;!lE*6>=1AIXymP0W|F^%i%u;$&{cb$RW0N!#Hz zzk0Xp3HJL1SW+CxWDD3y_DL*0=ppE9C%sI1mKu}N52?Q;-oYd+iGH)~U#*9QBdMDW zK|I^k+JhJy9-X3f1#l>wmNeFWZd;Eqnu&)C*PSSydOCAFYmf6F;v1a8;oN^^hgTo^}7M#%Lo@ zj|OeyeB58vEM@sHXVO))$^!qj1!MAKcZ5n6oPkKmKEu#qc5q_LuiSXNSK=m_=NuB?Jm`T`O2N@1 z=S=e{YxpB)k9&U$-uOs?PtgeSuOTYcOPo4OS*00w?iOvSGV~s&XA!v};f>)aF4XuM zO)hYOhOQGLfrda*fdc}B&pSswy-v?dJwEKDt;n)6o{W^(4Tbe(MM{9hw~a+LF{j+d zlxM46q3^6em%c-O=SX88cX~9H>`16!_;?66UV&thG|5?ACJtz>eLDXFch@qB1L3<1 z)vv?UK}(lX@;Sa8U~he}*R2ml4p3D~ZA+st-Lv+N(=~MOs_BuJz7>yWx%Lk94S;pm zUMbwFax>D;%x~eGj*K7vo8hE@rZK{D5*Kv8-C*&DdPYZyWXq*%q}ql|TWW(sgWblB zwRE-Fd03XV;_7qHf&m-vcrtusN=NN%kWoV1N0z-y~VV6{9ob}eqR@r2Vjc~HU z9H=P|e!C=+Y?HpeY&yCF3e*ea6fDOl*s@NCz%CkiENCHrY&n|M$HYGh+Pb$Qqd`wW z;V*rh@_mW!Otw)IJeXniFS6dtU#ST*`caT#3h0%&Rd=aTqXbcxHp$_Owk4vl3bC-^6~%|{It*QW$Ztm2S17aWK8uR_FA)mr8{cYZjrhegh7oo;uM&v7Lr<>v@PdY9Kgx?_HyFM+}kl-+;@M_ z4abP%MzQQhE_=C98w^{dOxCY3qp&;3wqN^_-BShM%yUH2I9K~8fp~kmcM{Esp85q6 zgv|6zmtZa^2%A90?{o)Dm~;DAS6;13#C@GTL8hVq zUiLBa^AknL4(Re=kA{i6{jdrq;f>SCA}nPD$SF~$Qd5>a3e{+w#fqN@dFcUZ+QF%% z7s?Kc>}H}+_xX4J_FmGR(zLTE3fB)UwN3xK8Pu$KShEY5r|hGrUIJ8+xcT2K-Hy4> zl~w=ktrA!1f|SA%W*V^tt4ItITwLJ4lbf14QxSD`4f@`0Y4%mjHnNVCbG*Vfn}Jdt z^6Av@ReRBQN2cDV3(X?fXTmqk0d6=j!ia>SnwP5?5W&m4uCo?vqbD%_wOBT%K_Kix zj1@_;Ve@QYeqr}a)uTm23Of$A5tvR zOhq(KwwDLI^YbM!6U#U9SlufCKKJgmkHakAM)30bcbjRjwK?CQI-e0*wJXV|)xs1U zh&k%E>S;AVM`5pz+|B z&nM^dT(t#uF&LMcqdo6zj7i-?HSy~Iw!AdD$J#eE;i21+){@`GyjF30U0gZ%%~s9- z1Z_vwpcGI3cM88~+S~Xin+%M*Hc{FNmli%a~q+;#0(ZC=ELC6eyyDe~ekV zhvz^7>$lg&IXj8snS+s$1sghzkz|;z%q5o{L`6DFwVY@c=nT_^P%2V(OYD+^$F3%M z*89M}6oe*f1IZ=)QB_lT z_&c#EP{cE@GHk($pSbOW{~QqnH;`y$y_7pYKDNby3^rTy(l7eb;Et^ybi09}-%^rX zI_{J7bEPi-Sv@w73ff4dJX%}r9vXHzMz$M6U<5=!?-D9gybWU7ew#vvyo%IF$-xw| z4D_C_=%PGzW<$0R;gAZ|jQehnNZFk*Uz+o0-1!C4P5;TctwSHm=(I3Mfu|TtWGsYE z;29x(7*ofCONs^O(VSFEU-4um+ku+Oc8r~6F|jollR(J?O0aBAkbxn~f=;3ria$dA zGJSabmLIq2Z<2Ci9Cw{!wwch`q+HaTw;vUnpl@gGqEiQp#P^6VF9{Mzp%^7RRv%w* zzAPqpUO_lHRmAny>X!G$^i(N<(j-dC#)Va!gD2f|{I$J;zO<)cWUN=Vop7^UL2AS{ zH8x${J$6*ZmgQLT!iSJaA0OR|5`D@I787y(*b71CIFAurOp`&PN8nd#kU0%)39N~^ z_nUH*_fZ#`5GHTRwujb&*U2Mz1q(!0^RA zwXR?OG@d{Brj4`)dg80;dH$*r^)7ZgmyshoECm zLbn3lYc4zsC4_2-UPOyL#n|?f8QA1bm|O-f!gHgYP5itcBGTzy|7~)C84M+B>rcaQ zNd*&Zu;;us4`tb_bCU`XD1jFsneq<0*afL%j z5EU3>l51%qRNpZuBTYyV`$;r*zu~&#RT~9Q!R=*U(;qUhE3)r-ITL^zH@orxn;KU5 zD^!+|%tw;3(L}ykd7dYQe#!C|Hk-bHC782N@VtRDUY=03F@|3|FG@k4BuH(OAYVHs(M=d>%2^ zRy2}Zzj=La$}`lwn^pS`9;4z}3Bh!zrM91_uUH!l?6Ue;kcX9<3VHMlxhhIXte!;0 zqxJ~2xvrHaC=cjNXI811E!?DU&N0tkoQ)A--f1$A{3+9f%LoyTHBI-0ku2S5D5imK zSne+aLuOMfLkNq&Wx=jp)1$V_YlR1S7n*KUFf9Bcwj|`iDT4)bTs~Z=qU-Nh`qlGW zV~ah81nZVd{-K>x6ZT%21yUjf;O4YpVbzd-wnQh(4?6O4DSQ=C`Gvd z_UCeG%T|}pSUFSGEqpzp88E6H#Pt`hvg_KBy#2awE3Bvt$XtEspK;4~go+Aq(=7c_AkH6&bqVkQx-(SZQ`5##C z$h6bzpfL%WH5u}TBPX>wGJz1f&hVo0qFILDN$w)T1`D;D$tZ1ldt&=tXewS3`YBOQ zMD0$;Z~#U_{M^2cOmt8yEyc~yIJ2vJ zC!RI-W=E5)?$hYQtuo%lW!Ld6l!8!xCucC4Yu$~4QLF+6+IU?(^cCwg@2mB}XO*4X z0Wna-O<#@kL|8R>bF(>2bCERvQPD!#8_X`XuhE#LMhLyM;L(x*WwNi*@Lp4l#Hh(Y zHZ+myA4Ay5DW|~&vkw0V7UOR3BeoLvzC#8bt^+v^j-wiCJgc3dq-P(WY5hjzJlo+| z;$QO0{)C(pH5q{s>hO(lQtCVxyqSD|4K0+zL*&Hgy+wk7mcilJS>F&%8`8&nuy`?|VER5pR{hCeKszM31R(Bhi7H#t5uJRKI?!a??2c zxf|yX*^0n4-5$ZakTMH$p<$5-MuF;|$-ivafo8AJN(vTivwEBi-LZ*)h55k0^ zNe)bGqQ>xiCbx_V>LJUax%KVL?)#F&dTrHh3DKp%pgWIw*J&cIb!tu6fC293@KU;# zVeB^+!l*kzL`~<=V?6pE*)w_mBc145z~+GY4VL#T#TpE+N!G@#=+_ne{4%ZcX%Z!$ z90G4=VGLjrd0UFy0gQ}67BhD|R%(7KK7;vhD5`Sc!yCd0fswq`D1*y$oy4LqCp9+s z$aBq@+SM3R9U;pjE2ZtEHZx);$_aZw{b#$Sm<1YE9nCQ{u7QNS98tRri3|Iig(EW$ zch%>eEp^;WK+*b@xkJi;uhd2)dLNa2YP@{Z5T03zjSOrL+ix&MX=)c_0z~vj4H?)PecZd zo!|$>xg@`oaDG_^F3x75fOdcRENqIwK-%|X;!Kx zK&1m%8FGH-F&IW^B@#rv{e_j&6A=}QKDMaiNF&msAo%S|6liKRz`bLX>-Ax?BV$iC zd|;|^2G8+*lm2Gv+7cEsLdHMUeM17wjjd*NqC!}ZN>XNvgj^zTFa=EaNmHobN~@Xg zuG{EDIA+4#Gc&S#eoxK&37_@XT=eIyrD?>5n~6XtGR~r}-8U~woTbu2LT*E@@9#3a z?!)$|@cLvxv@orEttO5x72@urU*rlkSDn1{%8Hq8|JuY9eTZoM`HGD=O;w$}FrnP_ z7QL<8J-Yk;y!;q;O@74=I*I<3LgF)sMi@Y#SDE4?tpC2ohg2J;X(mBC;k+=Isj4}N zKQNINwhG0^l4MiL0X@L%iJ; zo%^XY&x&uft9;j>j8S}S2UT18*Da1cT{w8qqTp#qaWB#bE=W)Xs6VF{f~yUUp;q$# z48AdIpBGSG0+CHK%35u_9~_=Fs@$6iGWlJ`{BoDSDfS3v;Vi zw2^eSt_%yK?FwTAHe=4fs;jsQ9#!OFN>ZvHmDvLwR0_@Cr_=Yyre===kL016--ml?k%FY!}d0t%^K^=o6bO!Fl7dv(g4H#osqUrluyb}%i)OE(q-SVlo3Tq=Df zwrXLX_k%a=F6&iMNG?MNlQ^+GP-@|ZDSNieIegTMg)Pj<#thicsZrNBqj${E$qHFv z0?OiHXRQay=!dXLQvK&tv)*xF=vN4KwEWaDQ**iQOO)_;$QH3MQf%n_yVCe7<1t>O zvspupNH6lty!kH2pn`{w@*1O->+Sl!t3bWyr!evYT)yWrYX@kVpk5S+D?pd36CDz# z7*%atQ>AC)0YwfFSD7R|yT@E&ULiHyt85;2_{AFRCmYcKVO3l1E~51W-@LAuxqZQ^ znbq#(fC1&ADrjU&M`(|wGBB_V7ZTU=zn`@$tP&UV`LhtxXJD_ zNE=~%9z?!GhPzQnMX&T4n_tTj`;WqIp=1~|o&72KOTxs}H?N;!{EQajGN5-WElc58AsXz{t5*j7K6mBNUC2CGz|RnqWn_iBFhdzE*6U&0dYTnex(Ja&maRwo$oULAEPuqT&az*3 zNT)~+26ndnp@U49ITY)ty|PE*7;X*ZChL{oreU@cpug*a?1#q4>o0xx88!Slg>Vr3lJ_n@epm?TfI zfwliOw|F={AAWe8zQg(kG^O!@;Y(enl+-WkMgTAum>XXuzJM2?;CCW1yI*yStj>;5 zHduRkl@d3!A`ESGcKX)NV^&r^J`y%a2bNo*g$Zdb?j0tU6HKaL!Lp?Dne}+t89sQ- z{#i7_LTV;f&LWQB+ufxodJt*GF>VHRc z!@3}{ok;g*hU!bpVQ4PUGW1|D8wadM?DN~LIYIgj z-o-5Ws77dH@={umY|bU)pLJl0l}i=^MX9W(-!rlWd1EW?8veTegg|pA8}htNtSyk$`bp1*@h zkBP!+N3w~F+nu5DMG1ea(AfbR4AiIKt@0^xqqrIid2qZ6*l>$1B*lEWKEM#y`8WE@ z9F!;DxVGH7N0#p|K$%rHdx!YNIGny|}F$^@KZ6DT|$GBSJh6@e;JlQxCKwa?!;uhaXmw&=*!&S)Q>ysN!$0yFWCUXJTX1r z{$5}yWWd*m+wm8p(k^xd1*E_ePA;C_sS6&gmz1-bfGO|j0QMjS56sJoUb1~S+$DIC z+y;|oLZ2fgQ^7PzEQ*A0``y=p61t~r{ofr^zIT!zjv?`oxGS?)v!4tu>BU<|JrT#i zexE7bIMz81SzulZgA06`_I(%1pLA?`!AxrInJT=zOC>i&*5L2KSNiXl`@Vr5c088% z@FOWl{|?HI`#L3mRUIjP-q`1mc})`h{CkXKzY%`P)c)SFe@oo16|+HR#jsWjx0k22 zkV1gj<_XFl48Y`m8t}v4GAe6^nuds5UB9i;mBrs=#nhxa3Q7Wo66XDCA? z)DC=F9A(TU+1v&h%luWwrSKL}UC=8u$(a~0@{_$mXBl8bnTx%{G+Xi*;h~u1aVO3Q z`KJ_iZKDa5=gDq2z6bc__~O&MlCx)D&6BpS$+fy(fIkfKvBUF=`%m}WTSx)wmX5$g zD+3oc3l$FIMU>gjXOfV>3iZk~_|*UnrmHJf^NEzWZ%WQwT_8kF4OB&rbl^ZDqtt== zh7Iq&RW;cyT~p^mv5Fa(5lhO;T_#Hpi|jHdg)vcLFDrhK@>n)Y-jLhQFW1t{9Lpx= zSCZY6lx4wF7|{%>jqE2ZU)ab44*eQ}uNPMTh;&RgEG@!}78yhFXWb&YoiikSlyhsZ zV9XqN&eQUccho41&z1={y<{LfV&U`6?)V1q!TYlP?i_l{%&O^un#Sdl6;JJ0AK4#S zFsmgV>&7YA9Gi?j$k(16D<=q)8x}Nb*=?SB3Kw~yi?=F)8N7Nqj_tC>j<#EWmYX&c;DP%bLPP8yH$-c{a{{?75hhnbD3x-(@0lf(8) zsA1FxwzVv|WLJzxlb#)R6R=35$#LKz&2~3J=-?h5s;i#9R@uVds-Ye)1=H6HZ2lZ{ z>{$vw)>-38)D2n7_8MZH=UmHRu*Skuz(PVk;M(4iOJdDJCp}O%@lr=(l_3*wde^^E zhmIC!x^hQ!_&K#^FrMaNE|GI@89q1tC!!xi5!2t>{p*i_?dmva+7;^F`9Z$+cMiL? zvA+s z19cNA20L=c*vMcAsfa7Q-jZnOcaK1ceRX?%nunE!jXlj{it&^i+b)P8I~>McZ+kT8 ziP}RxPil7=TCz>8XFXD1^!Nf74onIKS!}0|8&#Fk%>AR8-h8SFEH^z}JbN5I%*)=i z4>}~9%p1-)!9Tx;xzCSX^8;`5lMdQXGetSnlJA=A`Ki;itH2;5Z>5P}Ox0Z_L8>dx zXvp;?5!mQn;`?Zr#?MOa0oyUejq=a6ZxxdyrDs`< zlS@$J%|>?xUc$vxhEC;^tHE>iwVkzX?|aKOnB&u%hg`2)6;kWIV#M|iw3A$sXO-Bs9nk=~p751z$bHRc;)W|jBGAqtqdg;(KY9>lHvS5%9Qdk1i_HMQ{ z(c%}f4h@>E;U;gTQcMra6BTpk%a1Z>Wl7ZuCf3e$;?nErv4OHXLj@jI)B`Ou9Cm<= z$^@*tFEBcAXO3wFRRa}6q`ND|wwi%P&iT~zNnl}8#IH`|l9^uI@wAoSez+L)V=(iB z4C1S3_`>4ev-s);fvYr^@y-iuRGV0LF&GIx?0Cv0FtO;dc3mYnU#MmAKC{}#^wt$p zwEf((fvyF5<>4wFa>;L`A*rghmXk<=ny0U)(k}s4ABd1H9ztNvSbPZXlQ8J=h|V>? zI%{DQT8(4%arJ0<`MvRvPRDkef?uWh%>E(y6s9L5xT4`XOIvhLE~qn}hAXjWDIOgS z#j0zjiCeDoZN4S##Ga_seA4MY-#~_jkax{qiuxK z8J)QiIc1PP-rnN*7v#P=+jsJj{z0$BuaOU{Uy_Z?!L=7xhjXSV4Q+Mm2r}KxD$w_F z*Y$*&Ag7d&-f5hGYD@w24?#gL;>}(1 zc`XHot)!YCAw|8Q!hCbq+E~)-vEc+-$AohF2*pt{_ugrX6SX}yjuW?3Nu}c3 zDQ(R$?W@^=xp@|Qw`fwLMzxIdYY@#&v<@@9Qezn<8uf7@%BHnENwc-U?=DFde<7JD zrtMx@dF?}6YbK}d37mx7Lk0SKWEBUqm}gXi1AviQNW8T#BlVUztdW76qzKN_!*#K+ zWoq<+55IUAD?I4LA(AsX7v$T7N@gg(?@^g$MqN8I0jGK3G8aV_zS9os#tIj(vfBx0 zS4-TYy>I};1l;9CYR{(3m%^F3-ozFevZcT9eSn%?6%iu@AhsX>JQrCXfn>sR0HK=~ z&@dtQ?cM#PHRU{=^wLBuFZb{Kl@2oYbOc2CI~(>_#Sz>embAV)Uh~c0rgUEir#fwN zAu;&1&m<6<=Bdm5oiM2k?(OBT+BY73qdUAiII$LHo&+fw&kny2&Z4C?z)6wJF2_A6 zRyK{uv{;zRblGq|lHB3-;`W;x=jckvzx&24%(qJx->$PnMTuQ~R}9Ha+Js`+4F2s$ z-NHAxGz(p&QJUKLyoe-(gB{7%-hsB|i3>!KUn6vVFeseGr3^)H&>{?$41ohSTH9eS z9cJsreDWjKDw`VcvdCXcmg%!Z-&mPQZW{aK0{dbL&E_`k`fxdKpCkPwG~)_A=}med zj8o0i(PS--WN}gX(wXHtjJzyb+~-BjG7ks;vg9&jjVLyRv+Fn_;~zuI{s760)4R?b z6b_|G@0R1k{8!L&4$IA9H_EaRVj)^9l zK$MB!J_X-kR;Ms}EwkC`p5|;i{(ZFecm1G&C8(TTTQ8TV`p{!>IuFXqPFE}mH-PcP zY((B~T4x~{)p1r|DXh)00=Z?Y-G)?;7b?j=qDhnR5B|GV_(pw7x?h&+7>>< z_%OqRHzcb&ZPVxDzpQBahC&|EQF=!)l#k(nT)Z~Y<-rQJAL zLVaV@a`oTDLCs?;F+U2rHO}3+>Kl`+elp*+;y*(vRk4^07q+px%9gF$x)R7?BqBf|;B8m085+1#b{Q=&>C-ZEajNcP|Nb>wyY0SO(UA^p8 znKd)UdNK!9Fxa$M4pu?(F5}4)%GFKhSTrI*{FZeqeu}6?it&Q_C3kxxOTtC{ae@#Ag~RLSOdhiK%<-j$NW% zyqXIxC7123EYD4)2bL`0Ip-pKWF4cL$yi--hUKW&VQ+5d_Vkl?Rn*>3wOYG6vXqEBB~ft{PtdqS(gp@nwx!xBifsT zWOITi2M0Dv;i}-#FB(r;>`idV4ouG^tB%HJ#^^RZ3AO9BM{gG9jdsY6x2idMp)sVT z)vUO~VKka*-ka53WVF966t$@h47zw&d!>*i!~EI1K-UsxQaFOprE@}D(P^a#S*Hit zxMRf1x6hPADmBB$uNi+L&yq>(vJ~>@e<~VV{s%qqEM^0FBqTh>ym0nb`a|E$ZIi|y z4uRmS{S2Ap*S645i7d>2+R&NoH81GrA?nOo<~;+7<5*d3duTIv++gLVxJk|Nvv#e! z_e9=`Q|c1*{IQK6)juuMuWDv1!3WL2aNUXxw+~+t;Wro|az3eND#{NuKUVz|V~{Rq zjXR%Hmg0A~)Z(}vYKPt;+B0S(t%)C}sv{$l#G(!Lr;gD)K$k-lAVZUu$qG)cfM7MZ z|L*h!9REXi09G_1PdHgfhSi1olgJz3fkvd3gg}KRuyqHo&ym2N^vChF_J#%8y;|rr zUyf08H**Z95!a(#=^!;k=ekieFNPP<3Yc6{6r;SxKEW@nb%-)HLyTc785v$AG)tgJ zmjtX=@Q}6ANFc}ijF&MtBtL!zz*%cU$` zh0z($Z)i-kB^pdd8#2td=|{vGhjRF4Ae@6wgHvbE0Xp$Jd<;2g9Y1Ge+egdHUi$0WP zRcOR05z7i6rUgn)q7tGC~zU8a|pm%>|G6Pc!0Gf#{6fFm#zxiH} zNNHV4_cOn`7UCxR$=i$g4VcIA?P`A?-ZayL?<-J^Qkdp&!H7H<@MNDy1JWe71%>C! zf4`hC;WF5ML6TUfuLiHuPDU|*(~9?g9XluI{|=5c z<7B(}>il*abKcu>Za!~%@@K3t$?DKBY*eJ8Pq!E6yjc;Fp?aY_#OXf2K~T(YKZRpS zOvabd;_?Kn7_Izg{1ECCr|o9cNrZ?yGI&W{4BNtjol-qPE(&)dn6JWwgt4@+f#i)XA{YK*UFvOSp4IWVl$( zMAqRMmpWjq)MP1?OxO-g#ME&P7~sngeYTCqQx7~3KX>s1SDcHvp3M2l;Q5|dc=7i` ze)IsbfECw9nlW-Dos-_9AC98S8@e+iZld5!YwvZaVx%0ZcnLDL#ej{=(BsK9ZzOu9 zR)$B^;Ec(0$fDh_U);M7Y})iZx%)B6r@#>+J+oIN)WNMs)!5oE!l-DYab~g&$}2^M zg`HCR(N;w4$59#Rb#@~U4{Gp})$lD+L7xU83&l9X;h#g>zVmGCGGy*SgoB?_`)`L%%PI_P{Lguv_fxlrza#S**W)E;cvUjF(Ka} zZ3cqA5xS)<8FN>d5q+!m5$yL)HCjp{*~$WgW0t=Kl>dk#QE|U1l_5!=8^V!B9NlMN zdI|5mpcKT_zHKbr-pRXH<4;@*(Q5m&&qP>-Xi{sQuntV?6M!VVSol83W7)^e8bkSYzl5^eG})d#!uElC zdM&b^f;EZVSeMnDpaV5h=T}}78bFX7@Hp&mKMO}`21!u7!kmwVlJI&2k}Xk&SC+$G z%`T}bk~~5g`XQGYU8v#YQo<#Ni!YrUDmztO#`c?uh7~B?#MZ&@*NPI>1K5zCg;#tS z8+0;w{^>`~&k}$r9D~C5!rH7RNrx}{t@a$MvTDfyvnR_U70D4vH3JDrlF{#$ihF9x zQ>y|5YkbQ>DJm+oa1{rkN~(nW#EYfu5qp7QnoUI3uXF1kR$~pEq?_9qt8}2GhUTz> z$YZs4)Y=OU&VdX&mO9C)n>QWeOXALN;MikwFxlq-IEuG(cewz?IhC zViwc*_h5IIDL)ny&3=1q*SbS|Z+u>z;rCNGfYuupf+<}YKSIV>+5Mr?^tXct{hkp# z^oF-h@AVQGFuW)R5?a%zXMvzS$3w4u2G$R!wdbsIv+d0z?_d1r z>RH4mbqPE~(kN7s$XTZXHs=GlY)9l4}g7$b^WmM?P*NBERq6ya)HyG?GL+PzGSAR zI`Ug6QrW{WfPIgRunG>(c&9d@vPN7*jIVA3f4JO%IYh)|3+TlQOADd^J3z$0g;7*V zLUQ)f!$FD8JjbwDD^BObthWLc}jv4t{gy zYBPSNnNuOzWY%VF=h5coN^?)MBAQAN^%w26q(LcNqZ&@$0U}Y~^!6fv<5>;cP-e>00(5Crln^;@7zoB$g|n(P7H$~XhvnA!Ba|YX zPEy3*p1OcGzM?QXNh@=-SL;C?Qxb8T^tyi^+Qvhq^`ebr*rFTPM*@|Nc~9{gB%G;6 z;%p=;MfpNX|2>S06cxc*MP5vAQT~j&aCU6VV#){wJ{twIM>7sOTRQE$%I;_I1%@BBYF3xeFWNaXF^{n0MG;zEI{taua^M zLrRTHR)81nvdg}w#s6E*;Yq3ivp+kXgzmS$kpa8&#}j6ebza9h^GKWOink;M!?VjD z`?hX?QpwnWoPkb9TjaY+rWuMGxtc=MldZMMTWkb8f6He|y0hVBF(~t~YUi+V$Az>3 z_1^6BDxDbanwq9N#cGGzF_gqVs+#(B>aLG()^y}B?BM0POF?W!d|e04C%uIXqNe<3 zeRjp6t8a~Q?_a4#J=1S?4pk8d^2sz_n;%uTSiXK*XgcB^tD4PhM~Q^CJd2>4JhJju zA}^!C^rYL_<|`26J>{NXFtmKoR=j&%ntkI8uHHcAlqefs`mxQH*FRq{*i41rh+Ak% z;opoE+0^BiKCOxkduKxD3-&(j9}{yPxs{XCx7Jhb%(>PoxO!g1fu~D998)>p^GPeo z+&V_8OTVW&8-g5bb);shvTrPe*C;D`-?X@ONTzK045}(+a~5AxzNYDN_3da}Mj)Q2 z4drK^X{c!6iOgTHf5Q+7O$lsS{; zp#ILUjUNhQe2=p&ub|87V_YuJ0wADr;*4WabgC7v&0SiPlcx%J{Q(~eCO(z!5AH`c zaT~Q)!ht+jZ}k=B0MZ1?P(@z;Hs9;>-@-|X%Plz6Y=|+TNdWBo=K@uzB|OrU@!Ly4 z*;-UoRHbRNn&lwte;H0hVvBEF66kB5k4%hx1kGPt;cdIMg=*%$MQw^>OTVR3t1id@ zi;;Ufzzl4!+L8UHsd_AotREFRR$;0m^f=`(tijd7ZuIl#0>MD|b0b(UE;1cBchrKw z{Jm`Ah=4sjUJSl_xhnXILuR-M#_RMHDQRDqV8t4pg@-F(=0f%pI3r*`Ua4HD66xaz z0}4R>dWp{1>DLSz>={Fi?W=4l?tSZka@oynkvlMAVG$G4*JBNtF zE`AFRa@#iW?);Xt>QlO>;K}YydWqZV-Z4(AtKMKzriO&V2#wAm&~^R?OVcoVQ`3}T zZI3ad|70a2`HVDv{+L}7`tEPa>^4fD0yAufi5AUFq00B7-12=NAL=K!T$5ZFKBRP4 zOa#tNRZ`8Y*cd3QOzF=gEvV=g@Ob{HwJ1KStoz**?RobgmD-64 zXJkOK9uOnxTCS8LooJea6@@UzwvL2GaCFy0{|v?Ng!a2UOFQ-V?vCweqb}VWfiZOS zl#^Q>CWbx)6rDJ zW3cVBFO@tL7)Vd+*}k7o^p?)7`sPN-(v6V;jQBH8ZMw8G|Cqz;^Wy^HnL$+6ksu4 z54geET1*wCSj)>@#Rpx4W)4?DSy1SMtg35jh)q zx`S)iuPUzl>{{_>zOzP@LaAxWt=dQzQM#)+@GR;@tp`7X_I7n)zF%p?O-x9`Z%I2h zJ$%t@1wo&oCRLqKvfew*@kotL@xJ`qckSk`>&(;;`*{1nd!b-maE>I>pAKUuX7sKA zo0V_0Yb-*rX)Or<5_VpROhl$zu!rrEVd#4v7OmGjYN!tz?I`Icl|^)YZ(;^HUv@Z21W230g|P)Iyi z6v|=0s-Zk05|P<@tWTYKMimoIq^$Kk{R z)LFr3Q|41B^B6v))Ru-i5xUTF1mn_D#*UOl+un~J`QoSxYvMuhFz2Qm*$>*0q=wXo ztKQ1<$@}LR66NP#SrD^Jgv_C7QTT4`4-e(vY0=Hxt8=aqRc>#bb{XCYZTCZ*FKzGD zr-le(K>W;xCyg%|XAb!rel;KpZ0@I7D7+mKZQ9!3poy6z< zGnn<$p$?8M%+w^xWN>V!U6GN?4YwM}<;v15)i1ALwqnZ}FdTm$kWmzs5!1xlI#GE& zb+i*i88@QG^UrT15*85<6-3fw?i-D}PHM|+!?QaRNbhi%sdph!Hm(5L>E~M1yTYCf z0d=tu@!~@ZKNvsa#&{@s{kFov;4h(i^jd3lq%g5>c6Z&{^#%}-+w2Wh#h!MIfpE=S za^IA>K&RivO*w`5n>ftmZ<6TZJWB5e#^NcwVgMtCI{=zu=WUs}hvWniE|ATZh_w<| z9nYCIvgyLvZj`7rm-x|~Uxk%D7AILVs+LgMfF<7G?kIaTz z!d$V@puAD$Z6$WfRllbE3V}+zQs~MWs8{yH%v^=8a*@N4hb)k{D0}$xDm^nY!f7Iy z+(vEFLNB4-@CotwH~v8~Ce%}x(U&tEr@mMG-=1#1`nXtGEp2o|tHCsg^L1no?~{}mx>4J}46}M*jAJXPs-C98k{-#U&v>R2M@g7fG?#uT`}w{WF1h03 zu9?(GOud4-)kbCB2A->yk1FQ_Y{KBh`tO>L>OZYLDLQr*53?T5p@ zZO7m#R2&ETJkQeYsiyu*S(yrO>?(0Meh84SpNn2PT#QB#;>V3c`~`I)VinbS5N3U6 z@LL}E(8GU!!j_th83&2{HV_|tSB}~C`(+V@tS&iVqcb1L!;-)2pc2wBUs(4hc_BVE zr%S{8!Rhr)gKSre?$n1;4)k?Rb~>CqWo_d_TG}*3T#0kwad}hzep7eea?-qS3GcD- zeK7|s^!AI<#dM`^5DvnMmJOuX9p^S*-8qeqyP$Aia6J<;%kgw3lNSw{tRHb?L z;<#a_#WBtzjkW2^LN7Ol()g{Jk+bNTCn<#UPp8zT-h4^{kZZ^x(3o-$X30fFqPSu7 zkQEA|@y;6$GC#VC2A9roC42?t6m!`3nK`8Jd>k2=DlShX%7%YanqiHKGcp2R|5D&~ zen{C?f=8d?kC86W#3DWJ5l0u*<>5RqL3#$?97jn<;vvcd&EW6WZ_4e`$0rfU1`p() zpI1Q^ago`ZiCy;6%;xo3dS@CTna`wMoe~<@FRWAH4&V@h?T_7c*mYUYJ6dg zE%VX`Ngi9_twO&f`P$&k-hdVCag3Mq^U{@-i}UGcUj2H?Z_GJ32Vq1cLG+I>rDKt@ zwBnTEF$;e?_9$#sTRP6M%ctXSk9|rftV(Sl6E_zSBK4hL(otFkEGP0#uq`M@< zMS(U;6gi3Gt5}@G28n@oMbvRAS-8FkUX~iNT=qQ)Pv^ldIOBnwD#ph>0c*F^YF;lH z;m)_qX#MeGYk>YiMA8dL>N99*D;lHhwV+dnpTp~FGo%>GaFqdlxD(roC}GJeat!rde!hV9D38hK z?_DR`n~h9mi>7*f(1b#pfA}y&I`lwc2!^UKQ7E)%5M7g?AVxSQi4joGt_;7oKm+Bi z#7&HCKCU~$f@JEvd^@A=OvVt7A@$G^ufKc3Zj!v2d>u!!G5Ishk3ARLg4_+((V$(W z+vT~_kj&IXhXV`(OczyJed;~h%_4J5q$|%unt#G_Bi?P| zo=_@H1?t$#IHhf|5ke4_9dy%fbd(GFrM6$=mz3T^lE{w6nmBUE)1x$}Q3h>t>hW<~ znG-DO_nLFz{`i}^{$jm$ne?+&i_ZgE{B41WT@Udwin9vipKX#6n-F@o+{)Q(Eyi7BSK zn{H6Kcx<#?jskRMUIy<~CAJfGWse2uiaLD-0gszhlOfU;RD@2S_G}@T1cmeX+3)yp zPkyG1xMR_Cg4J&{+R1L3A=;_ucRJY@W7vNwDX=Eo3uM8!BXAdJNpHJr;9WV(e29kzpJI zHoCyiO`qdJBsr^m-)rOHY{*(}m0d?VRbtbEK#IrecQm>WSqI_huXa2l@AAn~P}fbf ziu$BXxLLIM;m1xk6Z&oMue$g$+IKe3H#mWKL|^oQd({e7XMvaD-WmVGSg6q7t@40o z7m9{gmK%nrpH4=xkd z>*tFseoh%Hqs<<4*M~!y@kSx2>fS%o7F#p87)XoY2wTd+q&KOudltJNZKeOK>o!()kc=q{Ub2H;7X{a+u-VRrYw}#(bP7Jq2lP=>%#+G|=&}-Lm`@2bB-duRa$Bu`~lgfF~s@WjwiMu6a^Fl}7Q7ok6 zhr3aNP2o@Ft}52i9rmqX-o)vbMx{y^!M#P|*sBZ@aTjv4b22*pz7kI`?-#FmxI!y` z_a4ri2*d$Y9ZxH)w+%C#g>xn==&~(@&!d_AI_Rn?naPi?ESNAw>B^!jpgOSu1EkZ5Y3we!M|R zN1HI^NPMR%ho_&#-1m^+(ZD1TA*Cr#y*+BV=tZHK_@Oc^nKXMmU92>UvqK zVXBhrw_xR9o5?1?$o>%WdU zm2JlGN z2$#VX)_yKv`tLoM6av~VN<9I_InN55H`A)iuYyfNiz@B*o%Mq5lS`lv*@&2q7>7MP zO^gF2623IYr6uoC&HRvqB^r7~t7e&HL+$)BrpDyL2o0n3gm%Z#JCMr5HpJU)YDUV4 zNjiMTq0R`Gvd$!fadY}@Vn)tWEY#D^mq=KHXt=!z9>u@mI$!K14@fH!9lzV+z2xsn zgbdfh!wsHgD8VkhuiLSg)MuLSnH^HMF_l$Z#=WiisG^rBC{4p@-Mc5%iDiE5&znPP zA`%Q@?8ciu!62VSd&<{r{HUiPGbO0T)>Q~IKYdH0F=MSxgVAWe5FZ5@@$<&Nuz0rwOt}Lw1UrjnZdMW~~9bI_M6uEP29TRAi7eVWZj)RT-J72`+aIxTnC9xbi0nKf+{OpL<*++@q#m zex8CGs!gXK^t!Fw=%%UkyFUbDV&tmfg{!r1iNXG^K6pkp8wl#)urY!*FB^>TqcQX6 zSEtRAKlF!NUC1=EWK+l_$jNx%U2(z%}i~7A8 zuFfia@psf4tTyf-Mc{u(^ZIt&kJAd?-<~T5Y{ISaQGvKv7DRXX7COZ(BBB}y*`w=%xeqU7p z76`^E&?;uLo5NM+U^K39Q_rvF$ zS4J3?Exbv;w*pXgzX9dTci7qJJccGTcc=%FH|s79et9-Ey$LG(x3R9jkDMwuucwf$l;yJ{Y84zx>oUebvvgmJGiQmAtwm`n#)wzc(!K zPPbQDwfaPXhp}WvziSb$r4{R!;p%Vx}s021M1^^>rgiNF_*tTZn|SA;A!QJ zf~b)rHOe13L@)8UN=8*tF;z{m6~TUD6?kdGIJuUgJLW;wNRO+Bs@yBYKokrK`?aXU0A=p|TOvm*nB&)Vaex zUx&1f%2Gf5w?eo5Y2nYJyYTU~HFVbN#03)W@Um-zy`^~*HR(=b`~Y{bVZJmWi4rTq znFEI}Mu(`{U_?keUE!JY0E&dp)!1C{o3f%R#2Mv1RkPf>Yk-=0o>m<{A^}t`rggB* zDG*!wv>{D%gXU9TaeZ>Yi|snCayXr9MMCXd>ldWd_9O(?IkT~x`ba*avNFWQ@g_9~ z^*_#6A1*~hqzx-kR-pRw1z2>>$UgHPi4hZ@H!H-*bM6A#2SJQ2(LGZWEqQUv)?E-T zK@X2g=ezg-d})hlwh)T2+Oms+1@=^W)AZZ$mO;YPJprjot}rcMq43b0W0@K-+8um# zEBBjQHImJYfqU7J=kyKhKrBaL3y_$&26CDXekVekE!*@q;HpJ*5+*hidFsQ0BG2%T z5C;+5@4ClCUUER8hDU+N2i*3PUxZxLkoav?*^vJIcLyPvqORsQ@!^gA$dFDS$#Ovc zotCGVl6b@r;s854ED!r=70%YdXgpv;gt?~oFy4Zn1+HKp+#^SB26uq7|Z3XLHobkE*vP zUh4-}ziIt?nchwb;_)C&oO9Rc;F866fBS|`*;eHqq_8v=SUB(h;#D3NzsCox9N zM+I!}PYc{m0Yx-et!m>vlwqt8)!&h212T$%eOVcEi$l&6KLe6K!Rj!3XOCOd+B4a{ zU+P5oh->jHm%2H&2G4Poe9OJ_YU-@oR1IeF&xkP1h$+So=^;K zy}Mk^P68c+eChll#3{fD2cfu!gSZ__SDWtlkTMA>0CenpNN417zhD^UPuh;dBq|a& zhApI7UewiWg1HVkE655 zavTVv;FZL%6kE(t+KHK2ryupaSxtkXl=(BFZCCAf6vrxc)V5)`l6daRGis;UT5x1} z$!QYh`+=G450qh@-jQzz*)D2+q9XL_c3@b@5?#ncT%CA_t)h+JZGNb&le_IT3pLt= z6}_okns;Ctfgy0=9$(Fo0QaD(vvGqF+6#g!RCQzKkU6B0D6qR(TJ3RWBEJ+t{o8?H zLh8ilg^!Mmrg`xTlj+US6hRge4eFT(nAuNifWOrr+Bngf|7;Z2@f>lVaPJafRkZ;c zZc(m+t8snna37?-RUYMnLbl(>`Q^XGA`+4@!&#;%1NNDq67wW2tdAq2wfx-k0nOhD zK~!_DUh^1?s55fus#Qvj>4SN4e$ZBvnWY)U6zOza(Y$MA+`$PBEV8QZ!q{ih_aAHZ zgb`?neK(Fp8ec@iUGOmsMj2S)X7Do)q@@kqTNn@2X!6yCKuh;fuv~g59l;2R*7+?3 zUNJ|0Xp&m})-X(P7w5gjxG5FVEBz5w=E$YNrjBTHo*VdEXHxK64|}#`F-!=drhf6NkUu zEZ|cBBJ8O%d56cke+1WgQf}JqCeA|3d10NwNTa;jWvM63ZLRDwYfJZTM`!pa2{Fue zH-^JitL;2c7nf$XD8&}d;E`LUS62wR=BM+#9wME4wWBe=75(ZK#6xX}PaO{Lvq{G* zJ5;qW+tVG{(3@u`f#;5h3s6Kz;QR$&mH&Lb@`P1H;B;r0w2|YSb|iPt*J|1eOUG;c zHyVoWnGCP`pNbXT%1zSRiTH>fZ^Q&De_z&Rpz!c_Xlg&QV7~#Vsi&Yis7lL9@lSC@ z$&BW_q0k9_e;H`)3mzJ=302wXq*c4GXY~E0GUXl@=|Ff4(4iG(gV3kam}qcEcmv)2 zJZ5RejaguGEr_H^-vo%)XM0gF22;8tgjk*@8zrGLg2ht}Kn27YlD1UhFK{cj)#|L3 zTf=5eo{rznx%8aC<$=J=$NlWR3r3SNN}=IB1{Vgawb~a>Cz<*BpEXP9RnKbS9%vuqgBTuRa*b@vT*vyw-zg#Z7ORw^~I# zS534Vb9+{AI5#m@<(R?qV(L0kx-zjjwXkJu)}tLJjEALKHSC7C6S#xD=<;<NxL!T|M-tMvFVw9WL`o=?b z88~n(%KsqT_4{E}uZ&tSRWy*Nm)!|7;De9dP&ZRUJ6X~dQZ!U2FE?|2w_n9zVQ5Dl zqk{xxaMN!g!#QFZB|pv@$!nG62q#>f%lXw!J1a}2yVlh*2O2zFOk2hWge)j{Pj8FK zW`2ACO|+cPPu%_hqI-*yBg*XmowP-%=^`C1BG**t))DZbkpr{RdmD}}E0c@>^F!RrP1cSAQMd7~DFtNAHF-Y9l#A z@>OsCoTq|izC8F~{D{0V3{teZxzIZU^Cf$RYD`;KdF}yg*>=3CyS7|mYVu;TRiv0V zE_^Kd9?&i+RGCVif7>%C;`Iuk=;v@m)M@9S+4r;kSlW3Hdlx*WB$Zk@@gcy(g)CVQ ztGIwRi5BaTNi|An{W4fDis9)!sjm%tFJn`B(dXy)?Y)xxZS^ZL^@JSFS@$f7>41*uz8aHATX)GsEpoK* zE)$bnZozGfcbj6H8^}VPbRrnsuv0V}z#(d~+I<)W!eIs$;j=!ZxQ@D)B zqI|SR;-~_Pg*&0Np)xQH8$P@whc1qVUvN7tJRut~V(*>{7eXE^w^+q-%^rk2%($-i zBL)t-T+%{>UX6q|$KkB&67H6wQFOPO5AiCAZrvz6G0Sz0xe%R=RfY|C;Z=f+$VZx0 zlQWC}O$ug`OZ%1GI@(B;OO75bM`m%)NN`Ob_?IM?+Rm3(qL3y~9YU4H7sza&1U^4g z?iJ7kv1;R%aLGwp6G{zEhtjEcRqaPPXO>&sg@=DztRp_b*f6_VN*yY(qH_yL)Ber=0y4_0C8M59iYkAm#=R{_ui+KM4+wIEl!Db)@tmrW^ohRz z72TA)OOG^q~4Pqh&bIIVx;gTXD68+Q?5M4^_{}))*IOGmlaqenUugKhu3i)IZn|k>d0l~( zmPLuypYFRJvbRlDTTUor$9A`mUCl)fW@LOJKngQ69uI@Y#8n#mGENBC-|FA<#7rb^ zeXhrL)HTly|NQi9nXYJ}h~#n?5if`<-bYyqD%+TlVnxfwGKXjg59gK#U&qIO$Aw73 zGy$#l-|Xg1xrS(@Vz6v*IfkOdlgfMQ7*KGAs1xMkfd|>+1Xtu9U$>XEH zxcrTaD}lkMq&2?k>o+9Cd9e&U{NkqYULw4BfRh!_Q8*<##GUPQ{NWjtA*a{q7ApW% z@;2Mv&7jBW-hz8U0GC#fsYJMQ;?1`)L zrz&L?c!t*7fW=V7Hgn+CGXCkYVA1QcIlZJGb8kTta;(RT{T51x%us48R!;+yKvffV z*$WbPxrjoGwRL+cIdI_bRGiE2mBC$nI5jx;6$KXL-I~8`YhUa06i{#Qt&^|{W9B8A z7R{-`B$P&$#ScJ0{~YuZ_{j&K+;w+u1fs>68r337aZ{+&8889SMPb3#biLiHK-~CfhCl#M7VWVH>WS0G3;^5HE2O~3Qi5CqpcEDiTATtYPv z&5{BlZYc%wRuS^Sn<;?W(bT@7xHkPG1n^}9A=z1{T6nnG)$r1?#L=zzFAB=})}vP0 zUv0&jm&D@V=|XDvx8;3b1qYP!LN~Jl~=LB@Di{_)3o?C ze1|ino9Ott5?$-uCKfI9wRp*SmCAnfA@rE0D4@@CUn?eS^hto_Th&l6LyTN>Q?ntW z-OT%|N?NCdF85H2`flzC4baZNP5%|7!___a8_PYSM^p{Bp`|0%fG&<9WdT;Ph(xtD z3uYIzJBwmLoXtke2Y{Ij+{Jw5qLTw|(AX~-5MzI@MoR*UI3^YDchFOWV>k6zFBidh zCoWboipa(C+Y0M^Zoj$wMH{t37xQ1Ceh*xIW^o{5h;s*wq>aYF)>osSq}jl7seryh zd6MkoBx`2as_^*z&YW~xfyB1x!CZZPy;BWLmm=5Y%gRkme|MbA?~XD_Q?*~`>+j8y zJK@Tx`aB5WfDW=egB_6y%Ipda zLA4JSSB3ZtMDL*KxcXLjtYcqcn2})6Bj@)7qcR0OXK#lJi@bfbmcy5VwLHQ)@rlOMCF2ga+ zlHxqi=3GM)rOQL49443`Z3mpb+j`Pe9%i1jv-kW(rY&(lHN$@34Z`Dcdf-$W<$G3h zHyDe0U)(NC74a4E@>c0$H!-IA9viN6+ZJ=t%V_*`UH;npD|-LHVWV3{6g0;edyC9S ze9drK7=5Lma=Q2Lt*tOIQryuL%`M6hX7xdMG=L*J0aDzTn3)y5IYML>jnmCmlNBd(E8pmutiYqjHt^wajA>C01QrPPSUtt@6LprC4J)Qe)} z*~5et8CW+R1dX?kF*U8VU8oFR5H>x!N+fSicgb@`%-0?%-J zI5&hi=Y@tOtz9k?zTj(u!px#dgw&5n_Kf!Hfv#M1%FV0hlPE_IutFIYU)fn+Ds_CY zTz_Qy5Y>w~(&@XvcfDDO4Kzv-)~sr1$^0~kIQeG`?`w&c;ck;oMUzbX>KI>P`|iv^ zwlE7lD)vz<_SO%!W2GM@7=p?k76n8iliCI1$3e(T-N`$-*BY5T^W~E#+y5Q_J+g1x zJhaUBKNK!ZbC|Ckf*An(U!&%N1Z2;u(ou6UJn<|60vqV>7l??`8?@G`=#z2aPtNw8{{IDHPCid~pI=mB#LBZZ}k1+_400ht1jQ|`Wd$ogPh z&92-FqQ)EJj~A6-8BgY%ZbOy{m3z|F-Gd)peJNMic9gD;xhI&hw#^dZ_tl^H#v-{)s0n zy)6C=BW1AvB(ukNbHcQg|0F~ePG_c>qymO4+^pWJ8ZA%D=l7DXMyvdj2yPpgQ|6ou zSq4926UU+;yNXM*e*^gS*9V>t3=(nw=P8!{r6WZ2-vH_Q#*^5K=RU)OzUIaJO0do@ zqx?CT#$r(9Ho4z}!Utg*?MLBLB|x%!sc*L}QVckFWWkGHl(huk!3ar01Lcip?-_5P zAs;{`pK09B4CUn&>?J=-r-Lnr=X;hY{Aftv>Q2B~ zU?VxMjqb=p&4&jsvnVJ82NSaWxwB@uY!E#S>A$=rp$+?+a}4LwrZrzb>t@(6IL_)4W0(^kEjB zDlV=gj>x1JiN)=O)SicLc5(#WnT}s<)yttb6Q|?XcZJsVLvqiLFDC>>7PAeQe z$|bo~Y?GEd9)%ogbHXFcK`i$nbzkzh;{kPI&Uhu;Bi^qWB_mar)WC4y${#$O%ae#fo!s%pUzX2vdaKPPmo>VU$dtl{yV zc>zy+`4#h+n1g06g%VRC2swZi$>(R`FGryayoI?!zh}`Q(deRgZ=QdM8 zI(8yx-f;wVARB+^o?I9*cBj8Ye(RKnREWc>n+t*?V)a-lHBAyEEyeRD`JZIUT6uT9_@ z=~*xl9*&w#6hbK_G9|seazBp`e3*=|SQUpP#|5Ln2sT3rA`X+ARzefzc`E62l;4ZS zQUblSVl*Nb&Joem{&m*+<0ke^HwIa}Cc3%qOlM9rq=T!R3>7U%FH3+O-KAY?(W-o@rYtOU8w!!ev=p z5}^`XP<^S|tsi5Qzss8q|c}QNWWfA}2;l0ywdzW5jFHn8I*nX;1$vS@_G6nnDDzu}cD@MG5+&ai>OB}r> zGLrc_;4}3>`4zD#CCezcxaGe~HCCNf96rRQzgTQ|n-j_yJV$6J;(2R^o>=nGK)VHV z%frL4wnBQO&hL-cv|h5!T>YX0cG20TL4CFPi|AU`?HgBNU!_g}&{PIXmx2`=vPSQE zeZVWeLp(MLSK#umi&QlKQhktnVuF_w&HePc^qYSc-|V&teMz)nRcXx;-cN0(Ltfqd zK>F|zyS}V^&%;;z(Ju^wNN%z?=oC#u;<(pP%oB5*PD|A&59L>)iQk_0XL>fWNs2t& z2O$CiW!80MZiAwj=S|Ppv$(pjxd!xK;}Ns*>(g`g#`Lh9TKOe>yz@m93Uz}m2(0pMGbmqYjGCb(^NyP22Ib(Y5MYj)~+(X##oP7u9ntw_; zG|stJ_|K@du5tT$$rHH~zH2c$6~W!6#(3?L?S4VB?95Lf9b4i3u~VFGx<)ag6W8-O zT}Ma`@-$`r1pbC;eSEp-+N+1;-x_SGdBN5=Y2H{kC4-LfC7~Cz2Ev+OniDsMF6ddy zkC-m*b|f(I(n>`36Xw{I=BrDlS~SSOig55IYe8cH(xdoP!guyGaJYqZBy=40(kHHy zoe4*txVEnO^geY<9kljiL<6TdGn8VKr(KDIgftOT>a8@S&fH%mJ(QNhp1Pdpo^D-t z?ii(-5{K}-cB3t4sHSq?6$J;wlq${)d`O&0-mzr3d3*{XEa-JPCdD-I)Z1+RW^?&u z_mqOKv!%Or2FpmQVC9ag2FOUKaB6kTRxo(D!)ct~UA&`gORQgkvl^3M=k#}%bY#3Kn#n}0CLMGnn4V94Qihmqq|!^j?fc1C)H; z@n3}DdsXI2+E)IGjyshJx$`a$2AhzjSLpED>5!!6P>BpwX!u@;Y}{3??#V{Lyc52B z>5rd^%#VQ@c)HyFd<9dZC{hgLZ(j~%0pz-v{8FMYk6p$&Pv+$t6`Dr-%OoAo;2{#* zYki_92?^RD!Ep@ro2XarzQOWO{o*U=*}Rit>KB=~bi?li|IqOfF_x5v(;$DxT9V7y zE+X3X=;z>wV+GMlQv>7G4?EUPdDR3&xdC5>vzaK~7>vU{Fdx461M5kshoUI{ZXWBl z?YXIK;N!%8&}(tC$9)0ny3kusN^z)v-y&iGS!YN>p4)kDj$gC-%sIxW-CIP$1!*Hb z%xM}!4Ps@y?S8p4U(>3*E{wr@eQW*gX3jMT;ii#K%h&q0x0=f^T}vdBg2Fc$Vpp!N zDuVfM;|}m72YwiBi(12aY6x7s`EtJ(4LaS49f9i1)pScphLv}(QAf%k<~OW@`K}oU8i4#rRZjx=nc-GX;pcMtLytr`v>jmu4fJSd2^KLy7uFF4*N)-%&vae2ivedD=Qf!Puk8(`I&3NwOXyUH&Y>M z@8*^K+3NbY@f@Gh*$T54rYXdl%)WeFvrb>wioJfkDfx7NZ zOSR+Lkej`*i@Xh2>Y%q2v*-O4dUwvs-4c6_OIe@{n=uhn1Ynj~t5KXz%V^hqZ?V+G zcP`3&@?auC#nm;fe=Di{pe_62&ii7D$i$zd<)Mr;`z5tBlTjS;%&l;%YK9MvFMRd^ ztY?q?60&XpLqNR0YQjQEM-E&T{2lcL4H-^XDeTi`$6yvZG}p7uIE!I zjW*Y9!Ypf!^-Ch}%JTBrSf?DAv6GXMbZ8 z!YKY!io(R*px5TYYD0t)X7NkS*ope?9F?l=FWRWC2~fb3-kA&KKrADj0T zwEW7U%`q>-@p4UOx+rYwj^i}WM+p+ax%Q)9-(Jd z5OYZ^iT+3)rhmYO{KOucyQb+-0DbQw7g`dcYKY;=d1@gv#3+de@TGuO&og<7BZ`G- zdQJHo*3(EXdqt^`icI&R@=SV5Qrj;4yO2q7nLa6}H4biU{pk688#93DpE0+NJ6IDH zG=5e%(gocd=LEw(CF;tSduc0_qW4_3oAf$Nedt!E-Umhh4Zl^q#HfpVrFm!6df7+H zXaIusp@y{mNxUp9iq*mjxhQ;N0J_)gSl&>IU0YT@zlK~wSQsV4v~9LoG)`jz?P=?p zhq8xY^ZY(Hulc~85NAYtHUF@?1S-NtTXLk&1~ZC4TGKo@jY{1cFJ4}JE%RBJVkzxG zKT(A>Y2253=qc+N#zIew^(afRn9=G@@8F!BYVRgQ#y-P0!_U%tBu*7K7{ueoTtm0$ zYL%(|?gGi}$x!*s3XPn_?kuIpJjcxAuC2RBYv4_cJweLw#XD0e*aE8|W>Nu6%J457 z7e9ahK5kv$kiV_wkREw*Db>JrXI!THPpu`evsKMCFUMs~cjBfE(##Aqf5Ej|5^I@D zHHq5t1jrwsLUovzlXqw=i#a(H$%`C+7yYM$c=WBsB9IUXCKgKLgJk3P zRN^cDITFt0MRfG(89sRWxmcWLdjl`lnub4@wDXCNb40BLO-&P$4%)4KU4Y$uu9BCk zR$Pi>;TTS>nTY;MViFxQ@ziT9W}7(e*mTn?N9L<@FvrBL`u4$Ahgv~s_jikKJM;;Z;#W*+pBdu1u1G;O0`>qQ7k zHDx&{^v5Ke=NMICZ^~N{vAQlW1NYg>cxZdS?pWAth%2MlH17T=i>YVL4}x_^ff8k~ zJRFhby|@FEv7T(c1Kt*pzt`wQh%QfXJmK&d7FCNBE!WeyH7%%syTUYy_z7;HRjTlp zWyUVh-abz9GIymUahuz?IO+KEGvR%9vW+#(U~V!$dAgxrp)W0>yDl=5Y~pWIClklB z3lPutv#cxXZdiGB0kTE9x76;2G4^ZlICpV5%uJ)BLrmR7vpCJ_!67RuIdioU_~T|` zxTsS(zFEecEz%L1+%0hBlz*3j zUM0Mp2#ub!?p~)Y=0I!&>-AIa*0&~!SggaMWWei7^+>{cYY`>V6d$=HnE^8?Tq;mk z1}7>tiWMJHeO}v#^8D~dkvrbLT&P1yF5cX$;bBWxG<#{*z^uY-$|_yT@hz?7C;NRq zg6*yn@v3^745yb-HzpP0SR(p7HeQ6e#^tP^ZE=?!*4b_9mSyT1dp(wOlTrX7ItcaeoGF%mPM9^+K%vX+Hv)~S+eAq?4wGgID530gv zDS_8kQhR&g=uU*P0+-@-U$TX=f;+olz#pO|y)EAqG`*D2;?^yL7#A>L+2)xW#(muG za{fSnv66NOyc?hTzP$qmH9Pi}*?i0+@N{M>A2~iP*V@qmX1zqmi63FQ@cqmjA7g)~Ww*G-O+BjX znZeji?#*4RkxgSK;M*M^x`TBv_X+X-Q5=(+v zk+F^hC!WdjiHK*0`E_z3UXcCPyO#n1C%oqr5A?AmH}k9&uFnACX1ZDUNk76-6Vx~y zH>b^=+R8AOb-M&#vC?OoLZ)P394%X0P8zk(-&ss8Va}n;B>j%D3TY&JS6l^C_|vg2 zr0bjKE}h@nI%C?l)du~}{uAj?#a*}*QeLk#B6=Tx6d(GC=GX7Zs9#LHK4C2JxIFXV z1+HOCjUDC3dA?bL(b-r@ZYTJW?aa($XSD|tL)GPXwze``#3Fu=NDTjP3$8anBU3pP z(r`gEPiTKNo%V_0kMW+t?>jgYwgTuWTDO}Q%hQ8+bfm5p1^T<$Wvg$7!|?r%!wOH~ z>AQqnYhQT#Tktz3%EPnV;-oc|yZg1@@F*sD_b9eruqkR_4Xy%Q6Wz(@aR6C5xfqDn zvPOCR+r2LIv=R!&lp7Wia}2OiDcz!1H#{6=LPtw2E@&(F^eNS1MD%QM4984x@wz)LCWM-~3u{KW?9GO1p?QxbR`Oc4J z!6RANyi_J2Lc5FjJaGJ&V}r5}8jX<(b+Buvr+e9MI*Qkp!hNz`lxL16qH9da8W@?u zWvbdhm3>yK`a^75J3NuA4=hp8SbaDWDjPvHK|lrM-T%;4x;$^+0!+|#C`ZsM8Qc*@cdE~*Xu?@na%O1ujy&YZP{Jd9>1qAKges_9`~Dt2Dl zd&mGSanVMyh5C3P+$WKeGawI6I0q`Hso~PwT*?56Z5*MEcs?A6KIbwCE2KUXD$wR{ z%&Sx|jV?*W2R~r6az8*2`*aVhZqV%gZmH!D&1Flm?6n$ z2}w1_WE7j9PIQ#nAl?x}d)j@_e#F`CMV-!~SXfXSkSZ}s6cs}|_~UJYgAoumym)UK zl)YO!mrO@#%Es=DcZK;ao{z7t&98tDXHztwg2$wJSi z)u@>TZs1|xj6A96&tfEOY4U!5Y_MP+&cOK^b##b&sm@P0>&+Sx$y$Wr6K;$LHaR(j zjTQFAk-*ZJIrL?wavAGAecpk# z(txz)I$JaPxO#R{05*6JIJ`Dm_K1SY<@jo4uvBg|Xa0eF5mtkmC=_Qe5uFh{k_YAn zN+aFfH;>r*<7E`e?Y7MWX;wug4#v2P?_*tnDH9c1R< zv_}NNFZRoSHZAT?ESvPB$9->xpKP0*ZaKZ;Apz6R`{VX&JYyXsU9cSP0V9%VJZf^# znH6Yf<-gT!w&n_d z-JsJ59H!lTob-7HZU1m^N;AJ-R>JcVypUQIVZG9gWAU1w^?WmTsv%T>+=gTsd$6W^ zgMpggP1KM;g?&{(Y`&2URc_YeuZu2MQhXF)el-M*Jh@GFV}0|Pv_h5sJAqIpP%!u{ zW9#d7u-f!ORXr&M-z)P$vx;*|LoGrdolUbUIWBryte94sDBsb=vrfze^YdL5h)b7; zMW}Ko48~H_DCU8y@Vg)@RadhLss#$>B)S)9y+`?k`;ns6qK>L zPljLd9Y++gKDZ9i=u1?|-!;&R`^sOOYvSI{gytSM>hZ`X+J;Bc@8tF>LWgvciNPL(hH6^|lflLQ3_`Ia&+G5carZRrY7i5Hm ziuo{L__~|6D`>@}Kxrp_6n>&EPd6m4E8@E@Ld#v#-i%FHRoA|289U&Sj0nYi1ANlY zxm)2v#$A#B!Sie^2vGSrzwLIsi;2+^GT!nM=af#+ z#16Wr$2(Qz7hm9M_9Coobjzy^P4^s7jB^^Q?VZ)K=x=;4NZnh;M`nP+fZuj!;_+C! zhQ$-cnwe0L!;tt??!Kkw3LNAib9ZjcL|F!RDHqFpdY5GJV)^_!d+NFFB?~s{Q1Dh~ zgfu3Ml5G9*=$@`(HufQ@yM&s#vmxh-k38Ft*-oa!@g|AP zKt-)F!m^m%$T$f18|+e`U(3&$%vO%*YiBA~`-BHC9)H_SmFQmr4ZN`K2LDj3w{BAV z>n&H0qh4sTVzuv!6cq$J7oIxNqoWb1dtLYZoQ7@UAqrnH(oYLTUF$F9-TDX-XYHxP zZp5;OaJ=OC#GIDY)n5~`sl2aJW6{P3lJ^Z^E9q|AWp|agRS>03C*3;q@XlveKlq>{ zkp)Am`_$Z09S&sa-8xI^A?sN2laIi`2TnwU&yvtH*#RBL_8e5N&l(aLW7v6gCFp!1 z%{mYJ-3h)~!);J)YX6><^v1KZrqrR)_N^WIkOLt!Q)Y#C|6n9Z3S)aR3h!~hMlFA@ zZ~WdD8F3vhG=?mb_2nGNHL~pnvWPxf?ldre^z7+Tax(EPPa;qYDEn>otlEgM%Z4}= zjQB=i{1++X8pYx_D0c7obo+ehtx&L?yU2s!FEhxDps}}Lc(-0=8-x&Dqq%mWpd-!7 zHeNi0!Iu|JZB*F1)hCX&Ng}!t=Gqu5RG(T@8$ z!=nB+Rb5nt6aAZS21Kg8+S+4bd6;FPS;Tf~eo=NVRVWgEs4>#XjQJ>+RWwhLHH4pFPl&WzFknpIQ(F!jf*vR!P}oH+sskip$XQ>7nnT; zSqf!T2<}vUAeCu3>Nk^GD<1Ok^(qs2yB1L8Q0yZsbBIWs!FWFUe%|?}^xw5~)RVRda8n%v}Hnl)fLxdca_x-5&J z#dZ9!Y5E1ioR0-#$#2gL#g0p-?D@OaO*Du7?tQ?h1l>l9+z&(%%;HjFV@JC_!|D2g z2{bm@JPbIMyY&|upf67n`sw6Suy_F@!-D1UPP=HWVAK+p7>w(9lXMa17~2741W+|v#~1vk<{-{a+#~a7w_yw`qpav9b3u!gS5A0>xYOZvvfL;){E4nFxO?zU zWp3DI>qN2arfbH>CC@gpYHkm{nz}erE0Zss`IR@mfBC^^z2>i}a!y)V9(_EX%0&r- zkt~Y+^AW?I^7H}woyhmrpX5hZLcP~x{dZgi652++hd#FOS&fzRM}WI*lwjJE*oRz& zy?F#^1!j4)(eE~4Y>_i?lUn zsmE9$ju1xq;7Z7DpELc;-waS|u=qG=pMF)JY$e0TUfgdiLLRm8^LG&-PY?B(bRJ~) z+3q(#Z7CK;zolBBqW80YOiqD>`NaS>k=f)H%wGpq1BJM3Wq2|bt8#0R^EvXX>wO{3 zdvMbW?VRxNkG(OZE`5J|uaZb|!QEy2@W#+*dnPnKV2-##SJ)K$Aaa?w4E<$Cu$o}( zIPnoYd=Si@Y$b6$#ii$_z%bW*$dD&iN5(FO#u(;)Z5(FUsR`5tFLsfiKwy<{y z%&*PpD@yArA*1F-sXw}~X?0bVvUm&vk`SWxws&y2=&X*aip!H#x^-&1&}k;5t5|6d zo9)sh0hCD$Fy7j?Xm#|SV{Z2O#rYF;!#6AKzBC+VrNc6z`$P$*miSPw&gsqAGqO{q z-bS=U4$+;%7X(@61H4eSj$Xe84q6eyAeGLFzrE3PUDQCOM!n66dh)@X4&Wird)ffn z%JEWBH7$UK&o^HmoG~@@;-{vZflDVDBn*)Y;6{@%WJ`|onYV=mlkVY6-(}7zT`WHo zGuKg^zAWP-7vG&jXLqILofKiohuJc`He^Nat1M8;i3k_nVMartIo?Dx@$I43q{|@D zdkA*Ywa-)On11l#xCOKWMn5hYTu{ntq(C6W;$#?wU+R$?+KA+nKB{bi<2@5A;lxQF zb{~~!fy2B!uM@mwsQt%PjKU8JoffAOJ#SS3C|XDlyJe^U5{E4{4PRu@~H583Z;yCGbe#4q`k8}Fe}_AF4V z`|jPgaqHcP4xu{Y9IK`tVf-%sgGl`Az7G(lBhs@4!Q`;a5WT^3TshFClUzp(lEeKB z&32#S`Kjm-4GT-Kar2{1dQHHh;fYLc&MrYHx*NeATPIHauCWXX3w_3$W;A_Xeu`Nx zb1`^t3 z*7Z;a;&u2+=fJxjsHxs^+VC`IOzEjoe)y{8juZHJFg!7N(#iezJcN*vPE?|Zhr_@L zrRf8s8xXk<416Esi75k`iCp``*5JvM?dN*5wU%yo`J8|;wPI6cmuOydg4`MRa9G`z zc^;P`y}Ge`yl9K_2bR&ONpl*i>a2fDib2zul$1QWx6|^Tc&E+Q9X4;F#qI71?Z(O- zu=S7Sd)lnJ@5u;A`>p-)5pK6pR)cAH9G`6iPg^K^e(35U;yHjx?6Yy5^&{#R9$Jbk zc7m%p`vSTE}W%HTQv~ zGV2*fL=1g5QlX-Io-l;SSDWSG$8XV^P_>0Fs&twdq--3!mPMKBS<6wwS8?#LY*yY8 zCwEaTx4I+`%xH$H#Et-l!Nc1}1I_qmlugfZC)R86NC}+7gN1M{=-L>%3@@#)G)%Ie zmQ$Lyxs*f;u8CH$Y0h?G$FWrLt&xuV)`p%`uAJt8RGVvn{&SaE7M}%3t{Tj8%U(jq+Nzc~SoZffoc&j#m#E7V>rAbBx}*hU2JA8Oc{}FYP!5MmjKdz)YgNA zS(VrhCn{$kdkwh2ZQkI)<1#Xb=SIXH(zYx@)sUxVzkXxXCTQ2o48rNBYPk-Bu}}4s z6lg9KM9%v2^Bn$x30aCiXHCopgfcMO%VsRX{p>4aWU0XrG}Njv83?}c_10z=ZO<`M zTNm8Iw!F|=hePiX_^d6T9R4Dp=%)g#b%^8vLSLIR=+%$TPBMaH;8RHV4sY|D3(=zS zdgCF7xT=1e7e4Mx?pGTKyUNB?Bnoyqlw6xm!;3!v-A-WR;fN2XUq&Hi;aHVO;yQHU zdmNdw9hq%JD6B1f`&%e3mlz(ZQ*~a9$ndHcL|P_}g)oG0Cpz^niP!=;8<9f_nG2*~ zRSDXUEkf}gWaUK&B?hAlmtMJq*`j2^QydnJVM`#4^@BI=M!YviWeX8772aq`x%61x z6=!#d=;?cDg-N>=Ypa-{ANtU5cwNlIpFfAm4i~9b^}A;nC&h9BEAACRReoUpErCB6 z_3{{UG*~&T(Akc6p8{y^VNky&5*pFS=6NrEFiA$sx>4PskN9rACe%~9eo{q+@SL8_ zff@-HpRv_S^WyALV^46@J^$eSYPM=XqArlmL9Nxw2IaSPTn8=Htq+niHV+x37Tcn z4;>dzLVro#6vae)aN0+?r^fUgN!GJ=&9NIvqNtKlBustNzeH?)>yKe6a4uL)I+%3n z!vGItn(G__4md`onF@X-p)v;j-l*?CjMU*cJtSyK4X(qg+WCzv`bAASVmzIpAKD*D zfsxZ)S{8ph-F;JsxA1B^wXr2P<0F(St;0=vHrd?>IU5T0_BypRZx6b#n45U0J`TUo zv;#XdqEH}z-{l`{@wAhZcyej(g^u)%EI|k}`3g|%2-)WSWl?_&k8jxQl%=qpq7!Sk zX9bM(0id0PGcT@LHC6Ak&p9Z_ZXD602OWnv(ugrH|H=(6E(hs^$=``aw?%dOv`xc3adtKajQoXu2<53Txl49Y zg)jDnER!WBlFA(XZ4aAD#Im$BFmQACtTyMT)o~yqUN#+C?N~LSkZH1^bBadO&vSC^ zZ;m&rpBTmxK7vCoy#^dx7#scXz{34Eoof=Xtb*+3yvlslXS9EKAJHMvjIKLV>GnC@ z=iw_5=QbU?Y(+voEs+;FYj3D+7PWEQz?j;fF%e&9SdZ|iZlXt# zD7-4ZxgN`2u@ihFKD$;=jtKkAU^U)t?r1MxykF8Y&&k+pq#0DEK09j4?v5pYo^x_a zxjRsg*|p6m!xvKt(S=t6MTd@r1%HOx60qkg6Tvt%tsS9=qQ*q9PSR{V8!;rfq7=s$Ij1Z% z(;-ANwj+)3MI)`%in8HhNr3RmGAMZkGcjFN^X_gj;FqDt+Whi zzd-vY$3<6xRsY6AaRsgBhX@&W#&tiIj;c-9@LgtXGnevZxKT_K^ zb;fKq~Rl{qxhEE4b<4rgV}>+1Wttg=hF zydl^!ARlxk5Zf4qU#_@)8D0obB`BOp&!WRSF`vpsJ?`bAS`xgTA(%vOw5EQLT?vj( z!UT-HY#^T7?w~*E_sY?=DEqM=%3-x2)+F#zB}hDf;KVGB@}4auf!04sYld5KcddQ z{&TO|G^Hs12{Yp+ln;LC;}JTi4twTzVcAWsOwrq&oeM_jO0Fx5(O!3z>nn>Dt_geX&Bh3 zu&>qt36f5g{nfG(xng9zue#DDw;j4MexTv>@T(=tp*zx!cq>qQ)H#@r<8czb(ku~v zRl;*?ZzYykW`u-D3h|Z1vTTSw-){ zjOtfe7h&Ia_Yl}1`>s$s31}~GA0Amm7?&nY zPYLQ%s`|cY?F3q}cnPN^a+YOdm3qf>vZoKGm3aC302A+v?UTXY?9Q{xoD0KQkt6~> zwte`rZIBpkAYS9@NkJQu!nH;9xeeGd((nk0K1b8DV5BC!sG5%tYbO@m=1J}NJKBB6 zjIU zm7QWiRl-9V{A<+uxH~butD1CCjYmYLT+Mz9J;3L>`VrG%L@x}gZ}iSYKnj@#OsZwR znU3&i$aekYm#2*9eWuy_0f1-+R;A2Yr_YtFeGcqmrf$7*b`zz;3%(H+hR=OmH|YwC zMFDvmLU^*&y_o<-2jB0l3;8jT>T7>5hAY*RmntiCo}9>WzFV_;2-!RNYD{TK0k*+? zb1%hvd@bJBCD}MF=P&zyg_&Kjo#&}Mjh)q!xz@P|bhYVyiKgj~k21n&MQw*5r=>1^ zoh1;ZqT;oUFm1R4{B?=VU=~FgTGsSh{eU(^c2?@9cREn~1MAr*r+l)##gyonISZ*+tA)L=e+7++ zLsl{u7Xg>J0@6xZPnqD~(~#d0B3I(|>eL~xKN9HlZ-Cm% zc_#b<%@pJE2)s_A^n}pQC*tB3TlaE??~dxR3eGV}CZ9W<7Kv~fba_b;5w_!Bll#72 zcl`YbW5E>W#;01fGkX%3>^Lf-U10=VL!I46gi^VJ6uBfe@3?57zLHaHz}b@aMVj95 z%b}8P4?E@dJeEh-aeYu<|Dq}$WerM>B9fm*PAuG7qo9+MmJ_{IE4B`L=O6WXhrdAv z##H9;d~|Xq2cIj4*7&zl(^*Y-tDzqAb)R)QMM=Ow0*WE9>|2lggMGJro31_Nu z??erI2=%8pNkYhBL%+Y+Izgu9dL9$|R*`fh_$vt6$Zxjruas*oizP#nnkoQ{fQrB11J<@6-?ndrs(vt{F7kA8%J;cuD7`C`pw#-xvg zX8uOeI|^sao1NzSjHUVAMbp~UkY#c z{wwx)#$QkDFg*Pmd%AeuEqv;(;WjLALpy?a!NOE#F9n_#!_I#ldAl`;?WJb-Q73U2 z9*S%&gz>nev^Nfi)s&=jgZX+RxA8~YIvz{CwTfa%>Kq-2zUzDhsuNZfVZ#|Te2a;< z|BXbuXOh|AvpP!OU!^Rc8TWk_jxwNyX{b?#nfnt0~G8lZL2PQv*8gFbxz%OJWc;=E6rktw8thh8OO= zfZ%fDROBXl9opNqt70J{+>?c79vIok;esUsX+9?v!=S%4cweJL95adQoBj&jIvh7? z&2B>~SP6AkYw-4J4(hGmo+iBG!MuI*eBBWJwKej6`%1z!)^XpGN_>3jM@JXKgEKNp z5T>RP<<=Q6*vkyM6P2#a1={<#@}|SeRw`0}kcnI6XFg*?)-fx&ByW$JDPYI0`U2RJ<;vVv%IsjLqu zEd{)2XSVj%yeP~g&f@C39E*#z_s<-uKQv@hCq&8V;kcK=V_GW$ICUs&VRM}lJF-F4 z=?tKLeU$|LO+c~F$5OR>TwyLad`)Ait2(hLNXU4?JAU}6k3HJ!BZyx_n7-3XrqvV-yi7IBQ@CLp)t}_#S zsUw>?9jLzJFUuqEW!CR8sWBqV))-=!A!NU|x_c|c8Jic~wSKd&73-QzjZ$Y}hIY3JAU-fM>jPMKWH%qk{igWGyhv1b_EgOVM*P#gAPt3$Y!J8#IF zw}f{CF0su%WG`B#_zQx(|KaUc0CE=#skJV1ciG6IWD(4^-%kn3;!W9_4q~F1pI?lTH zmoi8Prnw;Jo;=7qXBHosMpz=AS?%xnwRV4kadowL+>wveLu26;Bd4gCJFb_`6j=V( z+B6yD%Dt`d2G`xwHtckomM(sYk|Lc1FSI?Ca#laTyXoygeeEO~(tSmC58*171TFpd zP6^vD(bqp}%XOEvgU{Q}2(%a@5dgnO-d^)!_dSKP4O>RQ+=0r>E|z|Ea)ntKcFFAY z9keVdh6uyJL>D#qtxYx=`fR=XdFm zuDyVSKFdRMMXP;0=oQt$GX@OH%9MHvZbQqMhMj-y?K7u03X6 z^EXiR1v5Nt2*%26@4UC$hU!1Fl$Tg+O|BD~=2qW%8}E5p%G%y!yLBORvpIQ_IL6lo zRi0QO(Jrgt<|h+c;y2b=_!O@sV>>6Hq?iCfuS#J8bJHqcELlFk6LF}6k7Wr*+eR2@ zE3}qXt3&?8dlrd0qoxy&G+xlVxbeeLtjP}f^i;Me*SL@pWG|&7+Ry?-o~Rui!IJ}_ zm}YR##XdjRu_1K*wk_#m7;c4>cDO_|C2y22oGEYWSA=z&#Y=WcttKx>TsE*Q=ZR+% zQ+=-D^8Aefjrc zuGCy1QXLiaw=Ps-B1}%d5&5i*vc}#Nh!DFi5ub3M#F$W8E28C+RA1ULMjDuTlnnx* z2%3@!!V*(HuF-)s3rzT_MX1@Hp2LgT%0eSCHUGT_*&1_KygbfH2pX%qL>diJfZYOZKC;2XP$0r0;t0z$qK4g*@-tO`>f3_j!n-X*v-_uSL*9Nfukvv zwNQ-8C_N-dr}>l_WBrf`w4p7TTgTe4oAa+v>mOCMCqoimc+1TNJ=CvsS^mRCZeeF& zKEEyu!tI*@GP_6E94kAC1{e0*o9X=KpLyUb*7lsio4JvV&L8mfM35Qgoi=I?6ZNgE zBfUpA>lg7#*}u><4{^)W(5z`WcQiKt@+Vi3CEpwRa1Y{*-VHI`#^sI6R`u{}0{AC? zd5w(x-UE5ysp9Q)hBk>bkC007Qv-%YlRr=Ct}1?!=qJ9vRlWd@zPV~I{mmg(CuepGqP8l7AJ15iX?%MlJr05g$^;`fC7iAhI z@1;+;P~nDn8A(?O9d)+`FI<0b=e3jCexF*b{VVcOYbe2wU8p9{eCsB-mAl++~VQ`o;n<>8GH>hTPtx?24V;c*mJAh-tia(?l$ zyJ&M?{#=$847G6lx!vFkws7`WIo=1e4LPJO!mNlxKIV3y;`+lY;aMgk<1l%u^irbc z+v#7alfF9;2wl^s6IU>pfvCVB>g2eLD0Ck-VV`6ZmT^3wP@=A@()#UOYk#Q%hb5v% zl(A}KNeo?AJ%U7UYa}*pe>21V-&hYqgP}HR9kEoG*a@7IA`SgW`F*1k!XJQml;^F> zK#t`Q8LKWWkWJSZ9$Nb1p`C+EJg4Q{Ru-P)AJskxleA$ zF+}6eP~ypB73-DjMYpez=Xu+7=8+Ocwk{XUdGY8nF+3ue80~oSbU3=H<7L=pi6TYb zMiecnT_|GQg$jz0WXwy#lBL>B%cdK!n@?Thkyo?xcFM^;`S7tmZ2q=#B(Y8~h8%(_ zEv(>KElB;v-ib8t-kI=$0-4)2DkQ8^8q=|UVrEOqyQTQs>6q&`4AI~3(oG|68PGf3 z(^n7Q1@EIY34RR-|MLz1t>BHj$!*m?f61H3=ib|=C(vtviDr*qif^i9u85G~-O|u5 z9{&0mp7|sm)R~fk&-3Zt@08xo zUE`_FlZ?AVnXDoyehZgw_H*$96-Wdn#rD*fbME7LtP zb3J=#)93h6Au2I04jj?9LEjjrb(^4(U6xbIVQ9|&131J~gF%hLFr-R=%A40*#7>m|SGyC#~ridQIcdNpn3CrHO@QyiofPRYK`PL6UPif|B2aV)!6V}8ITA%^HBZdAeWlB9T)Lav(CYYL z`e(4aAIfjD2dzG)RXvsd=X9WQ#~>&(VD8G!jT>$!bioUjp7wX72en&VMtuVk@IkNk^h?AB8HO_QwtqNH zu2NNWIhbdP%*-^vng5ZeW-UDU@bENOASGwIRKNIgUMA zTwTH1UB2xVqnE}8%GInb`*{8mBb)MtZd)S9gyOxFjH3G3AYQZOWxX8CdU5A+oGIT^e-6zz{8wXDLi69(zqiLvheSVc^OPBAug?{zTXpWtDa$6bQnqWtr zb@*AfR7ZywAn~%NuIT=IWl8VD!Zg>=Cy9)j1d~=~j~Ey4mD>C18Q;sW3{Z88RGO@wNC>C1ImFTPhT2UBIU6*(^&H(kA6rgm}V^VW1MrKCJiJ9 z*H&KN76UYSSoAgeR9CNb~}cq8?zB6pWa z-!E(8p8L&P(JnAU&!qe`fVh{vOoUz1;z(7cg(+4+q_AwMtyTQ$RL!zmtD{xmYn2JIbBU5 zo_z-!t;|~ed)e9}8#{P`@VBWT=@l+lmyqg04~*^U{DzbD2EG#W=Wsl1)@20FzFP6) zjyRMJMKdzW2|yBI0(z5Cdp=y$5B@UMR^ zbVP#=c-AuQG))Zr9Xt~S+j|_OOSRTRE9LrE%maxtL5VzYJtK^EMUL!p(j)T9h2xaD z8ror9MXKifRD#L%t7A6wd8=TLxvTmYEXt-@I^iHIPXq0>v@+cVG`DlCRiM?Y4@bYq zU~M=f4!loa$w*_OGR?$Z3PRDsKbS5PW!Fh+3hwN6mM&NT`{=$Uwy<#^?0U8Eizu3$ zE>8iM>I!dbS)1@@#gIR4OIS;SYWh-dFw_@WBPLEZKxYjepi17kVa4dQ)<+r z-hK12IPH4_AP(mqH-c5D*FCo8hEyp`Dfa_Pv2OB;byy=7h|dOydff`wDVk(pRu_ z#m>;KVD*Bbd;w}#cI(uwre~!zZxp%!kmGltYUic75a+Z2%iYMwjqYF)OY*yt`hID z6_QJ@ovkI)(3@*DlV6c%l*IMJ>BIDyv}Tp*1AP|R8c2n;F5eW#IEZXL5iMq#{gtPl z9rYkIjIvIS?NCw;pZ0}71_c_0<#_9qNw!bfUu64g&;zcD0E$*aSouT+-K9@7 zE*zZvG9C03ODbvD5{I`}m7!#AwOk!T8|FkT__acEqPb>a=xbQQ1jmZt^IsL)P62KC{#v*DA6QMX8tt{;LpYW41(j6mykG;wdQzk20D7x zCY{Fp@2@2SW&jgjkNQF@Dw3&G=4DpRItUT4;SBjzh*A6{mh~aF>IH4?N*1Z;PB8>9 z%d3R}edT0T{U9qhHSlzYvSHP&-io7RUj3d&)XX&Uf{RjhdRUO_u3Ys(Uk@%@S9$BH zxK5^^A5Qtvjg=Ua#D?i_E`$faGCD6UKd62fP%C9Ip#l_=A_f;*_s^zul66HGnwhQ3 zCrmh*hqZP6Woy_V3454smGrCVH{xlCGW)0t^jQ`FJwU?0S}G<5BFP7DpE+1YAWV6> zFP_lTW<8oqqpu(g-VpPZpQd)FXrATtj?9{sSR@=#GM$mwhOnZq+nZ_5UOC`9(&?Z% z*d*&Dnqg*v%qCA+r{VoFvdC$r`6+FZJjjh)3hc&mj_0Zo_`MsQIbn2@m!oH~)@KWu zgk6m*@?Mg8%axQuTwyd>AmN`#)AT6aMLRTYQlM?|s~yE{mXKo_#$P=+aCUOEShQU! zs*57n{ZkFKnqv|}kKSCCPS~O=I(Uc>&JxDqS}kuJv~EdJm6BgKuersMm*nS9VI5c` z$KLg#k_`pvdI||@CC;1PuxH^%ULpmAL1_Hap&H9n^Y+7f2xIk~~)2Q^C;SdcDL0;Nz5f>gad`^m!3T3fB zIzg>KoqmWDlU8e8HV~;fEj>lm_eAVH<+AIzZ`?t7lI8~?T1M;%#>#ReAwD?1Qd99? zul+JrPf8argz-=}8DuE|i3Z;nnV4uW^Q75p6d7f2Q6|Y~SGHc8MB#4~YF8KoO~cfD zx~z9?Rs9f2Ks7nH_Eno77RC_|kHh*B%T90OgUdgHOi(jRY(Ii}zQ+0P*^!`NNP5j~i<8#dZIp@mMvy)QD+JW1zn4N2}ottJiQt*+_wQRa-DAxzUDj#h5^JcV(3-W4in%Z<5ER@ zu|{VfVIT?ain`vojaZt{I+3-(K~S-3iQ@N7)7Zl{ z;+X6nV(fu|*+iRUL)0-?vQP_a04&l-DAMX^l%Vft)NMzzrv#6YPtW(>^cP z^NQx(y3&JG8_QZQwOzRe-J7swXVcuqoq zUq*aM&GFoZE~4No|LP(zJoQ_kUvxIc81rsLH@tZ;bc_YchOmd~nDo1Ow5Q<>fVC2< z`(*LV*^r6VvGG((CDOGjD~FM|mMOvnicaJ_`Pr`H6NSFcmFsCHFn??knJ9CtX#LYl zl9=2dIaXoFMvSAdeSK?MlE%{QZ+@BSjZtY_L=iEFHN@}VN~dyX7#HK+v0rz5*QspO zHoEs`wtGQsFV}_AJ^A9DgIFvvC9Mmx!h^MSYp7}~$jC$0w9{F3^Lt5>Vz{dtC#{h2 z5v;-a%2Bi&+m@l@YrMQRIh)&%%c&>G?SJC$5F1a}3zG72xN6ue8Ez!A`0$PKZ2$=+hk*R^BGQy?QW9-@JpmqOB!7~?>sfpmC>UG1&V^Isuq=N^q0ZH-d5y1EC> zGxVG*?x@Vc{!w%m39bSG5WNykSmN%^4tJ-q(+{)!GK92Ly_X@7m6UydT7b)&W(Xk! zq61$?5-Q&1%z7LL)AT5FW{Wnm7b(`}P?-X>6LIJfaCcMD91SE|p>Ah+`h6Q^z97cX zuw+>zJ(P#RH`8`3R`KS3zEgN)cdO#uLn_02F7m11B=~@pa~{(jh00s>o5F{-(abJ7 zxfxubI82%YrPsQrCMcUtYbKh4bz%dICL$lMh@B zBEgLAu`t;}do#y==IV!amsW*Si=7LRT)V*G@FTL_;66D*I#Vub2qm3Sxyt1=7&CN= z^po+v=h?@HG{%B~;K7?{2Zkw*ppPY_^iQp+yVpsDQt>7mHERMtKf`K&`mA}N*T zv;8m`XS?^(DOjIQf9-D!asD#M?-x+%()F{zbNhn)!Y}u9$bt9M7BRGN)B8)f!o2`nV68ll_Rr#}^vmJT%8Fu^AtnVsXYLtHfOutKGuJ%~~+B4D}0 zwu0_BN%e?r{HDcmv^DOPU>i&EBHEg7&7~HBYmc7stiW0mFJZR*{an90LL`=bjcK3j z#4AfWu-;?uS-DmkAUgWoCfCte@Y>bM#(nUMv7DaRmOP&pMgx&1Gqc+SL5Nb7-w}VM zx=s>fs4bd^pudJG^Ri1QEp;>{b0!? zXaC1vu-NF83}L+?zMtRA$FGQbnNP-c3Lf>C>zK-mE05hFyRAU^9Z{UcOsuyn+lP1O zK<7aGtevh=+3#w(?9HTEMAV*4N*#!MhR&JUv*LaE7-_UZRNQ53mnD{l>Jj7GlNlLr z`!lgwr@4I%Lk-cPG^L+ok_3gCHaMu%Gp zCBb@i&t66iyd474TgI}p#k@nIn~*75Zt8{X8SUx(l5Vned&>ORdI#6`HUc|-W=gE_ z+5XJP%rqGRo-S@_n6Wu%E0=Oh*~i{76r~`w{VEV-p__+@V+l-l-E&db;~RSsg5NFO z6b_45ozA$?c25>U^&q{fwdD5v!0&gumH;EEY%78>3%##Q<5%<%3w;xscym)qoWM5U z^+Rjo@sG$xtUDi|hpBf*6>k;uU-maBuslQ}DuD_UIV3LBDS9`bMRA!^ zQaQ=J=S0-a8D!g;n22&5POu_G6Q3|f@owXQ(y3ABKjr7S3DHNDLl&R=P-V<}fd7d0_As=BQ?_on z&_ywa)+k;rT$PTF-$3^^!>BL(u4-d%)=26ms5uR8W;1C1Z(88KXG3iX_c@5#(Z( zno>03EEUz}VK1&PHQQ)gg7g6fZ;9qSd9_WmL9}nKtUU`&f8dSVY9`OZ-CkG7 zIj8|z%QP!sn&zOKlW~h~jP`0zq%#w0^Bn1oUQEZvnKEB7E9vBq4NnBHu9S$(=OsM# ztIyUi$d@AlsA>@j1Nz-y-~qcP@GB%#dPglLz&rTzR1KGQA{W2W-j`qmS({ zGwfskRq#p-VtY^P^^L^rKA8apE7HBc*g16e(tAgFJRc524@pNtIgi@aE64`?ZCF+4XkhujeeZs6v`=6L3*APN9@zj!$!S>hFMI2*6&LY<4gq zVVAU_ZsreHdpuXzY+y${5)Vc1U-To<2C4B|0CvQ4v0`f4r_Lzg{2oMtydq$Abu3Bd zg_v8SM(nO_7!@bkAOf~11rvr@xI#4*MwpREowWOK?Dzj5abtjoN?tOT^RXA?^oB;#p;Yc|M|Nafu(4FT8S&#gVcQ zOH~6U-0$iQ5BsWDsNhCR8}d$hayKY6<&s?1GKy~Og6+;0q8{U}kCHt|72@qUWqv-rSSS;q$p4T9!>jUUI^q_^=-YJlS%hk9O-YM{hRwGt0+Z=z zUe9TFuEtCGK{xlblj2b0g5nAy=BPUgSNxxlYj*SuMGw!eF)ek7tH7f<8HN&iHCvq}WB~ zHO$j*TLwk6#@ws;F_>e44@oLv__u}%+V1YV4$$=ufYW55$4$GM%AbrSc9OoRKI~zB+l0iLL^dgN8f>`RK86-hh!>58|e*?=F@#pddBUf zEuPGE529%s-8yj-Z;vaULDsl^+i*0*1y(XVX17MctKYLj85A_EhPFh#$kpBoX~lEF zXkXWAnB)VrRirqXK<#Xx7FeBP6xkrGd-1y3T7$GEB-8HMX3j(IaTZb!KXb49c70EwojzJ`xp{pIjCdEp8JbpkG3sgRt&&Xj?a(uZ!AW z!5t_FYc*hPex|e-X9SmIJ@?8IK5XMjFyi=7{E{N2J_uyjuS^p3#m; zYH9WGJI*&|O>sL+4gUZ8M@OwL?Fnw(_%c1>y2K!@?3o@+FV=_!L-#9~uS0?`SIIn2 zBep7g54C^h#3df|c+&eT71AHbLvb4DxA&;oRT+`{Epr=*iXt8_+gtm>GpYz0Vh&9i z0hfsuDR+KLJ8e{F=eFg2Q|uoBrp=rBp;l$|)hMv;WB2EhpuPkd4zBFwQ!v<-amYP- zm`$_P^|TXO*!3MW1Bu(ra`jd>Sxrod_g)^-Z17#Zm4L+d4-Gto#|5tDn>ZpiwKCQ+ z=XZ^PEbCmo5JlL1j#m`8u=J_T__MSn%?l2NKcGmpwtCf_yhwVOw&JvEj>^4F+33ch$U4%)RC1+H0WiPLa8 z&JF}3%k>G=Lt$M+$C#RNjd&$fx!6~XL!!hMk8C&AcOaKEB5;hipr3O`%2F@Kw&J6d zJ}%#~V21mgQKAE*rEYPXuu*>ah7aC^tnkw^F8ikN|HDUP`k~`32*p!KCiLrPGBT^8 zchsM?f!N#8{&bzvPa5T((eJp$X81hD=T!d<#et$Exw2<4F!Z-B4!dhjGoBm_h|!P- z!$msHiHOWChI5-THf;-rxzFclL=xpAyzM&ypZ&`H*C$(GPIUTA!}kUozF^pBZ*Hqu zYUgLLD5hM48?2SG$QvukQw75_oQ^|n6Tb|d#YoIM1v$5iVi()ahfEJqy4+hF-Zt;N zTnlm#i~n|B$GL5d$&SSqfkL_b9CO_V#7&!dIHCVS+O&u*Jo1fItch@P)tD>FtY&HZ zbVqAw`a1<^ZwSx#f6oIo0y1-54Rh6pNxVR_&l(ZTjyJ{(R#$KLJy*WHYH-*Ss;@eK?IA&VCe80$uzjjmPcqXBr^)#ePVTZ-u0%_&vL8n^i%qd~^v_b^b>WtB2;3WP3 z{oYYvd~fqbshhwy)! zM6BrXsr{PNPV?xDMx*7>c=`QiVPk(jyWP22hm9=h$Suk_ZXy!Jnr&{|s7_^(FcPuL zJSG{5_VbRAw_Z^|zvK$P%lAS1W zz1#Puw;)a#Z**ipedOmmS7QZK)`QU6$B&g#r%E!!^$mDemFk4Ax;rVB~P?6ix`2q9jOc zp&%FMeq3|z4Vw=7oGWr3DZhHqyake@p{5iiS(1K1Nj12U5^UC@COK$nQ3GmU+h)+A zF<?8S%`%iONvC0miIq*KoDJU`kQL2LJ#+vG?6=X`N) zD18J!#Cx|~o*S!O*4?WT%jB3NCTsV;dh9RSO>~C2KHp!osdYHu1`T)i`Z^M97%3Et zE3=|U^Z)3g0w7Y81MnOAIY&AB9Fr;S14jJ+R3hF}w>2p#eosaWd8#eiu9jyEqQXHegh^k}5 zJ1l~b_;O@Cz2mV528zexwmntCI0nTy@DrSv8;9Z@t-<8`8x`w&SdPQI6Z6_xXE1WV z|L^U5%p3dw5}&e(O3hr|5XXSagHi67a5uaYn-!3N?e&=x$ zWL7`)_ups1=9{zYtzZWR?zKkz>67xYc zyEJNdu+6kuXxywhB{ux&ry!|PGr)oFpxR2!K&Dr&7aiE0QQtX;qt*|wQ~2STjguk- zM-_BWvRI&g;&;^14`M!1*2=e|usyroKvu)AIc6%r?a^W(o>!Aw{gv&8L1U~m&Mp5| z4)+INQU|1g6v8f6w1NSCnOk)uT-~7;qTCc4K~+w3jjtNUh9gE&2k-W)sy(s9Ca$_% z*tB4e5?TEz*E;1B1c8MsRA$~zk{On>bABo`6p62?81*A-n`w-)+IkoHJTZX_$(juX zGX%w8q#MRR!<2O^eVhE-l$}?kT{Q)*dT22!w2~IGW1mNP?+01P86|CS+I!;`*c~(E z>xLjy1szODLeQsFrJ*;;knCDU+u$4%vy?!x59Fg--f_uK-$0Z`7>x#Jf6c9jL~cDt zJpohZY0cwVz!fx+!Vrn&1gYLfJS4T!nJ-0i)teZjy)DNb;WELo1)+KTDwIa^t%pi2 z53S6@=aX06q+X)TaFLx#5*RkpI$J!&ROIbGR=r(s9Gj-lRfHfeowptJSv$`PD=y~K zA{;vzjEd>Yg>`rP^A53*>8MBHhHbb6w_h%$%0rFNR9!7KXpLv!K4V%!_0bH8apY9% z&|5|7L#C}7#Ct9;L-yH44pp+PhMo-Lx^FO;De6keBxDMaN*^m;_&o3D!kTF1D9KgT zv3qe+>Wr3~AGkm+)Pep{67R~xMmJv2K^wPsqv2lJ#9BMqt4ceOhO;MFEWR`S%wJgi zf9hRJFe!F#-c&e~7<$VVasZNB8&=kshKgiV2<6k0Gk+B*Z*y=0)MC!xK4|t@!JE9r zu1z5##qSJxjg)HU>K+%S>!2l4`aUsU!y)Nq8S9gXzGgL4LDR8m*}0NzQo@CV#KApL z#u{TPVek+%jk#XKzIh5UQoCUEINQGqX&H=dS{4#ij0lko+t~h6x2CV7@1v~+IUJ&2 z^`HL{W?D1km(kVg*k+Shh8paI5yF#(dGN-5%AkKhT@4%B#*(!uEKSQAt=Ab^EZ+e*zzyS^a z2;4*W;LF@KxZkXysNyxaws=OZb)mJ>G>BnSLw+Z5+Yg>r_x%V=Zk;^HhmA?9(q_7qyt2*FMApNc%LvgDmAZR1R z{~rgWo1Z34Yfs=@d2aeK)AeS*Zt@WJJy|_2NY1`9Bc3*1w@p=a0*>}7J&}8m{(GWKXbx1k&sf{3_M9P-+w#M(jYDWZsd zCth`MQr(tY7Ovh%^1eE62Uv;JEItpN@$hY_b|Vv#A3*gWlImf7x%1)!v45QOf!**q zirUb*x$XP7XnSH4Cm0@f{9YaqJ_oId;QD`c=)_b?DHNMm~mwtI|a z=y8IDQs}Xls&l8PNUv9x{qojyUtm)C#opLqrF}AIUiRD2I``hM5ygc|*3(frJ=U+- zvz1N_`Z%e|)mRThizzCG)&)CecQ<2&dpl8*Y%LxY>)^hinZgc1I$NA%+hEYKke;75 zC-pkmuBVq`l@C9I(J zd;Ze;eHF1^WgZ^(pamnciMvAN|t5al|qT3NyxuA(Bu8Y$TM^btl>bw zxMw}7Ji?~C(VlTM!qFz*Lg}QiSc$S|pV>i5_JTTV8|yGVo#k6h3xllOEZ^{FMA{nZ zSgT^fMlEO+Gqq9QIzyPUsBin~lAW+4p;92_cmW zpW~v^^Ax)6is2h>jfeNOL42xnKp!DrEAeF8pG)Qpkv5x~XCPg+;zR}1F9EgJ7M!ZM zrgQ?D%orw8wp65h$vyF(@9mNpM$&?b+adBD|G_*~@0%u@F~>^7Q}8^C1})rG|CdCB z1cPD0rX7h&E>*a^V)=Z~dZ?-d$5h zE8&Dk-q;xn8d~iQf){_}XQHWrSvU29RYqlzq>4s_#q5jsrJ_F4!VuQR#BH5BS@ImG z1wEcT&O9^{=YzO~W`<}Ey#RUWQ)RGRBpkwHY$Y!~o zu+xk3O{8`fyv8y|B>80>@wE37S@1ju6N3Q$-Qy?)9+@X^y6NvXA1dycMZxmP{mBWt z|6es08$vcQq>~C3Kcbk>!k+t{*zB zp%w?G-KOBh{r@*7&L78fNB(d!@hSjC=*BFjsVsQ2}*%J}q z4S05WRD9X#ok*ed5p**k$2EL|7n zgmNABJJm11-5S$^gL$feoo{U<+jW;^fs0mAvWAZf>8@0A^$sDeQ?an??*7EUDkEh# z_~$TkAlLT}?wum58*`>V@o!3NRD_OUd+Ihlos$zNpOBwY=*a#{S6)TVH_!Mm+2v#E zmguIRM6~*W6IpFcETXFAj_jb{#vI6qE~I}a&(q0uFr`$XNL$j0JsW+8QHwy)jX2oF zLkeHa?(>)bZ2?4iTmNEgZ{)O=)8U#=k?3g<7n4{GucuXY2>^|K4c|DBvLm6+pv}P^ z;9-@&C)7K;;$>pv7Y3N7E~{AQozfGFC8(;BWQ7kM%7?3-4Gq(H(P{S@Gp{qA+AXQo zySJ`!VaGt2ixn%AMRV9hVoM6ae~+k#uzcb!G*fww<=tkM+cr=@ustIH-VN7#CH07| z=#%4!R{U~mG`!62RbKpl1$Gj*NU^!L`m5gg9iP-niQiOJJo1ovi`5b|^RT})Ap2pM zmaZO>w=CDUQfaESN7REz%sg517Vgb{%ss^AKJRfE*k!W-Cn==)cN67f(ws-QmzJKm z>599(0sR&`MB$gi-qz>xR18b@I*3^X0?hpQ&!~H}Z&~jD#Swl>4Wv-#l_ks;*;}D( z+WBJh26rN*0}n_nFn|JQD1(8IvPpZZw`OUYu9$(VFob3fG}C%gb7a%|ym@_elQtQi zq$_=D(#i3&W}bDQSx5Nrq-4@YaV}0$ znBw)cUlpcQI}Rsp{Ld{mf2XLOP`FTgZc4Q1r*(?lqEcUY4r4OmG`kk?|AHp9I6x{$y*R zfPDA#UZUN0-DuvHa+z$7;4W82Tqp>bSg!owrCy<$X<8jb^vXDF7NQl{N?_=x3i&U8*+>d9LX0FdsWcy6!J1y2{H3$TP^Wv~zrhEkn z`;vTocHI+QX0iqupS_64COtg;`WhN5T`*xOJC1yI;@QRV`sQ@F+2hlG^fdERmpa1H zU?%DGDauea44%Oc%@3O+#7?Ng-rksrlOeway6n6%=GsT9TIGg3 z@ZQ93=``QG24)C^F}6G{3-myhF>ng1dM^B)9!uvPcM5Y?Y`{C-doJk)U(pV z-*P_o^Ch%0*+70BzTp>m-T-b|m}X7HRJ4Qt@Fv-@;3uZQc&u zG&NeW5}@WM?+XXjF<2FKa;TgIoW8(lFmYC@7iPBsG;w^6)Sq~OLhCS1`w(|7F;sox zzj(b?OxY6To_v=zC5EcK6>D?iXWKVhc~&JGS~WCs)->TDj)^y2Kan)V%7au)V+@HG z_%VpO-N1upM3A#^WAyRk%*2RR<^UOtU0yeP4G*qDC(wq;s_43k__gG?%N2eUZKbJu zBleIwC-G_!#PAiqM(0dg(ho*0Lnx(VWyT|!D&08BN6wOvrk_+A_dluxjeJ75W!E&I z+x%Wa@{J09Q;&d74(T`62b61k6-zD+H$>u0TCW7cSeKsi#1<`r&TzTIT%R|A4>R;r1j!+sJf?XnjBd_NQO8WIGJ zW3sIVi_)_0f+pZiz`N_^XwBXB*^#6h1J31VvBKvGnoOMX;7K#0=kQhS!fT@-{rR21 z{|I9B?I}u=&J1JUQ?T;b2~PhYpEwv_t1D!l8CPvDwGhGN;O&MeJxS2>6)t3E$%dnq zb$sjeGQ&b@<5wN+$D5Us1uF2ngcg!lCXiC^_Gg|aof(Y>a^^L8r*QBSiQxSC2gO9o z0IhCHBay$;o<0BGZ={!F(g^?a06Ad}m?UvG>^z>_KRM^pXW*c|r%ME6Ubor|?nQVw z8J2(Dy=(}_jU@y9NJmDrIv`LorJtLVo{~?OJ)^RoQM+1~!1eP?t%bj##TB5EC-0hQ zwGd1MnJ8Oo>zg)@`dcM!{)JlwyG=15cU)G-?NWCcI}*9L*uA9!{tFu&Q>O-*({4$` zfv5Sm3#}G%_XCefANkNx^&JUjQlj*zrEtJxQip^j>F?`&7wfRf^bxvURbgvOnUF`n zVmFm6rG^QLyDZ9Tbd09z#W>)vuHZQ8o)07RTor;lw+yIRh~beoD0nvsXvsIy?m5)b zpGN^j);w~cID1A$s=XyNO1yiKV`#QL0;Tn$=Y4sliT%mS2tHVbGV9zyMX=@p_hL>^ za`@{~k#5nrFq!ri$Le8Ibcc>eHYkJ~p0adpUSGPxb#Ld6J<|5;(a~DxDVZm~n=PuW za!XELCfEkm%nJw$t~b4I!vx;1;@(2JE}JQX#-7%YLftPvu3zz*Dt5Zs*f<6fGB(3$ zC{{Tk zQJQhIB4AXq(T?pE4EbNt-~17C&-ZSk>8VTWtQXmX9oHEyB>FOdiR!|euP&G1(xSm{ zf%6hiY4K|&+<~p(b9_^_aJ1LCF(MB#b*+f1Arwr{@vzH9a}O}>ZLT?gRxd^{wuQEn zYOSRnldQ2hcCBaPQ|GPS$>Eagg{-wfnkpna;EZSV&Mp$H!$r=Ejx!D4%`4d$_1hz+ z)b-iDkEeG#8Bnv56N}p-Pgs@Kz$EEuvlqhfV~c*a&rU1%*1sZAI<)5UqRFO2@F!kM zn|!W6VQ8Kz`G_y5&p<{g;?%7AwopN;Zr1Xaw%O4B+cCB(=pvfo7Bum5uj;@f)DwEM zfnUUyZ)b9jP0HRXGmD$jPR4hLsLEy6RG;)sb@qwnkM0g={{aD9;YmsM;%$zq#?*Ui zoJro#QHJI^dwtbm3=%5lyuQD(WIKs*Of$_Fr54yihG)eIR||yrk8u_df+aiq|#mqH=2ft#D-6ad6w13CCx;yU92ZO{Z(ud|zZssS4V}6D?GnIVspN z`dR8>bTm~lhV5l})|;76zp+dMhvJ-uC^f;)QC_uufUQ3QrICly&6y;+3l2I{zGm8R zj(MSypM;A9dm*7MaC=mfGwc-V2#(?emxEtX*`rZ-EYBiYNwwf^$la|SM- zRS4~afRVjXnm!~N+_8$`F1@vrohgv2xS$r$y46FxCoV+u`qTN}K7~|vJ_mjXK1+~{ zH_oaE7K-Itj-#e<@71Q}Q|{N7 zwmjM#1^mzgt8lop`w?D&A5!*K&s$T2L@-=Y>Nd#=On5tDxtH?xj>^_w9liN|c^#Qags~ z8O-?!pD+t#(mM|J%rC?xL(9iM7DGS<_Zo++b$aRw*RuYWz;@JP&fEf!VQCI?{rOES z02)8D3XpIN!LBK7SUCiXkjmI<-OGY9Auaj+9>Lq(hLqfAI^Wb0a#OpF$rj`Dy&ppS z6hAdb6!lipx!V{H#9Nb^>w__v7A}?=c;n#1PA75DA}G7p_x46IICoO&7RHJkEz%Yu zune#TbCcr9&w1?(u3SO;Le!eXzB#dLev2Q2k)ppejnc8*rykdjnu;Pu_uUe{(Z;yg zY@T}u>3`C%PzkTg4T6Q)>_(ZPubEtwIOC$*aj8Sq_P5=rM++e$S@FoA?usW8C+z4k z{%bTBUUC{mVRJKjv^w>mBa^ErE;492MLCm0(AiG=4tsV-8)hO+j}{Bg0yj^$HA1_E z3?x0#ncUG~?bkIBXfwNa_;E~n*waj} z8b;)^&7|OU2zO>|O)V+hW?Zt3qGamX3YymoP~witS53zQZ-zRuE?C-J*b*e7o>6sv zT@xr_=GQWK5dPr%HL23T$Xgy)6?e}cSIbLO_%TUk)#eM#u7qP@i3HU9MQ2mV6n7Q{ zVfC#r35qC&sd@0~X%Jd6bq;>MVFq|*7I#>IC6YVLgIuPHRHr)>Kx0$E?!Qig?B73e z3}q^*L#Ug;*i}(BrmMOm+;ny z!quyySO4#`zt9hT$Xt$u(Rx}02i~M422aytLAJ2%3-scjt2-;?)Hp{7+VYX**`Tgv z8DWLl9Kxj`tr*B&$3;%j{KnGfa2MEQ{=ZYWP6bl^=6;uUk|d%YNs9YdI(<2z)P1F_ z`|_bPz8}SVu<#%QGa9vNX zttZboC?-@b%)$oH-%x&>A#Vxj^AA<~q{S;y#+Mj!LQ(UXtV&^KS3l6-LopSjvp<$S zAnJZ*M{Y67>dZ+tN5~}ntf_LK#{q2NtwM|3YNBu8)kR@ zT7SV)hd!J@NL4hOLv|3$!Cl=^L6DZ5niU1fbo(e_57;?>3as*^U@0YjO;L$3x1EwR zop!fR_WT`QAiPG^s1WGO<&9bJA)&EW&Ik+aA#wcwjZ1z8U;M7&fCR~2<98Uy--%vM z9QXydaaO=9_C%LT@!Zl; z!p5537ga+}jXsVHMz_5%OO;4!=QZBX9YmN>I8RYttlNm|_1^U|(rj8WeKu>ii8Jb_ zl($1$&KpL{1(R!)l(60PGovGgks>&0jr0j5Vj;ifCw_PP2l>ep0eh4DP(zy1mH_+3 zDUQ7)$^Xw(Fgj?vkUi2K8K)2KrY@+3_*|t|DD@G!tSTPWRO&c(S5_P0s0>eF%3r=_ zPrYWhj?4XfJML#}^y^1;j@BVt#>9!j#V{&^ZMx{OwFiSzrQEdIdQIO!3*O$ON$jgbi(wdiNk!bg%zw|!fF4+ zCtqQSeo%#$)#xcH5;4|=mdil2h8_`42+!bq%#8LfcAN!2G@3O>9Lyz>LSyXO{p~U% z7iRRyB~Ai5DQVBJn#>YioigkijVC7PYc?*$9=NlZF&~FF?nON0`@kV;=qlOUDMnSF zGa~3>*y3F%0~8W%D$Wywd7~H*U}R{c$DJfb`{*a_rwL8-3p-+uM71QL z`^!7A`}n4)rPexCc?tyk=aZg!l@RmA$_+bYe_i;E104)AjbTG3siMSQZ#!TeJW?X{ z`L$YR50O0FDB_1+=W~PO{EEF!S52N55yGrW@Ea(q{F{P2=Mp{d%3yd#o*C3+jdFKG zxkIKkv{#&;zMWjf*j^5v;A}isQWX;MJH+JkXi^|Mm=W8q9I<&%__AwAg{kH~f;{Gp zUJ*!Xj4w7MN|^Eo=X76Xu9WQo1g8p2pd5XOZy*+}5xVwj_q%?mU6{nq_gmH}b!%IM z7BMG37&+(zBhSQFM)%@2IX4imIo0;GQ5h`tPDixsBw^-(k^JaGcp<84P|Gnjp4@-luU~xK^XE(AfKB*glwfF=@I)V9CpBBrS8HcGoA2i{K zN)BncYYB+HXIKCAo1j-TZ4)yOHVdsm?uh)KYlK#ZJk^-r4|_P;Z4xTv+m$CdVhKwLq zavTPIp{E$4X%D+i{2+u7kD|X} z?k9mUupY`($&k-~)cFQr#D)g;Jd*ZquWIwF?M&dMI<5+$3OhBLK@kPgNmL>pD63S zjXquwelmun@}>vs5KP63=HuDV0^xJQEosdbj^)pUveMd$g>bYF9MdR~>|Dwf_RUP# zy=}!7Ns=5$*bsQLh2kUcaB}{#Gkt1MmT5poq+BMT@pU?xcqNTJ$$9`P+LM_ zm*tQmk4c7j13Y9&{b*Dt^#a{hgEZ@Rxs8oUSkPTA>P#ZoafbbxQp>>XNWKiQd>SI6 zJ0pcaQx<*8e^cQ`no1a9OXZ%udIuDPSkA0*?vRKDo^&!5eml!|Id>G4&-|u$8#d!N zrK6{HA?auRtDi}*_Y@BDSuCSg zL~IZ!#w8=?{!%FZNgFFI_2I&+b<`rUy1PecSKc7oJjclX`_Ve1lwXb{oBuO5xpX7A zO|uY7axOg@iP^lFO2bdIySD$YmU$aWEPKeZh3(P}ZQ+)8luWJ9;hkc_f(^yu?HKc+ z)msZ2&004;BmU7~=CEfiZpl-aE$9;}DzueWYT8!Fwp88AOABxW{bIX~C@M^FWMXWa zfg(g~+?PJh)4wAlNIw~vf{kq%@qXe@<1v=J(?Iv5l%-&YTGGrBW3dIlo{J&7o4-1` z-v7_?8(~H2=*=X?G-~TUEp!st)~}o?C!IqS-o<{lASXXJk6y#-)s+wt(;u(&XEGQi zmXv`145e81j$8`1Ir51cFqH#?SOt<)^9L ztkMM3kYOWkK*C)U#|%Z+SC?(2-^Ghh9p>Qc_rBkO7@A!9TwF~7e< zFXv8e1Rm<|@n#t{5ev95FFQio328FMUV|0el7aMOALGf(m;gMeXEO*)^6PRzW#_+4 z{QBI4tgfOvF4m|`&8A!hQIaTke5P4)UBn$ghu%p~I3N{GBu2Y@WoKuxX3HsY6Ze$m zZIwgZw16Mx)E(_2tURG%_%rZ1Ji7OcQ<6Cvg%7c>{EyG%wbsNpq99F{)5&ff>tC#- zj>EOy$%bK1syBx%NEFtr$%Uoq@+;_L{by#5%UY~G# zleH(ilv7o&WbtKzwYJqwzrd$UCO^BEy}P~mMW&1>#jJx&)sbA2gI927SwA5r3V(C? zjy-HvmR%c{8C%Gww5xR&$naj|2osGMh z;f!h|X=Yw~Li)RjO_OSDP`eAPk>}4-+=kLD{ zPICAs_5CAopXt+KHm{ti#c{drY@K3Uh3>7|#O-Q|(~?~L8A;X7-L}OyF_jQ{;4HJf zHi7kqyP8>NO6;(%Ubjh_@DE?Q<^Sr`yD3dMVNv?|{x|AnQ)*mf!n|25p78%Iu8Sx> zr^W8dar}f)c;ICTCVO{vC_-j&ivh%9 z?8ci5wVWkEAp`zWS*EbV+m5fg`a3HJHuhXC{wqYJQZJ>q30Tx%1p<$R{|L5;EdVh< z&cBtH7=PICkLQnO9D#8^gWuo?C5*t;*j3VF;c%TGz65hY)J9iy$ms%G-lb2=93=BX z5h?L!``~ZO5X@ieRt`eTE5VXa_|NjU&_UGXDrpm|>0<}&e_ClYI(sAGoMz3M;SuPHh^V&v$C^VFeMt0%iGC(XLH%j+FwDpH1E1WVX3SQ1m zXEWO9U8lS;Ti|gZteL~Z(9dN+6>$~DFzVXs0oCC{h5#Q`=8XIiSpqzL5oxCDuoDU{ zbqSW-a*euRPqM0N!sRjICW#FVUYv?qK<+6n0f$?Ekgucz8PC2~VI`|nd0X)cF6}y2uc4s+bds$c4dUNxZAg^^HhNLuFIDYq2_0mQ$c?O$^qY!$=D{2G&aWMIRuB}u zH*_4O{H!u2T;J=L0f?o1-pzq(AFE3?{08W*hI#yahuWs;!} z%);Nt$9N69N{PUcE=ZQs@)1b%jPoE+kRUVU)1#d~f9Wgq zLfHB*WViPU)$hBKt|I5%?OFhd0zVXAsPP45kgY0;mTK=&DV@CDszPM;3YhuuGjNOz z6;?>oa5YP3J;x$YONsU5G>Z@xt7yl#F&NPp2X*n)O1ApS4%HR_6z- z*zm=T6G^ApgZ1|49rQ*@-1@!B+mnTSoT6mYbu+l*y|e&WIIJ-soW5{?pHbnCfj`P( zXnD;oO*Hv%J#Ld~E4g`8nDaY2Si|1#<^Hbf)DkpA81k+-4b1PjF)|=Uh(V4eL=-f? zHM)tGD`MwDKSbIQLR+xrnq9<+J~Qi+E>RPIm7NVdAZ7$;0ftDo4Bv8tPfp{RU<4Er zPdqYu8zLmK9Cu%W(N|XD3z9K#tz}E^gU_liCJZWWhn^uBE0vqc7Ej^Y^^grLkwWmh zlKUmwelnH)yr~j*uk=7yp8AQ4Nf7|p4%rN(n`toIMjx~zVcIJH{BlU0Ac9_R`f0SX z_Tq!n5?H1m6<^U?_wLvRrtjih)eF%Pe#~jECyL&`wf~KrLk-OTGXyeZJS;~V#p!rd z_}uO3hyE_lSAXJ`FU>rnAapnRV34#mRXeyD{afC;p$7mLBVwpdjgSn4GxZxUa&LW* z$>+x{gBQ?uqB>sLM9D8Fp?ow8s7nFC_1oVbU)$I63O5qF_{Ov94LhZMgS z&9eGsr0xYzoK_SFE3^z2Ock}p{1_{`K7X!yhT_WLUT7u(jFCQ`Kw9m1NW}0p6pX;F zuvZSK3Cz{nmfhZ3@&(o0Y^3M3!YxSd5R2uohzIASs* z&=@MD$M!M@I+uO`zKp<{HJM`^Zrd&5^5{2a>L31%)irR`aVd|9rRjZ}cs(DYHOL=S z&vo2M0i0ak;WM=Cn1!$=K>daY<(xRxRHHdw@ltq{qgZOIwDg&%=qaTz8%fwWA$l5# zl%Yr_SEoAhz>e!za?-iug)r8hSgb4s8;nD+08Qzah-J7i;(WN>(s;3aU7OILoq`%e zjvtSSU#dUve4@vZ!o!%oktH}*9i*^5u0dd0bdN*vm=$EW$cpDt(Ae5pYmD4xO?;G* zvmc;TH5OR*e{YRoKUfAshkPF~fgjz9XX*{lICXdSt8syHMCxKvyL8>Jww}<+od#>k zQoxQ#NXo3+b0bZoJ7k-58MvHWk}InJLUB+HNl&Wy>qp9?A5??JQ!Hyv?~_N>=!3-5 zs$ijG#5GB^<^~;@KJL7o*}ruIwS|XaI5wQGk+o>xyjx;i!eLRF?f2~u`z6A2w`&Xa z)c9(?aJf73`PD+OZb5wWMjAit1f)IEZ*KcxC^KT9i04~e5#Gc}u6*zj>XFpaEx{Zr zHPi;Ogwq?M)hj;|rr?|xsxuwhV_(bVE(A%2oEeoGxq1U{ z?axQizG&;fPZ}_U&%HKdk#LNl4kg1gKL>z%SfZXl?SPSCk6g%uHXJW>NfY@CjeSJ4 zg&yxO*2KqFux34ZP^}MYf3HSTZ;=fe#yauV7{7k)78+PwlU_4)f`q@C--QO|N)kvZ znO<|dzjkYt^^sIB#kRh+v{=K7JuvLA{caZMx+T!4#kil9W&YlONg7y{f;u3An{uS7 z>v8aoR_@n130M&llpzP_&if}wHP3c0h+SeP2bB3-;-T3R>fE#{$sT-mi=yUKGxl~#l8deCUct2UYX!dUTGeA=Px+kX3 z?>bEB7W*1_;jb( zT)134@!h zr|rML3K&yN6cZhInzZ~fWq6erXWQ()5E?iro_TiK(l95wysKF8JTGecoLlsNZIb)l zs)uhHF?R)!u!mZCVNjm9B)f90JzmAJ(euf1zm(e;#lCH1gzM_9$s|ZUXl^>MI6rT) zHCg1uX|^uc;Fe0J;i2nAYImCH|yQ`&NL0}`(;wChax3v+p05YuB2gU?FB2Mkz2CdU((|O)rRE<^JH(Rst1KP_qE%snhk?4R9+{b(-1^CGfClTW zn>o5wEE`Qfh%h&{D%O3A9%>H~QfqJxKS1!-x~pGZ`D5ve^p52&)-nf98)c|eoA zkSj=!HObEnQJj8qf3}F98t75>Gw7@%%$6q(ovodRFi z;>~pqPIMT#R+s#mP^Q`{tGoFQc{qLW|Gd@j3_mypE@QH{nwpHg;?9Q^EA&}Hwlm1i zj1rV#aO+#Ez76Q_hsk7Z=T<=YzX2A8y&I%-bcfhiWLRu8{*#P))ci0CkI9co(jh)x z zo2_c)g%TNejIdoruTIEir*^w+F|m7A2X!PoP`O&P(_-2(YYS>1O_f0yjilkZCHw16 z3jb^mhBY};WZ#ZEK+I`)FzFPL_?|dL&9{I|dN3sMn)I=-3{ms{KH9y_2PcSLy?@wl-gf;qT8FCz{gAx1 zqf2K6G`+P4A%+MqJXlJ*8cmSh9V9vAqKFi$VV#lVyMyYKxVJ{9R_NAq2#z-qdHdPG z{i&#tAl@39p%WNHkTy`yW$Rk8RSo1xV){9+6vFqe$i{S)LSuY%n`)9=u+62YvSAYw zd~@j0v?Ek*#Bb%xHxt^(mTI?b4zP=~%!V@tvpS8@rwTMWa`>YMtHd?4*V!A~1)4?J zYywLg`B72`(&KMK5!>=3?KTbhycL+EBim;$U%5yzVuSxV& zw}V|+y>w{EgQET-QP^U~?m9Zxa*drwqQeQ#$kAHA85A`PHWR{!l}L?Qz7Asup_6 z0x!fMUNo`Ov%<_JWqO1D6l210#&gU}%p$ihV_2aJ70vu{I{I}oGd`1M{9mw5+9wl$ z=tQ1S9*bkpL;il#HgzVA?qf2EZtn zKB$7xJ~XdH=^6Hvy!8!}9ajM6YT;FU2A`XjMai3jdhBHj9VJ=q0S{gCY_c7_xD zk}MlzrH$zvmn%TQMy3bSdU%-!JtSlk(^yB7AzT~8k z-3^UvkjfnK#>Rbk1|xh`ag%H`qx1`aWOl>Y{NdND6*+7Ug3c-eI={jod<9OeFH*tx z<=fpRlmNA2e=aMPQ^F1%Ximm*LQY`&`Z6=Y;S=w{vJP~Ojb?mDYY8Cw`2n#-l}wxt1n$qmt@!RpHjhYMC;xg(jz-!(<1 z46XqAD8eC)o2~HlHhU(-P;hZ#`9q!D$NABlx;hEnZ}aU*j3e#OWEIXo8&9h$puqQx zm4*V|o6e1y?lUA%q`I8S(h(5D7#8-|2*iRJZ|BZ@pvn+AG;!G%)4&c&k~o^dVHgEN zIS=*h8MZ}2V6VIGA%@TsaLYowN|J+&lgZIt(&&*`i$L9Zo=WC&6vO@~v9^|~_v2&? zypHTL-)RyajxOGipmi=G&6;&bSJ&qwy@bP#3M1JUo=~qDu_R0PvBBbN6{7Gk8frg( z?q1E%E-{B4^h#Za;Fj#!4G&lXGseg?CWzFs!cmkLyJ#ypym-)fVi%R+iRNACxgy zaqGa>mu5RvcGXrJr{i%!>{9X?#B?jN4Vc+!VrPnj1U({u1a|_VKP+$imE#rHvZAT- zw2?L?Cu1O4kz-F6YdB;vCB|r&32Ef$F%lLyXbD4qSi0C+#68~i?XdVHtirl&kez7G z)0=*-qwIlGP#bvOq6Ye*e6Yr}M&z^QgZ+7a9o7Jf2nNROC8Enjf?uQ?hP&qXd3nk# z>3oddfx;5{{tD3vwgaXwj~X3M$CcTBpQ(=oS1ac#z_zaUa_Tf|hZyLr#}bcJqRf_^ zB7ih8qmhhXQ`bc#V@3jsB4vWPH+=CFGwi3<^rZ3k{cN5sJ`bP|0^UavXS!d;?Nsmu^;e*6 zv*%VAKgw4ylM^xqNTQyx7%ec;#Esy=7Bb8j?Mls9RbfE9a?IvR(WY z?ZX6HCMTT*TKNVN4Dfu%un`ObY=KCIV+`t^B<*m;FG8>C?Zhvv;N zJ{%QD4-(StqWE%YQSa{1j{J*wTNbOdqTJzWVX5+jP};fI zOJG>Sayt(yr)bUi@6RK9IiLp{PB2ajlk_Ghd; zQ&7yz(T0Rylbe@>bXhd}o>Z;)SS-8wwiUl4f^klHt`)l<_(TgXFAXWhB#qF7^&yi8;o}URIN!HG}j+~qyN4Qqh93!91d%vq?G4fV4G8F9CjIS|6T8> z;kgZ6U1Gao(Jj(I-p}I*j#|8Ulz~?WHOL}3v{k;eq~)7PQhH`J@Yq~3tF}_`g@Y$;>U}A-_ScB?Du+BH9VTg(Voj8MF!o+`3CkoPK^u~_ zRxv+zU2vpXfKq>p*~0zASwu$AFjT1|NIF-sDOg`sEj$ku-Z zGkLF%?T;*8b%bO@OBh`2$9b~vd^AwoX``o3=#_XtZlHhenb;T0 z{D0;w6{&Udfd>k!-q9N|+qG}6HQ$EIa=}E>OqMCcRBK!Cl<0D0<`ycOZY*PlL&Z`# zzPB0OJ;R~J-;LNH=h_x>huz=#@#Qk9CK-jenQUtmTLD^X5!ff@dpfK{bM8Jrs>6Z$RX9g(XPs3 z;zL1Gt>1}GG`r}Iu2c%8Y_emOvru1iY}hS79XLvWjN%nHIK#_>vpnwKZ4X$$iMjBt zNWek@QOA&byt(f4<3p5OvQjZ$D?*^B10req4&pa=Ft2j9axpIV9@j^Gq}aAW8=gU- zC?Si_rURfAglildVep3g*{HBn;3BfFKv|&GJD&rtu$5~oO`|79C8y2Yf0Op^<;SI5 zaJy~-L@b38OIrMktmO!CdwMm3s@d^mz{>|}A0ivbQR*dJ!+gDa9^>GiIH}(M~Z<@sYKTmIQ10tRMq8Sdl zu;p_plV5-q2vG~D*wF+Wai(1RRMyWzXE}PNR|bU>TDiDsJSU3XyRC6PU(2d%1H)d{ zHn8FT7J5pQ5Ficj#U|?9H4cqTDH=@qyr*K^WayXE^p8k8tLrO@Y>|Dom<3}F{k+na`@)Tb-2;q^9d43XH#Q-b z*R(?Qb>~^3b-g27c^N;DQ4EIrenn&~t6Gk%OR9Dm-|pFCWwRl1vM`p~N77j{?z|WV zC>2P6wx&3-Ai8X1;Jhl%VOTONAUqQW=7f42@%LU6Pes_Zw)qXJt27#FYOJZZh&kxJ zbGad}b0FH9+@!AzTsVa16|cfZts^6~j}~UxX&Cx!VGAYgQpIq8Xb3xC$Ws?|x1fZO5HLS{O<{IOM5g`*m_h0*<%_dlTW_w$t?j zYXQlmfZek_Q!nC3E^&jwNl_IO?4QLXO1JF1Q20>QPq;Blt$F@HVBK8Q`M;laxRWHT z)_@cC*ckiKs}}^5glHN@8)b-4nc<=RtpB^~@|I@@vsN>kA|R>@tzqhxfpIK1Ma}L( zyE_nvP58F#YK{0lNodY8@W%*J&IK!^0!tynN8|2{Z)EV>?!i%eao9ccp zz%rE6fghfRwG`9thdYiRbcl+x#};Hp0Of{hPsWkF)q7>0SlmX9|0V#{GlqesQZudc zzw~8(W?#yOxwZ4Ea&Gdg3~anm>l-Q!Zv0r!)U4f(Q6>T?SCS=7Z2n|g(cJEzB* z4sDCh?UnSdGt&DBAciBedX)=!AxCtRy&T4+#JRDk4O2GT7nNInCn%=Ysj6jV;|p?7 zt04EKrp*fSdMWC2+744kUKEE4bD9wQUFdz$Hg{ZIgJN`QdMTVc?}CGzfCy(ZDx%A8 ze;bmH`NqtN9x|Pi;V_s)bBjMZwAv+95->%g$*?CA%UV`|qgi750@XPMU-+q=#wZ&l#$Ydj(_}DKsF)$}X_pbYOP~MhH=BYPCtNtUkN{t!dPN~*9 zRKbeO|0m?C%fIEw!z-KF9`Bo8*n;zMKl4VLU}I3+IKbRSzubU|t-OQM!$%qyT}Ool zzTz`s0(th;LR3DsktZcmPZ-@#Z`2^Xxn6bveNd<$!J~PgX^d8mu6ka)#cYGHDr-pE ze8Cd#Z&R=#@ZEsYSm=f~r<>fT@{T0A(k^Xz;#0VKB%)O{y0V1)X&}3u=5YBuf6R^A zj4MKzfJ_rAGmYRODg#pLNcTfWEUa=>joGM2{KW9t_|Id9Aot#2U<-eAG3dN6f_4TZ z<&t6Xps$YHcdQPE+0I3)4_Jaz>X);Hnvud7mJCIPur~d~@pILThg#HYxWvWv_kNTX z@;2@R$ip69+pvBXwp<|QBZ4z}euABoAk%tLUQ3&t9%mq{6;yRolsJ=|!cqe?M{lR? zK0?`R@w^@DZeLvuPrNKCi*;rhea+U!^PfUf#5Q4_BQ==X8G?_9q0*KF(bhzT22me| zx_=DDf=echyPWXYk*gmQHOA{@DsvwoL??7hd*&<*ALH5kjXNF5t)+NaC4iVtPL%jn zAQ#y#Nm8*)4Nv_oT4-J|6?Qg*(L#S?Yh5*U)NFE@9}*THQU(h0QGrc zgM#^+>0gcriyZh$K=p(>-cIM39bG4cg@hK1+r`8|uQQxMHMu=3!)Elyzo|-)`Tz-V>d> z8vKY3qGUb8{ULw;FWXlU28VlSpX*N27G(sl!itl+4$Njq%foIW03Yu+K)eN`iArjn zYqc;wmXfd3yr1r&advj$ZBLo%Jm?GQ*R_e!L9;^cChwJ3m0E@I38c++npsuY;YTpgzbVO{IM? zrz8sC zxO?j$Qf%C~iEpW$A(s&4bIENyoIQcrw=$`3a&i=#vB&YGer~xLH&9H7s%0;L;^5*=W&nlu&DCTZ+iSu*6f;bq^;XeY2gwBEPaZ0Wtndh2q zg;ODotAu^FH9)`0v23Kj?T#~ORc}KfYUCY}IJL;Owb#xqLF_Dqr67@9=ML?tT@!iv z6e3zN;_)cKZt?N!&%PjQ+IY9 zSY|H?W!W+U6M0)Y%5qM6jPJsNyfAgL-wdP$Ori$`RFYk6h;kdH;fsPrAw(w-0-JC*e9A2gv-}}k3>z&ZT?O8!*36voHk3Qygdn3X4fPg)!bQz zBsPtK6i)C8Q-?Fq#yuOoOyNBA+l^wRVmCwxih z3OZ{;i9L@_&o}g6T7f#3mcK94^x;tRRVujI5Vmtemba``4 zVszE<X|8;|G^a=G-5yc=?HV2@5&#NZz3-f3aA;HVOOx#474zt)nbo zO7KUs%@or^P&-7~W)fYJvwn`BOYIk6W`|Gk38%Xd!;(QD%g*`Oj+)?PHpbJIRd6+8 zP7YJ1Wra0+&RNtL68VCfO5BX2jFO_cvqqVPPkZrZI&iwwl?k8<$*kv@RzBl|1*CPRZW zEt}~9O?gx&clp+(dTH-7gohZGo+WO=xf1vYR6q;|uNo|+%dd`iyrU<~@;S2=$A>}b zrW?j+_~^I~HUkYC()n0>$D{l0P>eVsq(5$z4SzAFc~~54@=9Dz2@#u>)dI4VuId;I z_cMp^mq1UnHaa0{iT2(+46+j4y{9CyC%(?{mmM^GDo2m96!&p(W)JLEFMgbw(r;cY z!D+Q@!G86wcn*0n(K;c6(TkXHaK_ob*162@I%Gv&P;sW9-V2Y zV~wLOm8+)wPbjhX0zX?;6N|`G{v**+(jQKjx@$i%-UwVf#-C#zfH8R&$Q9k!2DX#>r50#d4R-nO0dAu$--3)`F8i`eUFYm&Tzwa9xNC;lB zzK86AK^wv=z?G*Wwno+5Ay-7Tb*jK@|NZ zF)YQ**xremSzjMznK|P`P*v|<+v>hl%+A`S?QveeawH}n%2t~>`k^cru*Cj)w(0#o-I85{K+nsA>Z(hhI`$crs(cvmD9w_8@jHCELrmB6{7Qv=peV6jhipGz(R9ciN z)owHIY9RS-omh98SJaqMUea9cv?A4^g)K7nIFxh+ZPEwCS-m;V&5GMOPK`vv)bUhU zaFEeU%-fOAiN56)JOnQGEKKKf=YQ5|%jqrgMJKv66WcL*#ko^KD=Mvswfo_8j0`aI z8LQ8_^q&VtY7izo%g;waPI_V7R+7}E6Jus>#qMo8*6Jf%cYmp$1tq5F%-3KK1HLyH%*H&)tPkbW_O7el^j#p6Ajz1nFr-9cp8?5n>jwJj~Y#j|MkmLswtkvuqXywX?I4UZpAg`^>CD^Q0Lh5 zt||kE+CwAagcE4g&)s&Z?P3VtzuRmmTd6O~y~Rsn)%QvSQrbpSvE;!fOjW{c7Xmg% zZ;%r6Q}%~U-J8YTZzBL_=u9WMX6`N(iP3vu>G@(5bGj(tLFvTKvx^rAgu;sa><>}l zD-ngTr*Qq5fL;-~GY;{-@{qZzR9T1G>}gEu0Nr2HFUbQ7Lrzx|q+5SgVVfs`2*miS zTJ)SHUY_xMB-5c(5>l>h%nex48dfirr9J7(so}MO<+ww3;@nbVE0i zlyY_@FGQC{rk3W}Zr!-KTxVP4i2?&byGXWO7^ck!MTGOv0?Mwhf09kJ=Al88aF*1g z4AGtb{~YKTkfz0L)i0~>dg6j*GY5Aj;jY$N#rDvzcYj3f@z-_=Eq6thnCMv|>oC6QBi*zbqm8rZw%_u}5A zYjgM~pTer}h`m%L_sq9v8qbt05-g>o47m!emBDx3ws|_kYHNG?>+(NGU$vq>T5!@yj-SH~?(#^Bt4SbNTSq-)fQN;(wa= z0&lzAw|6H8?Qu}D!QG`g{uM~wjhW~R5V!l;#8-QD5J4^FLvSeAnv#z|p6(gIF5Q0m z-)L(t$@49IVbWtzIRC$~oV}6LZ`e+LZVap+gfh^}j<=D4+}s~Rp3I3VDJG<*^)dtO)tJX>4hN5h&n zjxE28{mgZQJvO6fROwO@3H5Odt(vr9XO|^*pVgv9Rjo3{f!}N_JBdnw*`^Ck@gdsu z$~IROx}LmSpe4WIj8wamq-~98JZ3AS7KjUtO~fNxbUzEf^P0!OJf}Iv5G(IrkDpyj zWjwCd^m0@oYEZyUT0TW%{2Iq2k?GzwyNF)W*KQH67$TI!@?=5xfDTjcH+lsrw9-WW zB;ui!{Z|EsKUNr@AeoNI(A%pP21B(!|OI5vW9cA4{J4Y?JpaQ|3Zmym%G# zFC@P+G0G4cHvk8|$47a5>z#)?dKAk2SB)_24rD}6cT06bp8hXX?(U}t6$zc!0u&7`0jj51Hh@&aPN~XG>nm4p3yA(>F7(T3PYD+Jq#q+9osz|b* zt;K!D@!;3K+wJMqSFx4bywox!N7C$5U5_W`@pr2$)YYd?uJzn%1~H2p1(G53#a*5OOYm#1{hSt^6X6|~-Z*wP zd#YpZ=KC{a``s0;j{G5XBOdcC(dai3K=SkT6A|&k6)vN(L+v|0S<}Owd_SY3SY^v; zFOkGRHCiqkknzrD2k}8W<{+%Fdr3Yje9yI(tMfk1Di?o&&5DcERuX!&Z~PJTwjoja z>yr?WdfzFzed*ZO@;420tRJ6&BDn+MHBzx=aCT#B(fePNyfA4cO3JRQ#N>$gLn zYQW>T8Q_IZ?}=S70c|AATW}V6qSDFJwcj=%p0wfq3WSKE#p%m!j9;EZ{~zYBw$Ff8>#hT zxg8z!mJYl_S)|a@3qkFgEu%^t=|-%ae-BKe^~a^(@wZLk`~C9f=sV89hEUo0QJ#g} z6;eN2*X`E=&RA~qIm}klXw&%;`Vl>gV|*ltPhdYLHPeHy?{@@wus}3bvn|cy@QH3c z?pTKwB%{5)?vm(bB}$~lkNw-2G(H|f(YV&sR4+P0tuYdc4ma-9jNk>+p$4HQ>Rqc| zQF`C-+>WnDkf(xoQS?hx5e*pYdQOJ8RnnHxo+~SCa|$QGR{w<)X&8XDZ*Ecn)pzp5 z$74ytD3rKA_{4T*g5P)()W+^?<7QROacDD-#eGT-;{)}oY6#8$X6Vz_mw&^J@d3egH_ zZ7;u0eM8A(6$7d~LlMrRccJ1rspI=>y0*p0*+>RKUzOwtJUx(`8v0P`QS^!xRZfH& zG+J^)=m-vsj;JcR!w5I7ojLw}Ma?feU560U$>vM*L0>V4n6OZ(sQ)Z55|`+)P5%K% zrr>w~!+tu16dFeu=7x03L&}*x8`Zg@|KOV+73kOBvvKcle!>dH&cxB!YOVI3=uOV@ z^5+x_hn-h{*ke+FEEgtSBV~ep;yN#{Uu19N;nlSM-m`k+wWcKt(CFL~q4NFY@{Ssq zCmeY|OKO{NOw!9sWxOwIwD%lQZ=-;UC-9lGwbko6YGT|vdQHl zfeEh%87PFu!oi&_3=1LsNRc7}-mi7mD?cVuoU(k_P-Oc-R;^H@ri|BeUzi-Gr*>`vwq#njnH*KVnU=-qPvY z6wBdqnzrPZN9#j6+I%V$q!e8#S6;Cbzuzk{^e`#T#VTkp4B2Eab%U#+5a_V2W}c8O zVj0j##4w|!BQF`@BmsN0U&%iLPIaY$ID);PV)|kLckXT}R#{bXRmz1!pye>a>h79p zR$#BB8k*r=)wB>(UY4N}>kHeqxRgSJ=8YQU_H7Ly<|h_&{7F|8_ty}QlTJIeHL_z< z#SHhW@TR;>V3D;|#=RB1&iRWjHVRY(ff|?+U82V6NWVTA=CeLc__IcDO5 zsr&Zc{rZuk0>8Vyj0h1Pe#a@{k*Po0*2%260T1`g&+YD7VKYobu{gbyRn@uwhdB;A zFPP8KR-Tpf=#_)+hum-9KvqJ5ENS{jiwJ>Q!qR>xNrK#1+Mjp>V-klO9CEcUhE9>b z94sc{ISZ;z>CEChQ#!j;TN~{sym>7kWG3H9FMpe<#wA_`sV>*m@ zQXo!Y7VT^Rl8Onbj<{=l9?iom$PPa~PfuQeMq@iDs7XgNnE<>g`hdAFTx^ z8N%40)vU?EW$d@mhiOVX9?hNW+2!e<5Jc*jpV=QuOnN88FG zX1@QR{Nm7}3~pqDU3EzLDsXK9z2iy#Bo>RQHxq%MWuYvjZk<6q%^2>qQ*=WQo zjTMLzYw)PTqeVm>rE5&LCt4=kq*6D5J!H&J=eO!q824{oiX%lSNDXSmX6h$nk)*c( zH^Qjlwt~yK%O~f0XELLZ@mGk;LI9zmp&|h3zSG%$G;NFk>jIUiqyzr2)DD&zX0+FL zMSs~cc`#IfV`3)-?TV`+WtUPmO@8;J>uWO#AX(2fV1$GeNKdl4Ek;<)qcfR6w4Oz? zEgymL8|rHD`P&0QkRSSzqO6`$7tjN2u=vFJZq_De)GH5Mh{PH$ucsO|({S5BzP!P5 zLmuS0Ko=EZh3WrB9!NnYLEOi}FXbE)%-9BjMCz}`LyVvjTS=zZV1N;Ly{3!$4vHy@ z%)>SR)fJsw-nl72s&N-b)SNQWNPk>n*Afqn3sFBY94P?y9@ORMALz%?9!Wmrqr}IB z>Vs5#z=;Zy5%eo+)Yg)&lkhANR(Sn3;7))3fMN*l*KSWf3JC4NXP)`gNHtk?UsUj8 z;#Ox*mQqe6)eNqO)3Uq6@Mp%rU?lt2#AF%I9BHe%#|j+#U*;uarRb50c|-R9LKj^d5a5+viP|x=zNNfmiRe|$Elei&*XcsNo^cSEFZgvnEE6Q>_7+iA1G# zU}q053W85xV5l^k!Da-_)cd)Di$_(UGGw0ijA;J3CkiV0~ z*)@x=zvVGzlwEg{3P~m#ES_^sVocg`hYEe+2|T;Dpc|rBhir4vyMnx$F%@@@T;W%t zUq*O)RnwSgcXwV&@?2rU|IY{b9FbQ*_$;gv2h%qM<-Es_({gHENm+P#Iu6p|RYAC# zTC*h_9aUrhT`tV|alaJZMVuBMw}8Ta7>ypEe#;mnKVy|BE!=~KjFDTYr^;qP>f2=j zwDj^6r>$jS+Kav?UFISuC#OqCqhZS(HN@C9Hbln2xy*7@5c@}_2>O^zZs3_|CC(uD z%%Fsf_A{5ZCE3=5Yo$}Crrmd*p37VmN4e?_q)5nwTPmU-n38`{?|SQu;iu>htDa*x zeKa@gB_>J-n;C^Q-b%I?a5HX*)X7+?>-gzH)oNa|Tr9kSa4*4J*;`<5H@=z6(ZPPC zvF@zHeF4ooZc;)PTD#X#Ts^&{PdVlUg?g4XuDM`jB5jpx>Vqao#CsNV!)$jd_Z!;C z`qhL>UxzbT+=|nG=;$^XplPP*#J3v=UYs{!Vu=q$>;5)2Vzaje!_NQR8cgG9IzARi z3SLiKbcB2OXicx}C`(pjy0>M3P-c_ysbEP;2yD7tzC^GBYqLn~RWfvbwt}7Y@v_#W z^@&7%3;k8}T#xfT@2D?7ig)+M9FAH||4g!*Uf-%HmmFwmq1}eHAclquB@l;n;b7>% zyc;=a>m=H~S>Z=%KKOQ`*x(rz(Y3LXSK2eLaDT0v)kCu|f5Wwl8G*6Y^!a6ZXFms{ z8>LFd-$N%h*r5yFi*~o)7uAXdK^~!owa3-IG2y5KyAx95AW^g%>*h<)b(ivEcug** zU2-A>mPK~uD;^h(+%pN3^y@_?_99~{9$In+qZ zAJE8i6D6zjG!*3d@GZCj&RH*!rQ@*h!7YDY2l%!*KIiM(~w(L#ehv8}gR zM~~|y!)eeXJ;>gti7M0bulgrvZWR4U5p8H)GCDXxiB)cO;Uf`Dt>jzehu>@}ySU_$!ovApbmIu7KvW!EZai#}W=#5mts-lYX+M6B4~;}AlXhO+ zj89Ie)-;5!KzWcHmggafj|lbb00zw0srq1DAv2xIDirtT+Qnkr>DP6@m&1Ho*WofN zOfY|l9@pi%+os}#&>{NQJgpBhaD!@O07(hQm>rv?*qDaPfVUgBEd7H`kE)jd`T6vRl$Gt1) z79+gb4uFWbcUn2Q;I9;yg2J00uzm{Za%i+8brA%$KKSp)K*<{Z4*yfEb5HLLa|8(Y z$B{<8BsS3jD#f46@4eoq*%s0UsYTy*{D$fVfXBrDbEbL zJZGWJT{dGd!f5npOkN1{J-2FNpA|4U6v+=KU|t0Z5U1ZX49WC{47-`|5cCskQN4{9 z5qVf)*W4Tc`9 zE(!_`#y2c*_@_ljT-*=+^CdQu&|fX-FGna_-Ndy*&QT0U<0TEltB9iz6^|(}`0e4i znrF83%(WnQS(o1gS(1aKSDD>eUj6DBZPcQQUYZi)Is+a_5kar&6Z#8ytvIQwBN0PZ zzmy!aYJTda@5PZ-y&)3CZmLPB{_Zba;&ufg4&e<+w?3*qGI;%F-lm@tzH+QI?4Ml( z1#a4qP)vFPlfk|0XN{^~R_90;bjo?61AC)AO9idk1OmsI`gRE7vDeR?!L9(=IE4z? zC>sl%H0Y#l2ZE&OJPvB>ps5vgA+UGU*Qw~(CDfo4>E1Q+0qVv-)?)r(s44veA`yRh zsG&VWB0l|>3sV*faCizj72};yVZ&$msX$>xWB?j`@eJRvu&9B@XDLuWjt|!dVtlXb zYRr!Jucmg9ARm!1K1<&Y-u4sWn?2q)%#-ut`*YK=rr@zU<%GJl{0^>Fjx32w!BlXm zUdrsVMjVr%CISn#kY|+e8L<(``JzR9#M-{biI<>m1@*}}R19ry>!(>?QNi>{T4K+5 zgvjnZe!Oc=zfhJE>|?i>Zf|10o2BxMe!s7HM{yOx*2Vv+&lNDUGv-==&VKV|ke1Kw zl+?aeoT$3war_eiLFvkc_6Taq0=gkT>9-UFezJV_$j+yJXvtRgZ1p(-#LyKo&_iQg zyIm}7A6|8Q+and@ilfsp{|}F%R``t(Y*f}k9DlDbto=c7oJsKS8IQ~VS}x&%eXwSZG0*Ex_QsN$0;EJ1zOkGA+H-L zu8Q!YPJ)G1;Q59Lq3%FJRHZf>&htE^k;pqSXooqy94fz|x+*%ahgZrn`72I?{@oiz%`f2tY#LV>`2>>O<%FN-bg& zR8>&6oVuU;m83Cvr8e7#`j;V?b6y*R z|8NkVW=*m~o|Xq9FcVZ<7)m+a_p_T@vOd}$R*>bbtxKXuoFf`{ z~4U9(9+x`l%V;w?A9--P*XEL9?S}S zmC50W%O=*C@a6pctTy+zInFiUSCeJYo7=ILcp9?n>y4^P9C}WB_u3ta71+-(7cPq2 zLSJ*-lQO-R`TkK+<28|{GNQaoN|`So>~{G0thS;Z=pV6F9!+DocoFJ=;kMsEy&V>@ z0_Qxb*r{GpR9T>Ugu~@wW7CTJDK?uX%)&jA=I`#8F+7Ppw&T5@HJMkEYH zPP($3GAu+G!1as2YP#l97;r<`mCqMR5yqJ*q69 z+M!J7)wRGJOeHWZNi52+KG+sWG^=BImVVWBYvr=icyTO!!<=W?vBFx{8CCwY3(?W} zK1Hx=(iQYp8YNAKZ^w7&CYg*2&RE#oO`J z5~M;#A#sGD<4n?Jh+j;g!7aa*=rC6C)CDxUmDRFWli|z3T$lyL+5$n%C(8XT&%RL1 z2%@yi{9`%-r&=FY+A2MmL&!RRy>J_;CE3O1EK``!5(1*8Hz@7PJkIwEwyv?(1exHM zqdJBKg>8FE6c|vj^01y1q`1W`43DCk)UMmxzDe2ukZjiS?`P38 z=v(cHs4M9Ls~PbFW>s6ah?~edujWl3|E%!FDVr=^WT#_C)ga%)z@_V*_IYR@zA}xq z>^<1eBII9Z`sa36C(IAK6Ti<`_7%e8)tqEMp^?U9)cDNXUH6yY+L;W>X*0d)7xn6u zw!6=eY>N9s@lSE`#{-wuwTVg@cXiR2icmnAZk%UlKYWCqr>t~@?g&FDM{>jeB07P# zVQ5J#=aF8!o*<_*uRhmfAu7v8q)UbxY$lEZvvI|@^qeD#W4gOA2S2?@8xH+-kZIY^96bth zF4uv6@XU%eQibxq2>L`j(jgb=Jv7$gi!rK#H>j1XtJy4^ zgdeDdLPfg?uxh`8?+)|@B}ZgYBFF^j%T)ib{Z|8_;Dp6`T!Z`b(RgJUcZWR6ZlRq_ z%#iD*_rsG}T?K=7oPa(;7sw6O=*Wb5{F+VTecq2dEk3sHH?z)+AS zj_Na$Vx_8-By=htNHLh=Pf8rNqqh3_waBDlc__5aW(`MQ)`oce-NJc+qzlGhGupqr zfb(rEY5fl0j@CMr-=fcFo$E{zfoC`M7cj*Qi*GGDzMU%PTIuc}AX_IS3}00IO( za@*S`H*#Bg{?(s5TMibzM+37|>k9gV5U61gxJ+7>lll1k4U|R{lhZ=*-e8ok(O>D- z1cABaA5zv{wKVU&H_y12E#C+)3@4>=`;k^B|S(e!m-a{(KU zo}LC}mtaM2b{QeQ;0F@D=5usG0cX_8El=iHM}hhexSzL&tp0~%N|e502E&6|U2&(k6nYa6n< zNU95RnH(I=Gn$5GBepD~L4*d2f_5zyHFB8+Oi0?Jen55Jo{_&dR(nQLX7xc3=l53? ziom88~8y29Is8gg^yZ5mv z#i{TMD{t-Zl41A4_B!&>79S;Ka8*zvG6(wHngu!kv5e8w(4K+-Xy35ZXzSTv%*HA- zDa-Rw7{^G&w_gH3*AcUX@G25wUw64`)5eE&qmR))P*xZ#Lm3M;er$*rM43FUszy%I zIcYfaZaOx8s3eFG=NQ;3{s)WDHAHNr+khxRs@M`L)>6Zx%F>N^&->IT<1WSAobv{Q&B>1WIO3E zTQ_Z(H#3ZU?&^1`KazdWiI7EATJYUICSKd`!ZE0yDkRzUsCu*DuopILcq*{luGS_7 z1PKxO5cmUb>~QPw_^9+9|8ponr48PfckBeaErA-sJQ#94U(0B_k&jdVU%(4p@kJ9| zT$lSC8Hf-6B%1xyqlXy+S%bQQbetW+z_F!s7Yu!G8RngfS`*_PtvLx(H+Sz;I5=rs zKcx#ITT~c!i~WW!Lj{|MM89@BP5tUauY-h=jrhPla((NDtzMo*TU*|!t&IBWkm0!+ zl~By*UZ#HXv$n28u=&?Cn#CE}lx4*5B&8u8@MRWeyc;_qx9Kq7xA z8SPAyfZ|MHX0xtJuJ?%V(Gs^&67bsI7J0BFu8`8_1OI!P){b8(%HlxuTmh(JTa2C{ zFrTv42SZx2i{+RFYMNK16;`j9d(H(kNS$@=?Hyw=!?^WRZ=jj1vT3@Kv{{HW$1X!Zqd5sI7t?-rAN}?|BZl~E>vS@T~Ok}_b?PS9kkx@V$27G zyNAh^HQCafV!l(!2U5LwdekT61A5I9RG_QCT2FuM@pXykg>e}17Ee>Ysh_1HK{)on zE+e8I76RK3h`Y8;B$J! z6Y#Ds|Nl^}dLk=|LO&P;CUUrD6JXyZs>-?=_cRM80XuXeYM@|CVKAB7VAO4XR#B0E6eA;3Lp zPnMxRxQ9r!lEDtf_t_-TpJw)U4fJG8I%#vDz2sdd34csQ!K413ng|P`XtlW>__zz{ zX&)A5QK^hEd?(o(9P(Iq!Fzmx8$-!L*`2Iu($r*Hg2?Gkh|Ytf@&lWYd$&vYnL(m$ ziW~BRfpHu$r9U&;APzN8FGnj)c!m`sFp(hE9Z>UpvN@->-u*h%CfoU)1{N$p1 zxr=bLfnis3>$PMpNWBaP?52M!4OHL18+@DNp%%GzS9XbwZ$f+Gk3uxqxCGT2gj{%L zL0|{O#!*88Lb46FO(%}Lg|h$r6dQ zu2SyB70DrX?J=Hw=>@e8bW5vxxTdJ1o5I#wDz53o^l0PuKczBuyw9|J5Q53G>H0Ce zi)NJ-Tw#GSwwner5`L4D%jX`m1B)4IX~|EoHQ%e^c0vB7l3nR~OT=xzxu{_i+iq*4 zzM78-b4mfXd9OJS9q2vOGQ5V^tJJ|F(u8b<+?Rw^SvACw!}TsSy2z^&+g+pQcUg14 z_1iWgwPvtN?bfI{SNkpT+o*;_D)mr!KBnWqmDN^`D)RPePn636#uIZ*tsKW*J0sj)t()d zX&eWQ>iCG{Tc(PbG&IoCG1W68b#Y3@@vl6JWPQ8rgEn*8nonBo9R;VIb#LJv7v?p+xfL27MmAroK|tByRBEMP#vAOCLs__Mm8e8ZUifpz0_r5 z*`7Yql!dFUVP5X~FJa{CO_oLr%poNn)_;1F`N=N))~jqUq)2uNNl&f8?m`ex*c$b^ zewWqtp1T7?FM4@Ms$YLRY>6ul4`=4ldd>Kl730Sjqy@1lxQ@@wD2iSMW$kc|@x&#D z5Eu#W>uoS*=ECrViP-S3ppU&Qwe9VWTyLy*k~eV=;819Ds#CucpEr3jGjuESWX}?{ zCRwP&x9`^X6rEd0)CJf`V$*w)#o32DOU#dPDH*NH2B3sQUE0K(sV3DjH99 zk7zUKdfD^bpLKeXNQI>Wh||elGzod0oF?auviOb2;u38)7^&3B56||Fcw`Lj{O>9( z7^9<%hT@4mTyD`--D1}j?|~0~9hkrVKo1H#v@{~F+aYGMcye22`dy%}zA6!Up*-C6 zi(I3lC*BF4-{aD~f7LDHIQyNJ_IX;#n{RSQKoqAzFEvBSZq>&pP9ifYjjsR$!Z>OR zr>F((ejtV0Qera4u*7O^iZR7#%_^jy!_vn3K5b4o_4%*|!G8Qy$^6~lQuyP%EbI3% z{J6Rxr#(V5zPbE`!PVV8l^$ku+IznmD)i-S8+gUtZ}@VUFml2@vFWqmjcpB$bX7eX zZWOIyPUui(I^V=tC8^7dSr#LcQpoW_JwoK}%*mUKCo+deu2b zH96~P3ASC)cdMVFuMx=VjT9)SwQ6Wx8;QfZBfF3iiD_$i%)qp$*Eqj?yaZ44#Nf{W}w8<@pDH{K}YXH0DkU4B0 z@OY@s^bT>H0fY=BQxwS0B>B<}Q)!a2iQ#HjVWD%R=2-nMe1htIJ)}EUjvh)T7vHhF z9);F&G~qWd_WZ%vG?gm#N1fnC(mq{vm>mRf^aOP^e~mH9-8lE+Tv5aC7A%3bQXYqx zHI5jUe?@NlY2n-@Lr{&1vGCPi;(PG+Js;@zz5cFiN!XCjM*tNR| zBEpnctf}{|>sd@V*6_&DB6f!COu+ww)glMDkbWH>Ajey$jZwgtJ5TJQG-X-FJvUYJ z=l*64RqYVHE5tWwl~0!TMgpZzCLAUWrVW)-CG$*&RyXusSJFBQ^gu3A@4_T6mLP(U zQg()@OB5^SI2F)JbY8*bise$l5>={1LnEPC%medrk2FJVa@x zBO-z(E%i@>+JqtM=9YhZM0XN~xb%4o96cnz|97v1+TSLubCjsy^mUJ!+|hT(jX^j; z8WUCRgcQbuVA4B{r56*<@yWo6`>|Qj_sO~09Nzb*Mm6DaL@fan$Jhrq z%bHt*0s@a>t^RdCCE)7j!@Zf<4J5mek}``a*k0o!Azs-xZRN5OVFi`);(iqns_}7z zFkG$cc_|;PBx-;7kZzN122JuaUm7yvP81Y{!3P+IUe-b!*$UTshwxt*Qlapnm>eg) z0H&rt_A}KUiT~OpF1R9dAs(&qPTOO#$dSz!?u=baZFLPckYLn}5U*!Up^Oy8fa2A0s^Ry~;wq)gp?=D<^>tD^*;fHIs8gTWUuFWeq9+ zC6a2#1XH5^?&44Bp*+=M$b^6)#&+U}N3aDJna z}D*MfOHdwy-cF<}=d}rKd*AU?!5rA8SY=;suQA2tR`k?fAB5h~CSx zCgMvn5{_bF>Tcm58_ZFKN=tec2KlQdN0f-5G4tqx1r6N>J+yJsT@u!^2TK!OAb{Ff-66L6p9vlSMkon-e5^u`}4 z4WFJ0Zh@p;?yeoAcpb43LJMt6N%@82)W3^Via=eaKxno4zUV#4$Jx(!Tws+)64~k- ziMN6Aj;`o? zgge>2HsYMWU1l*o_t|vpH@L(!b;IyW$L3jxh?5pJ`S_Y&C?nEM4`EezN6w>MeCfm9 z+6Ebb!4V|#wQ_cSMQ>u=x3f@9l~Rvpi2on0Y8d9ECgESDNvW&u;doW-M9uZ`%m8kF zIKoR8P|>|x|F5H!qiAJzT+$7-It8WIBV-#ZMeGG|;H0}i6^dplKT?oJ z37fK6FwY-cNbPe?vm6|eXY+##2EpV-ae|_KF?E~a-=Xx8mu@1t9udV-KkcFPfb4&z zn^f!%*7P&}x!$c6N{CvT-gm(132!i$#?RRYNZ8HAST4vAZ!*k1sI#2u{NzAgvu0nO zQmshljLy&R=w9g{%(Ch&?*3b@HQ*RRh2W@N#7p<{#?n%NylkA6aQ2V}vQYAS^zt-- z(-XD%2dnByEFoi)*!YM>YR*(?&stKak~2wzFU=XvddoEQ)=#Y%u0x-8*bu1)7EC;H z+4T`CI4ML=LOBvb<`2Is`oX%bgWiZ-ZQf7HR)1D@+%h`R)_%k!k^dx}Ku^B!t8iTC zHCXL0v{@avdE-UqCI;9C-ywZ&Q_#RKju=p#5OJnIqg9t{_)u@xk@EHbwjdCYgk~Q8 zN@g-65^MwQjFQsCL<4BKZXfz&NZ#6xk-6#fnjyyNgN;-KJT92SV8`{l`QqpE+x5u2 z=D`-0#$=^XyGH66zqyB~l>QG;1@Kq`ME?saVCSer&#Z^d5(qE4YKoG9_7}^$GFZj> zom*e{Yfh`uH%aRs_L}GpJcL>Nd52~~nt^&kSOCN%Jt#q3mdG+cEq~EzyU-Gia#kcDdNd-dI(QeBHjkd^Hw|3D^W#OXoIDp$&+2G-1W1X-Woc;*2(2i6$h}4p`kpZy#D$ypCj;CAFe_M;L^M7o8i|(Ocv>GH&UK8fkEEX}=)E9r+Ss zHn~J0cG%w_sUPfY^O8^mNLP=>b0J_@hB?Ah{ct0g_*Lu0EmheuI{a2QXLx7b8P_ys z(|Q`E7$Zx9OZ_coPl3P{e$$~|eF#d1pNB2iVwA3Y3%MoiVXw!xGMyWt9uI&sWzz@n+kVC!-qZ>nUD?`BYQ7>n3kv9;*Jgh7^g|js6F_Pm3 zf~WlE6gO@7gU+=Tg4lN{+6ru|L}f9N&>i3NKzz!XX_w%I@Kg1jEmK^t&SYC?qXe2| zNFw(;A0%hlmm*53l*I2m|@xxgkiu`=RL4K1+Z1=&GF6rV!bCB3Ou0g)H z8>NMRfzbt{y=Z2r9n=DNA5BIZ9tQFyEY;wjW4BXvCnD24&E=fsh|PxL!VwW_M-n;p z&AAiu_X=FH?m5+6HZ|eIdfY*kjZ41Ud`w;dO83^h49;_`5{^|+o1r=O#f<4p&sU=k zH=>Z@=`QA**k%*Yl-_&l3mvgOLV1_8CgWp1OIW^H_lq_SKCKWDkU6z?DPUQ(hL9!A zvyK0uL#(|nn91gKtZxW#uB)4|Ep9(;Cc`vp?-Dz2&?;%%;kL`VT!FYK3Z$!4aP~Ko zCVk(<3IXD=$C)X)*

eouc112nbX?+twT>b$`dPd{F-4U31sp43HIzpfKkL zlZlvHfXc>H@@-NeDu&YSeV9>Ykf2Z_L-{MUxGfK2=}r& zl8OuvjrR$7-Y=w-(j|p{yAKc=%p_KrJwn0LD9ENdtP*tGXC!&Zn(?fXDd80B{K5iH zahBQCS`#aLS4XI+XK1XWOScz%KMX=HG(-5!ve0oNbF5KpSOT;qs|xUaP+O=6VE1d# zY?HjUelnLDAR?uqUV*xdiOsN?n2YeS_L6al>Yx-I6`nrEU6W7_ERfDs38iq*PQR@w z)odlwXQ?v&tr{*giG%~X-*2Y1s z_P$&QU3RFoMjV<}pw`VhZrvLBj>GL^)5M|lR(2^%?D1KGpvnoNS0Sp2u*Puxi`z50 zsn{GaAD2W}H%dxruW}5sow0d0`%qd9$t23(2VHIKjV*3>t7utWtn0X_dBD_8TTWC|5|Wylk`W#c1aw#VmB_0Os6tOczO*iYq27 zvCdolwy4`+>k4d6O!qFJ3CNG-=Awh>xX)=5#4JMj-zV<+V_#s^#q7Czm7@7BwgDR; z2EVi~DN(wpeA5_;*rxRzkqJ91f5McKn+MnD-w*wv`ppPRrbUTg0yb`Gl<`pF+Jnzl z75V0P8U|hVJ5ogexo*qjMy8JEn+s9dk5~>DF&(Vf14VPrI#&|I+B##NTjGTd?bBlH zPPTy%Ikf)0T7MW;*ONypVbI15-FZfHmtuyBeKNklhZ3)B!xea25=>DHzh}--PL4&3 zG1a&L^*v8!XKvQ?7CW$$JJ24J1=9x-&s+2>)EupfAZlrmA>oIYqgvEzqL#8EQ@1kq z|3-}tU#o~S&8Pa^lcVw*K7`%8-Fj~1*@%yR14+99TEI9f?u}ns$03pd>{W=pb?mOd zFWTR!3lV-A0&*h<2G`is=Ze#bm5zZ1n0wJZiU_7-k#xl*ofd2 z!n)H3rQmN_+q|F2MJEe-Qk#|NiaiZ&e-%cn8*~vOumu_|(yK(ddECX(XC{)Hp@u*h zlqV=Xq*IGSN&8ydj#wfth2CImEbf1enoWdd5rj{CMKLvhfXnhobh_~FMB~zfMtBh% zR=!F66e1%Ax!{f;6c29(vBLVY#YVjxhkvy`|j@_SD?i$OecX=`#kVAn@cUIoM zSm5$;Z;4iL>Xlsf7@Iq2VWp$CT=w(1fu=jCOSQg(GABQNUS1*xt7|(Ps*Co(4#qAV zQ1H7n+eqh4FI1A?yiHzsiI+dK+w(iK?S8$%@dERHAt;?fa`lttX*98W*$<&!?WTDN zg{X7ZR544S(5E0NHFg{+bYy!g)!4*dy|xu~L}v_I$M3TJrR?EBmSqnJ*)u;2 zHfs5QRTa5S>;~0;!uxw*JBZgQ_P(g0NMt-y1^{Ixq&^8IETK*fXRL@f6u}e7x+O?IS+Wk+c?$8#QIl za^3U+E3{q8Xr9LCYtY<{x3o^yD0m?ZLS9R|2Y&F=^nC@KPCsyvPMHs@oV||-^I%*T zIW*pQSNeEK5X0(xzaG{;O&I3OgMJv14-U@iwdEzN8FofN#nUl3O+2;rpVA>+K%%6o3f@~$V-2S1doRxPlImo`N^ub4{AGJwd$Q*K>jg9dmL&Mj8 zo90UGuYMqtv$=i9#QoG`I!@ruWB-0a6vg?Q5aeP-G@4qx+=fLD;aB6s z8RxNL3GD2|{fyG2rV>*!ph0b(__}MSAj*biFQ|oHgUQ{L{KWGK=Mt)_oO_YXMTTu-X!X-rm*KC^ zd}0{MRGc-|c+YX0@UzN5)kKtt_O5V3yp=fbkxtNDQudN{l~8Y7`9^K*oDo0kw7RNL zWuLPcmI@btfv)!qy^Z1j*Tdiw134Dh!;dLkM&ckPxT=ukH>edSS*ILU-^AjcdSb{Z^K8I1$)7}HdWpdG-@yP>%gr=0K2C?Wapcc9FOuxaJ-40J^|gQ3rENG z@9RjHJlNHZ;lR~AfU=^?a+lCY0ciT%1<%d4^W6QKxE~-(KVeyCtX1~F-mz*`^UDCg zfV!>p*btav4>CMZ0)dY*)pYBy5guffk#s6 zY!tU4<4HVFm~c>gRe`%q#`9XX3&D=!pc`(kt}Jgf@c-s@xz)`~l=-wfjRg5s#%&K6 z;2$g-7uRlAt>tV~zv22S*q{aVQ^Ci~nFQ+=5j^s-SX=d#v=*i@AJ}0F%rvT#fscC7 z&Kzj2w@G-}?f6m4U!hc7#`NZ55@@Oh5W9W)*Rf7cu83tBJ`!3jj57$@*Pr2kG0A$N zVf2zK&AJ)rbjx`>RN@Yxkd83|Jj>s}5jMX+V>@WIZ?5Mig=s(kr!$W!lyoZ!Ppx#d zBMYVldjT48gCywkVL2w&{${w-EvP;0zOQ$ST%0#+D+81E1+#;_uj@!sj5ct^>J)Fr z4;p6rIjnoPXBN)mPy8)nSRd5=z^Tu0r1=1JNi^rjsY!yPa+i*LCLNb%;5i0eY(B z-O|!LhlM8Rp$b8Sg2Q%;VdZ5iC%A5WbKBMESS`?VTRr!JUysOE!AJVoZSFW9#!sdO@8ITcO{)^DN&uRdo>Ssrb#VkbR9L7n?Rfl&i8J@7P7h( zkCo#jg9xizgeM}UU~+rdA*$6v+Q;|hC(911Rq__D$CtX8Fw*&*wzVM|507F{f$M=x3x@Lh7R`-u}}#NqCC75GRhOzAB0C0cKmV zU+Po9YJ+LcH*D6`H`fFZ(pN*!U(uDeJrEN<(CPBP7?15bvch^ zVI;OydM+vDszA#wcwPfkn}v{PxGSR_}_A`n=^AYCK1o*6~E+kpDQ& ze25KJTN9JLwIBDCCu>AwOo8GPCBT%P*P5)BG~iwk5h?$die%26Jm$XZ7%%M zo9!97Ux}Cur@Qw!X%4#GJHP;a5nVJ}IgUJ97+Ssme5M^H`X!fI@F!JxJ!6Bg(PTRUz9QGw_N4K3~EDRCcmb#r(dt22T6km&s6OWUkwe~pP-UhIW z?d1Ne*g=TFE@PK{joQ)GI$BSC2Mb#V!9U92rtk zknf%biBOy(1dqfZcxY%1wb;+3aq_3SOb0z7z8?z{nq|#)CMP~jgU5Pgle1gq!0j1o zU@sE0_)Y_-F%CzJwsvxYh=l*S1M)>aW>3mg&#A=+CsHOsM4}kzb+_XY6dfvPDso$=Z6MNwf@MrOuz% zRvvbeoGmOsN=8$OJBr&*EbkDP%_(sLdy{fcPsiLa&4?PJ>j8-(xE&dK(w&l`YqMpv zZNJfXo7gfTOoYj~if_d~^67v-CH#L9K2__-Ayx+K(ui<4q}kWN#hwk-eU zBTgk#+4wA;odF-TPIr#U*qEK)Wzl6)J3#x&&}@I1;3f-J_o70rj|4o2*)u$vP9y%8 zf6+7c5xp_KHKkcdc88#NJBWuYD(3*xzw-TR=lE`CV4Ke6dFM6v z)^(Ka#71sURvWOb7D=Q+jH|;nNrdm)O4QBGgq$jd@=NGC-Y34+`d8Ub+)G;Py-FNYoO{a6N*nI>~IPqYD zW2bvR3%>d`+@JAurgcM_c`jHxH#wcl+%c;d@dJDNJcAYEpD37C=Uxv43)xkKq{kHu z#A0a-;Wzv|(+A?y2hv~lLyFW@v&-`rRSLver$UJhhLlhVVpBBdT0&5Kv%d4Hr7)s{ zcUwN>$FxdHsuPp3^J??|L(;;Xw3B8e1h1>Q%MQ0!n&OO*dXFVkJFT<#QT;br4+D!_ z$+T$%G22B#1PBtGreBu6uZA2=gx-;=nD&OqYU(r8tNSlk1cMI$K<pc|~A{4tJWxJE?6lkSH2@ ze)rs$+|&4@T^tp2R~}G0Bxv_VV4kjWv%9%XFx?f zKCjGOQTpV}^vIp}a#R=bN0cDZ676!6hw{nXbOECeVty1;CPd8|<{OejCXO$qE+X>6 zrp%2SB$^U(Q_)fniX5rX{Yz#-_wkdseq*m!& z^w+~^j!B-QXu1!hYUx?y-`Hq=;xQzno*7ib7OMWxK!v-uPDu5-x<2G>JWWzSpoefa zy;oBI%jg#=R>&n%`23`azL$?IwaT8?p~gEsYI8*1wcK2T-!_(4Axb@Rb1!+3P?Ejg z{TMyo5qSBbbzJ&E>7n~^nuvXK9&OkFoiG#Y0dz*Qx!Gx;N6==%xp8O39%&tklvyj! z)G)FW$sloiLF8BdiFjSdw4R&M?on7&{(tk?(j-ZhehB_-K57j+o;_>rVJ6W!VWS_7 zfWhP9U&vR#y|uG`H)(@c4oxTaa+y`Xwzgbdzql^?d235H$T@sKPgm%0`F%E=$hMW# z$`0o4ms7XmAqoR$;jZys^qehgES&gA+1iaumqR}80M~}0ILnGSP7;xY#GxkU7+ig~ zp!hED|F`N#=?uxYv}X3 zgP;vT!9Z`pK=EY=)LDxbUX`XhJ$2W6=2m;uM5!Dw6jza97?-X^GBHRq1r)1{9EF=b zuSTm(k4*1>hdp-ot!G@rh9^46rGmUL^-!t-!Il9+Zhb2fNcu#K zjVqj%t%vxivje(^fzZv$*MqVdV9KevEXh;f)*g_AIjp*PI-DVKk6cgisfu$a8HPvO zw4G5-0e5p(kEL|+L=&|Qf1@p)HS^;|M`5FbVCiz1dG!+RBw;?EGnfGaaVTjI8@_7 zQ69F3!;jyE*yhoow^)v)nP6zE5ZG>srr<7i(`~WF_hosW{06Pt$JFi74NcXW>F(9f z%k{fDQ{iS{1;x=A{WImNg>*B}g34))e2RXT;RB)|K(bM(vRLGtL-qgnqK^HP`(LB- z#;=-C?z&@#aDW6)(|vf9kaixQ6|4{Ruwo2~Iukt4e>^p5NLL2Xl0^>(ea-+swOy#c zElZ?~4^6U9gtTq&@~14dD>d}TZ~5rOdwe5qCw=kWA8;#8-f|dr)0bPbZXX#SlZW;q zelD)70w|fcrFZF_;#S12HnYyAJu7L15E)iaeyqn&0cI})8jBhw%3kb~ebj~lX>lZ) zoi+s33wq72Sz`J(7%*%-48(zeX~fBj3M*z{`AeoL{K4VnJ)Hk&Q!vAkSPE<*8={C@ z{DS+}j}$yf_ve@54~E!OG$DSuNIq=zOa##7IDNpqmq2(E+{dl!+zkvOwMD5LxmB9I ztI58(3TQ#f0b_1?oooe3i1Ou@BDPUsv2n!k%LQakx11kpsTqR~XlbK&mllQ%eabM> zrP!v!xZWu}Acu0dvXOceoke?#vyWnu-Hq#K z+p|8Aq-nZ1BqUWJcT-0gZD6Fcq~aEmd->^y0{}ljz`yyc5(q^13tjoTVM z+fTA*gyn)%&=!tXQoQ{tA0GKTYWiyDZt~6`kD7oMEw3Majp(Q-w|97z_`Fs;y#l|n zBh4b%enY_gyxxIIlbpyWf8Ei6>SsowG)>#Qx{bW`;jVsS6aNed$t!HVHy6F{0Uc{N zJE;aYjt~;NYU45@2KCZu_?{G8_^C`>zBX?ME~R3s;YyZ;p*)0_>kqz#$rFm^=P9Ld z&Xdwv*4pbo^@FTCQ|C|%|DSse;5(M8{a4d@f67+)nv!ZBrDVb1U&FCvttjASO`9UD z{8mz&0}@QoRm{cApuc6)(~Egv6*H0$<~3#94gDAPWa5~>p}dNRcWxbUd1V2Q zYQncSG=_Clece(w7_oXdoC(se6$-WwfSy;Ya=Da>TpM2+oi_^_fZl+KMLkdW;5S4- zyL!u@bi>4YUd+_Cw+I)#Op{f@bvJa0B{|cYsGISvq=BooJ{S zgRU;FKo*)S6uR(3;K&8aYWvc`-!6jPuU!9rDOt8;?wAuByslfuY603ZlymHCA9hw4 z8h7i2(lihVS4Bp|FZ5b`dViEA$}(<+v3=HCc{o=RbVikjIgy4bx^Ce|`zCpybrqeb z_vXE?zzsnH{K7E>`Tuh(^Hy(cv~w_4tj|7$LvgRU1`Q8dq*i)_^rtE2v@ce<#*FLA zfv%lEN?0KAr&Oxb8T3eyrbmmot1<0u9VhmY)RDT7Tfl98&fPy}x1 zJj&X*P1#NrEWMg^DMj?usAnvssT=B&#I&Q!I*f5q;Ez=+lFPY{IFN5)9r8dhTQ1K7KR|_Mqj?)$A`(#W2P8qN^%H~ znSJ@H@a*E&2(}Y3vgPF_{ib$Ly4HD6glVzMC?KUB+bbt_xStSxN0dv33)ULj*7g&HBX zY|FCkp@N`nH{_=UnTedHc5YbBHYPKhLLYEKG6Y4SCqM5X15;6|?K$CXRIF?=m)jZ-rS}x-7a9=kN9bF~NHjp1*;OmR zmRq3~7qApd!2?u9{!zzMbih_ikQLTMxjF>xez~)gi}J_ped?d^D82(v!D-M1;BsMo zxeYSu>a=UWCb$sfOLfVqctA{_h=EvcSF0I5=s~tZ;UiTJFz^SKWR_0dfF@5-~3;T zTzPxmWF%UQWuWT-UywR7T7mohcekEI>a4^3W1ZdhMKBg$CxD`cYWHGg3+f< zrTEY>)J+@m7yX)jpl4mte*;LIi8{A0)2XL9mW^wjy#Y1Rg3*S~?17Yglgt`+RMRkv z>G+^1IRqS44%Sb@n-n0LiV36FaO?HEF4e5K{BRY&>6GkZehdy>Pi0^QDExU#$H%bC z@YHQn+;Ulvh3HCe!!TPt*0Rfd8$fqoLS?gmmjW>UVZY_BZ*>Lj$gEaY@ z9ulhAv5|eW`rTbp7$=am9WynDJ4MGrHVfG=OkNY&u@b}|L+*%30rObU4@od?tGl3} z(hwrp5DnG#e@lmIk+;E=i;q&D2O-o8;DHtsZVK#KNKq{(?^KlTDO5tq!zId`7?K;cTAAzCy4T& zcrt1b|L;_&@UQ;Mq&IgyXEl3wG0n#h4Rm)~Co*fU9uvaBU~jVrS)pRZm6YTBtxUM? zVqm9jX(zk=)!y3E!Vlxz7e<2Yk(At{-(EPDDf<*>E4oOpkW3@?-6?Xifkbl`LA#_ zZ5NU?*3gke(qV1-;X*2hR4beGCuyFV!2&xw5lRKxJ^Njsq8%@^VE>Hw!=Ef` zg1BPx_P3Nx_BcBSaq65QD{qp>JQhmAxLYMFsIg}Z4i#nEX?r0pE@fcqNxzQaWxC4n z$Lz*cTpU#2IWa!<{4J6~&;~cK58-t}QQi;Sas!!a`?f+#j|uZ;ED_$huau+TFsgo} zf_`Mok=8J&1px=Ww2MbZl>LszmQ5W$@GI@ptZAt0i0X&zNcvF1h$ov4QjvEwYlrdF3E_3^8{E%^k5l+;@#LT@st;k+8#aNTE_;P>nz}`; zT3X8@ULtHgrxYu^R`!P8J`~)kMX7SWrN+WKCa8buWUGcG0LdPsX-%o#p#d#-j%;PI zH5zNK&!=vRb5K;l$AcUjBJWQZ=9#D&wr@tfN>AxvDs&x_t1QBvT3@I+JtAkjIefSJ zq?jq>2-&H`taQeTMEHC02+3HMobjk9Uxi$CJxbd~>Oyz`(N2XIRE4($?p?G%kD5LT z6kWRO9F6Hd9qul-@b4$HX~YzBrBT{br~_4G8ozZ<@RK0VV~dKLgog6w$KACQ$zOg< zbCLZHLuNZWkZJ&nI7gMVG(lv%zpd>IbR(%=zGnL`xOH5+l)V}URef@&X1Tsyb%S#c z>RROIgy~JxD?By)M)GK{SEjYBvWygGY`vHiU^!R8yeSRVDD-%CSCusq&_D^B?Z27r z(tjMC$FiG15Ji6p7~+zgb7n^pIcL7!Jg@93yCgJp_dN%pp+7wC@f|60$dhYyc*HE# zxmKF%pIbVPtKIE@&+^sgc7Aa6?U2^IG~43*D$)C;0Y(Cg6GD29uR4$Hi^sdB0)@#@ zX*vuis%P){wxkeF4kIYZOkmb;?eAT0CASxI{OK1+_|g42F1BwHTkMRCv^=uU^)51T zCY_;1mvc_M<#|`0oe+y>6oa`CeBMh!OWu_tI4qaL#%mH2);avPiVj1M5@!ldS3e+{%n}o2ZSs zOA5(zqwjP$QJ~AEUqe&eLKLI;+~%1dmY`?RwXyrHO-^DX-I|*Z%E6$dk}ugINr=kL z6gwvUn}4UN5&0{{Lg6i^xD%~+`}^TH+f2U==Zh3wz{_yfP{mv+*sO@LG$lJ@gU`8! z`2=Q{2=fpP->%#+^h#mm6*8`8?zZ@jiH|Nm}S=HUC5TAY>y6KCSo-ZOJRFfeIs2sT-9`JAQdtC@yQ|yQ) z+;*C_sBP$1D>!?4NMx-nrBOYN6HQL)z-JzM-LqcLgP(?@dDhi(waxpx9lI^}v(^Q8 zJ=qm;(}Gha2GtBg&y3$R48FE7h@=$4>DP$$Iku+Wh6e|&bH}7utv`1B1fo_l`dF$> zd+qC`>Wy8a6%Bx};k@qbVGp6!%g zexB>U7?F*hk~gl8<7)@&?A5>-jV@np*~FdR4|SA%$h;*_OK_%45z`ozhiufloGX!Z zVfit!6{<|AjtY>|rRHIVEwv9bob#|g9M(&%>2VWVT7M1htP5Z<+z7)1?!Gfbz}OoZ zr}?FX`)p#nBsE2N@7-sSdRio>JIJDqkG+4^nZQohV(#b&vpPh-GNQbkB;}B#M8Ss? ziaBlJqrNDnt!uqOY(@}z_e-p9eaCZ*{IUN4EMraXRZFDyVr4^~7Ao+&hi4S#e(FS8B#%3B|9L2! zn6mja_>eZ^JG2gN+T3p{i8YMPGb(L2ZJ@6*4Y)cr4~+^4R}W>h2iJ93**iIxvO8mhp zUm9q`*q&l1bHEe2s2GoW$Qk8cZb=IpCc&Evd24;b&hR0yh$p4^yk(h>3c~40Ti8l9 zzwRBmfUa|tv04b)`fg9?fuX#}B`1ZP#+ofY-TpaG*{mGFBf7sG<9c-x&0d*K-V7$M zZ@6vF52;|K@oAi}MmiXr>}~%a4c!A^ap!f8&i8`=h&-PKwMcNu|CL zTID2*svWL#b}OTc&T{TzyA&l8|25F6d(ZW~U}a;{?kQ5dy!Z*2t_eqi=>Zdny|!A)^T%Z%8~BU{s=v?yC4lStTYT zDtv27@gwEBJTj_R%y&u%Rn7IUlh1L$)AgdVkH`g?ez2O~EgzCN7j*f-OQ4Neo?8@{ zq*;s}T$6cxo$Em9^{1f~cc`&;6pPcz&C|QCtXUo&YC2*rLFTY`s>5QHkavF6vi1Jt zLV#IGAGRQZMsqPb`?aKzReqgb{@)C9JE*Dnc-a$`>MW00-&;LdA^uxpEJQxYXO*WZ ze}RN&`>MRK%vG%NXt*)xmF^9rLh6cz0bCI;Ef%tq2c-EFfoD^moR(V zH{N>-b+oml>GLKDvsV%~uXWTw(p@PmS=BvM{l?^G^qc4L=bOGqe8e030!JxI`<|5e z1rsUAG0V0-K-}KjP{X~pE!3(;VQ&i8X(g;f*I?#mz^y}92p{_XZ)WS^s+p(u8_;ok zs^z?6th9v;v{#fPDLm3L56#nzAtS(%*^md}#_d@dAoRB~l$jFYNkV}D-HYD}#$?aU zLgzF|Dg_>Ogm(1)0g3XJEMosN8VA-s@lZN2`NbD>ydt@9SsXJry!JQ-e!HHPC2@Qf zOCaQemY2((RF(r<6!!|M;+|`Nt4977=pcUL$*z9dk94)0Sqhx~sja(IhJ9(Zr5P8>XZa0?0ob-uR!-q(8LG7yB!J8Gtn#8%kTSCww zgWXYKGb?#`8w2Qzk3XHEal}HubhmHkvtDjxiB`e6(%C2CbxQJhobJ$^=VMT1IvWfW zi-bn$8G?STWz zIStoW%C&X!HXF$NZxDPg;B=(T&Ay_H##blElscO)o-diIm2{SHDU3Vfz7XAgsb)-# ztv8BU#gBj2$J(WAMXQt?LUdRZJ4Ho3xw^>y@SFz;&!CzqMu?+y9OmOJ7*dx<1<`1Vtqi)FQfaQ%)*>PqrpEbN|r}pcQPLSc=l+Khb z=~pts`x}+GydpLZ$pWGy9%Sljg5{D$88l0Ex@~S0!u|I5LzIos!+vY`JblH`1As%I=a+<<;_F(vkz{lRU4JWpcBIp3wRYT7E6YGLz=Owi;gOQHSY}QKxYC@d3_PDO;$M3h1xrrmtT5NFXxz4#k{vM?e_6S zp0}>aK_JD^GY#_wPZEjpKIv3xtsKdgkJ3VVRt!l`2WWVH@=u*q{!Vh0R;I*$e1cU% zgo+udD}e9>y_~`CS0y<4%fa6b$7*W&Baeqfw@R9f;gPVHu*utjNt3?Lw8(Q@NRSm5 zR^6ZQBRhSYmo)LO8h{Yg`BRPhm}L5op(0!AC|&U-WCuTCB(yOYCw>1tAp6>1P>D5w zw~}uC7LYAsQVrcVI|!hgXW4QVNdEi+L9n@@jT-6lE6<#+x5CPbDQ37@vJ!_w4i zM~2bQ$Jl%-3eC)1S6DNQ;cVx2_cOEbsXuU=>BvxH@ouAyGQT)%W__Ghq`+zqZG zSB`M1;U5fftf!Q%d{WHMQOHq3!GSghiM~Twv+(cpEgP3B%sHshKjps#n{6otq&; zq%Y+$wMOal@Al**iBz(OW8Z{^;QEx`v8NWQar$gvp=7HuX1 zQ8xJ@AhG5E9GBhW^|(bkg!&@tSeZEhCNfrcI{1^n0``R1fa!NlCLJ zgfZ`_>QdE15HK;DYCcxCnUu3g_oNUo0UGiS+o6@_c`$*dw(sgp?jHZ-x`vkswERSOXejF|5 zP0~I0Ep2W(gjRbX%bT2NE7if{5`&|Rib_*amk1Pe)6uOnwDOyD#6nN(orxsJ(3iVQ zMYv%b7v+q7cCuFb0UWPf@&0`S)qGJYUrRibyC?$+RY|m_aM%MWtLtmF8; z1=y@(BGtJj!6?ETwW!%XHl~`GMfZBtdGew3P?4BWx6nWJUX~N<2si0*>CK4r&JzRh zXygG~aiD6+LO5o>Nu!*KshqpAg9)^_ zY(}#M8y35)SgqZB3O+E%{@5)H)N}H&=NmnYOTzVdkZ$GjeI7D-68$!^y9X`2t%wxA zI!x`Z7_=JR3E`$lK1tv{@y~lOkHQ)7;rfWWt<_I>6YdmG$788s=b16VkdP zGdlYhkJFi@N^$nbwA}s{o%`?i8`n%{dL#V7j1Sc`OX}7W{f4O&om)V;iokLm?X3H3 zPe$q|BrWfDtG0CWR!j((l8K(21-WQcfG)GrPg*6PSi0|#Z)C!wn75V6abt!Lq;Q%m z6jFg}d~H_lPo4nt?^a!LIwvi^UgsGV*|R!}`NSii!W5N^q!{<8fr3s&B-o`r2l7M|teZLuZPoapn;PrBxK`+1Z2mSM3DP0Yz-e|x2>AIX~I;&Z~JQ`t*$JVSEy z5r<44@^Zh5zKUr}ku>Z|E9^!Xio@jo5`q~2oEkoV1$V%`nJ7AwiAOSlu2>=7#i2qe+e zvR#HKaIf@Vy!YFaGQ~0&L`u&$uZ4Gz zRP&;1hqT#0++JG}LK~W4+MR_zU>&aNCUZ>@ql8hy)GkwvhA0F9GtCPM7M@1v1^dU$ zRbp}ddtkk*AY>OCUSbACA;b}1Vwz4Z!VQKOTe}P>T=ok^mXeB+R31M-nU+{2!eo?a z!0kHoY~b;^G=Xp7GUqXCST1^MU%(D=n<65Mr|5j_LofQ!KHBRQ8#8>kel}HchuE=l z4DZ`@Z1*Y6RY4ze`(+x&XXFu9NaF}J&ZBV<`M=(tlSp^SubRu?CQiTL+#mY;ap)!* zQ)f`+=r@zx^UwZC?k#a|P6=CQLM7y~a;Y29Iv2t7DF>5lW52XgOn;{S14($*KlatW z=4TGvg?kUaTv^*v{2trB0uo>FKE!bo_ zhh*O`MIyw z3TTT+EF8+DYf-lveFn2$mvq-NR|e7jvf| zCe7gI4g5g+Km73&88*YlIp6{wr#Tlt>~6342YPZ4pV~bjmG9L|K~%#;`oMTieu<$g z$uOr>EE(xz$@r-^ga@KQVwdm)wQe94e=53*O4UZQQ6)dVKH13ok>Zak`C!EI*{6Tf zM}hl+<-bg*V|st6%z+R>!q+6q%8SKqrF_Br@z1{dkiv?Py|6AmC8w~TtS~F?93c-s z$f!|}dE5|Ol@33Yr5a;>^%ZjTkF9@{%DB(QRzwef9yNCl0nb82 z?8AIzjS?AlGuv6^RyT*M?2d$2ShnF6B09r1rAAwfFr^N>m7CCDvbDvzOtme?L@`dT z0j9ZpXBU!R{Z=VVgn&*ptZdR|QJ=lkuMfc7cMKggSX$-a?Ai(Khq$hsi&;O3D@w?- z4X~1vHi*R_HHDl}11~S^LR2Qp5{o*oy1KuLt|bjx)8 zwbVzu6|U82`M1`6ROTmYrc^v@mo_z&_}NCv_|M8A=8^F~=nF>bB$Ez4tv&f7`K|4e zp>NJJ49VF|iN4NEqqdI9M7+)IoW2%=L)GdDWP342Ul8`M@?p2bWU1C4_*K)1 zmqIcicK`ajMT}7IPdwbGwz`-4yzyqnv--T`TpS2gYm*0p)mwq%j6K1UfL{W&lLF5@ z8~PN*pE*1)U6dhzJuu6uVr4HQF2K*Pil12HObg1~F0(rBBxkt%A|_E)_U4hL4%*k{ z_gj;XNaH*`23zv3x^_2DMmV0!Bk8wlRXY)_N2MEHP`)r_L!qB);N5V!V=mXqw({|) zSuRd;^2l9v#`yL^1ws=#!7Wi{>+0pyEH77=yaOSyph@*Oxq)Unq$&YT1l#cK_@gg{dHpC{}{X`L=(epj< z5Z+&CznW$V3S%q}GttuDiaSXZ&gInVr!+eLmkYD&)99nlG97%jw+X_d;l`!!;c+8z z6Juki4afB|J6xBY;6YFY%#RmZO;JR*AffrHWmVtsT7%LS&Aw0Ui{>z=hih`7AVfrg zK4MjcXmSEww);GQCr06zhn%UF{_tLISny>c<4T6hto6&gY#tgY67b z9{E&Xsv`@i4q+X@9(_aXTJPcs!C(fOgDoF*@*3VL7fWmHWd7W&s12VQ z-`(T73lnw1s)?IrahW8<8JDvJUhovPJm)k1{hS42KBg|?+C|cWFW5f0W4K!owwhxP zf{yU$IwqX)gAp>->##NT{DP3VR1 zUbFcdxBR?*JS>yCRUK8K*5#39-+X30d}B#Rs4hxR_QO#<#7pCtiE4L2>P}0zz6gJr zC$@>$?Y;s4C9&3hrzMd(B(G5`hYoGCdC07^`rr{wdaWp2q;+fhn1(}cCX+wfXiefr zBx8zg9g0bm-#+~Fg~S-UC$do@JEmF_{@UyAvTyv1)G%NdLsZKsz;Q#x#v7#!+sB+B zKyffO35R-F-7Z|7c0USTKhWN&j%iAaawCPm4|y;kL8e|W8GS~GR7|8rs$`|lMdsgg z<<4xqmMc=jURDxWR;k13l5q>Y*n^N@j3f0o&CvVtihEaA$=^zK`;(+&UiJ{wI!rdo zhl$lU(J`Qw=4qL#pOhA=)*dkbuDO^fs27`qmDOLqGfUxdxZ7AM<*Qg%yK_|$LJPVu z(i_}5_I5g5^|*5{ll0D9Q0pm`xstragZ$eqhPOav&g)xB)b@J{e3Iyw8BUa%o+26r z5++vdHf69+SRpSLH_Eb@_IdM1G#OEABo5U+M2IR)-d}=`YSUv0dUL*zeXy9m)XEoN7C4V)ioHP#jx&1K{ z8?8aw(pg%{FoYt^bBM8xhAUhwS_-862=lIOOfZ$E!8l#2=ILf6Aah)9rw(=SyVE83a0FnjEa6{onAMZ%Z=qCV zm1He>M+)*YYOMCoY zPkCq?TtjRwoSiw{7WSp-D<=sM&&^RS*afmEAM))ku)kP>wdAL_u4d^R3tsfO*-`wSboE(b0 zS>pBn#6phTLK>50RyF^}C-DsRx{v_`&8O}lb^zgmCIapWjsX|t9W6K#Dt8dYpViAZxnYj4D&ObN z@57%1f5=?c>PJ-aDa*#qF`hY>vbg8t@oX53J8LKP*E{{3uCGv1kTbrV)W(#yC0a&y z(#yr{QaXKQy}i7g1(+Z=7IhQSY(RLrWR`JrL`iJ_wPo$D zd??~qn)chFhNHJSW_W!4e_THlAuIJF zp!>+?r0E)Zb$uH(j%WJjeTwCJhC(%lqx%)ts|!m5W{4w_VWV<=4OGX!Ja@$sek@{` zBW^H44b9F4pWuoR>nOsRI7rFiO&D^LY<|my*mX{<@H3zWx0B9b9}9(1TJmu-2cP(ao}j?29HKGO}*oPmNm z#9JE)W`Z5Vw@$fjc(H7hs?ySKMiE|FHOmQN%`6Z5Et_eh&cwFX#-aLp$71>i*<3y& z%}N~fuL^UoMWA7x)r-+UD_D1OE?DtV%1T5xwi;?HJRuk&lV#>+GNY1Ze5pMV@f+$5 z0Ue@qSam{9Kp;&=*iAmmd(Qtv##Ef_b@adCLj*B1W~F9#31@&~Qx7_2-`Sr>3+X#6 z$P;1D&_0pH8#ijT<{q7y6^q-Ju`qRxq&xWM1#E$UH*EO`p~?f}<9AQY-y+*56nyQ$ zByLENi;tN%fBPCVZt@7N++#wU@l~w#9f6IxGb{u+Y$s3vxhLC2T)j9*_2h-gtW5(> z(4J6K|7!RPAU__Ez|){+T%<9se;-ELGnWhHP0>mC0)hyWA=&6(l7!P>R6klw>oH~! zi2L#Bk?AG$;P;X6?2bmraV76)7ZWh#TCSKP`RrR4o@ez`Wq0d|dy~-4ds7cqv_8@M zp=U3!oWaxvIAYdxGlO%}o6bJ>AtEL2OS`3Uek-30{C+o^E9?HR52UQW)Uy6*o8lm zl~f0*yQ)Z$2>;Nkb1QkSU(YrpM3wI}XLv2-@k7uXzeYaqJ1(Bni6PDw9<*hsUSMEi zGLt(-i6rQkf;AGgbCrW3OfPq@#xWDHY|M%7H!Tiqm%mkSl`*{YRn~uv_pKLRM6rvW`2`vaQ87Uwuoi(bwh#A3V_Z+ne47V#CkWKN#f5R7 z2~3{KIv$$k_jWOtYo$C(k}DE=O}qv6DPz`$)IsZtMA!qEM~=}0VP`|u&vljW%~wE5 zztUZ^MaRBR_VmNX?fTs;@rlv3yFSkb_>;{1f~BCcKv?PeDW-J_QGly)>8nfp((lzz+_I<}6xQn=l$?Eo5o z5?7ejtkyO{=s4Q?-!xC3t!n26eAId zHNz$V&CwKDJ!<>?=u^kf@ZK4?>*S%5b)j_o082tmgA)IHL2=6Kjap2dG z^J%pxti2JqL@)GU1|&wHA|;Bf^^jfGc!T`5qGm1)O4p15{r#A3v<8h}&NQ?u9NB&$ zx@<~*o>7UlE~KHyz#H*XfAuJj4jz?m!E<9GBfDIv7`6bZ!`^SW~%kic9 zp6cJU%S5aT&WBiBP8Q~RE-=2`RN*Q*yzTM>OOh#T6;xjvNoiQ}bFKA3R40;G(djBr!i8DpDo8;L?)u5ht zk5mck;kfBAy{EC*kVonp$wjQSlt}=k82Uvv08JCD3>6s)%ZUt`hbp(4%jN->Y)+s* zWWN|!2dy7OLB4LB)uptKpk@BiKkP-LEj>=$ zI_%Ed7f5@ub)zOZr0g^cR;tOQ#{o8y-uIMAU44wUX~Lf5Pq|2oI}HsErFXXZL!Tmf z8ole=lCPG}!ep7#cSugF93VLH0UTdwo2ymy?T$3vM!aoyOT?<+z2I$mD*S%IL317R zu650Dh#~g^V*o_WzD4HdO=*D$M zc4Me|nju>QH+dPcO~am+eGbXpr*FTB8`~GJaQb|>(0~k!ZQm4}S*0W2YMI#CFP_>+ z9DdERMe*xBF;M>lk995Ai97FGW)$J7oB0ry27?G`vN zaO{pq)H$pj{18l7B4aMu?`SSsE!h0Mx6=q868XfsD*hzBO~WJ-fAmj>sia}fDR!^v zqs4DT*{R7XN>#t{Pv(Xj8~Tnxy-ed!;;GHnr;Am!2&F)XiNQovdt+S8y{F3CcNm!$ zn@mNVFU!xa9ghb!0JtHR{^pQ&n5R@U7};lTY$^KOiS<}1qmDm!p9z#$1a(pQ zT2hKkp%3abq)4}W*6zS3ZRjOd7R-H^Fc82?wY}<{Ch9Q_i;+o!pRjL!C8;Oc#((Nt z`E*>3N&K!*BYPW*vpg%V2@3i2X~MEhS|DO0wCug5*@w94s*8}%eQr1~ zH^wuoiJ7d)u;GNll{~djqK-TmoChltt2C)e4)avlGFAm45&fKMZ-UBQ39pfUy4CyR z4;LiIRbxk3i-UQ2yxpBI(GNC&yljntyqlzYj(QTb+^TJKc2%okdgbGO5ZHH#!P*Sh z-WTuJ3(9L<@py+aL#UREJ}z-HC+G!&_&9|I;D!4BhjjKDL?F3;$E!sNQ$5kq6r)>V z{t5*xs9jT}=GMKa-Co>!^P(5h%#Q;a8!U0hR)X*6KF7+a;&*muVh!<( zI-2dhHaH#c+v}8xxL3BfRL{2YAQ`5fTSz4Xl?_>PU`$cn`U3;O>eNv~humWs&lOLE z?N~m!l(!yjw(iiBiM?iaZ)}?6p;kZFwoK;S&X5(H3bpC;G=A0Eby20e!Jx0Bi@Sdl zlH%ZpOUQ&6FPNyumO8QC(0%>B)-cRCUNt>mMqe`nun0s7XSNGOE z{GgYH_SL#ELa^&(${+0L$UcZ>^>R6QMGQtp8n4*aY$$cE&_e0zhIbgO!I1jIIICrB z_x{??p&ka{mE!*_mk>Vfj}LV&|9(`X^%$#!<1+Qx;T%O<066eh)f6bN$g&CVM$G=lCax#j}P zFb%)um{A~m@6QbAlgw@SvvV9d>OObWp z0q&>b7B(dmIMtL3t?Ia2X_*^HDNg@1I1(=g>m)i8aVl0ut2S55T-wB>y7_FLl?E>ub$)cCIK>I3<6PrYXOb zyA+x%RlXw#EMdy+!Vf~>ej8~a?!*&pWBIh&Sz$rqsl)UppwCZc?N46Zmj%JlC!D0# ztC3&O)O3|{_n3&tzfdLIeZb#DK!zi(?Q|{A@hrEJYT~#dda*2ftA8-C{Zy}95j&o! zSkmI8rY8T8tdL}O^Y~a6!$uf2{l@!QFjT56_&ws6+M9aCgQPT%8CTkiSuwOV$wr=f z1Ki4J2XlNM^>(m_*7hzmU1HC(ItZKi^y@YyRl2BSA(zO_)kj^+gx!4zgxEoJv*a)7 z=dVbr>&7noZlO8xH)eA^k7jBgQHSb2(=;oB3ZerXJ_4%4v1v^xO$lK(ksWV{4>5kI z0e0f3bUkE@(;y1`hi%tq@5ZTuv_8{Vi&nkg&uRqV^we%>Ms*vHnx13{RStw4@!QVh zJxhG0aoDW9XcwM(u{fi#lE1Rj0EZ)<#Ss)PHvO~Sk-H^khz1;v=FgRGvs80=Olm^K zYhIZ8=c1QThV4uN^x$V=w8_~!zl8keH2WAuJDDMllrN6hE;8AehPf&`=-cLIPtnlx zv$BUpKS@@FnNw;+6%9Kl&P8|Lc8QHfYL{;BFoCN4n)>>!8YN0SvRns<2jzm6n&mdR z#_izmtFLyr`et0C$>+|mzR?}}yXbh@qLuKtIc4G%o2P+e_*79Ep9cEw|CZ^z(#J-i zptcEN2%aT453dX%?CD<+(i&j=*n%IQ;U-**wur@yQl|Bh%d0CFiN9R|e+tUXC2n<* zlfq)Et8y<)S$J|Kt%bvET+iCE~bytbt4N>rg*pGn;=V?Cv zP`lZ3A=73qtXM1~DyA#4nH03^N;6|S+OIfwhKd@M?U|5Hzs2^6$0KM<9DlEYa-*$? ztES%KvwiEUC zRZ!e#YQK$6UJ0wHRKlQcmn&(1#_@9VP?4^IDq^$IYRkx;wYL9EM5NL+TbI01=NePmIvEUhr(8$SlD$)HF(^fp_kDr8eQo#82Ne z`>Fxc7i=v1&=Y8?M+Ax?ZcPiS>=Sv*!nQS)^vtUJNV4hz@l-v;Xcb$?VSHhsrFE|e z0fnqTds2Tz_&oD_v@G%~-rL$}%Jz`DyTRFsnqK)+`DR^YK4n!slx2Z{1=d0N8WY%2 zSYaIUGyJ827J|CiU0sRa|wda(uAL&I`{bgmPc_%c*R+?gUN;G8%@9n!w*W_rm!e;SCwQ5DUHOcxR}7N zQR;TYS&>AANZ1a{x!91TQSkK|7aKYaZ@#UYb)csLb$-Uk%wn@NSw;xn{&}h(0eL`^Lo)H(At@ny-?}>V>CFo}rtAqr zmY9Ucbl5K|Q0WHqP)j*u;u_AYxcQiExGG^WPpKfc+e!Ly&(3lQf^zZSkA0GKO4izA zdsGm)F0WcM_1NcH6pQZ)&~F29DKJjwH>wmqnDd{pJy;EWUW*LmGEFzCKQ~-HAl~LY zJEC7vupi+b&c(Z(vB;flh!ajE+bA$e==tA#rvg!=v(Ey1VcW~ zw7D?za3KxC@r-HcnJZeypV(kfAsR3tqOVmn`@{wi5{87s5GAGwqy|gw9Mn+0jn8F< zt9&K7bQ&Cz9_YxHm|f{At}?wJMohg$R7KNQ=WDvoRIR2*sfUE~XXtviA4@_%qcRW~ zjS0$3K|FY}q^z>8>zf)-v3>Si9 zIyZyTXeS_t*A7I~7oB1x&+qtPPcL=E26&0W3x_L(E-p0s6LDugI(S;T&#;9PfC)qd4cS zOYA9V4+a_g553bIf@YE3WM|E;qd^V_nlXTq2mGlDG4;^~z3gw$GPUXRnhBqm_F;*!*^=$v$QAJ$HF5N_4l+ z*AU9WP_7dhn4}W0R^qJEq*e^OJl|MSzKz0{*-dm;Uh{e50ATsNrzvh)> zQMQu)ITzFgFX{Ttj0Y!N5ZIVmT8-iRl+GeYG@6X@N7ZFhWb*t3j@a~J{8fr}^)9VJ zsMH-rU;(+cl*wRXKpa3M4u`a?GqF(7kyFSz`Mi2YNk(T=c-C=TOF9Vj#!C;bN;O#& zg>|shx6ipG3i|lNzS)`liE_PkYdw%SZzp43s=B>3=&&d|VJgHx7*mnK%r?&?5($VK z#viY6xv);2Us<@*?;hwVaew;mMXbvzy%Tx9l*3ph9sZSEF+=>+bL9EgJD!aK4(>+s z{jQYaNbzqL1Dpl5vHtR*@xm~b3nhWRQ&{#jeL;zhtC7%xF;;eHon1;YntdjAmLPg5 ztyNW1#g=r^wM$-lcKd*V)o0+7T5)YE)C7W!bgICXl;$X}Bp5@h88j2quPl3@K$<7H zW!5i2ZK%SunW`jYsV>;T8Bjt49cdi9#$-Z@lnMw|A)^9k(warX*baZ+w@d7NS?uJg z99RW`(1;~w7_3-zTEjABf~DKsS=%5M${!kZ*eS%bwaaISc^{G@vf6=B{4caL(eQCb zhoIF`sXQ)4fskL&@QqRkQsN}>P1DvZMkR5rgz7nA8*tMG%vtK1r6gRH5W!(XGX;o3 z8)l+WGkRG#8~3*G2Xltgy4Z+D67Ec4nEicSSBq_Kvu`TT<8t@h`F}+ z5}c0FS>*`G*I-YZ;$GzPo;`04K~OOMeQ9pmyzR9=(J}u{zFFJDT-)8yCEgcg(XXulT7$b`2e$Fci}E5sE%F{Gjai8P-rRz0Y#-zjG) zM|*d>!CJLY016D$v=e*R)#nf)qb+O9u_0%wsXVWkbrKdZAQ9Wm{ToK9kF{jiF^f*-KW!Rp}t_28JGcAKdn$!M+WO zX~tYUX*oT^7djO`D#W&1)1%fo-WTszE%wK7+V8rk0Gqi1dJcaQx=ln*xwjSBek6{P zn_o9mFMMN!_2tc!(nbKCa|wDpQ6;LKnB!Q+oq=H%{>rBqjGDimYbMQ70jBl0Rd6w9 zzs(~X2&YxP@WJ2XODxqu;#hE*N`4y`tQvHcXIxpgfYY(nfbd8;|8aztgQ2!LNrHl`LSYri2(}m*4Ceh9#5v|yL4mw}Xq8+6la31O zWZWF?_e>zOY#%HU^!^H_9X_=~3v#0*T?HgvrnN%f1~t&1*Z)E(k3Dwix7g&6bN?#% zow>+5w@N+?YaSizVBbqbCsmzZK5e=Cfa*q{gIJh9mrC>b%axd*azkxJ2lCkNgSu0r&sY1@d^9} zpCBhfl6bNDLND7VF?nhVsIbL@d`M%iD#MiG_}dzeb6!xw2&&rQejl@D`w3_1r$e71`>vs^Y(u=AP8isS>MHp$EGLw z&CMkCs*FvWG_IXdAh-2{U1^9?X{d^-^1%*V0+rkoc77r5k@%aFOS1S@i!So!!ekK4 z=YtM*xWJWRy+rYtx86WXUqb;5(#W;Bdp~5x83C%zErBYyEn7cu^2IC8mA0G0K`c-F zW{nCH0$;2Jr8=QXK3uXyVP1c36Xx%y_li|GFx;+MfxOi$1cNbMj8~g{aW)(wn_+>b z(t&@9Fa91Ay-qs+U``O0Hl#jZn8vJ6SL~6H4~I9885da>Nq)IaLqhOQ=X&8Sl~x1& z1_zhvz;^CW_S+52Z;3y5N9sM(7d(sLGhnEr!c34EN5_;nAv{+?QbrtNqDyxn<&PKo^>t<(_W>Wg&R6KtHw0gb4g zl`O-Rm@Qe#LzlMWnrMa$TflhT+4Q(CTU}M7bDyCcgG{)|QSJ=ODL8zG>M3Z2iaCul zxh>f9H9{tl=hAA|23X}Bmk~wCok+Q4&i05KuPhLQttkTH;6``cra&}=i$}h%yh0VL zU)T?f95d9hqJ7yy_Ga(d=S%g}RP)yvwS0{PRr}Ge{#GOtA1i=*n}& zg~KscJf@jFV}%@*5?3RNK^WDfI-SpI*REW2j*1xFa+=Wm1%6X(nR|w48}yNj2#q+4 zXQ-{+NV($bNdpoesvUKAJs|dHq0m|sXKp-FgvS}aWdhMfaFoHyOHdHha=&5^^6}Bt zzJ5sZbFvQ`5+-_RFSsz33^s96dvw0NmR+L9jfxfMH$M4V(P9U!B`-f+b_5Jb;^qAk zUgY8%LSF4YS-B7~8|hWHqWN-{>P!F?^o^i~NN7u=%g~HMw_RDLX~@2`G5sbeHVG*e z`qBFOp7?OrEOt^o?a0i`#txmtv{sg_12_GOe9?#?-A`U|V_%1jN8a9eg7vf>O;m;Z zLqM{FTlN$!g$~8pk``%Sp%N>1$-RsL+%twZ&FFER4 zmrz_sy3P0c@;2)o*cH}C6u~(P-JtDDOLP{@c;3Q-YU?WLFR4Hz7`Z!sAro@i4DF?F zQ;!#y6(LyKG6H3-{OC{3O*Wp2rs*RAL zi{G|u@_S^QW-|U~G<7#R8rBrYo0T-$PWzV<`ER7(u1*>_4h567J{G;eO73-qZWDk% zQRTqlRrj#Qq~C{0kH6d%{Avf8794uU$-kPg&V0*K`pHl+33}qg>l4G+Nr~wh<*n(h zm#{jz=C1Uyg3)`Xu*_8$u{#;zHLNP+B=c%}k~Jo9g9Y^AvTjVe=u1AX)}EuV6r*7)D>f_h5)NpM(uvX&Ze1C-4VGT|AsYq;Yi~J> z6RpaQy|4kr!df{x+smBjj60}}qK&SvC-&{z1o9JJQ^rBvtj(D_KVA~IzJ!KCf8hC} zgtU6tl^6RE&Z3o<{2@zTDMg;e>DAzd1&m2F^eEQjm@>`yCa{LW;iN0zUIg}0iB?0l zcYJ+JTq(WCd8T@~aEJRb{P14*rM5|q>f>pVZIgi5pA$Wa-7E)E0TOAyQN%*5JA4r+ z+(Dk2@Zh(JqhjZ7SpXGz^9~H(bCeukWjkK6RqeZT7&pX?^s@SU#(VbbKwdh|?4z)F z!F+P1opc+c!jtY24p-PY>`(9zPV?~Ya=y3ztcB@o<27-SD$HZ`NsskJqs9wcY+7mV2*w2O3<($~3jPJsz1 zY=DM_aaZVP(vx#uX*rB)JYEh`>sWUs{1hD=4ou(miLs(7>B`uka*5nRs z(WM~g{ie@Tw^Ko&G5K?76Vq1NSZHUUl8RPgQg;^EJu8ip(ZwpaO|ql_iKKZ=eFL8}pM!uH1jXKxr zNJ$RWi-b9k@dYd1Gd+Ikv_gCRX}*)9{xM#8=Io&4(%zV88n6A!oWxf4*R6Omf9b6% zW>$AD3fMu4VQR*l^IOor$$^SD8;b66s?SnIy-@aUb*{!Qf&B?{H1yV;4t@c*GC!hH zwo+IJLFWZ4KQ?6P^jH(&ziAG_qH7J z7??E?{fLk?8UY_8`~CUJQ~U^H^p-L|Cp9Ab6LUz-pds9bBs}`1zU4|6sY>cMlwfpU z*$ay!3#Ge&foaOzA-pjj+sBFB2{z|u%%w=I3J3un4uY8XL8@Tn78|MBN=r})oF~LK z>zNlb^es1NseTYd$No5ZTG=A0xly*{uyznT!Voy42x_4Wog2iG!uVLge0Se+V3K&gP_I5b!%ItXNq}O(eD{1ZxC$LR%K7 zn&Bz!)=xm3MDbHPuVzVkOfi4+^_BW~-}~BCQ_O)5=%p_fqDMy0t_!o8X!XH(e?X_V zvSet#DwV=ypb2V*ifGkafH4h^w@KTg|;r=wRj9`)sj^wxTzgj4` z*Y(dvJY12wrOEJ-xDVPl!z5*zJw1@W$i3`VZkiv#e*vB61}qK(l)!s}_y)ks!7=RtE!jBg)YQ(z}~mL=^^$0|-7K*>h}(sM)G zt+nxHjTiJQTMq8C!q%?rGTUyHhwFlU0oJ2oFF4q942U3}1hxfHPZvBbw7Xi>M+RO# z3_6WdSS&x3Wqw1v-gVm)m5nvAr@ouUE}!he*JOFu2Kx+ay@(F{S3C-=iQxqCs$+n)G!;a@sLGwBTQX^v?d0Z zRSQ|qGZQu4*iG~^CztLmY~GI$0R2K7zTGmONO85y-Sx`1l%vTy#E`^p>$acCGyq{q zuzYsB)kZFko07Ri{Oc9hw%74eqdNwvKZZyn`P>duRx!mLE(pxe1s9n!<(k-b$De{E zo}aBO5p~OT>Tv>%oV9Xn-tjM14MqfiRpa}LDw}_sPDquRe(fUaxh}M4kWmI`z78Xz z5T^&b!g}z7vO?D^`a3b4oTfp|*#7=#=296qd!z2cdYZ7qlwI0;c1_uI#l5ub?(9Ruv}0TP6Q zWv19eS;6n2Wkj$VM%}~T)V}1jS4<>vtGd6HmS8R(sNT;>%MN+TQ?x8P(GZ0#hVr`M z_j4nKW#*)n^o`HPw(J(_Hw+ZRtj`pT)V)ILbH`?G3=&QFiGl~x!wvJi;DHl-?@^o2 zl~Ea0MuOhvukSP;`v}v7jQQUl^>U&LbBn3T5j>?Os+a9(%axBFY&9*Loz|kC-Cotr zzGQXthQxxiT5|rf>_&=fS_rm$XmsAN3^3?%W8k=&9xwe@qyz0)(QUttPR< zg-N+-bAAy;yp&+~3k0F)?``4Lr;lkZk`CYa1`EnugY+t+0!pDVW~MGEqX#J6w2r+I z#XLi-a~$&sScXW9%W|{!lYo>0K7qFzTEBN2-V-`>9TL_WCkPqL_JH&8hEKqYVWkKc}`lESR)$SEL zlnO#S^iw^rR2^8YggY)@26#{f8Rf2g_C~!xI}WhJAccRdY=tR(c>|nddCiG=f^)3_ zmebIlJhN*Gom64jjX;sAX7R`ay8kx9XOeI=q0y=HidlQ&u&OSnRlUVVY+6~pD9h&& zT}XNY6x+GFG2vcolii=4Qajw${a#}af;Kg9 zla6v?wJy&WoEWVmtPDk4>EdVqo_(m!!JfBmA!0B)0f-R zsI}BsgA_lvoO|Oyc}4YEtAy_M;|=4-n&!)ahaNi1Q-&W~xZ{3-gf#4-8sy}l-e>HO zpOxW>a!InCaAo1msX(=_qTWsHXUc9K&w(etu6N$ez>2WY+6He3>~wL-^_=wt3)DgT zm?o6*!$t|n=~ak(NLDXK?|J{PtbgjdKCU% zCf$s>RbiwC31Ljl#Nk`UiLjGJ_StA@w^}mc(E<>T$~Plq^@y>@+dr%U5AnyX7nek| z(iGX7%rG`82F!<^4&j=?^%L2?Kxvq>AAEaH;Hsp-__Mw14*cv5f=jfV<&^r;L4?2saT zXunw%QDsKD@QH~|KS0(z&yv4e+zK4s9oU>#^hMcl2*_qYAx$@u=}WsTa{L~o$uf<5hn4aRQ;t^bF5)4-c_aL!zsdlMgYm-%eAZ0%NGM1x!1wm?*Jx{R6( zuIRQ*43?>EW#Yuty6X*g>IfsLg3>ate9`CqUd&>I1L&}>9ze1A+P`^D5L7nc^o_i7 zOCs;Y^>MHt6b&9=^p{mYCh{SNX9oEubha;HOSqLS+49TWGHs zrEd8`3Ril|A3!-bwP|qPQJ%h#(u_Ngq&7s&bya1^q{_oM!!d8{R_Vbp{qC+X1#(kw zvLpUojbL9Zu##_T7^r~k>hEvK^68DgD596kJ`qiKe4IN58l`cjXJx__VMx@0#7t}Jj7Uq~ceD4Z*DAq#1gavq*yu-X% zM$Ps_FXUEtI*OXGM3A9TA3%VWt5IChc!Q+$oGwKqw6q$8Xnb97wZ14Q-yHI6(@il| zb`1fUxV6Zm-&wmYkkywkzR!VIGrKqIE@^sH?XC8vtL|j_i@3$`Tda7Z5^v%~CuVO5 z)fp>sc7`81!>G<>eT3z7`wZ;7_ z5%&4|Oo-wThJS^-jtvL&6O#F&R))Y;MP^5u$_`}u7{_`{yGp_*eM{(3__f*o_8~gX zIaEhTq+W)xj?7k-8AQvVU6!HJZ%Fv5YNT>B$~k&RV?9D|`N-}2+IHd!-{Ux3wH@^( zb@KJ+d=NGcL|E`!x|@t3o_DaCzmB1$mEMS)lZeJ@ zIe*iiCXSER!gTq? z%-zve7jmz7pjNl4)`~v3OD%W#g>vpPe;=^D8Bc~|xzz#11n(@;BI!l2x3sMCv^HCe z;IfbPfIUoTRc=8BhCkjyA2K<4`ydGV&IN;jh=&B)Q!5N=hN8EyJb^*d-V<_I|0TmF z`b0=pp9L_2Q?N|VqlbtA_&SnAZNPQ)H`ejuLxc^BiayD&KTM|X(jsnB{FyHr@I=6K zi&>693wobePy`(R2z4bi4Se0#?cyAGR9GU)NT70 za@d7vkribQ_|$BR8w-xGzw=s}JSCDQ^eWvH=GxXmf$CaWH*RZr7*<_Dgm~crIo}Gf zX@~f|L&a(Kj1uZXeYNsJtK&x);v8fw*db^rKJI?xB|+;>CvCImEjAPLlqh3o;pX5= zXO^qM&lU$MI*$`Vble}&*B2!?oKXcp#A)Uzr)Gu=k$NDSlsD7kuygm^Ea=J`Z%%d~ zcnDt$2alF$GE7D-;h|rz#0Gz6hvyvc>C4E8Pu#L3;NlYs-w1h$6Mn@H#&QgF?m5WB zs(C_Xf6%NTq7j7K772Ti??gYHi%MWwGwt&|ikIshzNcfQCpto{qwjxDqO=2xXQcNy zYN_3QNIU8!vqe{1GW}MFYiciEGbUMbCZhDaY!U7rq^chZr?jE?7-(bdLM9aY_{Y>4 zIR*Zi(*6qVUMWj1I6QFTr-inWn?&ISf6*nJy%2BS8Tz_?99w!Mz2?-XApJ}*hgb&r z5$zt8+6zp#;+ID~bT0zIauB`v=0V5@i|?bG{mWJb#Rn!{6Cq`B{AGW;%)s2&8j(4) z33IyJfv@%~G6m5NMexv4$4{nrIxbe#pL$&UZaC{gU46Evv1mSUEPRPVY~&8nhA;bx zN6;%JL(5NK-CZHrafp=?5(+unbz^{rfWZidw4J7Z!^e$NfSd9wkVIY0V@PQxPV7c$ zp!xYzd8zcb@WnkJp`(3za&J?hSp^Oip082YHmcI27{Ef4HRwsrz0&7y;Gq=~LG_hx z;zij^ojeyV5jYj9``)O*l$zW3U1CV~y0F_|PCR#pQYl`=0Y|M;HyGt`GiRDU* z-rgx2wN7aL3}zNQTX-pMT$4CE%Cmc23t<7hzs_d`3(V?7fT&mX7VUYgudPZ@!9k)F zY-rf6IzIAzUbmJAI&9Ya1Wz$54}|i9a6G_*>f+z3{d^XSwXof(Jom|A4AHK%Eqrm8 zK=jnK|4DjRLDb)!WrVoAOg>IMe`Th{FqWs4!%vnJ<2M9Ka0Mx;1-ppMqyoEv!fQdT z2CkEMyRPeYPHqFjONaQZ{qOQif1Pv(UZRPfKfjwKoq%u&-$^^qV3+y@G)cL-lH4Wg zVnB$RqFUfZRsVMCU0Ms*aiwvPA}3Qw`}+Q9s~tBXY=e5G@B)3d-%i{LbulwG$&E=R zJV|%fEE+MiY8&K;!k^e(8yx5eMBdQ%GGXJU;wjDZ^*O37G(wHYttht9RCJ=3q)4wB z3Pv{*7w52YLwFw-c%LX86&89!_Yy>2XRN5PP*w+SpbR;qq0to9;Fnk-hAw3+CnvB2zoj1zKiV586w|}4)nrhnx#XyzTHHDTk z9~rWSB6B!mU()A+0%^`amis)=vMpAT?KR4Kz7$bUL}F>eE3p*UK6PS}t0WPq#%_P) zV+m1+n)_=#c#B%bYCQ`TSaX@Z#)?5XrT$P0o@8E@Tf}LnURzc|JMM{SXulwtAElDe zgN_Vpz%DV@l6^*f9f-DH6lBOMD01s=c!q!tCzIT_bV8zETgKEcWPTwYX+u{b_+S~a z7Y3=T=y=m&s%@{IIA2?gwtDuLf4@>={zhY#-4t(M;&PWy zIMc)5E>9Iv2snUkA0fUHV_q+i8hK?I6gg0J^4eOS0>aAs*d9%tV2dQ?DgQ9YKvk)3 zSXB5bV942UqFE~SlL%d-T5P!#-OOrtBJdPVEDCFv{9tnS=d4c=-R$%xFdee37lz&Q z)6g8{QHWWR5Rr!ALTqd45C8cjiCi1I26XoD(%fhTz*gvZF6;;Ql>4k8e67dMrxXzj?Qs_)~tTZ zYU`~jFze(hCMrxTq9#ROfk5XusHEuvFWfIIZCdB{Gmg7G)210q`U*#{_)OiL9h1vO z=H6o=FqV^mhE6vnh=_(AbWZ%_q)0i{9)D>%ThPTC)NYHKD?clp+eeH3D$V4T`FVe9 ztYWr(er&xpDBV;D`a}GlaFqQ037N;&QGGi#-?&MDW+kkkYl`WEIuPdg(_=V0;H9Bv z?yd8B^r~IlW+kAc8)5w6h{!{^B(}(Mq1U(m+$Pz7V?5pGM%RSQ@0Qr%bvnEk8 z$WS^#N2OW6IqkzP6`wuz>s#ED9`RdzlD{c1B)x9hk*MW1U}CYpG{dG>{Kn?u>KlH* zDZN{smq-0WW{#fC%kca2vsk;pAKo5JF4r$D>^@U=@T{erEPby@5k5#|v`vD$>v;cm zS6ek!<0$;S26j88l9HQGJG+;Pc+ZbFSmgwJ-8x8hURe3yJsY2{Ll?l}%uHe!eo$TI z7mXu$!tt6T`!0ra%59~kyrc-VeoQJh8bIHolP^t--&P9wy#A?n$VLDq5kk7|8=Lz zsUvUhb+oniC?NDChD`2@@@O&w4ab#X*o=D@p9T?6#cs7cJ>Err@AGp@ETuX*20wH$ zGK;6USyK~yXrb0%O=sdhIQga~Z)o{X=TaNt6@}cW3nKPOblljW-YRQwxWGy_6nBB+ z<)8E(?Wfl@Ih-}A_(pr9vXs)RH(0>qSSNcXRL5!%wy?w|<=Bmm?@abM$}Gq4(x#1L zV$hoM>fEAO!uE)@dO}IAe|K7@cB^o;5v^V|x*I2Bz~s{D4dv!?7x~C|Idpz7D>nqN zHqKQfDYdD<&szt-n-_`k1?@uwu1n3Kt`jVD!}`o8lf7wgT5lJ{$6Zt6m0v|1W9(1m z&FZXAk}1k?YGr-vq}tUFgSa9H9F5cR3w~d-$-T%#I^~BJd(U*l-u<%f@1{gi{p@&J zJ9}O`gy>?g22DCzWC(&=Rg4>Yw{zwi$uu?1mV#MvBG+l<2fCK+QsSta_VU^R%UIhx zbC@($X*&4LM2?<6{oNf4b2RV_ydv@Bf`zwER&fiAi%Dp(Fl`rc_-vz;YQ|SNO7Sbv>&rTGMKzbcR#^VV%QdY*8{V=|E+I@p1W1QIwZqLn3Fj z$?lwHjo#q=*1aRU*l{72uD~Ms1k*uFlvQA-{a~-ml%3bFzf<>4$$ZGsh-nEW3lO_n z<3$P*t2@N@tAaFfF1$d=LN3+W>@7Ae4KRtTHmx%jM|!%Abmkm>jtyCQDu0k#O#rzrOWDC*JqAqNXw#qUf*2UU9sG-i2Of zDp=M0Qg8U?)f#TCJSHI)S6rHE!1$Fkboo)JiRUIgy%VH#KDS41*AH^>c-gg)2<zz5GRfM+7Eq7ioJ=;o2@5Nnf?4t zUk2hu;ZZx5;3J_dB%p4O4_)k^SkW1_+`DmwkNL9rfWtp3BMEs(6m;LVD?rDXGUi{H zgSa`R+8s&eQ>b42`FH!vyPQXliNKW#L_&D&RVaIlvUP7U3|!tgUWR8B%}R^9{!2l6OA$K8PaW9~Iwj zZYPg?q1&k{M~Ruxn<1yoWb&JlzoHkd0Y7Wd4wSZ%7oxHi#I9oCU-RWap^N3i@5@3d zSTnuGCZd;?f-64a_q%n|ysT7d)zo%iUG!XIn}6fp*{RoD+ z1MH);A^xBkBB~!D9V}+=<8MoQ{Wj^}C_q3}19%wPcAHR4fX~HPA8C8ZX_OzngzsY< za0IxyF{~*SfqFAee+|uxAm>_~NeYi$rSX?~a(lhyISjKnq0j|*N$}(DPjepbR4Y~W zSl)6s&gPcR76_*-s5>_1f17vbxO>R+QXLI;qV}n=?znmf8l2d?y)Uwk12G*@-Q~=7 z=k`L!R)`qrLrLuE3?b(pn*@Q+1%MGBz5ur#?Baz9|IVbooObqijy6&pi`>qIo2lwF z{)D`t!vem{jD9ceVHi$02|7r(npb7?Dy~?6y$nJG6Pze#Q6>;^aKRM|OJPlPV=O<9 z&cm*}=k;;H<=OyO_8S)93lN!oirt4_V)gsh8R(aL%dpm9-8_WIg*m{K5?N9QUvP0^Istk+iDpm*ZaHzMj0Q40quI45X zopskJD0t$Q0Gk|YOo{zyQ@v3 zAeIT}%bM~GtpfcStXLw1`|(CCjx%nOm@|pX!E9s#Wll=zmhMZeW6uQyBYu$ai?1KI zLrQOc37BDt?T0@F=65dO-kt(?g!AbOx$^B$S$<2+s7Q7?O_VEsI*4TFcZg%dm}0gv z!4e2fA|b*49tPvFhKz(Viu{BiQ-fud#xv3OJEnc17SJp|F;6geVLTl;(wdrXm2R*C%Vj zal><~wO}h1UJcNh0xo_G;_>BObj<-}3pGLprVfIi#3GS=^BAQJ4)>3u^V+c!hN9>% z(aR&fTzZE$dT*xl^}|R>q$rxX2K$`7)&%T$+4?vRWRgx;DLO*AuS}Qj(4c@7G9B8k zo!Ud+AeU8ve3DP3aKSf0bpV}kR6Dk!OdYoJ*{$s_GJ27DTc&0Ll&i+Jh9MfG5lfsiX=6_Q) zE_vXpSHbG3X-RQ6HfItxg3%6e-M&f6Z9mTPJ|jm*Iv|y*GR468sn4MF1h6dm;-;LI zu;DDco%OFWH3`E|cfzxcoZse9z7+GRD9Dwr1a2Oh(DVBTP?zy#OyOeAKHNh2X|ho7 zeC>vDT)V~TRU*X1uM*gk*BO;aZ>soI>l=nkpLGG1I#~n6^4w0EiOrmi zxl%Db_YtD8 zayk=ZEEd~K*K3U*xflFVD7_8k66}z%fchtiPrl2}UA@qS`bX>$M-?UdsD)B@(L9x2 zZKDw=8Ly~>IS`3a2~9;Yb*fic=^TYOmoJ((gt6{O#LmZVtL0WQSoWaD`}8j8gjBS> zg?kUFwJ2q&?I4Dt;Xqq>ZSrJ;h{>CW9n=vMm5llPu{F>W{;;hHJiKua_Ni^&Qhd6X zU$<)LdHebX-N`r|O6A}=h^6IHiLDlfbTZO=N`+#q!9Ng2#tIV2HzEHY#ci3M>r|XQ z!B_vmrj@#2_-%N~zh^}LpL?*B-0^M~I&*s7EmJ_y`E17r_vMw|6Yy7nNy5P$1Pwx5 zT3*4~A^|U_4$s%*l7?a$Muf$vb5*(3zTTfuP@aFxxBAy~J{B2uHIO)6%8*+n#-#nibhN(ZBr@OmlR$Yb>OhjU(H% znfQT5$682;$<0|I7mQ+18lHEPKqZ)%&^9gyH?z#LDYx&yYZv%X!Ni$&Duk=`biJ#J zxW4RmW(tF&Qgv=ZEG%o<3Tlrc2CKS>_=NheNgix!p<_-B&&f%cxI-qX@)>Ny<}OJs>w-+pJ( z+B1V2!!(tk9iSrTtgYft7}5(fm@UoPVCh~Gea4P;I!Hf<`8_D~Y$$`h!yi@o7S3KB z`!>9%M(Nqp)U+M`0NeE`IIX(q7#`>ODY7nSfG7j$JN4R z=CU2B$Yoo(A@QES3!ksy`7*15mLWz5$!*z6ffA)hW`1kx2YuHs7z;FGMtM6Cv$QZ2 zin}O?g>?7PQ)94vVyA^TZdvH{a8mus>#jBz_qH6ht)}y ziZhUqLvJKwi*Bn@f4j%cMsTh+mKX-5c>34nV;(;JZ#d>`(#MLkG8Vj_p~=CogW20@ zO?3W!t^~!*{d=w4i*KZc;_pMFAkV4cWSCx>GjDV=j@Pr=NKWFHjsoyuUIC6+VP3Z% zsph(7sZD|N_&lCT{>j(e+;9kv|Gtzm+FI)kEr;L~<4_L@#;XoFI2+rvIV}1ApcvJ8 zA43J9kG�A*m-trAZ1)5u^L7W#ZJEf%M`MHqIl!`KrEgnRB)(M)WuYtEh_IP{y2 zzSNcPvn?+I2KXvG@rf4A_MwBBxXMHb?!t0;#atAXVyat!=RNc{n?3Zw*D~Wm1a_}4eq zqj+D)?7UY6oEE-vG$j+mvdFJGVWa8UywD<^C|Rn5}f6Hct%PF8E`{8OP(Dvaf*4sjJGHV=5s^EpT`(uoem zI2jREC)}Ijgi0@l2?smhO|83q7N+#IuKJJfQT#``=T0>mJ+n48yLF!Oydv*x8%j7= zS(hNM2sM&lSQ$Acn@Fw%`NJ;-g=~>eqmwa!5S16NmT(GE&$X~6Lp$2w)ISV@#Gp+{ z3r`M|!Oux#a(Dh%F=m6pYMoHaj{j$kxYs^&t}eH%?V0F;66>vlEj5vjOR&s@aA;t| zN>J3u>J-B|(3eAK{7vkzS{|$*ePY(81=dS5XM)U^a#dkInRXXj-<33%A}~zx<_STK z#KbbxDF-O4h*iek$UEzf=fg2nSllMIgE#dDqR}?PEx(+CrqQpUCr-eimaivAJdH8K z#XWVAZlSAZ#CDA?Nra>dGKmy*>jPye_ zP#OA&nTLiL0nePJ@^I6*{TR_P)G59k#EpDJa-aW=DMD7-nIcZiOw?11B0IJE*_w+s zr*^Xav8?8qy&VS=h1JQI<|?Wg8G5}@YUrSuotgzM4MMS#uWE87mIzUI)aus&Jiamo z-no0w#eK=C*s%-uZu!n|cA@f>LT%$owUM~RkRJe6xz(vNC8hqJHbP#Z zb&=`g#%+hYi1V~dnuoe;QZ2V}(>;Et2Mj-9?#J4g)~l^u@>aCBirW-^QPXv`RnX5Q z*`aNZ0lBJ)EfVqzZeh1EDXhdy!r7UfaqJMED%UyWU!bN7KNB^B@VkwiGcx@`Xx0c) z@HKlrL&H+JdlL@VYhi8U!<=q|;;V_?mLsWp0b5XU8_#i>L6iNO?ygx)bRUuG=+x)t z8@Pt+CfXRIt9r}i@B<_( z@vC4>x6iaEc^F2{_SJh@fUQ9(P5IEACQFnW#YJqsjY?pshY6u}ar9qjz0B7-P*}5q zo$&4UfG+dx{4r<8D}*?|IhvrS0lo&6JHez=b?;ew!Dl_;>mh!SD3E?Fr=Z?dLS~oE z^P*6B@|_n^GJjRIPgQ3qN@j$MZ0tjOA`3CFf+g|vwtxx(=(LP2!!YIK6p6I_K8wG8 z+yWqZ7_wzA;CO%LQK>H~EBxHx$rx_grbGagconYEn0~!_m5r!w;NGeFXkrEYicnf>Tpft=CSu zXThm$7?keT$w za`RQiHEuOox(t9c;$M9_wS2$!v1_tVk6lh-cOlJ?t^?;N7_BGbgDCDwm!ieqcO5-p zj;*vyiEGCRn$e3E=5~YN$W&k0n&`JVdJ=YHzc{4n7h~5=Xc>e=J=Vy*CD2(Ti}Omvx%%aBS=@+dhRyr!%MVhm0+rIK zrC#Fe+h=~U4K#(}G&$R2&@(2|rxlcCw~@MtJo=Lm$% ze&{OJtSy(|7Iy~Do;8PPXh_dmv|^);dMI%vYI@x4$yC($&Q4mU_8~4W39$&4>B_;a z{#)Yd`~)!?)g9aR5&pmLw{C9aNg?4h9#*w0giFgb+(-o(3h!*-R$bFuA#Wm!!PU4d z!d9ofH4)e(#D+fD-bWa0cLT( z*K`yrwKv}P_k>DyGNbLA-uIn-b&~%bG=m9nXesO5PcW5!x8-_`&71TbpXB*m-^PJS zJ>g#K2Hg+wfI_*@?N1_1s*amyS$iX~rADcU2E&59_i?o)n&W>(C+NMXRbY{d#AbwXzY`8C`jaVIwrt(8j95nsBG1pKb;fFGpp>LAr~r9^yIJVrqJ)P#r)R={6%UE(7no)OO)g7ratf_QSDYWp2ErR7z#ZsNjj|kmri> zmG0%Ujv)L8`_#Jm4Pe))>%bu%Zgj((T)T1Jk3bUshrIOo&TD9D#b_+RM>j-4JXAq7 z;)0OvC#ui=Yh}>_17+9mTwo&q5i=aa?gh_%J$2L)tCkZL?DVO|W$H0Ay&<|c!B_SZ z4iSes?X>Gaxn9NF5#W?}ly0vwZ?_JetpBB>fny`)npMNQEB zaK7gB$5APf5~XFkRfA1%Yj%Gv!}alFR2ljvF- z7xJCH>_YE0LauJZ8y#KcAEw533BQzWgm1a%2UO{Z?m!pyoA$KT0P6g9lF;9IyXs%1 zW)ytkR6ps`j-1bn^pNr1wX|MY;vZylPm-i$y@eZdlueE^KMlUlk-^9X9pI#i*x=BW zm7PN^N07hgG6cXg}>!i!plcands#U(zxh83TyU^BZGIxVU|dAen-eiZt?8E~lGlw2g}8Cz-4n7>w1K7U>TrL3 z3Ktpf{^e>s4e4#ec(J?-Bb|cS_?s^zUBWNMLrCKf{&~754UslvbO?vHH{@eh?ab$G zq<)|DJF_>Pzk&$nF)`f8{E1s6^Pl;0_Pxbe7*{o(L^&#KZ`@&VI&efm=vvI*Iip?l z?g7Z?QC66)`YZb!zIwNeJ|KCQPKvhBXD8SLHd6?Li0FAat4Q)n);*Gx%q8y$%t$!v z24z;s!(@*S;p=)HjxG>|W3k-pucW^smi=NIv6vJzqdgD3_8N%@k#BwcwybF}!rG^H z&>T&^3fvO7YnH$-DzD?nqJc?lCENjY4-~*`g&#jf{#1vGYNS5wp!a%ukjL6sXt*$O zh9f(7q;A1d`GlxxjPI8~Y?bKDXZ9pP;B4s+EFrWld=&4LP}vF8|Ek-LojaT-0?^-Z zRErCOZ@O`Dj0cC6i^YbgO5vX zO4HroH0~wnUIVW$5=J(u6ZP~gk*r_0DBS*fu-2<1&hXhy>%0c7kQ`$0PjEpWu$B=l zJ=cevmcxcSr||bx&1fF~hask&Jqy!=Ly;|ER^58r+fHJ!Bm)K&U1Nqu0tgLhcQ%c= zywD5_H5<&mz_>*rsx`V+%)-N}RY2<7I3ORNr8zHrxYuy%hG}Avx~ib94nfG#ta+~e zzGvmfipADhVg5D)-1<5Wpk!pbfWP}}@^ewMtiKNwQ`m8ghU?u5@{BNt8s~5CPjf81 zs9t7?sXl43oE+6_TmSy{;^T0RvsEdXV&lfxVuBRvrkPTf=GIx1#h1%?xS=VT zzD497Mw@tKn2#5Co9ehKi?OR}H73;Hg(I=Hjiq0BZ8!!AB?CuR=o%EY5!naujd8z} zM8E;+RPX|?9c0|ah9bR`Va{($;G}SX=JPHe{Inc<#1ICS3lTh}*+s?5dHdrh%+IrU zhbtQsMlO6(&ut5C?3u&1@uv$)o&MAX(~w=BOH0w9Zh#~2zy?)WPu!zj^X@-qUecAO zej@&uCH6(ttk?Au!Jc+}OWC-fSmu)jqbiHDfIVex%sO8A^Vp64sxl*-M+D+TvnHu9 zRsOkWN;e>6l3Mfg)a<8^BUS!ym3b|tLxw%a2#c?WOq$E>JxD+K|5!_6 zc6IeKxdVp6}cB6Y2WCn zs-UxN8`2w^E{*QB=Q1@Ry^JE7LvE-V3=~1@~=9k zfL|r|qjGNV+3FW+J&7rNYUtGNG;|Q8QMa=88Q^!i=kq%KL?LE<>>;fyW)m z41BR{d=#=XnnpEA_0$rla@fVBhvNexniU0!)Lfwlaq*`f#%(8cuZve6el{N!HPz4d zub|SIp)96IkUiueKL~R4;^eE@Bz9-9at0(jc;|%sN!5%&B0#b_`Ug2Yq)RXwC&O3j zQ1L7Js7oYWnSx^C=Yys@_Dj?1HAYG_EnRxX7^|8?r%EwrOMhi~zmaO0M@D^TCxbBF z{8P_5_O<2X=oyo&+7+ijWYxW0P6o$mM>U`i(7uGb`s|3BG)t>&$v4aV;~5fNi!IJ;n#;lApqf2LyqqJtRZ8NFb=gW z<3ypjk2dS3Z9Zh|j0pz&JvpWt+bIx=A{zL=3imp7A=<9cdCVnXZg$jKhvF?wCRp z@~&1R2_w~~_CCAx*CaUdMM-cO8L3i?rsoxI{$!Smh-8G5I0-xd~bX4gLDr*1fJ^<8bGI79Jr zoFik|0^~i{{n+bj{S<5brs0c&KF^+I%1nF4#vYTM$X`^?iKhL&jL%H@67HTp9tgk7(jx2>CCVBay6+@SV#ClRt2J~dW-Ao_)ZR^sS~T=j zFpgLYMJ`Ye;iZR}=zQ_iR3|BnZ}&HV1OofCEyh_a1$xr*e|?Mkdi z#X*!y>Jo|%elWkyJarwO8#}($thWoryyv1Bv%-j|F`rNGsuuLq-lEJqy zE&wA87VqEXm`GGdGcBP?E)`q5>o1W%mBERJp)Y?J5tL@Vu|Mef#qGKEM5}nO-hW^a zruUB>)^MGqIyF=%CVvMx6X)Cq^4Oryy-ak%wtAo;gN-3!l$d zQy+RHuxcVfxaf&HJ=6(YVFUNbmPO;` zSv@V2$;d~QF~E+e!WgKM;q!AU#T5QZR*s?ekcRXdrBq^`qqHHE&*dm8N`r5)NNikmiFkIxN3wv?V7gNE`(mGz_XZ6rRPF$gox8buzLN$_;IiA7UgO ztNxipgG}M@?h%Az0Ufl~af!?~*49~+( zSf+s?m`WwN`zZ@WdP4X3zq_5+HV^V~%~YJUA8FnV*Zc^vTI7a=4pYHzuEEIEKdc`2 zr~uOq7tbWc>DiwuD=F7j_S0FrH(T)5eAEzINt&SRP5Hy*4OY&K&a&b+Jn&vUmzZFS z5N?`Y?RM!L5n~Ilrev1%3eoB-RE)LZOxep}6otPCt|dl`LitxqLQs(s_t3}^Y80E| zQjEh*!8Y-Id4dLtx&~}9H#A=NKAhh*i)X3~HgYuC*8>d2);TA+kxTI77C0U6<`?9@ zw;dl$j{VFLEI$FNe3j!!y2>OA*FR5@Qqhub%uwZ@%p`IVeLU>)J#ij)Xh}HF^1`wQ zm20Vl_66Rr$&cfUmvPw*>2jvq2rhy%UQLrL??a@37K_WM#nmrN+zX)We2?AA>E0jC zd^Y3GBIYyY20IrU)cHxi2PoOf`+#4Rc5f410U#^TNM&Z5*Kc!dd$oz@a6q+R^Aq#Gf6k+=2;?`tT2zc zj&K@cD1%F}@z0=I?3B3?)#|bnDvZDRh9*!5KYnHMP9~*AYWBZVoi?4XW=8H#M4Vnm zk?{Ctv-=N%=foFPeCMqzkC@OgO9tLB8gR*iS3^%s6ALBD9m7Ju_-LeA@Y^-h;fb6| zg9&i{ph6h`ojKf3S0!_!mfz z_}dYf*s5i@uGjD3iF7zz76#YwOD#o(4*8g_TMh+GSEW7A79ru+mz&Fik%v@PAlm^0aTQ`gwcN{VTl^-frPu8cf}zIL<45MH+z5@oGGd%DmI5RdG#|4tN|nPjannkjQP>`&INNx z-$)E3b4&T-L$!r92|dd=^VmLY=01$eYdu=0h$yd?De@2FZJOTO+}18ccrGSv}H_V z(|j|Vj~>C%G~^f$w|5W`s&M3@!99Or`cS$~k*4$_hGTrE(Rv5Ic9K}x7{!N;h{SVJ zi$s)Nks5JYfv6+HiZ)mXBZ}cLkhWV?uQPH|Ej$`%@+~gE_tB0K)Jf@We1`qtadBqv zgo+LNpl*y}we;e_NSUtW#wd3W*N=I)E4$jIt8n3wD{xTtt~jHOTn|O{_lB7ujZk=} z>yH0nESQ2DhW>fl#ZV}Q$Ke)Qt2?g$kH3j&#D|#{KXcXf}%@7$D z#eH#h3SVbz73s;mgBiC^zEU^yk})J&epaZ%j<-s5%F_ent8d>Q3vC=pMpI+o)*x?V}yZ`ppeBtMyzy zhh{KN%X|C8{<@I*DfaLhhp)shJ%-Pn$BuaJh$@axiy+8wO*@7E6zI1Aoohk@y+>I1 zz(Zx}0yy{h%*#c#)~zrOsU7Ec>m4Rg9T~AxQLPBFE70r{tnPZ5h*%*(Z|NFCF}*=c zb}t}glOq~y&CE*VU7NA7z<3zB2xT6ugI=h@=4BoxA^f}=>^Dqg>-?@YiE$S-y6Yj= zkN-2|TLg;T=|gX<3pQaD4CqwE6DYqpDjbQrtoYVw{#xH-(0?_0c*IG*DfR(x>NQ=d870+}>JPe|jy{{1w?%OCW?OV9tqN^I<@w{*8Q_~IBA3nP zaxxdpC2NdKG&NfJ7Pfbi$aBSXM&u_)#FcSL^KjSo`hRToJl@7xcCmT|N_=`FCu`4@ z>5=`doe>RXht3fsil(GC0a(G(3qAuCEn&8%y&l@s~VlXmLl#-b#MP4S8cwO!^hrvsO@fBnB5BDVgBW2Y3bR^Jn3B zqaHO!JZtOnN#R89(rptZ`IBePzk&e7xXPj1-I>8b3~yEbaD{=m;Rs}E6n5>EWSAXw z7Lv;=m&e=7<57V$_&wc54}Lu4_&|_U#{GSBjrFP6-4fNk(bw5E?uoW^Vcl5jyad5T zGWvwe1lH7MZ=ePpB*{f0felovHBeU_}greH+%PsfSj2a!xSM3vfh!Nb^}sW&#IO^QKAO)F*^=kF?c`DvaJ;Su#3rgdtSw-dk0=9^Y@(g_A|2@zrF~k5EPE%?mM|on>-wiIVU+C&ovFfR*z)ZjPWaDfZ zQ5AOhNTSavo?uOv{id?ROk_29na4o1v+Lk?_Y=P#HF<_g6j;0yAJnGxFKkrhQDbDnlP1-qH)aw{JIuQ= zPxXw{rIt4_1kNo4hYHhA+e#;cSmmhh*?;J1fP>i^+yb zG6y!~MCxN=b4?;Qp7&6P)+)fV_6tIE14xbwj+@gjB~I%pvlerS=|i8Y#Y!Lcq4?qf z&E5t5SPU96e@kiqBns?b>qKEYTxi2QI~0DChM=C(5hnH>_{Z(i=`5Hmx*4uN*eCh& zEazdtwGNp+8kJ0RH)j zWdC_zWEXoQ6s3@}=sUe_mTJC|9Hgu>rA;;;LO)$WhZpun$$Ql-MdFS%Pua!}8~iwk zYW4yne=)+JOxHD3^C{_?PR`HX`H!nc9=B~4&LPZ1!gEOsNigeWfG)qBgT!Qcv5JD5 zyZ!xCW64r86^>llR$e*624-4D=qeg>TfmLh8mYHJP3yZX(oD)y&v-;_BnU{>dC0M# z4?2~t%73F2BYSo!PNn&y!spaeJePpQ>a6dPySVbPBR+CL3d~>=x@~F7xEUe4O-I*d z-tchr&f~eB+>_&pKgjPJ2*bn%d!luS*{eW$$~up5ITdZwEJX6|hCpL!4p|3vRap&n!YgV;Oy564Yl$1M^&&f?djVEoN{-Tu^$E)_mk{{ zkNTo3Rw5Z!UYH~M7UeX_nafPq(#tL99W8;N)zRD8V1g0Jn|X~Ddg;XWPO|6Gei<>6 zUU4?8pq*VT+$|ZcU1#vHg`;8%L!X0b)^S4>jkqX*5DJ6idvM5-dvIrYX(cmM=GqT{ z$;MJoURN79h+pV!upavn-Rgu<^k>&5U!quj4wrQ1ks-y&I=0LtZZ(ofuhLDFM{CZx zRUS&9?&bzbkFp4>zWgZB_5YEuIK@7ex$4!awb>tp#R4y*Y^jL6-apqab zYwkK79$P?{)u+^myirgg8W%+`^N$})j;Ib(<+Ky^r+@GPfejGa@%e%Zd z-11)V3Xfwh%(3 zd4G((ybt>`pi?ohTW#b!RhlAhf9v)z9Z-uaYNFVC*B<5`wLsEUu$A?x9f7|F#L>C! z8U)Xbj?R!Gyq*K8IHk}$dXb^@ulrV9*50!KCP+wlZ~+rMy=*sU$!jC8thTX>b6yb{@-kGtdq@(ZRU#5Z46=d7k|S-Y;w3r_D~aA= zAK)cmVgR3Au*rG)b8;S&rSJQtKiHb8dDr|)Rp@REx?8~qvbDUCwRpwQ{I5-t4~6bp zNn;O&Fx5N}hYXVvoeVGe*#fIZJsULFwk#z7jHR(Ep^E< zyD!GD2y6ZPmi8tW2}%ik?t&8v>t z<%s`m!%5d3JNUQuw@*}e1EF?luDn^P1wygHNI|{q#GJj)wyu-1JZSvad?BhsdIy|x zLSU59ymj4uH;l~e?e-+LosFou31Z02Clt}GBT+}}tsu~UuF$^OvhWWFuyjE#nDIne z4=NoOEY*fA{C)$Gg|R$-s&?Y%I4LRJgWjMDtsNwU)__wF464CcV=geHno|v!5|g~L zsdSb6T64S#%lEgOCg{ZdDsqeS=+g2KY7!MxY0x?{5Bpp-{Fchm7hO!^GL9Zq(yP)Y z0z&Yf>xjYENt~QX_ri8XaY5(_;Jc}&Q!oO~&n%J>j>%&KcEhmfwh7F&a@_f*y)|+( z)V)cY>(27|F-hKNL`M4mWe|KKl41JIXjjOkV_>8-DNE7W@asl15Y^Lp-wLb*YZ$64 zlN_OOf~{>v&jhjE4aJGBBFDl_tgvFi9w>&#xn0hbkS98|0+D?y*3>Bq_SVWeYx}LK zSfG|^%kW)zA?L0b)mFihQ2K=_AlH90mPBfV$#whoYK~V{J?CrTD3=_(UmHSCm3un< zzNz>gag8qokgK{01xqP({ey|y@Ls@9MvlAEuN{)f8L8jwbCu`RRmiFjCZJ^TKMbaW z@&P}!K2Xk?rMZF07`n^e(!v%q$hnOiJV0JiR@Lx>tm)0~{XqlAft z7xVtCp0`oO){b;dkr7&FyxVT;Eig3_BK|imM9A8dVTa0!tfOp$(#p=rP1K!Vx&&W$M36cu#XA?z{^vkMikmH5L~fegvC`AJ ziATmZL8eEpv4j;Jtqh+TK~!OW%VB_r`WA?yfKv^674!8(Nixx`Nq!zw8wh8oT|)hS z3%SjACpefnQ!l45#(8XQ;vbHUL~Ae|CPKp5h4`X|tLO`2R!jmWO6~18TzWTZ(K*ouox^Vq+PZL7!5l9W!U}|>#k@2|9h-<&nmJmtUCvHUtP#HgnAf=j4O_Bj%7Klp zbAWWwVW@1yHD^cYW-!BdYFyP$X5y%QYaVV)>+T!(KHp#A`l=PEv%N-CJ7l)UnH1>x zeiW(Byo_0dTS!u^P}eYOp89^1vXM^=o~<5ZtjIBt64>kyFYgSKO}pXktS=l9)hxgI=%CR&dsADZ=SUxryaT~Q#1HPbFm z^xHd^qwaNyw_lS#-qUIda1i$2*2eJrKye*1lf1=^4vX3>0 zg{I61DoRBYIz;~Nxu73;1w{OtZ(VCWa3FA4A=kJlK4gNzM)RMIW<3<5=~>ouTLooC zqt5p9tyhbY*Zq)G!>au2kTf1zLkgfp?oStnH+SF%&*&DIn+A`&F?aJb_glIO%-mh4 ze{(1i(n-~2@i8M%(Ppd`Jf~i`rN)BhzR%^8_evz;UPt-l`^kq=>NnGCB&>9N9D$ky zV0D1TJb92T3~gzN(}ozSKldKWk=!nR*wk^GD^J=0R}#{VzL7`?&qTo~Aot0!A#LwC zNrP~06>qX<-78bZ!=&<$G~A*6G_T0+&o>=XiX^Yj2G%ZbE>FyxG5~1H0c(Z;T}BBO z`wTeHW<401^a62-Ig{MNlDG$k(rd9z-;&kJHDeiGD_Day1m!nHndKNdCe{uK` zMkHb>R>FkfrTwby0zm}?=$lo(3Td1Obl*+$*S;g`q3*BZPO`d-}qL6 ze@41h<}qM1me%@?13%ZP8tzlRPvE&CGh~_hjT^gJ!L6R;nv#laSrdv{;=MEp`(5>K zQ>6xGG>Aui1fEMRZ->&Ret1`r?bSwX zP}3lV{a(^+mA*!Gsl2ERYA<{535m zsL;a#*4C<~NXhtNJka=71Io`kzr`?L&DvVpx({rq-);?H3sPviBnQ+4VeDuwrqw^+ z&h=FHMx;$R$yB+esJnK>R zB$sq|SJ6YpVzP6b!P4>mXW;SNIf7~txuM_@X(xbdFDDQ0rH&Yr35QwEJG|A6U2lR9 z7y-ctY<~&?Y}5WM9a9+dnp4!an+f{)7zbhDYPft*a;Q)rN52|<>%Gz}JVkkea%?9e zk}H9RH_qJqP>kyJAh*lfK*>EK_!KXpmSkPXwWCi-0p6E@iAQh_()1YrEEKV=;?K-I zXN6C7pLf&25Qi6Jewc4+5e3FFo{Ck1XB!KG!lJJKa^%x#~!e^dTgGIo=$vm34hy zs89j-6$xb98Dp$&1^vW;e|96<^xqBhZ_FjiblShyaffrF$nah*%kQWD2fxqeMlBLG zPBJt-j4H<1OutI#=+EDJBPg{)(Tj5uw3Ua(Y0Qi`iI^a}NmgP&$=7FwBXi$lbn)Pc zQ^u?gW>rt?vn_ga9$pSk@CWR=HjrJ-n1HT zD6qi0-n}O0r9+NwXQ6ib)K~hvsW@!k-QGe!0_1+`*Hv&Ln6%4qoJ~2YluyBS&n z0uMHsNnDm;V-4n8jixOKo-U8l@gP}BWrRO0^jPo0XeW{oDfhdN9;1u9*;emiYTt4*H|6^ztpJ4gPT^!wT`!==h;45z8GN8NjDfk(##y^W}6 zPY*p9qB*72EgAHY`!}dEM%-odGLJ`gge2y1tP)swg_Hi`20r6y%R z*~?Ut?G3NWVo+5BTK4Ff8WdTaD`jbu8#;Uf%8|It(07M4217{A!$P}vlzeaemS1T% z+6su!XtQDvoC*|F{xb9To5gOzd7?z!FjWxPa7im+lORKws8ylldz@I*@9ZwO!K(5D z;ONM^pVN-IHS3-C>{c@5dUjKd>YqX1nwM%KoQtDwP6h=B1t(Zy^p_`UreCh+Jc z;pJECho^82bn;@?gdl8;g)9MHm_`y{^*~`HxdYq~ZILAWPovDJpS~|(VImf~z_&jj zT4i@KZbbFNw2D#54Vrmd&f*MV{mjvw+eGdPb#SClgq$QW zw-QKLuytP)9Ik`z3~{XH#M+n4IJpoT>0swKX@jA0d1J+kJ&F5{ZG@jz1T3O8=lfb+ zW=(pA)l~cCCkHK(Va?b`#TvulFG17S*OK1O(+kV9=ZZ(8>r4?ThJl?)gub0d;-QCa~MIW)C+s!NbhICMjqm^ktnA)Ld-5c0( z{w|htREACgr zctS)O6xS3w(_X=D;rX5<-@$1y9LZtzOY`GuJd+toEW%-86B}ixJuUE4d4R=_iy$8x z9>}rxi=w7?5}|kU`Q2I~>t+@2@ZywtVh`2jWqw{|rpQp&3k6T93(`gdY-g-i!$mf! z3l#0`!*m4KIO;x3KV;f@ny#fwj$q;lFLo}Ib$=Wm&x8YR&ObxnwMxp#NA;hV#*nY) z6~|P%kp6ImUIg)h&A%G9+_1Ii11};HI_YHMKEl@dyTU1*A?E3ElHH4ds+?pX+@7`3 z<$R`7VV-;WO&zRu37mFXDZQx1s#F%4eVttH`z%PhMj1dk0@+-=fr z3O%6fih!wl!ae=SXu*?ueYH-n6Uxkhsq#y5=p9^9=|f?c;ms!Lv@aSYBItDuOSZ@0 z{R4^Qymo1lj-qf)@H+6#U?XnC+d6?Ihe>-R)jIpD3G2%r9Ldmyp0DpdHVCt5JCC#B zj9RtJ`_xA0`PF~XIZx#{ehD08gPuaGblL;z%|PCNIS;$&c!#Kb8{o`#_uggu`ZMQB|fxP;e!}K*KIA?tA(~5*sBK8^5r>l6}K{evD@(`9dZyMPgO?e=mHDwk%SbU?nC_bvTmWc@`zPc~R+LJ0Z&bQ%Vh z2-&rQQIa{bM8<=$1Ge3mbjRD}6>{52%SxtawaV6T^g_$H zq;mgTj0t-pu9f3QF$+9;?WX)c7+dfd4lg52yH>UlBbOt(@q!c`tX2D}WN+9jg+b)9 z7yUj)f^S0=-qloWA8&R}wY{uf885BS@pxIG6ihCks2Z?lMi!ns&O>8qh(qnMZN8qE zzMig9p_>_XPG{;Af$IsbH`LWM1UJZ5mSHRGDRjaYTLhv?bqV$VRe#mC#d}1o#X4^{ zl}e_xA<>Z0z|^Sc(tmDymyjp)`Sv$D6mTu9}gHG9JXQwk}-d$s)ps* zjVwekV&}!wsPW^CcuKDul`k&q@ZH(dA73$i#eNM2bs?eatDpb9%PUGg3^a|<0Q=c? zb2#Gk|d_iD7;*3Pgqhkr&*Xb{Wfy$_(t9!aKyl9`B}Gc;u)rm>h#Ta ztZDB2<+&NWTt)#(({ykiQw6yegv#VO%A7io__M|Qka_ue=7m09QIOEly>}%s~ zr?7)M9hb1}0#$3DWICr+2cspS`czwmC+Z6e^!*?YK77p@J8?k%_Bt$Z_+}<}UXqW) ztZzD&1+#gJ|76J;^IOswOT8y7zFmA*tMp3@TRn?mB`Z)2?AvKd7MF~YuWwgxr z?U{|UmxF9C0byuG$DkMfjBFy=n5bM5w9UR431l*SeG;gbSO~CJP*#Wh&=1<#smfJ0 z*$Q@{f*~NtuJjwb^Zg&8UvEWzk@bHV7#$$;b_Bt3R*Kr~N$&)L82!R$8w#ae*=8tF zZ5rZY_=lJsA4?GqeJ%ap;Ex-YLyT|HoUAPN&h?7|$V^ zhg%Pbk-MLfTeZpV^vI+JV>dF^Ua2x%1S~khx`aUC&MdxYjj4Cd1eaY)PW+2WY{CZy z@FqW*B+8D>PiyfP6{T;AJ;P&q^g2{Yd!pYL4&>0h2SQJqRt{sc?DntRg7=`rW!WUL zc15Nd<1e}bLO)_2j{bw2`z2qW_t#wD%sEOs+vtD85Dbl~P#Yu>K2sZ!lPl^J*p^zo zbcRoT_Wlk}V17I~ju)=*tyG!0xchoKyEbbgl$w<}_1rokBqu5iE;*P3F+LEy9Rde* zTvX-%T_*ar5OTFfb9My+)sBEKEY5BAC%SU>#^f4SRssK|r|0v>Ae^8pKsm*itkAkdXp3?c z;?G%=$Y4cRl@*5=M8z=BwY(o`$DAtS=RvFR zQ?TOt`ijFnXO;RU7#;DMt8dS5RO4u6S57$MWe{v;2 z&CBGMe^Dm6mRso+OIk6iLdI&%WE)~k-_&>YLphhMxV@AnH~%Hv_pg_=<_|5;2!4mV z#+_6b&c+cT$Tlp$%^ayoD#j;;L?KzT#(Rym-9lMr( z6s>$|d+hJ-)O5_)kh4fSDEd`7M$wCK*yQw4Bx@3}=p=kx{uY+_IUM5uMqzMxM27`c z7kn>cqWbW*U@;(UM6E0@cac}?lbp$)k8(3Pl4KSfb%Cq=h`E$tK7|JN0!~pY$j0?3 zu+b_SULjxvm)%HjW>i+tL*_kp>ew7zkpCy)tw@c(R)RUX;qbqk!@ruQ3G17>qlbo# z%xsFi`gBQVb9zx2&wDrK+Nw^R${NK*T(Nf!rjZM4uDxy&t3v(dz&fRC8j9VHlfIv> zKQ6RFysC3_&kV`HhsIPMPkc>jaKA<4<=9l?D%Nl`@NO+MJ zniVfUSl99{q(eOXX>cch6#u{7Js3(D3EG@^&*qY4g(R;lM4~EvwRrWP`Kl{wVE!QA)%Ni+h6G~fU6uG^K-=?wqd>aThNE62 z#jYA*->&)a|IeY@o>DeF*IEVIz^6?0>g&m&U+-B*bd? zV0kBU4=cAO1R`DV$J`*n;csa(kxC@miDJlM${89fZi_YS8B$((3qv*x1&Lm z?RMVQa*ZEy|a*|62QP)E062p@m0EYDw%^^>pgm(X{3t0e;!R z(8)quHMbO=(=5n@@V^DC)-Bp6G86@u#muYDkO~O+O+&6cmR5y>CcCe!At_A9;*|$- zlUZ`Ww|n7CeFH%fbk7&4#o+Dy-(gZ7!Ud95n)*vwBtO9a&uh^DH!KJpaG59G&n>t* zR&b@>iVwCQxT+3l_$`x8;LTEXnLRovrD_6-CpYv}`lAQieiS!mYQV9VmZGr6ol#9~ zDs%cNjgr;U`9$EW>%FcvX8KgEeShsqO@VOU%oZSibFI&O_uz`nmYu)GmXp5M<>U_{ zPGVe`TWHT8TxMWq^&dCRWhI@}(5uqRrqEZahuV=M&1Hr|zl`YDIt^_ZuWmZA)~a|0 zO}js7EpD>(Gn+Na@%QL3@2Ri5mummaiC#LtEVhoPv^R};kAb_n@#*>iZQ0GIZ*?_O zxB8m#nUR~2qH|wom?J8I!}k@VS!V@L-4+ejsE_hPSAEt|dohb#oc91RK+eDXkIhwU z4s#XJ5(#{(=6amp-~$TNvh0xMn4SzNEm?{?%_pDEla2-M)ry^`VcYcTP3gs5tRk?Y znUOy+LpY7Kbv$;_D_!|KV&_evB22}VlsR-bgCQMSexc)yQEWMj;aqsJxgePY%U0{g zHspNiGaibWvHVDmf!Nme31Zo{3Y%`q7KWc;vs949f~fGFbnfAGiYJz*H@Rsu(~*&5 zs9B5(seMb##Vp6?MJZY>C!0E1a^Ed_^;BytGlPf69L`(x<@Kr}<9UDKH-+c*#A{*5tiL9p86o6C<^lqLxME0k4h(-aoqYvJAf zs!6ZIDPu~3%XLrfz}t=??qJ&()oXHc|Nh?%K^6MDiYf>J7aLr^P~`?h;WZ+ykH@lS zRrN~su2mTbUKL;OJ`Uw5i71mH>J1(X=4vbt)jx?R#o1kQel?@Kwbc6MM%!vamrst4 zIlh7>{=V5^>Q*c)q;w7jIuz^G<`gElGW_x!ysJ5cH$FOgs*{gEV0wa;k&i=^YJoOi@v%0`2F~Q2MTo8!$M-r ziuy1vKrYkEHpZTYX|=O#Q$DM!8Ct5WJv{SK{?794LX~h11qcCG*d^t6ykQGY;v@Nt zTQS-pg+HiQdqF4uY)m14jyL&znVFu(^W64{hfy}oc?3Nr!4wQTYa6N)%MZ>@3cQ~z z2>FV!{{k28r;wJe6WO!n!y(UQ<+9$-s@*VFu1(D=L!es9l+-Eew<6c$?upi|09Ax- z8fWEWO=8RLf{o>H7ptHwu>xdtX8dAPY-zgOLP~2ZW97`U=C48Q@tT2js(*uxBR*|9 zo$lf*%Eer7lysWyoae6rw{s>3X>4q)@h#269?hDp<-AC#_%G7GU2-HQ0~0Jzxs;S|Ka7`Suez40QSTn2 z+Mxd$rf?s|I56%DOE#B!KHi%H(+r`ajDt-Vuz{3f9d&Q*J`+1+JYoS8oPs#h>`&|W zptU_1^;9lA)-sO@>LEnp3tJ4}uIA!#;n8MKUc@v9Gsf!^>hjx+-o5|-0C`MV{5nP* z+Qe*!q;1_wm#ophXAWFM=L3Ec#v8aRg1t3{Rwn`eu zDaaftO!d2&ek4~Jg5=46B&73ry`HzpW!~V<8?nt_y7!Z)teRQ)5SB4I^6btwYBYsr zffmi&M|8ct7eLmhpAs&9zeQzCY~D0j(5ShXDc4%NMyQ_tQhf1 zn9oKmkKc1rA|>Jq_7az!CNczF^~A1>I9Nv$cD;Eq5fd}ZoEF`6^%H(KK&PtL{4vCu z8(%ITh!ip+D;N-!@FXPnhhwq&Gy}qgCT4uSW+N`RXXAW9P$^%EYjq(c#+pEuMkbeL zRdKS&g(heA!Pf$b(jmlOBufmh0C!}oz~b#QTA@I>C6a-xg|#$z`*aFh zm4DGTZphgg$}f{C!@O%z(l*Y4I)O#na&jAkOK5(So6emOb@TJn8JlD-bMy%!tu)rO z?Vs5Z%e;NVu`!trs^K~c4LtbKc)Dd|b0)1gOatU(c41XmAq}ZrNAk^-4lU;YuhzB5 zmBRh-(DRNFHfd+|`EbpL5R3y&lHZcHoHi>dr4rOpN%J|B@9AV02r{~mX&H0HYEXIH zMF`lHq0%1JQ|{?-!|a03uH!m>T2|Oc+#q6f_ss_hApI|hSoL^{65TfSZSe{b}mH2V_&O2`aY1+@UyZEJlQVZf51IP^71mI)p?auy> z(rk3dk9901dv+PM2JahIo2UgDbAgtzv|#7x(z618SZ-B61Yj&Uds@n3s`2v6(rR2z z4=NmoMClL`Y#u9PIzzNY1X}@CR-amV*h! z4P`9*GS9bmjM+Fl_hFnQt`Yp+hv^X&xZ}usax5Idm>G^0fv89uE%{{w3x7(%X@yGm zA>9uZe=HkEf}EGXTXkAKjsf2h3A)A;R=lF5SR?e-fmV*_kc-giSTs~zsb75BR7O;I zx{jAh(i_Zf(;&*P6)bvDGg1|c6?<9tiaz2MBOs&B6jt_3n-4IP(o1`QI3aTYe)iGgj0mUk{| zK21~p#y>2e1_`^tJYYY;CSIMTO)uGdg{rb57bm6#mrWa;@+3K8Y#Uy*Vge_y=`jY_1kiX zc8{H)tgqq+%7MoY(IKd(dm*` z>SxP6{4)GNnQU%pJS$=DWdz07;sH*^C>|{8f&Z8iKCF7%-B_z1PH8q@vmD^Rt*1IzILL}o4r7( z*ErDmm8yUTb$(6$$+IIj$YYtV{OkYJ)D7FKI%BB#KR_`QwoE9MNt63reT6Y{IF@-a zDMP2MzlW0ER}$*6W~Wne{=Zx^_-Ri2mTx~oSQDf~A&|~6G1S6-;b?v~nUZ)s#2QZV zKne7Xn>C;lY}N4U81yC_M%4kwhzKe=T6Mdlo(>4=hH_%D1{|Ay*;CxK;Z4S zhlc-{i|a8h?Xx9nr>{$v#r3igW}isnwEt(uM3eE$N;mZ`*t@Ft#~oET{FLX1`PrG^ z`*Ys90)flRm;{>4b`KRk_x$u_@hI+PQk&wU7X7bYQsVe}%NWSnPmjIXwJ;|U)VY@= z%3JpG!-vJddD&s9O2-yD`XNKsJ2yGb=0`lx*MK^cFVigWB;`cI>iF#XYua29aP zVda}ligFR3p(u0&YeeNyy9Ue&(z<)|t*x(B-F4a2Joe z1KC?=0j(te@j+bWwMB$Lk+ww^?zJLZc%W4{=w5T3fA){OS%&%)1xrIQuUGW zal&)4>V%sOjY70m^sx9g@%LHfpe*)F`VpB=ppq3_KvZtV=bzd{7$fu&-X&D)Eo0Q4gKaUg z%NZyp^9 z`#@P@Wy33q06D>BM$FFBJcuivqQFDZ<{~uOL$kThdFal-DG%%@?;5CnizQGSX ze3>PW>kn7z-5GPjiUbf+%TtP_-r)O!t1H6cZ!1FnR991+s;rzmW^mF8R<;^A%&|n1 z+h?VtvjW2s18Tp?JHPCPt4SZB?kNLktqKKTW@DMd!Od@eOIHoO`>6;)J!-`KgROe4 zE4Imt%$l9ZX~tq1vtiauIw_)uCUD0<)T30?%{s{1f+5(V^IEd-mBKJ|FgAFUuJFO{ zf+cm+J|FlA!lKYR)5&yS!gy@^#1BZ36(2RQ1metKX;Z%2gn=yi-ARV0x$QuWi(|zK zCK)C?MwSx1O}UPnpPSVdTOiMcwYGq9r5rGg$>kqd*D*EK){^6-ch6EW!diEv;MK#! zRKZ;X83-C;-pnmapQ<-lmmxvZmOnVQ*qM-aBv8G{=G1!0X_R{k=WFB{hp|j>@3U}~ z@HxN)G?c5$Xg4=c6fr~S0&_Sb`@z=zJi8C_w`{+wu|K77e;&7^h1*a}dU94`%b)-+ z;A{_-cD;9HgRj$3DoOYoRk)D>HmxbVpO~jBHS&K%?;@1d^OFm<)^5L;o5$xBl#e4P z>RgO{C*}?@G)+*~rAoqKZm2jMWwVmeX1hx96*(DwQAX_pjdtUveEpGwA(~{cI!yg7 zqgd1oYoR#I`vp!Bs8?(Iz5j;UJesLfTLihgPX4Vr_}G{LM0?Ffw=N`WV}@1jwh*@u zt*L(g&fe*jb(79-PxWMit0KOC`{gTi&JfoNb87?Ys$E(Lh_W7*)Q!;5MTT|W!Tw_4 z5#OQ#6RW{4?AxWQXBS@T9$8S^yp9;dV@CmX;u-AIL=`wJ8kb5^1TxrRsv1~|iCm2N%p8o(`ZC7)mAWz%K#=58k zv!cj}PuhZa^#Z@ge7`BgNQBn!tY~SeGiUMODH~J~%vNiU;7t_i_@Gx4-Rsj-aO7I| zwW$)+Zu$XLcj7EW&oH$*sccc6*aeBZ2id$z7>o6WrYgzenTRRxhAL%Pj0X${6obh1 z<$cX*8MuzHahA4XDTOYGzzf3h75n8M)u35GREI7cya@wJO_M*0<lV7z(0SLIW&u zm&Rd-yE~^JW;xRWg#W8v(U642!zMWT^g&jSTin=^-2mQNmH(fMrJtZFxQDQ2Xw`;L zBG&jc$>8D}1SMBo!*|~0(U3VXikf)K;Lwtr%!;*Z2m<*n$wZ@155uEpV}>q<0q+rx+#23k@T2o25zViY8ehS=4APBj7fpmmwq>iVTe3bSxI`G- zP#m67UNW*Z4pz3C9zNtw5dm`Np6DS&F=A-m6)S(>6#XP#3J<}QT#V6t(akq>O#*7p z^x~tBg|5u!iw~#HBCT!wPXUxbe!E8p6&e#ZHmFTfGKPBqg+4%c@4ZL2AxKkA8}^+* z`ZYSmw}yAwajT`hmc`y?x|vKeU-kqPjq}?}Q%l38ov_!cLQ;iJcoj{#qp_w^fz0S-!5Yvv-Av+1N z%Xd+sq#X}@P0aAMAzW+hznBDvDPskPgJ^Q zG`ZvDyIUGv4@laHDIDGAR0_>5>T_9*%1|f<*Ft$@5S6{8JhM8akyEvE0_(MNhx++- zKT@tvJBs`(M6D?-*WM%T`J5~e1p4@CnvBpFV!R&aEG5hsTiRdGVhY4d%sRTfkqCXZ zo0l;y?K&SI#*uGUdh|w5i&eviJ$}yb*oUNhZlvgEWmQB;Dq0Vk&Lj!?3GXWRMwmtl zuSSc$f*cvSl1uk$Y|H$K9$=!KKDlQ!bzrR6DJqx=pfmBkePY;3-_(TXOUr~6u{e-f zX?dK6utT)>iONgo{d704`BpFE9&=(c_r4|&+@D=kad{ra`UWU+S%5Viu-?C;7Ui(3YsA(iL_`S$*1c3oOyeb!A2hrkE zwr7nsN=(D~H9!COjib-uh}`arfIHypF9a~Tw8mcsHAU18=tzqQhU~RWt?z9R> zPYMH3#L9yj98zaq8~I%oN5-5Pp=hq7qIjeAzG!CURwu0e=ym&w!rbVNXCZrr+wyDS zdwr|_$t!;^tPGr>WFU!^QvQui))U@(fzMPG7I`y6lQxlu+#VgY6*_ZF!GlaBedH=B0=I} z_&n}Q%|znR#|O)@x_nPUID)cD=tbdN%Z;S|a&6_t8;a&6iUsXNU6lyiMxkg>7)A6I zqBJdA9xnEkq1$w&p~k`T&k-zQ%Cz~Zn{Z9_10f^D!brnn9AiBS9Q?M$=Ee+CQ0__5 zNRaNsu9jn8yVQ{dX?HKnAjIJKU+)k^d++mC zVXNQb&Ty88JQD{yX8aqgTRp6s!!#k++#Shw*_p(-D%qFei7FnLaw#}#j4`P|f+G`U zs!k82jZlP*i22?=LitchO+tR-#ax0bgeF#vT&+po4J~Uxu81_QQMrKYt}zXIoL>(U zgl03;oma(po%*bp6RyyuAk$eO8*N@^FHsYc<0yK|hGDZ5xGXY#eXF<|*1YE~-l8V+ zxqK~f?SJ{6|Dfe)^QzpqGM!&P&^6hxf@px;nWiDu`OBGm0be%*DKeJ<0 zSL6zpCC=%Uze@i?Ej4YDXbmAp1_j~R4kI7?q4pkpcx1KBR|(4@$t0oaAz}`#x|tO> z))Venijjns7#+f_*)&~ixa~f~5P9q^DEl~NaE19yb~Fsoyv(Xg0i~XG>(qXeIFyHO ztjXcz^ecvxh_s!IDv}+Tfd};s+m2L|?V-Y}(7*`QxS%VaGn$ty7e%YFO1H$`*bLU# z-#4Vq`ColdR(Gy`FhMO%{SdWnB79Q=@g52!7y=BFk8_Kmbz^pZd*6Xa&!YsrfFGxS z=@>fwX3m3q_f2~#N2KrvF!3#%-|C~@E>I=8+*@%mQ;-|&Irpy;QnnSBEcza}@ zR8BgF5M`a=EJ<#SsNE-^c!C@Y4aqy?E%Mh45>&6g!1p^eh(4$56}Q?Lyn`XkK8akU z(3Dg$cUDe!nMKJBr$z)xf)Q!QLPc$QC!XU>IxvrIqr-fyyvCyH>MP1{%(~@Vfr5Ub zZe;X6Tt5`-a-F){X@zf5Ad~Z%BuC`@wm@Vtd7ciyLaFHBDEs})rzkwHMs-(#^9hi{7}1}q;U-#)y_lsb9{dp za=%mCF$Wfn4@puV*n{cBk*>WMsDfs`+cX8D!ei8G)xeDCfuMFoE zd92v?i?-8~9?^jEKmqfW+d-wSwLaoHf&&|-t$(2p25Rl~a~P(L4;*ID zO|Cw%hFo_iQEpqf#j~)8gsA$%ToT3P;ruUAe0x_GR8j&CR-w+i_Wh@iCe}uYB#o$A zvO|`{k>{K0-Y_z99WY31M0v{ru#uu)0v+s??b4WNwI3=CU1177cR#_S*_KTIPZeZ@ z%2Ie+2N7U$I1>`PEY=SFSW<|!?zQ=vXK1zTJLN=`twV-G>9nIAE`Ga@y8~{FC*Dvi z*PUZwSqe9%ttDK{K69ZNB?QxYS?Bncrk@N;vo=zdj+#~W9DDZa$J}Z{WuMuXhh9IA2UdXU(iGhDTOIIb>SYs@poO96H6a~tpPAqey)i_Bd#1#N01sFv zI>AWm)Yi`}HM6So#M&=S(;<25U%xdIRobLaI#!e#BFp1Wu6-cw`kyeVsjn~w>K@lu z6po2Nu|zThOzP*2+Xu{b{TO>-LMshuxYpF#Zq|=<&eX!6rOb+q8Hj zx@+=rM=~?AnEa_ne&>In#ret{J!LyX1y_~=2RGXfc??G$BXz?xwCH*p->y`-IgT8< zHixFDWZCtq8Vlb9RsTu#^&5@_Tg9VmM|NsM|2U$@c8xO}qCEUuH9(DE9k2;Oa=0s3 zm#EF)kwZK;L!UaxG+8ks1$u0!of}cS{()!@Ik}RR=-IVHsV#(jP!-`2?)kgYI?`w` z^>{yGY=&_hcaza^)>P27zOaU!owyPPw z&6Cw{FUlCqJ9tY*>D~;bCUKpNWHDxYu~ND1BNK2A{sekm?nPiMY(?vy`BSdzHaAMp znv6d)L05gem(O2Up`U~}KkBoGM1r3+k?fyei%CLZo@yiQ(kn*#KW%ejLSG_~p5I7| z3uo;sU;flb;jhqN`a|CM1?W&vh^2CQJ|f{Mvt5+nlvHX$$kOok0L*K06M#O&>_`Q^ zNtmkH9B$^aeodgxmu%;J%*lv)C_8zT*dk8MS_`j@Q?qSY0@%hO=7kp{`wW)o!)QGf zFGGJFJ*wKX5ljt3;#ya{fl!~KSfuljR}xo}f#gt$j4hsZm0GzVX&mYD#~jxyZ?{g^ z3@*b#@WY9eSWQbl^(X52POHnY_tDu26R@WkMi{7NiUJNEwW=T3DBz(xxZE}s_~*>Q zkPx%F4En<*#*JZ!mMP(0!as>`DyT$PikF%zHk;lV+T2lm;=ZM83n>@w)6u{U6!Zf(>`aX4yvG`#k(Qv2}MB;#U541-@+SjpO;0xUV5Ta6S z?q-Wxh}WND;Z3fW)Bi*U?#V|MW!{MWXv;%X%-_r|2M$Etq$;D5=-IgCD{`r?#O zH?~))BjH`>`)eH;{ZZ$uSqR)2U98VvM^iEp>08vvv#DTtt3^$})cENej&qnVvx!eN%x7Kui_OO%MV#sI zF8Q!}^>s+IQ75VWK&;7X&*_n>K3SBF&o(3Iup8WwG-JJqy{qW=kdso-U){1d%FcT8 z#t)u)kw~RdYWylgY;`4EhU>DON!N51D`AfC{qdvSe*2$E=P+n^ zcRjTfP5UEeM=bK-9Sul4FD^G59nnt1=M(??&U9x+w zIDA@u*haOM8EVqa1PYeEfq!~X8rJ4HweBFZx~?#zSR(oCgu&bOE2@ckmw$Na0$J<; z^{jEn{Pqow9euaYPprRFJ7Z6n%hu@81^X5<{)9?2N1rB^;!uP@ny((capi|rJUK+v z7&IJOfG)$*BB*7Hk8elVNtg(LEipF~Q%wVT72HjPagAa7Na#%f_CEVe3Bs-uy9gXq z7wyqD!vlE)qO|a#KFWQ$S!4AG^=5)_>+CcspmbaNVp7@zudj}Qy`zw_U+fekF$Sj1@4r^3Qm(aKl1@1IzOCJ<2kof4%ot`x>{V})Vfnc3g$w92-prrIRv*ZYDhz`rK+>Me>8=F9v$oe#T}v_mj< z+SK6Iylcb$X>Vy@b(36z4!j%L&)29?RgRxTCtvPja!ft@;}9}bx|j{9 zPzHsiO^2a+atviO`UzO~)#0!Bl%!B9bz*k$VFzFTtP(x!EM%gExgd`N?GF4MoB(}; zMVpm}N!?x52%mHB(=2tKLVB(g#ojS@ub@T&`+g-s7jD z8%t5Eqz0+*QL(Pg&dFRlLbbiPC0m9;Hl47soFf43(HF-Nak?w~r1d+)yVoQdKEc>nV5bJ0?)*0}Y0W>iH_v zm2T{m5&IzvKGM~F+wruMIx?{@UVRX)#KadLD2CXmo+FIvsvAqAi%&SlV=<_Wda9%o z5KrO4LGF2F2R4m8a0yme@cS%KwT1ypi}|zZ50>ime-qkFN+h8 z0iV-=Z=Ox;VsQYKH0L38xp1i+(i4x=WXQ7WP?l}r4J@+e2pjsXol&jPjZaott&2fZ z5-z%Z-m1;sk$8-RAo4j(mE*;4S|OaLQ;7U!#)D!xGt)b}rJ61y=iCOf*fy=5>0QvanbqTqdAGw$Bl@xIN9&++lL&TV<-AwOi z1OWHcbSmy0TJr*W=cC}?t^ak1Why#H$cmh+BIAd>A!T2JK2zCB99R>sa3qn)3=wryA&1i`e~P!nbHY|9mV+&vkC-u&2_dLj z0V!PkdU#k`zqY2|tnKFQi{|aBDAfMahP`Z*Ikdf@K_Le~X`9VN&BM{1|aGw9vAhr`*n!u&{6s-~>^E@88T~ic*?zs;VOhe7=VUhXoo< zRWpwnc9O+OFs~D{B%rFNaqeuMHsJZ|&x9s>;Ij4JVQcYGxNYe65TJ^Yqewurm}F8i z`MQ_y9czY(z?%3!->&hukPfgC#5{`??M{NWf=V8}eDYEaxzZF%;cK0gHQLF~Fjeu8dv7Rl9zwC?9HG2)s*_d0?F-6?X<-_M6UJtt>kFS$-$G$5K( zFo&Dukrk|B*!9)vYvT6(JQQE1CHU5VPup3`*cqK^wBxZUFb}ciV6Gl%;0TqPz10^R zB+rTICH1HatT+pdt_Wh2^Qv@5a?TFR*f%&auX`k_KzjsSYPB0wjET40L3qq@cq|`k z3|;X1X`RzBW)G#iU**VvsIc^kN7{He4A(!`MTh9|wdrVuKuDP7vaLPSF<+l5KD@~U-Fw_?gA>2Si@5rKdZA>bN%9frtuyBOtXStOAxqcz z&{86gBS~YeUBgs^5A|4(=l#{f8^S+qB-`8FCpnE8{EiqF9L5A637&ynZF6O1e2w7D zNaJC9^d+AiiF;2`1Ny)zw4F8~T`8@Eq#oZB46}i>T{Jg1v7PBNUT_~LNVv=woM@29 z#X~G&RI06>NR1}?+P`kh`^WHuY6F`t8AFtZXe$=F$*;z72M*%q+jx0|ViO4NdD#Sg z_{C8;Ie>&CWiGS=vL7v%Ho<$8ie^ z*=lai2RWjzdQVe>oOfC#d`#~0bJ-{de=!bu1p=TB^<0q;K23?Ea)U6X?u^kkj;`!rhp#g% zB`|I3;po>jDAnd|o7&ned!lu=q)`#|6mmNEvaMl{y;l`>$^Yz75y|*n;p6N=PB9)_ zYCcQ?>26Co(=S zC!~!YxJYj~?-ZeLY-DSiIiMYi)Y>b3=n)6w|G%&v_I*8H;jdv-Y;TAiDk8zf;@7DM zGomKD3sie*Fn4B}^kbInWAhnb z?lihxC9;Ojeh&H@b!VQB#eaL4$dS{L&!kGkmDeq2W9SHC2;#+RrlK7Q%Eug?pmkym zvDJFxxEBGBG57u{nz*I|?1wb0V1F zOv}i7&e_s46WfPXwF2n82$A%N0#hEQ*dR15kZrK4sAfrS2~AgbSW#G5Po4C5!tNvY zi63RvnX~Tq!4}~%NRN`9OsFzR?2^s31m0UEqb=wJb|>hO?~><9yTZ^*pL=cpL}#rR z_ImtK8d9saS@m)ZdA{7p6`R`;h2Y@WL9K&Z{lk5ZTFAo152{nE$&sIXR1 zq5L+vGaw}JHNE;DebtL{{S2$|Cu!IHS!%J)vJs>|uN3^cvSZw%rKKV38J3=^l{KV< zBq;oRs7QnMS`4H_!HheW?K+STMP<+J_#%1~#3`1IU_p?dBag}J3bfWS4oGgOutx^c zq87r|v0p4n{SYDU$U~}b_VwhfehEt4zcC;py~Itl8wtUvdqcrw{Nk{K=$XxJ?k||) zoFdKC>!Y;rRdp>3(&J zwyX*D>cdWxU%gFbj&a|!jY%>UF}if#ULt(v_QWY%pF9j==Ri-&ku{$}ww;YMcdDkD zDkYj+zoEiVD{?y;M*F^r#rLpBxK;EBe6@E)+B%(~SJe_FR$~%X;W?RO&j()Qt3~7n z456mw zrD%Tk!VQn(j%YNE7>e2lXlziia?{$+2-DwLp6@QvIyruyd2BgUw3a}*Of1j2tfRDZ zpca~_bhnCo3Z<>}qQaFxSN&iJ4DhZTSG9$sC&{qyxl1o=|38VZ2Q7xUZOnNu3_GA| zUQwD|tA(96KWUI;b$g5nRhdVt5wW!tQkoVD8_NZko)*D{`~EFe(ZtRg9G?dow1vCU zn#3?dXD7%;@7@A9CZUj23K*$f6XJquo2)VO>z~Hon(NckE8+L&V`$K%iT4{E0&%0I z-i?nDR5X}+@~c#*qfgfpzHm}<7#2NbOfR=b<&F?PH)mg9W`n^o*N#4b;C zW_8#5d^dwrv8lOP+g_Xd{Irs(yI3X{e%L90z0Z(QhZ8PE5MShUDHKCWN0pN`80a0M z$nmW|apTaDSpwD%Bh8piL=xsp?1>LCdS=P~vwbe=L}DLf!T%t<6zNWc?Wx5KcXESAtG0gcvgUpC`&6QKMWJ@ctkcSH$_q*Bt}QSng7!Bzw4@7*6@49RT}PMPwh@RAsrdU|<$g17NPSU=sxCTF zJh|W}V~Gpd7`@E1v`^C5?{ywM#{!3{)m{*+)oElKneLOj%#1YFl^301ae_Nnd2p)4 zwqhDqp_9$kDgx2=MZP^rUkSyZFZ+ERg@0ikIcRj)bKgWFMy%oC*W3a z+{fjkHC!|K)34_VWIEyBjwux>hZWmG9Z$b<)2(Ky(8fI6So5^E{I@_q*I+rSc(d~I z<<9+K!eC&^T~!-_qABKEsbdmP8%x{G1Cq@7%8R8qAcpH&$C+hYMF)0@S6A>LaB;WSux% zfdhZ7_zT<2JpqZ$swtZPa%;DhXG!FDD**q@oJtmv?s-Q0zq^Qfe@Tq6UThaj$-o9V zb`4r$QK9;nhYB4-=lK0_4nWOvpVXvGR!Z_N(eja$t8+JEuq_qsre6aXFEM%;va`w0 zQp`scHn>aqMxGyByzU+2b{b_#@n<%t!KI?Yrka2b9UzSs0{cOZOadykgMRi=&B~@U3N~myQ79C z*^EtsFfNxmy!X1eDmpsacb~pKJAriIu^kIBHr^~yNMF65 zg?DsFu@E|(f=R&!*N*2KB0ccam%0wIZWfvs08?>BZ@*x(R?wOLq(d=MEgpz=dQjv# zRdzJQs2hv^jO6O>9STFnQM#vc_DKTy zChE#I-PbwNJ4X)AR_);MZSLKe*j0}eWxL#QS#$25HVKXXcosoG(N#3sFnoEP(`(3U zQUWVU9B_(j9V;x0m2?WAINyvz3%PM?Od9-jdpt9G>dft)M>y+rZ8;pAFnJscOPiA$ zrv$%cA0@Va$oMOF<++WTElIESTm{^);-3D_`>kNU`f)(P9FA1TLx zCx8;Flbnl_%uX_114^z3ziDwaj0+qY)(!nxIX9gruR~%XF7R6n@Pm&eB{ySnm5%}j zi2mBPIt{Ai;yqX2rItL?!L6cdZ~mIj6n`I2R!w+2xFL4EJ@~$R+Bd;)U8A5YdN81IaBHUA zYoBFFu4B~)xyn{*y80HMq;N2%l{xp+U18HC1~n%7Yva(kt?8#uZuVM>QGoALb8I<+ zAl=}2reVWa66j7TkRItmSA`TJuhR9RYi z6xnFsSVmVY>W}?-Sh?5@S_oDOjcTaq`1j@GM{6k5P+Fy8q@!j!BS8Z*-66)(+!)uA z**eQy(aS8&k|=D6TAtXgP%EK~RiLnloAX_R1DXeMMwVKb5~*9~XE;)``Q ziDZySvA=tXN0CM=OelYOhPL)e$L)sE5g~+&PVFX0pM1Fl!3Y%`kLIw;&?5y0CY#_T1X8ZeL5-R(QeV4)K>YN%@Th^VBOwHr#aTn zjvIG;Lwkr=cyiM(#g26@XzqDnEBC^Jj3Ca^jb@Uwrz2-z#ho3=?s2V9tquTsgNPGc z16*yT64jG*5crwT!$Ra2Lt?rCH{722c_xJ_!JLYkS7XqVc3~WHqnKpEDpuqE5{U*G z#akh(;7cXe{gdCvZ58uPZoig<-Jdw9{2e#b#pZJNkEfeH*OKI@nSvff`f$6patv$b zvWjI48MCxz2+SSp86?Ioo;WB7**>2<%zdn5Npdg2(4tIt%J4`TZ2U`MaaS+TnJvqq`6<^g}9&E!nDTDcE6E(P$@tX1 zR!Hmv9vE7n#JSbua%_Vi)vu)nd)(_#q$Bg}#biAO8`sJ5CxlF|qWBPl-c1Bt?|L2F z3`nuAkm%Px&bvq|A2*Ao+ttjo_|eN>(pHU9GGQWlp;}DDG!ts>ym14C`KvrhY17Zyj2FA_noyQjV?^XS22u z>XJ3rzay-Ebl`7JIbvz4FpYwvcZ+5Tb97S1nMm#qF?+$rNk0Wk%S@-GiWm_+j97e@GGI>EY)rI#GUpa-f%&W)RxkOgh7QPWHO+>PtMI zhPK(Y@ld;6AL>$c4+4V&vBn8!epA@R3q*nUi`go2TUVsrp*D9Hrj8#V?wDYDPM87t zS_wub34@9#eqp?4HzbE3RSmB7gN<8l|78+1VSzFcE*4CSI?`V!Q{m8I(uLAVv$F@0 zWNRwl>mAE(r25XWTbB8fvZ3wOnHx@hLCZ)CMO@M%R?8EL`bzUzZ@|dXFFzgnAZ~Ut z7G~sFuf_vHsWM3xGP$YCXyT)>e0V`@SwyAwc=+uTLCQm1L@Q`~hYE@^qczg(e_w8) zmFPS>rxNRtSOjFWF2~8(7QqseW90o;$BTccqF4;n1I!UJD!!2El?NKCh#w&~<4fpU zuh$k$Geu`cUwBrRgF$uuIfTBJVY`3gtQ?xMJ6Cg*z5F}|@Ik?~9U8Ujvj6ijfS^@@ zN~b7xM$D-2$eDDrz_$&<>E0I=d*M?%;Z;!7P<34O7PE$kS*i^G%mVUFgGp+^S8(Af z3&IT`b zY*7ON;)&oH650v~@b%CLMC&g<#pi!7f0m?UF{ujl6-hk$Ujlc40S06Rd$ zztu3sW!Nt`luIornG+d9f;v@tX^r>P-RlO@S;Pt~4OqJZ#gVz{?9!^gf^S{}@r$Vu zyi+-5%@s#xKCt2d-bOW+^?P_+wy%7y{dL%lZF{lhtg#5aEfDp7lao0z+oK5Ixn8%U zmP8U@0UzV;)*rg1l?n~(052}Lys z$ZT&tYS;H0g=QNmm)3&|SL{{X`2bOQyI;!E@I63o9Ydk8g<^j3iU7jL{@G$r`^69K zx0A+z8e4%-qJli`J|_GkXEFrGI`$alK8>&o3=#6BqYTv0HA7QJ_BMRD{f-hfRz)cT z0zo&f?TrorN!84@&ZNQ)e!C3ASAm5~M;c*oUkF~3os#frB>qpuoaMjU^gjQW6#DV! zp8Fki(F>m6Yo5(;O^$p7&ZoccEd&B#yRI|48B*0UHX25L@LU=17=eZ~Lp2$)4y4_y z6yY$b4}FrQSu_{2an^w)bcF|q;*C$i1-}r7UW@tO_OStlrkc=PW~Abqp9<RYu zFC_H78qSyKWA@O~Y3nSLxN`37+yYFXXZkAsE~$rx9qAO;RnE1?*ing~3Fe1R?n49xh|pHI?PdEF(n*7I zUZ_~?@e_?=>?yL-wdxXY)u=pD8FYe%)@j4b(xkeToDA4~w3v;XRs>l;bL!WdL;Y>a z739p+&h{4Apftg07rD~hT0VX*&`c4Bnis9>b}<9U)NCc+@_mpdS11bV2p!2vLus1oy&ZHsl5qz zG&A(-zW!8x8oXCeO;3O&at;QU9T&T!-{2S!gdEKFaj3418EAswX|Jk-q%fOUrr^AJ z@2g_EVb)z?Wm@A!@b{wkcE-Vodmj^^f>JM+rg|vm6Ct;(Jo^itywUAi_ zcz~|QB+(4!PaPeWUGBc;BB+9lFkU_wFF{wVR_TXN6MLfLSmv)4V_##_% zRHgeme#$muOgV)pi@$4KHw(b??CwKOjDaM6lC#6Vl9WRgMe5go`e%Ah3tEEyw0M`8 zc+w4}0c~Bw912a#(j+N1rVm@fDK*(@E|uP3!uF@WxlUCOdHX7$C?>>7Jy}BZ_(K}eXniQXzY_QESLHKAX>!7N zZ;qB@_N9{f2{1V$;foau22?S|M#IF;L2JV=am0=Ns8iMtacE!p#TYRdid1ll#Wh(9Lvm2~cv=sa4~UfGGw-z%I*7Wj;8tz+n({_Tlh^o67KLjGGQne+x&67s1VFwI5r!0VTI%rQaKWRIh zg9p4$=^lA`Ds_0%N!CwSzS6WO81#!uug67%aAMDjTvOVsZE|r7YZ+5+Q9tw^we8`^ zlc96b($c;Ym5N*%U;+i}7?;1zGhq1yYq)QJB*43i**(>~1T2Fn`cXVaF^Qp)<{Eo$ z-j7DYgyR`{OkwuwoRhJiFKM@&hIZE%Tw;>nSuBbv6OE8%R7-bU>h(0f@Z$K%twnE} zwaT0jPBUD3U>X!L2$uui7T;)H@MDWc?sBJke^SWrWurSZRK`84)w3zqkdkfrsuRb2 ze=j1Sgy$F4V|MG7DMO^EAp1A!`g@d^n*OhISVdaZ7V|S8`RwFaN1ON~nh@2C(texysRdkEi9elPX;I|aL=78XJqHvUCyBIHK0I|*ar zhbA}s(Jev*fis;TpHzLc95-dtG5D{c`uVY2=so zQ)XJU;FMw)mZC-t4>SpqXjI?4O4Uq{s*!^F1aO_Ol7p>jPU0vCq&MC_>%4vzNC#GE zzxrX*3p967Gl?B45II4gYWD~!RUZ^P@2#z{j`JkdX;GGF0Ywqo@*3*VexzXt+R@^H zROJnrZfYwcW=?Mh+jh%jN+W@a=Z7(OQ1FSpUh~T$2JVx>F=MEI;iU%@qD})8bBkX{ z>kXy0xtWwIx~q}F1gk5*sUDZbwuJ*Zjl?d6_Yhc78u|5aWVS6klw^kUs1GP37fY+; zpOErefh+AcIohUhL}ByV5bYHh7U3swWmD6bN#gjcIH=Xq#ry3SZXZ_HUM6*qRx^m4 zd>2t;X??+DPxNZ)C_(*mw5$v!I1MqI?ST4*<=djKQ>%EQYif?c`>hW9OH!P>0ThsB zmy}@NtCJT$W_jh)dyezmY#luLoA&9a3#oSzE#%KP;i6K_>?6+P=H(%lpfn$dl~(G% zk%c1tl36KfP*1aMQ*G7KzY+m}f)aZ4^KNHtnj=i z8U%|=DYE&MzCk;ioc4%CJZ@foFE{#gwKFw;412%|ls;OGn+A`w!n+gILGXw%G=j8k zX}Wc!YDZEujLoXQjJDwF+`IkCgY+ITJvXZGJL$j76fdVY8cq#{WYk&#ww6^oWc4yH z#1~Xea$=M9lF7j9_pqHNQ|KSkJRDxkXHzV{yS;psjvgxwV?+znK{d@gnh&r*nuAPy354RV-8T{@st0 z9{Go?RrSb6>&!zP!|WHq4;NR=Qh$eW>2#FwiTT`*TV%zb1YG$yqfZxbxw8B&!xvj9 z#WjT(f{ZI6ZCBH1n~Xv*mGma(1fWOA)XUJV@-^WBhV4P?2F6dvik*bN{!S)d^w23` zLv0(`qso=uD<{JuvAXg*@AZ~nT|G12as_1qrqC|%eBsV=z;d|9l>J&?Q^Ch9!ky32 zmP~rF5DLl}w;fg^%p>~Y2Sjk4c~Gvb-L`~{;IpYkNs(ZK#lN*l8O8mBQ9A4btrN|C z7zQsJNxO7+PG{_uU?{4;Dl4@bL_Ee{;a8}IzO0FNqR|hy+~i7`K$NHld-w?}14*$9 z3v8NvM0nVb*U7PznPv0bzX?+qDDO;{_UtnWj9$)mg1NyIE42HQK}m5#ME)4_D7=#R zshB3hcyN2lN1aW%lFTKFi<)6{MYA}*!e(IiUn zNOi_l{5GVi$HjNxch69Y?@@)HJ4YYrRdJB(fvX+fvAc<{#Z~(1`s5PW_%#&;ntHUM z-=rXcu;aA2zuDhxm z)|BxZedUE3hgv%r%3%p})*^@I)VQ{I53?-mHK>>U&t+l%_$g*92s(33iJPXdV0cm* zbs&haz}#+i=E+L*sfmuhYO#3!HnxNsi8N~tL|cf2OMXfP-q}FnUJPOGLFHZ$$rY)9 zpdY7=Ukg;NlE4Y@&IkLmbR%3}>+^9;|37O8v0#1b+)=i}1(yk~%eMlF)*s2I#!%(! z?NU%^>+mNtbxd{?$Tlh@7Tl?T=#OFlvWS*=Na8koQox=y|;LVUT#f)!>Z5&~hSc6@A zuC<(QF!zL7GSqN&hxDm>kWy=e5YrJ?(og*nn8LMg*XafRC=7dsyjyEq4#hYVcE7=Y z_X>Vzw^x=-SeA2(reGS>hZ7Q!r@H=RnZn zrTBe&-~Q7CB5zLU_gMaNKh9^sx@O2#DtpX`Vop+?aGs3GKy)>bY_jpy!@7?61>)Wm z2G`qF>WUDZVLSaY(ALjNQ2$iQ&}z~P)h|OeA`wHZ&Cct77D*S!>psN2=}nl;o0Mr{ z=h|-XbaKfjGVCyCT~2aQW0vo%Pc6pPs) z*4UaxWuRYWTZ-oO6^y`blnXdhaAX_;n5xr4EzXQE5hCotJPf%5ha;7^=t@J)728-6 zH>8JEC>!$5mD~b?BzrSvjK?z=LcOV-ghecG9-GpdOOCi$t8oHjyZ6(XyPae>=_e=X zI+9Q!M~F$S$B9?vK#+Mvy_~2++;s&-EwXN6&;?UdpNv@VaxKblJd9aaub6i1s%(ufF@x`zozl;_#QMK6X`j`1D3KJQ*g z9}c4ANp6H|qu?JvHCH#$kza#U?}w}UJeM~64BQvzlcuPrqc8V% z@cWQz%rD!3Ff!I}Msk%&b?Ueuj;t^-l*B%+F~rWggcc7r623`Tj8QZ-XAqN*DqlU( zo(V4;IQbrN<8!1E`EJ@OQ6rX-YX_S~y0OyMk76s@JCC*xx@h@vrk}&b-G?@RyE(r| z)7uY{4AN$Ru&*uTc)99aa9C1LVzUUeyO`Yuj3rz_J+kdKDmZzr`pF!+7wW{6zGxB$ zJf>vgh$<1%JAiX*N46B0*m4)ka=+;np?g&aJ25VC5q+Y+ui~Bm7a}bJdeqKMqDN!x zUfoa>Egnh6c3+fTi7c7>xPHcW9c_X%n{&$&YH0kx^?++rZtii~Ck};VzX?-8;^6ek zSnbD#{XErl-u1)FY7x5ARW34*v19GNOkCF3=r}2N&CVBDPoZBS=rjq}#V(E*cZL5v zcgZ>#Qmmz>kw!OPTnpC7Udrc%+Zmu7OqLa{9FE5%-Gyf}Q>c`RFR$+iE5lmgJCI(O zw{8ERtTNh3Z`k09Y$Ujky3g`2i+p5J83dZY%EFu+?b^eyqLXbqB~*I1OkTMkdKwyN zr>};6v)2hN8zth9y43oL^_~|HYNiG;Mkvktxlcbh8x~K9qB=9QaW-^)NqvGcz*p$( zz0hB{_u)icOB9sz;ndi16y;bQE2BqU`AbE!b^`gX+Qe zIl-@}^utJQa`ut=O5{Yg7?V*Z^y{}~m3bZoWj_(Xe5$?i=@?i`&ck`XT0R?TcIvi> zwF;!h1$nC70r{xtA?Emks&(}?T9RfvwIq@?QvnU5;CSN{qkb_KzNBL+FG z7;SqdQtfnwiJ#e?Oz?7F_0n}4O1Gz?U7nlRmJqrDj4@6gVNEk1GN6S$R^B>2<%gA@ zFY1p}oJ8gSFmSE+Mhk<@vPx#7Yl_T>%Q$L-9zjJU-)9<#z?rfYHDLL1UrT&|*~I*y zKfifIa-lbKF3i%&nN>QuS-ThV#jjr=s}hS~?5&Z~UZmp$RFo1ww;L2{m>kcGopZme zBrmz-{}J=w*Kz^VAE@t;HkFR0=Ivi@=%P@V!M;r0cq(;cW3$>y?KbF5h^6@1WHq5a zU#s72Oh9gqS;X`-3;zH7H^Y{cZ{%SVq6z3-*f=-0!|WJQ#+=ek4W;jNp;4c%m4M%Z zVXpRI38hj=mHIZ&GtC~O5UI|=XB?+W@S^J3#P`|7g4nL5s5J`O47F17&MW+&XJW7; zDtruTAuzs{Qs*Or1%meje;+Ax^`(MaWFOIep^6EK5;4MevOTGMk!pjIm z82hnoj=b?u46HaZ2zv)E-L$fbW5mv&K;9v6?z_AEm7%*0BD0!fM6Zt9fxeFpMGRfh z>-3FG6tTW+i;!lX-k6h@DhNu#lN2;Z2$sfSoU!($m)^b25}q8!~Crtb8G zK6=g)`jUO6Y*&2SC8|&X?*&4bLrz!sH&1tIc&~KxvuXA#og1&FaGNDgls&ztFbZMJ z3)7L>E`GG(zP?askY;8~lNkx|D4oJpvwPL2gO$=phDh`y!vC2)*;sFEr}PAbXgj-E zMIpPe&NRx1Bj8VtOXiVJ9YA;wuU8!KFPNX+Vu%HTRalkAh)YZ^Elw{sf%5<5 zEpy&y3(=aLks#wfC`OTuAHcf`>fH@Z6!kOESAwX>Xdh71Qv?z@t&S|6dm4uIlVq1| z7{F^;Y)E#qQ+W!^DSqL5(#ZA`kgMj?2tS>P9o*qGs$wIIr3?M3$s=1=QDGk#swDW~ z1{uf-`1pY((@P)+Uc1pYy7>Lb^G<0{e0^Mgd=|hLQ6wPrXQ(I;o5xC4vO>4#otzy+ zmA`m};WK(G>wgVr9jBj3aPxpbV5cZvw>`{&gs3~bR+lC3@!-GLVWT>t>i&(f7U&S- zE{vos2~mo1L_pnWr^zDMe%qO094;&D3UJ8B^W(EHg}Z#5S9(6cbcvQ9(W<}ONfCG@ zk7`McW3cs}e;?-???s7?HU!JN$@j>VL|`T|9@Vi}E{7 zS{&eS`x@qzH*=~ALY>yEJB0ljQYD03fLw7ja$RY6IK2B$iF2Jl|1a=Dh~RD46#C4e zg8XZJXTo+YO2n5gfJAV~t+!fo$}C08MOtIafo(@_Ui&hh(8Zxln(De~DGNSa)bFw~ zt2vG*EIb0F$8BMu)tm0D2D94MS1p%CDC>ciE2Lt{UDvgLh@7FBb;h1Eg%do2ehot%RVMAR82d>iYue9cr@Hu^)lI2OEZHT8xr zRxJ;V5k_AxoR z**SiO5YJc#uFl-UrLjGL|qzREy^AzDd{bkrzxefr^+@8{4?4o;ZDyco$cYnd4w8J>?(hdWc+Nar~e^Zz*k zJRr)&WMTcnQuif<;vz$Jh>dzROWDh`A4`P@BN31WTq>0?r4rqZF{Ps&bDWHLG&|2? zbp<%)*09x|Hk)VEHcUu-es z2>BS2yJQQ%1d!$uhQ_O`Yi0O?5-UGF&WL2*gFkr6awofqV5Mno&#L z`DJ|^D=a4+NT}4JTK|IU`g)qwopm!F(4=CMFJg|>3)#A=2?=FL&-9C4A!BE8>J;$Xd`B;S=BZ&~Vw#q-bqziVs-uzN&pTg*kA^icGR!sDo@3FWo6v z9%k^gn?zW*nyhiT=OtPW#|=umw3WZS9>Kv9Ra8y^ysRJz@Z1(@HCpJwt7sdEm}`Fu zy;|}SEsu_&&7O!sdNj98-srjGn3cdAn6op@yj70_6-4#%x)f1)~;gdypWnZizc=D4Q_qxQ(+^8+~RQhc|?*AL7t=?+ z;dX_OcODwi$_LeDepfomK|pE_Sem$AY|5CERG>Pnw9OV^*pu!#Rq*uQ0qo;2UPf`d zOdN5T*Kc%`u}L@bIBZsOn4#PpG7lx9bFp`4lM0*;%QKQ6iu>gRl-yp8A3@^z)<}4@s1?R6E4xj~HVfY4@HubzDdyX;47}UwGQPIsEu8!xzI2q{lM)Rj7-qXVggKf9*q(M;lk$l#Lw? zf8fTrj0fQK9^=#dS@G8AN-ok5^N{_!B@}H!wLcIh_B?zPo%bOIIH_^CEr}_M6|ir* zn0=^rvM{M!*?yf~L*&ocG?^H@R3f7h8jWafx5M3rIX^bF?A#&y-#fj2_Yo6eKnhVf z)i-*lE5)|9n+`LVrawk~O7>q+psMlW2chh?n|fP0{&@fNHxyQElNW4WV#zSIw%1Unj+ZoulEWZ&?eq+I9>Eo3c$EG=m5pH|a2=SMN)a_es8Hr*)CkgBx4ev1G9|={yd*@Z>HJ1^@@U zDL7H}>D<>qQUhjH!x%o45k29>h#L`Ekj-31{fLvQemf6d;#A#$eiG;`U^W~^U513e zxR>8tL1E|g6?SGIhp(#vk}p_i6%m36h-4(Enj!&B3mwJs&VK225~VdE#f|zMHD`j{ z99s8`UEd>SH1i!I-OU;dz}6_ zJ+Vr=C~php*Wwie(;m@E8~f?=~`i2Wh{$qsGCSVbQSEd5 zt?h7+vR4dqETv1t8F^U#*0QaM4={s1OtvrHgpt+M>fR?#!q}F9$-GS3_|N|A>>I~n zf{~z0R!uv~bfA7$+LAB244spF{)}&ErFjel%aA$s5wF@R{BU&VBZd4U&vGbfnTssO zf)skTuagNUb`&fzhEjT}T<3s=j+cOTlIxf+=T3D}UxJfP!#lRXVr zQpPI`c&>t$|JjvV^eG{j_@%R6g&?F0lR=wiIVhW}c?^0ya@eTgmORVd&qygQwUeyFq-oJG=`|B(GmlY*LlPaC5VK35Hi3}>!KQE^QT08m{OeRe1whfJ% zb2fvOB=jSBCKB=8wh_Ou;yYReQXU-V#+gvs-l7 zJ8cHZ5+Cy)N$lAv3z1t{@Cl^v7xf;UkH(10JF|Mc&Evp_TzQ}LFH)fGTZlW^USFo@ zI^$R0qJ(o2J7o6k27I)KF4!|7L$YO4YTfUQ$kFX^&mzE*9C@VCZF-Fd1(Nfb;CXfU zwZa;!$jE{ge2n6v+(z3Ff@PUKJ7lFk(9LWQ_$}mt_iXh zXY*Hojx7FM!#i_iaXz~iM`|%QGO-mfTgnvYN+ zPvsu+aqVZ-Ls2eIR*x=&m8(i9LHT6ksHT^HdUWVc*c)yvL<|FMiwDTNO4l5M5qvvU z!ej^*n|{TV=myujE#}WADZF1|t-e7kVQWaeR;4&43%Y@=kh=HsY7*t?c!)u<_hVR! z5x9h8E)M;P)a;P2_5S32mYaPRK>GeVJ#3aw!392-l0bw*T<3#l>{xa))~+U#%?+oC zBq3`oKfH=yrQ<%Jm0Lxtee?@_#U|G!i%u3`KBl7-0|LIATuAhi&dx%|64DMhvvzcS zv_K#lA#Bnl>&~R*u>IWm&0U%T(dW9KM~XyC0gTex3qU6tKOTz?W&01z1<7se-wp+Tpa)_CO@|q14R~#g7N%Z0DXNO^lr%m-B|p zu21^9Oa^{77Punz;Ck|bZ8uRUG$X3CCY;`xjB;E`kP~WNxSUPhtKEq&0I(Oab5L@!_Eh2*8nwwd_60jtLry( zdeTFRcfUp1Xk&zeTr2g-kgB}cSk0? zwDHlpMTnv|43FvS-0}L`yK2JP(@khF3qP2b%g8(t+OF{;=pZr!gPhs%b z{b(0<_X^uzUsnqB{voYzzIX}@Jw1i(wrl3btjoESiO5bl7Me456n~@CdEDffbIi^A znJ5p<_A%w*D5dxQiOf31=a5S#lV|C2j2u}_?O}ZN5l+EoapeQmAZL|6P9em*F!lG_6Q9=GY4!{;VMN4vhY*F! z`DeI-$~uCKh(*s)lm>SRA|@02j0mMke21koID+vY*AmGr9DIQi9kA|NgwpXS9H7tR zAL>l~NHH$@F8S~>=O}e(iOWCMv-vgS2b))O)|Jy5*rL2qyl6j~o{s;U*#LhC{#r8P zScKJW35uc5>Qc$FYc90Xu; z&GmfgSeN3nIEFD3*@kuIjwGL^7e6(p#|K|9G!-VE*FN{$U?$f$6{Ff%l_Y`F14r-e z#O2Q&?oH?_!@WU<9x4=E%&+m0tfqC0c2}j;sOd7UBH7%(> zOLxetrqZ~P!=G*dMk9X{9{u>jJJThAxXwiZ1|A@^j8)r|vegRUd5~ zeiw!<6a;EV(WA+1*8Ul3sVbU0*%LKHyUTZ<)^+dAzSy3RE{-K0tMd?AxRb&3%=F|% zI~#~s#^OCZLalM#Pen?~m+_}FgyNuM?-nR3?&?N=QM1#d3fd{P%9ztHVN_RSFggLq znVcx%fg#q&mqLE%mayQ+-|zPb+(ua)l_wBO_%?K|5jj0y-@`^MJcbvQJQ}11Bl0(P z&v{N?ybY8!E_0O>HlqV3oQ*gC<-{w}qr!yITGkI5mCEvMQAXJNN~W#-cG=^+5dYz3 z(Mj<2w0KIp$tJ)O)y=Ha#o~w>evlZ8wxs_A4vL<^(ayZsKKJq>2s?uco^zu#b)423 zQ)@33jgFt+Wbzg;VP1TrztPL*S|x2m$x>uwY^jdjE+l2fytYC1rbvsk1hiSDWr;-R zR_ebHo!VsmEwC@4qIusb?d1MO48LX~EjfTI=|LzNhUKN`dQzy*p# zeCZ8aqi&N(xQE+NZhJ0LcYcSDU&=N(g|f?n&2xOg3t30XvvjgEBFnGwM} zAHR~NMxnHVg?RdfhXZ1FkCMGBkCeQpgxQF=>>J0MpX?|hZq%;hIgV?>9RGu7N+2lK zK+j6fH9GXv2?3$tr*`P&0V?w=lvgA3W7Oe}m{`RZe!T}r)CRyVElA@C4cu)`<~y8A zaEMhF^dMs2OBsG{hy}f?u^IVr)oueQLUn3FkeK51Rjb)kNZY(M{EFCs6U#<~`mBUd z5FzoJB-0ITT$w5y^Y36!^HugQy=fh%Ml%mb?x?AGstpXTZ(8CU6HP?}m57m2DrpM> zD&Epp!8rEP@5sg^xWImWRzb{D->m@8j+ql;y|i)&iC%WH8JRJGUUA}@Y{WupiGM|QA${;SN&zud^&d6 zJ0x?xHg>epj^t_#CgqP#Swk+h&kv~(Q{rpF#2J`Bn7#@)(JJ-bE``csz63MF`hcn+ zR)sbS6)R`pB=j3e3Q&IRU-IkCex~L_E;~`QASggrQI<=zQkDwH2R+m$#d*b9XCA{J9*`E?+NqMk@K$B&FjXk4&9^dElA&}`ma?OJYU|A=2fx6Q zR@VZ;7yvmJV`Y-bMNB*E>OEdI!m>C^*v`1kuinnkAHV`q$2mN7@lH@EBcw9#oe8R> zyXMy(uKTu6w~fLFaMI}kDzVgQuxY8(ivhvTtKE~XQ8?&@FW`MP85*hfJ0hTCBri&8 zV=vA$%ouJeHgqpFjVc;%Mb@axk(MwB#h&)q<&pA#=Ns|Tu?~lqRhC13d zgHa-KWDLy12lmU7SV^~E$^CNl<19a*}l06GgQ0Fq_>`Jg(kp)Dqxm+3olX;lDiHfXI6vGxl6e_=3*{jDH+q z)G)*NsB>(%Nf`?I#lai~D5=4>HJmUKDbR$6kAz+^#IcCMj*dBYIQ}TTamY!180nqJ z($t6S!T9uc)OjXuqkC+o>2twq+i(h24PsDjgv_KPQ zZNn7vxy(Md{8ZhJQU2_f)904EKDL67P5ZnHcm?P_{<$uM@LUQVMWP$60TC$9L`7r{ zRjwz`1bU|qxnK)p_4%lok0E0Cm{>v(8?Uds5W{>+g9-STGRH-5_-W>jaeuf)JJ_xkv+9Gp?g-KUP4C|W8zvzLrK}XpO5#-XH|_f zMp8C$O|Scu%5=FbW5VRuxsiQ&qb<)&+5i-{m(Rc(CgcV!SB#XC%2Jf+l_lhQ z`|tC|(OGOs3IkB|m#{*XxVy81;O_SIL(gu`>6-2$B=6l@1tgnXj~vuYrKi)c^`i>H z-w?5s3emoOHEIuuB{HL-aB)ZLj+v0s@Z>VG%Q7yVt`U(X>a-UR*9_cHBhL+@$Phi% z%^QvbP29A)(3kUlQ*Rch&24kZilO@jod|_ii5oyrY6fsUDd}?*dv6ghHG|(y<@22; zJabSrA)}^bP(@S!rUPB|3upyjdm(FPq|`OQHF0h;XMaRtB;hHN@C>0<&7V}T}bnA!x{T0bG*53{hjr9=@f%gguOp!oWNPt13AsX=)MHnMjjRa2;GV;vKKmM;5?Wd1?iHrHo zQLPbAUZMA+Gj0;M=rG|Xb?S2{UaE#GABSnrju|GIeogC9w8%#^F=}{eUpVrzJhx)Y zH;V~`ylQAz_~qa-VlpF*4${!6^i?0@s;ap{2KA9 z;T*>Sy1Lp-E)sp($#Q3$<4$2DE38xR)Q0`2(H{Hq>^O9`K&pf|)qQj^W05L2`lwmd z5teNMvI;T2NM7q*tC5TGFazLnvPw6aJCi|M5+8h0BvFJb$0>f*dKuC-ZNXoQ2*iaT zzM=4K|9;wGG}+jSNZv<3BxOT%mTn&;%pFqgF)?O&l-0JSklhGK$3^20_2;Bt2wdZs zGYzzfI%&aNY(2iSr&bDmvQn_4!}*Dj+06S&v~D=%KSHZ2KH`p_CYGPZM=&z7>D(IH zL6^>Z-W@y70e{iCv-o>dPldR(pOyWzgs>ChJhf)Z-OeBTU4>Q=OI{j z{SJ5YMbXAq(-Y zm9{ga23bt$ym3XtUki;ZDqNwKw4V8w*tKS}Oxy0-FnO4*Ao5gDT*7ROmiiFa*#4r^ zK2J5mm`K8TGfLwV2M`5mE6Q~O#Lpi3(AV&W?(IUR5HTMC`FlK*N<@yszvF>;(Ok3O zSMMBq;=2C}BH=ZQ_Piy!!1AY!y=v!;zdgC-j1QMMwE-$!d8ED^MbnV3^@>PpaAb+> zt!-nd*FVX2)Z0ZPg?8c3V`A!na8`(3ywPx^_ZqiDI!u<5!_OK@Te33Q%HIuzVU!J} z!A7Qr&7GcbIi;)G?PVRrT{KbTaVeGJ^sZd}!!)e5R#BI(_R*?4OABpW7bJ>L=vP8I zZxk*zpJw!1*bR;@x-FUqAZ?_L!8_A3+&^k}D%afxV@-DaZ;wr)|4}Av=Qiz3^l{W0 zeuV2~2I@1HkNl&neZOW5@shG1f^AA%e4s)YKvoo$bD5hvLfqTtdfwmG@!d}&&U9DD zWAX+Omcsm%Y4X5dyhL}tU}+Fi-PO)Gmn4rbHR)OjXI|g}WuRMVQHX#}iWQ&K5CjrI zL|=$9q;3yYeM{frZO;ma=Ht`e_(e54bESe^Fm8KLcPEtmx2Hf;vj;11?s(SOUb3Lk$rJ%L2iY~F%V zM5Kk5!7q#N)$f}>HjuJ6)KzePiP;xIA$EJHb8$29m;U#7LC)#mQG*iQ5RmN}z$@d< zR(YGw)f=m@Vv5#1&a$_28h%=M7e7)4O9jSkBxbe;8AvJM+6HzauFcC9Rd-w z^tdRYq#ida>~s)Vq_s`Kqi}&wdD4uqNItD1XtuH6m&==B!x`Tvpf6KKxb+IuNq8$w z?wlNft+WpzV58OV9(yM>y9^b=4GXbfTCs_S*+w-z)%Zjr{h@~ASAqr*8^h%EuwjYW z%Os=g&gF71;;t`5yc{asA~_h7f>|f_4N)rLF>bFz!(UesDSV$Y_;p{aeexUe;!zt^& z{jCi@zVJ58Ml5d*2K`o<0q;?T=I$0)->?&QU5*U{*(vrmbmNJWI_{_hSC%U9sNmEs zE-TFwUud_8r&#dBck-3Y9ckw#Z#hiff<1miC-O-UXNYOrDvi&g5osccbzO@jVP_rP z2L#F%|5Whm%IFJ^zS3IDBFw6^IGcqb*Jb3S{aW%W@!i=mRa`*({5+{P{2NJe!mlZ0 zjg-k_EbM+OC{4($3qkJ_w>kinIv0=QB}@+`Zi*94e*CkniQPZk`-GDHXByCky$FjN zsL8%*jb_D(031f8o@*Zm!EKFn{x({2&~KPxcDf_t9v;eLx7sHvn+IYDxtaJns(YnL z@bx*<`I)8etJ(ZA&*AJg=p*s;HO=7dgGbdD7R~la$Wt{cVj7gHlON*mgQt$FM}*TF zw|`C7XXDqB^&{CZxGRYkZDpE8(73k-qUH4W;6|!ow(4Ll_bVKU#J}PNrs#u_O(=_g zV77-H>EMy*$r5U?06Rd$zxpnvCoLN2@QBROPfg!j*#c@(J6D099A{4lQC8C-VXD7R zUh*E>Y=p~?d19B&i_`OQ8fa-_gjacxrV;$`jm8uSQ^C3uv)QX9qA$>l4q=E$_P?vx z!=La*Af=-YJYn|Y1qc4FSNI)c=JIb|Y1Ar;et(-1^+^C{JK^y;mc?gR@zB~9_+wu3 z+q*-)xFGm(_FWeu$Xx|M>76x4|MYMH>_d50L>gO;f7vlQ3@$b%Tr2t+5;GT&u zuYF_9cY$Lp0LASYzu1raY<+l|VKwkt>cE93PK0lePyapcX9s^Dk;3pmT@)qlg3a{( zSdmYKfoT5Qj84Cw-AU_T&VL@?wnq808Z&`RQ-cGOD(C=tp=^DbaAo~ZO**E!OacnD zS&chAwcbKZwMClpQG`@Ck%$aASND5KrFvbUl#YhE>#+yBt0wU$2yN0rji9rc#>_Mn z>BO>=YY`&Fa>C09*!=b=Dp3?Rr17va#xu1rX3Cvo&=$H<8#kJXxq=qf`re^9$zNNm z^PFv6VEe5&H^13I{DVwgQ9>&mCnuBP*vqQc{%$X`HxP!M&7f@fYT!akV}G!@I;y}q z4SI-#k28I4TQhu*xkh?EPxID~YI3`J%eg+2k>Fps9P@9`{iTDlj(EISQSgl`_bv|K z>j))xEn90)VC1N&v6|o{jx3@SSA2`=qU!>D;6*`A-9WOeMNE-JRd*$qNmP2KgIeGh zVR(k(z3%Vhm2aBc`U{#sdh6&sMf3h#=~_znLsg7H)FI+DhZQc9A-DN)-Lh&(yE~F2 zjTC0Y+0DBjna*y1oC4q*s>DqEWkT~bU)4N^?2&s#-61TmNgFP(x%X;?;F-M0hN z4xff57q`2?8JTf3vzr?lTQvN z(~hYuHlF&eRZ1Mm`PB|=J(*<>n9IK03iMdVsn`WIs+HhkAPPiE+d)xex4)Apk1fB{ zs^z1zq~R>vA)(M(jP;MDAq{Uy&}2CFiy0g{!0GT|YyDiM%dC)~8V9bk!LUxah0Fo! z75@5(2r9JB8+`NsbyfpvB&plY9u}iEe#rdc6^NH$y?f$FgHRy|E%R&sUptB3_3%lPJum-N_LD zAKTBe&`*Y~yY!hH=wVyVT0POt&?kY+TuWRu9#=S!ygMFQG6Pb-@+5vKyb=;BB>9gv zbS_slc^co+%3h2la3{G2WeOtvmZxA=yNx^S;vE6e0NOV`6N{Ri8p##OTxLWcg)G&K z>y(flgGFKZcWAn5QmB#JF#OJHsZrsQ%2ZfL{K>z;s(wbp2|m?&UGb1$g1l53v*GK zu~~>4a?X=ah3DdkG;g}wp}ws+%KlR)y~)8a?9~BWl>-a^K2>B8*wx}oeq@-1($^u~ zw}^P}A>}4-uVL2p$KyXHK;h{^_5z`4wd@!aTMC6&EIw`7?Gh=#-vULZ9{Iq4*lJ~G zH4*#>;(Q~zWk=q3#o96KdM6IZnLi(s?a-vwH-|b=Cg3*kwjXe#x;KqY|nH-TkU0X={ z{!&}Wu`dz665jh-Z%)Mq*ZogEYd=V5(a#^(TM@*0?r~puUMMN0jb@NEcl>XWt zJvoCR!@T+ik$P}WbHltN>12$ANN}hn7Mw3z8{ysM=Eu4IBz{WlLqCgMhf_pzG_?_G zl&5rNgB{q~zCiY%AhX_IIUHfpGgg=LuDm%&(W!k<#-}B@9?F|g9c>~EJ=plkxV)=q zr+-WlAwzm-5(~aAUD^9Pf=mX)n*7x?f7&Dk{5 zZ0fd_uU8h!kGqfRCsp=axA+XCJJZ>?9MnmUkst_J$lp3yZPrI!buQ~HSD+rs7R1#v zkL+0(P|T6UE;v{sXnjUPg7)tb=g!j>R6^b`n>PEXM)Go%ZTi(q19(d*6=pQH#+Tnm zk4%AmLj{-ZOoig48Y=118N8;9!m~+@)p&@ri-gK&BJAJba}1Q+aESES?hWlrOH(nn(TG(R?q^vZM!bM&Qi>&r*-IgAtMW=Ueh za;6*C+NbDteVP_OhlF+>R{e1v3bT-(*?!x4oonN6#Wob!7ze-ld;OxogrBL3y&g?` zY|D4s?^cy9F2K0HS^U`p{V2!95CtZ3)W``hnBTCL?;0k2S?tOTrn-S#!^dq0Np6A`6F5?6{^v0I2!>a>$9lCR750E?E&) z3xz`>?Ld;K@BA{AXK1fm5vjBOeT^8WI^|(%xTF(5ugW&rOk&*|gR`C6uCVL}l#BCv z(kb6;9~P!xQKh};7$d9ql73c-K?C;wI{(T$vsC%M;%!2DB{FTqe6I9Q#@J2kmrtZx zIDqM>JwfNC!e}%&UbEFMauCPoS?Wrh#{>DIBR}15HCh8G32d$ zKSNd6wl&J9$*ZT?Uvlq^N1R$jK4?qUU) ze@ntBzT#ZVl3N=_h*Bxi+fsbuVx)CPnfD2kSlvmPy2C@mqE^z|m}gkUwVuBoG3;RD zs4sXSh6~PX!}7rOI5sPI>VEJK=Zo4_;`SF1)iFmE#np&-@h|DAB)VNhS5tu|Jwr$@ z!+DSs1_WUzv?r3m$NY=x*`;yv&T)8uj8-F^Z3HfI6-Cj(SnvpwS3uD-3{|6<7{`3r zDL?2IM%31`-tDaTxRa|l(($!b?rBCn^Xjlb0cfI~VgyYHxx8l=+-UgR zd3{@i(d@>rK|dW$F>%=AqDXMg3jC7j@rSI`J0j@Gj(~jW-pL}xAuR9b(hQcGzSX9_ z(uDVbfZo#XYeY`$`da6Z*(M)4tF?mjJ#INpoRCsB5e(l1cL^g2R$h?H6{GM4-3o#S zd8cLQPVilHJ{zbvf)Wfa3;r(uV2~7+CP3}QYF)y`tZ|{(AwfQt+iRW>8ovMq`|m09 zs-!U@yIR+fL=C-!L)Di&hki78(S40Xx^NPRzCuhf=1j68x{mPR!Nrt|J~2ZJF;7)bnkRk^D@kT(D}Q<`j|$ z{x+nU%s)8#+X+m}FfXmmFH{?zc^#JPM*r8g2y#D;BjpZX;V&csJ3ed|KDu26A29qz z!%I7@8W3#_>(_u7X$xv1th{@esE+_>f$N`oGv_pLx4n93(PbeQoE{%=G{xig0LYn( zmU`(mGBusU;(b=C=trF(2!VTVJ!;UAwWKaO&&gu_Bx zc;sJL8t{@CIma>ic=`qs#?n#0sk^f`%MFJk6?GRFDGM`hUAc9GNqXy2SmxmrQ78dX z+71HXtUGJlrOLrRrv-X9I0_C3Z4X&N*(I10bMe;A*=2Ey9NU-^*}aUpAg|WCul19A@iR#h1=vO<* znRw98=?X7$JW~=R44A|;?QOI0G8xj7XnH}jxQojLzAR!3vM680@pc~(iDF)gr}7oD zOUy9rgSk%~?O$A?>Af6j!O2F@Nc;S>vHKQ_>t&05W!Cd!zW!Y9B**p$A)_J$-|_v< zeHjJ$v294E)uen8Z(>a0mOwm*0R~)yCe!EdawvC+9|}Vcxi57z->HrgTRUJjU%{da z^aCFiN>Ry)F8`=AU+fq7I-WkZ3JASWXc`QWIWI6oFZywd`>%F~W|rFeZD@~FChHYM zrGvIug<%V{gGE%g=u9MF%9kONEQoUneWAZEHl1NrWIG-S!Y|Ti%lzQDQ(WU^twX-_ zMY9(!W=;5UXU#yIYpwXr=6AxjP!&4^3?pfRIL4%i$j1VKVb(uKS+#`BB33b_Hm@vg zfhTQS=6fcbaZ8~`RYdyp`UlPIzlRas2ae%!B8m{0h$EOM4))?mnrzFxyBg}*h!PdC zoxLi4^XD#2QxLab>1S7hk-3E(9?m?J2N!Ty1hlT!y@+u#Co6E@9gtHY%N=Xi9IL&N z;A1>^i{%4pp6YQTqH5anEz;#MSGGdYAeHpo4}FgN3>gNRY(V*E#AHZZrlz6w&#=GT zkKwypz2uNVei#6Fkt^C>WloQA@xnO-oF)B6mFuLS<{SyopQs<%ut z{Dp3`Cl_gl%50L{s@v5m@rhfUd6mq8mqWXNDr5as9DrnP`G)A^>FL5*gOG-_G=nXD zNvAPbzMn3nS{RY~yDC_qLIxJ|g^gyF(Dm1UE~9x!b8Gm_hqIiaBP5gy-asmWMaX(F zX_KK)&E2&5o#SFXeomFyX}`zK(Om~YG!rX%M1>-jtjfz**B#^iSk5n3GkYO)RBgx1 z3CZ5qvBm!GX9V2GpjAUful-wltkU0-}p_)DK5{gc5rr7d%uM&tYRrz zSD4iz^~z^6^@j^M9Qv(e=-y08PCvfU%K^BU-puJ!SMY_@?q5nLH{$t$PP^7MAm>CPs^F1wjI8L%=P?AucMN+wf+xv_GO&1$OFrg4mhk%^c3DY6_vH@ZQ7>fcqphv2M33qB zB7#rZJ_BjFjflTD$Cge*%WsAc-WWkRyW?gsRZRjT`qQ{CxbbSE}v7= zl*psR4L`b&^_F6O7TYfK?!E>S{L)Zpcon#lP}u3HXrRHgrmkdE z@v#;0n?hdq%e6;uC=EZRPh3!lJ5*8kMzHtP(+;3ZZn;F~t)a82N;NMYy@*Hnjtm*` zUz9jI;VrT>eoF1<8GNe)Hq~{~`mI%XN>H%f`Ga88rv;tDm?7eM&-P7 z&~rNC#!&UW12ZV2W3J5>UcWg?$e48GizV>DRI@L&>^Ae9(wcC6HgWu2m`!lm4PJ&? z_+C+45|Q!ytE0~O6JPle@y+Nyv(c=~)eGXP@v}(Js_&s6oW(~q|dznBkwZjr}&{HW>&6iXW z8VMUS1aMRceUUea+*9&fm*%umiIon)8()l)12OZDMk%YF4%>1XEC;t)D>^R3eR^(8 zvZ)9sUEIj%9^X$6=8fOtEn$pbZ&~7tk4UUy zd{B6z++@1~J4umOd6?<>ykHb)$DX#D>!B%d1Y5|ffhfg%!;~S#x z$#zvYqj8{?w(C0xk8=%3ycqE+B)SSwf=(L4n=uV)I~1i>rmmy}30bR>2vO;DGtrx% z;fDGfPbg2W4hZ3fExJ7qB%t^`zkWH_P7m1y$-2hnN>1rwNWco+y*2$fBwUb+kTl^q zzHyC~%8b|Q9M4rae3`0D+Aq?%_JLoO<+NQe^5;B}rZ#WJU=m3nyc?6|AHkxaq;fSp z>3&1E?zN-$Bv!>%1I=+pV)I?sea?;e=7@kOVO#Q=AaRFPLk zU1!Cwvb_;EYpnCgey}hAf&T;y&B<2!Rp*;cOx+e}@3De6wx~$rB7t7uXAYd{q@BGu zj9>R;UxtzW4W94|PrkRO^f#hq%E}ge4o63s*DWx*@Ml7`t)0!$QVzBA&nBYI-rqvc zuO$`J=r3WtQg9Zs-#MpSY>fNRCB5FL*rJNGz;pL-0qK4X@OtW#(l&Dst?GKKeL8Ez zn=lJ@Xne`Drc_bURY}1Tr;~W;EMt3Zx4NF>iVL)+N3S5F8fQ{*=#b7a{o4Ty$5TM& z+J}~ewOvn|0xC3+udLwZ8KKsr_ejjEQmU6|d}Ffmh82^_Ca862PR$AKb|fd{`yI?2 z(l9buYlvjle5v%^#4p#fl*TWTxNW+d-yQs?aWxD1Uf4Gc!4cA?Kse(?O1UGbxf*ItrH)#PgF7B`vD!s zn{af~TYkVO2q1 zrS)lDABv{OCaN@IDlj5tV zTqW+B?9f8SLC1?O`?av0JBPBI6M7Yv%Y4Tz5N<6=k3fk=t#}XNI9$%kQY+DYPaBTy zsJLl3LXAS$9TCsmEHD!33%Xz==KS4i|BKo<6ht8!A-EWU@%f3+)AINbiMGgfv4ZNP z)z!i?DDhX@o#_X~ngCW%7UIKo4MSW%8@!MF{3EpxI?VCcU~a2J4@18c=@uz2(@1wk z+HLfz6HQc&P9D9L+i_?Vv84^Vbi@Rxqz}k73yR!ctLXzOfdn@hYGLc46Axstdq5Ql zcD{AjO(^GSIPCyGd&zW$~f-=nu*ejkf$vdhRSzcqPgYGcqww)MwY&vaOHqLk`ay{HT{&-U>NZ z3wkR+P`Yx;Uwtt+oR?UtR7%Fgir%sEiqgiiw{TF!Vp1S;`wHcXveJ*}r6NIf5AJY= zUu*ro(GIn;M$+NZdKG&<5ElH99h+OT+{Rt1+H_!=pM+5#DA)Wm_SNh4)w4KxBAb*; z{PHUrC=>rQ+}sbyMe>cSuE-I>EZb)AzGp(WI+xJzf=P}$b!hJE%Mr4E zh&cHB`U;V=q|LwuafK@)l=Lkb2-zUk`h^g1ki90V#Od6*hnnZ z#2h)WC@~VyZ|!0qnDm+n_^qMS0lmFTIfu>6#;nLC#`xa%U%L0KKiK&3*Wcq2Vhyjc zs)N{R&2nAKWeS^^z*cF(+4vj4&qu`aml%A&7xQ#S4lcPz!HnpB$iRZCc1iPG;x*X( zR7?2n@tjHgB}I%Ur_^m)i}9t4%PJvXx5B4tzoK=75|uLEeVj9Ft=Rq+UmM+d{~l5O z7;hmMi9Q)rA2+r2$!UK%Z7DVHkp*`_!>UVodiWxvj3CFkL*F>mnVQr-M*+_@kH!4E zT1KvJ=(03raet3z1HKAcAWvCl_YgS`fdW-8%A>xFcNU zbNY;uFkn+|6gTYIsWl3SE6u5SUl*t0#YAlH-aS^n0;$@e%V0UPVaP!9jsa{nYc&+OFUr=PCxZG;SlQTnEvK^`GSfA|#=Gvd(}?VBr!^EB zIN>nFMWM_Ugd4?Ys7ti?nl22U5u!_}?$#^Hs}I#G@53-Eko&gTu#$zf6KA7KA;(+w>=Shsj9s(g>=ImfsYGEU8c!$ zaU*IF6Rsh<`nXf6v!G*tJ0aMlS!M80~ zuidGgc5JwJ_eXS@XsF5f)Qc4T`&^4M=l*fTVU24Z z>N6|6G=ZqUFnNB$X!vMEHO;=>l|XPzGlEAmglKroZY$c5;^oWCP$#uZjKFwvr-Prt z8#oncZFigpngBc}!xi0|Ptm znpzwUF{c*oOyeZjD+ULYT-dy{ey;Ol#FDuZJQ5=D+K+4XQp|{6ZL)tC;ZVbvIH2_~ zC?oOq!gOkM_u=4*-xgp{`pdL}twQvuE4b!ZW`pA%`9?!oyIt_P5w-#pveW9`Cz`Zr zFI|=RN3I$%D#eqe8rN;n?-oKZuGJJ{)QJlq_(_(;!Tk>idR=$o}i#X8x zfvNk}ZBr=i-hPgAxRtm-RhZ6^CCv{+v+K z@qI`TJF!|DaM5dW;DO9g2GHo*#N5b(qpL`;z^o#&aX1k^h@TD+({LC+jQxosfn462 zDmhz{c4dW{&#c&K9%8qDBxfTo6N$bS^SDGaTd27m^b@@zG8uCXPzL+DgC}7!$BCU% zd@Y@JEP3cLVxgLG!(_~76wY+(#aX80G;lhyHKQPXfo}7^XomM>Pd(lIoJNmkm!=_U zz3qSkL-O;{2JJ5X?HHS8&PL_9vrnF)W{lk9)mIXEG;0!y4M%ok1&!ZeV)_(!ede1@ z9dEMnJPU8tPefP{sEbZ z1V0{iVi+YI7YgUS$$5ao;odthbAr1uYPE876cuq!XpJZ8eX89=xb2L5&qgZpY?5$R z8bUFhb($yZcZ78^C;#v)La8@_n z@K}_g(d5%FG3HhmNo))MfKG0xZSo$^Xr>KD@`C({Jb8Zz#3rnQaRcwk!M%m_+Z@%d zW0pP_*4uaZfl8a|4eslANY(F-x6n4s01<{N_e($ewMhB~p9B+5ijwP^>hCYIf+A3N z!{0;7wc;bGlxG&ZAI}j=FFx2^a_`hfD!ZG~LR6R*AWVQp@H|N=eO0Mms%I(Jp);X9 zykJ4M(NP7qlyb0wcQK`ZiyZ0(oJ@^MG&Hi=>_HeuorGuULB?R|cJD08sAte;K-8_n zV5+VgH#uZ)t7ZvVdf1w#cJa+L9B7@|WZ2f%9gmcsdpu z)3q*in=%uvDG0^ZD}~2c=P~kJQQIDjIEE*^2kC}*iCaINlYvhFWMz!*N<@?pN;><% zyP}mhGJW02VH?x>_uT6R(jGoFz7ouXIDFJ^_d5O8 z+ErBR2lVSRk43orc?W&9-dDQf&|3LZXq=|N5Tr#-15;hf53VbIgoyh+6z&DWks9n9 z@FF5lXxu_gTcawzL@RCftZ-(OG_VnWybEoSc}K9HsC=h{sR0l;!RP+Qu8hL#V^@ zjNoX2MD43X)SbXM$dK{8iXikg{V<+S4b#|C!_rWtpC)Y1^Z;ks;9D})=AgGb>?Cc^ z^xb5m-;xj9xFON*(*{l`>1${hIzY$YIS;F7sP5SG^o-;SUF3>%nYI7(9b>Air`14@ zqTG{3CoI3|wBn#_>RY`lPNzo9VCJB*ZQo>=R8U1^#et zH~Y~K+Yr|<5R7DtLrG{_4YbjemHymZgmftIB%jIAX>_29Z#|Clet~_6eRkSz`@`W! zt%xga#wlrIc_;B^e*rLmdK<*O#cZ|nnUwqevT*;-`EP@bd;MXg-mUryEIAiDS@Kdd zh%M7v-zTy&W^%G|%L59QgKGBZuuwK|QDo}!KWHXkjiahZtck-4YwJT-JNW%R%rw59 z${(FCYxdS~vkrt9^GxZ-+lPucH+{hTBX2FI_?N9DEb#dw=zPHIRc>sGs20zIhJ{l! zZS_0gUnpO!@RI>s6|6~6`{J|>)UoGd%!!G1PN5`WJL6{Rkt$my0}{BlL?jUN8* zp&s|HhYPPKHJYOfDlnit*=}ee<+dU=OKV$jv}%;|3v*LB2sX(U-T~?BD+-$3Z0Wf0 zlYT$FyTjP7MHLDxmhZ8sXC&|F!g9Ism}>SBhK`VXT=z#6c4CV^>PhS%%*&5V*`HiQ zDVv~*?3n_KRg1ElB)!NF8=2)W5ymkjxvA=^rw`~F5WR8XSv@Ihr6dnDy+-e$ONsNK z90z}n@Y0%p)h~Ak`pC0MrFTl}mB%ESS1)0TFoxDje4>Gza~u%j3)!9n1?D&Ju(xyQsV;ktTf6Ne{CKg;6QwJQyBae~rGhcK z{N3UAtnt*FsD3ifTW)l={W>^b&JRQt?H<=q9*;oN0Md-F*$Kf0SpPS6TwY()1Kayy|o*Ph=zC(ogstJoTPk%jCb}(HuW2u19uh)qahV-D-fp5Vt z$D3jdY^a+=1sN#TdDn?vX-!~4XlA36rcUB);j&IJdz`%ZVPl< zDdCpv5-Q8qM`Bn>Veun$;8jJ%=<`7+G@(gear5EYLJeM~CVj-Styy%c9k^a}owg(> zjj$RW3~79Y&>aWPnLc&CSGsSm_VZpA=17}Z_9J^OZLULn*5${_y02xcs(PmV*g?4l z+y<61wUe)1R6P1ZIfIvn)_>DF;%z5|op&<*?EWCwlKtW#Y(L1-P&9x{5j7=b6)7bJ zT3&ulo)~VZ7JMSE{t>1mc$gy5?k;&(;=7!l@jaWSYw0=K8WkZlQe+U;@86~9@HOar z_zj}7-Hsc1+~32E=lr;^NxBv_AGX_F-u-3n>8LTE3Uk}3ScMawvS^4!&7}DED}-yM zI&&FHNw8b7O&gVf8)~MOFGd+)WK8qv+kEgJ3CnF0u)akRJ16UJ$PZ^hFk}3i9aqIr zlvZmAH6KOdWt%H-r{?x4k{As~xp=sQLJuNMMtg$#RYP>%zvQ?R&_UdmF{8TzyhUv& z<>sv?1T8^Q5};|cdHxgH?o*C}>N9ru>l@}u=OR6P<$RXFArSC9<&}%hVI26Bgua*` z;G;Mk&$x@~9wVTniy$_5z5L4^zW$TTk7@>s<+SGP(7E(kZJvHgM^>slD#vY67o2TY z{eDwkm+49E$8R}SGNVTPq<$xs=AZ}afh?m*`?h>}9vMAE+EqyWbBeWk%V>v|3}C4P z460FO$}ojf*79cDxqiOUa_jG6gQrc*-G#G3FshzekF-8>z0;N2EOe1{GwzoM+l}DZ zWml6LZ}-x#%(yp>AARF;c<(hl_e`?o`~eI`>!5r_vA3zBn6EvvNgeUN8LgE!6squGM$R5YcHVI(2a7P`* zL4Cg@IN1}{6wwSr*r+5(?NQl@FW!P#fKY2>FUd$(>7Kk-P%S#5IFjAcTi zE)onab)nARTO+`4Vt-b99_Mowi~4!b5-}y;9uT#M$MHg z!)mAr6=;Xhap0(&+*V1=#@hCw3T|E7i%|EbDzMlbcf*YM^(Ob4h>lYX) zx>X$4cG+p_q;kNN^WbTFiCzBs=~sZGwUMuMue;@(i$9JwYb+S=zbP7 z>rn>}UhcFDi=p~^n{EKXT3r3`0?Fcq&tK!q`I3}tv`098xB@2kuT5{{Qr$91X;xi3 zR6OyS1l46WqQzf`SK3QCBVl8{U)y7HF9VW|R7;3o6WI<}zUil!FetcfyLLjZl)MBV zkD#JwBS;pGXQqg2-YL3Do17nqfe!iHY2^tBqvER!WnOOs;$Wg;hs@hqABP7GmMU!WeRkVtG0{nM${E-Vz%uIL zc?ZF;DK(Q~)YcmDw0``~4QRM)k2cy++BUVl?iW-76~?lzl3{3+?1IrAos|L)V#{doxDK{^gSdws$Dib}4}e+*XVo5NggY7s9`}SnC8eo4)UXNDdMu}@XyO&YgyqT*#jkMLKVDxrQ@hjy1D z)guAyNR?l#pNLe#Ub+j^7ux3F`Um&zoRP2nu%iEpkONl+5vjvAG;**H z9K7mPX(s0)DD+M;eiAj!U92FQ`K(&n#|f*~J#9|3Zl17bxADsLgE|ar>zqoXOqR4h zw|dA7nfX|lUQm1~(S5Q*nN@O8U}a5rm%|r(R6z+y(>QD6j&lZOVvmV@?2?vYrsF=c|-KL7sAE!-iD20ADXMs zH1;p%Jx%nIpTuU;DgsnJaxFKT!m;loe8wdRpZ?t&5b$Fl^oTP&-osKm|24)~cep^c zCs~c#G)&{bP%cf!>)eskd#I}M`W>9%_v4zyJ-bJB8Ywk;augs>2)(+ zXWoq8DjsNLh0o4zaqp1;dR!v7W(3T6JwrK-&*5 zM{A6xI`q1R(XxX39Yjzadi^rI0b=u;-Ss8#nH3tZoj4_J#odIWl-$&_`sMX5pH2JG zW*C2X6SHNTZS)kIudjqH&XjfKPf0GX14gWezOrACrBYiL+v}f9K+;^rI9evDX}GCV zO1BanGM2D<82@VK-R6b9wu)%yx3Y}NB)DH2r?#`6US&0he1W-k1DZfDyC2egBvetM zmhw|TbJcTmg?F3g2Mxf}D94hwznq=Dz8x38H9Pkut?6p|g@~*nT1qY&w*1|llX12- z2w@&?G)Xz&=*_Zq*M(WcFqAGtXq&QR7~u7K;oP!&js@%TJc9Tj)yEn{z0AUf{# zLRIEvu0JwUf0lM5HnX$F!%P_#EqE8S@0dtzBlKA%v`_!ZUq-cJrDkxIVYsSAnN|j# z&5Cyx9qzQkH0OmQk(YitvwAfuP4eS+B9#PHa1BdRGr=QZ;R?s!GG`G0}3cm zCmVqzkbvxILdpVE-<>NJpgW&R72f1_h-o%_vH8m15cBja4Ux){^jD5K37%#gXae%Q zOe(y^gL^bJ0>|GnpQo8Gy|f&@s2=2+9@C^!hyf7Ho_S}ndQVD~3k0pD7T@OmG)c3i zI)Ry(6dk5RC{4rn)U;)W2NhToRSw6S3DoyjL!G~1n?#MY#IX9M2D2dOb28Cj*keSb zm-+6;Do}#nLdrslK^2lDB%YZ}QP{Q*CTBZj*2A!qm>+qG*x=7<;5+H4DAOJP9DW(C z-h6HiO<)zf*}z_NT+@E{!k<3b9FnX^owc?~7m_V?s*|GI~{HImk$+^jJ}uUbh;$@PWI zCxc?ZOxku|HT9qWl$5%8)}DkmGb+^L2tXpWVa255!gesmHYT1jRKGUeKYW=)+ZkcK*-Oni9dPZU_=MxrQkr$k@8{JRkb6EmC7T?+P zX9`DUci6I|Yg_->x`amyTy=lHcEF;)y>YNF2sL)L!X8~Z+BBcOeM$SlWj9>n#WnG& z_Jm7W?J(~vJ$y{XTI=%6{bgW~@j(x_E_9bV?N0R#HmM}WFP6GYDnn?~^TB zG-yy+c6qRbg6D5vleucyTXO&{J+Xt=XP@@Af%ynW*C zQO1=9xJ-drR1E`pZ3qIEK3pR}CjB&YyV2fdPO0C-fy@tz$XhArPL+IB5X>ol1)ja72&f2Gr7=rBeOi!BnJ(WLkrMGukf9Zvn1i&h6Q=XGr>fvvlPKf4L8XfFyJf7$Vi=?T7Kd zU8&Dkb&uDeZAn3PQ44vj3_(1tAy%RwWJ z*_?PnLf?X2gO8}1tlJ)YP^zMy`~Hx!CPK3t=vhCmqa@A&am)ZUkQ-*oRJQ3@73miD z7j2YZYt7MPJ9uaPJwhhu$~ncZrdh@NSr1r`$+MKx3%CfgZCXmWrWUEyG1XacB_m2H zYHByzl>`*tbM%68)mAFsEyQz5OP!w3vdq5)Bw2I<#hcMO8U<3wW7;Lj1ng@jSe&#j ze3L+<s%W!JRt6u=~T#rk?Xs>{~5$Z9QfRG3KOdp}d#XlnQQFf(4D zQ#2HVdE#5-FiQnhAdzXROYLtOAwGw$WC!b&W>5&Pz+**K{a-!^Eb|3N_JieVk84XZvLyM^j??yt*UQYizvZ;>BB0IUL4{)K>^J22=)P; zE5@lpYU68dWUwA(1?LAxnf90k)dDldw%QlXayTj7DzJ#)WAi{=5&pt_=iaQenqu+%)WfA6_-84b)SaUPF192emDai3 zIZnkG8Ix7?<;IY8nbUO-w3L0d8};1F@9(-mmSbgG&h;=?RnGXQ^%*DFei)tUr*K&? zEyiRH@OZJBTIGlAL-Z2?K1$ zw)b|nA$i#2C~e3|{grAaxr58KCO_Hq#{kdN?2#>99kc2VZaW6MSV};sJ%KSj18c#=aBPlgInmBKXjTZ3*4ua3K?)3Xjzwa#)!rD`zF2 zAKzXZ@dOuDk`i+Jzu&uX8AZJIX*o=LV3RQ=E#Lbb%Elba5=Y}B z^>G&8e;W$dtwP*wCzPzrFF!h(EDAMPe}*?KJwE?!`KK*&ybWTSIGe+{P3u`v3+;qL zusS{4y1R(8fMVhSM6PX?PI+ZeysJLZ9S?+%?W4G=y_dv?Py>$kgvUQY2#BLgx0Gth z%|v^{npGFPr!k(p)0?Lyu<^%Lc%9tmadX~Qyzvx?W~Pfo9^+E2IybQ&F2Wa>m_{kB zj`4@C3myeP+B$Hw;)2uS-)`c>Z@l`5RNE2$oi~ie@`w-cQC8qNx5E(dntlS^)y8#g zyE0NjsXP82KcB2tEB!Q1M$M8}U5dS$yRWK3NxO+kL6!X`#!y+P8$__FA3r5dy5~)^ z`bMyC8<+v1#+h75`){$YaxA@A(HcU%Rb)7B@~s10YpHus_@aYqhnp@4Y%Jg#-oS&R zUx(UFuE}Jwl_} z!~pd~^VCY^$P7S+_)S+%r<|O#!CS|H9p%(zsYhIRLr!Ub+sBU*uKNSdaP=N*@4uI1B#4Q2PyQ&jE4E528bjQ(hN6}j|eNzLYUekOh+ z7mHq&uy!+rba7xlE|nR6+zWXg`nYEPal2#!6Jj9=M2MJ3bhvwCrn~W`77=K<{9`eoF z>h$6@LKX*T{}!o+Ji&{AwR-fIm7S1p3U?4q(cnfzph6Xz($WweQ=L$H|~|3HUL5B4;~ns<7MBzFaYY0wR0 zc>y~fOjR<{4bhe|302yA9J*Zlykt&=aOnq}!{=8GUpP#oi&{YouzIgkVI=~*+CTiO z^fBbDX@yjSxs*v=$2|Mfp4B-N8na9|*O0_Wb;;R3z6j*vd|k_OnK(@oVfTLiQFRFS z9FyBpv(N~^HARvs`;krIPIaOE^suJhNON;_g^Z{1$lo4MYL0#}AS*(V@We-gH}^f_ z9oDBL`NXj)W=)ejl14K9xqok5uuT)RD%}299X(MtIn;Gv<6ttkP`G0aTD)e=uUgv7 zhaX7ijbJ3tJ8#nSc!6=oicy2)PUE6c>D^iNV~WT@r?7C6lr(L2Q#3Wdb&XGds6fAx z3yc`E&-ctW``UccW0qysMj`9}_E29d2n>N0t~+KR^HD=L!d})^fsT(jFv%aq4=;(Y zuQIYqSKWA2{N^BLDcyD(oTdEVY7^A6s_E3~{?;MKf*1What{;5^*RR?p z8HG@~2T0{-j4jAH40Lfkh=v`y%3%4oxIL$ro=Pt4r(_Z6g9et4kXWqnVghiA2- z(X}#|bcc;Ny`Et|&@bgay}MnBVOEU(NCKVKGd-8vwWK#5NTLQ|q19nwPS9l=k3Xfl za;|ciVJx7Y%xKl(po!Ep*m=+T66kK%reuI5&hY~^;7s&RwqY<=chcXpZ#x$*R5J_$ z&corl+vJpmdv|!@L3V#kvWS-2ydBN+oUDhYul6d#FE5|(b4Byw%ACqZFFHaZ_1?jF zoCZUSJa@&D$lmE){C1gv&9drVNhR{N)e4+F!TDbiLZS2X6zVh0une34%OjIN@V-ZgsK22_7a(GbWU)a(`OZjqqDzMK3avi89#hNQ$u$ z9)txMM!qZcb77Y$CP;D2jf*M2sCLpsr6!fR!|;v6e(@vegP!N$@-3#SA|xM9hXU3X z&|C;*$$sw0(d89PAMSBu-gb;mm;5!PrMPXa^t1D=Zz4rp&P<`FQACsqb^5jqn--!> zfg(-!H9s53cr{WUYMgMqA4c8aO?5{V7d^ z&W#A;Qz`_!SznlWKUz{W{eB48NI`C62vjRLIR9>St_mgk$zWBvYd06}TL$kiE%E6+ zaBhkUSQzo`LEMU+Tc23JQF19_B(GL_jG8Y)E-8MaW2Eb8Y=1o}l7tC?z^c65u`(~V zelki@A{|v?MjDpjc#wRAU{c`wA||*#m7Lqj7MQ~~CqFXL8!ew-9AZuF8)T z+&v)4x{BF};{B^f0w|`@om8MnFDK&AJ6?T%NDoUiNpXAY__*`HwRh@Wf})0Lax6az z6Cxkf{+{d|z1+Z5xA)nv%i&PwRaR2UOf9+Q9n5AsAN4j=I7=ED|0Ci{`vY`>%d5C|wfM^(|W%ZQz17%Yxwm1FW+_dGx6 zMrj+BlGhKz5$|~j(QTqQ5B|JkC8&Z4`;~;WtfjpURzj(bVcQ{Wd+c>N__^_VV&p@y zx2KGkqucJi%|4A5CeqFJFsfb$E5@FJ7Mt9yYlgFVm@CZupz;B+zkaELGx(`a@uNp9 zpGrS-YH7U!NbhgB4X?CZ&J}MVQ{p+N0fhb-acWl^_(pY+q;d39Lx0*V&)(e9#eOb6 z{657mHb*tDi|Rv8a#)?7TRXRye@I=B5`7g;#0enLC+lXYWJxXc^OII)8M(%;Cag?~ z`Yh0PX)(_tm%IE&NIW^6Ob#mj1?T!Dr=7bAU}hYeb3vT)pczQNJ*$`S?nkKmBkD`P96q69icJ^9=o7}F!6aP3;NX)52niT-@*w%l-iyHSZNr`h{V^uP$JuZ2sVmU<#l=J))p zQZ!({Wh|;XUyWO3yJdmo>_Kh?;eA$hffFtCQ~RZt@@o{FIr#NLEmoCEXZ^+%53`wN z44yR|g!H-A9AFa+JLVn6X1xe(b4l>aRR~9jXii32FQ##=0$gY6rO1NsPydmP)=7K= zG-|lAC7D`n`1xI(j|v9~y*IY9PPpy0RYAj@0KP+(@YoF@ zGoss_P4<0>ef6ng7v(R_kYqzWr?zI_eA}fiTIez(Yh3%m{T_+8ju1wW`NLaDKec-FdAOo1W zJ=-~Ay`XtF_0CYJQ{oxq6i*0EvhEWOHQR%Hk9%(a+r%Z6rfCW!I(*Yh4Bc>}O*tX5 zwb#$8M9>5`VAWSQsbRM@9fP~>r0uap1Nl|rVS62=){X;)j0=(M?a)(Pa4L=|`mJGI zrY@N>uJDJFFuVy0!7g$DEf<$Ue~&X2g{5zDxiYJOM-C3dyNDV+n>7pBfDlT0Ltvg< zxrC^1^D{^+DjdT1HS{-(3thr4q3;q%ht5J2j9d7$i%XV-(^f7{ARb%G`8gyAzBJq^ zswYJtF@&HS1>p`lOpqfnN<_v}o{1Ml7KK7_g;#IK|G{1>BUOkFYk5t(bhn82QE#k{D zj#N%wG-sRboQ$cI&dDz|e)^P~3+%)VZ)E8}Qc=!)_-$Wb(3%zq$z=T<8)hM}vd^C? zWT+)D9#!2)@fGPN&J{d|Yhp)Q*|XZjk=H}{Qg+Kfvwhv|o#hGYwj+gw6$OS{CLBr9aU?s-;oUs7A^CV*vjpu$A@`pv22lQG=9+x*5yO5DZBqJ@$g{@ zgCse|tuu*J6`M?XhPlNjZZtH*^IFYpX-E&;&^y>|2}>i?Elq%Nw4D3F;>{9Bn9f}& z7WQ#X61L!Mp~`tlCwO)vWNY;Mvn3bJ)C+Nm{dY+!c%IgJ^R$dET;$6Q)>?FGuNL~D zD^0Oebxq=X1~7-nkMKM7HK=uin`qS)sx+P_we~E!BF!QY?un{kIvJw?C(Gq2GIDJHq5JWsxk( zuCZKa&IfcAgQJkI-jlPblgc)hLuPYx@*#^1;~28N5KtjzlEy=T z@>A6Ic!XX(-5^RFK26b+1xL`537uQ2o|fKcR6TWzKBX}i!6}m7>jJ-qWfYm7m4>ce zjuqsDqhlo}=*SY?YW}|GG^XHQ#lYxa)!{6y^ypNy-ZpSK=@bS;vRxSeLQv(EgipHV z8%PGwR$6(p67kZu4rL^$eB_wJ)!^2bK_nm-z6`%5HPT3Im$n=SODfi`OT%fA#utkC zcV*_$Rn1iEpkC))r47hF+9ww@hVMJTqLJYHITup8C*F%qI<24K#Ooi6p|gSEy?$1Y zEIX~}E%{X!THa#xYV_*sU~upOa~4OXsath>`Hp$6yR#D194hD9TIwvWbM#$qtdjKm z6paiIn2U-;ax-ywC$xC>-};*R-?f8m7_AOU!%wQOs)ye>-}|W$wzk%T6x(%%XzyEN zNXm`zK9gNxG&J+EWSpx@QYm%-2djj|g^eKvL9}BJeUoWfwOfAsv@NG(c1y_=!X+$E zsh_d6%ihw&4`c4ltWs?^vLGgFuBAhz3Z(_3oZy_J>pgJ!NfLyodJJ93w?(s`qml?* zi9XBAX%HfP!}4cRh=6uE%6*XTGbueiwgsUnrD1J_m7PH!rYyo2Xvn`na3=`)>O(l4 zcjggT@(r1HIT^G3bO4OJp(p`ZBVaT0F7v-3vS}{Q44+9KJqt|criX>xc6ZB`TSV*c8Cn)DRy8PN1jMa)WX}$a&gHQfZ@6+ zt#pe}<3QZyh1XQl;v)XGGeVA_!o?$42-)66m@Jy@FN?@KXWhnAj9_IbFeY9 zwy!tID4QI(_@aM{7LMUUz1(^)e*GKWtin&4z(S#lMI+D+kJ_uLxp&yY7k|uR0MUrQ7QPSXG;p`x73qTci`zY-U{lEev_kJh~L571_SoOixOTw&K9&q zFUE68@b#(BaUa`uB94hn=WEZ5@OFiK+xHtxTZVNh)0BoAMOz_5)w7DBDbel^DhZ^6 z2}fPhpB2Ldn5V@em2VJ?%BPl$3$QWA?W{^rH@Yr$SOYS&E1(<%wZrN_;lW?vsUZum zi(AmsKqpF;$(B6>5jJLIboVobw~Ia&@Z5UZlS?bf{cm-1Xee_+$L}7MRMV0!+5Fb6 zjALl=yvnJBmF$&D>O<2I0aW(%W>}OQxsT|wJz4cn_N=A83DHIVL6Afs4`x}bOx?mM z<~1Z6NEM6c%s6yS7Jo(|zy#*)ukYrxT(NVXJZVfv;3?d6ey7BWby`^s(+Vf*&H@4=uN# zUv35%4qvRxj4aAsAC+=rbNih0u$)^fmStPDkJ?*QAwW?B9kL=6_v1sfJjZ0EkF269 zI&|26hkUjEq$ylPc~GfX_v@jmWW(c6KokS5Tl*M0_h%t3%kVepNeBW(IDNQ=Xj?C% zUmltHPsnNdnG>P#4r&22M;yqLZ_&edo})7VpiKLDT4^Gq&j6`gJo)t9W0<7?wmT(qyRtg;;sYt3^`AJdrY$8-kF_j<)hhSZSrMEQf5+MSeh98&bH^EFXGjwpRUyN`Q(L$s>^y`3~3}A(Irfm zohM4q;l7X=&UtJ-{8PXv>m?(Ls_Eot!)Ns80U_&=-1SF^B5GMo;fSA)WjdCKG6SA* zl{YtFP=*|mSy6|9hB_4TNZDFw2UV)T5;oXBvzw0=0wogc^E)9Gb>hdH*s!`Y5!AYp z3~}2>)_EVH#zMN8tt;6CkfI)o^LeUP(d*dH{v&^}>(uvQ5-z{$Ej@|x)LToa&9p@w zX+FId1~#vB;|_bsIWgZYt}^1&m(6D=$COiY)oo9~&hz{sk(U+J;5ySzJh*H zWSuDG%g;B>pz`?|KUQ63R~5BTY?L;HZ&4@waX}*M8dXz47%~ndqW?HLk1aW2Ad3DH zIV{OJXLe+0K;-!KX1wyTeR>)s)vJ4ZXo*S@O<6wuRv7~u=F_e0;rAL7RL#aru@TNe z?6N83ipu23mB|*J+M8vDmVb;ZETzThPQi#jOcK?wz!{6&^WU?L>ZFclPZu=WZVb*C zQ<{T;{4L5xRPWjdko;J|az-?LpkF)1A~%o-AAv~2`7wlJC3|&97VQhlZ;Gj-a(Bbk zo2h!RXWB0Bht@yb8{6|evPIi_H_mqW?5F;Gyd(fmp3b{Zo+ps*TEs&^h?vj~CXkg) zKW4cU;-Z4xPi#h`=PhGx=<+#Yc~d#VkdQX~$JJhc%GnVnTO<7jKA=?j$_wKgeQ^bo z!0p6o%{q0A6pM~Xec_r3OvaFNsj^XF!yntZR=4oR0EVZU9#E8ByRGeT*bl)BOC?zd z%)Z>##k%5#^X#c}tE8_ZWI@i?rmgYDR~rp9XF83a>FkRgkt-FcZ@#G9ym26Es~%0S zCm`)?DEyCoe<9DhNge~bjKzb%&Q`qPzS?vffcrV=g{+xj@bO~m4j-lQ6#0qtdD+Na zOVH7b=MX|wGuj%P2k-T%B>eEu{pGUw#d7~;^CUOO=E9%7ae5}={BJPFtKSjZXQPH( zEtGDlXvqw7M0ii0xq~dHB}cAseEaHAo37WV{c+IYm%w*jVd$Q5j88s_7zqP9_HFl}3EW~{i?*sDC}nf_sT z9Ujj!VmA#^lI$<;#cB)3LGAX_v=fpJH5yqx-=KAm(6gUqYnCY4_|G>b6d!7xFQwh+N>F#MNce>)HmxM+GL0+(FKh1SDa zc&1)I(bO;2M0$vR{EkPx)A(s#6LMjHt+GzHm*rKV1(#8vMR% zLQYuv)~FA5i=rvWq*)H0g9^IwkJd>7qBWEYUhaxnCQlgzx!o%W=nB;!PgKGVUMI!! zAn1joR4*EpM~q%nl5T?0QBdJj^O?R@tRBl9Dy!T4y@rcVB`2 zzE>H8c|0jLQw8n8i;^AY$KR2I2|H;{kWQTZtRuQ$0WlfZeSM>KGoXbJ(l1n*V{=n= z!3A34TMds(zd-PA;VNvjLqSt=$eQ8*~-duz71{m!xVdRY9O0YdgAt48j=3Y zpUChu(R$`Tf;=tSn-+f)J6)0j*1P5$HIcTUAD+ZjCc1wQO0vF2&VMa~a(gu^hgE{T}nYx=S{>d&sbo7<#p zRSY_JDq4xoU{GiE!aJ>D{5}m=;rSz;&qe4xmcx07Aj|OM@*OJmGRkCNSVn3Pw-yL{ z$!Dca7>#pv`i&K7TpY`56=jlq4aiz|X;Y5WF=S-*k3{ESP6;#mu%OCgPwWD58k{kv zF~+LEc)ug*PS3TPt!TV-9nhcb>|V>YA16$`k@so?bd5QqbPu0uqk zG8yzGF-fwEFY<^Yd-att*LEUa^e&<0J@j23eKcM+BV5g^M_S66TuV>qY0p}rLg2V@ z;t@ASG=V6*Vc8bV4;SMz(8ROtRTDt>?1ow5_Y^ci_WA|dpvSN@mjVw}@tvQh?s5;< z6SCpWsHu>ZNMs2nj(CXn2FgZpm?eq;xzlD%7OQAJ^g7vx5h6 zM-rLKz5SU#Ns5x=V8U|-av2T0)n}WMK*FW?I)Kpeb>PDpHHWVshS5m?$gI4WOezQVf8UNO>Lfn=a zQ4#^7z_~137DO3q&;oLlldcOQ5Nbo(amg40|>AN*qnfm|9nnAz55G!yI4YWEhr zrfIy&ej)M9+*?i2M@Wa~elu};UuSAt*0JM!KZCFaK&?ju-F{7J^)@lfd*s8=yX^P9 zmu~0hlk;Lt+A*``Z{#E@twZ+JPYrE&;aFKI8jMmhA~l7oeaVxI!%TD+dI>T4gEu^D z{1EAv-*RlYn~f%Vje*OvK2`@y#yI?hI=Unrs1S2w2!VUOl+@5)K0O?rp6o|+nJZ+B zwmM*LKPxOTdN0EKw(SS*72%`bKqq`$ykx>w!t9$Wzo04$NVEW-E-2j?n)1x6HubHL#aMjQSwU*dFDGVTyg#=qtrGb_B^oSgd`S|Kf7fnM5($;U zE%h_Lj6dBv0!a~pILr6V5de#fzi+&C%T12813cFyN=&oDMd5FA4)2f4)28p8!;1Ek zuKX!x_d03CrA}BlEvE?o2+snaiMs7rhkBVc4oAVhSz6N$Ut|GKl^MFHv=3AnW4xrh z;zWELX7>n3-W{K5Ty{4o`MpX8_`03c#0+ZYCb;Z3CLHgC=j?4X_XKEII58UW3S zO&}3OIhH;~!i$`jddP($1NfDWwtodE3?Bh?ecE(zDzuwC?qOhl?@H*7W4#O>J|1g1 z7$0^=3|949VN{vP7A2HzuGXB z@t+HncktP`*1SFnB~4r^`cw=clP8}yGd6Z^$&&z0U`oD~$>;d5Gi!%?W#b^OiHW~36RTUcn5729m zGY{Ns^6{C?F_L#LpS7{?z->Fjy z^tP8Bteuwv+-1Qv%B81#RSs??$`dzB6=^Nv?+ja(ce_idcptm;LKKU~2ts1Hoc1^J za#);=;b<98+}E*?d;kIRy*H{O%Q)gfac)plSQ7U0KDitcPJeRFs8={rHBfp$G`kR) z6d@5SH^W3@cZWzB2*_33LPF}Pnz)vjJL~a#&@xQuDKXIOihLB|V|(3pgGIJKF{kz@ zm9&)*wQBC8mO5syR*>!Jb(WC#Fc~H5EWZ}!*kI9m>>e~n2i{L_YN?b1rNFnUFPN9of$w=KF^mrdi*?)VCK(jn=KgP-o1$9_{CD{HYIuKo6E+Cvmjx*C{rFNVig@uWTnr8d$1^jcOYaqOzk zMjsna^A^cxmlcIGpb5=h$#H7t#EIUv>Yn_rjS!hm>tX$vdC|2umMF17FhMRqaT1E) zCm_?`gU3+FV&HG@((8c*ZG4Chnec4H0&a1rx0|7}r9MHB&>d=uPW|Qej@PF?i58(L zluyMJ?xR#Zs*{&3zaT8|oZY9;k`zSx7>_*uc23Eg7c9!gKKha zi|zx5{A&sqAC@mnl)ySGIkv7htf6`p8yY;^xB~obuwvVF;`)-m+YP?Ck-E{aO#Zj)xy-Xw6_^#?qraW3A_HKw z+#k0=#gNUM-~ap`{EmGfpw(Mb48c)7xPyt-%*JzgyIr{P_`s?=j(*pXhs71n7VR#u z8^`GFjoFIb=ob}d;#Nf<^bN4yuMg!=j!nYh=HLT@7+$_p%HD>k2B&#BGwPCJ;$iK`=2Hz*?nC7dq;11wIE=IU}XEK z5tv_~=f${-ySIV^DCKI0Qi9fa4)n<{rHoGlGLB)8U-JujXcT+~cZ*PbA;TzilaQeT z+=`B8;YMk$RC4rTAEq!9Qrz9fz1KC|oAMIZPoeq-s=bR}(_d?t^Pv!%k7+R_;QiaI z8u^zGUo_V00c*|atHQHK@b9fnq1?5nR3r=%+z?05jyP2xMlW%|P(xTUmIO|Tyb#*6 z1o8Cy7!asNWuDMS3z<@ROqxE{l7RYDt(e@4RmEbhk3^q+^r{`_x;;?Li<{prSdP}% zRZ*oRZRD(gE*W9qLOM1|pCGfxZr5fKOs^Q0`daA#zhRA#hiIAmY#>EH(omn|yL<6j z^UkBvXg1RbhE&7imDlh=iL|)U{Xt!=g-kCOyRG8?*sm?iOdRkf7wL2pc#poAH$iFr zGOowfJvkHyP@Wl%BLabI`X(wDxi4llql{6;x=l=S84{DXE$Ri*R|Bc^t>o0x zm=>H-l3GFAo;zUq{3fc{tUC?bHctHwIED9Y#Y$3e(d79UGk*Vwn+-Kf+^U4^sMisR ze)Cw@I3N+Rz$Huv}AWpP`6b3Fnt&xr~Sx+I~{!KX2mD8}Ulz%!+ zyE+8JP*CNk7iz%JJkv1cQA5qHnS;!52&c;%HY>T(lD=W#bYYd_u_{-twc@&@PD@(h zE`^piUx-!?c?V(gYn>fFerz;-!T=pVR!K*;I+8Ay}=c$k_;Bp*-StJFG1+hAw^-2FGB^5fNym z{rqZCGmibuBpf0!LX$6IyugaH&S_1nBNhR8YpNoR|ke@QF4SG<{t6mjDErig*Q|dU2h{n+) zo1w3)gpWR@H@jpOd^OAuB5gdMl51G%Gg0xuqmlS0I=cN~%+kOCm3h{?x{|A9IWs<{ z6k@WAEdjh2epcujx#LS2C3e}*T!^=^H=fS;?7V zUR|-{Rok0vmDnlW*Kdne8Huvi_uc||Wn1|>Y~U8q4(XV@#)(!^!(XZGGvc;YWDCxS zMPqy}b*Sy?lgpAXT&REh1h8xBn;$D(RzP?imFH6!C8Xrsn@UztpkjqwuuEIB!lYRL0GFkW%w$LahAvf z>V&isuDLbUbpr`zZ8*_5|JGhAm5oz{*p3b*TJB2Eso zQtb-$MMt$C8djyy7j`^;Z9C=Dhs4wB_4rh_S78!ZtpDM^F~jVoK5(*EOhqeyYCENm z!vMVOGzAiaEF>D}_)>WjOsYU1<_Mp~2>KiwiU;~w$iz8UH95tDKh*n0Q{t;^L~f|)CuyMrqBmcKunkQS+*aUI^g+4FVvQlj@>dav;-%nhH+6fJ^|DPn z9!T=2JH8vnff8|cH^e?&sQCgVLAnGERZ>THJcny?KObu3GT(dBYgLipmEh)w6qg8O zone_bFJqK$`3xm?slR;gS?C}hh^RPYI8XPB+z#?EwW->5S&C0@&C97;a-i{W*NG*? zz6;NTsrW?nyQ!mrY&3=?B7i5td)-}JHoar8u_j&y)gc`BTvmCVOF#mFN?WQTDYrf7c%R{Z`2g&(Hku%^|dHR2~zXa zw>zty+bG^jRYmQVGU(S2I}i-MoQJen*lAu4s150W zp|VGR&|57Ub!sUYoCK@(<9-<1Q+lmx2q_Lf~%oE^?#?ZoO(0G z@`v&)fcyd3s1*|KZlYxM?}_x4Cg%gjZ=}zTd23XEEU!k$=!(RM(QtBI;j;Ni0ku&a zayDq{P&F@yS50bJ(&opq4<_a%M}+#$M@Y3S=0JPD6>K@JCaf}vzn<=0Vf zG})ALAZEvC_XHB=7CHqaIsSAQyNtckdul7li0Bq@jZK{HWBKxfqbL#u*E-M2t@NBUsJUWV4jpW1w00>64aqs#zAMW6mtO&yJcZqi;uB z0qTp-MVN6>%*pw1{Ecc|iZwvu5_@sS9Y@RM*s-fvf)%`;Iac-T`RcfI&O}?W&UuHL z7Xhmjq#h5*d0S?Mht8}xV^CTC*V-#10LGTOOnQUyzOzcA6l2x8E!%P@%(^AYA3kpn zLiC$1iq61C-?sXA%34f>t^FCm{B_xntp!Mf@yi8G(J5uH)8XB=Mh_tt zzG;`4zGkTwl(=r&%pEw}l^ZIzY{SDKtuLlnS8ua?!7XC?YnHChQyLykAkE4Ug$P$q zw9Ln7f`KIV3AdqTg=YY$zz zre?Y7$k=X^gN7)w9SkrjZ!c3rlJBC^gCT!D$%i-xpCu@-t6olr;K!3vL1d%b_L0zI zb-Y=Qcy?CI-p7v9=YY>G?Ij?AOvB}^1_n29OFvop>Ke5$6^vt!y4l(M!F|y;N{r6k z1P9FqILv&$NsEHK4kn)FU&p621o91|Z%uQT0L(^FCH5!vLw_-8%y7dAoAamb@5LqNz+Hj|mk*TopmDZbN3Zjqrfn@&tb9P(#<6btfj&`#Z9PjRr-wv_HJQyjZsmi(88} z=X7rn?o^e2n^zEP8{B=RopKjls|Bu*QFuBLBub#zA~F+PT#m92oJdo#-JWfXiZCEg z^a4n>c_`oIL^}2nd&T@Us%rEcSj+KOe;IxAb!YShlOD!@)Kfh5Zyy&FE)2%F$CNRd zit@EvmYOVq6LWkoDwHdA*)#+RoIK~??_XLB%U(MhtjK9m8j$0h`gKj_1Gh(_-T!#| zdPP!hA^7J~<<{=rw(B;kA%k3zp8ip&7vKA1ci{gCH3Mksxt+k$CUI^l#nqzWhoB#EVpK ztid*49=6!(E?O}6TNIb-nr@0!0biThE;Ao7{Dgmn>7moA2K<^}3F*&21UV1)3AlBz zw4p3BP3+cj`?~q4L)@}cCe1)547Q)%S8KUV9kr3~2wT}k%uGd|MsGhPCKnq|ke8=z zFKl}frU&7`Dm&$u&`4SAJ&ImZe?hf>Dca}xgOXi2Cc(P|#A196wSm%Zf!1p_Te0_l zvc3(2Q<>$?Bib?i7!6KC(CsKTM2M&dJeyb0hpf^m{d`wYmr3I(h58UpW^U7gaQKdr zD=%PQ?Ofc%)!Qx4v+;U03;D2_2d?bhy~PPnFrq(wr8#7e!njNa0ENrhmMsLf)fOLL zCm@}dr&3#s*wKNuks>QFz*P$8?*N-8G)J><*7HE3IlKmiW?;NHhl9k*cWfI^TFdnP z;Gs>lFau@1&!uND^(;J@^2u+=fYKPG|YnGTnN46A@AdBFeoNn!s_Sh zE1}DBh{lIhqz|v$@cB?P#>a+3pDrmpwVzTXRDV68ndODvNwdr9Pvo&+3bvXUW_@r} zyG>+HcX#t%6}dnr8y(d^{Z#;9ByK9}hpK${XiNN47r3&jmMCJ}$kfR>((*@I-QhOy z(39DYyUJ=RO?<)2_uf|0VnOR_^H6!9=J zeo>|+Gmox!O-iX6_}6FETA4&)yxR6 zaq9L?Kk8+F@b7+FwG{wfk2lG%Df&Z6QM1HMPe-%%hrQQ+e4{|iDL5}PAZ|XjVyYh+ z1Rb7AwMF2sSw_HA+9V=NM*tBC07!2cdz^Dyg2U}wILk5OyXkI88pt{}G3& z%C1OoOf#Fh(FUqq*uu0G8@;jXZ?Z8>$Rw_9_mLB5nH21>Jk5~7x4qA2qq@z!OT}j^ zF_aLkT!w)g@#MHoRQO)ra(3Fiwy8+WV|R?E0zCrQ1GYegbY*^_ zx$3@2Q|CQ&wCxu0{L|X4_m1m;*%;|B6mE$xphFQ^z8O`D55>ue%y5AZLhUB`G71OA zZ3~=Fb*KsWR~Bu0crhi$`B8n4bxA3$jn)a>=74HBPXD5XxRvpiTHOBoc!WmnD*R={ z659tD3H3{;A-xgaeh{G&_FiDamArcUsT+aYUK)AmHQwYIw z6{1;iVV=A_-u%tSs(7Z}RkO&tV~a3ACC?K2L>_p*=1NC)32Sq#3KrC@j_MWdw!?<2 zUsDTl&%+~p@lfCbF(jXcmuMO{i-oVj{rYF|wJnDk#wgCMi0J_SH4&cZIX` z^V1#2-$uEyyElp!8hWaJEn*L6^UE_|ZK1MY!^4DoI;`@oUwY(6+9xd^np0g9_Qvh5 z9mUwCgj0(&7=~pJIi`PE*lf!lT9ZNeN$hN8U#AjmWw%o)57#hn-qgf34*CKxGOe0u zMPV*R*{kqKS+x8I^Lsm(j5nh*rscK`isj6mO9qzp>;6t_)t-9kjW!3UyZP(=dHLCK z(`~N{Tt>L273xyO$QjYLZT;LY+&5Lou2?Iuoi-#D!Cvpu&{A%(6K(tK21{<}$ZW&R zFp^sIR2NToFDkl$n2)=Ly>_3lFbq+o;(d#mnSKQ5m!VU;k$I z=mbmsdHUpMxpcXO^!#9Hp^S-G?zwshxIxL$LkH1TY;!ix_3#<4DqFbiK0)Hid&IxD zA;}9a{E39AX=b|IUi6#2i#JcPMRWlNdCX)W(~}~B*jZ4uGnhUlsYs!JjY1^j!*bD$rrfu{x%e$`%SCCWk@_$FSwmYkQhJ~^NW~bn~s}_UYuJ2 zsE&IG*|(Qi)yv!kDe9T?*bIrHrJ0-hr+Z(ULdS^T{O(7wUSZdwHc+G>{$zeGJ~Q?* zNb#zhwjv@Lx4A$H&%9%$lax5`5vf@Dkg>?GWf@(aGtBg{2P@f9S@1sx`NW%j*>l{| zJ1+a3l;3U`Xbwi-)-53SDSqnfW*&w5)#LS-?kY3+iM24^^ko9TRwhgz%9!nFe&kk5 zL1^_=9ak`D?05<5+kST36m9`=eF`XtyDkT7#P5>~*y}s1-PMW`;u#my((HZ14O7e`$M+8fo`HeXgmIB99Y0^2KQ^?E3 zgy_Dr6Gutl?^0g!FVb)P4X)_kHA#&R=9>wtYJ%?lRiC6Oc&?LQX5e`VFH}qJ0b}?5 zgtNZXKA!m$AqQ$<@}t&sjedzskG)@25Fs|9NC!=G{)Z5TkD7MiS|7U04ExyqM|BP< zLby{J*X=qC6IdwO>kMv}9TUrWwB&W7pv+52=)nik!*OSs+@H8d<4rTGgbBOAnxwE!b3McqfiUv*clbCO-_qO7sH+^|_XlVLebq95WT)(lVj6 z_Cr1}0*|Pzein`K9vNxmfIazW1RCOt{&Q>bO$j>cmaikF&aqir)I-MQW6_=zQ@grj zZ}%=aOK)@?7Jr@pOf9k0f21=*I|SqE*VTFd8@_!qTt) zR>NJ+Q8zKzQ+^l0;!?6^W;97Z`BUpPNB3lh`WQ7!nlN?`4sNSk@At&Y3XapJf)v?} z{Ez9h%yinaBO)1}Yu!;v0cVFOxEcMY>d6z8zdW~(UdsW&3=;lkC7Tj63NM|X)Q{Tt zuA!vfdoxOsYtF!dyLg8CCqa8p0B>UQ`DWSlgSgYSH@34eBd7uJ>HwHc?ni3SYjdw6 z%(S5-)e43eVs67&^1H>R(YGfTHGXl~n>%sDmlR^w@`*U!PUm_Enfma8)h-L+ZGF#P zR$NW0qcPjCZTTc~JG;lK0`*s5c$V`TGv27z5V7yvXlKW(I=~wYnZD-kiQIl$)xPU2a@K7b59cKkY^T?m-!X=Ku=ng zsdbQ_A(16i@KdyTyMWGAw?IxZBb#Q1W%syy4&fQ4Pjf2@+Q=3p=39qEF7MU6k zTydD|2{B>apYaXSr)JRZc(IL($aAxBK2}5%85?_z!*i>o4ZG`DPwg(pqesyo1u+Z` zlFxD~)&!QRO9B1ti2~NpikY^_g2j8zahrZvhQKXT@^`F=PbHe^D#N25_ zu@KsQ3=yh3tQS7jaiU}O*_T0oCnIUx3rs9ix1Z4bY9JJ(+BbDA^E> zeD>2|ezICWCtKK<;HhZ8ZJ4y3f(7R; z{vD3>di$Pcmh>al8B0<@y@kp4FR}IObrOpN%9aE&%40R%OEFh z2*-(!-*jLXPUJM@$poKV&>&zVAx3YPJfwXIgz}y?OnteqL>i%Ki^y{+O2)T19QM>h zErF_aQaeKXu*6SB{EruuG^6A#4Ul#x;H)s$UJtkA73xdC%2z7Ov3&5SYzWe_TlQ3} zU5WzJ73nW20o)^UTa1cZYv~z5j1VZ)2Tbdp!qRAF`jlC$4r^c0ic|jc*MPmtaQ7w2 zGS_<~j|JmhX1?Ted(zwd+Zu7`ethSw)-nQu-3$_Geeok2X-(72HcF~-9MQR4ey`(u z{{4N!fu2f4iRLYtZmNRGY*Xk)oir>XI>Cmr|Zf}9Fl5fM|{ebYvj)o?Q|wmAFv}VOQlYea68kv)CkQQrA|ob0^~4p5r;HxvD9I7Ne5b zU#Y_Gx{er0tQP`W8#qI>JgGwWXp?7&LX)R_uwowF1VT=x6lq-|#CY`vP?qJs;nxZEMCt@cJ+S;~KyUFiI`o$kS za661t%4osgvp*q3aqPs@OD!5~m1*V2K6U#3{^5et%U>RZ$Vyc0C9Sor;~7pR22nC> zZHAj+9QZGUFrs$NZ){xZ)BNaZh#+l8!TzNOAo}3%$=SD=W{at6PY>$aNbi}Y6Lg>$2H_95 zE2E0N92*X?O-N`DJ}7sT1m-8UkA9JHwF22&!>edf3(#%x;#oU7qHXGRb<_rZPze1z zZ)Q@p&|JUE_gwJ;*yZ$H`uFKf?RrURMrdTFbP6*xhn3NKsyaeh&lm#6^#jjUN(*Yt zVE;^bhw5H=^;$o#GwFvIQF)g;SnUap5wWlEO%x3`cDv7%t2Z93cU#6`T_lkwOypY+ zP9aLh)Uva0qUd>eKSpUz9z5h{JGcQ%9C6h4>wB?{%ppUo1{}AX89{nFzd2)4=sTLl z6H2Gw&E|iSsL4N<^rHx_HBS@cnvfKIMMxA>+ z-wOFKkFLxm0fl=u9Gv-s$k;conQa0kFd5sSPscso4bLUHtqIG+NPs@r!hkZ6;{ZvD zMT8-dnRWo;`mi zPR={oskv(({fPz_4LFfA3U_b^g5pd_-TEO+P;UHiLBf)sWYRtcjIUjRr+a zlO3dE3cjPu2;=5}(2~WfW_&b*relz{Lk@4K>U;+N%>sm%U&iNG8~?ea1#uvsr%wlZ zy=_m0bepzUa62CQURp({fMDlp526X+%SVHL@=SZbK#g(c>8;+RoYmCRrzgTR5_87s zd4wC#qJ9z3be5}`=wm=vET0yZSbW6X_)16owEPO=w(60np$qQbt=%JnN;#JD&Q%IV zylldo<5iXm_Fx1^@O_6a$|lDm(h3fbvFpX+AyA?!gxmCVSG?N}&L-2>)`^xr#%7$( zp}w-2b<_Ax{2&NBcdIqF*ZU^FGoml9-{VXYv}kvV6`?{IO8os;_y_y8<*n`0u3Ke? zm@>eo(LyqQ7C6r5$xY1=_BI1)q|<_Rep5EPG<&YQld<8CKX ztE8Li*snJ%L79uuWi;OIz~$6&6;qLxVfo{+p1$$FHWSE*LW`!2Mk(xmrtrwd7E6;! zu*8yXNi@%?1RKIe#Q7uuV#4?0!7Vgy*pR1jg%AqrA+ATpJBjlmkH4kdNW*D|Q@kn} z6;wAt%Q?65%y;oC+S)TJM+GTWdIe8F0YzJfac5S9^vTh$2@q9D(~ljRx!q&Qbg9w>~1=>aY5TRE4%& z-1#$7*44w99<^R$TMqK>&i;8u7NK&fOtldLw@wRnBhata69%6#HT9eI?iZa&epu6S z;V)f9@N`WbW%QEOP5Ll)RFedCsi}-X(JVw86wqnWe#7B6hHZvYksLr26Kl6pmI$T~ zJ(LJzXC_PIi)+q#F-28KXm5P|BE%Jc?Op66=h7%8&QcBaY%+|_5&hw)uuXDSlkL=& zx_QTE2A6s7=jjoN(q!*J1mf@d&-m+A2dFRy`-pQ^%>xlpz2E>@<8-PRtNA1AMHla* z4~S%McrqeLA6n)yS*0Wv>(5Y`jI9Fj;cn;FjRh@<8q#3W>hSmUPzj@vQe>d6Z@aK_ z%g#DXSXMuu&sZdp*ZaUdQS7Gkj#Sr4sv@)3-Qe0S2s@rPy~ol)#By_uWqDHHm%lq8 zTuO0!WhY%1wc)uvuv#;ncPL$Y^a;(2r2!LJHwi%M zMN0)lS1@w(z?M#gQ|FJs=c_(9>PV2j+HZpE%h9gwDPQMycdq(0+>%?YS z&qMJ|i6TY&NQLse)sw-#Xrr7I;OYSa9b`-77+uC=YW2qB$i01BC?ac{F z)6NyIM*5z$P(@i-8rkicmVS7|k9kC;d8aE%4@;F?V}?e!s6*pR18mmuEBkiN-wPV8`pX?r{KO|XCtErfpR-Gpq`z6<@cK@7Zvgx$gyE!$R+CupzCqV~&ZdG0(R?;bC9Pt`w@h|#oS$#;&& z>ONL1A}ZfSe5gL@x@MAHbKk1=7LHRacKA30m(k%(_<+^B?sE33Sq#+sjOU;D)c z4tKNYjntIaeE=KuPk$0lzp@ea@Yc~sWNLLU7a*sU8KL0Y#!TQ^n!x+eisqf^6Dfz-+zho~{~of~}cLJZSkB9}=wpQz)SQn)u> z!jfz|{8%oC>^BC2-G9k+rY8AhCUnFRReXNKS8i zEFMi_X<<{!V1h0?zo*lLIl9KBQ#7yF9}?6X$6(7*j+rI8JAQWIh*BizHKr$ZAd68A zyNDk2w>Bg?lnrRwuNmpWvB_PaeGB>N4>I5|`l{}@l_156&BS8obSifaO9rGE`LE*Y z*RaHF9yL>-aA}lbp4RnPg48YpY9GOjGHWlU2GrJLx)Y$?PCTJ?wQkCd`bKU@(Dd?0 z=t8eqco3qxeCE+wY~99%n&$S1A;YO;e?kjgw}XhcLgkD|kr=?SA5`UBm}tFx-7;hv z$2(4Dwh+?vf$*Vkc-;2l177ekGr{v*94DAFK!EfpG?Xz~X) z7RJwB3M4L6ijPtfw=UC1M?_YC$5TwkLlzNezRse!&pL+uzzFVoSA2ev=89F2p z@F$tlJ-WBU+y^}~T1=-*+D@1NuA%lxHC=ut=NY^)l#0YT(Fk6MFKH{d)DL2M7Xb9) zTNQZXlO5NH`zwsbAfDu&oA|+sQ`@8CmIfHYPWc-ALcdl?E7S7M+Jm7J(oGcKFPg6G zPNV%&q#T#klB$^MhO=D_bF_D1$Zp1B8jYOvx-#(7@nH`=Udsnx=U z%C|6S`!T9X+YJ6#zsen#Sah`fRI~At%)|*Xce6{UX5Id^Zu<5_&o-=PjY%nRGsSvv z&L|>W^I18TWpVinnYIf@zRqCCi`0 zLhm7X)_uzQxEU~ZPzxZZ1J>6;O?OyHcG~uqzv?u`U)XVY`1`Th*CX3n;n}gOCxMtO zRBh|?;?=vYJ25NhI#!^E5(1s=q$(e(9Q7YES`N$Ymu~g~j~4i6qaU4x4;%UE5IA&O zR-7E$4kTk+UsI{}j7zQ+orjylC0>h6y%T`de#qjO;~$Oz=UFSlz1e>mLd71&c_pY& zDS#s?m{SkO8ig3s+cHHAKwiHuZTQNNuEp;y_lyjBA8VYc*b;m2Mm@$Dbonb^|W zA6fjo$TMgr4tXn^AEaxaT<7)IjJ|F>R?NVo*V8^fVa?(~b;jW21Gps7&u*Zn9hl?L zo^BJiHWc%9JOitDb9In>mo;%eL_ukrKXAw=P4+niC$mMo)4qsbFbd97UMIgK3bFJM zBf}UGDHBS_+r3k!I_*S7{RTuccNP%rDGNF$ud-MD?S;a>%rGS3{4s9n^1w^#TTunW zuQ4^xwx2#|Lf#Vzcq3>>YeL+Pq$|>~CJVi63s^-B0Pg}nHwcrLE;@2JaN))&22sm~ zVbf2n9-6gkBb*)1KAW_|s$X|a?6J?S@z*BezC3*(6Io7cT$zZJ?HgHcexqFG zPNvODSHzOZ+1bm%U5B~n;Ro{$nfk6~bUHOaX%W|^W}qVev6Kk?T)?EIx6S#_#(K|i z3G>~pVMN< z7$d*gj*3fEt_t>L`jFijbRcMPe*8C1vDJ))K zWZeWVgsS>E8+Wh8mXRy8zGM@fgRyaUQV4AAh3WP-f$4eQd3}^$tG2e1(z8)M9}wl2 z6|&vMxr%ElTJ`Z=Su*kKQ}olGb-+o+UtmqLs@daG0Eh4A?W!Z7yd}w zAqvXf2(oz#a$?&6R;4HyOT7(6R@}3Ti!AOz$ zmQ;ytCYel}Aaz*)^O<)0TuwfTjE(6pVm^i~pf3qud!+1wmdnEVy*~FliC`$RNvu9x zocPHsq%3TP%nK46NasBvzmD$X1@KaL*Z+agsBGN0^T~) zlU3FWCm0gRu-%@zm{9r1!aF+hL0M`_#UjpHuLb2w6$TKs9Bnz7c14!*iQQgg(25h)%CI<4}xx&Aamr2j@Uu1TG=a zwNCP#d@bK(%U*liIQ6f?+}GQA_U2-o-t1nMA#~}D)zJ20T&l?a z&H$KxrhKGd6Vvb`Nzh+78}c$Jjx@&n^2%X8GJRKQo9;WdU9PWucs;cz8KMG?rL2CZ=DDmhbL0Cf6Yl^)(GXf=d`=bA_k z`_gD?Z0^z*I4=d0|JyV+uT%B!mk-|$s$k1W`p1FHIr!v~hD>Sd&Q6sa#}o-rqoIB| z1YC0MUpjzBobReG2gM(Yv^+}{%`~hLm{16Y zATX*`2$4pPB&zug?z8&}gk~pOc5*C@!-JLKpncL9=}{>1VuUXO@fc|^AimRMN4+Z! zR|jM8x!BzuZqN1Ws1GneSn%`xMtZd__e#f<5r~CUud&^br$yKziudE}VQ|JY!>^+5 zM-rYUQhUU5ZeS4@vrJU!O)&TJT#`2Kec~yZC*y+}h8b#=*J(V?W4Qmr^oa74m%*df z8yQ~DHG&CB9fcYbE&bDT6oDy?bQN9^3_p=Bf{!Y#nJ6zIa?4U8f{j5Jrh9L2sW#`< zI1Ob|nQL@38etXQW{hPJzr%z`mb251?PQ;;a$aiPZpK!n>aC9>bZ_?(_qxmNOr{sTF}K@)*j!t0gO)O$cJ! z)Iu&lkyV%#&_05*RzLYOFiX{DB{sf&+G(rXS)g@-m$+S^&^-5G-6HO2 zqrAK3489VsmWn$|o&_k%w#L|jNO5EhY}SYPFcy1vl;n_*VRkO*a$O#ZKXKx9R!K5B`MvNA`>2pl)lddg%R?SlhNs!fOJC=_p4E?}5aT7Us}fdqRUs zCX6Hg4vSPJi{N(Y>tTVtjKyIt^zcN%m9r&r9q!He9*;4(P!m`tekoq@A*67V z%BMtorN-1Ngg!eS7ynotWf7me1b(T{>43bv0*qZ}Tuj_#%~~2BtF#_KObknNDQ)_I zqQ(i?&&1qtprWTXbIv#fZez9y^Y;jqU05dzOG9Ucfp;uvmzWi#m^q*VvN>gv86>4O zib+hgpT{Cv_8it^?{6voB1^;2lI?Z0k=tdC9A(OpHOV`wU1!vPaub2%&3c|~n_IRv z15=YurWH_(#t{TYgNQRw<%hL^hLnu7D*o|u9ao(jM;kZZ_{A(XL~8DI+iEQn6n8Z` zdL}NQF2;+T4=g%a>DB{|KR>6LhJJ5AhlmyFmKuG27LWN4NpoN2Q-f~)PBqlm{b~tD zDy#7x=A z3=4Nv8%HiS9TW!LJ8IRKUT?X-PbUjxl+S|GE`wwyI%Fs)i+v>?(IQ{{hmN59nhF6a zxO6YI>gwsEPg12cGDwKP1>00n>2ROQ@8+`@EA}z6 zIzP85)J5xOHGb;rWK0u1Kf=05Yd_)u(H3*MVps9@j>Z;0ESS|XtJfb*$kK*saD{p3 zSIc3o$z2R|Fwofk)=Y2=8jKh|@zQ$X2htks+>rm6P*OKeW3ycBw0K6={Y?;7r9b%gH?xXJ)T98vdFtWtkP=p}b zQJ&df)#_~95>UMwiut?OaY|BY?FI;7$>}BvMBYJ$;0XgsKa6=JVtw60@QwRvWpdE~ z)dcv)LFBZGa8Fj(z|g5b;vtM%7il_xt*RGj*q%V$u3n;0)ucxX_-*vK_Cw(D+Tgq} zR@EM-NLjwpnpaQN;J#(eBK-MZ&RWpfIoASrt4*6-1ef|yCQrHt%DfD)#>Myow&r15 z2cIu>#?zU~$9)R&d~28Cu^R`n>gy4uR9c%Shf@Iz>`gSz=64v{-Upsu4;}L`6tYM-Tv@G$WbQINb3iEX_zeItw$>&IfUCM&nmV8HaWe`T@3I0HT zK@V-s3qM+DdS{wRwPuy;7n)T((n|(Nj$MIQD&0-rUkXNR4clh>hPbzb1N*gA2Bqhd z=3Li?@eZkb$u zO~8s1So*^~F6XX}N zjZyiGspppt(!_RVa0(NdRN$S{!Aa}9IpM2cP>maf#;M*P?wDQd*(;j+4}W|+zt6U)>4Akh;Il@`~b!^geFC1MlvjJqNpp6E|GBP)Pi$$ z6#DRR;g>*?RzVKYOGo>bk{eupJ!6{_&$6uO`j^M5Qxdel{hInD3vA`^sE~D~oIK*^ zi$?af=>X5amLUkoF?9Hd)jg63ux#*n?KAbJRUsLr8OnK_(L_e1fjRS)<4*g5OtsUB zs<+E&W||aEQdccKuNLw%lQ|+w7g1LVNQmejg{M75Bbb1zo0OyZ{mXd{; z-1F@S-@F?X!!D{R}NZ6)NDtievFaxO0aO7Pf_qLB=Ls)t{^H{`Al% zBv6*P46Eh6wS&Z6hTuVZHSQF|0%6B~&aYCWX@ujE z8c#@YTu`j84?1$4NHQ!U>&Rz)4NZ}Je8!yhYp`sb4wqpUwHs$kvDOxGgC5bbaK)%&?LEPopp z7hemg5dIi$p8Y6tNk&pr&BD>@QlE_UfNQ>V1A+M0yWFDnxERosnE_#{A*-4PAgAjuyIuq24<6U~DZ z|2#ht8b}G)sZTA9$zQF@c5~0n<j2oy-03wB>ZkPR=igV>!wpVN_zVZXNKaM&8R|<+%brK!1fdbb3REhM=tYu z(oV(+O4sp}$5m&g_k}Tc5iEB}Xe{p0k~*{E>Li0agY;tKV~(~K`UQSyy-#E3%z1*f z9@JT}ha#9|v{LUyYj0rV!kOm<8C0EMa>$uITBsmDhd7PRiD_$6DNzE7Qej+bw_8y> zb@wkdB6n}xmw=Ea_?myjueGg{jEzyF(~HA-`cL$A$Tqw=)VfgHg;1bTo;g~(<;yxLT{6Wbmk6m;r0UlZDbwS3zT zOr!-@=vl=vJ@&rq<1+p|F|Ho$Rar8buR*{`6IHq!Sl8N(ae1m9c0SW?yuR7DbMD7= zT%I8)L*MKjrjjkbkZ=ofYOx&NTuyd~$|kotx?G!tH8u}9R3lnA`_pa#vl9j=3Ft0fOCZN# zE?NA^{X?Yj!?2JY<^@!sbpyASfo&(9dx4}%tOF9lrS?ag6v)7q6+Z79o_w@T5!a<} zM78p9s;2R-gFa2}ff76HihMS;5P0H#1LVn;j;AzKSoQ>=Xebi`T?Cpv!Qm!80+Bqr zt!pTQJ7|Y-N-DIoonOhP)-xk|M0;<~g2%@}2KZdq%d0knbMb=Zg@EYUwrQqB>U`+c zz%}K(=?tUF!Av6lmit+8``!uhsB!7F>K~+9@QAHb|Ect5&BYI3&8!?6lgR>AW&v;N z%P*U<9`YHS-FVXF)4trts$NB)j&BVU$fzW}O>blun>R=Bfq<9~!-;WE|40`$L1rRR zNb4#ZS`lQG#| z6sJ~uionoF?r<8aQVy9w%G5 z9iEm*_1QYR{>)KIcWit3z5bmdsxX*1rpr1g&Sq*X16O6KK%Hzm(R3T^jtbyr%zZNw zc_#|>3xnP5t|c3WJVsH-(lM_eUa2rPY0(X^bC-X(?}5o1jrcsW4u!cJPlfBijdNAQ z#Rv7~xgZ+?%UH20l%gqg;miumiSDWT%q4FQ#~xuH8*|ApCeqeC!&`+rav5MHhc8Nw z{+c|fd8tmYY`%Ak#cXAjcCJb&A-A)f(`zcPTK&BY5MG%LnVStVJTZCsfk;ld*+ioz z>*5oAm2p-v0uqL-Zw#hLRm6J_I|TW>kxkm=5d9Y8cX}b_-r}DT!eDAy51<%VUT5)T z&Id-zn@ecZk?)LmcD0W@Fm8KA!K@_0r9X_B(DgUF!fh>=GAr6@)*oBnISkS&yFy%nz1;91CznHtgaMe_tH3+q?(d zFk6(M_T;sQDj8>k-0!dzXl(%5!>%D3GF{Yrcm%2InX4f4;uVpk31ipBzSfxeCO;jz zy?y1$;j-Hzrj}}KOt#~o88l_v&-Drehv@PF5hTgy)`hSEDM|;uzb6nMxFig1hpu_X znnLq-$y0@`*cLC3&*Z=3lViif#lH0o9D-&(O8#Iz1c-j=lS0A$m z>}i6&`)Ok-!Ekzke~&g>{w0BCXt{jjaa2rKJPn5K@0q-z%k02REG}WbPzP1rdJB%x z1;oHV{j)}tT*1jTk3u#!9B{v-lh_#IeLtny_o!7R4a6@mS**f%_Xm4ewtK&o25A+YFg64)_KnJlFjWuHIFD+Z;3Htj|=)u-+* z-5>Qja+?0k;R?C;k;QV>TK|yx&)KqwqqJSFz4dUwlKoURy$tMALv(cUqwHH_VipmGO!k!ZfKL*{N=r7g1n;_4Ye|apTF1lhs(EiW zy6Mc>xMr=M4E{D!( zvpP|EhPztJp}2c5vnl#U8YV$o4R!tI#v^E0EbLG|k8%>=ibiTb^QB#9Ff-v%Fix~A zgc>bD%ns38KbY39E$Gnf_Yd~+mjLlHKbY1{RN}UY0>8U52;Li3z&?6(3zFmM2Z}^! zwj+M5@I0BZwmYHvtpCn~FX_S;R5T4KC`MK$T4>j@6GM{{ZkTQ3-sIWRK7Y!O zj8kMLT?Mu4{#iX5Nlp|qps{g%@4aM^ea|!>hYgkF*Q8@D+bsoybC5JflR~4KqY%zz5oTBVnu}aFb;WiwLojEO6JGiE3^J$=MN7A@?apj!) zR8pXsejo-DM6eR_cQ1ML=+Se!icRqj57j?{))8#GAZGv_-?Fe$5^;uPa(apt%g&;hQ|I|H zXI@8D2gImnJ{>zQi0>#2iPe33<7veuAX^e zi+8tV&3kWaa@2>l6~z_V3O~`ob=_o%YGJ0q#hy_71WR~m9&NM03VPobmO}eOK(2lA zA>T-9ohIEcvu05hn83#`E1t6$lk|ZY{3Cnr_t!1{LmTTyoQJt?bz^n}N46kkS@lSC z$&AJW$b9oI7_I;0Wj?Q;>MrhD)EZ|G0R;eZC4@*yqi8X!>u7GlUss9{jTVoCesyK# z%OLTIK(dl&?uXW`o8CRPuE?!#VLEN7seEV4HYUXu2oh%5CU_MyRV6%FEqqv=J8$n@ zd!||HvAervS^aTTenjPM?tg{G5g(hp!KKZ)q2_8Ok@ic(Q?08)SK3HnawHG`eac3Jw}OD@1?>T zjNE1kMd;**wV2)epySm`WOr%bUglZf9Tf@mAnVnidRJa0M^-NfXr(T$qOf3Vss_>} zhn6D98OL6HslM&os5MhJi~U0^rL$LPM)wh_OmM%5Xo%#JIu?)S;p0RLvU-tnMN@6c z6zV9i>SC+#l7USKG87kN-M?4NAmiu);uPsNb+3Ras4Adv8j>^kH_B>W zr-h5N%kZ1g)Mk59C-_|Y9jc2sDm{EgVw&gQ2N0~|BXee%!0%JqUOR1(xDnP!!sXsO zARYUPwyzJo^pxTP{dcGz(}vl@ElDMa?%?&H+SRinj4RW;yVabsGyUq5Axa3d=q~#k z640uEw%SobG9U{N8E75qc~8tp^9}Gb7*?pIK3g8Yg_jaL-kh9Yd+y=fd`Y*NGco98 zQbC7#)^ZBC2=yv$Zz?l+Yz0)@{FL26-(SOz?>TT7p^?mn=2$sa*O2ugNx;g$KcBdR z-Dx)}?0_^e8@HU$AbXjNW&9*adWle4C!wC7)tVpNzI%FOqkmGBg>l0YI^{oYJLZ%a zFF(levi2q7kFm(Mu)22Z1ba(TK_^4s>KR^APk4^sinB)S+(nBG>)3U3=Rb5UrJt{xo-tf)#v;TC~2vKI5!hJ_krh@>v$D?uaUQYabd{i;}&YVY>Ray<1ma=m0L^x+1o!T=&8@h?Pf5~3#eW6&I8oxt@MTz6h*c5?S3R$TM0ODIIzt*WmhM;5PV z_xV%XkA_5_)6cz+xkqbScXI3_YT?zn1GK-A;8WjnVA9N5=9-|8bkd+mw$bmJcO&ZZXuh6a2?>ed?Qz7SW@XS_cu{@#82+#E_afQ^|{4 zASpgFu^daxf1+bB!0%skvKa&Zgm*bpMe(#Q<+BkPZQ?rM@dh&Fo7xQ7HO zedHInKI!=M*z?De``QrqPsTpYh#XexJ!PG3dK3MfE+s>p$M^4b7CZ8^e`n9nqCHMF zqXOkUZSeyg9tkUiqKSs@Smo;6b$zFj&1YOxvn~IKHOBqS_#LH}O!?Q$w>-{b(*kMF z=4yWF4rztcjoa2yxo>I&MdHtlNzsY^GaHt*lXzOM&d;^%*62mti?Ozq-Q1;6hqPI)#>L*%yzz zB?VwbxgAp;d@t1reLh02U(21zNAYTm`_|H7-tFKyH;G7Fs?C{uc+%*UAzw5-QUT-b zOG-E`f#h=m*7iDXG0>4oU3h*_t?dqk5R^x6dN!19$>|{8?Dyx2U7xaQ zFcs}s>ne44mNgP3KR>=qIg6ai@Bs*}Jn^Zzlpa3plkf6X(4cvAGK=UBpi5#JZ$_Ye z`nH1xJ*TUw)KdscTi%f$@e;-bQ)an?)@c*_>9pT+-r0J(P)&x$)`qaSmZ0SdFg)&L z3KA0vb(JbQc}UqK9i?e#Bd^d@eR@QX@j01V94}>Br%AIGzAqr*FwqFn-YJj)!m8RK z-ap~C;}FF(eZ0iKD+SYVUr#ac2>iQbCQVvaG)NLzVK9U&!{%WZY}4{KDE$u5RwS37 z)N%7c*q++Y`C~>2FKy{X@lQ>zw6khBx3mZq5Wc25ZK14(6L^!^*5~goytM%RiJ*?$ zJ`1s_^;x+VL(;i@ZsTH=E0h2^K*ql_bSmP%0sRihI1&X3&8yHDEUJAXvH;+(R{fk1KoTelL9TNWlV@fBhvVbJBNy=R2_{CA{tR92GrF zrR@=8*YD2_X^hp`J%#LmnylLvh@fE0z;_ zZG?XgyGzscceG!bs^n3V*I@XG;@39Ft?D-Q2WM8K4yU9vpt+A>u5p|MUuLHT+bqEX zOazrN`dGf*MVTJoAQ$|c#+}EWKrej*eR$_=(jSk!JocpO^&@-mNOV~~oC`btz3$xC zL&Ix@B<3Mc2ULBw>ELT;SKM7^`=*c$LhsGHF5Ktqx0{x*t~efZ%-j zD_p~1#(f#3BM5#%=9>j`iqbAW_%Z%Y8eGj5Uh+1b8}(p?nshucyZn3&dWK*TsdSvA zs~^M1+jg-DGqzi=A8s?rcp3Rw);900i8TnA6Jc?S`pM4S3I4Fa2M-ZUgi^yRpe&ma zTkJ8!+friHi@&}~#BbizC=g}sK!$M~F&O1~rWxTw>W+!2mY})bb(^hNXPx20g#dw? zbKTHLQ^GL3>r3EF;wU4Y9_+Op#3W(`{kQvSg1cI35(C`IY^#+>S|f=dnB|kOZ?F8F zuxZ$9z@G4j=uyUMGHE<29*4!AKt<)^wU^V^0a4iB4ONijW=g?sjqqz4AhzbtHIX0% zJZcZL6UeQSSoiPR*p`qidu%`TMaLX30pvg2M7Tq6`+~ZC1FyMq?iR9)x}6<`YG(S) zE7BBL?UL`RuW+xqE|C5B;=Ou}Cr~nFiLBC6vME$veG+^$Bk^}iMfhjM4xj0CYN$ZM zSkAO^URh~usVwr#5v#e!WU!BLF!HFZlHo3p-owM z&eOMH_or$*!F6fE5VgepT9JATNvGM7f={CTi3E+`H!&^37O*dlngL3*2}!k1cRo65 zHe#Jo=1tBY$@m&B{JyY92V{=;W0NV@B}W6ypsO&l8x1k@YYH%E&(pLncSgW|u)>i| zqLd<~tK|rvLE&`aYx(XH&B4JysPucYz}#W4Lrh(3951#+;Qi8gx}(ihs)L3RM2&X0 zV;;@L8w2Hiqdx?~k5~75EBQCg;VIZ8*Pwf$m*IoKF<2gJK6}6TwzRLGYy~ar2ITuC|3Q1ZV)pSeRTzS0_cY#+Xcqex z5xHKHRjEw8TlV{sY2l+PrL(e&b)lm5PbGt|F{76tF` zK56FHKV}VD-yz(d-I11(I+&-akudhp+oEpd;zo`;@R|sxGeYxZ_ zQz`Sq-PFhTY61Fpyn20Xf;Sg%_oOC;FX2(yUSPHr8x=ykt!TuO1ag5H0Fi}JoSEEE}b$D@J;?{~z;zZ?~dr{ZPKwOWUy ze4$>NU)hWOfvNg_&YLkSc+eb0t=CJ*rFc)L-Y9d`v*z)E!&J&7exG#bo_fnpg5Hv& zPMV3E2dp+)jtlCWpTI*ou6nH`9^ErL@O@)8TUg-62B8XQ(tw#4TbwOjKVPJ?nv}im*IIM$>)K~ zdgBzgHKZz+xE~hYtZM{YPUG`B0iA;s0}E+hNZcg7wcVgwV7TF&mK?(4|N5Q z)E_HD0hjWQg1HY&9K4FX#y)OOCUEy&E+EJB(DC^ZO8_i3Jl!N?X<}{z&2_z}?DX7b z1{ongxXMjNaU>84-yM5UeeHtzOTxKhR2I@8Ej!{PVnaiI2lNo(ZC&CEc#m4=G+yEK zDbM}E7QrvtS8%gMv5!eNEY;9{>esZB25y>QkLX4(y^ZI zft~y?REX0aqoO9Mcq<+bW-+aiov7<;03Q}9MhUBsdako(3lq4~>=`YnXye;ny70uC zh_a{SWx@CA8tY4mjQ34HpLq>w6f6tY+O=}NxK%&d_**J~9uBdPv9J>|IpP<}-sOR@ z_8His&9$b0nTH1Q$K6RdtqvN6UhH(z9_}|>O0`SA3CJEM21=Y}(f!u;p(?Y)qzX9O zELqjMxc)Lu948grpMez^7rnT2T0(X(q;bOh_%R&ySYn4g^|n`+-&FL# z*`@1|p#z2}oaLXA&Qo)+(zSQn9;n(K>q*3fq{GDz3$@eG3j7Eb8JN{sPg3U*4U+&t zMkXy}Bk1aSSWLf}V!r0-Q+z?S0^J1FcONi;LFBL1a7U(HVI20n(wS1FOET|D=WDxE z6z?8r8HOuD$|`TZ?{se@rpm>$)&YS!5#PLau5|(j8^qjEeL&7of?X05jn=WB`&#oy z@V&w6;k#G`cXY@XXmUggE0%*}QS)>S&J!L5Yd_ z`DQio>H2qU(ZR}Tj=l6EfBd5CtH9LiF_oK!G?lPpWn9BSpG|0UJ(`tp7PrJ<@z~DHG!!2U=C;yYwI~^M>Xps zTM!RQ=D>~FXoEH~D_f~GH#1vgf8<_d*jxho?YMjEBA>Bm6cGeDh7^kHOPYn5@s+1qJ86o%~zttujqL2O9&;GyWWdMXB+zAqT4Gg4f; z>d2u@w8gpMqa@qHImC4^|0Hf^ei2o+{6&TlmqP?nA-fq_c9$PWVP;R*V52bD=&wLA zRMPEJe}+(JG#}IMzD29TY z?<`b+1lc`svb`A@fPBNRNInDIu1734ne-9)_6mt^OHG~@_aYp;6fuZ|F?l5WWTmLY zZ_-c&pwt4o>-cynNJXcJ11b+&ImGR>oPLs9G?e|_<6+00`=-?Rpnvu5%}u5)3|wwL zUhXh|e^%7~AZ%{h(pcC9wS(%q&TTVQ*1*ZY=XIHdjpzs6-}HJHQJXWkH0#~Qau!-z z_orL4qHt?W@_`sIh#WbXQt5u?ey zJ3$=HVqMLCS{;7fioC+e+z7sH_>;fhP-H^*?^M)xk{Nr(5vK5ZXFXWPfo<)1ii2BN zVO+YtL~*K2L=65BIjDF2~8=5J+B0n&7rM*@XO&Iq4fP=B~06wL?NZMSN*-u zXjV%ZFFxXLXG!L%BS4fuEpfMK3QQ~bPg!)DH{HHAr$#-C;~VmmXW3epLZ;AEqGT=g zkk2)Nh(d**aLB-4V~PkD!3m4HLJHWL6iJNJBz_U71IJ$aDrE2J}^`lDjAW;q!=>wxcf1?z2ZsboJ3rNo;_Z7Es;*5~ZD|X>CX8xHcXEv1? zcJKOG$daGS+OEe#<)0sF!l(3!&h3Yda`Ju^YY-bN>-$`NPJRhL*=3tqX@*ch*U>kK zUsfD!n(w38X|l7dFo}*HNGmrthhB3+cd{=KW8Ez@oX^9Zm99i$%#KiYivane)WmB2 zqVFWQLlIc%l%ye~2o?7w9HU`)Mq`df;rNU4>wCIRk_g){3KT9BAT+t$z1iS`2RJAg ztAJMz*396ZxLoR!YrN)PTo(=?1z(S&NOpYkosrouNMrI`6-+IZ6gioJ3kor0r!phX(f5T+f@5-LUp#Kq zHVHS2fe@mk+bAdZem1F!wPgR|S@Gg*PNT+;C8;IK=seL)->4FuIc(-t1eBD0!Ss<2 z8SFVvFE^eFAyv#H=ZiLXOV!gCM-w!Bs$+UqW27^nQY2_qJPw%6x7FU%!!7ot48i>( z*^hRaV!{ba2|qIRIui0ipYWw!N|l9`=!;DH$r*Da|A z?xYG^C@%xv%9zY*r?Hw%5C^5mTAe9( z?e+UY+$Tl11Uu|L0cx9FxV#}E_o=Pm&=2k zZY%$0xa3aBwWQ;4e|%$=5RE|P0p8kq)cqudV%g zcJ+~+Yuku_yFmX5dYoeMNt*P6ka+pcwS9UZlZoVh$mwdl>V2hO0C$_GA=0=S~@voldy@yBj z%&vVRp}R45p)wU_x`(jR6@d>!kx%H!aRz+Q81lw@_ZKp$MwBE=l^#Kn0MCV#ofARL zQafr)kAI_IAfs*@XUWLm-k~D!udMU+NpXx^aS>8#S@>neO$myW2cyE}hju@9dU`e0 z+b-I(xwY7w-ZV&t9(_4L}FY*vZ6h3B@pC1^9UdvHl7#z-DQg#KS23G49z}SZa%TYI z)yc1TuR(N%=OJ)HpR4JlRB{wIA+?Zi$kb-E(>SMc`bQdkf{inn)3QnmSTYH5$+Mgh zNh~?HLUBwU`UkHkW-spT^VOry1Nl{_8BdMTb?fiZiC3aOR4En#oVqn}JbBC^Q#qV%f_ zX>k@Gy?%1k)ttw;GLbl1*HWAEeaOCcbZuH}Ht9CsEZ;Zm*{f?ShS8V^RHvf zeMD~0P@_qhu&&!B5blRBV<(ZpKF>c(*3)F!CB23S6VV5XFAY&)OOyLyqNB{|L;3Jo z*j3hni_9Ldj!Q~H6C8L!sCBpH-ijFuEPq@Bbu7RkHZLcHSO9nr?144H7zyevS zIEHYUDOJKAA=Gj}rG8}oj(ey(nC=f*oLb*RJp1G^oi-37h8O|*-E&1gu_^?0TKg+C zyFFg5szKY1J75!KOwV3W-^B$pjeq*b^fNLE)O{L#Wy*zq$VW~U-xhYL*-x3FeX9Nx z>zwMZ6>+3py>GXh=li@7~#?6t?o{ENX?GQt>W2$c%GeV z#!ew{Z#l6k#5yTt;e1dYE`3O&IL~mZfqh8@`n7xEFkk-we8y94T;J ziN4wl3nDaNFg|>G!VM6F9rG0}GrL#4@Opeyc6}{r`MSJ_(Ie7-lT5*1{FAv6(MfEY66|;AzFggv@ z#MM(*$60b8`V$1M&UjKO_=o;kx(0{3!fBF^h%v2j-05H84Vf5E$U2?sIn9ki@`XoW z5;7kFo9PZMbCgXkk|vwHtR=1Q|IJ+fhh!(;6YK{bF#W7s57>@seU!?@!&$7@ZxV^* z1d2SNAL{Z7QY%M-rxu+)JdNo6AYVX}W~h?ph`gptTZErOEq1Gx~J-@?u ziKnC%Z#W^p!(l}V?d&&O+gh<*aPy|f;m1vAK{oviXPw5|55~jWTX9uhUx=rhg>i|Q zJzJ;tWZXD?1aW{oHeS6m(sTCLjM;rW_F8|mjAQnRy1A%7fFBy0P{nqJK z7l`Dly!xv`1@JUgGE(4mvjo5z1Xg=qQJ(=g{h+u^j?jZZf#U?%mP0S9`j_V^g4<)`=Sye6puF_dbRVojDV~I zQ@InzUYGvvy(;*;rwU*rOSLtyZ0`>tsM@bqoZG50$y<(3)DDkE5B_Fd^gaB%j(!a_ zYv(F-u2}6j%l4UNI$z{iCD66+n``-sowls7)@i&u<*?23$2Y_VLq6@#q7N1cdB3^^ zdR+a0sm~;EhNC`x9k+bol zpendRXPRh^D_P4*A$l0d*3zvvJyr&>?Udg`Wad-YUq;-j?$kXOUd!(w_NYrGQzJSC zcyZ%wsj~V7ZEGyhOQYX|z4DlHG#>od#?&;W?x1Qa|F;~!HeEvmwK}4Wq`1OUS9S5} zg7Mpd%jZ{3(@wg=7}gVmJV-KPzR7pcJ+vGd_FNS++|kR^mU_YXjxb+L;^u@QBOi?9 z!F-ed5-Ac-Nyv?UiS+kpJ@mmbR@@VQsD%||3dhB}1=a0l|E`JQD}STCauUzy6V5`~D*c)J(Y z1#)mtn@v{(RwLt0S$O<$ZCSiO#H-oB6@8YPOLR9sjJ`tc_mH#CT*R9ywq-?TnR^5q z?bISNe$JUH5N}q~{`*wsecppmv>AtaQ?oD*)oS$jT)Xtj0%MW$+Wb3+fHJ3nMvQ;C>L{D_h0zNy$-NSDAvG|S)@|etFte=>$Ni!I%FZe7W?2n{#=37 zP$p=A5!1#lf&yTM^+wN^T9!QNqz=c;lq!#?aBdfT)lFz7+OkU?$K3bJ0wX zw+wri}E%>sXmecj)KV zMR%%0$~1~8yJv90;J`{V1v=mjVX`byDw z;$CMV=d=f^R8}tYELq~(D-HB9I3au}6z#Y5?GTG714~Cl5*ki7Lj?Be6EaTF281b| z^Geb3klr7xor{~$yQ|lra~uT^mK|_=Ih?wL%4lx&6g+;!%s()~n1a->IMxI;);teD zZ?;EAAdkU?1&ESO&dlju-}H_r`INi4bu}AaW^6UaR$bzzN=sw{fv07om1JQ*V~zS7 zN@q7Ov5S4Vy%q!sWs?amy8GHZyZIpO?_wF8bZ;|H-7*YI8Xwye6}&F85~Q^uQjqTS zpmr9CsPwG`J*IT=zLCWm;L;9H?~ zL0A-BH_yr-ghMCmbds%_UZ-xjyH+h-r9S-@@%Ou%E@{?@+#EzJXRP%~WdtFAjcWZm z-y1^c;VO_g@XQb&Uz(*GP1Pp%oNr-rVr1fACOR%F&}`w~QY{op*TD|Gkb3*N8#d`J zJBNw0#pYMJWGaP)k@*rmsM`)54=~m(a^qE#V_tfwtGI4z4P^C^PN_o6-=vSPeZl1dZ1y` zCMrEM6X1G=ETrKAtvB$ttgq1ZMxc?Gnoee~g;{5W*3c}B@xtPz>=01%nd zB%1hgnG~@X*dW5&sACzqfD%+^YXb5F&?BL0UNtRGKm`PL-D=`{jV6h5ccJG~8iGXv z56%WX6oF4iXFZf`70F4JkNFp+%m9zYg!Yf3vSWb2+91bxug4?lP*V-n@H*)~UjHh! zKaXIEmefTzk4&d7$j4g6e-fMoJEGeYKKJ2sJ)V&5+uuQPfE(~Exg0S{cV6#kL9!UR z7R&eSog>94o*hITMi!~2dzwU76Bdtg(=$xgDKWTuYx=tf^+oQKCs1ZH@!rJjlay-H zd^(&YSdO_83R_zA%n2U|#ICwr`J2Q~$h7hJ)gr!n-uXpStz=G}HQSr1Y_Vrj!Uqub zy>DJHFlSefcVcsp+dk!sA!1_GmiZ7+4EdZx_1#?<^3E+DLn9LB&vZYR>P0I4R3ZH!EZy<h$f|{EK?SWu&4gO7 zJ)HRu#md#p@=lEqg!A*OWQ*~t%S_>53U5@9)+^4U);I7>F7uSsPdf6tx4NH~FqRg4L zb{2FUKP#~B9g=!FksGi)Y>RDZuy{(Ky?bn@k!ikuP^I~U*H7=?^qtr@)=9rh%~mZj z0!qt4Qu5O2N*+bd0b0A|Zgx56?6RyV5?9w58f8m=6t%|7M_Q)tk^cSi(?VH% zTj9UiM7ouWwgz1`aLEuiUeJ#aVL|78SA?~et@vM*hxOoRPyFijfMnH>JLj8Zt6%95 zwpk6H4JLx^%O$XW9rTC5x3B@Y0ekoRCbZxzxh1GAxzKXD#JKgi#2s1Xa{hXI8O!Eg zPBVD;;h8}2F+NryN70L&s2Mp1hf5vzkvhzj&zpgAAS^r@w%;-&W?}h2RWf!oa&$@u z=}Ld!5K&NB2`=OmztrP&lbEek?BD0}=4+LEe>2JTK87rxrj=49SiD)qlJh`cuah3} z_|Vz#%kO4=Co@sTJf3MMEf6$af(q96ae|!ZV+UHLG=jd0Qe16~&7q&pa7))+{LyF& zZ2^)R9;t|(K9_^k>j1xk^{9Hn>FRG<9hBH6(&d~(QO`UDa6G4qz;|USqLdwMX zeTGLDI%2#$h0bBgc7G`f>p92a{;BeKo!$&Oy|HByO;Y#bLhwK40?L#_v3t`Q2$r^3@z40k$gykmhl-;;=L zGh8a&8ccG-T(}W454A0PE>Fh_8L0SKBDLWo3<+U7j;b!i!mpjWX{M$`#IXyR7E>31 z;G(4Aucu|kIpY!TWTuz2ZtPifg)eHcQY7C?1jdKMe)I0*xo)jI3f7jF9)uwJ*O~GU zv+C1T$9zR#{VZ)EZT;vL-a1IuL~#?KtJb(<+YhJ%AtG5E`Kp|Y;(>E!Te8~^GUiRf zopimG2+dqec(BB19o*>Et!X`MM984hEO&-=MjZYcC+DggAU0UU56+O9&u1$*d+k8SgbKgx|v! zDVjs^)eDHG_9Zv^BlZqp)!e7mN>eU;Xb&aCIFl+#c{Q<`^&J+%yM8m+zaId5)d2Wxfd^|WNJ@ya z@6#zPGKg3V_gaBn$;R>?@d+dP5P%sB_OQW;I2}kv8d=6x)oB>85VxTOdNmYo4|LcoO`%-vV=c`6C%3tGw~IhF(<< zki;r1@pKY-TAnBWu5i;Z!*8W%obB|7jk{PGiV5f`nt#u&QN`e255ZvFr@H&2-h6(# zII3~l&MRbrXT{-D0hb+5!x#>(!7;E)gi2no`ZkrVxp01Ax77ml>$haa2P;BqJsDrp zQ3s-kq3u472@3W_jr%}l7O|cf3zug^2?=h!e_kpjMcanNHAFcA=vweixrYR!z{Toh zJ5~;rYarPyJXFe;=*4%r6SdYuj6cDFlw=!%fPXUV@RXB-M^!6dRlF=jBK7 zxG&qR`2CJbgPiqelvg+GfGcyRydL;FC8uHgWrFcOB9Vu2P41VT$TQmMp;wmDvSn}@^luL&V#2dl-BRt0dtsE^d+P|Q;BC2H7~DH` z2j?=rUF2>MO+x=3PKcwQRFpFP#7JCs48!8bUeSKH95Uhlmak);VccDfF_%r4eyog_ z9#Zh(SDA0xU&ICC3d|E+TM;!rfg~fr7&X2beDNGCngiFR&Tn}fyFZdduV!i=y#ju} z!e6i)(1b(3Cklkvt=?S2>j?umjt=c&(I!-f0ov) zR2j_Oq(nbACXgaJ3@Y>;*U#kr*GeosdTT^RGxru2^vAnHH zs+(RAS{|w%?9d2XYNTgldWe;`?Nkwn59&`=iG0v$_hHZPPzXxOmw2AqrfYiXsM6>O zpAWP(mW?3h-87IX<<3d#DI)-L9Dn_I;mE~A(M{hPcEic=rQ_pO@9IotjQbAHTL%VVFH>Q-3x;y*j5r&0Stg)na+rn=(Bf* zW?@--PqqKZaL2}4@Ob!xXm7IJVZ@n*Gj-1~!sql-KTQwLA~MHDLQ?z|fbh_|U0!r$ zwlEM$xpO@Q7cLNe&Jn5GYDAE9+S?N20;zPEb;XWHsM1ctoSMGv|OqpVG0O* z%GgfK`mi@ZIh=*~S$Vf$-Vug+^|vsheZ+h6<9^J>dR9&hET!{}nNRk$8Z`n%;j>tB zwP~lkr6`od{bC9(!m~Nvh`73(*&JGEkh~V`L!0{4C0jhdk!ZpYE6>QS6{tAj)|>Ysy#_ zs^qQC(oB;z6!ctih9YPQ$c?Q<`#nEM6{JzFU4gt(bgxX^WTXqx7=f|ars75iaSZYFNF|%kr>fckE!r97B`^6qZ*Jk)+8zNB*<_rq*dQC;D3v$_s+3&Rl+I0 z9E`-B>eth(TLmJ=)$G8B7eVMbNZy{=Doc-|5-x)P`_-Rtg8_8h_`(yDt?J)eIE|sP zg?8G+IG?zIJW=-B4{Dn>x14N0x%>mU9KGa-nC$=&QZ7&yZ|>0tV#=DQcRQOQ%nmAV zpMPe0?jkU!Ur+}D6O60V_dxWz$g#1z(n=&8)=UxdEzb7TRnfeI6r!XEh>-8Aj|ALq zlAh}w8vdy?H$8>!D8C0EilkTJU@aau=iH$=#Y&(=>6UMYVx%RKBcuVZLyU~3h15L? zSex%><0&v)Nb*!vXm=oPC8L_ms9uXu;0T}{Dol(B_lwil|=hJ*vE7lJNRoM5tgcPyi z`@3h&PFJ7nZwWe-iYsOCO{guK-NrGbOPE8!jNrZ`w`ZAM5gwkc42snXH}?Z`GR=q9 zV_g3BknFM0kBlq+KG}le0hneO#cHP9`qW`8yx3QD!EHyx?%RE~^LOY_V%Bu4G|}Fb z+T7WJ#9hBhi6uhqLn}<(FwO)+phENNQc$!Q^7Cr)q3y z^%Ygp4H{hC64H@l>ENQsdauG9BJel(D>f&mtl9}91eU4L5)#)Sg=sarV@VSeW@+HT z?S1*{$M2T?c3`p@%X;&L_Q*Q>Yt}XG@>;WIm1bqmI}lD)O09B;n#08E5*~WrrF7$^ zKO^G}#!--=O$cR4>_;H8Lp6Z6*V%(;P zerud%#oF9UrH@F{eF34B!aBW1>Y*F>G>3fUK1R5Qv>urAg6Y*N20ycInY%(kw;J3fed*|86`7xx2>7? zNoytr23@qEvpRGxpfd3s57ORG4Nan(ZrjD(!zm)?TBV%Us*iZ8V58CKW8a7!U%ImO{_AfyWePDDw4A&!EiB?UeU(aob^P@1bZQ~d=$3C)g}9}; z=0cP$T4LSGe^Miy{8pmnwI}(QBjd~W5=cGQo|W0d%Y49XJ<^i=RB;F1QxakM+9;pu z%L*Gu?9xPhRN-Slv>rZ92IOOj7*Hw`d-i>H?0yklJ%z4?xC8c9Wo&Gi^BXP8GWQmT ziFc;%Zc3l##b?mz;Hpz5FpX}5q;Ww$qsP2W-g}q-2}K6$+zVPnYh&m86s6*7!NGf} zgDGp+=&zh0Gq_PC!=d6(vUmgGTyjN8Aiqj{qwn?TV6Xj_tOHo&V7j!%v}Pw}7a(6S z?0jk_xOo}Y9`K7p#}sBJtnvM{mG4k_A13_--s2_1M%kyz7$j94ILdu$Fa!&^W>V$& z1ZmQz2>PpqrW(~bqV!zN8a26sO;p=Lh{G^`Xb?t=KI213=bPS=U;D$Un`CL>Ott@Lz~@JrJ}oMso4QKn_LHdjMH z@j`LOxXFdz#Mfb}nhow#olb<9nG5>ur^I_kG+7A!{s=vY(`UiZq*{>p{lqSEB@u`z zMl(LgS3bU&;$ZH{{eH7G~_;gcCstmo&e4Snv16YWk(<%34tB%L)O zNz-aCkA@?)nbyNZ$P`1hSx;KccZOd$#@N0*z3~?rn>20x?|Rbp1Z_s1%D3yWg$FxN zd!;pm`{Dbm<@Ex`+2wG*BY)Bd2|d%kj7zI_ys$Cc&FqaK{}@H-QJXY-KW*$^#L7>5 z@*`o<)kgZHrA9oHvaqi(>4!I*iVZ4XdhQNbA5J5cg8p3C=e3@14Qo|5!B*0$pgdS zcZ$w+T`=BV6U1{H?H$Pqv6W z!=l2x&!Bm1FQA)j|IH%%7SplUL#hVC;8H1!2e!6za-V@z=)rudYg@Y8XW2<7vE0K# zw^{v-(PW4O^UD5UZ2l#VRs+X+{xofVUYOi0PRf1TQnB--m90&_+E6ivFCS5nu_DC{ zcxkcu{^{{i<0&&y%1`ewTVCg^d$UDRd`}53)c>kjwf34yEiM)=*`VoT#^6AUmDXwN z2vZKzUuZVFQDF(K5j^H`%;E@b49L7|c6DV%`L%Gkwg;rC01B4&!;6^V-E0l93^{E8emxc5Y-1x%=lIm0A{kq-B8 zJb|YOSj&_u#Xd~3BM%KNJdS2d`!NjCcJ%uDqSuwLgW;8AJo)zxDp%F@bPqeFLsLaD zTw-=R}riwXW= zW+YoWghjPNkH0qf$w`o47=(xSk5MU+*wXOma3%fwvV>AOTf;q#<1Hadennvs5!Qi` z-dlr!Zg$rTth8QO-Lv64M`;TN3i}(3`0Ex=c(*3Ejs*A=SGth1Hy2F8w7XPosp2MV z*(C*ApO{TDq#mS9!zZ;#$%Sm1#*+ONZYURHe0sjDcbvpfW7IY}KINL4pjdczr=U9h z8d;50*-k?fgvE0wu4u9O?g*sN3!5%d@^<$u8#MusXm$?AUI|FRdq`c^c-I}$5V=#h@ zdb1fxUmh{6#4y5>1yqvm4Cn|KnHsMzQ}@7+ICr#NxNXEP_2X;A)c8OqkfkCym@OaU zE2HtXKWV0+y`@+BuzMJAq>Hye4;v_{gOzL{%M^;STTkPD2zQMFn$iF#`i@$dNiNv%mn+DMsiMb+Uy6UFI4p=R!gJ`q+Ik5x$^zKa1eBY zO`HpB3H_itciD80oz1YeD)N(XaxOdrM1L0b>!?qwQiIdt=@lsH{eiU|KolG#vR$N7 z*rV5?p^2SlEhF_IMZ@ux?Y!rlbwjUv`ysy)-dCAfQ5L)Cte&Vp5uf@n`wGDT=>G3q zXjS;c$erauO2SPNoq7@N&sD#1B1!5;zV>OU2$^z-5Q}ql(A^%fjp2k{HCvP~az9rz z%JJZ;pESH1An=0%o}R3}@ILKbt}7XDp-HW>63^pH1j}y61_xYrb9eXb+ULp8n8MC4Vj6^*+LH?dW=ngo#Q}RftU|L-39(KQ1ZOp9iEwq| z^Jpderigc+urs8(fs$Xbt^S}H{gL06=F{CJ_A(~DU)0%0HU?#nBZV*WgQ#)wat}JR zCHw1K&M8ycr_hq6@c8QwRV(|^)I>q=U9^^$`gy2EM#JjMxuprYG2j@J-O)8@OXrzr z%%vRiy*$zyjp3-!fPWNkD3D)DGNN)2uDUMn5Ms98SGLKGHDId&JE=d=@wjxbu zseJj}Z6~Wq8?u`zBrZdgSkEe#@DO10g^E*UFoF!(>&ikvGDD8O8M$&JTRW+7OnmfA z5&ycdq)l}`JY2qlXB+Pl^%uvzMN0s2&uKtQP3TA0K^GN%!`$fU>`#!ns?SZcpsq*S z%bMb>=xACsS`u*B2}ykoWfXXYq(5biVy~rRm?V3;D_g<2P_s$;smra0FZ^Z;_c2lG zm($4}i|zpC`Nb22AgsT0OCC?Puq6Z4jf)g1LOVOFp45kou5{0F(p&dkv}9M@p(PAW zDowg%QZ!n&&8I1&tB3Qq&`UpWpINKnC2?)6y;{X_npCm?GuL!>7Z=1HiKfp8xvzIf zxL@v$faqr0#ooejwCh>xz(=BB-a7Jk18nLehR)4q(GKhf&-RNB(%|C}QLlov+`56J zg%WduQL-kpdYqrw>pYUO@(YK{vW|IGIN>;=K-{TbLX@OnGUJ_Gj0_Un2q=ElPNCEl zgzvHHf=!YAJB|cgBOBSP1T!o0BIld-uh4vOZnYdSkOAv1?B`5d0%ZLmZ=-0bROK(I zJZapoA_GiVN5hr62qn$P*xN3SQlhXYkkZKIKc&Q)pMCQhJiWx)M-?ePQyTk#ro3|A z#U8}TLCu!n%1Vd?vzy?K$F1RY7AG<=}%a$`j+if zL?Gprh-_Y5BTUR#X;nVhwsI09Cf=6yLyfKU*foWIk0rF`XpICAAw*RaPz!uY2mD0P zzh@w5xSP+zL`I1CHc}oDq$;^CB(&Xdmo+hXy7s|Jva+vy_nAiDB-o1)I(yD~buGjZ zJ5Zi+b`KS(t=7au%}jK>1Kc9knI3~!GH*kmePg6mh2eVKRRf%ugn$Do32OuTlVT&# z7C3LD<=gS8DHEyaA?kcyk$H#58C>f@x_MsRk^g zc%wZ6cVAx6D}p8z#sw>uqW zRe1yc*jljpDVNgk3VahKQ^lC(4sCq=U@_o2C}CKJVo>{#JhH-T*6^c6!u8EaIflKB zX+wvHMy(u?>V9?}PtHq~lq#Ww2hyUo9vGFcl*&$`LJ*?;ibknDrv zTT2#y2ZMTdE3|=|lDVny26<_qB4tKPrW_}fwWL)>>1Xw8T!Q=i6#_}T(_vzN4p;@g zfbdNFJX`+|;64z!XSQ#V(a$~<49Hh7mj=79BepnJ{@z4h8}?DKNA{iC08OvET3TG% zqR2mocsQVa#xcRR`L5fgI=2oMGwJ1Pt~MuYITMTe(h@cR-D{q(a7875Yc&(3ZoH<@ z#p%Mw64zZqR^GGF^PbM$4P4)#>K%p0gDV z-ahM{!tNcM9W!)wirdqe$vw7^=gV}iyUyF_fqTw%G4G2N6=v_&T?m_I8Q&{MTqnp9 zw5vugW1Z;qMyJeZ$bq)SI`1onBQ%-iYg!xoj37g+$a+NwT~chP9E5_y+ntMV#4f|+ zC!2Q?TbcyM+B9HmXia_N@mvu2GRfK&*@1W-4~x=DvCx8$UHv;tt$QNG#v>r*rDpeM zxJ(_Fx{#QseK{^naZfu3a)PZMd}<|pFp3e*>k+j2G!D`@;LC$wA|Dkg&3AC1=TyO1 zPT~*U8ABIUv3Ai7IYtnlj%PiZzqW;JzJgCF@U>@m3ffJDGh+xB6y=LVj(`_|ot-(s zmy(%ED@9m}*|(6V9^19tI7iEPfg#M+$<8q>sH}y_C}|dx=I!GyEo%UjAucMipsWe9 zYVYeCbz`;LC|x{)UG3GMSILLupz_4W1z{P+2G3OLJ&cEP5EZhQ|`76o309&?mw zn2NMjbD>4PD6Zi5x_Vm@?-;Xc-P2;X;s2FYA3iIR;CZ2YBCGwZ1oDfy001t-En{+kDC#-{`G{XVBvXB8@qOy5*y{wFmaHD?!R&XW{+* zEnW>GGWBKjGWrNOD;c=9Yl6S?dVOc(sX+l0ktIhzNKAE;e zWTW|FSYgfY<&X+_0GekM#wOf<`-c%ykB&^mYITAv8`6{cFBms~H)3t;_#l*NG2uuy znUjd*GN0glmtlvS7su&?L`PR~YE?JV_$~UmYi@V4{ld&eA#^dHD%PNXztj>Lu8coEW{lj2J^SpB9K8n_rb6fGF|i5+C~j5j93o7RN@!;mgO={ zf#wpsdbVAa&z8}>#g|!0X=2TYmD0mZr+nDQ5X97#Y_<|;x1*i6-!>aBVY%;){TWGK zl47h9d>SRGis;Cc227fXW9A0Clh=!k zh*iE~B0F%I^3=kEIyvsB^wlWmrEnB+k0+qrpiTkJLX9#gN-v#)85%DQ01VOrlM&eAG3`K$53i*?<*xj5e~lR3Ok&UU|ZawZS7 zd=$x+xE}vQb92bV$XZwYL(cH>L3nWw@rAl6g0iJZ#R{t6 z(58^90G8v#^%ASRBz`H0hxp-UWG9%S$|aeG0^;Ed=|qnwFSO^tB*@EA+D8>|HN!=dQv!l``rX{)8$qWd<&?qEJ?=q{~7A<Fs+;zn*ZTOc{|i_dxrkk`=e;%#Cqg zw&uL85-i+CP&4sR#fw^y5I)|lS}PW-xWT#DZvwvS)M%YM6B=7{G34!F3xH)k5ZH>j%4UXp%ofVP9a*j=W(TUc5F8}5r^r!D8s@@TP1;p*S8?>$vXcVxN5&xP3#_={?AI>AP z=$Q84PTLwi@5m(?L{Rqcu+x%mX=#YExMT!%6EEK9_E_JU!AuX zT$xRbmtKfv*e2sEwJm0@2~(q;)roa7rD9lP4y2VlrUdKV4UmyMSupz>;;noPW8(r= zv*1Oul35xnys$T9xsoq`ujkOl0q|33a>8qDjV6j0_6LUqwcT6_a=s$EFIDm9YN#=i%ae}cAnwJ<&Q;&^ zWDJ^qtWZ;M4O34(d1sK>OZwt!31me;sme+LLE~wt=$OO=h;@Z3o<&D+sD0-#QtN!6Vz5_j}>o%-|tn^2`Uq z(l3ubWt8WRhw@@^&lnG%`cfCrvtUjVMS$XPlNLA)BQ69DSVTo{*so)Yj)e1a zD=Q`4NzT#Gy+3CGGhD)Nk+E(+cC^e~Qo1RI`S2p_Z&m53Cf4WZJvp*CUc62E_{Vgm z*ws2@e14)()>=pC2mJp z^(*qEX$QS^e1~dH_(_EwO6+#1clF{1_48ZB@W6PECuxl|m*yVHe@?Aq+SoqYbyr-{ z-rFPNG#7~nP%W2!foH0sYm1O-15Y0CDZsgsU{ zX|ubFFOu@Tswb7-=}(Xdoc{Mxk zVxA!%$RsN1RuDPX=@`2`v5i1XJ^R|#q|K6*H$;dWCifl%l=B+%!oOv)UytUW<^A$b zQnI8kpoMS19CE3qCU!AkSaLUtc91x=Lq@iwGq(weC}#8om5JI7NyoHmB#Jhf+earP zSNn$Ma=+{FWV6`RQI#pb-|qIpzZ^$5sjkVlHqck{Ieu7V~jXT!4|1PvE~{3Xz40uza}AGS^dES z5g3oeKV77X4Cg2)XPre==VhU&{C6}m9|$4nv*GiVXv=V$(l;&VxSBc(;T2T=JfV%F zSb9}ZrbA~;L)a}`-Z7lKU3}vWz76CcXp2iw9|=Y>vyq$|2u9k^*+b<|j9KXYR$P4> zyAeKaQQo-rk$8&Zxn(Q87N_L;{Qx1ecZ5$FT1)qdT1dzQQr!L>aSM-Si?5#}Aa1ZB zv_2**MPU@v-8T9Ku@ZWP;?wFy7}$JD>=s~B@D z4;&*}-j-4~7q;4|fi+cvw!(AY1c_H5GC+WOs`K9lXE}V8Oix20a*q~+tuTeW&@ma! zn_1jiEu}D=hhYS1m9Et1Z^YKzR=9Co0rN)a3dxSuQj^?wWDHdx)yeX_4?3nX@@~$r zF`<|Xes{?5RV&0#Gv$n=v|`3NxRmF(WcB$FIjd#=U?TB-i}n#acCuf|NcaH|CNvln z>^c7S_1Dy*@$+KLN%!1_Y8o!Cf({y|25hMX<85V zH#Tz9K9XPb_d{`SK$-PoGC5DZ#wLc?Wj$pn`Vot02fJD@QjqPM%bISiGq$DODq{ z+3$Ybu$LQd!NVIngEgYWB45oLw(>>p!cR53UlQq7+mH8KMbY^%nFr;5iv<$ z7BpA63%_qOEhD+vQBuAfz8T&Ru6KVHoL0Y&O+UPI!6Yg=GM*uGIDH%u@_tb&rplEJ zC(5sz?p6DdVFX5M#J%m6B+JbhFXCi_6!5C8klZ#r80%)@;sir7;MWjDt&=LJjgq>ZXRbIdLz@+0>cRrdQW~i*K5jAPgq2~UZHH>I9mzAsn z#k^#+n~Tj<%AJ4hWDyOrON;K+8)Ec$nHH_wm4UibiTKTJMpX!WQs{hB=v< zNY)yTnKt{D`XSMFI)_Cl?R44Rv2s#5zgBrPjuMG`yZlh0-FR2eey2(t0kW}>DrMM{GUN6y_0{(ue4_N72HBk-dutv++ z@)Q{-rD05Hk@Sg-k-m))6Tzw%85k}Y%wDgGHu-!N&*13rRgr9<>1Q=gO<_Xb-a)}w z*xuLk@!CSZ6Rw{@3UPnR`T}JtG~&xo$};m?$qP31dTq2JT&&G3HAW)!_7s>dA)apZ zvX1xdT)O6u0Fh{*(sl99kwXzAd;^{(QL8Z#$RmetK`0@R!y(N*osaJRiUmG|*m~AB z5*#osa*0z&g!UIfs0Zq1tF`)AZ;%afsrXmKX-uDY#&5*1iem{P(~SW9ZC1_IK}$uv zW$a64#eZqIGjmgrdG~l@%cmyy82#NH2t}{fh_nnDkg58t6yYrxlvN@+KtV|9X%Q`olVBL&J{LRXx;xi#msnXAaLc9P^%2BZArSCOD$m{ zt6$B=3y>~STRSa8E6b{V^FsXr{?H}P7mT@w^B3#r+zg#CyL{C+tU~K+auN{thV*Jg zK^*(*fTgF`Ijk$vBg`h>d~T0jd#}0Iq8PFiN}FdHn>Bd=`#XEz( z*qHoW0acC5)4Q%0Q&=l$P1M9J!BC8ubX8yIP=n-{Ca^G?R4cYpSN__!+3TwJxtmw$ z#&Q*Pu0|9yNYrNI6BKjY)GneZLX3Q*{^*D`rTeHbT0$%c-x+iTY%`%My30J+LFs0ffN6L9D^1-95Ah?-ZC7J#=S%$r9&E1UwfLrRepMDliE``M;MI?90Z8 z6~bKxTZDr~zv2U3lJ-OpEt~5PjgI@($S!@3z3)n4x%G99_i?L(1vRR8HjZO*(?~fI zup+z=SxyK@+kV-F9;(r##f3$gi@%$wNRJXI-kM>gecJ<dy(YfbiOP>16vcK5D zM&m2M8eV4LKCLp4SbxTI$MKF6P{S&b!`sh+vh{xX(_jD-p_-M{UTPurd}D^29v96p zc9VT*HEOLSQQ}gSb5%o8`&~~65i#0#@|-vE_k~}16nDVuG(kL!Xd-hL-S&4YC)K)X zhf*bA?TWjFLS>-^4qlmSt&VD7DJlTz2e!=P~v> zQ(+h4*R8;~>e`}&D+p(2`{?vqiS$A3)X65X|M^|>E8efnG1HGB0vs9`fvy%AIlt%ud5FqGCW7Z@%*}|K1eTgB0v|2l z8a!&o*~XC;T*S4Ffz#jjq$1sE824$D3MflN$Bi(Acqp@{A^<-5wia}d4EBtyL!D%DoXgXPWf`cOE^-3fUioAe_cO~(9g z>YdG%ZO+@OSAik-b2pnvhxAWgO<8CgBI&pj04~UYWO!j$gGqspkv>oa@6ngn`QjCi zR;sKUa@){)f8RRm7=oxtCG)Es)=>N~B~$FqKsB>ey^|TPpbo(ZYFpqyma`lX6SgN(SY6!KS3TWaoVDOG zIgWUH+2fuKJ`Tw74^9TW5zl(BX+LCN9_7q%MMe_$jn*ji$+dt~c)r(~L%;~qOIY8o z>4~uR@BHCqU1P$sbp`e9Y%fv}+@JD<^4~jkoPd1ZGO#G*R$%(EqT9ATVNd~nJN`b8 z;U-3ecm75&3Vhj}l^hr}K^Mk{vU&?2U&0BM3~0vIJyyN*DEto_C6f4J??*r;JxsHRM~Wun*u0Ok z6z+&^WNRw*)97HGl7tTQ#M;p?FeyR+dh!t|2sSDbc1hqMY&GxW4Y=eo&*m+QB$XDU znC?=YFxvkas>eL0czGdzTe)Ii!*S3!+3wJTUAsk`9hK-wGf=-*u_Zjzo&$C&Bel9I zadCsYnCy;4#)Z;DKdl>0GTZCa6~1rYV|r>28f$eHGEK~U()?!$K9`Sa~&k@aUjwn&VZYIErEnPS2rsDFaOxfg2i^jZjbBbM>m8&Pf~=-X3ry}O_X;ilssC!;ruwQsU# za%JOUND@5ooj6->CtaP^o>?zwD2|Hp_xH&+QWo#l z_JLl+r|<}+_CDC&*J+Me_en$53XOI3ssB}451I)20ZgLFf)%AGUHQUv*dG`_=OOT}i~Av4SlP!;^kr^&a8s*peF_h7_p z+Pbx<$=T1%W=~NJ{}3a8ps0mU;)%SK={6pbZJ7C?Ip@yKqHhrFWT!z+<)B-i8U3Cf zpu9Ax_IK*ItyX^rpe~N$ud^(XYEQzHb}ZxZnDQJ*`O=+74KSJ&a?3T4tB5zB_vwLI zis2+A`G@~UiXZupii zxWdAB-X<49m@bi5Qq7zg?zcsGYvWy)Cq&0;4#895xRu}6P6%a8c3>a8d_`O#}H zIa|i5whG;M(@rouH%$+L>9vh&6sC3e(Z=mySi_A-pR{(tpyxM+`2hoWt}S)s8SJG} z6_=@JQ0BW&?pRG9y5Tct#E=~0#`fxQthp@gQ1)4u)Ysl(nh7&)B-gtlH(O!jX{AG6 zClGC0!$oIRxS1w&?HCr{$i5V3a=O|Zs~04PpFbKemk|A(0}cqKe+l^nhUr94_8peT z0y}FVUF7@KpXH8mptfVPL{F8PA6y$M=9g#Z1SCaU(`^~mZm`&IE^;Z&((K<)!6Ec_ zRd!D-kauin0$V5Xb*(`^v*BcVm-~p4uR53XhIQlX=Y*vZNB2Sb$ zqs+>~f8{k^rqMMw=BL-S4TnzboNtWqW9DT!4&V5qus3Op$!x8b${6_o!^=ixV==`C zmaw|p2uYxKjK1|yW`w^N8~K=3jS)AISkZ_z&Q2Eh;rKPg!==f13!>>@I<7CX-_|K2 zn=aYZl52Zx$<`I__M^{M!byikm2$JxTiXUnY$~b}G(S4NvfFmLr@sN+pl0r1>VcVK4fj zm>ZX)?3wE?h+LuS;u&RB8KEwUk4_mO3vUh)Lve<=A3 z=DIi8$z-AlS2&6$9Nnx9kTy47h8Rk~PL;jP?H{`mr@}+DE%v_0*bLIAlnf6VegyL> z@vgc5yh>Exi2vK4E~HvA$C_A8A9WZB`z-L_)nsrpzQgYkqDrfyRbN?!lfnhV5j1A!(U#C=_t}TZ+q2r4)*K%Zu~(yr z<)7CX-8bVot5~w)PNHw(@}Hq-X0r*=>Gb{zR?A7oDMhH=|9c$%cJbLLbi*niBBD#; zl7EJH3$<1e2f>fyVL3ErnAnf&4Pr1Qg3v1P(_)oi!R|7%7XLyEaw;~g66t|kGy&^) z!#k-tR$$GTzqdXXo%pcUOj10Hm8FG@<))V9Lbe8n0|6ZWBRQ-R24xx$kd?V4p*3jyGJ zLohjE&(SO69%C;z1w>paotsAYhGrtip>5x85rD54c2Vfl_>oe7+fbO1MJ^vw-vaFT$`5@VS(STLOKuZv*fy*&AchmtewLZOgxBGBAl1{pY~RMeC*lD4 zGL#lZJp`gX;toRQMc?RoCSCW@yy&&UbMZA?olt-F-`{K_BcWq{C@)z4jbs|iWjJ`E zzP8<&>A;tUs(`lA5G3w4CxVanMR|wa{G=f$zE6siS^X(6Atab8Q~1jFC_{MW=-p*e z{I=GtC<(@&?C;6y*eNFb%gSj%9kg@k#SN^p!)8ZL&n3=-vwIn#TgqUtzFyT!6lwgx z0m$|qD|#gbHLXBAR6p}T_plTVZ1AkV9iwvP> zj0RAbo<50dfqd?DcmKXsDpXkDuKdC*yc=?T&WCz&S>)~=d20W5tzljOjT(Pn8=_9{uw zM5Lflm8jI2QEm>lpb^a-yFS76>K4B3BMm$4Dd@7Wa=@LMefah`UfN-|eza@B;(hM z72u}4R*XjEv2a&@B1QBWAG?TclFxSL)p5gI{XTBYm7}f9@H!AkrP#?GNZ&8|CE(ELMYt>bW1-vn}xe>+-Jd2Enz-p+OE$zfLPz>EtkE4InTvA@yGO-x#ZLB8bOjv_8e zj4dzmj}?>}J@w*spoSI>*~$<8o`257VD6!qJ9=e|Mz1Nq-QsuuRJ##kq*n=svlb>q zP}aWJIv5i>`1x@Z(q98YFh9$;GUMQs`+Gj;;h6~jFo;t%aa4cqumB0W8en9q$wmG? z>SD=TsSu*Ua&P^N>B{VwmI=}V*KMnz7S+F7?YrIJChas;daTCAQYE6TVap)>RfW#U z+r-11o#IC7j*S@Ho{Z|Wf+4d$>o)u2GO6mw=CiU~K>4V#rU)wnZ5OL|%zqli6`leR zOq>?BkzTzQk=BH;a5qNvn4{l}3Y`kh6wlc~K3fr$lEq@CR=qj}A%#?#VwATfhI_Ov zp1ni`d)1>;J9_Bw?g#w7(|J4?Uthu*J=b}~1j^2qq|Coj`#GY7Fnuy1hfpbqvd`xl z(InF82^!rpiPOMO1v@@QF0$=|bxov;)E(g3J(!4{NJ4r*O&0a`-J~~_D#kT8m^ogO zu{Z8Oi)ZXc%Q8A845To0O~R@3f>x-4;7Fu+$1m6#6hqyw@l}amTI}>Y#1s)saRu_x zxT=}l@3AkHoCT?wX>?|HhB5q}D|P#=vsfO-2Gz8-K^Tk<+ytrqJ(I=l{$-+(On*^c zaCbKrKH>7I&4WyNbdXB#+?$)gPx1wZ6A0az5vFt zPg81kxnFaJ&=7f5)q~TuDblRgz%JRPMTJPdl+i)OkW9UjDt@?FAa!V;lDnKFQ`U2d zRvvWZNX9TBbfuflB)$Y(-9Wag#yI(q^<%C6-ILCI8Q7{xFaFr}oD}Om3X&U1@bCoj^ z4HaCyKZ3xZ8cl|A-Rl=m?f2LAr0~Unc9Qn01|-xZ&=2^YEg~Unh-aHp8ws_+erTD- z5GrYkADmImmIQ46o^U6!XOVpuX!7eKj26vI$QtPmhq;R~R7%5E9BT)5GuS|OvSN3- zbG)pPg-*k!pO1l?+FE%!^MsS|srxf*wv(gAcyi(S^=8;7fza;v-F;XxF+rgtTWMFi zsF>hcgpKd8-%h*c(Izh>s&5T*>%CpFqcvD z+>E0giAIJ>V~K#w5EJgLzRN|mDTKw0qeHC{(|&G$>iAGI;VHJ&cpr#;(1}Y0ldqLQ z(|`hIi>{=hTf6hE?|Ins4)lp7ev#>PZd9|gZA|ZsSp>GRuqi=NkmcCJ1u1ppXu+tL zFUK2|-|Euixn+y!bXR#SrA(~AVBEP>5L4|r{^K?{glMzeB3ey$7ZGxfrS+g3$hTSb z(;bId@QjBw@VdV_=`c;p^M3u>%Gp=~bqGlmC|_NC2)EjWfA9wmvwMeNIGjs6c}Us# zL=;|jh=kaNU|FWFyxU4E8M`dDa3|5TDEF;c&M#l^2qC^`yYUMclihVi7G`%g&jE3A zPbE^99QooCvW1owX*2D_n9NV{VMAox`8{yG9q49&Sf=1HZV~?&uomPiIp_0|gxghR zDK|~og-NhvITRJ&vI}`#Yn2tiCLZr$<@r#7MaZ~1GqbIq^gR-`KpTc0%1s8~XPbWW z=`Nw9EL8`Djln-DR=zG|;P6A#x@v3!+^kNW`=lT;Gbq@;rI37_#kNe<129yr`RPE zRWJCVKz;hJ9^}%o&mNG>nf5?rtDwG5h<$yogo7Yli8)$G3Yl4Jnxha{7v3M&A%n~7 zGfldE!dhaRrIKnSd#v$_nNa|IA8Am%rk^H}i<5KM+TkNCa#`!lj)@B~$oq^K^C_y% zP|!NmoHjM+W_5B>iTf2s)AC?nCzNO7jl$So9b=X3X!+Zti_Q#Ii+B^LQBjMgME-$h zfV~y%&2P;W(p^5@I=|#khXN*DFk7C6F>tGoPpm{!XynE%wGFiIvmG5<)TicCek30F zsikI?INfinUEP=3&2vjZgHG}{I7>Wk3|D9hg-6N7=PT~J4;hNTFfMP&nu-cThV(Fi zK-z2Y7%;T;M6g(=F{1f>R(@pJYQOw{TvAUq*inv!AzEdcMqC%6uQM%#$oV8axdnL+1Oq@b1;SyZQU^{@ zY9YXu?p(Zf_t@nl{7+dC+QJNb+UZAP?gjqplWQr)JiTJ{)dXW+RN^PkBa&f1mog{CZ^1GY$56aI<56>(*{eU3k>Gt>;47H9+5H8%=D%X67vFK>g&Y<`Eu_%x+y1@1fk0C*AWn*9qq-yQdnjr<2!S#n}TAffPLFhygLQ zzOrmph$@o;l!~qGk#MIiST+?cm%VPytBLW&9D{!on9dN)QIQy zS?Dj9p6Dl3r)_g<_)3GggLC#LKOvkGnTzClbs={4b$3RrhfUhhe~M->T#V3VD)91R zMx}ukN`H5ud%>l}0sOoT!P2u@>SwnP1I7qWq@gwppGo5$0?ePrLt!%1pRrHv&f%fvvIei91t zrcvAG-s>L1(ktQ1qsWYqf}kxr=^bp~zgJ4|Y=og^m^8zFzifKVQrSBRT0R zfVM?X)Q3I~gq(ANSqnSwTJQ4!Fs<5sp4*01W{v1_fjvL>q`h|d-r38xYS_Pv{)VG_ zrlwGvt2_pji=7A=x~sOPOVE7z%YNzQWO=89=)=Z{n#Ob;pq2Ytwe_uwJS$tGxpIyV z9BKAef<4|6`{^XYL}qki--t`Lo#ow_lmP>6UpE_zy+7H`0IsXWOriC3$nsQYYq%@H zv0mW;V*VsYt4qHj(F0-l5@Ggrrep4w2?d{8cvHPvD@P27hl7p>^M!H=nFGH6ifN-d zM=%|v z)dduie*}D_8>$0&gkm9;>mbZvb;$hX+_SS}6YE)^W?76;zlAut729KKSs{5EV`YRr z&R^q-VjE=m69_C(=QBzw*RgRJa}{~tH`izd3td#&uQQ7{7c|>0TCL23@CX+7ln~QA z*pxeeNCzuXkBemYy>%^5#!K7RU5snw$$?g~E;*Ck-Fpjh9QI0m-MN~R?*sZE*5E6` zHe|wWtYQWQgVs|=s_o+kRfW|fh(LdPZOXAiCJ?t_s>F~nOkkSR5{Klq|*7sk_^U)I5~%IiGT&F(HU zPm!wv;}`nf+n3*hiA6wl#Kcz^Q9w==JBY2hr6m-7g$I}JM7Y(!pdaVVY~(G$oR)FX z;FEHxepA*hnA9~659ij@)s<3L6zcNx(3)%H`rFz4GI7Y7Bc5)|HK)CA?f2q4dZ=_&;m7gMnuK>l1z+@_B%ua;dX0qevc_<@jgLi# zxvUf7K=)zHcc`x1;uyKNXVj4yU4_;T!zFL<>+G;i)dFGe)%^_Zx1K@BTW~Rs1Mvb%lDy9~@G(oN3Az%CPHyxlMPpCMleji_3K57Y7M#H%}BJ^-X`AlQh zE1x9MP`rlVB7%*NcaJapSgvrx2+MV15_Em8>)1>g5Fg(s(gnP?#|FFTh5IuC0a@(x z5*$7sg`QoLqCeV|YjErp+fpMIZ#a!f>mOkfcVyspE*J=rjC)#o{I974xUpXpa0!Lc z=ALUhQS!Y1YP5#1j-c{BjDIu}nbu~i>4b#O3L~MIQSIIOuZPT0=cEbGdmTJVv*b#R z4>7Bjou=%YZUeJ!e)e$ld6qT}cv3rCl&f&UJ=0Q!Y-CTWG%{&9%&Zpg1FJ$xQiHwo z*6U%d!)7x5AnEY5v_Yx>QxJ4Sy@{%gZ%z$Lz4C(Pcli23#n6Uxi$nDjz7-EnRJmSxi@p8PBCd?0w)kVm}91$3w;<+x{TcgbM#95fv8~3h_UJ)b;i3*yrvn|iK&uo%C zd|0uUt|oh7Qq84ksxfLe{Edo~+(7PcDSfm^MnRGER|Lefe12wPBeR4xvujRziKXAt zt6%fc=5XD>G=CqRSbkcU#133>XYHWATlVy7UF>8mX$A7waWsR=K|sh|k9^ZOq_6hr zyXc$>^}khMZVfQjq^9XcJCOF`mGXnU&*r9w?q!_x-{0@ls*>z|T!KH70Fjf7a>cL1 zx?MSn&Y$3O=|=~>z0k^WaeC5b_?K`8y{%m5pP%#(VzJt# z=jzn8kGVM16N7BOf-@(G*k1QXP(i8KWcnpszT;z8A~LRI+m7%2Ma++1rv40!&(G%F z@`;!jr^<*3L1=_<9?*{r|2B11kdyvh)z4^(I~dfXXc-M%c%}(~;XfRWZ_`Z$;UA8)77bdf> z-pTY)16b^wY8gSg;$h-M6oP_(W)u#cXb@P?L)|Rz~N{PfLzrV~eP&F@1K% z;Fi~I(+0+_FJpx>`CB4M*j$vve zVK}Z>{m987cJ>YZ2DL}m`&mK)A&^HwL~>9i^%ZpWTfzhmN;ut#Bl6xZ_yceG5)dk| zsS*pd5fRA2SCHcc#A25Nx2^&NO!P(TqzbinRVI(bvJ9BRT_{cHQf6rOW5^W#+z_2h z<%`h7Mm7g^;p0PTR3nNeOXLJX?cO5WJ&dbxsdky%Y)ISOnDL8T_TX`EBp(dQXGxjO zO{l(sgL^|s1*%L%?f81W|C*;2R?cA+EPgf?jY*qP-w(f?tVV~?(3}SdYgg2_3+hKb zRHQm=B5U;}SX1V?l=-XHo);r5+bw^`$5Z$_YV96s)N8wxWpMZJD9*1Xc!ub+^d7B7 zMR~2hhU@ZdcP?yBxGk6p(hh&w*QT!OI4=+P2WFn;?ZnBB>vqdC&14$HonPa-u$;kf zrV5})`u^=eD;xtn3&ulGCae46pmcFttX|NTrf)bq(T%WVzUR7Xj_>p3aSRlX(qjA>%vPc-AwF zx}E)RqDE45ghE8$HJ2V7kLSEqO-^#k@wWxNp@2S@9Fbq1dwwctF45gfr;{yzzKNnA zWknR+JKMPhCv+snBaDPyx3t$pgzI;u1@Mq&~EV*{fs4oHud4vAD=-k&>d&}I10ALG`7l7j{JviRHZBDzLP8{ zrCyPt0_Lg)C=By}ldm{#RN}m7JZ2ppP#ob`*3#grfodZ*gJPA@uR$2GKGhXzRy>g$X%rRPz3Bn>4t?U5MmE<8@{Z%wF?_o|>lnhjZ8m8`wACv7gqc zibQfaisTnJDMQ@T1mlnAc5ZO5wOe*kR^JU)Z%CIVtg~iFL2HH1=O1{$Z*XrXvYp)0ef ziRoHtuB2vpRl!`xt2haM&KILh$E}(^ae*GphY$TUyLg&pxOpOW5X(8^p+k zwbZiK`nt0mML7ql38Hz8IcpF!%d%-jh6d5@x^J_J(JX%it)yW7 zjA8q-gae7ibcT6W7=B2+H^b?#&`IX3+>!?3&DG9)UR-q9jBhGJEUd7kGc>vzoej~riJ<>?Zz#(6y9P%C#`5J_G+xNrP{6}wfHiQqc}gN_Zg>tj<$8X z^Cl(273S~W{!6>gUsfq!YC};_>ZNW$2D7w{9jk({T7tVxS*&+52PbK6 zcXH2V1=Oz2LPNd8;(Z_1UEM`rG4rm=ZpJE6_UY#kxO3hgAM#V%vJ5CtS8DiOZ7_|PMNTTcMKBs6_89cG4nvUAjJohj^nrDTBAYym zZw*~{ruY5#vz}4@!Zca8z)UDn@zp%~AXA0{YL5_<6dOgd(D#vIXOEW6w@vWkzjDX} zkFxcSJlx%eja6e+_`a?$7n12fA(71{{35B@B!1a+y5c15MBA>et7cbe-Av4etJ*#J zP<^4;Gl{4YqwlC4;2TAdE`V4{>MP+`A2U_C0jUUo4k}<;r1W^0PmE4$jw3q0V&4Q` zqg}hi`#ZSBt_F;6KeAr~RjD>g*v7BEc75$OtU!}6`>Bi(fk)H_si|IFM=}gmHsm$~ z(MF!IXMmA!JEfF)~vDo8=zQxRZy^o)|8i4UwkTTqT{@j@gtGeI%Sm=&I z6j};(L`JGrny22un4j=801D2(-F-=Ks+n6GlFAN#ZV^Xkt$Rcqq&T#PNF*tM)-@-u2DTmIY$1A}qZHvLQo9)ZD+JU#q zHqNX;_Hh=}rxw@zxT?`Z;@Vz?V)`!R&b0&CiHi7W-bQV)R?Lr450OZc){U~2RVf&N zG~hVn15G6so4)1qsDtD~EKh`u03!m4TK{x5={fuyo28OVHP>B{$ z)8&rm8s|svLZj%?Zf9O&X6$!k|CZz5DQAajnfN89ZlWtfeNhNupsA1@YLZi9aN@dW zFz0^lcaG18C}fW=e!Akg5-diPxe>Uhhp7+q|`uma7zS_Nw1Z2+z&PnYo5i0Ruk=DmkZk$C5 zNYHxN9S$8=J`0JFs+H$RI;9hY)D$Lk`3f>2HE>#~;m$^aU|-l3mzg8yGYM=?jvzhE z9W!5HT2_sU{oEBHfBQlflq5 z*y+)#SJ;hG{m>=YOIN{1SC-Y_3XMCKY7)C5>Ua*r4I|JB#f`=tY~@>{#T^D{NmnW>GiwZ%m} zF)flOCib*Z&*Zcz&Q18MD;BaE1=kN>J3WrV?N)U7H~Ybr%y_=y1{S2J;|U6`XrqT{ zh}XzuCx)X+cY;c>Je0lCO1_8T;e9ChvLGl3w_u9$QZ!fyZz)jZB&C2T(kg+XQP6`F z(EZDg-WhTlNN!Pr7oNJLjdhDkx}4A}VeP(&ke;28M~6(ty(>P>gEM*laC-*-4YDD$ zCF$c~t*iAb@B>+(%~lne*D@d*v3Ebv14ZF0^FJe3M-Car9mA50y-D>-TS$%S zt(c1Dh8pu@2HS5GtVrvT!APT2MjUe~D)no&B>$1kH5Oo2D~p8ql(>ePzLws@AhB)l z$9{~v!81oLqgJDD?JNGhIrbk#x(r&h%%v=@YS?i)X2?-q6Q(1otcj{|OGnWtRu8C6 z0E&Z3mXl*)yj{VesM_bMjx|^6?^ndy!idw3wwe-Gm!Zym_#p0WA?v|+_*Af~B=_Mw zdwIFfjKmUqjP`6&DVZGkPRAvWr4Zh=2-Xpsc#s3!rp9O!bi=V38ZS zG=(f}8?-graOa)?cEvt;bDN?bW&@Ts=cSp#c_OgI3FW27Q}HE)#clU=F`6IC__(^J@lqSM?8@~ zkZa$xax+*nA&!p~fxAWyTiBC-!akq>ekd+*&&V^5j&L}6CdQsg zn{p6!CiI^xqj{^av~O{E^{v>YdnOm{Ou>OxSAp?fU~FwcZ8%D&Q4<4S;h#F=()y9F z<}7e~U?_|0FYm()HN#AcKC^~Z>O2;!zhg`5Ks6$Adg7`E#C;Fu&tu9;l=+%Ck*uH5 z*d2mS^(h;BQXJ6S*To6F@|JRNz+TaXv_k1vW#Vs9ayhTZ1_<4ou{yIfky&aN=1XMz z9+T3)F)V+TJSPAs$j7xK-1v{(d7kdLNr^I$LwEwTwh5U93z)x;yIO<7azf3E%X{J1 z;FC`9C8#%ClB*+P^jUBn*+^o@5Bn66ukop4ZD)Vk_esJWuYQj~-RGMOs1(&P>E9uQ zS#uuM&^!29R&&jM$cyp};zdkZJGLL=#(5r(^ad3}lVAaDAm8FHOt1Hs6THTT+wbOC zzV|DNwe^Ib%kEHA*I*z`{Om=lW43lWeEEEO5NSr2{0gAR{ZOkcsGXmro4N5Ri zvEJOm&5J<7A#1c(O~-i)ZAbL*X;x9PA@3u}p!P-PxwRCFROe5r?g#Bj@)-m;{`5A$GSyCpg(FBd63gE+6S8%8?qAUM1IRYcwi-g2e!}cRy zcaf}rSofYUYAnzq-h_H9uVuZ;@0;HT8l9iN&BjIGduMQtbHm109A19Qjc5 zpOYYGs&}bToWGdgDc)OiT-A+)Z}HXU_y-OT@^yS(X8gce1ai#+oppNW!$CmA+%|mz zB{CK8%0>y?(e{2~+%or^iIggW#-V5i_4z#*mWEMiEb3QNfcZ`5K+tHXkExIr`6xAj zmIyO~t|1)8nXrb0t?NxDw6Qju$)`S!DzFah#R7_VvjV47n<-j+5ts3G#yumgR-%%# zrI(7WV4pGNDa`^O@8T8JQt>fzVk|cEk!G4P-L}w8KJ&x=eH`@_w{vF}-9Xx<<$OhI zyLkC!>&-HJw+MX2w@W1#T)>plVv(!sc*%n9Tx}lAxILT7iUJqJ|~JxvERcBy@6Xps2> z7y=bH2N~Q*-)caCwZ7=rt|N*;h-TF9!fD*24_2PSseP-iz}(LXz1uCAIZTmk)evUs zZ20^dVbjO&cTc(*Swl~tlW`D%Nri;+YeP)8yStsnabH4QByS7QTNT6X+?5sehN1D9 z@uLY2V4W9w1Y&vMcMyMSmW9L5PJ8z@V@Pv^&&Uxm41b|~rK&(=Rl5GJT?Ij8vm${AmnIhO2bG70F!V1Ut)(fPNoSF`5^p7`|aAgQYu`?RRL5&D87A&)~}5^BB`fcCZutv1Hx_Ky%M#{@fM5{!yN`8H&tsKh=xkP zhfBCo#UD?1m@odYkv3(S_4h~}JVwlfPne-z?-bo!<~px5jVA@ih9szdaj>Ja3c_XO z(F7Ew8%_wyOW#W7R`5u+lml*| zVd|RL^biQEBu|Q~NvuPnEg2cU=JPH}-EA|*J$!E|e4TPA%!r3V0FbtXyAL~ADAZo% zz>1y%^tf-E>)*Z`k^9I{D)}@92l=01pJrK15-|9 z6U&+jPTl1e@JqsFxT%4@=-fLDa|LX|26A*wP!X2=6>gyub}Wt?INIgQfM{!e| zRR>Jm1K8R{x~CX8@2s|^PSQS)7a0W0Rf06p{lqhS-pNj(>QeVID;Wx2_x?O5s*hO> zXK3>(t;alk(R0R-LRsx%2%{RF?r`mr>e9t?JM{yGYSCmA(YUaR?vKnmSW_w#5yrEp zH~YK_=hiXKnHSe-gj7};#-+Fs`l=%Vqr6d~BbaCfZ5S-|B3|eX+cr5Kr=nAi%%NT5 zNRTBw&fgFVyz8L4rgtef@CZ^0N@7y- z@o7aXrC8WtRk$9uKSA?_pz;K>nWaT zbkkrTn%sWfpz;|lc0gsQ>SJx?Ww9y+o7EJ#ERBey?}fbPmgG(F9qI#av?BXcNB!@> zhM&01F=dw6=jb+k)u=#!t*Q6zmEb{imeCF;f{qr}2_+q@T5H$eP#11sh%}$Zb3lro znCe5zdHRli4(=Ksl;DT!|D3M1YJYY*VPd8{uQ*UP*MK}q${)*l=8(jotS2a89i6yn%cQ%zTC+JwJAi2*Sy&4UtyeU|-7n~ze;NGo25qDAd zn|r^$uZd2;q)_ye(3^+ zk#hTZa6XO8`Lby6@hp-}VSeo2hBUmP7tGIfG~B%h83Tod%m(2xA60so?Tn`a{imet z?_5`t!W+_)9wgr8kaR`NaV0Chf>9jDa4ZR31$b6PIfgPN#F**JRDF)wx_KYr7Hl zIxgtrlb6)o>Ki?Dzy53J==%2zOX3>37`FI4D@SyN9Llj?P~j?hGx*lmh0T1JNlB+e zr-*L#liSxcd4qMfwge8hG;`>dG?uHP_#Fwdn|5AGJ&l=S6<0s* zO03x#Q*5<0y~GBwN#H)F6)wa*zC~HS6tEX;!F?9JAODxBHN@E-I^DuBnECOJ?wkZU z$xWu!(N1I|<^RgP7JBoha8rx=ain3Ns6;`^J^P{g@znt*Z(SpAE4+Q;VS0EC4n@-F zxlexB7SJbbw*fDy|2IHA+nheVv%;omdgYBi4$jyCFfXw+-qkkC=XD zuh#eW7=JaML1PKpkM(ONw5|*$CJroOUi2u3HE!^dP%pjv*kNylxn)3*)-@WG1hq}>FKN9uR!@D`_hpBkhjm{sY zw9OFmznUF>%1DmNqMELhthi*^8I=3F%SR- z;|iGaLu;Ko!T36vxG;d!ddfauHF`b1pM~>hMXydUkuIMZMz{Fuu*QnV4(|kw(?Myn zH>fq2nTkTRk3GIJHM43d0v7k8Umu3)PkGcMsg)W0eAOi7+bgp1yf$6|-{%P9;+!@0 z$opKHopeXj8slvn{J!=73F!}&=Z|vfB5_K-T1-~b$3&@z?aqF>eRSBqVdlwka0)M!1Z(rm6YpMa!tCjuQGsr_?cR z=3|f{4E;+Qj3IskMoQCJ9fibm5tH&Qv8i-DRu-Y82VWMOmbg=|}gv#%ox|1tc zH40>qdj*HJhhov)SDkjZmA6J+KOSF{cNjbd3pei~C5~bhJu*9OF`PnS#^t1K7HL2tjR?Z~42* ze@v`YgGc!h) zbN<4&gRf{YIp*T7`?CH0LFJ9pv38E`LI&dS4iLu7kb zJmq^rX8VmB$T6|~1dF_y^;|R2oOs5A3G^0wCIK-AzZr?OiJST_sAuKc0M5%4|CK#r zIRiUb4E@l>LIJQw>i~%lju}FKGJDFr@@a_zV^9OdNICybHky<)7)FXve~6SE``^Qf z#{{lLgtTg?tzYA zU2ks;Z5ZCAXBt@Zr3@t`^j_8w!@943f%A!zue>7pygp8GPVB06seI z3DZ#%`kmYR@AGo%RPI!k4cr<(WAfX-4nSm0ynFWH3yhRmM!N@dvX%|x2ux7Ptj=lH zblIPfG4}au+1%N5cccyBe4r`Dc`oUUk_%F)RD1+%ko8=8GvApey$9QKpN9y_cWK;p zCQ%Im_M{GQ9TViYNvmFl8rYM~VgzHwo5q56`Wo9!S;!L$Y1FA$U zq?2gmC_epZE_)?8U@v*9Hi_jzXj9XIRq3)FLR`gnBM->H;szzg=>iEiRKlctUaz7? z-P)36J&ZmZSJvoP%J2CrXgyy!xeod!q)3eaoqC>pQASo#th@aMHinJ6OZx3xu+n|y z-MoVS82|aLtz8hUst_JAL&S-O=k|5SelD4Glj1QY8F6n?%pAjlwTU$zBj)g%$Frvv z-mRVycFD|iqW=4a3h#tUGfQ?uKpheEV2N!=NcuG5yr<%RK|4`qe}1#uJ}RnN092T7 z4pz-sBPko>ykq9AtCVU^HG>B9LvjNCd4at4D1mP}zPHuAZDb%8ufcY=$kK6?a_NtZ z9h{H$@cb5x9=DBVI0|%C%}~H*U2?4p7E+gjX^BJg`um|A4Q*ak8g}+Nw%-cR20MZj zb_;r3y@6C`FatLjhf8z4mJ4Tip+cD+NmM^1q(2GwC%Zi zD8o6iB_2Dd?i?Q;egsVP_2x~04Dq(ry^-VVUBKyk!T_f`Q<4wi7e)yOtQ>-$SIE1`vRuoK_rzD8 zQ=!({rah$PNMilT;GxT{FC&B|Vbk2JV{2K;cIo4H_%_ysE=k~rvJ=+;o1kOWN7T$z zO~9B2IEDptw#a~YwF8z|^&}7EgSJiPYk91yQ>pJe^qiiCD?f4)8SzdBZM_${g$%mF zjtkL|K}nh6B--m-YJ>e~mSKJB@ z4fgRn*_rS=Bt88^c|tTzkLs?LhTV^o1GKL4>GF(Uo=r}*vZ(+GrfZgwbz1P55RuD8 z1uoTW(&86Rufl9aB;4VH0SZ97`<>Y&0|)5Qc|mtWI-6p`_4l}BXdi7qb8n-frlO~I zql@X7ug60tW4KFq>v5?W^Yn*X08#f?TLiOt9?_r zsJ1|4t9Q;-daB;Glw}!RdF)YfA=UwtL0xX)N&D&*VwTcb;d+GoPDz+p&#R-Y zt!3UYr!3pW&Lbug!?wlaBqNn12c5ztN+MZEl`cpVw;0iwjo{{x>2jTYXlWpJMjwr9R)FDRS21_ zo7%lnkb9{k`bQ_kE8qngWmmi{xz40Uu1u`6?wWHyojdC>0;Ku`jP#gjkfgAh#|F_T zrK&XAM}p7YYoS_P<(DLGKyx`A#YBw0@S1+!;v`A4B( z*JFj2F0>p7)tZeLprHV1%2C^AR3VJd!L$lNPlED{-2*Cr@zfd;Hmb$Y+VE}x97Qhe z51Mh${4Q9WU!xY7lsPjGO??#SA)JJazTVuOUza{K)0C9OudeMMkPHOfA&fX6^ux@^ zubu8yrQt5Wp?w2wY(S~z>9yMp2#WFi-yHVBtp}Ml?Z{GHpALbCn&D)f1rcz_4^1%Y z?cwW0j-MbK+{?I&FMWP9&@FG1#phvBEI6Z7#7%4PR~}P-W^kY^v($>C-ls8z7gyOU zZwFh$(umB7-Rz6K`|e{EZvrn@DSGi-Ck8PUv%^&w)C!JKF+N{MLcG0Wm^(UO?TMi4 zT6&2{#`{thvSKy4tqB|&_gwchE4ofKEL+#8rJEYn*BOF6wsrC@8yCjKqh=CY(8Y12 zu6LsruE%LJrm`0&F>7!v$uK8GrsJcjrno>^^^hdD>8L0m9XOPGw#p$@vF@$xnM@Ru zUS&vC1qHP}+)oIoxm%Fx=QHR8&W1+`USux1SK4_ZbL9*Ue9SJ9WJWFY-HDI5e$;J9 zyRl+ZUBJTgfe10v;C5!Uhf7;fo`yEjn3+PiHtbJ^T}-*!wD~P+Mim+*vd(s7*{cgy zmTZXlUD6=Kc*mCF?m^_zzk$72d^SzkRZ&>7IGyA!Gg;s-5`}U5$hy}op_z<+IAF_F z`PYXOxe$*|@0+8Atf$9A+ZTVu?fTabpCUbJe`q$6%+C_ORYle6bn!Ipy@wd$x<-VB zqVQuU;Egg_(#uSHe=1;8NP;{ z%DqFth9dF&E}tm*YlLZ_ zwT=8l>_ST1-wuLVY$k62<_h!X66|uyHHNT(k_gB(VUDe;KlovjQ1UlMr2oFu=uuW0U?+m=|z0Hju%XyTk(lay?RpQWWLb0}>D{I_TW z&N4ky1sq7m`@l}^IWPgkI-$b~FQIiI@kS}+axa8-g2eRCZCLh+yVx3v-nj`^uJq|N zs^qYs_ z7o7DA&pqz$Sh#7z_BUo5?^D0ULwxJ$T$=H{H<)yP4A8aik%(14y)Sy*+uL)64*2o{FF+9^@G z8UH9`B!=2$gp!bFnDX4Tq?4F5Ca?2_Mml*nqXtLWsf}kgBR*C5v{D{xSKr4N<*uDo z?T+16(z-|QsDod7=TS^*>UopHsK*P_8g3gE8YsM*ubJGZe(jA7zf_myN%s;d%Jkoy zaWOV#G7;OSp-@NfH-Qj}7wj-)(AQdhBlOU{e8+lTkjJ<5BPOLFUSOQiv`xCVq3t;2 zA5orE;bX~&ERXvnn1=q$ZoGPLy*qI{;+6WNDQD z&LMi40QZ9)Da1f1hs1rkZbBxCyPlPL`9W8|d2=A1D>;w-;E==N!cK`MIY{+E^`AEw zwc@Mn1j6)j8!Ti~vNU|D%16WyiPk4AofZm2L5K&fE#x>cg){S#)0Pv;D(GxA~3iaAbwZ5{YIG|EZ`_NfVRl%CZ~z(rD76w z4jeU%0e0%fyjoq{k07w1^|!(|AI_8wYqLHW(642RUv+IoA@X{+Q+0fvGEwE)iUr)Z zwQ?Y7d>eWTEuPYhX(@ho6Ak&z`oa}_{z`t9FH5Pu7tX%GpS((tU>FVI^~*#aT8NK& zvAL!sPA{9sb1tJ;wY))BB6E)aD04r4*Ijvz=7wdoUm7Eli>Znq>w0LL4QIXg!Ypg= zcHCT|?@01yjz=|anGuxCuM~;Ys>|`lIU#V0JTpQBLcXY|3uUsJzA8PRVdCq$l1j_y zIZ<7Ir7w-y&R1wQq^A7^7)BO6s?d-SQITx#P+9a8oOV7eOLHQbDEiCB8xJ99#WQ_< zNhfmxxmfv7m#2Hv>|fa%J{u2`#$5pwTq8wXhPwW2*|+p^%wB7}&_sMp<4ya6JQ}06 zQf}QADXuz_h4G``q`eP8PBA5!znW!cCH_F40xZ(A(AHrG^_fD=TG}v1blBc6;~wVC z{IQ1!=Mij=*X}-g%@H`_EA>XBlP6bPpD)x~sK2g+TSAx(+=wp@DURJxKEsH=;G#K& zW6qY@G#P8D;QmGa=kCOgJnLrOoRrNjd8!oWKfW)*w9W+++6Pq;MiXAjoY*$-gBF=o zu{$9H{Mcu?u61JFAl2TE3)2)28z+4c9adj)g-pWWifwZF*7_PImWr+32aaG&ARdJF zvaue#qNh69FC(uc6!IE&5_j6OgvKegSxHn6ajMziYq-;vyv|JGBY2{({qxCy+gLw# zzO?Qp;yb1waGRPtTF2PRB%)X^Ebvh{S|@u)V<%UChvIx@7?$(LJuy+NP7Ys31jQo( zjY&}xx^YG_Rn(o;GD_TZyHHfW-(K^*J-G=5b~!zEdBS7qr3^P7+>(XngesaVD;AY% zK82TI;!}KOGJPqzHKFe2YRa%z)Q814S);~w`m`ieZi-n8KEiXgf=%MCKv1t|5xj)w zU%zfTD*f$oe|AQ1smeP=Yc?y3yoT-H*3RG!1=Xl5+IK=)(>VpwI@MxuML)TI>Nn!q zsxpqx#{a-bLKt?<*>6c-^e(DDuC@0}6TIsasDnDDwv9I!>F%(QRVMhL5!*!t&<$MY zK33G|#hiO+6%BX@dU%W*)@`P<(uM`qY&QSL7nTHU#e9$e#Ob#C}b#n)WgJ^MYON$inx?b zIQc%3?qd9U)JYqSzKud1AGakd{HD-6SHU9>CY!iq%M05qK)>(s@8EDI^B;v$*M{vm z30He~Igm*kL6{L7Jt0U(hZ$G72;3CSn`|+6hZgMs;ZcvTD+pJ>SkCm-N)_khY<)!X z*q?Uxh{xA}&FbnQI}Ap$Ofr1JduLdKa%PafF&-svCH3swn5HT~%W(6#mp6N3U z+e+HB<0-0t^5Vemm?pL=jFRU|TurUT@sN?kL3f*CeSI*(j7+p9Xf!)LntF++1)e+B zf5dY_=_}~KXmd9WzbhvBd-0>wDPBDo-Oz~qSN#R|+w3ynFvP(TPKz!0oVcZ9Yadc5 z`6_n0-Pq)v3o5@UX?67Pb{qY4#4%Dc)jQTAYA+ZBmL@62V^JZ|xm~HA~xKrf;3I3cp#5zD84=;D2 z5iM`QJ)@VB;L?RaHtOphiSvz09QE45WFd0w9f2JuwID`Y-Egs=& zQXy>9jGVsFetD8VUljk~i@{0tW!RXnJD^iceEnxu55EX< zRu}B?2&msScgUxg&$P8ZeQ^!K62I0U%UGl+jgQ7q_J$iNcbSRtdVR(9#@tHreU!S0 z$ig&P(kRccP1CY%pbvJY$1Mg+GlSjWX=x1t@P@}Wk?oIUiOW)cdV1o{xO1`G^1*(l z&8YXS89W>X;U1Md-H6x}NhTk_>T{Gl)NI!z$e?%ga&8pYO%++-I8LSX5HZHF>z$hw zxDxYssskB^N)T7np7U6VS?L|a=xigkQ>%RiQ3?Y@VaAAD-zK8jhE6mWh=MSh!hWhS z03_?1TwXe(rZ**6at$h`Y!X9rSv8rr7GzMA;=KPpYO45qyQYh9)gZAqQ7e$`i zzTnAFirlTL`PMD2%NOotBttF@rj<3IB5!CIh;4DrSliaf{Q2O(HhRxW43LzEmZYwOv!Yh6Ub|T+qvTHjBlpD(4XSKKcBa!;vfcX^;kES1b+0E4#_#!jF7ZqkYI>-vN;Sk*yj(T69e{cp?c#tACLa$k~SBf>C91F}l-3QN7g<-0+_ z$S-3fvg{F?K(tX&Fq_wsUM#fF5$=pT&0K!5vNkOfa3<18#wFvto}TR-e_XkoAC6s{ zB>(^9Sn*;IBBIuy$pVpMHm*&SWfG0^o+aZ2;8?&1HO1$C$mw|kQB&KhKnMwCeg{)i z-4AHK^fqo)<&S4F$36&8gofXCy!yzAF3+{_DV^+TpxUBZJ~o_^)s^}1!gFRA89Aob zw$`no1p(?KCjGG2_3e8vhRuOQv7BKCL_qELWR)Kw zXvC15+D}HTc!ke*(TKEvlt5~NH!6B7K~d&md&w*wxrOG#tB2`pxQsVHZ>JxpHR&f- zyVv03ryA>@ZFv)Ht;X4fY$%B~hile!nC)b8Ow{ICp4xFx+tBAs?}sEm_GZhz1ic0a zvZMW0f!l?q`|5mYUbBbcAC^)yN21ApL2%17o-105Bg8(ZRz!e$mRrb$et}la8usTfcI3G zZ>M$k@$}&?0j3E3hlI9BrB!j22En@hB0W;(B}2}d3J16@pSa$}5Hda7ALhS!#?z)d zG?)H7s!axae|>PKb$?Vq@<&Ba?3fbm4mP5)m`(!0GzrGe$+y`LI;uE$I7UWl3VOOE zVWu#xu=Bs~RRPY1gRp$}sqlg!G!FH5s|^1)3tx8Xd8R50Rs8Z3n$_nWuWl(%x=LvU z@Qd%oPh!~2m8hzS8*vV=DDA+nT?KC1N8$84rJK`)Ihv(&eV}~4#`Tvqp;@0Q2J5KR zY-*C8bRTlZUEdgbd}1$|!yV&PLeI^_Ueddwi)H@Gf__y+Co_+cPV7k0MnUIi#X#nI z2$koO=s2XFo`I8FQ{hbERB~+o`NQZq6a7VmY@5S*Og@A!Pk_O=|Ii4^QmgILKjttD ziJ{;}jxf>LlFN;Z4yx#ddXn#b`Ihn8QrR~;Cw;^W5%;oRDR3Ga>)9J-v zS&f<Ww!Bx#*TbDB^5cCmf+!TVVS6qU{-&Kg2G?nN%GR@kOhq zz1%nK$0!lwy3lj=n79j&DCItJA#nnp1o%VCA$%XrHcK_-_y`k!6x*}ktzoTM?kzrF zbCO|hNkU|^*y6w!gbKK4l^y(9AdFmPTK(dfw{MxcQSRGbuR0o&n^A0nTe7P8HC04V zaOez|_)HJWQYGJWc86(CZ7v8(s%3gH5mdbeEDuD&|uYs2ZwT5nR^h2Bf^{WGR~>#V+C2Qx82EJYMxsi zsan}XZnb0Mn-=#Z&m@A7a?0#qiHDq?Y@2^+l51ffyN3Vah%IHTve#-sz%r{Z=6ROr|8Rzv zmvKN~AH$NRI#odpx%OseCtEWxRke)M=ZnAM?e>tK4u6ja&KzxW9NC-#+1(6#?{Ws4 zZ&19W#BVT^BM!1V5)3Pt2ls{Y;U}F#sh?);jg3aGu*&3_9VHI;;O#XYk5LnD5Q~E6 z(`Iqb!I3Dlea5cY({nNE5v~WU#;#p9G(QPdevXrc%=pov{wqSsTf4a9NDQhGJ6Y`_ z$ksq+TJoHT@$VQ!C;4FzS1{+Elib$~#QC0xv|JsYPw_dfRBhZEmY_JeH1*D(W|jOc zqGC~`rp5(D{-lHIJMh9%21n-KGg^#+kCi=vL~&;OHe!xS4*Jz@JiEPVATa|+yh+gzXFMEN zbikSAkddmO`@v_gVT6FCpPK(z^+HSymy@}&k!s%z8k&qjn5sNtxj}dEPazzLoyrN) zxjb%Xkms=QKH6dF$4+&-j~f9kQmt7c%h5c={Z74UBs9)A1ow~7}j!~?3r z%N1wH9EjVfWWLbbWPV;ac9S|tIfsGe0{zd3Oj?&re!yPv={?095o^8R9DL5ri_C2p4uNR`p*mb!O8fNWR3+7+8S z$)3E^04{LTHFGtFxp-O#g?%(up|XWXw`N#4yY?9&7xWP9)gtc1%y~Qg;fWl=M&>lU z&Gx4TW^oHp6RwskUOTYj=6H7-s$9@!zDKG(@kh`-^9qoOI7F8Z+X9u59H?GWAKg_X z*Wc671s(>p>D?WCUPxh zrdVK}^>pGUOLDXWbUuO}gKpDaliamEQ85@|QNwAQMRTiji&?#R5{&wVQbW_GdvX$p z@!$?_6DiOgZ@bF1iP0*T+?-57Cv|5BhX}cUIgvAu=G^_JFPEwG{Pl% zdaRNDItl$K3&ob&Ue`NPxwm^{^kfsza`D#U6pUV$US=WfCsKNC!#s5}8X{tuonykg zrYFg&&T(dbPIW03v-QYg@xim|dojqR&< zCP}ujV*9n^A;pEh4H5#YL*0Vkzsl$G@K z-{^d3F2|UMiEWYptQp2-j|{Pnirfcd1Yij5=X!04cHjKExC;3EOb=00bqWun>Qw+a zK*qll`w+K2?#EvI96%qULgOOVo;ngco>x*siR+0A5@aTN62ZqP&t+ysZ#)O1GJ6T5 zu(Le4LW7%F$%wEFgcLjT&Q`X?#fz`F2;%zM^Nbwa#;JZ$5!1jw1YkxLLhORr%I#vq zyF|g*q#nP)H^xU)X@28HgtBuQdm-6uzEu15r(BJsebuRMU;89$B$UL4+WHCB*|?nt z>Z~KC`0J-fPTKPz>*Yuh2Is{_A;tiw#tL=DkBSyz;o*Me=+}I7;z3y_M(xV{5zy-- zd$fSQlZZMR&o?B;%^-pDLwrp+OfBWeRE^w239mgE+aPaXqyJi%z^2_z9}}{v7GHT# zCS{cd?uO%=;W2yNIadv_FvaQlM38bE@l^@8uNH2%VIJOL*QEXdUv@~tbVajI0!*tH zPe8mq?5VS8GK~oDs+*l|jZ>EprgY%~H&lzR*-~t}aO8!3xvHi54O~~t3arKeS>6{+ zcErWTco-Ua(VNlF4NofUT_-TtmEe92U)X%h9>pqiupnZI#-eX4VuoQ7@!Uq21b4^2 zr-9iXDquzwOGP3yHehVIO-@1GIzpu1O7APtR@hE;_hB@S`x&1xp5r{;fjb`xDXq;V zL-=oio9wC&q(vaPrqDhU)#a9zhx{BYnv@)2UDEZlAG!EogwGUS`IQxSC}ZH3v|`*> z;&Lc_ss8NdSlx0CB~@cAhO_9)M*tgkLH0D? z%!W(qsY5@oYMd}k>ZCTpKu~43L@A{7@4QiHTez}XZqz%A%9G*a29x>OnCgCwz>{Rj zNu<5_71%i&h`Y~4p?MOaU#hFP)pfg*pFA~X-IC{e)K^izZuWXgma_v-&u78Hi2v$! zbD7-QG&vK>2ka1`$C!RL)q1 z^flD4b=#6FUk525Y(6?~6yr$p9Bda?rsc1y&%>=Z(rT@&(peNq5F;nce;pcAG1PwJ z1j;O|lq3x=S+{54yDT6FP;|}s`vuGRiqxkdS{6>CqG<@8mWsuQdvx1UE{EHQcf^Rvclx`ifn+p=8rtRN?; z2f|oA*=@tS7>RMNeGjSc#uGw46ApvTGi!9?8B4UI=A%B1QA2o#G9-%C0hb;n=3j=K z8q7^$mmoLsEkVGcV^e<-5`^3V?>`?R(3f>H7AIdtsd2^XXt<_|Q&XM249zRz@%)ET zfvtL#l7RLfMdz{PC=f)!U&14fc<((oy!W23pPs|U#>UP_sHv)#DT!vw3{eew#9Oel zzC|sfaZB{#$AASFiV+7$q(L4%@N!}tQhbYPsxH|7@8!akwl*)vC%pB0mI#*I1TbO^ zHGR>I9aJNm?kuD7nm0RNI20HDy1~fMo0$>Q2l?cf}~?gDdaoqj6n#R~Dk)LbWO6npsC) zJK~mp7rZ!7i*#c%ENqvXEW&(wMd}9-p#>0G8Q3~>2D5|+Cx1WLUl=_V-NI}>3F5{= zh3YmOqFs>FrJ3RHR$qBS1eZKEpta8@2i^v2mPALdF0;=f{=}bRS}-fs*Hy7`BICNQ zpCJ&=@$B-4^Q3J4ZZvJgT>K_I?m)_#u&rY=Jp(ol+MuEn_-lQSljbg84@KV5-OW_MoV85)s(WyB~_bSasvt z^pM#$M2;KlF{uGN=H@;3li&PQvU`9;wvgp=6t3HBW%hhkCaq1h)CNBTFf@WBJR1C# zTSv~d?4+#Y`-LfMDO*r2k1Y7)%9ZIAytTbO4(7II%jf$5W%<35l6*a%na1{GGa9e4 zh-oNY9Y!#zT`d>Bs!#I`*ShBJT*gM;Dufz;a?_VeyG8Q2Sp;NlQ%o=dgL9>}2oa|Z za_Z+f0;BWg++oP}Lm+j()+__qJ_|*WzwF@LdsIZ3gX);dBr%Wp)Bu{~2O-~0YkJ3Re0=Hu;3&l^RnsK7_@7*11T==sg(uCH?-N-4>he*M{& zIO0tm^ztk2d8m+55hsvZ;iVn=l4m_n-)+2O!_%-%ACgQt`>vSw2DwRJJ}!+)&x$v| zllJLs`6x2)b?_$>xAS?(Uxq9Ls$D$4rso@m zOchI=4AryoQ3by=vdp3Xc76@qetIyuMtsi~%# z!MR|hC-(n)tc;%mP6E7J2^%~@1XtQKx4urnc@j44W6W;9>y6fvTqrPVV6pmHVC~N zXF(CYXyy-b1Zf%|VbZ+RIu5K3*t65rs6l4@(Af(<20wW(Qd`8U5|s0*ieLDdl+fwA zf2L@cTbUliW?-+j1{DV)+n4~`!0>8=NK|eX@VXOZ(!=T_L+b-s3`Pkx`0!oOM1+mm zJRlik+3y5@T>Nm^iWQwmoZOn~j;#KE-o8pNyCav(d);KikweBl8r04tmDtig0BaE= zvJ#>jgyk0Gy&nby3!g2Kdm~|-875@)JGIKKdd#T@NLXb#9BoV25s`0e;S|$|eO(4* zw6x|}GUf%>kUyhIaj}yLD_ipaJHT4@6SgWs6JN^fBKii^rFkxQztpt^{$Q{7XwJj( z#P5`KCOl-~7USXG3J(;JP8450s|kfAc*aiJ3WhW0!6qb)z-RS9T9jbFprnKJ*D)*- z^Y<@C6Ji9$qTl&YGv=DJ!(!s<6iG^W_n6Jax(FS)7Gt*Ocmecdsx*bz&@p*aa_`n`Ey5_1d% zR;DneGH9%u)ZAw|-6_5?@I*bYVx)1-K9;?~GoP;1dh+(roWSh1ZN~zYW*Y3Ai{FQ9=`~7OFp=I`%hYsh298p&cEKvf%ar>Z@mscy9AGVFrF~&oV{-FHG?%l_ylHdIkmYJoQ-x7Ed_W!PYl7<#v;# zn4Ngd3#`~L82Q7A%%#Hrw=(qzbej8m`qkF&n35=5DVppolW^*7uV@b~vJE#qXo@T} zN0g3Ze7tXFPrZaO)o{ks=er(o%PJ;IvJW42nNOdH8mH zuag}^H;U$;w;tM-XOKO^VE4^4zxH2x0-5Ey$Euuh% z*5IU&16zgz!)=CC8jIVOxTTK}E80|j1ez?S`sInDn0K?$2l~V;1^w`PoTWzO{^BS=wm{gMMp-EU?-?4^; z_dJ9P3nhLp#i(-idrd05ZCN?&+W^>ao0_)&G~>{75~)tAtlRnbcvekd$;ZXWP-&9| z9HVAbycr01P{U1q^zC73D~K_ouj8}yj!4@l5U# zXWK_81mvRp(idwObXem3vNv(+Qk`2_=++u6*=rw;&K_5vYZPkx?k@n-_%-qMznFYR0zyggW;jV1T_SD(S**njZ$wXWHOLtJ{;lp`S$=7 zd(4o%Qu=&!TJR)f0B{EP=>tRGrF^1QBiXMZv)#}_Btc23xtnn_q*I&{q4dTEU(wm& z{Wx{ZC1gkp51eE86kLs~8WLsMPm$Ol|!M zNmGd9GLn5Olww6|8qndLQ&zMhyb<*Y&&bv%)iFA)|F7luIjbpufr=7YLyv>V)H6M% z%FcbwZ~Cc*f=Y=TN$9qm%3zchb?E-g+p;_SYUdo=)iQ_?lS?^npi1t9A8}9q=LZM} zJ8Qtms~t;?VBAcLS+@aKtJOA;`6+H2lRm@AZivkniD=cCH(vFAL!)nCYTL;#uF`Xh zGz>uTLBU;#CpJx`aCGq$q2hp;Qf5 zNZ;V>=8$(HmWMolCQEmU{eqHXb(U{*&N9EPbTfkSFz&x5sC^XKf4cs$7qyY0KWExq zn_(emFF2C^F{>c#45teF)vvo%QN7gGP|=5EBwI1i#8i8EXUyD7G3WjmW>Q^&P#Ny0 zTCgaE$WVJPib8>(A9{Mbh2oeEkDZJntJ8Qvx3_s2f}4FGr&nebi+ml2DyfPu)XEBd z=oxRvDh1x^?7X?uc6=GvhAjE95^?B2=oBEx>~dK!Rvaedv}1=4+~6iwTF_QJ89bwz zW~)X*jxd}+?Xs4dqBln_G6yOBYzM`9Y1O5P=%t#IeNCP|hD5?;x0nH&LtY4&*H|_o*PStOF7=6s?+^|8Yw%#z@ z=oclk&1yKgiogfhg@a#85bfl0`Yq2%GWv1l{>CkzjhP8w={J{^LbkDW%az$^d^5LU zNTPl}IGEEax4^)yB#Eg;7y9reWwJr6GYGv$4+r}`jM+VgDb^tmTKt$`ThJ*?A#l_p z6up|>ctuXrXn%xlCQ;&`_CU()jxglEZo zYbp_aWM^9uH3@M$%d4fYHEjIi*@}vxlebn@eSZ?R>hhI9a}BdBW4BrmKmnDH;o#j;B;Hm&D_Lr2N{f-4#v*w-24UT zRZ7H6qj%*a@%#ePMm}UuW?SU%ga>o4#_a zz%q&V615<25AN+2b3a|*di(7*-z&rg`_Q0X8rg#ZL(Q}DggL*xbfs#d?r-la127k z!B;)#`i|)nXW!~v$XcN4p*(x~!$?pIN zoYHM*(ru*ER)6oqNEJQ0s12>~gZKQj*{F8iHt5btkJeNj4mwZXX5uCSM$uWaRslEf;`$Q#+(cf}9$tLOJ2Q}jX|IT!VJ?HNGkr9_ zh5Bh2EUkg*o|_Z-HwH8CabB$=gF3C{*iO(BUSS%XAH!7-C~9^BgRi)y`4TIDBCJu5 zvwPj@RpMR9)eRQJTB`EhJlw-#AtCJHuIMp&!>MaLUW^)`F*)iK&x*|Es#Ej1h^A0> zYaM*f73kg_1<=~L|x(%kApsP_n3M*cM{G4YDa~)K5CmUzgW9r+%e@naL(6ztvIhaUm zxmQCG=>eL$%%Htp_hCp2suSNN^#cy`H6Faq6H68$$C$aGMwLwmVifZ}8UsL9d$@1c zj^pb+ERjt(Z<0x5lktO)Dy34j4p{{rt9a# zxeX!z-+8>2icVVROnByqen&{{h(A#~EPXP(|0=9IToU@<=(6+bU-w-7T+LzQW!Z2R zp2co+kYa2h7Xt`{HH$?WrBPNWWaT3oK39tm+E5&NsCf1OJI^3eka@x5STnK>gWT(? z(u!o@H?McCq^%j9l%#`QQfI)RAz*Fzq3~$HePog$LVn?Rl{>EdbrYT8V>RN<*c&*f5K!xp{^qEvsqr{s|Cvy&F3Vk~+c~ z|F9FBNaUdC_xGC&u}z06F4`-Mq5X$G1N8yd|LM?iF;aJQox#Bae^zZTC;{_A#l%GJ zxVR@n28)*?4N2E!;qKA8%jO#7{C6HK77SdKghay2IwxqO-kSP(y(FDE?`L-@aYFD7 zi}#RE%Z$T~he*l23Ha}N+B8`v{fNZ-lq-b|DKPIyE++LlRMP-d+sf& z<}0|7OX_9sZo{0)9qbh6@6cRPUSWk#myPbNaE1YsW zt9NJj&!v1>$4h#uWERS!FNJTyQP?i;Q1!0hzW^f+lQp1a1X6#DneqGD(NpuivFYja zwj#gv*O|}sBH)Y|8)S)9>RfCT8W+OHni|C9)Ih22$NkD#823dX0 z*4FYVTJ@RxIBzYqA&A=I>dIrqO&waRzI`5~9(7yy{m8IbSjPK2ynwP!;{m49wK3c) ztMiVA&*%J%BwC|7xes9AoR7j=y_ur-Wwd!Txcjh2?V*L;O9w7FtjQ5H@d3S90G_rp z2msVW#=Q96Xp)e3y?5e91RBGk7jg&Qyk8*q9Wlu-o0LL?P$m)K1!XJ~;&l0Z-q4fkf` z3vbu%&E$=zVb0{Dxg8)x+{UmVxlR$Q`-M2mToHb1%7*Qr{KQ7_V0R)LS`e#LuoTjF z7eH&CT}F{vh|nO&d_X#A&IQzUlZGwUI$7}PTm0nST1hO3TVl3S8^F_(HN8*E)6KQ>3wGuN?VHQcX zj{K&xlvr$7#8L(#x;<>mjtrX2iX~wb%PP4P%xlK{LvG^u%{tF)@eedN1tcwnD^`C| z|D0+qc^#bRoa@8A+gD~EP;7&6d|2bPv561Yu7;HIpvy0Hi&1^|&)NBX2ZPIY9j593 z8}*V7NhQanV^Oi0o?=T*DFVzQ>z<$1T>N@YONzw`2u?WH?4m6Ei7X+l7&T;S7R+^& zX5^zCHdjIGqxh&-MO>DE$8C-jEwdK*+Dhdc{eBeBYHT6z+Q96YX1acHwnLiF8*F*& z=A3bS4Wb72t*L05&M39$D8eKuSD4>EY6@E12vV#?SE+5fNZ#-jtvtwqK!z`3F-1SL zn0@-z$me3R=hgxTvmD#0y?Ef6p*iH)vh8fq15l15}KM)kteGrT}E9klo!}>jz z@vm0~`v+W64vsaL5>%{<;a1is@-6S6JCB@yTl<)o&DqIkgsp`U#wa(o_1F!JJuW3x zo64kh_jF&{%qqv#JeI`+AzB*kz+IM5c>CCLxSIJ4glNoCF ztX#p4p+g`V2y!qkqy2Iic5vYOay?(K)#306v^Obcp@4SL^rUXF$66d+L?kx(8eKmDi!5>N`=axEybooj%H`phfY->S-k)F zx46Uhs1SSjTo?~<92QJW2cZIj-U%0J8DGR%?G*5$EnFxxd5iXVq7xc%%kO4jRy^;V zfu6D(F=Q5YCA1))!7waKlvGNYxAAf~#d6pP=(1^^k+XsjbEL92@wb9SenVN%J*k&U za-4%PpAP42KD{J~P2(y415$L2As!H|)}z}M0YO20z{E}nb42_Om{T?FGF*Iz?AKMB zc82DzcdqsR=DUfxq5+@qLnyCHDkgwUQS1B0%z9NvbyjvaLmUDtCSZxi{qWIrX$ylEnE3|cMPV?v3LhGwzv>AkJlCfPH3%p#E{pFUqOQ$b z^X^b_&?uenXzc&@z)K|@!hf`98*$k0HF+R3;~H|$;P2{SZIqww@k01VMKV;TBom# zV22~!>1M^@6w^i9xz*(x&2ht1IE)_Y6FH102&)gK*;jB!#aDq17{~i&>iEGI2U3V| z_#jLjZ|^aKJ`8Jrf!ziYFkl-_sIZh^xIyQqw$I~{5NivIxNId&lfXn&tQsL^L(o>b z&wp&fheN>5<~}p*Fq2LrJ^i*92b&k=UWetVwG7j;tnn%yC+P}%5NZdPCxfj>(^q^``)1LY5d^_#Om=v6)SyykJveKAQ@vTNn3al{QyDL^I>I zl+wn_1HCHi)?Q=HV;n9 z^f6;yGCSOv5{och<5loxJNvIBU3T14Zr?7;(;h(hIF|}}VV*l$2{vb<6wbfVs+|gFpR=~YeSXiIfU4Gy%xzEO zPhVc+{@#2D2?Oxy_lCuGa-NsBI{zgDnE91v!+h3iw(@ReNW=^;vZExq3O}l~k;kOv zb=c)ZaDD1mcZp%N%?^9+IDhk#BgoBsqQUPsY{*xS%$1wT%pH|mj)i#X4&D`%?gaNe zHF4iw`^rL2RDh;DlgPq!U&>LsV7ni0U43ptHg((1=>@DSA!PaL(Vba=&BL=anhW>7 zl}^HJ63Ny^>!XfHwEzQnZM{ah)iXl4*L;&TE+GDP-=_N#OXCh=lB_yy&7HEExI6e?J6QeKraG4%7Iwg9%FVCD1o?1+& zJM{iooTE|Xd4^>@&$QyO)A+83VLqwq8z@Nv05vgJ(CR8sQrD=~{?*jrUlC=@d5sE< zKUTciAF>blq%tFoi26+Fmwv(cC&7#Ebli?Ytf)3_h6A?3iA}REwk)C8vq~jplb5B& zXm6oG;~<|pK6}MEPNXy7Lx9eO`Y_M!7O*Wx_pt&5gQs}!g zOfS=C}yL`w;B3(5)8(^sJ9yN?FDlvS|zO5?U{3D1owf*g`XUN z2hh(xz(mJ#F$*?LWl94bJsHx?UbJH$Q@D3Ptrt@^kRPEPj4W^FjTW|0@3vNuhpaP^ zOMW>{KbM{?whs+3zJS=0~edm z%H}2Ur_QODS!y~5r_zcs_)*1X@xf>V7rl_=Sl^Vg8J!e4TV~$;+2De5D+jOMA|)8$ zQ`ED-)$OdZt{h|TE5VgZXtJZpZ&%Wrk{=l3VcLG}RUz&#&4Lb_mhkZ%vSToU{y8)D zP(kQ?P6{u}5%(UaTH@qW+Vd46S7%PFJ+O?9KOfG|dN>z3(tVVB2{dIn5iWEP!7OBZ zw+*>1X}Uq|9;%2$7fdZP0*rvI;zL7xeN*X&93nyVqmkK&-L3|g8wsng6q{q!7mlXS z@0jdnoo4B7@#Iw`kxNvk%`|Zmu90V1I02P;bup>wz<9Z}Y8CFM>b|`C11w(Ad?NYC z0=B6Gz3D-2!buD0-6TU9+3^LtpFY}-VppxfM|ww1enl-iGe_?|@ZC_DeKh=L&cjV_ z%skIEC3g_0SvM>&I<4{q87*QZgMF9>HT+XjZ&6dG=DvgEI*aM1Bf7*gQRD9+&246L z0dGngYF$#|YL|7JwdMbG8qr#rq4FZH`cji~tTusE6i1m~X?u#688w#??a zL|aFt0g+wB@R(Rt*ylil*rahLUu0no`f>h`{B1Pucr%+FB9pPm`7#2=X?SNG%X}T& zh2SK%=-t^$9tkgfxH#j_SSGDR$ZKh3RLhH5SHz{%*ycXs-p7 zkmuiB<)u;2i$ha@`4pLSkz{W2_gHTew5KCp!N9a9qB(TMxLJ~#@y8Vo;IB=V$H%iDSX6zn%SOFxvGm(M-5P~;gMS=r3dPMwhxFk+5?e=#A6tlC9^|+}%7;c-J0?2d5hcS(qfd^T?scMeURC z6#jdo)#^Dw{3_5*u~paqogqfE;CtM^lTaEom!q9P_;Jq(qKw}vM+w~XhYYxvxG2zy zg$a668X`aiX89Xnjbj$Ooy&`$hGSSQ>gyu<97+EQ{bjDvo`rvRFB!77lpSx`38mNK z>7f>yPhHxht0E?XpI;X=BU8pO#I7urU6-NlWCVuiybNE*ZkycXX<=ekzorbLaodoR zSN8uMP^M|a<3{tVN5C&+axs?r`6tde1_@0CNBt9aEh zMkxrR`&;)49VzPL3_54cbCUM=RCH1UK2BU4qq4TPK>{OsISoDh@9VH2)2s&jT&}U} zcCXXBm%EE+t`hVlFJPUYUoiqawgIjYDePZu%L(ZjWht0(UDQU#8#P0>P@!?6XNdlWEdyks)J}`Q>MJ5+Vx0+@4qtL{OYN z>C#i^ib1oJ#LeG~HaZh(IPN$qyuYuI_|f`uubqzpIhq-a82mMi1$wFFy()=#Z>G#j z2j_pP+1^#*uTi+oNuJ%u=302sE7|;uYa8>9uT#+ho z@Ft!-CZqMG8R0r*cA<2<;cD%&9htv8_$Ya_`^#)mchwS`S*YX6;UiQnm83tb{^T_^ z^8sd)y$@TVDs%scG}Dh*`Elv@q6T`014?auQZx>YhQB+Sk;dBO!wa#ZgZA!6jSVae z;r}O1{wQKQcQnsr$5<a^UVf8lIx5KfIX=Jtc5m;~`}i+`<%D-Gf8_ zrYY%Jte@Gb#e#Z(9m?CL@=Bc)-C?$!y83Dp)q2+Sz!oJxN zP5~#WF3Ra_Q#kt%v-NImpDLPQr00M`! zK)J=>gA>Dmsll6J{>XkqJ9kikNoYTHbRXby8PT=rU*rOGP}_zrp!^jnm-02QysC-b z7}#1zX7j=!Nrx()RiV;23s0p4W3=W=Rvsk|1G7<1N`2MYBI(q2H1T=O&;Omgm_=gj zVA@lM6`>`N;&ulOWaQ;qq@UZL8mOmoU8@mmtL2Qd&D0lh3a1O(g|ZA`{BbUQY(MSE z$uC2@zHj7{@TslTYdY4#h$8L7+8S_Pb3V;a^vVpaF8iMpqv_tS%KQvmT!Q2Pg)N<| zxBeoX>Yg(Dk>ebDCmtgwW9FwP9+r1ft}8w1I?M{=z(O|PVI1ScN@0YN>kACB0Qu6% z=!s5~Ws~*;vlC$2#egNkqwUFv-o!N$mz!m6pttLo+_Pb{f`)OzLHNHdy+tR3aQCGr zD^Zelsr+Ji!3TkXtcJ5mFnwW)xsJobn8u}rJXTGCJ}P>9G&E9G?i9mj@&3d*V+zMN zx={>-={qOYs0U2xm2h{;V*w~CiRChi+*3*mIUksKxBSg!ffd&6fUm*eZ-vwZM|vEI zSZJ_S6C*Pm6X&Sd;Csw(#{4GeW!v4TPo+yexrFcDD(yZFLj3fW$z8P0I2ya`-l7)i zORu4MaBoNx6HaJ{YQ8$nRi_s&$e8crEG%h1V5;~)hiS|u-kPqeW0kTk`Cv2ZCKQak zByOiOF!rYXUUi7L((oLjv43MnlnLY@g{vxYKvl*4SlDirt@a?KCaiB1=S%DI)FHQB zso`?&cDZ;U%zoYoWv>*`7SZH79SeN`*=j-huG~Wy2r^_Y#wb#6(Mf$XMU|u zax?zuIY@>DOK!5uI1a&jb;(g6hw%mi(e12-$)#ZN=h7`7OiP+5t}J>bQrtf?1jl`=!L#zKG;-}YTbjCl}a-hhKw7xF?UWNzb~G) zA^A=owP8FUci3=gIWR~y>ZaB1M6BL-UMfY5i(xb(S>zTBoKO4J-lcldx9c9R*7RT8 zrY~jOO>AI$K_i9$SFo%94d?u`57Yg`rMU@&?lCKQkX*ACni$#2simJ7xY?M0Klx)A zX_~5WJr;UxQKc5C#dVrnr%$WI;)7Cl}x}UX>+Nv9rfkx|d5CM-sHIf?OyL2j@4L3%?XpHdExMksM9W zVP8x;kp3Sky!<$azma<9-<(cbI@;TBW@ACJ$qujrx$ZC1t~$}GSi;7?-P|rj8LJ-B z2WQi-^5E3FTkt~~w})yU%e&y+IVE>HKDKF;#9Y*x3Urr{5=W^$YH4G|9)sHJGfuys zZF)Z`K<)J6|1YSgvxz~KUW?*P>_o9~5B(>q>u1aN6YnX88x}XxTFMN1$!LbQZsh}< z{@9f|BZ&!4Xc1w(O3cK?p(p4Dip=*!`H%j3 z>?>T^i46%tREK#B5=|xER$bAB<6tF;+q9<6(V#P{!ZT~K3FWfY0u}>Bm9*y{6)TnI ztFq~6wNw8Lb;#d1Zy3xRNyukdJP=?qGC6e9C4A5up)MTioJRzCRBuD7SEPHve|A6E zJ);~WX!^ey#PRb~=-Q!bbafeUd0HP6A}3TNjVC9k*8%)GYD_sULl>V>3Zh0^HwpBX zSGt$UmcW;K$7Lx@-&x1Fr^BbuW&APAxJ5m=o82c#1eGRPTUs0Y3dCF&Wc@`p!Yv_BXc3V+O#s?9I z^j!kJA7g1D{iGBN$99^}${0d|7X*rOoH3+ZQru!PQt$AchPCTUq4q<%xN&Zsh|`kf zclm`ro+ufP1n0&xQ?};ZC7t@p+kWNmUQb!gFT4-A7vui-ln}7FV~{WW0qdHXdh2i; zWVv)!*7qw`(NbSRDwah#C1H*>its1(Dj5qeg=KryY^&&^m~a{6I+=0YP~w*#=R8d( zty_1tgHmXOnUV|oj;AR63K`>XiBli4`=}^Oey&euKWX(;l?q}}g^$^CDXJ{OT!7uo z!vKqp9{D}@j;?N+&{>@4ky`EXG;7SmtcB&sWG75dYnQGGpX-YkpsU|jQABm$W)$VF zFLqDe>3rDB`z7_1qgby0cWIpB9(`PJcvk7A1NL!@zZ!QO9Es6fGKZ6*r@vM>r6Gum zW-IB*l}x$uHVEjl>uR1t{8%jPe^KR}{lnhm)DBG#n>v>7^??t>fTTMth@#D`D zQPPfjx3c=n=*Pn8nXk@uRivQd=DThEWz$H<7Hqd;vs`~YUm`r%zr`Qs5%VH~rmN{0 zQdcU8yYO|+k3a7@3|0@`CESL?e4{nZM~=*I!|sjc4?T!pvxTL%aE-bs>Ni5nbeFfM zn(LKy(H2h=%7`1Yx^Om)#C}&BDgAUcer4ku4%>h=LR@f+6zI1~wV$Kz$*TM?;w{GA z;E%GSDWoxbWmt(&M@bZ3=&IL2Fy_B@HlCVWWgA$Etc)AgIdCG$d<;QZ znbN99%}Z56T&L*(w+~U-W^Q%v>Jr83C_VH&yd$ELYVamL$(1)A95B9&Ciu~oA^!$= zAW&iak&7^`dDvK}73c>R4sGsRawW$c{sWaPcY+HTqbj~@3r=GVg$E}ml_HS-|017w zDbvphddH_LdPkw51uIyiGAhwId$8D6yB5qOsdL57?!0JvPo~#yAVV z>@)3YxKFlqU00A`)D(A$$#{zsYlAe8!ZV-`S!`$}9De{U{Ee5*&zEGfv|@m|?}X#A~b9u5_N%nBL~lNoQsN z27q+Z3WJ8tEO}0j4n*+(iTrH~%BFit3|Y+n6hmXo5!22Tccdj%)0Gvdzt<^vxRWk< z4Os0*a(-%|%2vOdlnpv&5*Gts!All2jGheSJatw$oA=35v3-HS4KA9&qKy)kZE1MT zI*TIo8TvC)V%wg<+igYPZhazx(8Az=5jrw~v9}7lHmG3KhSL~!fZmLAoU;pdU9$FVIfV>H#g z@>C|U2&%1uFtgXal3Sw7q*p25h*bG_NY5sCdV`fF!;`F7X00pYE8i)`iOR9lA zI6UA5GFKz!e!tdS9wcn^b0Os<1cd$l$f88hn=PwRz#?T z9iPLlGHK{2&#R>uSe~f3QrqpPjI5rlc6j7y0bvu7q+E?L!Jr}$QMe0<$+M1|7f;sj zCJyNm#)UfTHnRP)_lj#x4rEHJv1uj`3BipopOFAK^bzF7MY9|w35(qsz`BOpGQOih z7I#4fC4*N{3?Aqi$?gB{h$IpG5J%4ydPR0#+^mfQ_9S*B!fxts_vv?VF%8Us6&dpTCozu$PPy*pli6Fq{7y?Za2ulNJ5KL7d9HwEQ|?R zyGPeuYjQG%gJh;w?e(1ZcM2jVUlVP#Jz|$zs|MpiY^;POz@lBlJu0Ir(H84Z}k+$L*X}>W9VW@5CF6# ze@|6xuSX($ISbVoaGGbpn2!z-$)8(%z7LCHHlK=-5Ug-h>7Q633B^Jc)L&o44&|g1 zL6K5p`oyx@nBI!KR9nPTisbpV@+L#bGlknlm5b-?Ww36{J_j(a=ggOzPAEg2A(u6Q z;17+6-cW&#;A~?g$bsf(Sb_lPwQ`~i>h@ddNGhbYY)O5F>W~V7l;D9)2ewW%efE6n z0Ld`il~Zf(&9>~xr-KZqs(SBmf|umdH|QEKeTgO4Tx8n(k?}|hIPS=Rz@qOmm$?_F z1-!YaZ3^F%QZlb%u_86`v*KOnk-X4ANJ)yeIK0QD3rJxQIx=ucpi`=4kKtorw7RaB zQ>CGTnIUzEk^X`wwkHz)VuO2ZHx|#OZr_?oJpAs5SM)4*>xkl^Fk$Nle`5SNav-_Q zRhr0m7E<>wm}-`-3HlX(PwW_xc^d{H zS(@Vlpa#28h5+D02yKzrhGDh<9Y) zt!V^!d3#TKQ!`UZbdUTV6N_~BXkzm`Y#Ax~7j?I}&Yl?YO;i^-I(NLk^{Z!yV9Y+C zxRb&_By#iIK_tq1{j_Qp z^4SE*nNpA8SCMjj^Nw7SqvtlBiZiI!kJ8X0{i6Q!*J%$A?_Eu@V#7A=IOy!xjhU}= z$iLjsWdAMpNV{uBhsMR7c1uc1m+UAIxtXps$IDqGiny}Q>)y74lSPvTF7;8cy=9Vl zd!_5AT%JKte6hY|BVu144SnKA1>L0T0~G|=cgllosyr)sIY|artH~XgCaIo{q}HF4AR7H4~cq8CPu*jLT2ETaa1t>B?`=j(x1K_xR3T)371q zXacr?_ob9QITiBcQ(PbBsT(KD!)O~zc8C>?q5I;xCc0IR$ywIzs4YhlY&cOAu=&_p zh#2)Gr|A4qVqu%mk!HNXW$3_4g=`J^l*vCUi;o41@UfM)3vC8_8059xr7%aeA5Haf zmJ}txyQrPJUa6;AUj!uMSHFk&D(Jd8;|!%rO)Cm;cK$XZylsx9v)EW`xXpdP}<8RW?c0>tfJizs0zQ7aj~cVwjuEL1O`E? z3ry%Tbubo)obf){M0cU8fe~2q*pBku)rOl1H}#Wo7=tkVQFR8yAL#?#1U0*3cu!^a z951ijvaBkPYIg>H?s6h;i5N@(=^`L?T5yGbZAI`t4)TJWv0 zk=PhVWxTrFQg4S-&YYPvndpcH!8jC^Y!=o2OsGF!NR%#jth|*3jAts~d=7I>V3-_e z6JYr?<)bMfvZk}4`&CqyI`5Ijiff&)jl z{RVC#VnT&ntj==*{r1D-at682U1AZRO4Z6j>UZSM9FoI?D(eonoVg#|(1Q!)%{X>S z=-TPTz{*e|72_1c%WW!@C2(-6%GT>E(|vEMYrJG3GIRVHY7Fx0q^Uy1k$$900sMuRTh1FB zA(?E3C8SLKPagt(wi#1b(2nCgaGrIzQUZ;WEO@5z-#lEsyC6$OtVSjVSz0o?%Qw<& zA<1$0xIVDdV$YRAV48K&qqwb}A{H4f6Od}$VS=Ke7C7?Jn+p!%X_@^HH2Bl*bQE(ruP9sGDUpy;BvzU8qi2(*v`Y9M;_)M`b{_kEDAU9Dfn;<}oOo z-tR>yrs{kJXN1O`%-xUsk+U<_t}(GG!$+r(^<&xSyWT9uh`qNaDdlhG@lsM^&9SUA zrW)Z3)DN{Qgo3tc>j`$m>M0LUnZZ2=d)p;mY)uMN%7I$$h^`->zL5-{>8-`r(8tCO>VI2o0LNQOY9aD$RyAKQ=CgP!GRjgB;c(W3f7hBgp;fb zIJCW9m30~Mxi|&JG>1IM*V1Ix5%lbgCu0v#dGt8MAGY;-Qe)5Zh}a5 ziotWxk8)5WbcgihnFSf~MjUHtDk5P%6Hi4wpY$x?cvc!dwh z<~nM}DE2{_FqOCAIO9JO@a2xvchAwkN^i)?N7$uUt9+uxXvA_xkg8-ihq|$hN;%m( zMq%mFTwLsWEniES2`);xTw0E8Yq!Snp%1Dn6po1HW4`V(!A4<6O=*LsJjZ`n0mVW_ z3+Xf(+-XJyy*%sqq_(=6S(4+3R%=w*plph3gj|}_`=)7#L+tn%h@l!G$`?*sm5-;W zX>o7pXO6@txHKmt&l}9?4CHlH6RESLt>jmI%ET(0df&S1@>C8t4cU{sVm1p(1`=b& z66K$cU;026*>eoln)V3KQBhawY)e8P359MN$TfPfn9E`8v7qX6KrhtZGkq>Fx@eW} zmx?k`;+oabC%z+?*$>Xn+5NS4;e0xhge2R4N?I;7Q!aiMr?E6OIa3Mvy^(H?6(p!HQ31 zL@Mrhwe*eL1%G4*|55qX?d+$fdHtkB92w7T&De%t{yPyl%t%!PAB=AY1pm`Wh3fWU zf_{}<^%k$A83X~(GXC$7<6*ekA<vZ~Ne+>m+gf zvQzD+xmox1gKbqn!ZApO2np1-B4Sk*0o#O1-SpX#fHj=wUO*561bfBLhul&OTti94 zaa5I^$S>F(=WCcw>q8jdnJqaxN6M7UqHyAzMG?qGfGn+lVhKdTMcJ|WzN(cz zi1~|8gHSJys}QdSa1Did2Hrtfc*la;Q(`sNb|UykBBGpCOwZ4FLx1)G`P_z{_Rs3#wk zimpaVxJ{6IWP3b+4N5lIVH9w^%fJ?wAaSJ?S)I!<>@DTCDpkC7#ALO@cqj;?IZvRLO+J|T3fKn7q(gbvx8k?f^XN8zkyS@pmKJVmEZy5>1dZFFF&Q3wmg z=u9;yBqtM;l@|=(Y1UW3NFeaXPfDe!hq+>r4~;6yYhmfHnG(H@>)KhXtGtuGoh1!P z@e6m-$vsE+4f>+JdcsL^u2$bP=nZRO7oHATR^arwY}rS30GmXyLT}sdsoV1lcGa@F z6g$n=r35qn8Hf|_IwH^As0H$(Si-|H;E&Vq?RADX1-l!jW6pPyYk-RUsAwdv4dqUj z%9iFvnF(N^hawHa8if03{onXyR~ou$lZe~xPTxYMA@1k@b8q9wu6ol-whV9#P-gzz zxhg}`L41zaRo7_;u;d~7-1{W%iT0$PWA!_>z2O|6Yth4yKN1+OoABeHjk|@GkRaqr zIqA7?V5ioGc%#&bjRbCgEVr$k0N8ZH4Z4shQZCP|S$Jag0zxC&KeO5L{U9gXkdO7U zTVFFsZ23`%9CerJ^=%GGqD4Eldn7BDK%BvD&;NGHc*8-wWBz86IXusq_iaf%QoeH3 z(2_g(xS4BvSr9;;sfX+e>@)0K;(AhK%c%ut;)Lx>Ih(ox88lybaU;+vS0$2P-fi`{$E0~kJdAsX+$ zI)Acc9Q^RYVJ24G7BnY31%9eq6m4L$D&HSAzmmi496@MTA?`fW(xm~jg?b3u!5Z5s z`>U?jprL-ax3je(odf*g%Kejjm~2C?dHBj{j7H%QXl?{DPm&&c**KDH+$+b)NDork zeT%0@#e{3pS04j237W$U(uAt|j!D@_2ZXF|s&1-AG4Nc&Jx>*h>HWnM8_UN2Sn0kr zB!^pEGnhqbZNC;XGuv{(l3-F-u3tgf9Y-JPvsb1vUt#g2QDz>+q(H7`BC^xjTf0Ap zh)AzA%%?SaaBC85_|Sq$yUq(rD6)I}Q{l5|wxXSL?YCkbEj}zst^5`9jK=p8Ge?8K z`y*B@E+no5bXZLlApTSKQ-?RG&7FqI#xGFN<|MJZvNpQbls30VhIUG2cO`mGh3uE4 z#+`Viw!}6c7r29E*~NcP#?K@td-=r2(3v2*{mODIKk6Q|6N#HHw?H%vAuk~%LIDX4 zF8+Ty$KH{+qzyUra4AiaMPxSeD>0h`4!q^c9oy|h%l*7#V@&nJg7f5V+ z*F{;;MRU(u9|I`Fm$(M^;6Jd?+WG6NDP1ov`QMN{U>J8|mBCx^07Zmk(H7Ia_nD`Y z^(j^UdA~b>osC%hK`n@6T(%w(3`tD#cx+nNG{ax7v5kXp;sT)!Jm;W=4~7pyJ7ve0 ze7%P{=Fk!F(`-U3iv-e6682KEU*C8gi{PlK4r~k8X)It#@_&Eki}605h1QWoMX#CZ ztG9|hN)J&W?`ZM9x<{w@V9KJlt^<$1=cdf^cf714!F`q4fFpD0>l$rIrGZ<@v_=WX z?#h(Uz<|__&j}hXk_gDHOg2KWVGm-ZrV_Sq6sNo-p=+iBd6(ZwLoC8R@R&?>?Lgn6~7x4PArbe;TE9oOQ0j6L975{gZ^UGd#uF$ZBm5~9fiN1L4x-KeYGx(kZA zp4XoCY#;T_rwDqxbgpg8_d!=$RQvy=Fz))x?0({-9?j^_rQE%WVb&d3x!QJil&cN`?@FM798@lR)*$E*Jiqf z%5V;{0BknmS1It~;>@m*E9Gmo8n+hu@VL=m<9a!b&J90Zzs;=(Gtm+Tm2I8U%%2DM zdu_TSbwEonWYltMY`$cf^b%rSn;7`e^-%aAA(>~Z=7Z&}Rp!Sr*Uw>4U=(wQC>UM) ziF+%H*}dhfP0r}R@1o$(FXR(pdS*)_%A3*N+!%>FWW?h8)Hi~1Pv6At|J%2c1+$+F z!F^p+$YrcTpuV;-gu~vpg%XVGHtf{(xcGpF)dKMXV#0;W;bKmqsrn-2A*PNSI0kdp z*JJf7$mFY(mMaulGeRpbjp&9=iMCYC+Zj4W{S~lqhd%F`0tNrv54P#T_a8AKpds6i zzO)^r?aH|Nt^lCQ1>27RD4OeK*D?HU-W2AbcoDc;iqM@42%9PQaF`NrK6S}TbLr0Y zg7@}i8O*r~0(;$xMFeAQ&yZNaN{(@=Qj%9ar7tq~!8 zZ8L!#MMLiK(iMMmBJ@cVu0kU4n=Y61KCK+g-%dEZ@R&86g_n-#h#s$5DpdTso`-jW zamz603VG8*i6N)q)n)me(d-ujhBMvo%O6qe4llnAo@5>J@Z}BJk&?nka>l(__kXH? zX;ZZ}L6~iTfwkeLGkVK{M~t^Os)%i5(CsC~VwD3~KHrC#fD5mEB1TU*CXC!~>9jXj znxZYuOXgYk$l}c~`=ltPcR?KPZHMAE;(wV&BxJgCGngQI8JO=2^$HYXX+3-tj)Wp! zO2zw@$@M1KZzcP~5zyCnRZCDhYo7T^ql9ah{`bWG`s%fWj=6}KqT}SK$e%rJku6xa zv>aLT(KgG-8r^%AwEVYbG~f37@BBCE!)Y{JNJp}NBUD}?T7abf51kXn;jT(rk0NGc4kri^gQ^z zTNioSt>r`1-s}6X| zXcoN*FAXcz$@%jjw{i$tI%R&#XSWYu!o})u)suM;fq(80=j2ten8|zS(UCPdoFW>N zVaMmlh+|rnEKV4oHpEX*P5+i$_eN&;C2d2A+iAC@1fW2pc&FIb-m+aT9B5mAOIpLy z0ZTwopaUUGib3q+Xqr57ErL=&>a(N{V#u^VQkH&Ns=88Mw3_{XbcB6WVVDb6 zg*aVxY$fVSY&)v+GFK&qFq@!o?@lP}>jZa@ln5qYG3jPpdJf|WFry49Y=r^R?lF!= z?=9qBbV|D0yTJSSQtH2%Z>5% z@)yMF+YkLfMN0-O+U=_}(zjRtzk2Hjhw+ZWUx=>!fH16dKQ?*mTlqe8yC0b;*~5J< zXwBwjpkU(61Z#y-*@ls$VU9!KH3oin<#`yK>ri;eONMHZnAkWH_U?-|3a|5iauY~GAD^=HrZMBF$S$(g6bLek213cz z;IEm?(t-kRY8V8j@Ta@-&W&n!#+1ysPf=(;}P#h_8_ha41ArG^J^ z@CL*8JbVmcht`Pt-c)QyH>|?jfjaA>pPtS#=;FeyHA&DZpLGioa%NNk>nF!`qjOdK zPw%^y+_^=A><{o&9_>uP+T(8|lrDXoihUjZkwJDMA)U@nZNe$xB6W~mo`kl2s>N~F z>Bbyb5%%bnkcn!i#f!acooq;3|7$$VBj2v&m_naPo&Dmn!Fevi0v*(mTZ-n{)JL)V zpD_f3SL#_K*mRKnsvcB{)@}ZB^vjaZeIAmDwQQbbsoVUYzdA}|0pS&+V1~_dbWb!; zy}D{U6heryWTN5G(W-|OQVxlg8s7xHhA2TWbz}Df~Murea z!u1;UEJRJ9b0vq6zD&<_ zVj--F1=Ir^l^0UOdvWOMHqXUq4OLs-EHmaNVEy}r>w>mpTs|diLh~E#6oiCmO%M+w z$^I@Z;%F5N+_MAy?<|A-PyRxf{VkM(1X^0q~4eo^>s?kpBByjKlx zF@&PrPLvJ@D!JZfGpo*gad8qC1(Q^h)!Kw4X>e%*im0JvOZK;hz69rsExW9Q(RA6& zL%jPJXWD%S&R$Sy$Lv*0zPfHqJ4qE}doPMiZ%;eY=g=6-8k`mD(mMkeo)E)G=jLIDIpYSI1aaSJAMiVj!_)!L zN)Re|jk*#Fmju}W!(s7&e2>$tpXpaaHaWqaqv@)3EsauW0UiDBuGP2jduNJ!-9n;0 zcJh@_1lnzHg&>za5KHn{Y+UxSTs(7^AD7Ey7I8%U?n^Hp`Y!QMtH4!Q+VZ?3Qel>f za@V!<{~fcHMKlwZO*FbzU+jmMCEe&2&_FC7QN=Y*82HE<1LuYN~w`ZeGJRXo{pKrL&jBN;~W%x zOkssO2FrIOZ}=0E?;ff>u90m#6pK{(EN}{6kE32X9%P}Q8V63|I&I(R$KqKI2+33v zS^3x6J8fBpoiYdZJazU(O@|RTi{#{6ew~Oude&J zjYbAJtsMLP)XvFF?X9~>U>Te05AXupcW!#sA^^9g?C-Tl3xUs}EC?`HG?acu49r?M zEgO|EUa32rJ}~@(l>=>tuP^YuodF_^NUQrO-5f5XU#;cFVn_!v%wo0jV>UAK+*M#5 zgsqHPx&!2?ju7M1>svwj@=c9bt-r z!#5hYJW!NqXhe1>;E!Ev#D|!H1bzlHxD*L$^TFCQe!@K_GVNrS?w?K)7PMQ+X?>$O zB%FC6wy4rH9?iEpqR_?537sbelhD8hno~I_1b@TS*-cub?y?qTPV3-@J`7J+kY*I}z;!qZG4a!nQjeH^1jM*tH!f^*BB^PXWPs&5&B? z^&EQaj=j=UQZyV*rdvBx626c?e;1Ag%d}rtCG1brheSgsm|;cs44?CvTH2CBgN>F& z$;Rf7$uPT18JDP=I26~ySx37$J&#v)aX`^m@W}H!)?VQ2-2}SEh1t#aCT}_WX!zzL zs7fB|@Iwb-hc%A)zB~WR2_Iz)-gkN+Vw@X3`g)@X-|Lw7VV^P6Vs2WbOy~71Cx<&f zF6bWecw&XHN7?)h4?;xXb+J8^V51Hd6jI!VZ})s2vC5>1d4XXyRlRV`cX&Z``y1Dg zhHF2nP*-JSH2hnpY{XtIwpP^Ks{;B0&F)8;IWOELv~koU{_NLIL>bPi`|7d*=u}L}9%SR6f*W z{HtAJ9z8jN7jgm;6+Zvuvy7apG)p5YV3!Is?C9n2F6fXNR#&wn<{=zF;9iQ>M_(^bA-Y^3ZHoPPCDS+X3!4 z=tHw!c^k|5uBargS(p!5B5!Nr##-#JdnjALsO-R8PsVSghRqD*a3F;X;m0#!EahyX zcL#S)ugI0NNyl5hD5E{z@ujrnGzsv~yJgospQ8evQNTWovtE<_-hBHI%Y7`45D z6T3Riy@X`jZpY`A2Mcjf#XD173BJ5V#26(c>hJ{5(8R+{wPEk(W<|#8Ef6%rlvOP+ z&GqiV_#A}p_q9k)@shkLgoDCOIJ>wC81xL zGn0Lmv4UU~@pSRvxrFvOh239vdu+=^hJJ}na!L!>L-w!Y-vnXd_j)o4=kojyD@K}z znw5x4A}`Nx4~P|d1g*$FnlHqq5OEKT=i@hRN)jj;<|2gz;RM}KZX%io3x5{6)DTBu zQ^>Icp(pzNRHJVtOl5;~pF$#}pGK-0^LnFIc80zyREQ2JI0b806xb5LPH4VE5jbLC zAmw%rQ?YnwK)gG7ob){EZ4Wuh7OV`d)=We9FJfhoK8Cv@GSzGHaoFzM$@lS3%~^4P z8e~y83cSb6AaJjliii%~ZH1X>6j@N>={l-%2%8PYSsb!|E zZ0QI)xi*dK?yH{ExI!z=W#g>k^SnOVTJxYv8;<`mK7`^eq;dT`|93+Oc#ARgUMPP9 zrp0(^Oba~p8q0|(huaCqm;0(v$l5|hw3VH|0`1cdxp!omOWEp|5f-3o^gLV4GV~P_ zj!Lfj{3s)$$FxVp2ywB`gSkP^vT#iM=kr`(5?LO)c&0HQJpPQzbZqC>X3dV4l9~F$ zd_ttpDcJdK{dbRJ%N}J6?dL5bFlq!qnZPN=at1;m3+9RkUSkTu)*Gzz0H*0*aAK~WO&kww zw3jKJS+PKYdma;ph{Pzue}-fAbQasV2=?yjqUa?#MZDot&#`9|KbxQA#~qdgVML0Q{&AK=EJ9a}a-1tJzqWoqqU=FP7YDTPh9uQ)wyXq!=h&y*lqJ@@LlJ&nnMKdKZsy zwgCat)c81z1LuA@EDz%l(`6LHsE1TdXRtO*Cgt>AgKJE2LmL(vXEIx>ng&s}aPeoI ztfdX(rYW>4Lylh@8!O^MwY6r}ut{t{$E=1%4yEAniRwgt(#PePf&s|jX?$cbo#=!p znqP@c?~7u@@#1q0pyI`r{yu$NAm&?3d|QK>up<3p+zYhk9JYQBK+0uX8=`Yx{2Oyeys&88Dl)aj)cEJxX7`)OJ5qjB zs(YfqTSd9-l3aN0NU5wl2R~?l96p<8)b@tmu-B2;c2-`liwbH3gCU+@6uuBX}FApL#n5H*f#q1$dSs$ynoMd6tNN+gU) zPiYQGfJ*C`?pdIH8`G8hDKk@*U3?%z(p0aNYg{TtoAv4ItlH#WnHJ;=NyRH}TS!X{ z-o#n0H!)a<#FU<3G;%#A<1UbqIEG;IknveEFU+A5{N2rlSqGC}sR|!VP8NEb`s3)V z*5m|&F#1ZkT;hYf-r?@n@bqDG6ZoXpS-IPO+D^lQl03Y)T_i?3<&AOr#z zLXgUmt_doK@4npMl)!%FHl4QKOW=ngS>h15BaAcdn|kYT-2$fai|U?lJ-R=^>3J8QgVuZ?=wLl&WrKfv@=_ELitkpnIe)%Xbk4@}X)$KaA zje-oO8$ze{AZ91oDyH<*b3ifc;T!6*qTQHSWn80aS<|)k*UO)aNXwNVl+}}WbNd*( zdNBHeSqPR7tX`eEd3+(d2;zJ9xA8Nc`)5TObx+~lLGb3ab&^~~)MO(c9q)A$=fStx zR5VvDxZ30@@mG)!NKlq8hKs81kalNuVZ|FU9p}0eKDCj@i8epR9@92Xm}pO)_ZKe9 zeo*nHHEeiW63cJ3PSoqZ+HV=!Do5QOH^Cj%v4tKxE^rvg+{@IdXh^M zI~F(P6_87WBaM(M{Hd~n{BXnIQZJL`H-IR$|!JGfi zbc~JZlz}8KJ~Lc7K2NCR$@nNLug3gV0z%Dk*+E(V28sTBVXH**&M&6iEVeTS=z|j7 zs`rPE#l=bR724nK6$OJw1wMoOXGdE?DX9LC$5M}O-{2zt!-c*QNygk^d!#ed)pF&+ zMYgoy*gKwI^9E#}0SxSC1MrDLQ(T3_s6dM*p+^#@%NpD)zIwBKBxbJQQO1u^L?voo zhoPcy#6mFxFEw32%Xe7co{8$D6>k{(TzJU~5h!BNsSTyS9?H zJm)XQ5E34e71dA&Wr=Wh`s=tAHk|wenk3@DN!Jlq^MPi;6RA${ip=iO*<+)sk( z#9A1+a-WJZFQu|CoO#Z<1^=+fH6NDzbQZLX+f_|67|tXPRXHzlrSWgs4d**D3s-k) zpwFiN_on2Su?N34>2+ETsDdt*u%EV^mlZVeQDu%Q#cix zF@nVsd(^f=ci-3C(@D?t$D}+l+2Zl$Sm#`#IRqjZ~mYvWNbiMTGmo} z#j8$xISb+1IuAOh4_bBoFT+q?W?|xLf;t`(c z3Dp|tMNIT!qU(X{z27Hk?ab>*ZZ}!hsaLu!2LJD#ZC{L+}^=}Ok7b*o04KE z;%)2S?)3K>1Sz&U9D;Z+5qPR1QX`h51{TZ&zc$3Co-R=(u2@hC5Lxk9B`m=5c7mcI z{-%W;QmpH4i{b7+Ax>O0S-2xsM&+#u1mTP7@8gw(n*;BIlaUsan`(mg2EX7%quBr~ zWI0SH(A73_Ep6L!?Bd-9rQPdM)+vojZ*Yl7SIQu@z@#^tfJc!FZl9;>7)xl3l>r6B z!IAKp=r{Ms@*EmNYnq@virAN{tPctBV~jphHw)EhJeewj?wJ}-vww^)FhXguq*o_{ zEe*9cN;D}cYD>q{$yct>b~ zENhVd_u0sicFm6s?nr(FYwhOC{rXEUnyp&1Ckguto_2FuIv)DmTh)RWx35Jl;|^Q- zdx$9?G5Ua_%0^CjRl#tE4gPj-r8Lr%$`?G?Vuko->rg&P2D1BHG8C<3-m%MRgoRa8 zy$1n-#H$@V!4GcBR?j87KDHJ@YhhYTAph7z%7Q)3$@hIDa*eaC>w}nixhj^(SdS&=VeoRmZNo%YA^(68=sy-1V|7S?QE+ zx<*nHmV2G{07ewF@}MDe18DWMHRMiuy(*l^ttQrG5wxVOvi;1dp2GYT2}(08HwG+iqZbGK*WerwAH;VR8X%25ypM338)$MT;aAw-z4%QT>r;=j@)|DXg(vc-?I@a`4ReJx4yd(u-TP9s4B3lZniltu(EHtiqg=5wsmJM+Uqa_m;Hb`uv9ILVbcY7xe;ihND1Z zK&_;POysunlv^w14J+J1#mrfCU>= zXTq>Pb|5+zMr4ErhSk>bpq1CJg8mJ(&DKzVmW-6mUfH|%L*cGOPwhTYSUpwmX8LyQ zXupE5=UrBX!msOH$G!j$F*_D+KWR^fEOqv3@9pv+dTqTuC%kxvrNs3d65q4Fg#?+p zNKo#SI?MA#Y475B>7}e+Dm@GY-C_TtyG|S4gGr&#<-h$6tL(~cy{s6S7De+$wI%57Xh3-lMz49R=?|&Ju18e-J~#YD@_^G;9g`gleKr= zMm&po5nqdFs5Isqmza7+*JUF8Uj!PO%l2V^h~wN^p9cz@9(WBAu`_VDoB_&q3mqf|F zP#-TN8SSuJ@<6WxpZf;SJv<{MQY2(XqsJBv1H+XRUMx3}X6a!+J{ZBMLhNars}GUl z8nkP}dYTdpPgOV-gX|HO#G6nVmlYoe?M;ot+W{;|;+iC>;%=8x$L=S!1~(`NNbv*8 zJO+$V{7g+7@)|o1#WZJ6G9En6=oUCLk5*B2@YQit;pk|o3kdgUh_5vCcb__<5VT6L z=>sN&^ZAUPzB^@|)A(YzZlmzxw5@>QPt0>?!^ zsY1q@jO`gvY+4vINDd?PW_nnqsci|u0Mu)6*s5_&8c}FscsPl@@`TWq4qHxNFLB;(ANoCe1HZhDCq6cD7SS zrSM$qDE~$1N4G+MH1SQtW+SR=62EeKq3hy9o)>=Q8 z9ASv+z~C(HW5HLsJJQd)iP{F11+h& z$>4<~u&5s1-l6*h-Om~e0tL>qP%CIswx$ZCTu;aMz-|^r7yHq zhY?DCDO-H(*Dzvq5JS}*#?KA~k1wE$yPsB;SN=ug>S41m=Epa8!+7~_-9q?=)Afwr zK3X?Y-StaIm}P`b8ovbYDqUM?^(L869(cl)jC+Yu#>(xTU8!Av&w zkD(62#UIa+4l(cF=}yS(*CQQFHL&*p#|X?5mW+Xm@C^IuoNd&c!)>NDjG7sSy&MA{ zL9Na%pRvr~5<(h}%(g09r%#YPIuDcNgl!_F6v++&hj3w0p6gG~$<#8vNfdXpRmkI{Z>mR<>ieS8`A*FWOPDxA~JayP;?9W%hgnC-CDjX3C; z58B>UW}G5WrlIj)Z-}nQ~jK!RVe>AXW}_vL1Wl75p&%+4n|BF#0d_|t|NCz!lA6~~Q{N>NSfz5z`8wT5 z&hoRadUFDV5N0F0(0i(7GPj?@vMye>eE z7d!hsb8)@4-ArlAryJI4?_>xL<}U+|;JeU}v(~z}U7$sNXnAbQO@a`^;@3Fb7IXFl zwCj3THM#VCG1fQqhAgzFJp)|Td_a9`N$6jI9#vvTcxga`Bt@trA#3_Zxr#ns{u`z_ z4V~9gKA7iAjcVg<`ztomIV6MAylHImo4>!>i3ExC79@Ht?gqeFI9|vNhe^CR`;3M& z`e;l?zgK6UWHx1td%Q1-1S3l8s zeFN9-Wi%9kkGfF$c_H9+p_EysZ7GQcB}r%X75eno^}pdJeljJ+O?z2*>&`Z%9q6ZN z^m@Vb)c=fS6x*|mD&g^=c=2S>x+`EIo}0stz%d7o96jWjs{1Q6xp8{% zppO=uZdS51-;kWNZ9rhuze*MMqA$I$-SBZ8cH0{4Lyw>nZB$(_9~mIG$0F$<+Z(%;#d z^_P!L?XOohEv!ZBVaxKppa?K^Ul}+ zzR0(_w$)=2ra(ifpG2ESZj{CazR<)Xc0+UAci3KDb$wx)k5j;&^IWYY zV6-neOggaTDj(X5b24b6E~5^_0OnL=|75TfN773Vtv))2fjYULidz}YDoaE57)UW~ zDb5VE8)G{BLvqh5&dwM!m}4j#yR4Fe)M!ZDqETU@5y<1+BHptihCz;=qNGC&i;-k$ z@{!bR886QnrqInK#HR#3dQ;fsN%~gnDyhWm(TvrbYIF%OsXA+9>9JP&nMx{A|Jl0z#=TnLUv-9n|M=VjWP1??zh;E$Am@z+`{4^pK<&|T3rlrlmd$56lVr4T_m-4e#5n`vHu2N7? zJFrOcT{{XLpiVvQ`b=bdu)tNuOphDG9eT+B!b?_9k9G?$wXxB$SL%|tUmK1 z%L(G1>|Bbfd884q4IqM2->t;8S6bF@;jSOMh{&skD^6B;Wd1 zFj;PWNJq-3Kt;RuN!svtk}Atru$;#40UCKK8CgC2w~PQmK)%16LC-qNz$K{=X$}nn6eL7RwF&6@=|+x!4rv$Cj6G__@nSPx9q>e1aWUb59*3|rZ{nw zXn=uNY@pSOT;pPVBm((&7{c61m~;9DTu;F~I7nFM0iswF!yLVX%>>nbIWKBJqb`l5 z$eiO?&gfb?r6D}W=Bu+HyN2TMCXrLHZ)S*+5ptzCo+(h-zEhQlKf?y55fvfz(WShw z)vs$V&S_z%!;ciplezP8nvO8(F>`~X%ew}JPR{hX zZq+7AY|rc5dme&5qmtSE)K`}pzaX^CvNkmH)*k1zx3$#5((?J-+4vOn8#|z zD0Q|KQhl_vFji5mYgs|9C=gU(5bqdCq&RxuxgS(M^U{RRPr_EKf+$KvE!V((G}jLi zO5#Pp^0A%pisBXd@I)$z+;T|>LS`>1-mWgx`6ecX0QA4_%K;MK=fSp{-uFo3ejnR zrBtZJPrgZloNKMt?cjmDp{ZL_Qq*TnY#@Fk_yejRY0uaj3^-iMfn%EHxcbpiQ>l99 zyhfUvzWV>SmPZ!3o_M+Q`sX1QaRY58`h`vddHR&nW!MeU*_NSaxsU@rO;hlwKGLa~EFCG9Fga!=$}s!5bgpSz_(6D^ruFLOmX5#) zyU?5p4yM7`+>MdI6Ij+9QJUwa1d3wC$~GjCMLO$DwPxInKr`m(gN!LT{()I2DkeA7 z-Awqr)$dIhp{PMmzPXB+bx`x3=-)S`B(0emTpBM%R8sxn<&)t`Nelx6gMwJdeLh}L z!HZMxOZG3w28^89^Xa;$@yD{sYA7VUX_a;R!rnIZi$d&b{Drhp*_#uqbS17QhNY?E zBP^v*_t5Stm1s!H8Fd)~Kjvw$JD?s7-ookzPbbxP(-};0x4K*JD>l{0o2j679LpVf zljKoNX426lkL~_4Ef!z2$(Z*#^G`!0r>V{_PdKd5WxdJ6>RWz;*#Vnu9U2Bi16`r= zu0Qwh39Ncpf<&&p)kIfp@PCqFDPMLv|&A?RJ?e$F3HV)1*Hmy(UN0X%qL2ht`4wFt762B&@@ zh8uFzim}&gu4ZuI$x(gCAKD+z>=Oo^_JH7!bh3p&(erN76RP{L4t)`oG$~bD2|N4! z!l<6zoTgmU5{jX!s*ziqAk!Byv0u5-w9q`L&tvq~OidCGUT)@0UH-m+31$yI6Pb^u zFXu=@o1a4MvS}5%0S&9c9t&=zgk7$OUJ-bwg<7YMma$42X1?XF?q}~| zaaBl}%-e9}oCc+?6N8%-Y!a@E3LW9F7^s?TsV;7J+m_ zeZQ^g2O;oo^#Wzky6eL#zIsntdx^S6n(67|{6pwa7%Dn3E8lC40%IBf%X=`milU2q z{r1qh80*+fE>wouIqh20)-}Wx=1)2+DA0pr3cxkxob^biLadTN#L4j*d?ug;m{}GqR6f$jlfptrg8$ou0ONwyiDEFpf`c;5Pp2vp@{l50yfe^h|Qul8WNrA_rCQCFb zfWi(#OSdXA1784#4zbzH9vTjIay0$}CGoS4{?hDx5WQ@IM84Hi7IT{FQ~T7rLo2N~2C;XVE zEmd*M{&wo4^Y&&r65W~hh3A>%myA6*`RZS2Sjc>69NZsvzZtEmx?#oT=1FDvW+~3Z zLc~?HB5V+-z4P&w*PA&-TWLh-Oo>iINEHNaF3bw?oM@@ozuc0G$|Jsp$A1VHzvv%# zAw#RykAYOak)}sAp#UvyHkNAvGIgH3qfgESq>L81OB=)pI_<9F3=>8xs>Qw{F90i6 z>GjZFuKf0sP$JYk6_dgP%tU~Jo59VrPr*I|k@Jvk!MN4X!<7fZ+|e#sQy00f*Y#5I zAPArSR?~1JR({8m$y4F6w5!hVQO8}QMmCDh7os5dib%SgXl1Tmc_X)Xdr`uT3lTyZ z(H(`@m~u|Dm8sGYBtn{dJmw;KgXiQ#hE7b0Rl2&e06sfzj%U zF67ktqcu2+%znuDs|Vq52ldQo;x<5GDjr|-A?flh#!pTzzk#@_jxz_%@Z(pqm?c$k z4GK71$2lQssWO{-ZDGt@)0MAA7=F>4xg$ar?s*y&+^=pF5orf3d!`Aa3{wAnBvMS8 zH8$^Iz${bcM4aTwIwI6fa&Mkx(le;8jYl!&zRR}}OQ zpH;S=>|ZHJn8fN}e-xynn{3DmryS z*Q7z_YZ88@h4mE7^M(ua0tIi~{`5!N^9UE8T}F-)o3$#qmv5e{751%L#Ui|;IDzpN zTv_*Mt!&|XX`DZfD2CrQvHk6bhPR?B8sS-g!?7GgMUvalypCj{Jh^aF%WCxT7b3yD zU4tUrwUfan_>grN&$zVzY`Smy(q!J#v^!^J-Y_|;j{cDZrkx|s>FF&9YsG*3vz3s-*3!uFjT=q zJRHB|QK0zeReC~FYQPM_c2G%nWZsigs7WkNzjKreCAD8 zolY8kw1{D}X{kH?l-`L*ZPH3>k7!Y}_X6Hq)a-%AxUEmr?CxVe%%N&ud`8T5 zhteTgP^siMqKy-1(i>$cJW`NtVC1bqwnRscvLf&U$sKaEmVysjF45MjrI;FqjqBm-_s2@gs?Ex^fuWWJaLd~jcb3J8}0z6 zP}0PdA^uhFo3c=JQ?69SL;K! z75x&1*)rQ9PB2ZC8ClBkW^R=oj>zs8=R~vMJ~W9^U2RVtiRqKJMCqxdpOeF8g=T#t zSP0VfmYJDL-GtPTEs4udH28O4i)%l&vWXq%!S?)2B|R1q!@!SZP1aH@4-;vQR0uViNxDX^HIGK< zLhF3VG^=p&q?NTA)md2dQym$1u z%-AeZE-MUwr0MJ7FLPoSb{g=_Ip53QtG#fQ*_+xbS_1#|;wY6;ZQ}6xk zo!oeNsj)+@u}3QkOgLW?6FOttG_iFhe`1o7NHSzy?E|KM0mnmL?8(kVXo7zsC?z~I zOaGk>%J{C)A9E{8%JynDXOZ4G)oY2rMv#8%j6Hr@!mhe&HS2hS9Y#J2dwElqqR=QR z7VmPLZ3)g2$DSiWLr86|9y@|0ZsQ|P%%hjB5A~RC3uPUmPV$1-n5?rPrKHBy9#E9| z81oY!JUeJ>$pA*f3nNh;CDAPR)@5mAYn8<^$Nri%Vw5nBu)t@b|v4CU(gP*+fqZz4D(qoP5N|GGs1ec zNq~pnXrL<%aC~4g$FsOFnM{fJ{u-ADA|WdyNV%kqi?dREtw4(h~!wX(=d&O z(w8*eyxbM>cjme%x^${}uWjroU1)4x&u)kXE|h`!tk0ttoJD|cejm!X6pez$^9Q~M5)n?>x++V$=}pQ z9;b(y6no*$&tHv`?3`iSkX;6B_`B)1YMk!p752t$U;M&kRGDpuYBx?-+UbrlYK;@P&>Ei~h){4E9Rkzk`$IQb1Io<4H8MP2uNS>xnDZ+4*J8JS;s<|CWqFpJIrONnnRxLsI z>fyvc#9PMn(sPf#NwIQ4N?k3ToE0zOV?n!Db8$Pw93Gu<@n5P7k({-gywFtws?^0u zHrTrExS+7oBLd^{QiA#SqHqh5=_}eihs`!YkN)FNtj-{YyfKVxtIbptYI#T*u z$S<~ag@SPD!TPLo(iJ}&iZwykxnmj%(6h4y_tz8$<#E!RNMtvlIb;RFnWCfNCAUIz+5jHzJ9Y4S59Fj?- zONMkaDAm4x6D? zpTt;jdvkFmy|6>hc(EE5->(2Egv0Ey$aJP4`0*s22)|QATB6W`-^#IRHfd7Qn8%@@ z^xSF(azVM&VUii_GioZajzU%u6TcvxQ165dnzKJ0Ts7QaSX7slrXqfJ=Y&^pk_q{Y z+Xa_{M^#bJ3Goe!Gv4`(*u={&eA9zvC%iGn<>1v@-eyn!sP6SPxLZ#qO2K6g?JqwX zYs`w#?%jUCm$~;^z|_}dVEE-wjBK>O^@IrtfFK^PTxUbK)3znl@&rp;WEwG7DJIc% zSL@=B@hVf?Qp`Z}QmpTuBrU&+^!x?+){;36zqI%96)VS?tXclXJ3S=G;G!vY8cEjr z?5Mm}v!$F2?EE)Zt;OdJ-Gz8h_DG46<+#ouKO)+Sne8+*`e(_*4Ut!+g=6R5Rw(qP-7EnH zl8t!bmA}R(l(Qig0{Rv<=KS?{-EPhuS3-3f%;r}BdFWh?p5XOef>3tmQPnEFgH2*yQZ^gd^J>dsxl`#9-mdA$678wc)qI9(-8g= z%PLevUawa~=N~Pn-VQQx@oSJcn4sIpB$O0~qhLn-kFI8M5yzzcwei%ML38$)z*${a zVkXQ-yTS-rpslxn79KO%aj%ZM(nV8!Z{V7XA}a=`J#p&052fB!dA4(EbvmK6Hf;PP zZVW~Ie%iH)_CCPr#3^J*f|HaGHzd7@E}{X`3t zV~tC`M+50id&}L@V@PE-$Nt-AQQmSk{CK$_UF6Zygp}4kQEtKo-HJ4H#E!tQQv9+s zA%sDlU*VO4_M_s9QS4GABsTnhOW zcEJ&x}5W^e=xdY9Yre3yrtaU^?ps(OJ!~*o`l~&bGzJ^JiulehkY+Fc4kQXG?=CZSxzm6awLF8{P zy)1)HFv!g1bs;l)V(u=_YK&SL8rytTO_7expeq!vLT9LP11`}rPlR7x#(C}#%ps3? zmX)=W2{mY!)a_79R&uCIkv!?Z|A-6kc;aEy$u)%`)7 z85#*XUrm+l&Y0fQ_0gvMhmyv0*eaf-xRhJ2Mh9Q#kbZnS!rf zb;oPRAnnEceUxe+mYu_ZPeydeZ|t?}k=GKSYT3+!uKW|yT4|2x(LjsIa-)*sV5Y@+ zJ_3oe0u)0h?L`Se&JR?YSer+FoHx_jwCBWy51p>D-A0ATIedcm(+Rm*MU8@7*##y~ zsFKp>?bIk4!Xo6mQBYBlRvS_2MMGiMi&%pB`uI^w4{T-vNdFRl)R~F&7duRMrdN|m zdSLf18XLPX@IXSVR%^m@_^OKlM!2rn(|r zr^ZGGj$WdL#x{N;ilU= zE2%nHXzqMKjFR^+8Y&}b(VQ-MzY-++{TB)4eLS4yFZ7y>@!g~fR0Vxpb1uD!b|x38 zETiJ73=NdfTG%e34I-#XAgOe@qsn^ZZ-Xu2@UCXxcTdPh%W!Yt!Jm4eXIX})cEhC_ zQDq(oZOZ9AH0g-K+pbpk_z0V3pW&#jq*~W_I$5b zPof4xnfZAmd_7D6Qw@x)RtZDyu{LH+_?7KEtTe+(BkQ)`fSZyXboS${#9XYp!Y3O&?a@&jz&Gz!kt1xhP$` z@BH_!z*U7-9Wz#3*y@wsciHh%!9UyG)8CX<2#r_<{5eZCrsSm)D0nsySV@`Ta(~-x zkSR;v-8FoK_bC>9_3q*^V1Wmq6$of85NHW{`{vmM$cx1h?-yiusJW z*-9?IT~)`KLCeOi9lFU}^-)gr$&wxyZPD;g-gU(Qrw8?*1<^TK64PRdv` zxh^r{X?Uu}=K$4P;%h$h05y25ZDgx&P0!O>?*t}((Chq9OG^fM{QWGxq ztjD{jTtdM)w#_2W99Z7cs4Q&=VuZ-)Ouk9K7gON~aI=XGsXh2G%wS(HOCDikBexFj z6^^VP$+3qQM)%2WR%R^WuG47_N+F!}7a zZzggWsfX5WHEEQYZ2-7jBxni)FP(m!JGiAR6;v0OTLWd0$Xd7YuukIN+>;MT{_xAD-b?h;Fp9Q84Q)Y z*GPJXxQmVE1U(XEYx#z3Aj%}7&KB8DGz+xUGYUn&DihFeaM57k#=O~jepg>>+4L}! zde1r~KOaab_MFr}suEiR_8wF8>oo^oZLw353O6m@@)R%B3Bv&oSQyf?%(Tux0~ zqD1N*|OGt zw6caAUr+)g5i-|^T_@#^3HVzdn5F|58EgUV4ZV@Giw@)8#N(UeeI&d)HBW`EslrM> z*20zcHUKr*fhftwVZx?6f4{_cSY>l*rkbkjF{HWy-*H99@ zHXEQHgnUyRC(3;?wz3w|guM&mFf1k}(AQFSv8jGcZ(Gan@8r}p7T87&uJD^?H)?8# zzl?D!#!!iyTWvLeiJDfjC~XMvx*^E9&gFO9Y{8x#3dfzrr>RyBUYFFz-rMX3^4JPP zpW$$xD?vDQFiABT2xZzP#6F{QMBV4byx%gMmZ&gFw`wYtx<@u6GN81?@4pb({tHFlCmVj1^Fb`?msIWv1Jpcvms5ZH&QVk+=~RT{(Kl;j~;-2 z2n@B%?VZl_i$YJ(7kO*1B|x_5vT3+zECRNOoPwK|EM3~jrbv7`MV4py!t6D6*|Q_u zQin{YG>O!5dvWy#gnIXVo$>YWfpajZko}VKzjqnoP zbOD%oaUc_iz1M&i5mV^u z{o8Q<=dtz4+_Nnd{ZFJ`%HDr1Q`UP`I+u+i(#l&PxQo54sjO4awXxXOn%Ro~<;rl@ zP*Ap{wU8emj*WNMC>^>}yiYux$duyY;vBIzWi$s43<5X*V;;~4FVJSpM> zF!$+D@~Dv2?V6qZULm<}Idr8TDp*a!oC3&x!c%LmR*<}>-h96xWb@5_ADx~~qoy*( z44GKF-S61IaFq(;MY#gpkdVQA)-aVRSQ`iy>kea!&Rsa-3|AtBY|n2URETA!&|NMD znES^hm1$(SRHkE^tQtxu^n5Goa_5o1GhE$-Ax^DzTjn+2T-KjRjTj4P4l^zuv^YVS z{i-m*N>fo1f_p*&ZGAr9;Z1Y_d-rYb4WwLeGD+-e$lyI5-CEXd@9U?yNq7tipRed& zi$BV+Ex$;GIrJVvaTv-N&79LiGZ8fzPisik(5L&LZW#|>2lwWc)avrd(k)Y*#*96R zdm;XS|HMvlZ)Q$NEO=1ctDC0c$4px2gd8#-?Ov>29We~#JK|zNberYpy?!^Eza*XZ zD>J7q8v2@UId(Vc&uVs#!>H|AJK$np8n?ill28-FqWt-DS zHD*`?DN!biHCZzX~$gm$>X&$ zJ}&fCR}x+@)t>l$@5N+dp*M#dgIm(44n8u)sZvmCAdTN_L_Kk?f<%mer>|$hdak&# zTbIITBs;tJ-ejh727fd#v?g4=u7NAJ3znLB3)e-RkZt`(tivZE z11$HRU-J!WY%sgf^P8MDQDIrW&i0bYv%^hLWpR{hd{tI{qm;fEqNo^ATlG~!`+dgN zLzw(wfFuM3PM9OMDzrn;e7=Hn_?O~P;Q=a$Y%Kj6=|~INdDaV0!g;K2_-&0KB82tL z)voaK^T`~{tABwD-6|ZQNgU7Is$Wcs6j-BISx+WGW}@kLO;t>aUHWtQ>*B3#7F?MpOjDzWR+WxLU_t79q3ZJk{JTCwZG z-Xe-5aN@3N7yP=IFfc|T0>P}siD`IV->Ib+lup65>;tqp+Z$00)D_|FPSD_x)#~Gk z7QQ;j@HUfZLRixb>+P4Uoth65`y&Byp2=7{TMh#n%Pa-4#)|x+A?dKxjwN<#`=N$P zHvH}&Rv0R}MFnnlwGS;QysdDru3j<+G;{7Ix|&#mTuQ5YxsFSC*(LL0xwtMZ_fV@2qnFP9A){8!)F`>#r|!GpNZ8-=iLKIu z_nTm`R(HJ8*45}llYhv_13`dT`<7F?e1kTlU=iDBiwHF_Eg#}Tr+ckuwVre%F|C$<5?b7F)5nSjB7{hD;vt<}yHC7$O$ke4*%G&ecaB%o{1F z;hac)V~g>4NL07Ym)x%WFsh!A0(RUmYF=xjdduO~oN}ZoNW!fMR_~03E9#f=`c0_I zOn`LA3*r-CakbS%ni&$!qvIT^P$W6kp(k>i`(D(Sa+HL3+VpSck}t`!t@FNjg~8)% zt4u}&HZDw%?w66^+myB>mYxjUPnq5zzxYofNaX#+v)0yj`d#*&-gw^<1AEK$F^36R z2kJOaxjY^CPC`dc*&oV~xo}gjZjN49!jdksDHK#spY+qv{h0L|IVI~{y;~+%Z%XkO z=4B-jpY$ZmaU~LVa+{>Vge#Wss@XTRRx_gnX!8vX5)KFKAH%_ySTtziGct;KN_YMW+}p+D{)uv)T#dT zi>{#JjULqGlLLpsEQ0@pQ-`Y8=uk2BX%(wSOy*UBL9 zpX1Or_VY}XR`%Z)SQ#kqh6mF84xQ%102uTL!hIwxyb#B4fUCY-k*RpJL-roiB!hiN zdNSZSqNdr(;k_vbP}+>jHP5lN^j*xSJw1prVVb#U5Ff%f}xDnsnqe4N`h z<%r**F&d*lu*vkJDb;_^&i0P2Tm$C&?91?sK)%L))n3}qGu9+;zA^fqc6Es^DVCC& z=Aq|ZCCenPb$jVj$v;dnMb1;o2X8FANrFy0kBQug$Nws(*1B8z;RMq;szY7qGC;dG zpTRr@n0*U-!PKAfu*Uvcv?9IYgjbD~nRISFL-@4N)zoN|mi?8`o7q8x0N1_?e}D9| zeOj{c(f8s{u@lSUfk&Dgd1}`{}lTJOrEX|`mC(_xv||7LKS>Bw|q7UZ!1Zu1IoVW z9XkL*J@Cbm070A?>VC8_fSsgv1#m-=$m=DM5C#0oU`wKtEWT%#esXCnB=H z4HfjuhqGVl3;vp{EWI6|5*G-mp^i%40UF(3DPm2eVJ;uN^Bd->qEBCF$XXi&LK`r@ ztI8k`59$vEuU}CB$E1J8XH=~ZIr0+NcjAMaH>PmU+We!;A7 zvhXP8nDpK1!|C`QXsmGP+c5<4(TPL!ah9%#&=1?j7qYnAAZ;OjGm$E=%V}r*A8uHUfjc+BH?3zurj84*F3{LbySL|Vc$a|p4 zQImE*46TeR2dZejF^i%2Vma4?@r#0&qiOHbup?L@(cRpWDIHk&R%NK=b#Jl-&GKk5 zV_IStceS79q%;mO5-|#i(8pU`Alx4pjOMme7i@n}W*>dLbMfXSqrZpH&+@?ULWkFe z*-AA7?(Yz%aa7y&3hZ*g`t@Qmu;yy=0RVt{*1Ex+^i8+lM9^aBNPVlRt(MbXZ;;mF zr@dCCn4LL4p~^Ve`x~MH(uhKfWY@`eYc+n%!0@VLcCd+^d-dUFbcm@2V{5&ALQd8THZh9+P(%#}Pz`G9eLsc6xvfRI4`wzd_xOIJ^wE~thoW_A4i!n1S0YSx=``%_O7n!Lt+*>rw*IAELQR)6_PU}0c zmADsr6!P^2jBdUuw)i!&-(-nb+_0D!bIM}z@ituVj45p?v%wnu>tJgrM0lF@FTo0x zp%+`KIS+pvoWR+iU?`hly|}dcF!jytCvR^@H7U83<-AdQ{TG;_EjF|>T@C}D;*fmj z_1O0ff(OU5TSJ~1Btniw>m;()L=i5oFn0XAC1-}I+JD9UT-#dcban=^l5mZ40=sKusWf3v_!T87r0R}bAuc`wE)D_LQ&gh|*`jbC;!cOdah z8?oKT1#L_0G;;}RI#ulxM2yOllf(t+YvB0?Xz^YX(Ni3J9|gVE7W^W_y~a3+mwLr7cM{TOk1GkP4wPOce(wvOnYo z<++N9o382-V$SQteTt8)%dk&BU=p9_*<4chNWQvY%`vveScRn?akb4JxLu0C+3YZw zVDW;=N_xv#uvX^Z{pedql7hCXUHWjAaZsNg$wcp1X~T^6C_&CZjrPC)IH!!VPB>ZV zbrki4X~BP1pU>|)-X@V}S{ShYL7JrNLD@AgkNQx4SzUi`L5zN=*ZUfnb|>4!RiR`M zn@wfWdBwU$z*)$w(Tk%G7Rs2`oPbdWQ>2Q{pPjQH}x(=Bn50(;YMrY1|a=td;<^jKnTJs^7RAp5qv0a#1 zqbHPjXjNBOnse`(`rc37gX#`u6AEHZi?M+r--uSNt zh?cF|M|X7?Rj527BRJLKaHN>jfg@~)MqNuW2WsjUKY=5!gs2z@?QR@lc1(r*a@2oF zqLMsHTZNhr5#y>x$_x$mUID%eFpbE7BMn3x7%H97O2o;eZ8CANJ4?h0uzI5VHG}*U zM$8cBvS?-_U2xocKvL2= z%rYzzkbtyi^EdGm+L1627T zDsCuusd-h00jXqR-3ONz#p&&dd_HJ>2%rJ_&QE!e>r8@=5Zdkv+BZ;kxa_k8b|2`soVC^M+ z<3}?NWG^Edu=bdpMwlja_L6a*tV`ibgqcO7;92IS`K3|#7u42>njuB2fK8f%iNRS-4O)VNx{ z)fiBLG}@RnpR*<>&;orUF*Qk=U6|!Y4uWy{Gx&-}?P_}-xOEmgKSeZlhOiI0dw`RejYBj=uOq6-N8&kNZvYxLQswdJB9VSzSL zFE1lYCMiC&JB-qgQ@45wEH4HZLxiwhjG@=Om~&Fh+REaTvWMJT)lG#{?zPX)e9W0X z=H{+iJi0BYw=e<%E8|)5_IH?LP9Nk@ZiKz7n)-y@_*?xG)iJO2D38p}`RIpVFIdGw zWX4izl;W&Ue|@uLk8X^CF|ob_ODNfIvBX_5)*^{2DLPL1Zr)tzt7(q5Pva@jBN(z{ zfT0fLt_`WTS5(o)N1HbMlFYi#xJtTiK=f}*^c;gXb#-QnLvFkG&Ji~#Ka}CP%n=+{+QBY zwN~PVU-zT`wv!8ZcF{tuY>gTkkMUVeo@0J!NUO-#jOFsUn4aFPjW1Mu)-XN=L&HCI z^`1`aw{QTp!I>Y_k4pls-o|{(jaLU=PhCXk#?aWjkkji}$W;wV1eY5>kE=%h_<+VNA_tD2q)a_+h4UZON)3Y> z-X2(nqr^F+f5g=@gS>-@#2Ux)0`)EXUGBsZd<>^&PR4Uw~#tJltig}JYA=62@? zgknmSATet1#upDP?}ss+>kZNdzL$7}w76f}LKi%LIac9i(wKwPU}E(5O*Eq_46~co z74x;_i$i&~KMkq-_)BOC5O`b4L6{y=AKhEHq6at}!i`5dl>9+ZD}%7d5rxU&FdNSj zqVji^JjrRb=fLc5;+>!3RhNH2MsedF<~+xQj;k+fKRCeB3S(oTS3X~`g;rvnJxr8$ z;Z$>vN$4h&i6HL36NHWTjLt{ku0;;?1t5csj0w8YKnG0tLC=b zGA#cxeO=wq+kb?uE!~(8aYLnxmBl1V+g6C}#5{L#>ih|V3nXpx3rD*3W*NE91kj~u zxVH?{n`k1o~QkSTaZy$_maKptu4 zgL!H1Q$g`;e$-GJM%5m<>m>Vlts9gl6EvRPoro-^+y*eHP521YJ*+=62ruN2;jb;8 zGQyyac3+{in)Qda!5B}!R-ul+11sJMYD2M8Cj$v&+fF!=_6k|&QtC!!%03?KRj2vK z{v9DgxI1mV$pTnI+Q0l6X&l?~qf#4lnbBHYma=AYmh5os9!Q*HmQH(KgRUkXdF}r|Fcm%-b zFWzF$C17s80#OyMwR9(U%CO+<1mBEdPwWpY1*&tk+pk<}pfTJVUUB}Ao`=QOSjk-c zYofz8;X<5~@4^C%vxOTg;r(0RTgJpLP1?WH)R>z7BwLED?EcL+qIA_Ipa}8E6sY!Y zDf?Pi_Izaxm78FqeZ*}Wt9#^wzD{~USd0K9LFTa7g%e6j3J{WOtrBaU;OI{ZwCH>3roF4l6bq#aVPU6 zfp?!-^Y;cQwH@dkY+#-n+)i!uPIq9ZtwHvnc?Tm|WC@k)L z+4DDojSkYQrZx77*{pCi9NFLraGRmqCj1slb7ol1WNvV0)ko9dGre9hnucj}e+x!F zW}1C9XB($pY8ff01Dqx^PiU1?bWjp}!3aonJ1DYFVT34{4iog+V$ND9qJ|G0fgy;OuN>iz zWfZA4m9$I!!##Q;tYL*tBV$BDWU%_0@NVQ85AujFrbcKK!MOt`T1n>tV0mHLYzIe8 zYVmNR^HTHKH1mbv8t>V@&2OM!pL$;`UtD!&9d*6%jOP8^7}ORuN?3f(<#0$o=z7M` z&kEI>xhF;a6oelvYZ0sK{;!`JBOLIhr+)|Z^!PWC_YlEsV;?x~_WgZ%>Ma#2+bti5 zRaRAOl9sb5=g?HNs;D*B^JW{jxVf1*QkMn#x-Nc<=PZeWAR(=s7k$IN+q#-(_YwX_ znBeN>97ccZF;^0AK(9|@i-T|Z*~|*nPvQroC?^Y|3FpC3uv19UC7gzKvNo@0`aiSy zl=+*`#e=^T#^s#^D^aknxvj*7?BfDPTZ&GJ7>u;|boz66}V==k+hgX00MPQiGrn|f< zMP)>}!cEF9{y120?UNkuPlJ*LaVN5>PM+L*5~puPZD~WWrD{bN2KE*6u)uIX`B|d& zCBhg|Iv@QfdjI+OhNSuQt_p@5;WMH~(DSK?Z=Z++;MsznMW_}Qzdu3}K!F7|hYF+m zc3;$2>V|FUyid(-N44c9ENQ+0@-0M_wK!M<;mXqW-V_7Vr3*G-?t*vvr{`e-YZ0Nq zHOH!0+Tv27EI!)3?0ixUNDrkaElG^*Ufv4aZ`ih~4jJt8cyHb!U22uOVI%NaDz|T4 zHq~9Z`t!?|npT73Tux|5)Nx=7+*^tg?kY)wLHk2WlgP)h zi7Lx2B0vEz)f@O=BZ%C9m=~Ky9w#GcSCVuJ$sfBT&v~;I=zc=Q59DAcv7oSTim*o~ zy_}y3{s%nnGMo#w^C*V0nDsh^?5Vi!bV7ShTGOaIZ66Xz_N>g^1TV|-e&WE`=QC>`b#c@f)&9% zDdLxN*N+g0BDX{r!O06To8HatAt)2~r*)VY%aJpoI3Cvfc2GXu*oA+;1qF@v)4D%L zaz$`Do-6JFj4JeDetC7+Omd@7TvOAp9AwvA(2XDI*3E*FQg{#J$G592X1)d@MEbsW zbu$4CE-+UE?Y1to3ytednN)3V%@pJL4Yx9C`Xc2*tj*WmPT|B8C2v5Vc%k%@1JPbC z+v2#E5qmt1`I&cCRG+H9zrs*$HYQZdG}RJGpn0amPsUNpn5~C^t#9JyT^hM4gvYP% zh^J_>@yDwL!Dflr zAzFPdIv+MlaQvd8pnf{ODv8iqZAW8+Rn16=Ge^?)`RzkKFIi6MTfVza9;R#VHpMzv z_;u;t&`+^>CWF<|x(r|^tu__IN2O-(m*1GbujRuV`PBji&S-~uxwyFgG_}H#VcAM0 zNc*53<><90MpVXrt=Dy`fa6?TnmkUg0`50E_PJ1-O!;)Z#uSQ&+ zC}yQF!MM8t?>xS2{9$vKaJ7RZypwQYM2BQ>%s>_%p-v)oek17NShP2I%u4 zpzC(UU#*KESO$Cz?jxrixF@1!P{w#H;xDHmsR@Ya<@uqQ^tJja%NLprV(3e0re2J! zXLWLmE*f87sc>w+8#UnAF4~JoU{kMTuPO4jo3jIm0(7iLY7Afb;6Upa% z>#-|uRtDnXThWAM6T*)yUd_+5`JneS44Ru1Q-1R4=i*^?URMOfqqeP=&uQu47((^f zsxGXPG0P9z(k>o$7p9+#ec_{m|{i?jTSk>?;-h{9|r{Zlyu9FVv6`(%(|1l@XXM1@Z7$Q_>@49xJ z!N}hs&tp1Mi`F|AT_RF-8ffu;(1@#z$P$L+XP4oWTz@@lH#T3c0)uaI!AS*(mKnjy zT_PnGRTbZCK-*uL(-PTaQ=dogKZ?#`%~2qXqOXL@5{JTFcDTC*p1$eje3R};p#FQ$ z$pH1+7?I|yOVaFwUch#0jowFg>URQu5jiT(|BuMs8OcJbCp=FffyW&;oYWN2`KWm} zt@qO1=i7`8#ThWGnGj>DMX{oD?&zbl&D8txpPQ?oHA<_Yk=+$hZ~_XPrv0haZ*2Vt zh?X-71|(9zKex`;b#@82qI1Vv-Q_1?2~W+(jHr1uxf(N(oT%a|8p3O{*gxYJwnZ7? z>eagIY!7$vtakKe6xDPFowul}qW-7?m~G-*cv@gJpS0(8-QUF_g=k9SGquy-qd8H- zqy-=`*ZAN?oTSmSuu&-JQKbYiNq5QT0(`2Dn5M2d$p82CY_vvh)Vy|-R11R`?CrJ0N%Mp?u zT)1`aGXyDFoesICY4*}%B!;~BI}*{YdmY~&J);APcS$ufB#QI3TPG#`{5r^~DBN~= zqZKa25lXW_c|^Pyr?%^)&$}`sd7RQ@UGE93QLi6{ea(Qxwnoa6XhH3vlARJa^2`r_ zke;(8C9UawcqZU&b+^OVgFrbkV&fKvFvk@tPQIEl%S@lIA0y5Z_wh2NgNLc{{N%)7 zVpG}ceO)Gugf`=5PD8Yx@N&P5B}r|=tx^ZMR~brs)>MF{hTOOD6! z-IzU8+gGX$H)kIjcceCBY<)ow!#7jM@P9R9nKgQ{CbYipj4_c2gSCnUMNi%_I}`5y ze|*(Zj0BVud+hWRsG7a_Epc0QM?O$fc&fimF~}vh)XRH@gP>D}w%S5DQa$z<_{#~+ zQ&H@MEf%==pc+oD_G_K~=MHQNI$4wzINQ-3B&wL@SyX^cy+h1f@)IbgrdTl zD0Wv1&pPtSNMN8=l9{IPJ;4^@ff?+TwPQV4guiVs6-yQYcziO*<)kQYR`E;F>v_KE z`Mhxqj|GPIr`-Setz_6}7cHda2x!0-ZGM-#@l-D2W<<4(2|ke;=ag~MIKyS;Ffi|S z6p1tZ-TgtjaUrdbA+4yDXB4F!06V$pmU8U5>{%dj@}qBg0!xxiZ_T-)x{1cjKQ1AF z|61An{~gfk=`+;PU9yLgzh#I(URQ zpbxQhy)m(V_>Q}vhO)zNcydhS8+z^t7TM0C?W$cNXC`K4S$;Z24u}u!;F}hX^J<{w z`TV4cy$rGfnHo^~16C@2>rjByl(i&i7Xdxt?3{9Ml@hIvo-PHSYXBV)2(7&DN7gSB z5_uPB9WEMWCiGzXI|pROy2NXeeydJ==4J`WbE>@S#F?tSGYK}#4X&~YJ~_1*aIf5TgBP zI-4B2TE2*gNMlV@+eUkf=76hc_|m}|=Rd-Gp3ejmh)f+>G}d}|BIcZFh<)Q?QX?0WuAux`B6lh0(t!jFfmf>n|Z zRK$qv^t-a?I<{yucmX5ONkxKl77NofCSylSb%~4%!H;f#+?>a~G(50mWAtf;TlHz{ zb0>(g=7$@~<56JqF5}elVK3}#=+|Vd@93&=)vu+cuLI9Ar(!hdK*rLR?Mfx$p4RbC zy3K>y%rAr0BLkBk$126}5F1Uo#u?dEh+pux3C{jj9m1Oug!YO-RzyAt?oe1GH3>ca z;O@KL&Er~a;{Q(DQaUw6UuXIuwm~TyD5Di&F48;u07h&`=p@xC-;EOS@$aWQ=rjf!>Xuc*{vW#h3@hQ7bS!fVNv9A!kHv1d{?V=icg-1O^im0H$+J$+bP!}aSg zhhLA;2MVlY047kuFNQl=< zMNYzP+*C_)b$o!1Aq)-ElzimhgY4=XD)OqdOnQPNlW^ZL4tv~H?&(9)X}F*vkACh@ zTMmi@@IV@u=bAa6sjiLKd%t7~^@*1Ezr)xUT*L=fwLGWxb0uk(Y?|P@g3ofmScl)4 zdl9*zw7*e1EbWY|RKdaX&pd5FBYa)(V&0nvc+9Uc#XhzPw)cBUWB`?dS^Su3&sC$g zhg#FM39UCR{>rN!Q<)U}JoNZ=h4QZEPPxL#lKO6V{`_+(3N^8-E@dlQ%wdvpe%9Sy zUDOkjMDh2NOkk(;j;AK1>I~#NF{fBWuDvvlEhFg+oZXW&Kjj*x3n&andXi=5DR$Q^ zthkjF5Dh-YQpB=Puh5ChrEE_Dp54R#l~g6Du(eodi;Dxx!HN0vLhG#ZfN#2UAxXbN z$q#V+P|c^6tJE`E?0sbP+6P+1qwk;z>_cH3rAKpDTHMSmw7WM$b8mcEsUjuQI;7_L zsH7MKK-WN#a&rlA6L}VZwAr1UtHeV?IXPNp5t0Hj^C2ebxD#vBqZ$~Nn@I~NPuP3@ zeMaZhmzdVj!LS1SyhWcW_YL3Xre)B#^4ym%KBCai>RjVjA^5`P^G{L?TGt}4p{aPl)E|^VS48su zg0Z=l&heJ(c_x$?uo0`} zo%cXNraogZiIQ70P%#^m5owZSIHJy(v!^Q5Z%35~IQ#5ze8LB$Glf0_RFt1$6`g>T{ju@{U-EH2d51RseyJ-|!& zl{aCdPG-b=a4Ao9lutV;K^r#+Ic|Owt+xqX%JyZjKSBA}0mF3OiP6ZRXo=o2_Q0an z-Ag=LqV611zdv~Sh0$n}ACqL8QO~CYqH1>H0u@EHzH$a;WWDVnrw4D-v@^>$RE!PJ zy;EveqDoS$8;fZ7UkAARFbe9jCiX{N-J*%E4sA`ZwTc~zsjhj zrZYQQuUC7{)UjJ$VWdR=2IVVCA|7xK8bObZ(94iu)jMDg_P-C-h^2Ac237urhq%Vf zo*r8UX%uMK%;TQGcf0kvioA9 zwzxxfXU{2WrCbzrpQKxoU^L+{Sf}*{4JWwmgN8Bk%(AxnjPl8Wrfer%^^n)(C6Ef_ z%&v_2zg%}e`ZQC*5D2vxEj)w@!(B(Eie)hj6$ZItR7ocHG;F)kyk2N9JeO5WSUFIE z-I#;^b{gSH$11mgTg~4QVx#zwR@)8T_wbh4VNnJM+kDx^qmv%Ncem$e!H$Dmxh4=ki zVFdlH#7aG)+*1Bq?>1dXtSpdwaJ1(FJ50W%^8=}~)rmdd&$(94dfTy^iWKjJ z`S>nZ965r>|1jEdq&fJOYn(}h2Fi?({fNZ6Em3Ua{KD}PqA{j92(|#>ylMK=HYyGK zOuqc(m@e%{v4us>3Tf2}GbH7h=+KTgDpB`w?^UqrMG$anQoZG=SB;kmi3z0V_Iq3~E1Rr4R3>s-qx|l`Z)ST6gTcW@fLeN#_ zT%dkPE(L=uUF5Dp-QO?aTz!TuC!m2sP9Z$F`gg=;W>Q`5T7-i6spf~R)s*x5tsttN zS-SeJX|gF!&xP$!Lrl*bVu6Wb6JUio77hly@p?W6J7{&L0{o{Eu!ZpD&@*C6PVsPgdN6$#Ih@tPGMqo&`&4#ru*3 zhgUm{9#V#k_@`?HaqK9bxvORnA+m)OZ#UtJ_}$Y6<6}5zin`a=b-5|jkyLXlmk|1w zqM8E*ZAG4*401g{+jsx*9HsrZt|?oeTW?oEoFmg8`2mk9;-lhM`rO-PXHGq>3l7%v z&%W3M@h(xC=gH8$J4Cx})iPA~jvIV`3*&REqP~~^Te!^#;4~=UCSW!GL3vKxelnDe zq~0T%WexbenUvfu?GIX3p$*J=>Z>fqke@e0=dsT`wjVh4!#sBDd0gXf({#J+Reo&y zv!cc~dG3NoK3;Xl56SU2U8CsR7!~{oh3FR6X~)}6vnU{| zsQO4{s*a?GmYb(X*q24x((6#KOcwKK?{&e?VC=X11;;VeEMHfViJ`yIL^5$Roj$Xi z%-&a4v0M+a%4^Ovz2vjebGV*jj7W?dXpOrt>vo#d?{j;AKUgSUR5O2|bce<;hm#W4*m#a^D*RKpZiNdMGHFAM4Nh*<`E zu@2h$=LXEGANZH-X%y=FZ8*&=+KKGDUpYvL@#4P?Z_h9NCqUh^TypaK@cEt#;t>6BKZA|y>q$!(~;V zp~eq)HzVW;!?W4*Ct#K#+-ivW#YY6K!Kn`~`H(=}&eDNVHoj=^^S6QB|C{>iYj>3d zhyk^<4TaqfHZr|V?8_Z)U;csP(6lndY@%@3u15#aN$vVm!jQr5IA-@=x&zVQUh3riCcU{cmT?>X#7PuXQHnYBiY;-? z&?d7`Ja~__1$6ZtLb6gOcEp5Ug+#p^CbCZBOF!$ufg}#Zy};B)9Fp{_fX??FnsndHCGQgOk`|A8D zRZuV$7JLkz4gPKsRdWuJmCNBeC_YU8B<_q*h7Dysun^{9r4+B;DnKUWJ|}<3l|BWUEWKM%#1Ics6P|F} zygJK8fpGRNqep=Ag*z!r9%M1jMTq!VGx623g-E#7-{o3SF##*MJJ^?Q?E{*)S&jW~ z+xy|pruetJ)96o&e5FS@Z;QsJ5|bRhO&sGjPao%jr6V^h#2KG;;4Y3q^S~bpVs69t9}N0 zy#lN*d~RMnb~jXP1P9i*-DP9w9cZ!qmt=g!gdl;YmNae8-^pzA_EVdpDbx zxQi&YSpL+1b;RpIrnPpX`j_jA*P!#4ABk76A!rX@&Fu)V_q1qsxv8JVNvYs&e@$^2{ABrQ! zf4(75bbk-AGaX1ikT%zn^OIih8IzI4Xt18K1uoctJ5~f;!>*h^jdGgX&v15lH0m)0 zYBmGuSzzIWL34XA^F^W@F8eg45M)QciQC^MP&VC~w?%*FyD4hU3oNa8^7Pz4RtnAY z+#Jy3_lzq`bLHDaQIcj6!8bk8iL{pVzoW@E_=ll9twQ7UX-1sL`YwdxZRK1s1KE&I z@!>w(cb>gE3DvZ9=boNFhHZS@fOG%HeG-fd2iGT^4JQ&qgF3Y=dIMeZ;}Q`Qx!AaF zhn>=Ss~R%j;(C+O*sbCiSl3FW|Lvb6)jSIzJ$_PRYlmt?B7vxG!v$lotWzdC8sV_} z-<1)av|@MNVeC(vVz?~s^7~iSh@UHCVWyQDeWQGz^=rw=IUE=Ea#F4ct~c@__vg6MjR zrv}jnTkp0^^R@xILHxplKdC$!Dt0WHCH(!qLZ*Hp(XsusQKd{-rKVqfR!^DCxPvbI z3xCTX`X)-$4}aWzay|X3wEdeJ(qG=Uxy^H~m5`6#^hH&9ugBYh+g1jwUE^qPy~MQ} z^ubwo0=bOX&>nIrc)upp`I^0XXyW zqFo*S>}0-cDS`A~p)y5&Ibm!0J&!l3QvqO88d2;iu=W!OGo{aH3GY*vEneZ0L=iZi zIqTW*bJuedtC2GId=ys+2F(f4mKIRL6d{>a&|WqU6ROT2muE_Wi zR2wL7+r%1g8EiiwAkN$Bu?tcO0?941{E?(jN4GPGWsP3sp!H|Qu4XnHOlPailq7z_ z|2M49Sekv-aZYUa8Qg(b>XtJPYI#a?4kNV!+i!;gAS zF+=`;5X2I&(o8PuT~PDesIh9dfz+ebX9FvxZ`k}0CoQu0IYBsgQj+=Lv ziD4Pi5}L*Db+YwutY<}76|y*j_F1#L28kWiMN9B+Dvq5<@pStsr)9J)pRnb0mIIs# zFa7#h<-<#aEZ){~f)+GY%o`Enu^fS7uV$iN!bwH&L1?ZO$a zQYJB7wwEb3o+3>Y+4Lyx)B#Ns)uXD8jt$s_UgLC*Y@XHMWkYl&Qi~W%TM0d9vNLD# zvU882bo0x|oiTsMD9tRcT=krkBw^lwc)zn}TD3(g?A;_Sr0AK5Kur-l11oSqFgKpY zh=OYp0dEtp2-uF2{=es7*QhKb@4iS{6ON)8rUk)u}3LWkbcbo;5Y7%*th-{V% zb=z3nSmdG~QSuHaw`q0DQ0cm|xw=PJf;~YS8WwE00s#>;fY(zJPpBt~qZwk6uo>?o z!W{&vY=-hP_3AqI<|mV&WeN@(7hzxCwHd8+M5x8gpxmbaU8-EAl_tnG*gN3IJDQjE z8~mt0M)&A~+>C?-p~7!AP!K@NcmyKoS)c*2^O zgHw`VR8;IfCZ_E{(i`;#$PX>pWyVJa*~t^U6eX zRTERN0_$xUp({B_I3G%HJAwFfkYpe#MqTLKlCh;8I(Ul!juc%?1@s=;dJ5IegIX zB}WT$m%%F}ZuH%Et2-tNQ%uazLJfGcG}$}fvjzyDx_;% z?t%&#k0_1Gty&{AcN|OsYN$78iiFC?*Mq{$E zOb+_8Ku)lFU2}enkDGn6SGlcoGK>xRj%{nVrxAzKbTnvZXQy}nKi;Z3TkU~=`?r)P z+sS~i1eU^hZ(XFOJGA|Gg4jw~cNG_}Gv`tc;ing;qX34R<9JijdC=dT6+9#zYZY>Q zlz8PS=#m>?p^$i)O?j6qe0e>p-_ix-UJUNNo&BVg`%mj^b0I?VE-#ja5priSY_{$> z5AWk`-l2SdJQ_S2+;}BCAEGhHM^?#WKa&CmW$?8nt95A11TwwGxPzCjzB#;o;~sRZ zEXm2&pTDk&|BGl8d8Mcu5pEO|$0!Jdu=FS~g@r5eq*zoN3#|dB;;_lzZej?}ZTc^5 z?YO=T9=jn36

CQPzdE|mc z<7P_w*%g^{dCa^%=FBML|E8=7r*N~inS+iW$V28^8mzjHbbS)j z=bYA|`lizjZ#rRV1@omg?zs{X)0RztdN}wzBxqb8tKRHscht27Q*_rt$IU5TcEB*ifc3#XX!)?2}-HFmUERUz>bX<;~Q%S5lcZt2|hU}hT zTH3>A?h`$^)DT3dj(KRXHq2YfB+)BebX8lCE3AiJWV~_BWhzyebW4AZIgXJNXWBYoBoHF~(gxid<_>DPZqnumzAa_3qjar9mw-zU+-Y9M<#$wo4 z@KYJls}TEG3D-`;SMBx`KfQ&WmrjpZYCj|m(Xk#E;g$oohn`vawfXU&0~2uR0ex~( zvyXUW;Ss~wfT?&U3RLaYi69s?M<0~3*mksGI=#zDwRjfJCrVhHB7qltr2)y+SYYdf z{o^%WgrxH{SQ+B&Gx~JFLzvEo3^F8cJBe@xT|P{I47W4jUz}@UjOmKkTc*yOdsPHn zMhHFs9Z6KcQ$(_VXE_DOFPfdcnsO#uB6{McPr1V%n%q-lciZ-;t>oK=5Sbj3T8J0Q zVQC#)922Qi6S>x?%jO~oQY?LvP@9Z41=;^@5wdcNaoAKnH?#5W3sSbjuipX#FU=_V z5=BnLTnSLZN>dD)zzbwwogSYdRKobvQLap|29fW1somG;w^BEqv%02qzspFxaBjF0 zhA@<-q!w>Gth;Y+|Iu%c`tsb*1RjBq*M8`*&K&(#c4DN>-fj&2{)Ttcy3HM!Si!id zV6EMpu((>kwG8?#!^8DcKnQTd#8it?e{=eS`kpyh(Erqw>r^I;T6Iy_5Rq5`GSLTU z)I2%xis`{^Gy0rX{GoAn126dAh%42PsPch1N2$5_xQDUuSTw&HB|~=fh~>UA02xpr zG_0Q3VGNaF9;^%lQx#~^Ck?%fvNUS3;$6-`E64XsKw&|z0WXdyFDHpmpgC?thfy12 z(k#3}p!%jN1?^Hbrp#j%+p(rl>IrrI9+~|wfUwSOrEUJwn6Xxm|GB9UGjSHUgpaI` zO(yW#zEm~fC&0#pP;akf*j6vboS&JD@pA|eY`A|&U z^!>#D-#fq)rbXI*ug;#i#c3%svlI`hRO?sbVwy729lzqtP7j+;Yf^R!zo$Notd!Tf z;)xy)Ht^aSN-~UUkLn!Ti!E97{8;|z*pDTyM zpDv>?!cvi?FHTEk)w784Xc*ZwMwA^vRl`EZkXhP>;ctTaC`hf9m@ zK4)0;i+!IK&6)RV>SGu^_)*FU#%AuKMwgMu{C^Onwyc{6$6<%C%eh8kQ?x0!UJd4p z(fQDbu#}(lo4u;1eCbDwc&_C$KNpVa7zy~o5m_|B5ST|`KdKT4m+)?ePUl`;8rq&` zkag|0!ACyuZS~^+y-b&+X!R~u)$TUjuboW2Me>P0_qypS`c_5LS-d5DY~c6yL;fVd zsJ7@?o{bHAtv1D{%2gc|X7I)WVLn(?RDR2sk>~0siH=83pJHl2O}w|%XkkoAQS?Bg z^4r9aoiW2uj`iY9q7GwC*Qg8EJ*EWwV9Ogli860Y`I61zO$=OCZivv4(RA@HaarEzO$jv>kon2Q(eY zxG*~i#ky>NZc=XWnCPsi$fr)Iwg#J4+rJ$pl1Cy?M4k~Y8gKxejGSI=D16D5&)hF)2)ffB9f74Bx)M4Kg*m}bC?lfK319V*A`YXUL+K%A zgfO<=i|Gy7ucyOT%e(a_nDSbrLPdP;%|RT1T-G{%zIqgEVG>DmUU_y+!mWAY)S0a9 zIpL12=a_m3duDm9kMp5M{A~M(#=)dJBj>|#8E+)9TS!tTqFEhK(S;~CU?<7!w~S@2ppM@oNmhplEaA>WXHTt3m#S-l=;`x`$MJuK-q z*}8|)%YW4_d-}*GUN7qU7mj<_-fy={nU)!`Dw6twZ8Fb_6BuDly0bn@Ie){u{D3&Q zc#_C=-AJENP8^|!?y0R=9XT(d=Xa>>zTgE=S%F7>J$rJaOMF&;Lo&D3URj_GLgYcY zve=KQ4eAv*-3WDK&4soN+To@{5N+p?`3MpzJ@QFOlF#NU!F<1h@>4SBa}9$?hVwIInSd!j>68J}VUBOGlqG1v zSO0gVOFrcVYoeY34>cPuj^)Sx?nDU#N6+WuTy10&ba5A#V97<*WZZ8I6v zW~xt)uNmsT6JbG3l~E-IZ0Yq&RyC8saL4o@K1$NHyhi%Vx+nX7E0h?qG3Ywfo0j)A#x15 zfJ8afEf|F!_KzioPI*I7_*t#vq2+s!z-%U_6jBES?V#7juVM7x^p!L|KAcs*g6Q~I zH{(Y|C)1QBiAX*>EpP@F!IqTe-!htM=lV6(W7YiQ@b1HX^o>##rtX;4-P5)2Bm$s3 z3&~)wm8?rlT%kEjb4${DahmCn+yVC~o1cok3#^E{(%t?(FS|vXnD$hP$!-;!iyAgT z1gWJWv1;QFZYc5@sn$Jz-Bj;m1fPTJsPz9gk|N!&^*wRW+R0)WHgXi1cg;&e5lJq7 z+_ZlT&@yfjg$(q)!<71haU*qOq-qWCe`CH~E-Ivl<}8bt?k|r_h8+{sgN)=SpQ=-@ zoL^v=IAZ6?g${XRQZ(K__(>T2=l9*_Z)l($-%GewwIOHI^ed&X-Exo}c_N#k{hu3* zpGE$7MO)v);pX4evTk1c3{G=Ixzb;YIvu${B${{ao17I)Wfn&bSS(`GLzefj1E&ov zV4QO&`6Z0rG2$x+vrcIonGd_*b#`JZR||-{iB%6uLGo}yP_a8)LfHf}smZb36t;AT z^{+uoOJsCJM}%-fdiBL*kqs)0&>vhm_H0b1M%>PhH=6A(``g@}T~=xC{S_IISRE>> zW{O7%KFFrSZ+bT}_b^|8<`ROGdBJt06caZ2OIVBctRaQVtY&zA8HO8wffz8P`hz^R zpT#lBQIuX+6TkVh3UK;CGm{|zzncfxO_nJK7k8S%3KV<)bd_29!Oj|%zd|XLL=vN1 z+D&BTk&B`u$-x5FaN$UZx*{^`RI?)cTXil^{6^q zNxvczIS5H*bL(>9=>t1;_B`go;oTEv$F@5S68!MdtNu*@C!8+62XvWeoK!jP7e{{P z(vcdJumlvmA9QEEM%#0x<7=h@DYt`9=E2vV1y@_8*WhPy=a`>nA{Csb+Fb5Ky5<5M z3(wLxs%2R2Bwjjsr%@nf7mkLSi$~s}D>l3GgO~g4b@6R->pF8XgW)40&_>$)u;+4p z){=!a=J9j!Qh;J1&AaG+#*F94M7Mj6wmZi}<8`_-yUhNTTRZ~Zc)3Zc%izT9sC${& zYZb~=is1-P^`#DL1va~$FN%`>Dk3aUV&wDD;#D(mib=g*yp52L*gGO6u`s!AS zMu^YaDU@n2DYOq`t(E*X;7Av?JvZM%FG770ZBg|ur{>{uRiC7}q$Ad9s3M{-yHyX` zR9v6Npq>eO>=;oyUt1$}2WgVk|M)JMtp>eH%O0v|(nmb*Rqv!4L+X3yGwLKt%ZGuC zp>^u2Ca3c>%@~mqvk}Ilk#4S%_|IVq>IX26j);zw2}lj9#!&c(dES#1ok?&ed{p=B z^-pX6R;1+CLrsXW+SnfW%$JyPv%GN=h2UHW~;Y50eI{(iia_3 z8=#OYh%uwl1EK(R=W6^oWGH(!|7~$znc5jOA_q9d7!#;|K51^_Su?db63#oCl!UMB z-V0db-o}8CGA}#O`g8N1nX6lY$uS5Da~z~28?n!ot9YO6sE1j}4|wzdMG37^y0e?O z-P?CAq8N)7xHI#0I&=6-2(hIe8@U@i5w&v>jzzKE9y$>5@6-F39 z@-{08vS?yNSHYq*!%A7A-3g1EKSB%3#8v%&UI@G}TC(kYwv5d%6(^Sb@eiL4L(}DKn-oEcPK5Jg^}L5&uvvN@YN zMeWb%-9^u$R@Ftq{)P5=q(#c6i0yJZMPHV;GwCD%T5X!{{59*H(Db=aQ66iplf5y_ zR@H>NWeke;PJMEn<|u*ek(5sZU#{!Jh!BdC8W%)Z4=hEx!bj|cNo>|rnd_2^KN0%lFM4Wj%Czh=r^lE#t}N|LK^R1bY>LdPjdaB!Y+e&+%m)PM>(WMRG*%$cpv%J3N^aKf!~KO_9TO8JH4|=S~RPf9(n~2)pS;1jFXS;*t!}P z7gWJFK*YbU!LOt*&R?(s{QFV=lBa^94WU%oZlTCxl_-Aqn~^+;~czltYXZ(TUa6M7VQ= z2z>;}UjP=+k4z5P6aB&{lTF6)%nfGaka6+)iCYcyK`vLZU*6rB8)o?u75_}l=+;jr zE4=wKj!6^SZ%K_1IoValkTPb58^M!ftwrf;u=m5@?I^8I2#IK0D`Aiad{dJS*u0I# zj>!n$>L~2B_I1Ou{bIltm+cH`Jx&xuU_WhMxL*F1(vQWF!rbR(BAGxT{;Wz38Uq=UnUo2L<&eY@KUz9DVtJ;82XJv8P113 zWGzmH-*7Q3_xs*#t|jmk>O=Qo$Vfgy%2y@cjwxmF?_3JtG^;$}j9m{oMvV{z$d-4- z_^IF=l2s2)Vf)RnNPUuo^xtt z{-~2;F0zU!-vCNudW$bjAZuitPU4e@7?i(c0Y`z{U%=4Q*Fn5P4#<@C?CQ<+u0pZ! zaGY_<*`^FGqQz*eWJ24&5*)Tasr_k0hl%_sxRP!@k0f$|kYlkxH$-W-5>8(17u4`I z;Sn%XOA>2t+PkPrNo$<1CRVJEbd4)ycSs|s{$L&=yClsCm#2MO4n~7hNs-Ff+ZYS% zpZRmRozP#2+Aj+M0XC)Pjn4~6~!{+J4*oSwmbFCe**AV^A)_2Mwtq(AgkD?!hXu$-4hM&qcvpZ1JiID@W!8uNLxGo%=2@VH`@1U0W)L0zW5y>x9h^MlhVc0&? zR(Km_CFKrz2mke84g-xw3LmPJC~J@MqtoT8=kI-`u=XM30@bF5e(`vXH>u5Q-e~9Fj$b=tgr3 zrBPWrPQ8Bz@!nPh!ie3nTgkPzYX;HS^bM}f=y3lFI$a&*)C1)bM6nyzE*s94w;h4F zAa2YUuvJU7M(&F4#K>~XB-6Pp_BgB1;}7&BjiR_FhjSQsBB8s!ysR>iMtSrSk)`?GSXhJ+IM)qDq1=)-8c5; z!VnQ`D2+bRvWa;dI1w?m@eTX^)LCekZdAKjR3;zYnlkPxV-GyfP>t16!t;553(DCz z_UOhacJ4~`0E38KZAL0;Y+;?{y_Y|xgI)Zoq0Fj1-N`5-WjbTMkeeq{herTR_hmbp zW`}7s8lkp1NQ%n)nFR%Uh0utF5kN7kxSiV$x#4!i+oFVRkGn#i%G9o)@f;@1y^OYm zzuiqIH!E?AXfSJTB>m>Oc7_srB)zy?U+}Jb!uDn?6X!s}a=(%F&V*mCh5fLS9o>m2 zJ3eAYT-|b*1;6kU>j#A)i!w5E6e+dJfYFFV_$1jSaAs1Wp*2F#7TY4S+ z)|X+(iJ17;`=jD{>*ztl*)Aq+SI)A7ngAVoKSHu3DIOm7CQyuASjUQbKF2VtJh4(p zLbNijSsk%7iRG;#gjS_@k^*o3vF2TFU10!60^wonk@`)-LIQTVdX21hFG?`+Xr9K4 zbPvxi6O#=4Ct~+LDp>E+Yw2keLMcD(_u&f@=if)d(Wf-uj>7{s$Bo$ib%TbW<#rv@ zrF!`;MK&8Bl0LeK`pXvrV7T1vVJYr)L1N(>-SHE1h}B=~{t(hR*v5cD_tl#J*Jzde zb%#pGwq&S=9j1*IzRU*ZX}or`)bHn+h>ig0hwNov2P-!qKgo(b<%ErIOmBStvWs-ws#q_u=j>oJvHz53&73l ztUr zsL{vkk;6E?BNyHJ;YKExC{5HOH+Fsoo4n#t%6YDnDl%zw#wZasIxv!j-C2|s*BG(~ zAD?NnnRY}K14Y$kDFp{-_z|y(Un~0q!H(v1>lQk&!raZmllceSsE;r~>_(l@x}}JA zI?WIIP%ZrMFS+M@E4zJfwXq4LFFjYZo;$mbRhampO*Cx;slsg~dLwPTH$Bj!iJOXX zavxZKOc$FFku!$JxkdNG3d#azkHow{&|^9ju0hc2{jEbYEE+3@nO$7oc2im+&EB@V zyPnYN%x%{l@#I8u2PV0E-QQispPWqg{;y~7U7oJC662XjbvBX~i34m*vw2H&n5qfj z@|DW)Nv$du&*W_uCGBNtw|e=2wxQOY-VYhtG_qSB1-s4`-jzdFG2vJ#GZRA39E1AQ zMgMOIarTxWq`air1rhh;Lg$p%q6zW6t7pnhIS^~;P+^VGm0==uSdw)tp1L&A1rdY+Jk#N>VY7~Q5to0);435C0u za>Yn=4Y>#QJ$m{mj7tgjF!Vp^OZ;t{4Azzd{YO%%^QfG^)E_^B+wJ02y6>Mh#_EdO8?0}Si!ADYUn8MC@Eey z?Ce3&#s^i8>}{Wk_)Ke8oto8l-l)4O6ul_TR~FV15lXgB@X^vfO1;6Ii4f`{b_S$O zF-;fJADbijZ^7D?7D0oHx4`@+f4|1JTHSsTs_i$DJG*CJx6~@Vio4LJO0VEPUpeIJ z`ZcZB^$&NdCQqttot5no5__;RUCB!7tU(rDk-T34Y}knWEvAt}cH3x3sFAGk*59Q7 zbcGyA23y1tjWe5*%rJ@V_N*a#&73}!4@E_1zKH3W4b<|>02L698(Y**JO^d}ddIXR zHr8iu!3$EpEUZ+-;^$?9Ofb$8Ye3*xGy)^yMs#1Tb;_B9;t4i)3)@9uy=Y!b{=7ty zb%9tC5?bS{>-AwSlEBdO*KNS@Ijg0q?c)BbCQG(O2hFkDJ#|_Zl;wM^uSt-M<9>g6 z$Mh+n9gbF28(Yz7Sw=CCYD=}4nl)`zziO+Zv^tCFx1tw;3zyw@1NgGzV0hstulSwy zwl!e|5x;ePtCNj-#bEO|JF+)By#<$glf#yt1w&Zv@?G z;ZOzQ)0P4oTJ(;s*N6%OG=#rWz1^nSD}fcuGOw{Y)k+Uj;xN)70=G$H4( zP9$tL2`Ms`5jXgqdgR)_i+|D>D?bBr_nhmgI17%Xy^Dv*4Ltzvc|e}aLlpPu=e2BD zlu9qe3=1^|$F=SvnjtPm4whbHIMuV`;Iz#{$F57#mqi50uB@ z?CPfK$ME&bIqSrLKr0BX5~Mt%DGAA_U#QcN+`@#ICxS9wl!LXq^ss6jk(!>(G9I;y zP{bb~5i3$`*l_HxQpxhuY#98DT-qHcw0n5nUgAvfOy&(0l&nisJ zU9$n&)>33*_{46V`9)fuQPn+`QRc)%4Ee)Z)#K~q2S!5cy$KZJ4EDz-&wIK+rVCk1 zk5S!wyv)tkkVA!{vMid4J@E#4z}C=VR*^TwE)TzO15}=SvaZ*Y2LrL*L!o9Lqw&=havnLsZSI*II|nRR!4TuWyH; z-fC=$EBMG&s_anRkua2;BHgDdLIXekqQ5M4!qS`NR@ zZy^k%(Tyi*cT}F1T^f%_HDusfR|ePLIqr2vh%*S%BXpJ~20PaX+gP@j?YQnX|0;1j zcOTeBf?YNHAng}ZlrP-UOJ^>^lC!=ys}VL0o`aa7E>a(TBih@{A!$_E>=_kv-o~`x zEkw=2;OS58+(Jlg!fUkW_4+`$%7l~StgRF5_RM9BZlJ!v0z+ou>A?CdsLva=;*5EZ zft;j~$dT+gk=<&8@ZQzrai2>&oI|?5L1s1QWdka@SpbPsxd05#^84!Y1C&v#3H>{I z=5?I5IPMy)>a6o-_dPQ0%(QsS08)#}>p0Beszo--q;XFMRd>`5b?Y1yl(e6qyS*dY zte(9~;iopIp)vPf^dFyWiiyu!WjTWVkmGYZMw)D4c>c&^(4aAM0y{3YW|fP*riB^b z4H0^$4GYKkJkSbA+vLBrT|KcIanOt<)I$+8T*Ot%bs3nqBc3A3>5Lw5(7LI+t`#qq zd#UKb(gnOH+6NE1!RE2Zy)pAeE1-H|YSk>`r~Y5dR}177*rpTHiq}J{$xdb%@>DwL z;>500z4s;W+D3b5&gN6NyElxyrSA>B{_2fTj;gc4HfCnazsr5`r+BtdTMtm2(hmg8 zS@qGwk>i`PhL2Y&&xju=!1~76D1_7H7{fzTO0h~2$YrSBh(CA4sHeb7=oU=C0Mpsp z({p|fo$+(sG?V8{R9nQ>o=nzR9)ry=_omdkUfGtFPvoR8%l?a19Of8J z_@c40CDdzS7#&bMV5_MaCeM#+>cAlr@Tb-3u>qdMPvxGiuYLkowpOoqC&3ZC93E{p zTin%lISQ?yZMr=WILuec)0~!>*&Tp5B2Oc8ptE(NoTE^q^M)v6RAEHzXN`Iz&Ge;n0Vnh=Z#8Rma z%>=|S>x+ml^PP3UpPF$G%JE{jH**(^O$8ZEQ4REolFExP#jXr1=~;?{KNhXAt0ige z8g~N0BOJ6J*jG|49!u1^Gg@vQA`ZKX26bZ}I>5ndTQ8i$kF}{7ox(tdX3e{B<6Nc4 zz3&pCTs1SykaR;m;hCE6e+YWs=8;Q@UdhL&c6p18qKB46haLT3=`HD9R52?r5LuO2 z2rbg4^R{R1#v7OR$pZ1s`+^f<_}X?dOoXxZ2JXhbfJz}J-E3V#yYd%D4>VRrNNHQq z@7%$gi^f6pE4G8t^^CZa$m*+*-Tj+)rmA${punQSsd5lrCUZ?|~8c*hSP(!d1KTyQ8kAHv%b^*Pl~CF)~KdN~hi5wl{dY zkI=qtz+YkkZ3TN_^LmBLZF^$uiM!c|WbI&G4&PkDAKcVw@~77FR$zZ&MZwctg$9J# zEw?e@0`Gd5#D$Ykz2{ykZY$mso<&hHJT5kPgbNXu)|rChy!YeHHB!J=l)qHO$(5g5 z9)P(-KI{7orDL35*Vu+?tvF6Jzs2};bSVZRMWvmugqiE5-%8P*fjjAoLVT|3yHGJN zMDIm;xPs^N=+;pI;)l@^nUiD#EK3k3N$76yKe8Pjy?t~3u#~;iiAQa4_T<2Wi7s|e z95+@qCE6SzQ?G@>>CvSQe6E>_qr?p_$rA^S0`JOTXS6@nSYrKe{!~=9cE>Eb!qGVT ziJoACq}x2KC4HFfZv$gPq}6jTDk7{Y-Hfd*D@NIL02UhCfy6tTJ_<@)5jG=}gIn}W zrm9zO!la@+8&9SZFa1#da!##B(h$=p*dOfZvcY(nG%B^noND0K+Urp~GSP?qc!}wV z@sq@~$HLTef(7^x!Y zaz9ofL!nde`0J1%P*5+mE$cZ;;D02zB*qGvX91@T9}XuAuP^r0LfoHoY`M zi*NL~m;B(N5rwq*DNW)Ck`NgXQ#b*T3WAtB84s7sipx}huUolWJat1hg)R+b$rXqOFA|&jkMYv^4eh zXT+F(7m(g0bhz6&X~2`c@muuaN_MLkQpOfScneqpS1!qRAzn$dZ;p zjtG6zZ)w5yq?Co;9HgMD`c!$mLqe3ZP|NX$;RQKOF^r)aHM@xKia&W+4Ls&i8_|n+ z`6;q4yFU^0@G`8H6`MjKwMdwWyvlcL@CALFb0udBk^wCfThiVw!&+l`{q7ScGX%p) z5{vQ?4E-e?H@iNxAO7UWg*cPGwX}I8T>X2P@LjAA{CEvQC~Q}edE9criCJJCe$bZ=1a$ClFG0HvE!a%Pjcii#|#{>W4bgXlSf?;A?l+J>kvG%+VSm zp_^=s!*GsfkL}*!OffWM5|Fk>;!54U%X1wZ`fMMnG9inT%8TxR#Ru{^+xi?!$k@av z*`|N|^lNZ>3%{+h#wzz~nBl3yE&9LnY#(yC9MlT`-+P$5OH6vbbAlvftFkh5BqJog zx}jqH+M{s2l^&hkV%lJDa*Z zvflwHkv@O%l@#~Op9xOG)3Vgsxn75j8VqPC6`eU{^ehtOtGQe?f(t_B`oq>U;J$+@ z3l;2iC9FGfAYyJMVPkKc;_JQ|(u6?W%9s&sPxG!eB%KpDU*rgu8gA048}Mmgj_=msTI>1hKm?tC3opTl!iI0WKqk`AO1y z0H~qI{7l!MQCWA3+$D`GAEE@iieQ0VwqHQgz6_&1vt`;3E#UFsrW{j zLLfE)V#oE6xa5blyv3t@f-aesX&XyD^w^dok`IC&>)WKa(x+I8`$6Kva2HGjgXO`H z5kOGcYfK*)aLk8VAQX@T?cExnRi*jzV$!z9#yv%IOWX!XLf@bF|7iE@WK5u&kl`PD zWv-1NU!nOq$@M_)I5iZ|L}N_2zBuWTQ0 z_~{88Y8h`b(YQn%2U(bR1LA|8-fhc~!~P^a)KR2of)c;w@rJ-*nc+pj6Fwwd+Gp^S z)kF7Lb+&7QUXOQ>v_J8BTun$=A?~eMg>?1HMDy7R3Tz9Ki#XImTeMJ_XOrxPFaI;S z`>io*$bL&yHt1d%dPO5Twy!;ZB~}(9*jGqzL1985&qUOUo|*9`6%dA)ebO`M7KWk4 zCE5t_v;;?Cx72x=a~F(Aa^<}rO*s0Z0245E7dMtG28DWp{N z^)Avw8T%Uk%(T1bEG;KbMBlI9eQ_L#*xt;-=DlpUBW6y*l8kJqKZBqn0)Q^;9M0%hbWu4my*V56}uw))=J51h6aQH44x&f0;gekOv z=?WrxdP1lQQ^Jicv_bR-?B@M9UQXsPpB-*`NV|a!oW{ z8N&2&ib>GNLT`Lqb&B-+EX60EuGMiQ14go?s`~{@YiDwFZuAl#502D~iE*kzc-XZ4 zV?eS}Qk2JBxx1Y&@mm%)T`VKvHN6P>AeE|#Bc+taNAqX5YymtsIDWJ0#HGigL?k_SiAV;k1J{GnJ zU}fc#Z;GXSo_F_AY`EAY8evl^dhYE|wb`4+pXK<}Q$*d#AMVMe()7PG`bWyI%HapU_}ZJKjQPQ0BGt=F zOq$E49~$FI=mH7DIjWdl!;k#gq4?QZf^X`HlG-7vo%H+vs;b6Jow{q$yopW=QcK0* zm1KfuC!Qh|eBl%}LWiDC z_AA)dR&vY+sM5Bu9rWL4!D}iA()g{23{P8xXkhoDHmB=IaS7+&!-ZEL01zOOn3#MZ zf3=jNQ_Gf#bo6k>C&4F4jxea#u4AMESM`)6Go6&H{IJ{vmr6N>d1E0VRow* z1`^?T*atxuLSZy<*KzA}yAgg0#Ywf`QSfh74BRrJxPoqg@A!n?=mQocS zToR>tIQ^j|vS8u3k@HE=W-e=QfyM1Oz{~N)fn2sI*4^)`F^t2==%E08P?S-_TiFq- zO2Ekc(VAMoY;+Q}SC9j9H4lbulQH3QhkLP6;eNl*L?G{YtHyb)HhQM!{6Qn7J)|s! zfB>dGOvF#nhkBgtYI6x?8d93&hQtqko6VgcpVb%+*#j!-;U4cBM z9`QXsZHj;tix#h%U{&!QT+1kpf&qnebKc%>cjvluO##tRRLs06wXq^ga;idhNh)&1 zl4K2Zs~tNZ6T zn$$LIC0E1n+s|*4y!1!7(88tqR_!?|^zrO53UJO^n|p7<4(d!nfqIB~LP8i&#SchZ z?)X^Spx7o|!-I}gyMNX46K`4+x2sk^CaP<~kqa%Jt>qeSQ@JEx2@H!&z_8=#`bf8i z%=(NW{y_(G-Wj`>DTzQ90pq=y-$fB-j_CwCHeO!w`7e0TmZ z*Mpft8iQ{L9&>a#_cv_i5817?H5*P@h$jf0#1gxtye1uGJ1wXwL-&!^xWIz0QoSZq zo9kG+4gE#+VUZqzm6KlHW#R0$7KmHPBba!K+vU;=c}>(!-Rc z{gz&~ob4b&q~CUvv_K#UHI^U_4I=I#HRQy) zlRS6M@(<70T+;H=L2*V>Y7hsaQIKH+YQwgn&nk!lrm=c?13&`;_Y8-Y*=SY$VK{F z7TSA_3r1=U^;PS-5Z9Vz@@@2+{F)@T|9v`UP}+~z8^SAHeRPWj3w^WxDeR~uZ$I$=J&>=1At01m8AC|3NX`#P0Eb{iory8F%7JP zP{)?6b|L2Y{z~nN9FEHcs8(x9t*S+0$sSDDGh*37I_&hL8zvF{O7Ui`dL&rHM|@LT zPNGug+cst#R!vf}r_+uyp&jD%}etwWEzeJzhb}HuTXk1;>DLY`lMya`CZD`_Sz~vI#U{%RW zT{4gRX`g6jg{f=sf|0mpU&DCLHo%6h?d2G-zo2#2e3-K^ioIcOeYeEjQKXHAM=>c# zq;}QQ!w2oGIHxYpWqASbjq|qEX8~cef#71^L~W>@defq?1JP8pXg8s{5xr}3S#t8O zTjp0h*#vI7xOj^MdnsrzNcj8b3iT{EUW0h7l2Ij_x$1X~kz~SeL60$AL38uwfs76p zt6$^3p!xkt@Kpe$$1_9dN-ZjMmrIxkgKpIE1Ra1=zz+F@p*yktIeDmzhsKbZrN$b-z#;^54}m^ad3V)wJiS)^=_%hEaO^ z+mv}S5S{Z>!yOee)&%F|t{_UHu1sn?{4K*UL2q$H=pkRKUt-~>`bHC*n6<7D^G(Vo4N<18D;XHNB#xRBPrWp3E z&F@sVV_fN^(T9yM_)tVw>;SU4mVHH*WPb)f0d(lREd5_J+Hpv^287(TuLoYpd2a`}Dl&H#kAT8u(GkvSw*K*jh;% z4!S-yc_L(oAVOV2#fVj+5n;`|8H50nFbfu~4D9y<&jMlj`1<&XoaEGMO&j<1NiX$T zrosqu-y(eTF5{A4I7yae6XlmQ(fTEc$KCf(ZOe+|V zD-q!KSf=|;sB?DUE)Cm)SR>la#i`P9Lpo&A)_V}ZbWtRf-h`$SIhN+0EG*DbyXPw> zBQrY6ce=zU5g6cr!78`6J0Pr7RK(A67g+*_6A*&af+siZf$jb`GQEn849m;zMZP1} z(-)6W2iD1sNAZh>x$E?J{`9InApxG6915qzaK|PCy`&}0PMePgRet;Ru6oHdR9e^p z7h-fTOCCw~y`jHt(|^lwSOpY6iy(jJ_9H+x9P##!+}USV^EH$?+xWC4BSp7rE1`=~ z%xl}iI@QuHEBbP`$0%iBIj^0bDo%=Vz8OC-axaqeau($?F7F+>V@l9D^D_7jqh_^E z{DU3iQYWEcPK=u-rg9&K1yKrFpMG_3HwBJUdcBZ(>77hApIS_e9A9?#01<=S27CY0Ct6J9h` zi9%FLNfH&m<}BBJ^;smNfL)z(op6!a_$eU>V6kvp}qt^tJ!d58@H?>abJRGF=v+XB^|c{EvlXQY$p9ccxkuj z-3kh7@cFbEkhk0;$I!;%z#(-y>8NGy;Y(H7wz`PJRC-BVP#(j#?n>l9?nyci%&TK+ z^_#rdEa&UXTj7jVn1YL2Ba3ED>^8xY zaBkEL8tU8N_kLTL@wMdthLZ5S<&EIg9Fbvb(eS&C%m zen3X#s#CB_6HjkvA;`BCoEF(K^U9-5)EXI#CYT9%xi;8AQvs6GX`STDq7Xt;=r*tV zQS27eQ+?L3Zsk<|tiI#uG=i-SM##~YTR53Trl9iF!of+CGPaTu>5kIY>sr+7da#DYwm1Yv|h~~ge?qgC> zgxAl;-l&W$!&v?yRm37NJwbxpx6? z@X+1jQCH%azZ6$0VC~GYocz*5_RO@9X2I`naXxQXzgsrZ2UtJ3g4i4nTPbaooI=^R z5l3lAqpKn}Pnzk>UZ<;D5OpG*wM3F_er8A@4mO9N>{NM!qLgzfE<*LHM~9+@!DZYS zAth}4!FWSAO}aT<$hVAjF~vPZnt;q`YDBiqjb z8}|(|XU3M`KdlEt4n}=y-EvS0({Z`Mh8io%st+WrdXPrK*a?jm5<(~*p@~)>TMb(A z&Vd?%artJSZ3S|K8P_Ly9ql6?g#>k>-d>$@TcYo#T8k%W`N)_x|fTSVM-VxJj7uN zle4e$h1LdP=Bv;sc|4r5Sp-`O~>E_!WegYSQ86l~!D_t1dZq zo$ZTa9lHE7tRRCD>VM*BN*@yK;=HDFPIs_LzMn@oguheCV^s8k==jiT*9r6TQ3}mK z*_(vhC6!=oTxIi{4G=oSK+?>d_Gg(CzE@$vR&z^O z+U9f!Qy-_g1RAvQzgb-H)(35xSD#!8;#>yBb0hWZdb?tR-NnL8nB_=sC#zyRBD-I? z4^Z&R+Nv zD#eXNC9zNIP3&w>8v?E`{bhx|$xANu2AjtSQNi+_!jTm>rf9 z|8Y`lP=y)@no(#~U;&k}cweM2+oUyW@2`?G8Pe2-O6qR_6!>Jap z`ce|JwW0d%97nXW+jk_O)3fEt$o4@qFVDPzSBn7-wvK=hGmmA$Kkn+Y@lW=BKeP1# zF;txgx_rNdncH#h=cWshvf{m&Ro7%1v%ED|qOCrfOj528a?{LXe|wK_^RE9KfS(Kf z33&|zsWUX_!+Cn0CgL#eMoCI$3*%uA!!?lCZ*cc`>?sXcGAlYv>#PL9j3`@Q)l)OZ z4r_RPi_f;~ZtQ!vyWYHeT}^vTf$s&z>}Y5fI-QJykVWH3`g_ue9H^18*pAlxZD$ z)|wuEQWFMh%{K~mf~O)TsCbXs@^Nm!Q47(Q%)clJ#Dei<{p1EZ))3cC0sCNRzI; zi%_Zf1qM;4&Z0DK`k~=UeWVHrgTyI~JqRMQU%q6bsY;MXhCEX(vph;w!uhTHp`AUt zAH#v-7AdM(?qTFw4%+mVI_LI0bjjY6>6N(rp>KuRt%NOu5zj!;Cg*3onhCN`#dDBbqP^Y4% zxNWGWCr_!l)788jBd$tOmJ*6vSGd#IoNs5(8E+jWV&pjJ9Z%LLg8^SR+b$4grI6uD z8&~I3%X8J!{H5PXLyXd=`#B-rK)r(Kn!aml*63LE;i}p)0N$BQDY?*pto{iMbY|E!K@@Ss6pXhonmkuV`rXXp;ZJ5EBbq}*mkztq1M0JW#IEIPeke?&K42L^%a z_bY56t}N2^6KlZP6#I$;cI`NM1iaYn8T!1xG}bo0%Nb2}9mT$kd!eH7A$Z}NzypyX z<^V;+ga+W>$=2-aP%b=6{%fM;(Vo^Xi>(|9ZCwM zz-R#|*ZlewJg3?8(Sl#dYwP<1GC|J{<`Az{wgVwxKk8B4dTHu4@kN)^FBd0TZp(#^ zSr;KxjGc;p`tqo;s?sMmRBb@Sti-9@WQA@%7y5p42GXhQ;e7pa>u@&(v6YNS_(^gq zFsUVYvWM3DEnP6GC+-c;(PJ8mrpR| zyzjtaVkdRQ4oaI*cs7TIRkBz$9(WibAP==5AMr8#2`Kno87vmkmYIzdtV9~NcB$&# zH-uDUqqRuAbq7CT)F<2PAnV>Yge0ecxMpANRNY}3GHI3RZwqf#({nV&$6qf6dqw<6 z7){f@?O*Bdp}{$Ak#Z-C2vUq*!9`JG%)n}?uiA#&ZQ5&-U16y7I{SrjOW=m$CNp^~ z$LeOMpz6?ZpT##|ssVM;*+w)^ocz$ydp^4DwqS#U%N+~fJ}-nSM@BRLQv!2D?Hvv_ zRttANbm9-R5#$kN~RAqX2j8BlwN>#qrQL|+4iECsw3}=I|g|J zXyN0*5^CK!(9F0TqYpn@MHU2q4j=ro5?*Av9y5U#v=Vq1%bf@ z?XPvFZYN9ED?X(#jz7A)k|r-dUZw4vbjOMe#e2LqB(7?PpCc;yF3ieqUy0g&ne2`i=MFm2rKB}3z!PWDy<@Uu?{!@F$@(#(e%pv#p38|DxG8%wQZl6A zSrOHjhE5YeOH)#dp&cL{$?n3jZc2|}UGJj`qHZ_b8Kqw*CGm?7w>^|*zUFtJ7dv2 zGmEA}C(#{B8H>HOh>h7^AtxLAmrO#6`}7+-$N+9O*alrj5hEMy3-@>^53`v4;xQY! zssH&A<1ykJ;xKd_uXhOIx1=IDQ^=Ug-txhOHPVuu+F}n<4d)F{vYBqd?#NFeuvDVX zMkC*)a87L+au#H2cQ{!aXW8LD(@y-F^!7;STX~wt3`X<)*jElhkWuYFHNX8D zh|Gb0a|8#n!jr8a?NX>%{>`Zxw;wV&yu4@SpRKJT zzMf!*qYH^pheI+(WUo*($LnT~10{_ncfFd1g-KsZx3t^0@Q!szJqyXMn9{|bf+FWv zLt0yoMh+n z>guJN8@GuJYei|SAwQC89T3P=lMENu%dUsGI`_MdeX)amw!4G8ep%rz;`}hxx(ih} zA;5(72t;NK=}h>=i<+vQT4-*ws>aRhvF@G_vxq9Hv-!mA16DKSM1QQc?Uyu+W=`SL zlb>{T2dwWcBTXKOe!Fz6k(<+x1pND0lhJqPmynV;P@hP7^cL_VH{L@J^>%xriQ!uN zZ(Y%>A!Mw0OcDPTx_rtP#xK=ikM`BCQ5Fs!l|tDtLM*mmVWyq#W*ng?I}$S*NPz=Gs-)|iHq zI7NpievsrC!C`u3&jSlxEN6?IjDad>rcoW_&i!;V{Ef(KF@s#o82$9L$O!)?+V=c0R1i zrwk?v;j7pM+An9Ne4^v2jZ2`Ra3JrWAg$k!rVu(ij`@@G=ckg%?$cYeAuFArm6CV9 z6S`+i{M5;C1aZ{1)75sn!)AXZ0oEI~u%e}d!6K#f`*rcIBx-DyZNyjag;+yxz`;@66816XKC2Y|#J^o_vaFRV&P1Z?Xt zc(xr{1j;HPg@=@xLNZH;$^+-1BCn+LSK|t)EpoCJHoA-RWmCv4HvQgGRYVt0NKhdv zwqH@Jv>NMLl~H9xJ=#bu)e2GZbG(=;3UOn6i8@kXEdg)`;vqIE{*mgUVWa0h7b#_6 z*myXnMcE%uzo>50d`(JuanHl^&C2wH%1|A}YA@q4KTyThs!g%Ut2SuH8&>NV`#|fT zSY0|Zo{c@>RmhyjWOXTu3zv4GRP0-TiFa+ zceWuXPrjG-7cH^vBwsL*#OzeJKs9&C zfzh9qd6>e9m?}o@rr~w?TVSTW`}r8SifRep70)9yf{=0!I-`hpz{xjq#Zf%P18pd( zTsW&z{NNWQsPwWV_yi52{kLUG93q18+#=Y5M(v^_&y4_^ak|x_HZ|FZfvqfb{036< zbdCuQdK`WsMEy`Pjax`rUzx~%KNfX%O~_gZjw=R>9%l=drr4*D-RdH}zr(}ZC#lLk z6>fvAt}|uD%@~A3W>YrPQhvjWoi4w?=fTSw&>G zdEfu5=6b(bDWEg6oaTyjn$D9Pe8jNfoVJNQD`SS{uoqGXAVKaf8pNlf=Lzpuc!CE~ zbk|*}o=vqwmk3iP^hyOe?K#L#c=_FUXs#aJsY@ul|85RWnK5?y7KXH6<6XPTCySp| zOLy>s;K3>~QgI1$yc;D>30vRXrXmKy$2J*IcG-|g5=b(ZtO87;Ou^wJK~<)B+OaGY8aFs^;Y$4>L2e#<}- zid1&)(nYeYYk#!4I19RBGtfKkB_*SXq0=Z#cKP(%wHs7n6TuOf54_p+k_U9I(vX51ayD)yGe66!~F$CT}Lw!s}T&Di^ovtlQ$GX1qKE z{|x4oR!l^_=@{hmnBM)d9VEVq@@)GSmqqU=Q*{ugZ#JEWl9Wl=E(e0f$QVy%7=Kja zk-444@eXIOPKDkKxbmlbN3M3!$a3`_Y`2=}@zz1KY=kinl4@CU#xSw`*;DCj3_7u} z1w9mtq3f+2+49lGVcAZNsAEf6s5S0}biI*47t$Q=XZ$MzH=ZmnoK33pyQ^}XW~gRi zk^7$3AE`CRE@TU*QP8eY5*hhThZw@_%@IFoq5C?7JVv!nbS+>j$3ANqI=UbHln=$9 z_9^;mTbusu=C6@vP|v@P{}xtEeyj8u4PTUD*16r`=ZDJg$w?%0_bH3rDz7$%pXuKf z4Vha3hMIf&Ko-}NqR65Z^7Z1%eS*#-nZPyKSFT`EIhPl^OVL^A5!&@;w^`jq(Y`;k zN&-bn`=k}NL0olBX3$(0kkgGy9tFwYp`Ww$n~Mgd(!$3F?5sG|0?QljoQ`@1%Qd{@ z?tL;fpZm3ewbWC&K0?Mm6Zd6Z$1Nw=1*tLhYmaFY}Wid6sB_>8wzhE6^D1*pJB6jl#PSF5?pW}8$9 z@C$z$%Zsv)&y;rwVnA42-Z^=6Eavta38G-;3XTA0ME121p&OqXaRpruty^&*wpsRv zRopiySY8aoNzmol_BGkr<}qy1hH7#f0%F>xgtg z%D2FD216tjNu5NZQw%EH;)oIm-Ux2etz4l$ZBD#kO>PRPfw~8k__G=*#d@li;gt(J zWq6!WV+dz!?~2P>2Bh0>FR-Z-Lzz6K{%a=vW#u@+>sbzLN$ok&>Wkhucy7K}(6f`z zR#{iR^jkAInlIVU5gR|BCcW_D7m4D7hqU=yDSw}AvdeQekLQo`zxsbSA_{BEx3IQ# z=_Co~YQhiM(NKC=T9a&lq=+0zEGs?gHd(<~f1k{b7hh%^MI2(-mn#bjID6T(z7an; z&Izx6%G;VWj@8MWwJ@g==I|s!`j9ovbKdX(i>>1oL~(aJhYec1JsC|Lq-+mta$|ie z0Z!4al<{k0$dLkDB#ZxwP0CV{=w4N&`dePp@&(^MqV&Pz}f{`oC zJb!^LU^L-94aYg36slc|q*R=QSy#vuhG+Fu(8XEF`o#10J02(OR9h38f|{Y(s8;Y- zR{6q3<%@f;*QZQ<=-uL*h^Ucd-j=IQ`lBYOFyga{_YfooEBuV+Z+<2OU(xZbvTDvV zFZA=Fh0YV%8w%|+HKzA1%WH=vlI{sFEa9qUl9LmVsXqt%z=^4)PXF~jRzXb}4#kDS z_wg2Fhs#`Ae9@9g(MiR{L9b&{YB1uY`vHAZAG2&}PreE*qD6Sn++yn`S1f_0z zi|fp{=B-Gj`$)U)W#i;)9eRH_-cl^#INiT5MG$>%7y_f**qHnTe)CUHZWwYq)4ar9 ztw5H&<2KOkiMPTNJyA$8ow4{oRvbG=r&aHmgnNu&RwCmc&duSxM){@Fn9V7YM zxgZ-LZ@J+`4`Ik8BY#lx+eO+EzW@h4ATXh=bcOavR;-zs_!?{UXw?NL$;+AeEtU`o z`)aB8mD%OTy^8K?rkvER_8@r~8Nu1v5d5_`2PoK?LO;@)nho)GE<76+sP-^HUzWPA z012h^C;gv@{fd~LNi*NDzVe!>Dh@G)u;cBGnYL( z6<)Y*!~DtS6rwSUq8IK}k*;yE>U|4FKw;PoqC$f#J5#obaM;q2@Sm~khGHw_`%N5n z5wCW3f^3-*Bq8(Phkvt(Et!h54xX`<)%F-c--f=o-0}|76~bE$?~Quj^r64S2+gxi zxag$E)gd*G2q%1o_}dw!jpL^M#m&&xTh>ID?o-nfH^l>Q;pL@6944AIF6rR>f(uw; za0UTxU2PE2%a=XNJI4|Ktp?Fwv#*MHbc>vn!nQSdo!^HY$Gt7knc&ceWIVAY_!*wxGrC(cF|FoB0E=!EOPW3q zVpYd}^=m3jKL{Q2iDneQ{9TFB_>^nn&7zy7Cq{yMHb$$n6!M{F%GCR^Jd(E! zy^RSv*E7SAe9k~GWu)@^fojUAH!OZoL^Xmsnwda

2*AYVM!U5orIrTmsJ4LO@xS zK@nS)5p`S$T+BHGFHpyX=dE}ExbrZdy(B)@&`EzAQg*$DXi&XSmTf^Y+n!Rq>jmh? z9=NTjC-{O0r7j=V;~zKc9N9PCvx4%i=2x#FtPs37rc4>|5-VWGngxWc~gY$O#5Uq)Db?h&* z$zmCc#izQ}_>fQd{#|eRiwUSAPu;lelVX~EeVX}XknAt4!#zY$VE8^ zFS^45+3loa)ZTJB`td72j+SpMQ6h;RAn1zxZ(*U+F^x zNR`|qo20btMuJ3k=CA&w051K^!Qa$V<(+b$j+%D4CBiITuNTEj=sa^ISe3-(Yy^Co zz0cUVplGw#q&*uC<%;W}7Ga!fdN>x`ZXVNHsghDX!=Q{(7ABs*89Q)YJ?ADd2#9j# z7%nNWRUNBzS;`INJn205-Cf=J7*$aSylGqH=e zFIrPP8V%k?xrXoLJg><`yc$*)2j>{|v3-+v&7e77jmI36tEm|dyC=GiZ}ctj=%SV) zavOw+uZblY=q{-wvA){0KZj=4zxBFwZ`soQ%VLhq0 z&n-{Pb}xE3PFy>Fy?#;FRax ziUl@an^wnN`Xsl8h?|KRQiF?ydJ@&G=EcxM%#xcI_8hVZW*(d8-6aQUX4<-0eCA#} zteDjv=kS;JD!L(};nO_AG1=LW&(F+Sp`@Oq$Ydt-uKk6vRp(PJ(AnD&2UQsrNiJ=4h zO4h)L-Cif(7(1#hU~3iI^9CvgSsdDjrFT-g7w?i zpNaKp66z&V2osKvKX?TU=lYW$K|GyH@rN{1(TlR>v7cnIGEd_9IkriNP5FX6P`nkG zSleINm?`xF=fY=X(T6}!aWe5me2lam z%n#V9_0o2~&}?4tT>|k3%TQFpkrOn?^y?y(#qw*~aajAYuq*b;Mraj_ic?Z9mcy5$G8|)Jh>h?Y)e9mBi|8DlnUFlq6;wSw*MKzJEci!NHFRQjeS? zFbZcPsXUZ@vG$ItvW`Kf_sLhK=sJIp+@I$JvuDL`6*e@3NF-Be)6IT_J{9k25Y1pP zN?}9yAj2v4>YCRhre%+RaqFSMwv||Y^cKfw^K3^S{2f|PKZsD=fQr|jq$Mu}`(CfO z`AjOi7_of6IFp}*0nN(pCL^4+g&W|6gF5^Gp23K)yPFmCC$t~S>;&`CiPN=|v`X<) z$O2Zvm^*B{fUAC9lOZ_;A{6hTcfE&kw@Kpg9IYY@Zw9aax6cf+L;XOXlnkyZvWTtC zHZ9hXi$ya8-xgJ(wkm^%A`YB>S$+TO^UMQganWrgI6)cIKMVozB;tU=ZCx}HF1dam zZ~;9VzBhut)(LW*Xou#0R4CGI;LP|mK8psdh4{g`rLhpVr!sP~|MG3`EXyf?|BEk;q;uN97{X5vNMabR= zGZ|@G#yZE^-EpE}YDJG4YHG|>ACE?U^@ceS;+TQ$#H9-MXNcm-dHKD1wZ>-0v!!q@ z3Oj!oUP?|APs1KiF+0uJccKV*5D1eMz=7-ZufJ#FaA|s>pqbuD@Vx~y5advN%$gHd zN{6`$A%e@nI)-S8ZvK*^j!`&VvVI+CG1-ts#1cHPRE9DzPE+A&tvf1@hEz%^>qlap z!}J^Vx7j-kWWN|kRU+2rA#bx`)KR30=G0JZr;l(Adk|t<0yrqpQ94){DNyvVpEEab zB&u9^*UzGjZrt7*lJZ4VfrgUxzNT;wG=t2#pY86~hdx0^P`If)mN-K)mAW4^5WJ~` z6XIPXZxnJuTSlC4Y(tAHLNb#TEo?Z+-ExX^mh{#FH}ge*9Kr(hi5L|6Vmmdq_)KHV zztF7JuqKJ(ORY3{S_Q)If-UNkOwUZ9eEs^J5+aWX%o99TG$m&?0lonm4lotnuLM^5hCMpw8sEAU-8=)~Kt z8>bN`M$9$@Jf=L&*$k?H3EzT~iKqv{e!3b>JmslxNmG%=O{JOZ#)3`U4l$$cV`Ga> zS-W&J2PxRrFV)Tbj(aFKFk{xrQQ^5$c9GDA%((dZ6?2bBe|;1W>bG_GdZ3}QydoZX z1X{o-%~}y0M9D;0o|BX|?LrcLRe0l&{@8mqhv%pzj(CTVqJrik=U8X{i6X);Cm>{| zqk?Lh*-$ElUlGqoFRLHUHQU}2+QA^rabQOM%K1yn$4{au z+U`Sfh01g@Y#Mjw60*B6q6h(Yj{5z04)^vb8VPqCub_$@#A9xo@=#tVY)OfegC{jNW*cwbeafwK#fN9O>#$UfFUqEz6qNK>82A9uH|9! z`kPzC+cD#ayBAx7SHIt{zJJ8;_GO5)cH?S(aLMrqiz~94J==I;sJ5BJ%#_+#yF&*l z4|Y1x5Mv5@+R)E}BJ+ijE{-eU2QzN3DKmMsLT(gH8m4?ZiLyJEwxqk_HXY(7ux$;z zA;)v!;7bWI_aRCglc>yNCyy7H=J_-U*2=hC5_;lczj`m$OWF^LBa}%+K1{zQHK_Nn zdl5uMSL$49@nsPV3C13=3&(bS%RD~``lS{3LpePADOWY^f~G+nz5-pVQ#vU7BAD56 zHAsDjW}zY`?V`ALuP!7d6t&Y`{;EDY=E^;?1k<6T3#)2_=kuv@*D$_)vVkB4Tvawa>M-OAZ%7zqrku>b6P!LB9K3A>E=rOxHcqp-N3S=$v9V8z ze`4+fvk=2==Ng|5NKx_eYFJDcLCU*`*YT!!&DW%gUTYlAxR~%KW{vs*Y0d~mCAOfK zxQ5GbdY^>htKJ?`MBG?nf0Z0)+^>nQwYu=e+^QT48R~L%&1-Xx^pXn+`@36&=As2j z&E;J$F1mEYQ82ps%`wES6v@v%WKu0@zekLp1XrJ68=MAJnx62!4tm(s>ddEK zQm+Y5oHSJX^h-P=a57})D{if1K5o^%Gw!NMs z8(qkgT&qhV@e|6@jj`+^5^m_djj_rw&a?h~?@jeWhHu@Or!*6)C)Q;TSjd<>4{Ene zP;1N1X6i9FZ*I{s)7vaUZx_9iA=J|z^nC4>1#5$0s0C#Q*s^S|3{4N9QSl{`@Z=g7IQidCB2MP#Tcxt>~uZpd2lp!m$E}r^ug)C7Xg&OLneh(tQ>HY9yGx zIm4S^ZBOMV{APl>M^y!BNXUHZ;I#@%Ebi0}2Wqu#$#N zBp=KahT-%T6abz6zS)B;aV7pNhZaT_-z{G&hI}@Faxss zIozVflYLR`?$G2LvCjM8?AW-EhP3+p{Oz4c2=Zg4zx(IMF-PsWfw(^tyGtUL?uR3E zoaVlx6;S32Pu9!=q%+047yIbvWoR(y_5PD4_BovrVc`01zWPy}T(ubjy`Wsd{Xl6w z?YrKlxRpmf4tBd@(xL126}-J&Fzd;gi{TrwRV9!F9LypjD@qJH2qU8=`2#hsv6Z{m z-gM+XenGca*;)hcO70otQl>&poZJXh23FKM^jsqOkFGH6LZbL~`RQ81pX+gxgFt?v zoW`AsOdRWsX5V-l7JjXfIFI~27K*INqBIYv_5D4#a1OxwZ6r2yij(!Tr2}MePd1iO zLM4v|n8g%@JXYA+VF9db;;F_$Za@N?oqP^2&=QU5P20bP#(wg0dbyc@CPmeVuhe`B z1wf&eP)GuzizfB^0yFnc{(K2)>R-DYd}6tYf{I}%Rh#Jtt*_;lX)HZm|C(+>NJ#1w zn!3qtGup)7F4>Mc-rrM$MQ;4PQ07tDSD}8r;2{lJIbKn>rhZ_7uAeu26$qRS5WA1} zJ8h@NW`wlq z*WjD($Gd1cxeI)wPD`)Em`UTa&wf-lvAnDdKIGkizT;LxL@QJiLZD<$l7pbGQ*t%jMV^9X37JP>B%p|o3?o8(YUuUA3oV#L^7^w`P-dX}SA6<|tSzTAx) zwZ>N5#vu5w*si4`lcJb`MZz#rrVkKPRbdzTVQ$R18r)sRIn8!TT9Sr6dOC^^S{NpoHYL#}Z|FMo zPHjh(!guDb(mbcN%^O1kC1iPW6Pgndu4xITLxcM&#ZZGl-qmAPry4rE@{yAvGX;Zp z1a2nVV9drFCYQwE#&+3i+8)M34%OClA-GpZlcJ8OVMQ^*t3BZns-I{&-`$QmH!PU6 zpX98H8pGgM945Jq7#6LxboYYlT$bfsp7Z=gG7?FBKMw-1%4YhioY%xFo%=FO%P*0w8MpxIkri)rvK)LEZyb)hmJ zFdQqRl?o5&>@;ABo1HctbMZ=Jl4cRU$Bu zH1&c2$2xn{JcwO%?b_9kBC5`d&K$f%(;06!f9Gp zr0VAd`AXpO>3^%`y)LU)8Rkc;loMcozi{Kfa~o$DvqUaDgeqf}wwwIkq6`+TQ<)g# z7{+VK(q^NOq2GTTh;qK;vz4<;dB1FF!H2sXyJYV&_oF|}#>7p5+cOai8oN?4(x;mu zC$qzJADyzKy2{QBd&=D?9qUrWvi=^8up7X88fKVFn}J_BhngomS$pmU;7y!goBb%L z_oY*}IlvYTt+pUzFTi1_A(OnPH%dB%i+JSoE-bJ2d+^cR=I&q5Yy+;p@34Gx>4q?@ zt-Km*85QNsXj=8=tHH$9cON2*>5+C<7xfT!qX(2pmDAlL;u9UyPdNrwR_QI6XkT7fAu$pNDdl%VLL^SW zk3o!dP{nAT#rT0#646u(2S<4%R~P0RQP64@|3usycJ>=InwYwVMI{?a$!wfIw4tVK zq@YstA-OLcp4y9?X%My`dhQ|q_VUayIVhD@P!(6^O1IR1HvK^eC>|+M=+mgd=)I*= zh?8g38$C9w76@s~Ax_{WDmlEXlsTH9g|^HjJn5D1P3|Gnqu#dVhmO_jE~M}eeGYQq zCUbSb=Z)st=*#L<%=h+>le-RX3I@eyef9*aEWOj!}$~fGPeMI>jmnAxUN>$9V~KxXnklJE1>{ zK)5i4hyAjUAXuFry5~Nfz3K*))?5i>G-T0+G#p3xqqlX=FM*@2CyTyCDc@7fB+0Gi zRimsA;TY4T=~;yxTA^xA8CHZLym{i`t|0o|=;CVXGCGXUPhd(gh`E~QX+6ei;Z*Cby*`qH@Q!@JeIbkUnf%lX{mCb`i#rQzz z4*@O#7{1G@T5qaiV%$Fo@3_?CMKfo|JHwc^k$&LVl{FV~O7f?#XKUKQuR%XV#0E>q za5ph8yWSzJg_XwEYBA%ZVGoT~>{zx?G{XL5iGGyAZ1l*YEV!_GhO5r*<)?`v-786$opa45gmfQkoGYlulgs3=T@Z2B!?-Z%&`fm zhygsMJkrCH7sc%J!zoxY&dWFT%7={{cc{?~XH$e}ipt}D%&1X2D(VkPY|rHDSE>rLT3YU{ zFjq5g=e)H`M+DSahgj+Keix5NwC*%r9n4XtAE>v1-4HarX-h{RZHHG!Ln9pq_PNaY zX&d>*Wt*0NMu|8(EG(U%|H!iDE%&|UagEF!<}2kp)oaI^y39qGFi0n)>^d_t6j*c< zT9d=mX5lBm7shZshBW)`urXG?q1cVOri`O6KlWGJS*S$($CvGlFoZ=axy&Z2J<615 zm`s}w9h8eQ!A{jtxIf>mg7jW>(K;d2eXJW+@;nSXe6MQBw9kBn# zB1E6c8#0(r->86~jfTRv8vSjYLu1-sl0JQ*ThG zTnZKwJysknQ=UI4X38?+YHo`*KoymU8r;~&PR*usFu!{+{Xjk=RiG?36xzYP(ZHPJ zQL-T5eUpsZL~gfZk7)|x3!NrF9>u;7S>?Y3f@QWjZo=^)KbZQ*8zh^=3^s?1On5Ur zsbB|AD45CcYRH8mm`V4h-@{hawUqm~I8;ThLZx_}3ZTubug}fPy(N-7i{;&ea~bIc zQ4Hy(Sr)@U2$KSb`lo)PS8Vj7AJ<3yUAt?jWbng?ZQ>2TO07xbrf_S;jMasq>uP;D zEA9gi7BxLeQIS^#?#+587tc+#;G?;cboP^N8|U7>juGIUK!aNgZ)we@ves(T(Yt}# zdpk1cpY@MO_s%u`Qv{5a`6dD|VO6Mp7MGk9lt-bvokQcUO;}B{ zuJjy!Y0p#PPs7b>$)MAQOdqo6i%=sj85>b8Y0+*wSKKa5sR=?D)WFn)^T|@UgC6`F z9Yr121Y@lkE<=T(axz#TTvQtD%eAAjFV)E_Q7SXDe}0g^pm^MV-6b-~R~wI<`xPEO zB|Y0+i9%liP5OSVtdVH~0#YL4=%EKp9ClVjOc2V=vTjX9uC+@@@MDwQ>yX?7mP4YO zWY%ij)JEotTz6;gWwZgs@TDlmohlM+g~{w=94|ZqPdZt_iK22Q>v}Vfmeou#QO>F> zl~(IM_tADF${aFnx$AWgO2*ObL3je6KTOM?w*wd;0dL0ryp-yu_ukN`yrt`Pycj+i zm5IXPv_;$1_U6S|a;N5Jc{p|AfGV5{bSHqEX98|aXfg-JpP3Yv!`?tXLXTB{*iJ}Q z&RV~7uKJmX9Gh(uH}&^v4iK9C0lOVq=)QhZ!DSSvvJA4#M?$QD^>OpZk1b3-6^MzW zm>aEx%kB74Tau2qot7Fdnw*V>6jB8pql@J)j61)Gk^GumT6l6P z81&pwC>kU2h}IqJ-ZVQZxg@|$BWk8VVUA>0@2)3o^QyuMRbXzry%|-#=xF_RfyLm7}&o zz487iHC|+q%&)0u8L(n2{iz|k66N>}xbKCQh0ek*I`LZ8=T}v;?8!14!Bj;VEb}n! z1*61VZ*wdDo!%yF>&^B;I(&Yr(}a(W+?v-tCU99m)`LXl@2~dCEj&agsfz>ZJ}Ogc z{#v;+b;X_}pImBGgPfO}&6Odl0kkx!|%~ZyyRKtJr zaYKBWL$|lb^r7-gxKEJK!Ja?&n9j|DzokT`k`;=-0>+%^iLz;sPWwjWZqRg!XnsP_ z7i-3B#`S>#-MKZMP(vXeiJ0*IOIcNdRP*}HMDzS?Wa4aiVE9i}Bz^SUFmDRmvl->r zu#&&gB1DY1{jlW_y{zxk&gLUOHvZ}8Ks35FTLv09x7J+wZVma->Sm3}bxU@4{&+P{ z-xYYuF~=MlSKni9r)$qbipq0_Fz118ZK(PdAzldyR=}F2U7H}>>H@U4qBC6d@? zrl ztdBA8A((7r-KttS8dPHsMM2G(W|cl#uYM~E?Gp}^4BtH@=I{Yd)PT}N#}*Oy;JvBX z6Yxo&-XLRCtnqx~J{OW0Fi`&H$nu2?T=*d}6idZTxC*I(vCIn*%S9f3DJl)-4Gn(T zR$kaXWoLm@@|;~=&(ufwr%=I1I7H#zK3K8$7~o|{ zqi@u>M3&fCyw3vGnLyJl;3Y5C8pd`X|r5DLFr3!*wn-*~IB5ZrN^8NhX}F zT_7P$OF;SiVQJ(~+6~1D!Jim*16yMqDiPAlK?HBHcD%E?)3US~4)M4g0_^&Aqw#nUXM#;?;8uxMo4swT{K-ZY|07R_+UWj{Oz-9 z;;XQt(sW0*aVGAvvhTHt=G7FkRl+QcuSz?6zd&fqs4!hqN!QMvtLUc9QzE5EfQ4_g z=t;*;{}~`p7f|YMtD5Dv&tP57J1c361lx5FdPVP<0GqR#33kLQ7caxyp72FCe}Bt> zl+cjGXWg3bx|)hGv6gTF4C|eicSEzhtMXJPi&}3uKzT|}N)KI#r}}KQrDH-xxIp`? z2{eNm;S4Y6iT0yfe4JlpX?4T`QArwAT2woJYo|rYu8td1e@vcoeBl}JsNBS=bfq|a z*N4u1k>$c&6ACSw%joBh-Cm!EcLQm(@wzEQr{2WybNF!%xPfK^!w=}$3)GC;5YDIoV~ZD_8Z(G z8tdMHy1C2H{nmo-?*dobl3rh7=}d09&_IOz9`n>@p;T%Oh9<+8(CX__uVJK_9nvi_ zAQ!F%|Nib_9DWlWA_#eyP~4k29S&G~#=nb+KG#kb1~sqnkrRqbM9O0#@i@Cs)^xneHb<0gHu!8~u#jDBf+v5pCFYE72vhU4P&xXdN5Dl@!c0dB*!)TRW6y z`P95Wi7UoM1KAkCJy!TjZc+jFj`A2Q2b5Me_}@b+Xb4rp zQ_QFMHGQYqdaK`~a5GJ$Gd@o1vR(*>rRs@?Gnd3*S)3jYCsX>5IucU6qmh9k=&X|{ zFy%#LXT%~fahQxPX`InM$Bs}@`l&Bpw&BWYJ(_Wc^s90SPYA&G1LhI75z+(F{PZK#6Y#&1XLKFjZuEFYS+xTqg1YQ+?Z^zH6pXs zhKE&lv)*-u#jUM1vyBr{54_(n6#Bc%;G^Te?b?%Bf*iCe6ZD#osdq9rQp6{)Dj@`J$o_x|N41~wPVx*Ey$UYU0ydY+*kI>X#U8d)miT(BJ_0dv6Oa`=JaE?-%4 zlT=>G>+{H3&@J5M?abQ^r&jTK#!;Q}O}@tpO*c`He|%fKUg~BeQD4+Oi5EffjE~K!bh<6yjK&Ocjq7H`@-oYz^18AIuqWEM1)Po7M0qF9U|iWbZY8WM?Avz)pm2;1K^l zkOD6^RfV`cW&&4QUeY~j^y?LWv~+}wpty#U5NAuVcSk1E_~RVoN)s}kMZmrgIRif_ zWvL+Y(oXs@5gNsH#w|4UqL4AmtImWo-DtEG{L)||x&5mTaGXSBp8U-(Fh)&SL;rSL zk2!B^5!;d}#tP2Wj6_Mgt4>V4+8xaR0f92)*MR(j=<2@oe_eY9{l^mDxXQB5wDp9Y zy;qnW`_NmI{1!fk(8gxC8;R zxsK(Mukzt}6{Qpt=G*&2aqx)dO~bQIY)6Qdk=d$C8m&ZT4Fa1L9u)_Sr8M+j$I5Fz zf|Zt(8K)(8C|>T>**dG$5Mb`A54F9cL%%EC>0IXKlakeYid}I%X#++}kxRHrfOYX5 zmG*t=IRA@_C9aM5OM@AWj~E_$6~+4=$;v+S^e4Y*x|h1cAV{zHuE+|No;OLt<+q40 zE)`~DDAV?g9utm%z5#3opn9~c2-t&sx4#`ju{5LVdV85njudbrz8N$pHO%Q+03<-$ zzov6;ol~uf^8w}~5N`kj>+kW7T=g~qt~x!f1;O*xCrUgQg^Z~wqh9YuDqKl4*RkQ= zl0%4L9ON_tXqI*7t(SoIeuq;b+p5sD@n!9KyA`OLSuRpcf>ZEerhKtwGSXbaQxx?K zErD+-$O9KqO}j#&mSW(`Qpph^9(P8LQe%IT4Vu68BR*)S}mUrpBfZ8;&#!2dzsd_Lszd0{a z$kmZZ33QfuD*j`#tEKBMv%G~R(4rTDS30T@J&Aj43+sHAC^Mg1F8C7Zkg*Q@elV4t z!<{Q9mV;9j7x;qI?O2FO@fo%GJ4pAPZc!aMPubeyZ>J20joInu{Ho7T z*WZ6AomfBB6TKh*rIKPE9|EQN<2ANkWv@DMs>i{of92x)uJmUkIiUg=u$cN(eo5ZySax|H zF4IcaPi%GMYm0m&nRbmsTi%U)?cOG@sh{&S;D+qXAb)4V1QFK-$MxG#DJ4hN*MFC; zRA(VirMVX#6A%0>4%MbudcNihKsidgtq`Bu0qX?WJK2tTt~9yzcGMO0Kt}u$)mo8g z!{{$)%vEpE@KlhWAWcuWxbtU^Gk7sItdWX^7#{yP9>p#Dm7w2Lw zlpo27#be}}H2HM5iw|z7{{RqPnqab25sC_2G{g!}+NS9VURHb2W{5G3n8WA5Z$Ige z=p|q^jLRlmwLxYymv4dG*c8Rk#YEy4MXeC{tZ#nzSEwEFOz!}JhD^FL-D38xHV2lV z1-*9sl2hD5{Wia&$&~o(Y$|)T@xFtw2$Ei{>5M5KpSZ}G{VnmhD?Y?A24+3a#%ai# zN`=FkvSJjWNt3eMi1SdCr@wsw7=h-=@!Jr_I>IZ((Js#c!hFF(2&*BH*5{A=tf0;? zB-!c6{FqxvCZvj&-i&~D4A^AEGyD0_fHMu{V~m4ZI!6teC*juJzC7Y z!cXwTTP+=~ULzxOc~{Uk%bl+1@jFV*D+x3374{IP+xqpPFrYy$hV}EHebpu<)@3FU zg>Dq9Ki-a_8w*&*^i4i}>rcXJqcu$v4O?n7{pX@hq#umXbgokUe{LZd;ubC=ty4J>B^Za-| zfTcjh8Nz?@E9z|I;wj07SWzpb%&UH6tQR~!Zq(Dm(8sRTjB1D%nUx_*G(P4U!ydTD zoNqA1_x%WhV8$8|VEDjFG|^{C1y9dy3RX2!d0n+bv0uXA%Fx5)xPd*5zD$cJwFAp= zQ#~Awb6-aGaEbHOcdAODclw%8a;QZ#o6YEb0e5_JS8r)%I1f4X8>kDn2otyNy(Es| zO&2U4E3CSLQR9D8nBT*HWjz5?K`XjtFMHCrmBy!*9x3>*#N#Xf z@#jc;Bfl1hUg~*)ewbdByUc1rNLJEOBKqO&IDrKp587FP1#_*;k z)TYGbm5_O=ACvC8(3oj0x@C_Fsk;ZkG|;eoJf# zYT#*W@o2%a4@TB1mRWqb*1A;LTF~p%dckeqq}!HOP)@7guzYF5mL;s>53-!ZEWR_q zD6&<2#Pj-1eqk}fiX7q#0a6$>{X?0Z>3Nm7=g(-w71toewpQyVOkT8b7{ymc3veC3 z%COhcRSZ|Q7w$3Kf>sAfX66XoW+9yK=qbYsTM?nb}a z7(94SmxS2HC9NHMX0IctnReviQ?by3Uz&Ps^F&9@^%2lTO6FLrkuyW?NTx|gaTpAC zZbX9wnq?TlN}!dxZLi|}%GM`enB^p*Zwie7>?!YFK=oM!``%OBs_0ooY-WF(aFZF@ z6v>@uz~Za(OF4Z11QZ(c*cGd|DQ5=*ilZTmvu-AaYVzUTQgXC0$P(2{s;5WG|K*hg z>+A-w%uCg)$gROn%ZKqY6OM2rL))aCx{<7Ud0XNv5xXGPApX6*&5VWMew@tnw?1;j zw!88`Ph?|j+Nzb#0wFBe{ULV4KFPsCQLH3Gq&Rio;H?rBwDkMTl35m^*jDY_bV||a z?5=y3l3Oj2ow5&{K0s5zyR+q9CM>`KsSJsD=9^eO=+PI3+~W5N*0o_sp3vtUPHh6{TMl^O5u%MLg2bEHAEMv8h{E%gotvpyt+))? za8rA+Zp$t%3g*v5o}j5&h^%=`*64+7w)N1|LR3fY3XFhK-Jt*djF@v(;Sf^!E$cm}L{GMOWt zQy+!Dz;fikAFzgJf5#!4gWI4db0JwrqPz8}vMDjWH!VPum^d0snRnBe&I6Qh;>8dJ ze^!x`OsVVHaeb3LE+q~=~wi&G(>cEb=ggE%omnofO z8X#zF-~iMT%`QXR;H&?2m$xkFo}f4h&p3fHgWCR-+EtEx2Wcu9{lsw z5nHeCnrzw5_LQkDIrT_lKB9!qNmK-kH0ub?n^T{o z*}c0EPL-avecCYE0O@@i)AWH~$l0OtC^|7_40c&qF^3_YSHQH+qDG^5C}z2!M2C1T zheA3^5<04(mA8Z5iiJ!&UvW2sHFn-=xO+7CwnjIlZ>aZjJRCsfSe5S4@+S|jD4>mc?W_rm&UEdFKT02teHFVlb+U` zM1hjQ1sB=t2mJsU%~frD+*KrxY!eCCi6xl%OP^{fK^KM`pZio^DnD|JLFaf?jhE)F zay$@klsVC`Q3~!-z4*;E7eSAN_Ec|1B$+c5Zm-!dbc}WkV;3~6*sq^6>?{B{{ z?u#k~1W@KsnG)rNgye%NsE60qyk>Q%QRnHFtKB5z^$A-2~{}sd?YLgFGK=~QKd%7-$N+uqfx5?s6o|QS9gn38D z4|60-)|GpU7TAeTK{6i-QMAUM3CHj-FTK9^dhy4z7X^4mr3WqpvOqP0j|Hk)a?YFq zlaAHnw3;nGy`=do^mxzCpqA%p7pkA1_Uv;ulfiO$165jku&x%~NH604ugFWC9ymhz z?M=WsGZhZZBJH7%w7q`U!#PddIG$u5WW=r_=i*4e($Y@Ja3fK?PfW-=KCw4t+y!i1 z1MC%9NJx)QvZmqOn;C6e*SN32BQ+*9ySVHKsJ(eOsh)lYM&EmNm8lscRCD;Fuv%yE z9RE%46ppXHX2bwbpE|D(hR~g|bgNY^5Mr{n=qf3NTpKh#=N;kBSA@*m8Wceb_(17)!ec0wp-YC%d zqKzQ`kiJ~KKCgk#4r|xO-gQN1CRO}dz7;au-a-&k5mAMm4PG^xfbgl2c(PCG&!Tss zjcSklG}zyhww`^QB$()@Bbw~P7xWb|&%fP(V@)nvTd*Z*dJRlhcCU^GNVDbleN4-^ z%w<(qH@bt5j7Fj$m$|voDEda%_>oucv=(U+J7WDLa)IX$>s8fiS*66e?wUoaEy}aF zSdM4OTsfw%o-I3e8ddQt%Jg0BUMLFouc7{eP=k5gpNg~jtrEH;w+atq2t!?^mGyy4 z3z@OVkMi!-KOLTP0<`QD+Z?#kL9cNpO2i1CVcxiUIZ8?Ix-;hHohhx2B?=;HV&*sr z{<5LKaNyaZT(~}S{YeE|p(JrJPxA*(9(@W+h_ykRG%l8~*-pZ8D|o*h(#%uA+^Q3= zt4G`%wF$H2>rNKQmJ+H!{zziW4qe5%yq5NEQ|2s=E@O+=h3D*CC~f z@iArlNJd`;`K-IN_sRy&l7w$sYzNz}C^0C8Pu*Vc;fH^Snh!6)lwwaF46K?1=sCm> z9;}z6M$<#fD5hHV2rxDNAy42@zk7dQJFJNri878lx|K|_^ZcRCTNkyNsmCT?6ImJa zOX1P&dm)ln*dgYckGQx-?IKw~Hftbw&5u$=6byOcMS`=ILUUB%%)WjR@Q;oKZOG@X z04G9E-;h)kqjfS&Y(b3QG15ttW*8WFFS$eHql2#5C=WZr(h8DINOMLjEuC!6n@*k+ zL8raKNNqje%A7$5zaxk-@g@3bf80uUdc4xhlZoomxo`G!$m;5v)4}abx^5on%bTfZ zCSLk?fPLUBeJQle^wZCJnh#)N3{uPBLZgJ=KU;}>_GFG5u^EVia<@Hd5sM$YavE<% z>k?CksH{pfXFR5hvt=jgUFBlA6yCc0t>Ltwk+DDxwx#iK(l_U+=6Sd){1Y#-f*<5_ zN3cI+lZed9d-gW_(wTGHl;h}n-pU<4R&^_u0=9=-g`V7a7{DCL3#Vgisz$d$tiVU~ zUIef5gU)9h@ye*y8dq7u@7Fpa?Tr@AeYLv@H|1-apaqKiC zz_rt#oxV<(FHg+K!%-T$HK_AF2sgQ+d_$pd*Ze)ga^M?I@K-31%3>tJoiDPFYSW~+ z*|11t%Dx(n>%H#ppgR5J4us5u@7fy@{@pnrM_4)WF?+&P+BWA__FY~=w|n=589+z> zQu4M)U<&c!b}Fn%^Xayu>8NlgPjL;(IR(w9g4A^bQJiN5nOA+jpM6I^X?Jl|Ju!jGztN^!mJ?INmWyn{i*~ zu8TR%B0Sg`2X;$nzx&GE!8r>C;cRl#noYiy?cDlEVSm^UC49B{3@JH6OZ#CDzNfb= zaC~5Bq&|SP?P|x9$DaJ~mj$e-ETu~_*Iw_TWcQC7&PisTQ6aJ1v97RK%cazziQwg~ zo^26ax_5_PG%3|?^PL9gk;-w-Bckl)@mVJ(;^de-6W7AfCSYa?VfJ0)ttgh;F14gp z3xEe!b=>_Z!yxmz_A#PcDP~Te7iGi``f&cyswBUT`PD9Us0>4>saw!x9J*)=Hx0Ju z_K#uE%C0eVJ@39f?~e(-Ut`lhVmCd4$#$DFCEGoNd}JB87kV^M`Yd%d(s;tIqxM}7 zQfut#N4H!3&EI}qMkpi1h{NMfYQzdVHIcYk|@CvK5Ba2-!-6 z4mYm};oFAq=X?5fS-f!PTKcB$y7s4NpBR!dbFSuRS;JBLz4bZ_zenFy#$JWEWhH;6 z#V)NhRA#S8tvdDyA*(w7O%>{1EQRHxjP-;5ZFsM1NOQyLH>0_}JJ+UaJYO&}BvNb7 z#DV$smcOt$Taz;Ay)_&{Y2|X z@VHnbYh$2sG!VtINmMO!mpFU-klGokBG@8(8{8|SU>`9asgG_@VJbA#9F~k_jI(_u zDrMw4HrrXeSK#T~h6X<#cK|b2WbIGcuN|}e`;pz>afP(>q_g9TW2R+Tn;QZfNry{9 zTq_Zc^=OoeYxMKv@Q`F3cYN`T_8R|;AMf=T!~WPCUh+c0ThIgv(C}HSfbs0y?qTcs z6)lAPv_uZy4&ZIOHxcBu!K=P z1Z`P6_gG!v9@^d&+q#a??~!Nj?Os`tVZD(aO`O&a3K{f+5vb6HRF00;)$rfIk@y%u zp=~|OS=2ud_goTjPsRb2hwv*aWyA^4Bt*WO;AsIrtKWm>xrXnZwStrZK^7)XMi_)TXW(Bk3;zUX$#+*rN&y}%Np&>0cDBqPmS)Q zmM#h%T#IpAckj+9nDh0)hXjFtDN(n!YQ`KcXYb0LSn^}9O?G|3?2OM#5O(_Tewm2^ z#N2HPJx+Cyj2R9JBiHPc?-nVxKS!44Vd*9$(w% z`?Z7bcsLqqo4uhWyJnk;q5ZXUknB91BRzGb9Fb6H`FU20muqvjxBNSCD)Di#A}VBa zBQh7@bQva+@a1uUI*0g%kJC$}GTxfU-mjR^@PxPFLoEXyh994xm8R5smstW%2@nW+ z$lfRt(Zk8~N8DJfb4a={oU3!EdjiwHR9?Vn!rHOLFusr6sNLn$i(zTzU6UsMYpY^Z zaixe)ValemKLYU)2%`PjmBR>gp+BVl0rU;S6#Tv9VHyOxEEo~$)a zSo%rq8IBvG6g<-!(Tfd~eFqAil!)Z9sXw!Z_6yPY0{vldcecRKp&fvccgv;qs*p_K zG8<`ccECglKM!T$HN6^${FJ4zkn^U9;cdC*&S#({LcC^|BNLh`hqLHf3E@qlr|KZc z8ml6pg&NGFSZvJqSyRSz&b`vMin!2dL!s2^Tz_MMm}Wu%)%k;rx~4OT=jz9S*}S4Qs2EA zHe9i)K@%KtP}zWfgc{}rFuHIVzO%l#Ehd3nnnhZD{OSnjmqTF87$xD9ErRN5OhtA&d5_Ui85`wjzi5bllb!qU(O9z~r$3u5l zs6Y$s*x&a4K}qUfvXFC9Wie>j;Ys@JMVJM)62B_F>wU~ZggI(m*VBu|A=S>~!bu~k z9e+cxHO0tBq1DKabc+sl=iaAtEY2|4=R_SX<&ZR!k??&q)1(Abvi3IV+;1CM2*s7Z zDY9Xut$XD(Ck~5U&74wgYw2;U)jU0kZ^{>`r10~Qdeu@=-#3+G_yuidk_I=NGpfIQ7eMqVuFup083UIM=%|p82Gr zIX_E6>im>Ga^TaBj${fh-tS6#zQEA(7%8&lmJ{@`1Nvx!HRjfWAn3?peydUCCRa{9 z%qxkPI3FruHD56S`wX7Ud2k{&_K4f8IEEiJH0|Nr&|VzP%vR>6en|MEAM#UYo$77m z7ZRx1OuOp3>BtdbaitbLsxtC|Sv?-pI~gy0{buihT&=-%Ux8Jd?<797(8(U`ELUp; z)7bg+acxyL?eAKt#Jl@H;G^XnjZ>>$`i+M6# z`qnKKzY--tLhlm~!IegH+|1zVZ_6^Tnop*smO~V}v=0m`<)X9CtmS>dQAS1>v71#cVuTUFeMW={_kB#|)BTT2uDy)Qy_?T(hqv zlfUa3VSrQCRGy3&G`~QsqNqwO$^uF}2_+Iox*wvoBC896t3)-@zeS_uPPACil;_^< zv>WzP#B5DCWE`C!4ZB^31byDPohJA991GCjO6aeyY`V*03U;=go3>CpjsL=-n#kaa zskIU23|+`VXNLKFTMvf%^ll{j!U!b-?MP(jLA_u!u`?L7PpiLA3hBT1ugJqMSBZO& zvIOmt7FIoaAsk9VNTlgY+CxiDCx5h{+dK0x8lT%4XQf}1?YlR_8vpeh=$XEDE9_VY znGV1BK`Fjb=)Ley+qi9U_g;IvV=Hfv&Hl0Ru6`)(zNiTD5MIiJv|30)N+uLd9 zkPK0|_V>)PAe!h{78as!nF7t#bTv&#!ne4)vP+&&TPIO)rNwm}lu4r$iFWw6%3*UN zIwV*cg=evxXWyYR;nmY=(Y31xIsH$3qucvR>*x7Dp8Qhlv$MSf9O7yY)7|a9t~qKV zg^b4$^V2@cgL@*tds?RFIjU>vb+9#qGse#%X&?Gr=%o+2t(dTt5Az3xAmASZEFbWp zTrpAMM>CTGB=S%)7;&6G(1LidHP%&Kh#LwG`0VdxxQ_G2W|1U|Y2>L8{hhfKf7(>Tb?TD4**P$Z&X zCsfX?O1fmzlX6j30lv4O`6G07zYLqrjduIAvOL~sAEOr5*H{@QvzBf-2(-QCSMa9B z6n0#=zY2;dY4km-FGN?q%lB6)4v_a6g4CfFSBGZ;$|_L0K*&Dt0%+j$jBU3}ZqfuC`%1Y42lxSJNAi$ z@J&*`?-xoF=FD5fKLg6EZ#_~_S|4A9u!PTgtZCWRs${a`2suJ}aC#JOPx_&L@zXHb zGsk=w!}HTrgdRnb$k#OkN??Wqp>3A2w-MX(*{tOJs+{koaaR?^J5BO=g0Gu#%`9|N zg^bC zi#zYPS3N-~$FUoTKWSPOw9S2`zFnxj8%W2bCyCFHaIz(RtDJ?d5$c1cz_@intRk1Sgvc*>u;my3a7~l_bTxN!BXtB>6tsB|j zZv)8U;SoAZU-M@qjx(lmjOC~U>!usRX&Aai&}_GA35PJzNlBxy*IHiQXU5Mzy+n&z z^>=s6X^2HMnAy+MN>V$z+d#1BM1Ct9*G&U^ha?K+uu+!yO|H4(zuTZ*`!%3P_75T- zuHOOQPd2Bb-o2EL9GwWGq}ow-?a8?DGTI3Hy8y$If9da<8W{KE`nu`-bg*E>l!a^F z@d=(-ooaO^jp&^#8Ru4gOrL;(+r?l96xdaIp6CxQ+c}k!E%lwWT12|XkJ`HQjv(~E zu$h@(`ZT3R6lrn3b&(8O+34kkP=ic_oN6x8SK3b+o@yRm+YkN)+Hqg++J%U-WscQ= zh4Ssy%6-{g!YJjE*@k6o&yhT7*Bq(ZV8?GPL0Q%X*G@2u;K!{aPr^j6fPR2A4&swqnf zl+{@p!S?EawO&j=)h8PX9=!-<%t+I9xQr1zAg043l>6f!*+7t5NzZ?t|@hb zktx0(kmY-v)WO5cr~L4k(!|SFc9M`C@G$P*&mmw(8;PIeM;9X$(d45A+pSkaz8b_r ztzJAI4oA;WnH%}zo*+wB^mcw4$tK$)(VZFbQg+Dwlx$q0q1)dJnc~$VWi)CbSu#~o zrcuz;NNxqk-=i&H8%T3dV;DcEpK_(QK(rFb@Dd1z-uZtAt~&Lw%N`!=&XbeHFt0O% zqXl%@$1RhpZmo3?U}!E9X)L@##5S}})Q~?1snuZ+_i0)9BR6X4$1%M?{0c+BlT|ju zKYaBtixTwqfCj~RsI_I9S1tpRy&`He6ya2G>w5z!;qU??Dj0A3`<51vB0_JZn9P(e z!L^M9Hzfo=+LqhSRFBZS;DyyVR*%cY)Y$AmnkBUy@x?!A^edMt!?F#l667X+O4j!p zho0B`or*1ZQSM!154kH)AyXfh;-*MMa|a(+xteSefW~F&pK($O!R-|D$C4N>hO&57 zP^gH#zUIAK*pgilsKI(}$3*PFaD2rF#5`ORB1FnNh!{R*(-4cf?JFDpA|rA0wbH2! z=(P38;O8SKmEl<-6BVqv2ePIBqj)EK=wnc5Z^=649e4sFNsYjMuv@LgIi8qA?^m8X zH8B;%cs}}4BMwh?CW`Lxo z975Y_WnqFinh)G|6wl5vyF`f#{Oj;3ShOqUd&8jJ&td#Vd}#0llm<>@a@@gg&{r)7 z6F24S*6xLYm7>t=v7-2pq}T^?s*zM-|9rIJGGn2!HGpsx3s;|or0(PB(|^wm?q78< z749ySx=mV}Cl4PE*x{$YsU*rtu`&~=XFoLa@0UEyZ`f?u`ZWq>-!wGxssX!hi`DvJ z!eVbk+Mu<*R}0B8-nyrER> zfM8KoAW#=?J>(uuSV3{iJiT0KUk=2L*IR(MNun;_VdCn81cuhoyUv&ySII?Uy!Mwfoq z)|(R2(J!IX*S09An8C)FG`CeFsyVuWmZo+81WUk#n#UYNlJ$13JuTP- zeHvGtoTrk$EC;0kEcILe_o!%%vdOP)SPM1nDW%}FH6>dUe83V!{Fso$m+3PjUh_UB z_JBM8vyAf{A9OP~*0a47Qk1XU*9?n-9(UP4;9Nf(sywRHU89C-MvH8v zQK23Fx@j|PQDfTS?@g`+fF!t;qxw9_N=qzJ0q~w(RlznbGkVf>GDPK&vk71zG~@)4 zHon`6)hC*}%+3Jv$@#`W$TU{}ldZLag>mfnnf%PKcJ{ouXDPo#$E>^dsXNN2vgZA9 z`a`nY&+r`c6-8>Z%=QBGhx4{2lt*`7;{IEuB3w15^lRTcSW$MZU$i`Xt)Da+Mm{Bo4YKbsr9xz zL|3u565LwRn~?AI`9`4f*B8a{Abk5V?rm+s2gf3?4E#c|cZ95Akw&z%XwAhS9UAX* z&8fnepyP0XpyE=DYSkN;Yq_~0#p|Qf_34o;BZ+OMdt3zhtSWtXWx*6*ZJss~K9W&& zl0C#5w$(|;b%XwO0>uNo?nWqP&l8&bC}z*F`{`7>Y^=CWpmvNs-F-)o`4}`U_w+W- z%oM`Z-xR^hmZ*Hrg7LNsB3G)B$p(>hIe;6?N>e3*^eFgEUBijQ9uLN5@ph&v@Qi1l z&FcO!W?c_?rb*t5cNRlZV|$iLb9_1#dHuZ#sh3uC4=0*LO`j#+i8+J!yBj?>x{SVP z(U*3rnjvhL6^uCheJ2(sLCK|cnz6X73R?R@?OorF^E5;FU?*q#$R7JOG$l0~InvBJhDm?SY;#4Z$R*E~F{-e=iS|81M+-!~X}*>^hBDtZY-L+)Mrpa;*|ECMRUvm1I`Og1 zpFt=^Y=tM-4%L{`K4DZRf*_zM;%V-Atq{(|aq#C@G%~(v((mL~K=O@go0=O@-7%g9IyL6tj)A7qY6WGQ=frM1dg!r|e)Gi!v6SS@-{;2CipExb5z(vNZ3vC?h z-V%{jCM5H+^!^O$JSAwLQlN=VkJ4*;zF4aEImJMWJ~zsPBob}aJztMDF{-kO1pjn6 zhhbA-vwg(2(88U@Vqw>W8xypcOd)LGO9)kb0BKrcd2W{CDzE9d5R{$L-*o)L2J)54 z^}U)FhLGh4RrEF}P%}}Q_CVWiZJ5rMgKSvk2(Xh1!JWpQQ6qi~{$65s&nm$b9;NJ= zD~ObeZG67S#9D2F8d}nWnAlE}@sC2jAiZ+0&NE6FY8^^R&LKA4GYID8beV&@oG$0O zs!#Y}&D@g~XCpch#*FP_9I4OO_;!(BwHta>2eoRmUp3m zFw+n6MPmszsWuN~T!Ji_na74zWQPnIva!6!>Y(e6S5T4(Jy%KQAi2EJnd%G#`^Q725 z?Z3^^IJ(4AvaWu-gg5!F_9YMlQ@!XmXi_qEA4wCCuq9iV!uevE`!cFrgo7Z3rNT7T zPew6H2(-RN-htvuuaQxVna7kqqssJCYi~%7y$Wk&63J<&niEQ7m|oLPHWcHXuFaF9 zfsE+M>$E$?9o-L}Cjh8UEYAK;Eh)@2@h;j;E$*aZqEq3uFlU_9r^X7+jWZ#qyilh@ z-5&DeX=6v_ra9ve4NRh`q_KQO?H9RTlf5zFN5*0LNI;i(-M@VbEJlupmmc`QJPM=C zKPa`c7HT_bg45EysmRL=enXJsn}49a8|fv3UO+pdVX}RP066Q6F$WX;$SXH~QY1sg z(xEz4V;`qDWtr{N9^-6t=p<`n2 zZ&^V{R(pxI{YeLyYLU2ygRUa`x)Z9&@C!)dWXw1J&d8XFpN?5v{!i(iL8j&KV8Z6SeWSKihyjJoIJWFKbm8YZPN?LtEe$WZ$G^RjaJcZrPo97JrWkY>1gltRR zSxZ1XuVOp0oa^-jUp^hMNh+cti~I5`m&|7*06-iP#Yv+KCJXfwX)&8u)dkkX`4>@& z_Becpf*I3a^jQ<3Za6$cp*pq4UZDD5DQ68!n5?kDoy9-+ir}Y*cX;{^?trNGWs_7# zdP%fwhY%eLT+3Miq!Z$vX!;A|9k4yer%jhaA3iIf9Jd!Ka>CQZQj(m#<(3b&mUzXYW)x#cFQxI5-P1g2EK0l=E!|1|Y}rRLLn zFAUS*ip)kI#m_*v;hkAGKtz(vjrJKL4sq4c*xR3e++JJi1o++0hVu86Z(rKNxuT>~ z7X`H=yoe1#TuK!2_Y06p{qeSliu{{RXbILV->}Zt_$h_5Ga-)C{9&e)TH`yag-YkY zi{V=>0F$I|NUQTdQZcTFpvA4vI}lehrlzTZ0ol7lcd5&c*E!caxhnIjJqoH+nRBLy zdo^*2k=nx@t~O=J_s4o9v4uuOYl~0xTjp?caiZtvayN*Ch3<4FuEyQVUpIk&h>Owl zZppmdZ;o$}1j7KO4Hdf3Es4hqik2}bwVM+dMxDi7Z9L=N{V4$UxGG1gGg4s8(YzL2 z^h_w8S%SrkYKexBLVwbX<+LCOw*oGu!a8blC2|z_@(B5av@|m0Hxs?ytlwDLL{lvr6E>g&bVzhQ+;>#{G9*{fh3u6PTi+kj1Nb#u&2*kZ3SYf1*{OKQbAax zbjtnKSQ3SC10Zb<1T_G9K2>MoMc56WEypU7HN$<($-$~388_AG!C&$ri*_>YI^c?$ z6KLF{jQ6@wSx@>}H>7HBlm3BPq_3HVYoN326NMm{19Mh_PEVS^`+Li8;ma6~SL_v? zY#|IQ@6Ip0TS%fE;@KTUBlRN0{YB~OF%N)~-8aq#HKSk{V3Tl!0e5u+vxn_x$JPGn z)GgN_vrHVB!~S?{mdDoW2krgISG!HRWCLf3@0_}v%pG!d#L001GcT05VB_tRjyc8; zCrWG{#zrX5_no^dl8v$656V5neOMZrN-efC=0+djWEwUq+~{6Vv;J(TX}bv!>3*&w zZlaQklG`{QE0@rO3NDNhVBLzXCx1@06f}elbW-o>y!?IYt;2bZ~FmJ*9|n^>YThwTtug zV^J1*sTm)7lis0NKzLOtF#$e}3bhq~!@wqmn#HY!Iz>y@yHNOceneO{q*OE#hC&W& zl3G*s83?adMf93jZ1Thjv%XI1^@D7_m^NzfW{c7ou|&WbNGz#IIT5lSXfEWz(vCz= z$<_O>C|meVewUG$OFn$Cy|S@h&m?D3xR{CLm`ms^vf`>9!$z#3`B7?^mB-dk-BXQ8 z?4G#*X8e(Ps=T{0;o;5JnMaWf+^fHzG4y*>$Ec)MkWXYmsDrE5bshqNPhE)67F7$$ zDQxpOlqrMiw!C#vm4Mkzk_jW8CbYcaO<(QzVAOHCrm&`&0rei@OiCpLtK(TXA@Sph zDlS6I0JAN@ex7B%K6-nA)8Ho%Q~lW*F`k_o`8&jpO~MDZOeHy)quXR-w8kWm#;6oy z#Qa_J#CwpByrgoG^KZL)8OK{EMI;%vcjql*XDu*kv~ajiLzj94Z8^G(Uwb`$<&jFA z&Dds+W>cXc*Dn-O`jDET4^;IzjcFTsa#;bW;E5!=7ME_S=wQn8%I>UL29f;CwNM&D zh7u}ZVI8b)^}NI#V#^n2Z6Q43F{>vBRIfv{^O{TGzEk;kp{nUV^*=RC^<^9$LP%3e=8nE?k8<`B%6rE7nIa|J0VF1I8{H264oPZ@DZ&6MdV> zEsM=?>OAcEWozd3YOH_^Qws)#C9_I`PtMTq20P%VvU>bom&?z@yW{F{j_k(;W3!K- zxW)w`5T+o_w)-*&W|{8Sqgvy>TvSIk+%A6bu4O9-Ywy{bMz}h;8vrdp(!UWlw&keq zq?wfxOWnucNg3avkrbRQYLCT+@ptDxPec8XQlz^;jG-DEhrA*Y!=YeF0Y@2u+>?IG zRmcF~L0N7A*hX^2T|&}{b7c0UQrXN7(cK_yR^7*^=>21od+q=T+R&sE&k%=dh+ZcU zI9aUY)EQ~>k>4q4616|HCH(!|DX>ho5SUzPFDi^SFIZYZadMEeQ(8BuAuS-8gG4=Z zX@q3y0|S{_@gCgOK@$w`B1OnhcdY%Okg4?c?a$&Zgt9c$y%Y6A{H1KmdE1!sS()cD z&Qmt@BSw=7#)bm95br;(6<3o(D4db!5bpe>GRh=Ay%x zo)^b^NMaT`58D0^S7a&bXCx8QTTKb;N*FE8fOnq=d&wvX=|g9~NDWhq4qE$d;E$NH8QG*m`UrI(1WG5%5 z!zGRlA3cM+>$XWQkyLS!o_g4j(g@3rSX3?fxe=0(TwsghjkkBtuTk(9-y)a$N)CrJ zFe2yC47IHw9oSNYpc?3)tPoExDDT}_$0ZKqatts}KPyzdOA>*cm=>Y7*fnew>H1s6oM3=K8s+-gcnOG{9#c7tgGUc* zUhk#%TF1QWaNTdo_vl!m^oquyy3V7MH+YC1XPA1WA|e*NR8?FfaMXKhzSXFaZI=DP zi*KT?Y4-?~UmfR+Q>h3>_8=zD(}PJU%*oQth*({p_xO@&d~B65ZJNb?XtwRSJlb=X zQ_I5qJiq0sXLxV)Hun)-jEVY^f?wc|%0qkDn`^t=dvJke&vC@;R!!mDwGLA=reFp! znKFl5uA25%j^w!_R@`a;E@IeZz@87t_SQv+J6B}f0YQH0FR=O0_D@&pn2)=h%B8)A zahD=FRYz-rt_!H))%j-NWvOgeaV%^1c(_aMIbEeNFo}z!$u89K))H4IE6qF#vwbpc zu_&E20b;=c=+PN>u6E8Z9jP3YRw9MT5PIw+j`q3{s$42G+Jr%H-r7Fha{2PR_;YE4 zRmwf()WP|COSk98X7S!d>-A`jdJ;q4t#hZDxEg4Sa1Kp&CtU%&ofst9R#w4l<_Bs@ z3!}hxkp?mQ_M+jIXwhvz4u7{~6v|zPoi)SUj6!tHGIECX!8ltR5(uy7pYiPZmWWFw zb7CzFXS%W_1APhi!{yU(T%viz!1?+%)RDIIUKLJkob060tt(4A=v+fSJSp+oz4ZIn zRdB!*6LRsgeXX;po6pFnH2?C%(Ck>~fw;*NXn|5m3P^-+G_~mD0gUY(ugjqS6h+ZN z>7X7nlBIakZS&-wA1YTdNbZuwsEu~iObHZ=k;;3OiQ4TP&cE0MmiK0fHl$z+!UnBNmYl`lrAlt&+Di1F6Xu=&Mjwe>}3)!aYXfWsYl|XpqdOrMm%|K0F>^|ucyy%mO zMf)DUCG)n_;aMhE?36BQ5+!OQE#vx)=TWN4D}iXf!C1x<7I!_x zw-TI@>PtvlG3$*U**xAD+~>KXEu4;7*1li!k-zm6JA~HYiZ3*fVMcAV64ua`?W)Ak z+SRoOE6gFWxr_|;zb(&o=?G$^(p0pBfc z9ebx;8k}zABwQ+TDQ*j1if)iOyfGaC;F~hSV^YHLppL}r?esahUzqYndcN!W{Q+&q zgC9!%R_C@lFAgmuv|qEitzszaeke^Vf#ikzV5hu1ywLl)(CjYVeo52Zc~&xKlXbiM zHkNl%?snpiR*ZpHCegd1l{ZCn*^{zWyDYszD@jF-_NH8~S=at`=)M)@M_DD5w1z-6 zG>(Fin`r-=K1nHCe!BWDsS{l3v*1Tl?sX)=%QlkhcLR}xu_a`p>Nz0-hh?ibv$fd6 z`r)_Jj6uT>SL!Xc{bXs-P%?awkKjeU29q&`;J@TIJpiVKV^!s@m-47!yxFno*8;bs zlNBVIe&oZytHlZU)tuFFw^olKn9V=8E~baK=f}6KyIK@jCB<~>V2$0LvbnOD$_kp% zs}5$B=Bg?fYb zO6@6wFn3I`F3)kx*1g-bZByw~T5?E1ecCblREhG9CD!t0Eb8!xeW1onk~d8A`#m(B z`ybc(jC^Kn&&P07XEz)|J7u!DEQIUgdqx<{$AK+_P?{VGVQo;A4m?|bPo>RH8kyq)^oQXIsII87pS=aE~o!qzuhWJ6)VaWOYDf(7Re(&Cij} z!xDHqwGb6De*@eXA${T6&^YR9eVlGdq2tFDu_Ls=*r-J3I3wYfUVMaEG zWIGzy(2wD}ez_!-NR7kWW+A#GIV0}wsI-c-k!_Fbi1%v)`Q^Y|CT-EClr##kmDi7O z{$h^c)PrEnAFTOhxuoOUA7_qPjwHFE&qWLomzBAasCX6*ReWD$=2|fbCZsQfZX;QB zIUH97$u!UboN!)q{emv9GUPY7r`hHe4Jh{&m>;~uD`K3oi-$HglWcd1yK?eVs)|I% z!_K&SSjph(rsB|iD&q}oucd0dW!pwc(bT?bpmroHC_LsMRBLB%vrFoB@?j}g9o_kx z!$0VAMt+pD7+jEa&bUqp`7~*BEU{wY4o#LXrULDc{vJ+~D8P}|e!_^D&@eFjb8Jn2 zYv(47eZpaqTdL-B7`Bg|;S({d~=gWYe_9GRK4z+}5t4pK2@q4_Y5;?wLKzbAQ8i=K&0h%w9_` z$#|jGGkJA(4h58Y@bpO0)GRhs=BpPJd@4UwDLn`-U49oPN{%M=>sc)jmqSML{3Lpr68ilR+wg*H#}UULEg zeI4@23K?Abo>6P1XHoBRHPx>WnpFYCi0_=8S$mw5_9myGy?d{R#oBi8u(MbBy;ZF| zO6<vfuZiE5a&&66m`HqP)z85OkQU{xd7|F8ersBY<<=9! z8YFo*`ldx-)8c8TK=AO*{-~?<09EU1t&}57w`O6P2jeGwZXmE=WQCzj?#1E{4WEJl zV_ef1Y{f2Qd-eJauA@3+y!1;Zfo=nld7WB9*nRK6b2qTn(tvVwE|V?RBu6(*C}V7b z$kD9XwzBAAYoeR1O}O@BIuTFRIHY?Y{gs<;U2@~7jIy6 zsLS0Pk=4>BHp^#!BFO({xX{b|>c1N8ZpFzZhJY<4;FvK!tSp2v=*&PJAF+;QJ$|1= zk}I(;tf2*Q&CAtVE6HAnSfb>dvv<7nL1e#uPf0_5;GIW{#|0{UB|h+B#de_IuRltN z>;=vmi_l2Ip*yA(*KfeGPnkIil?R}sj`bw^uWeSF)jTtLQPjaqfSilN;XAbBj#-uO zbN<@FxICa?-1A+Wo=a<4s1vF%b&atd{hu;PsPUV zlWt5+@w^b($dG!R4L%8gz5E+@8iQBNlh;eJ$3|2H+ytS^Uj89fT5J73>n`nJ=B2SG zCI+$JkrpZDJVt|`-|>cZ*VhWeo_pUfYC6XrT@g>tKP;v@1HECNxAAX}+H4VZA4AJ9 zNyc}lSWW;mTLd@(;~ClpAqCY>KXC}F+-*|-h{>(`8%pW?l~z4BBe-xcHyYWZ2|&^c zqNKTAzQfqmY<7u$L2O3^71rncF4t@X`_P+NxFHN@G z)oME&wdJBSp1IFQ=D8|#)c7Io`?l_cNxP57$*+ZH2;4osF)#t5vwqAxYD#<7rRi7&*_JVDG`?AqmwLVRK?$Ed z9Cwb2xvM{E2eqv2`P^>gXhjm8fFRnzgwhC2?_w_6SEj$WiWZpvNfct6i9XbVnHN$5 zJMlGlkr|>r-c$6Mwm0kwHb&WYwD~sIQoO9lL>6KLm_6RKgvN*oRjjM3O4{=j}gBm`m-}7s! zkoHF6R_zVo0eqPgOW)KTc@fCO_!CN?IJkqX9>@_#{lIROtZq@TR)@po_ z7rsB!;Nry*6_j!?X7s6_xGf(5Zo28kX-y`*O(Ef?6^BXo%+fU;Xh}$3OsW^^LiO#= zSRB92Z0$~MzQ-jzB;iDi;SO>JdyMWhE|n%THzI zxGXfj>M`bwM?LLM%CKmOk6P9*A6IkO%N&9XW{nk3w#e5Q@UMPsU$_tT+w z?L_d8w8<=f2eMP6Cg^8fOP|kU(KfrPP%(r;@zBDQuIM8I{H#HEp#2Lvv*#IVo$ z1&J+B{AF9;(TYu3i=s&LJrT{)Nu)8p@|3&lb}UL=%1m=N9!*cgCzztU8Oy&zPbxNU zl1Uq%s&?;G_}apIvrj(6%W#sHgG2%~x9rCm$iv&b#hfGLVR&2YN>Qc0%JjRljQZn_ zbPyrmE}Al|Y`;OZDhQaln){-WB4OnX!EYo)W(x-(<5SET1DTA@R33Hld3TD1zemVk zINDm%c1o((yC&s_>jZ8oT#wP}$WmsmgBZp*MD))i%<+o9_D~9OHeB;|c!qylUxhm@ zc9~W;3yFjV4UOunu2soEtpbjV(6@_Dh*QRf8iGRUD#opmfF&r!2ruYvra1Vt_u>~% zjkSRNB4;T>RzjTXddI3GjSEcCh2l;A>@7d9bzdBfvZ`Ornbeb;e&?+z#tNBnGi%iy z-WJc<_E475vojv1Eq8L~*bd?rvA9dn^spa4A%38&we~i2HFL}L^gN>F5)5J$GB?yk!9`G7BA-_c7()UN#VI zQyKjQ+cI}=SD)MWtb8!;G9p(WlYRl$`?eqnEEbVW(3TnyjJ@xeXM+|Y43VmUf3J%K z5>uAtfQ)9QVTdz#ZqJCJW}#)?UZOb)E?tYE`0*s|25ptYtb0%n?*ArI^ff~ef*2V( z&c{CV@}LIYg8nwVUK>svo@Icxj@&0E%;!e?Ufj?T3`GlRP;038Xv&1ZZ_#4_p6q)) z_=F))2@pYG2qv_fjaV>f^c%2?P@o>lq#U_gx=47|%_&*-;eg)+zAT1^^HqTQb&8@p2!=&851iXs>xPl!=hB*IBSXS54nx8jlZ9k#dU4gSAGBKy z!r9oCSlp6j;Zs9_+Ts(yg%12V#n&5$VnFx59_9$eQhfLA%Ay+j*`O&i)t7pBCPp^N zGLfE#KYUG;tq7lo{@Og> zAFr907CJ%dJ-Ey1ev&F!zG(`?Vv}@`bUB$}{F(Edek0YL%t$x4|BeLIg+gLhO?y2& znfu*}`Nc1L^g(cP?pGBh!S$DBcv=~ICw*1(Jph&i-B@5r?}G9_lbVdZvW zNX?G4DA^g+?Nfp?GIQr74`FK-7N5fWJhQ8~LN}xBOmBMM*dev`ILPUp3bB{xocqz| z?C)(?Fmc@?()SlA_3ernM~y%%Ojlgxl1Bgu5pmz!6;gOGjiuT)J6*_rGucGat^Wn0 zH(<3Ku>xZdN9B?zV zy>)jY$81>QYg^}k&jIM6#6F075#!tLC}TacAZt`@sn)wv=n~3lkb=EU=b2E4T7G28 z?EL(i0DGd)ByLJI&NGdddhp$-Rf^Hf9rk+5^mwLE;98VJ3}1<7jDqtu(XL5GTkcNw z-0sj|bN!%pS33$wJZZZEkeOU~wj$irpdf&@!}uxNx>yNDT;fq$2? zeIsK(&q0Ne>+G3%1(J1M1gg-P#XHda@q+XGXaXZj(>9H?AQYR^7B3=|gm0NbzL*;| zmHs^pv2?Fh^@;m))thm(-R!))*Mi%vgsd|HRp8>i(?eiw?(zoUyDqyhp7Gu?Ur1M>9UD%h#bzr zNN>uJz0Q&y1kaugw{Q6sMPg4i&z&pA?~LigO2pmo z_2RZMLVuejkIK1z!0|e_e?LydV+6zbSDK2!8czoA?thaRMjHq=J~54wdWKr+3C&XoyAD#At0a0ibt+Qwfs|7QvI zBWX%F)zeWmG-&R)9HjmZ4i}-{6f`qjM_C?hYKN$_7eI9IT))qTJps}<=r&<{^DwmV z@J6QZUtn<{#WilM;%BWZlP5d(SmMal_lUs3{6cc8{z*ia8jZzQ$)pC9VHrY;{Ch^aEcYp#~ zTZ>JO1aj@vVf4FKdp;#xnGGUfAV{ewa)}9{|F*jO`xFpRUnEN@1I(L*U0BF4$jwzc zXBBz8TR$;uQd`AIE_<~Ani+!9_ThuQKqL}V`ui|wU0vRRcID|BQ-O`e4Sg*)78A)$mfR-PL7hJ4SnI4QAt%B=SqJMoTE5vox<(hH$d zWG_{E#JofTR!${6abA}tRK9}V#q876RLL23Nz=5&b=mRRr!w%+bC`OP)Ap=Kd} zVU|_lafl_?11)``Fosld|2?;*!e9_zY(PiS*@SG;6agx~%vy?g-jmaot zoA#X!MCG<2g4cu;Pkc)+;&iBTZ(V_jDz=DZIWLI6GeHIfSwLulzf5%2Q5HHRPm5T! zTsmZ=BiLIkisg;N^oW}-s>Y+(ZF=b_pK#72UPaCI2Y#Ax2>ppiZcyA2Ik!Fo{L+4L)gz!Wils!mcO|mVcF}XB)u6wHiZc#>+EL& zZlO~+DHW-PYpUIO$^1seEHatqzETx?zi}plXt1(Os%?kI(j`uhHG2iu_F~4Qo;2lA zS#Vr+7F$p%F55iu6@+kszMWq&h+s1s=P)k%z4kN; zNS(*Ajj6~+iYC%mEt(>=ZCr15$eq-+;~IwWV&aJ6RGLtGt*~%oW&6HFjk1)Xj+#vD zZ-%I!fx^4ZEZF4Ww@Ad5RM&pF1cxZwbG3P|w;z8`;p5z|kmlWzZ6LihgwxW+LuHl` z=diZ((^^^MNhE7u`nP@U@7sPSzsu$p+t%MJCMz{bT%1CCAc(o2ZfKk=qOqsi$`;u~ zHN>VUkeCAo@)eGT;+BNfgZ%Xt7OVUFB@aZz8`=Gy+i@V)8#U8r6P_OQmJU<+c9)~$ zU%@3s7m@Z~w}%&r8$x3#@TxqCLB)#e6=7~x8Br5T?7?gBKq+N&=eb;AYxJy4BFj9D zFIz~|6@#xq@ly>%rE|5Z86j0i( zGP-g8`~&WU9J?_(VEZWVzl!`jGMppu^GGU_7k6Y%yU#;Nmvf-=VgiJ~bt-HV(~S1` zSrZWbSid2S=}6iRZgp$iCXkO#CKI7P8Q^d}X>zQkdTglPAEieFTV*CoYYR`iqsCct zW#Cpp>HN|Gu)aMur){#fYtOjTy6~#o`homys!X2^5UZ$3a5Vn`%kXvKh>P>-;>Ek= z%4oJtL8`LzwVU-AxK9+yqFgV&-CZ8}+V^N6cXPkDv1x>L`?G~O%^|2p^nO1mtY<6}-<0|-=YzNT75$8J}jz|rFxAYfRs3thcQiFiFk5BIq8uuO=W zjq#dpO=HGy!LXj^N@o`^l}Oa+8(_>9dMp8Eq_y5UBy>&HfALn9w*g~edvXmz5l!IP zs5S`&IP_MS{)x#Q zenR8&mChQKdE_td_}D9!tgPa;_eVjVW5VQEK0-H=Pp!I@5}FiH?Ghtxns36LeM0~Q z1AJq(cP4E{-{eXyd~$JU7c@w_()kV6i(CLNe%)eDViFJtLkK9t&C{MG@Bqb$J0pmH zRYc+;yeh9d><$Sm&`cgku$J8~Xzi=h$7dMPCGyCDToPo)53GfLUh&-)o?WT-B6p-@ zD`t$s`!PSC+p{cpDgBkQ$fu!WDwL+o$4YlZ@|6YMbwxYpZE~UP--dM7g}c>Y&M5Na zWIAhz=x4Zco?2n~(FWVS@2$Zp?Kiaeg|Me6#!g0Ql2ck*<-{X0rIE{KIBMxQ>E{X} z*{jjw59gdp*_;qjA+(nQMjvIQ4DP1c%IWa%a$q_g3s zA!qd3#%8DVA=veuuA!9w__q!EvNV5uddv(EqRY_w5_907Qz9W}MLks-@uyS|7)Y-I z2xkY}uCQcwUi~j@cM%WGo_y}4LW8SZbr>&PN84VO&9Rl`8m7<)n%No{xmybwj>l=5 zTZ3?N)#rX1oER0~Y6YZNFbC90sIPQ4Vr6DZ6){h|e%o6m4>ZEXUcnV~8EA z;f>K5xGo9TwRiNRIEX3c+TOv5d(~PXzUxc2P!{uwGs;|K9R}FPIi6FKOZ@u6Wz-P+ zxpxrmkLy?VtqRaEg=y$(%$_yoQXa~I#=tUc-mGiS$ zA;ymeKHh-`DfMR;0-7U2~6MN#9A!6Br<>mrV zG>dw{bk~2V6XhNz2_a3-tZ$1uxxdu*4gB@&Q{FkGI>h1+Z{bYB4-FccOLB2paiorF zou0=Ib-x0`b#XLL%uJJJ+ZJtBN#TkUe#N)z<={V{&C%J$+KooGPN%*rM{4Xxto&_; z9nNW{-x&jT^8QNU~A`})eo z%R==$Vxo3~?=p%tsWM*?WbImhvUi;cDHjS_x^=#g3d+N`a{8g{zT)=vE{*z#KU7o- z_uSTA7RxsptF?P+e#gA;gT4?>y*-SbL6?f6_ z1%K#vSFxw8lT0$XD$)RJFhx?>5V)0@TFN%UWPPMDlx;Xa9%kGHQ_NzEjx%d$f>&)uX;5{J72-tm>Hxm?<#y&$c7h0Rc@;ETT9_KvuXcA5GeC7VT7~@>EyWY3L9$!wCp*6pTvWN8?~AjMn<9p=Q%w$Q zaiND9X_FzWL4yyqNSWSwQx$eEj>CrX_@N%noeTg@?~Vd7#$ZI|{C+Tskti^zrj%P@ zt-{EWIC>3(ctlOZk_lX~Hc#TSehGv=F#P>kI4s3Xw%CQ^^+`F%@-Qi4W?JQl{Fe7v zR>JpyH=e;qk`^|OQk^+|m{4cr`gal6elD>7!JxSz@=?U0f~rWzIVJ6QnyRjXA#+*} zE35Y?&<71%{BSVzp7{-WI0!>s`2F!yQ-%&Z)@R`A#8HHP54aVL?)G`kB-F4@?D&n2 zgaDAlVG41mvtN&AkhIbcvJ?HUNORZN8{4a=ISQ~!TV*)-g-K8}EXO-#iKEg6D~G43 zI-*N_2E`$k0RR25t=BHi5U~r7BsLh#l~*ALP%d8J2EzM<9XSl-u`QMyuOf$xfe}_x zK^CtK-8^omj3)YovGJB#-$U0#0QiSX=RDtn(Qw9fg@PokRgu_?MS$`qb)*j);B%@e zh&Nh?Vzlf`BbXJgO~tpN8JvaBX>3EHcTwQMx~(HRO?SO5+g|ob5N8TP95G};mr~Wy z`=1v)41%J{iG(am8rVRFX2L!i?wd`GTIaYJHHI-`5#eD3FL&k@jRO5TaX1B9cd&2^ zBq8zoFRG7l{GE^dU6}3}R8fG%i#R*S+&gac*QHQx{!s*0rdDpd>;i9I8L%{KzCWlP{iz z`4)B^ zyRtAKx(FRNt1KVJQ zauf=DS9n-#Tr(5AivH<0$SYq1V! z5NO}u$0+wOpy>34ERNx9Csf$)XE;%X9-dP(+8Rz)*h5bst1ii5fAVgXEQwc< zh{83y=*Chr-hYpEcqM0gM?o_fsU6Q>2#~)3acl)+h<2)6pt&52tBphSxz_k={%*6v zUyQ(c$#7P3g|esg>2tJnh=uQe)4QcG`suP=aeA^Uswt%|E61FnK;6zhOKEFEPFQ%m-R_`FtN!q zSwT{}Il_SK0zA9RRhOz4LFReUZ%cxyy$th%E9CDee^*mSJA7Rc!&iMAky2h50Fgmj zDc5b8i!Yb7Bp^JqJwMlS->-K*+$}~|&Cp~#8cWaCqZXpPi73>YXo^lmj zk?5x)|J^hn-Q%t{mhZKFI*$;tPJ(_tcez-X!#!^1ZkSlaLz(nSv^2lAuT=!!j?SW> zUTQ2z*sTc9eyaQ1EnkFszjU;=>ymQ^&|1h$?gsWg;m1h^0qf!Up>4%)b5T2zQ1$?^ zP~-F8Glx)>(rx)4B`#wg{bUHYojs!Xm=KQ&A)RevQCvlmv>ct{p;O{BnWJ)s-qr@#slh@^~k0d4H-!Bga}9 zW@@OY{rw3}vgNx6H-l$GLbyyTqe#^j06#p3sprt>=Ku%`y#!|U5CohU6fwOHSyqQv z@A-rMyHH7-9^Sfa)VW#5+eABleJ(iuLcWg<8xF2zr47Pbt{vrOUnIk_cbl8>u?gP3;oWk7zor1z`VyN`+hcl%X6lcM{?>sT*3xo z$S^M~NIt1r`?JVrt7?V9aAh)PMyeJI`x%!eeNmrttVN<2HWV zX7zGC@o^ScVVImK_Hwn`fl=LYOfIRt*kwM9Fw(h_I#>ZkQEdjkQLJ%H`|!GS6_K|H z4v=*zuVJjxYgL>@kSu$` zCsPAGhrxBrws3dD_;D#cHvAz?Hje>7f{1s3z*Q~!BXMyxlg;QoQ1-6kHBJ~wzhtLi z$9~*A9^auAUIiv#vv~)*SsOYhsvL8c5ELvT2F34b-g6?b)k&5(*ROV&mxY48DeT_;@v!#Mt%fl; z$aX}kApnE@quQFaY1F#+7vAy;#cO22gONmu0O;2M_{9>#xSPC6{qP&BWCX4-D_W%_ zKYN3%z`Rqu`@T7Xc=kbaPV|uO`S&+!hI)8IJR;5BD(T5C_I`&9$HzlV3UYF6BM zp(^KZHKxc5Qzy^_x+-h-=Mie{rT8mTA|aq*!fJO^#FdA)iJruP^|(U)z~lD$WAmf6 z6<^MnrcITVH8j+uDoYvz82+PMWX&<_vqMwGF&NHzR{n}}pm}~Pf=HLT4+@R%EPMB} zoGbbkm&Bt_@77;nIUVwkqVw3269%H_FOkEN3{4U_?8rH1zJ8dE&#|X%(~_#*y)-Rh zSGXOPmq|VZ4QREPhHKZUwN3c!bF9SjZl4j2PJ)2^8dD}!9{%)XJ`>ef#~a4W9#hTC zS4}075m{^qvf@#da~Km6J#9cfUC^xPcE$MGSl{1B;a%Y39>w$i+*mp?`r8>pC?6ls z$4!D0u8Aq;6O|pA+b$xRmpnvu$Z&U3u^~jqH$lo?KS0I^$sIAMx{=RF@yF_6%a(*j z-|H_nisajdXvdw4RSr_;Ntx!p(nt{>$Br&PQdJz9X^oPJh@wM(<*fJF zi_MjgkXu^3_e1tX+)pwx-*C`;9|rD$*&(_PWs*W3LtIRqcT!WN$ZZ>0s~9Z64JW>t#JxADe&E=TgoKB^A)e#RP_8n>EGGQaEXPUK?2Z(LW$ zxwA9~M6Tt}b1^CFc=Ro7$A0r?JesK0tBgBm-co}y1L3KZRyU%=wE2Q@g8G62I%dTJ zOTT)O$N4uU*lBL{TFu+2^5VGY!J;V*oUpG`xtQOaarEO>E$AAFJSRA3hyRSA!lVV$v05uhR{>&TP`B>9Y_&B%HXDR&`6^CC1uo| zl_o(BZlF0bGlc28=#ISL(%g&-J2GqSDm0|T_wPcP z`RB=Agqe5$d^~=Tm3x9)<`1vPnr{9$k3rjn;fT#Y*pR7dXd3K1_^w_einWLcmH%B2 z$ujTC=nL|T;}8@FT7$-+)`33axF;4d*&qhj@#IVQcadxJaW5>ir3mpyoD8+)^?s4i zUK+FEnmwMzOyt%`tP3?Fh;*3a2}AkIY7Z5NIZIP+hI+bTB@aAd)5V*eYc)QhTr+>L z?T&DppR$h6oii37^m=}U7KcbxFuBp!nU|Aa6GDG3RmF};KuB&EV2Nh6#zawo#t!gG z9g#oVX@z+9RQnxPK4{Z-irJ%vGvmV}k7*Ui#~2i0Vnkqwk^o5noi^Op627d4xG7nxsA9Z{X?ta-HcBN^+!K39JHn!r&jfzot8|LxR3k(OuJq41N?k9n!1lAypF!<<|>IUv-`d5)w(qg>|{g~BC)r$-x! z%>%1eV}|kcTlFyYU1ruYt~9=YMDVEd`-3g`)KlZX;h!NC#4lLY9QXNj)K5X}Wx?%Ph zn7Eq(Hvb>*%;~MCizaj-S$S%R&L0>z6fCFHy&0fw8`97&kHS&~==hbkv>hW%hjQ5P zl3qCT!YGBZ5*gIvgTdAvGPOm39Yjo`A}3zp)HT21E<1~x!Z^=x(TS;wym}3>e%Ccl z%gA|eOJPBsi46`LBt8{)=8HxzLR)4d=qB=FE-fvKk9s-F(%G zafVUh&Hb7mmJo~YExiytx;rGql+iFw0iE03&4Cp^>hiZcbf5V6Y1e!JK|sF0T)g=~ z;jxTbDmk!0t`cWLtB;iPt5>{6xS(uFo6e>{Co_dy|240WZ(QTuRZUp>R}{;vGESy4 zg2aIFh37E&-uTmJI&v{lY)6#HCl33khpEn@b+FLY2Q*N#;mxOtToY?_fblb2M@@h) zl-l^52|`P*R>9}P{Dj;+l1pa#OkQGtG3vTjHNANIBgAXn2T1H@HMG{&VWs@-DP8yU zOv{$>#{8#VXS6=`g9bvB7~RW;x;R_#SnA*ymfL2k@K;Wl2X?3ra>N2+D0R_@#=%c& zc|I|h6m^^QRWM)vJYvnt>xh!$NzZV;g088l`39zAln`l>=v20~U~|7@ms!4qCE%?s1b$)syM7uLaxR*fWbkEF)Tf83!-6KzMQ$9UF+ z=#`HR-P;L>IfsM*LImCX^q4z;`sFJmW|1F0R_~Ny_rBHm>0v*{A*S>zGm=af?|rLO zpV8tM?B={bg&wCpuF8IUpDbZp9+5rVcI0YZVvD74U{UaOF^lu-PL_%KV&8=SN87Av z1-Sbsw9fMY2tA-iv^b(Nk36iSeoKN^)v#M!u12}f|IOA==)X#&=GSVH@pU!>%TQkb zNP~`zFS>_G)*-o!+^h$ZbzSRAyjK26kd}2@)cUusLbts@Lj&;wY;HYdHciaMKkOZe z#o${m=u$!;A$Q@&n_Y{z$kY~+H0+y_Q2naHNGl%vG_7#X$TpYZ&|k|Te0;*i(@L5b z%!e;*8{1oXabH^hx#y~kkQwO6fP=5inW;zpg#ipx&>^(P;t@RWv> zL;O6tGHc8?A>PC;oR)JVACfi=Ij()D;DZvfuJ}8=b70iqoR>rdvu|JTxvS2k1;dRV zaZkH}qML(lfX3Y&8FZ3e6EGg2Tm=KR&-bZ>UsT^;&EmsDonG}ikI|K`VZJJ|4hkKk zV`L;0{n&B-9QhE(6a~jLV(P|qEuCxVkI+@`%QB=|5k_XGUOnry88T)u-2>^73A$2x z59@2e{9qyWky4>(PfMP~LHIU#=830gy^UWCRYzqzbW8xYFzDXld9>igjm^o2q`DGc z9KyEXVN9Qn;!p2zz7pBPqCnOuHU$NDKv^vbsHhLsnQgZo9GwAHqIIo*L5qHKN+)D> zvu}yZjz8E}74gVGJKrOOZ$4R!r6?6-x zB62^!Rc3NQgN8(Qp4szi(Y@D(ajT_x))q&9XkHCR;$#0N)NfoN!Ft(Rzjz>^;yds)u%IGFcJAK=0 znmKoRVW4pT>%dxw>m|6iKmK5s66KzLhX=;;Sa=)!L4bQ6uvOXdX3oRlRCgk=oK5Y3Mb7Ghm!ty5W3PP3H z4s@eTHeLmUIW+4ie4~DeR*7x@Xirk6MbBXaG>0UQ1B_T6r#fIlvigI9BREE_<#t4` zW+>HZeUbR0=SF`E*sXr5|K7PkGyVGxGes#H0{0tiUDNM3NZ9yj;(I@^2*A6xS3?A= z+f=$=rQ}bpD*L@2T6^v+vzQ)7r)1y_q3d9Tsql3ltDM0|zXn8>N`mmC4v3M{=XEE8 zv%16`sy8^5yVB!rY>(UX*Ghgt)Ux*i-TG(Sp*p3V78&;bS`wq+5Ye7!N(wqPhE3hM zPRJZwnSX7_7ANpJ>+>5#tw!~q6`rPMc!pgomakX`*&y`1^3*N{LI7KoCxhHn97VW& zpwp$$U_qn9^l~twvFutce!|X?P&Ez<2J3mGSvur5)x8Wp8U`F$x1nTZ_!H1zGdDxL;~&SCuaCX6 z2D^xZH!kfp7C>~1{6=~Iv7=$Hed>+lwaOhUW}SqHiQI76AA+fi<;id3 zG>A@ZJKkNmDc|O0LW9M~i={7v1Jw{@)g>dM>Mucz;A&X&Vvy#iHsf8S*d{C9<>%m~CcK!7O~F)vDQu(nDf)0g`)2z6=g|2g7$PN(3g=$1m{@v7?C$w+o{g ze4=wdFG#tGrVDmY#e8L&{RxXA^V$F#j`MCh9?O#yO$`2#M54x?BC28n4hPxL3hwbA ze)FSA<*>Q`eJ(y_$8EOl?DC`^uoacqBOV|~#fx}r27P7xh@VMlo{d7)s+8N=h6)A) zWRO`M4$<8&1_28Otjz9BaggVoUrOZ_y7;x&@65~yUVcB3nrH1y9|cjSc~2A0je`a8 zY65G}pcuKGic?=*5fu)bCaD%5+5;0(`~B1mHOXJ+6b6HkvX_1@10U8_Lp=8wDMIo) zAW=FWT~9o0k4rCV=;h9(s!RnL*2te8cbV7H=JfAy#x1w$^Q+K@akGwj0G@_%5Ljq3 zt)p59(fwuChp6a>xwSq+E(f1xaOiW}?U*c$NnRG0n}rMZa#@8GEVGC{PMt-PMX5u* z>I};mGLc_L;mGl^DVU4-o@5cur*Sx1(O`F(N>Rc(MXs^`X#%gx5)3Z2&Q4_G-eSl0 zDi-*7h?QII$){)>C{Sr!q_ix=pK>*k$EZcfbDXzdt1-HQbzd!58c*Tue}QnZ6a95u zJJ%y9LzaYz(`G6KA&Q*A#BD-Rk%bqlHEzpb^44_$?#Cq`3f}EIr~(dxh&e6~QaXgp$uW1ezAV;!nISb>r-;?rh(PiY z_l2~sSbXW%7WYx9RDMt0`3)u}?(6Om&`4&BdEMSpY{rVl62&QbHeZLMT>1x1<8R_j zH}{-L=aZfcBct9hUS!}Tojo>C$Y~CR2>}P!c3aC~v}<0qqtU3x*!Hkfqd>k=BcJ{& zsi4Ny*}LeO{x3h|_oCfKzkVP0qEJ+1r~EWLLDy&AWA3+@ed zw(39vR$VWyF3Pm>I}Bw7!IP-LQ*@rcbeU^!{3@k|Qr(f&&HmwQ3%IT4kjT>Ge@s7O1P^bk&4yR4}>PK?VyT4j`<(L}&fx|9@9t zY!Exo(-Rg$=knvjM%@xl{pgw~9=b#8wlW~ZsJdNx zL|MZ=5XZ*f-Hsm_XPL*A5kh#fHu6q1%v4qt*l$Y9`G12&Rnq;k2s zYF?X&mTL(cI#LhPHuloWTDv1|$P)`K-)-0qAd-Yk(zfraX6a<|45 zp9-{S4e&!4vK6W@e1xwPT~*hfFOSkvKYx~mnIrPH6cB$vMnL5=>D~*sDnK3!cPVOM z``?7;$MdGdQX{W8dqGBI*5!Tzl7CdYsZt+DZrZ5)EAhm-|7Y5YYsc{Ij7UID z<$LBsIZk#T$Q6Otr;Cq#jkGLVPLh``@Gmlt2{%(a#bMB!<$e(GTpphAn1_WI0};z< zQ)cZ4Jkh^fIK%dM^kw=m42G5AAR&OWyb3YDN~Vt{MmF<^n?%A37$cwQF>;k-=|w`t zUMdD1TU)A9iRO!R(Rj4RXymTKHWOnwz5?@^xr=luM~@)jx!8y-R?H^Ji*jm&4oTL( zYe2-{Ccg?thI!1=qpK`$w}++=>AMhXZ{St{{|Wik-B zAs%SU{dS*Cf%7(?OAZ(6vSw%Z_iuJWwV!dLRL;&brn(8M-+=}%-p)j4&0UpXT~24H z&Zvo|46wvx%nGWe;J#`-`O=AkLJuPIJNcYePv%C@&A>1{FPk69{#0JOhqd6C!J6{Pa zrQUm!lOWY5ab6a)TA~n`G>P}GR5^WJ^e>3x#zOjD>Z+c=l9qBcCY6idH=q=L1sp56 z@mXBva_mP*JEDf=%1*g?+NOuZJ!2wz0`Jnjv*KfK=X*uQi?5pA;k*5t$U_`d3mKWs z*tbiMx9%W48ch47WA$*=BVZ_6FkeNE?pvMMM=uOMa9lP%oy0e@#+c%W$kYT zlmTu)u;=h{k0prEJiw9rnw#3kmeh>T(&J|26Q#=dIE*qLCy37sfmg=r5qa073)5xs z&}O}oL+i%~U3RNK^=oVFSiroFPUxt6g;ni?5M$-%mJ}Y`{R#K>%IxnV$U^sw)Lkis zTzl#6ikp!-RQ}4sJ7l)+H{;jNnD~Vv+}{M%+(VExY{iXR|KERyK!-`DcT_d_q%jl- zh#_j1H_xmrs^b-Z#m4AuZ(K+4vY4l7R*9Zh?uK^Q#Tcp!p^CJA082cy2ItO1M~@hmjtJt zf;MK!UJX44VF(Rx>jg}u?;*J)z>>sTo>voMj0vxd7NO|Dx*3&Of5`Ba#-`268Cy(K z()Zbx9xcQ>QmY9W{TR`VKJRpQIO28avKQB`AkCS_wqfn|*Cb$ zy_(KtiLf^p!MyfdWLm3*Ge|z(L-l%@eq+L#CGWSUEJ2^SUYUk(=Tj{ldL(C0;YV6# zJ-Q<i|M(c`AIr(9sSt(mD}n_4w`VC zJ(ZR^oVoRcpQ4eHnPOOX_Pv~~d@^Y?#R0neDe~Ja>RVdD>(y>y=p7Cw$)1`Yxn8-$ z85^cyS;yzKSZfNvt7G4)8z`NIYfwXFwL3KisszaiyO9rf&ks88;F%h!`9>gq|7kiW zO_OM(Q#DW_-I=a{j|M!hcwD#@OB8nI>3hszlo?D>XhE<=PNGCa%V+bVzpsW=Kk!6) zqS9MSCAf+~mZrN1mESaneryoH+<@*qqjB$_2iN{ZAiz@D;9C?#!AMdmNsXXjtFtiO!~L1{D{*zyQWiR$gdTP zeo+;H!MzI_k}aF;_2-dXO#@CnXxH3?Gj*GZuAjav+`nQ!nD2)M3^I395?yWlVx8#mY)o_(kcj8q}Z+vCX^J|HrmtNk3~ryj0=wO<#0K))cGS z^v$}O2zocuj~1T^lG!3Chipb=A?=di`GdYTTZ%p*L5insX*?6uZyaa)e>dFxDFMom zZ=pVG-=Gm1i7hl4>W}!4lKG%HqePTrLdCqL3D}}Waj~Ny3$w5W*SIJFHzgtF=wv7C z*>xPg>D?1XQcrq2_l}Qh){)eO{BOt_Z-G_`!`)r0a(ZhWwK?P)Uq3l&bsT(~lP5*r1mR zo~Qh<@KJpTmd?CNm+@Ovbm?&A|I?rzOsTC0$XsYt2x_1XtIsK2@>M$^N?@0nlLcqd z`3=AM*BoiExOkd7xy(CVR2{{mA2V>_6KsTPVC80DJx z-%15%f^OJZH?KM)vCSMEw&=hC6kWx_@XOO*%q>B~Htvc3#HqEm!&KA+HS^1=19-Dj zO33EC$kDsm-kPh#B!0xNICw`c2?^Ud)#zt|6yge^{ESUj5|_Kcdj`?}{CUKohwTe4 zO`f7oO2SOS{Z~+P_$nIj_LBN4^$(O(d(hbs+0)2BRCOARSS>gK9<$P<)*=MU^&t?{ zEcS6ZUb|et(f9_JSf8v*cze>ZkKmJ%Umw1C%Y0q;x(Jyvwy)2Pubq`)`M+HT$D7tB z`)etS@#&o#GP57HwTfit8TB-myS^nO9#zlJuWdsb%Ew`y=lih(cU3lgRiITVy>i@m z=;|2vkUQL=+!893Uhq*#u%lpg9#or%bZsp@{@DYzx>7V`P!XN=lw~>7Wu%>U`v{nk)#Q1(Z8b}x3O7(kKyQX7M`;rClFOgFVXet%wj46gXF*YrDs zqUOXBsHKK!!Eg2u+V+?UBMcw!j?Z*FDZXuq`kA3D&NWt)6-I5_oMIT-1|8@M1UXj| zt;z^Ip=-?QPbAS1mm8yRCCDXInBh4*9>Dre#}j)K`d0?*{7eiB zNzXxIEz6~{m`Q^g4TP5)^L*n7pjA?k`dmu&&sD?#@e2f-_AdN_yZr>ypgjiPRJh=~ zG?j`~^fHdoiNHj8(i#29C^(Xh4V<{8t!_+CxZ%1bLP&9IJcp2GUHx$PZ`moM7pZjy zzaua8ip@a!Asa7!9h%vgr>Fmch#?YI|lwB1+&g}G;qM@0`5iY>mr!NZuK)=KU9Ngf~^J&AG>d@ktTP{k$} zlboUwc76!bp_rE#dT!IJ?51Vh*4ktTukzk8dK5EkevZi%Z0hmx0oB?e%#4!qpY!3a zw+)^_$Di&HE>UKfJA20O+K#TQ&4_+zJP$sJ^(};*MpV`)7Vd<0cD{wB?KSFc{m7dM zPAX#(Hejg8TrpgmoTj_rl|ojQMXD3!WBd>AoaKnu0Ewu>YgFC9w%76?cgQB1FDYZyg1y=mz8Eg~Vo79RVK)m+u zxk|^Ut(LQXgHvJ)GX$fzM>m9IB+huO;L}=J`B_X^of}KPFaXQm5$F0|fGI7(X*W*S zpkq3nZFcO5gQ*`!XDpPi8<8*b|88YXLs`EMY&-pZe*=EU3=`Z-@Uf>_9vNZ@f>jf4 z!#w~GM1dGt(4CjzB!}t;4I1K*xhjA}&P0fmaMWBk3WK$JPVRv=DKvEcMEb^a^}-*_ zr~#)>{Kj6^YcyNr-Z$Ya@zg+9kYwT}?jg|&N1uZ)2Yi);^=imwI?Q3RPti<#(e}OD z&rnLz4ojB>*h!<|M42rN&#>*ve4A=^tl`N^%9z%NJt*-gmB8W>H7KDYUXIQ`229A~RI?;rR$mPEU^_BL|bL@1Y_P+WgRe)(F z$jb^%QlY+vx_{#V@vsyFf1qTooD>fbAehID1nsE6gadkT&00r5+PAwEn~!yBc|Md8f)Gn*_?n%arT zZELj`YE4j&CJlYC7SA?v6b*dwE7?H$VOfS~tu|Lq0%ZO=gc0eLW5(5nuO_az)uM1w z+BFnnPwx&W@sih%3y<=BH%@BZaG}~6F1R~awCq^s=~kvYv3cSdm(jW79G>dSvi+K? zu{w=pgbQ?#8b$JLY?YhjVGb`=9;Y~=x!YGhe$C|D-S)~%{@;XHu|qi(Ueu1l-DW

Jh~qTVPWCP^_m2N7Ra@?X9Uz9g!epP6Mr8mwX&=iJ;m1i;mJ^ z!bdMuHz)L0gyJJUkzrih_3#C6fY!^9qP0=v9nX?ohEx#}q&82w`a<;$8q;rLY7yZ@ zG7JahN_yg)IoTF1{^_gdap|CT^I8`g0krsbG9FH3Wetd%}ktb7QnBHm~ z4262AWXr_1Ca-048X95TkBCpBc2N)^4=Kt(nOYd$eMbZq$HLR0AMkU=#4; zK|g`Ip9+`Q>8$NKr>=jhA(;o_*0;VP*^YKz{cE8CzH=PpG`hU6xkA0S%f|J*V)%iy zpRbmi+ik`(^J?X1`o-<7Sq9Vc_~YrNa`62KA&2f!&kCX1yX{8T5KC|mNCxQ6{#G=o z<&Nm*a9uFX&#Mk=+eekzOUC(*_VCr_WX&_yVyXIfRrj~ecB)w=V_Ia-o^g5{`;>Xl z?_yU00tf4?z3Jd$uj+>K@yIaZ3lx`GA~9G*!UlZ9;3|@R$>87c@^W8?8^pu~^saWe3&jBN7iHh$6E*1W& zwHI)sOh}t6z2yOI$FtA*!ptvP=>tVHX-qUEFA#~%e9J)D_E7TNI~8O9e7R7pNd&TGEz5M<{bCmb_iAPaDTHWu+Im6+0sVp38xyvU>xAZyAO%kDbx^zRPJJ)~-ZQ`Qypk_&{NyE2TjDBIK?~M}Fv?LPEZKcOI3HWgd`bgciq6jM_zA!A_Uh*~Ee4|Z! zup3M>GtmcnI4}nVS2=A?EmhkJlcJi}|9bV)>VdoO%jo?Shvzw)8h@B#!i6m-MW^_P!>cVb=xNs?)(6XOJUmX);B= zY7`@m*pV0+_hcM-UQSy3YIQ_r)9;W_eVn!ys*)(uPE9!HD!_vVn`vZr4FiyGtMCdZx+7vXsLOUtrgKux=L~ zUVtK!-1|nGC`nlvZ+^j_78!QrWWVe4#*52*q5vvhiq5s%SU9?{SBUQ_cciK(hTP9u zTdUc-T0|x=5uQ;w8G`Vcq~98w>u`K>l+pCg_dKKCKYuu;aW#K$X|v(hp0t))EyOFww^l-ladXnHOrDI>nxhetfc+v zN~XbSj&<%`Z@DzuNI_I8P3i8bZ=5j_Z7lP>r%c2_;%X6@a}y*>b~c{Iu$-?T1as$d zGs_3WYpqi==%o4VQsDL^^i}I|7XCJC6L7x*g-62IEeB zw%~LB@hnfQyHE9i8UkhZa&$rZw))&?6m^@HdR)a2cCgaCp_Ox=S*u`G%~8u=W~vjd z??=|fWn5%B{p$0kn-HFm`F{;8p?2X^b67~{>eAos9Hdw{;$mPF7D|A{eKWEBgauN; zq#b#hqXikWU{W=?$;jiK(zI?&_%)XCaWlt-GJ81e*C;f;5$ntOx+@xyl<9Wjfv`2C z+?=f@AnA=9;x_pGaB#4fajM>6atngS&V#l+Hn1VAy1MU`#aE8%T@OuR(pKY{pkowf z`2Pxu^<#$E+|_IUy-WAn3dz1n&Sr-K(VN^P{@W2s}BM;1qS(Noie|E<`Y6r#L%2yi! zou@%s%z6{}u?$H%%birrZJ@<%x>9X(paydB)x#n8&xD@LYR5OCJlRUO;~_21zQk}{ zQCf-@Pu@8lZ|6wA+(3hPxb-9=m$kEO>x!DiU8JzDckD9>*3foXquD5z%XusC4gVwX zRzk?#f@2$|C#+@nPVLOvnFXZdsMlVQhj$*bm)ctIQ7qJZ`Ne8$M|4R#B?p2jwE1`z zH#J3X<~Ak{d^t$>UUFq{eB4cgMA@)!`Fd6ZEQZhoG$R3=Q8!T9UFRC;k%L6~5Y z?Y0JQInb-VyMla6+x$xW7d?Y!%2pg#EcZ6y_W};-s6D&tVeU^$tf|l@NWK-lUMK`Q zMs!iNOfMuQH`u)3=XWfE(ndJfF6%Yu`1OydKx+ujaN;4Qd7^*v4>EsWFq=#IBdEk1 zBM}PzADQmp>fyBqb)Rc*a4lXeLMh@OM?IYD5?h%Vei1(xjoPf_Gb1G=;-s(8k4K0t z&ZYGM+0?&mmCv|`wz;>N%^l>1lx>$lc)%04*d0sxWhu(q!pC>Qz{k*h~)ofvcZ){iF%XwFU$ zLgGjaC6rRRiu7mfk-5WsyFo3w*12{XH2O$~|6_uej=XJzN-Iwtu6OQc-$tJZ(d#|k zoAJ|Lzy^w63Qw+EQ5G76J6Es1AWzbcz!wx853GQ}5T=XmY}o zX6NSg>hVz**BOIoT}FZTxJP!Pw4QGT;pYdpf5zDv4?ITIL4sfJ++-ZBPNb@vOU+aS&>IIzzghk*1PjAOc96K&>JGs z2@C6N?1i$BNk;EFSH?*}Mh!yD4pJ4-YbQyau_*RB7Zhw^DYp)^VilyY+QF@DE@DeExq+-E$DYc~Xqma{ z^`JM>S2E#7leI;oa^61iwdy&0XIQx-*pTAn#B9oM8WKt2i1d2LeX<}PK?+3XcFX3f zWMhcIYSnp6ujh%vTK}4oKZo)n0`<<$WMnMHD#RIG{!f>xYA|D%-B6^>=y{CoQ&Ih1 z31b;e4$D#S&MvT|NV{IWZjr3PVa zXKS1#`D}6_r%H$L_ZK3sZa(5kavD%$u12${#HOBauHn2D>6w4&$raOsm9lz5p+$S^ zx}fH*Q!jUBqL@u;K=L98!h7Ut)H_rT@WM65Aa4>{iuUC*)dY|V9Bx5fx_6@`iV(7k zRW=V3{Cb2Ay^v2lrpIUXDpfviU6Lgs--xh!6anKZnh^9&eQ8u4YM*M&nSh084KGoC z62ZOr5J2X;)da*WChPrDPF?0{*rD-Yn7|2Bz5A|AR$qqe9Fsb0zgJ(P^j(_{*djX1 zq0FB{_o>0;=1B2}F zDr|&%={J&YY6#G1{&iq0-lm+=r~TdSrwpBm0Zt@P-!JY{7B#^qu>|kB-agk6KxLuO zs`475X*LW4l*;eA#c@ofh(3@Iyvfa?Ug}?((jhSX%vWCN$UHr;4QidYOKpKrW4$*61*p{AJ<$k34+u zcZHK>!^#xK>6A!T(~HHMbRU#N73>YzMXVF_b#Ce3zHai<7_Tnf zWvD4c7$X-0%Mf~$Thu%3b9$XQZ&^yNIt7S~R#Ft-c<=)pxdL)*`(9K&r ztVLe=?Z^?%6t@|LkM+QAU#@cB86I-C(NZ`$3QUe)fW4`LLtk030OEI_!Ae22-~eOB zzS<_Yb$tFlUI%U|p=e2!CXzCPzLnel!c|B4I+eNVg9=xUe<@pQbuAVgNEZ=j;=)*L z%*-KKmx8&EeNC~MlvcQ?uYi@pI?!X3h0B>QSq^raupt>g7Vkz6<)a?~plRQdFTDPm z0}82IlYCc&2j&0av_6(Tl0RZq>1;*`MZJgaBg7W7Te5YfWWthGzpH^Vi6_{42rSo5 zlyyA-!)=gvgl5G<-ffD|oy;lMo=2$2-RLka;&opu%exe721k-Gk^X)h2;#Yo3QsKD z7|!xr@zD_L38SCqZecHz$4HeTimiuZPwELzX_)Oh4h4P;#BJ5l4q3S{Tm_WBUwEukq~2l(fD9mYT` z(l*q2LV*&$ETQ)0L9Lf|>PvpX#H@hskmMuoU?)+2L?kv>(cdL$TLIR13|2#71siTs z>T+DDzNyR)wzz8FdXPYaS&d-nYR(@zF}$4S%}cxW`j{?UvLBZpAeV^*O=3$*TiNuu zmRWwQNQKiU!yzD-ClDl@#Gc=r=i&it3?wZm6g*O`kz-i~&adrBWZjwimuTJgd9zl~ z3p6M)($?$tdU}lfq+Iz}VyoANPCp^RRWbK#!pZ~)XQ&1z)n?!6y)$h|1< zz|0ROG4D23RC?HIgF05)&B$ZeyBE5O`F!LQkr!TluM|D~kZ+n>vATYgCN1p7*E}e? zvcM(H;BC6KJBP!{6hN39i62*`bC4um32;aKn%BXNewsV<*^oYbgeT62buU5~d}_mu z)f`{vZ!$$g1b;lIURT-y+@d#IU;^_7KxyH+Yn6dRd2|J!Fp zO>XPYxCl?>Z+G~HJ!%)h)+JN_%}zCu2;~WU@o}Ywj^2$XW>euBA&2ikK$uG68EeG_ z#d>p+Jk+w*J-_<$))$LgQHfjpbSmEllIdoQC^zK+(Iun!bd~gRvwW|LSTG! z3-R^Z9=dwiP>kzWn}>%We4TyX!}~& zjNGFAO>-$(rLMJ{nzxB{F!ny_Dtf4X`Ba^a51>#9+MGz5rQ%uj_+c>gLnD@A&_V@g zj!^1r@NpmRRX@Q>R%`3m%{tBMR4MUz^u{0Geun(l;jL4cT|YbM5y3-cKYo}YRav^K&cyeYNUzB_IxY*d1PLib zPGScnU3I?@ED(VEE|D5y(?lA+4VmtZyh^SIdd_%1wU&?Lb9;E}kFDD8cs0ZF^f4`$ z;FLP1O?bnK3Gq0zH49rjj+rUdLZLHFV)@>RGC1@#a)iEph}5Z2mMQ~J;uzz@15NK7 z{_A|V(mh6|g?nqNIfCR8-r>??biWr)29wk+8sgub`d-JQCwJRQCbeR+e9Szh#K@Yk z1#5~(mGX|HR#F6wEeMuEk@42>3~3TZegqAXD!Zy`e%k`pEHWg~lWvD=sF=u+M6l}O z6fHI-MSl3M)XmOXvLEc^Q0$Wtl{Q)xNA9!5^QnrwY7;`5De$UxRB9els`@rXLYy=L zQPm&ZnQGNFL?_Ba2qzDkF3fqLb$9W9?2t>9e+jnt{-5)=<+GDWd~i03CNT)ZhRjaq zv8vzr6|I>55p``kHw7>_@of4Cv=>o|O*?*g{^RrUHVuUhh6)5%JamE4rw3L7dKSj8u4tUi9OZuFtpgW_fTW87w{QKYC>-c8TsMxO_E z%S)C`P#9(rr$DsTw4WxZ97q`epS?D|j?Ud{i~gJ77pj+p@dMW~klz#$NSxkKL=0mc zWD~{~1BaE9dx?F%KQosjP)O=aQM&eN^=1f}G9fZf`V8AiR|3B`KuGY4bLCfAS~l7C zUm>#9H-mh$CFjFpz^Z9bztiN=ZrL~9@}oo?*jE}<`oujNWd1%s zId&&eVON}ze&R$JY8cySQfK2Y!lNR2t~)aPuKrpVtKHsVfg6ufhH% zJ6+TuYC^@aft}*itjrAKv4Xe(ZT#0KLgUB$s6B`@h*s0(YW4zaq$=U9Yi*;PcU5dr zO8coc4h83J!j$GytB!L|4-GbHD$)gW3Z{hiFC}z@;g%HsKa+uz3pYS|4b^@pWap|Z zduMq0?J;jwQqI1=E!kMSq*XwRq!P&)KI0`;G9pR4LPu=$<#N_)p{NzeuXnnbh*#8i zyn70jlZaG>^1sEE{U%P9&OA6&Hg{#O-yvS=91x__ixymCh4`r7Ik}U**>DsMK_Gmz zvgn&HzcuoRh5g#Ecoc6-xYK!VsC5@?mc@V{(Vn!wXt$=qWB!m4svrWU9gtWs=b}r& z4(}{dIfvz2Vm;!sa;O1vTn-@YcC}?7ka7us@1f&`4&_R6qJ)nII zF*zpvgKBqsuHa{u1ut|l>a+x*$W{<>+Erk#oO+cE7WwDa^EWpO{xAxc64X-))6!bi zkL~Gjl95FW#JBy@XlTsv#J`aBvt`l2?M~@A7Q)Y)B6KDQAmoToWp;By zv(Tzh#fMs(U2r7W1s95arp)s1HXQaGaX8x7WB6h&J+S+~d%1>;n7xV-K8;Q@Tb;1m z1^DtRrWF-tK8UnqdEYS+cS3^%Lnjya?jMqLxHbQN)vIvq@l|5yy&-PS!H=(|xkEE9 z@c`NUT79OEcu)FhAmaO~H@S%fOZ$kdH#6sw^m#kJSmf!e8RDI`NFJegVOE3uO9;&m zLWdFfJSJt67l?7v{ZnFI)UkQ3 zHav!1{;e!5ORhA1R=?KdA8W3E5+4^AWAnHflsLmuYx1CvC|?gBn30?kf!z3uik-3u z;a>T|TO!*?fU9_q8Jt{QR<3$f= zk*3Qee{BY7QOewqVlQg-^X&S>?Y3PR76wIQ#|=i7m8Yr(t!8p+zmoefv!NMVx<)4! z-B7}VTDIY@MC^Jq=re|Q@5>q%O^`i%0)1GaMPQLM*jY}K`C}3F01h(I!Hmwf9;p0qjw*;YL9 zg7ZXB)S7u~P+Nn+zqW_bSX|F*`l<3L9$6S^0JRSW*Awb9YS6-B=76Rp4}Bhimp}Sj z11?3j1@Q)bYbl66cX;p5T~JIC+}ejJPyBJm9r0S+d4t+V+O-gk?$FJaM%>k8Tb_2n0ATk&!L-)1`$@SR|7D!Ta$P#;jWB!HRpOqx#SyPySt1)Jq>a=d(5fa*~gzyTMJw z1jEDIvtsLsd}*n9!ukPkpLk>2>f<`2IM^>>2rEK+;TKJWEsF|fCa-6TOyx(EVVTlt z;e~_bNKkY|hiY10mR>xE2lSh1?-fi(#z9QIN0YlY9*!ytBsnt5W4h^Mmr(QCi>9XV z#3uJ<7>s~(832r9!WJv^-_AtYvRQsu`tv11a$t&HeT;t0a_^=ze9N(ND>__Xg?2mS zbh1Pkk`hj(zVc6%arK*|enjC4bV_&cP|*QxR!5tBCZeW9T+CD?-IT(q6Q>-UNawFU zy|v;&+~l$s*)(Wy*pV|RI0+7PSI8E^xlSEwq)dcLOQhH8m8bWG3MQ#dEh4L*hB##E z_|(0?A?&n8NKWn#dGFJab3$~)?(XEQ17%e88=#QsV1)ywwrJzL8YJx*y)gs(!ikCs zp=x+dnQzvJeTzI4VSR&fpin$q@KDK^-eb?i5;Q z-;0_DGdhgC*FLA;OOMjGL$7^8`gF6F>Y&P6B$Fve?x$j_EM_;e)_anux{;asP0;i$ zNi8~)b8p{RM&||i*?GBq@8S3B^>=lL!rQb#-G1_t;U*gWHg3ZLb<-W38xMA@1@zu* z+qvA$^1on=N2ByjP0D~8`yhdG%_#O0b^h<0k1hWRz@aP@c~u;@J6%OEim>=SE5S6B zxyA<5>+K{ai(27fGR5lGA0bMr4NXL!Hv8Gq(H>g`HdKgeVpbscvqIuU?Z~6_?moMq zkYAHOju8{9eoRIcJ?9kGe9 z(n*p@EN+teEFM!#en0AiI}YTAmLk7~OGrgfl=Bu-bdV{Y>-JaK=eo}mWkh~{q?ldD{bUx$l)5ShohXmmt-->+r9)6&OVeXE z(&R>c9og>NcUEplaTSJX9>JH<3+Z?=xR_-aCArc$QxC1l;%s3k4N#w5!Zv3aq)9vj zZO)9jO}ExByLAKF_@YNI_y_{TRcPb1ybarewdmUUAa-|8F)o(gvYrsBeFw;KRPcOC zp-T;Y_R7Y9!I#F@C(}AT`gLAqy>8R*BdZ8lSYY5RG9u}6t(f~>M>;=f*~0f-*X?M) z=0TW*`n>U62Ri<=ej6=_J+AnL9$owiu?D!Uc8j;!h@*w@iw#6H~O1d7FEIF_8QzU#N%lBmTNOIgS{nk?nF|)6@ziaN+iiJtjNiCyo zY4Dt&cTLkSEm^>nWp&Bs^W^AtJK5WHXx3Qft$B7H@Xa*r=k!!lP>rRv4f5`$Z48^~dr_RCYU*YV_{%87-iVr3i6jf!AE^#DFgPBJu|N zlBynm&`{1dv$aYtTA>Aev4}~PJG6L&StMQYI$8BEs#h*g(6t|wvxKT5*UuAFUU#K1X6mOrJT+R$i_i1seo;9q3(+r_1@xT zOA+v|syX6yS}a~ao5si31i=4}LfH;kSLqcEwZGTHwJ?<-&9 z|ACVa+6`5tz+&TJWrBO5C$YLxKm}_kJzgnaJ<0s*XWcAWBju1XgE$H~)n3c)>xtsn zg+Qu?MOQJYR|sF!h8$~}+B_(*6!P>+vI;RPwQHVa&MTN^1B9W%b(Ssp=ZV%7Z zhUz_otYv$FE}~IZkd3pM7YX-uqrOuZ%X2aB2b5H@SuRN3LD7p4ua9|yOQFY#ubCa= zd!u^3tC|z`%R7w#^&tfHZ?HIU=5TKt3Gpzn_o{+eK>SDFPuC(e1_fdd)-(|HB2YLt zz!)=|cRymLQf_%Mj!D`%@>#>_{%T2oMEM9w>ri<)59N6GjIa0#)Rfn$v@ExyV?Ibs z-SKDaZN8FHKbNlU<=9FvK`hpCjlqd4OZ0Z0OvG98$~>UPTybEwAO&2k6r;i>&ZR_> z0JVwowFgu9IxiGmR$ih5W5^PdHDRoYd*+UPMG=JUq9uF@bT-A? z=ISC5Rw$3iTN|Vv`Hc{`2EVyler)MRHo@7Y93k%#-uZa?{}HTz;gg5!>!5R#u#Q&A z8679eQS2H|@7P9Mht+F~mnEOIY|$)FPjI5B2A;E$I%$Q0E*B=&gKiJX`STVFU`^sE z!Yc7NgxXmL{52caD{{oj#q!19DR^y|ZdS=6oxLjT(&`ik1rn<&po5TAL1$`;ntUDF ziKUlKTuII>nJ0)*YKj9g`b!~nrv4m~JQ*dWZ+`WZ^C~rwcu);r4?j%_I~lN@baPbs zM6hk(u4p$Z`p7A5TSM}O;{E4K3zheO=pJ1A|`V)B<;n8!Ol(Dj!;XVc3?bGaT<)hKwg`$M`6799EpH4t@>oB%=@ekiX#;*qIAXeGL7V>qa|@(P%b9Y7S8vl6PBjn)3L;p{n^_`{>Iq zcK0x+6b43i7Fc~TR#3}#cNj8JZK)MODwNMUD)D~MWFR4@4U(oD$M0QBkC&W&Z6QL5 z$nKS%%t6Hl{>hTm!1^qS3*CEqC@SdNqhqCr0mNE=C#0LmG>^Nd$!wURMXTIS&P#iO z4w1sP$xn1FXNwu|pw0ZNz(-!Ow(#|YsR>T)Y`m{WStCDK{Z%OL72=+FbfVyLi&LCT zXg5v$L<-9Dz%vr%@$3N_48d<&(nKo0lL$9E(euG7%ThTaTRPO4Qrlo;twHS9k&%Nq zks8D)Vt9BXF|K3S1D$I(>KX)p6#`gQy1gO`bYT3b-B1L$sR=8|b1c6?jhFQ;g7ehZ z5oU>*5SAPuomCBGFFNXDT9+w1kTRk7OU*cw5M1Tg6z6z90q-&++d5`7;O7^0D2~j}^f0bS znFt6A>?FopPZ-hpdP+RSppH6>e4Vd^cNvB@ImH~xv=i_onPJFE=Kf?iL=G%)ZtF+* zR@$+%vZ(jl+gI^Q?w!)Os-U&LfYNg1u4mn^vjLOOMwvN49)E){G^fYbc8N z)?pzgAhnRdGlY~E=qR^cztfiYCoPxf4&Z)HZHr&sPh7wJ(xeb+;rRWgW~H~N++N`O zYz1p{S7A^WUR*Bj(#?rHuAag}Cn?N}3;CIkK)y47Xyl>C^jMP}S}MHzBeF8>Q?vg} zVZ$jM*HH>WC(<_03d3Lzk@QtpeipYMfXxxCqH@aN?FlO-Vl7H zi84g0ro`ZXM5hyDhM1E&JTgP|lI&1w;pPg-CKzH0Q9bm_w=}phh*!85w3tNLGm*F- zey3gxe>DWFQ16)%UL?>-6+hdTSTPNz!AI^j#2T@%9>JIOH80=)O%0jjcn*2aaygt} zl0A6oJ$>Pb09E5(Ya|Y|r>-CMy||^4h^?cQ&H=So8q3Pd2=|!YsfgRWB!!lJ#=AHD z%I`;sRS+K)u(rH#vV9Vf>V|<6mpN=;-vTLx_S~doJJi#T@Nt8n@v?qh{d(*z1pUJz zs5SJZ87X3B%Huqnxzh>Hc}ZN!D9_|32I((Cj%dbyuJu(ZORGh}n@6qS7-q_>$9g)g z3r_1qlmA$rTKj}SoP1rw_G$OsKXX~Xw`U?yAOBaoM~pE zv~5S6D2z27T0Lf?aDHq*7${;`<|cXNXr8}cq!McfG^XaizM$jORE9iBE4#2^VpPDF z*Y>MMMKAgd9$~JIK{eC`W)T8#0Eb-qg}j~QUVRrxiHv0Tn0v1`<>_yp=Ztf!gafnV zKXk+##5fAu7>P<%@C>;4@y?3FNT4TnLfcCD2~yWjggkUYB2YsXD=km>%=vQ$^94~X z{_m%;*o}MV0VvKzuA-&X0F<6j#34P-e~Ml34OwGe4wb7BNU2#6T>j;hMlxYKp?wv= zAD=kf@+%M}9l1Cn@x-T}hZ?dnInUxjCuqca+-`G`UGFC1NHa?Uis%xyjR-xW-6ME= zXL!gUH(4w89*<05d1DEK4E$O+vvyQOfFe^+7)AQ_+Bi5wx~V>|-Sv&HJ|IO^Xk7gm z{y&i9e=2`g4%yn>k~rGZEYes-6&FpwjE$&L-HdXfA0ldeVozx|)D9m|gP#zJvS4Gi z`_orwyS3o$&-mo4alskfSNOvb!qD5>5U5p~MJz8UD&-kn(z$$F(p?I|J*ln6JtE|l zO1}KH0ZXf0hvj2AOnRo~z@{0^a}Q1E&I7^2Ybq?&A9#%I@QO4ZB*S&u{}pa1sNC3w zCiUxk#@NR#uDHwpeJwGA55TZOE;y*Qf7iraid!RPRU~(ra0OG8C+>`@>~BJv@3WmpH-KOODr@QWuX;A6`v4)CApQ0*x-H65TT9t znl&ao{`7(}4q(fvQFEIHDK)4$He&_6Lfh#-&TJL=~P(OZrlX?fxN1R$Y z|9?kc!5U|$Aq(!FuXH!O>`TEHkRlG`rjyoY`|XlxOWDYE0vzy78aw;9*1Y^458yyX z%sDefS0+oiQxN}{qbCY-bdO_I+rb?$T|qUU5^fJ0_Q|l-(>;SnbRiA9J28r0u*>)z z4(D+64AuZC*z{V~Um>FQQo*hwDr1OLowH-Mr_(xGs?s^o@iSRF6*_(?6G+7}1E+e! z#Lq>2c^bG9gwXb)DLkBpEaMG3RY;H4cm);mbbIv;AZd%4t0$0I9WAC0COOZI7N zy7v|%=irnv^F_pJP6ql5^K!1a_GYg*>^7LZ%*I$w>@FmHJc|MW!H(xNwTK8Q_U$If zr&IDLzC^lpWM7~hJHx(S_3xMA$FF*7Ch2fVHh=1+#adgsb&0}{m^@p{-5kD5dJ*n_ z<6gg3v93sEV0!X377Tr5+>d^_oz7pRsVrWYNc*I8^*=2~KgJvTc4NJ&~ht zN5I!mt{EsIxc`sjx{dYydu7x1s$@$_4VVIR0)NWQsy>#s~O ztJZ;R+lwPAf|nRYJnBFDx9UGuUD^9L{pNwrn1Qykr`#DDO><}`-7V7c&enLWsiNaC z!Boj)6L&4wJl1(*fFzq7r}4SBdmC#d;tI&J5M3(y*&})GI}nqv3#X!Q?j`$ z^V2gMJoN(-mB0wANbfN6k3zk)dGpLCxIAS4vMFA@8#ho}QG$_%x8)rxt_E1VC&!KkXS@fTwU(h=S9a`FhkRa ziCSmEj+*zAVV0ehJtt%R-N*r(d;uU|d(i>qn5!YbX6Y;!8PFv&2FT=<4zLKaZ@W>~ zS>}K56h;hQPtrcNcs~Ag}V2laL+9V>Dj);yo>h6Gw~z($^XG=dndZnj%~MHw7zY?jb|ALsnc z`$S&RmV}K|RYywWMw%jl?YZ8DoxESu&G=UMNvgv@KzDGB%|$Dn2z7kf$r<>G4`zx; zI;J_H#@5Hvn1#cj6r{*)yTVMv4bo#%M{z<9#8b>VUM#$4SeiBxWCCK}+B-gP%A z6=m#-!C-kFsS}sbj zZgS=Cd^~+kxVaiB&lWnNJmkmTx}#A17ob^;{ZyeN!yIl70rz2WKA_p*2Or)9Qrzrl zHEl2yl(AN<8x5uFT+nLiuVh+zDJX}2w-AAa`|2EQ4EciD?m)>hSda;`{CfK_m42nd zyJ*(tR}(baAx4l$#6*kaR%XPpG)7HfiRht}hpoNy$`o%xNVch)Hkjb>l7;!;<3lO* zf|hrEyfQrfH{(yUFl5b?nmDzZ;44^_RB=0Dj7oHuU+D?%4#5YjS+)T+i)|4T$#B&~SXqY9*C&Nz1RtO?7r(;{b7Gbe1}T-gYbs zqD^>s+%%3;;4Yk_C5>es<4neU_vXpq7@fQge z){j@1nmYGH_tbCE0;t}CLygSNh5Fpbbi!;g%TQfKZsu$)vav`om7CqA{@3xdMUy;5 z_}=yfed7G4VGzw8GW2F|1k3%ApVafjD#A2`s0JTjxjsJZ`Mrf0MA68nZ;?!}Q&}}r z;Ln1St7VO2RVJe7JPLi=#Mn&TSZZjtWb1%N(w8H}xW(pfJz484a563<7WgIct&ha+ze6&v!O$IhF;oSDTCG-NJ};)z@p znliqFi;7NbJloOjb%45$e!X(O)n*U#Ygl3lU$ni2N*>R(`Dgh<-~_$fYRZoz7mWm? zmg-M&V2&Hz0f9X!0R@B^;Zchaox#xKGYHiGN%VAAK_Mzt9CE#KqTTH7|DC6nF&}EW3WJjl zwPaG>zayeQ*QCw|7x@MPh2f1YDp_Rix>-zyhf^mwfpS4FoxBRb=40<;=0J>G4=VEF zqJA@pIosM(9MHnBIJ+ZK)5eD%jBsTxew|S(Rc;n0vJ>m+=qTb9F-vRckgSI$-^iT| zY-q6xbf2sjs~YovgLOP@avHnhtV2P5Uf754x&TEGu88O*fA!QAwbD(4Mt|_f+!yvc zjUpe9GaKOT6awQGBc!h7_kPrbiM5?9_*WT+yMDVHoi1neS@wioYKIN2P3MhvSg%Z; z@*rA?9)CYTV|nY@^U!D(r7-iBcYN*A=Xfj6ng4kB=6QHA=acZ~CD$*MdLKGGSeSsv z=c8{4Fjzvj&fWRGhX0>|jtf}HF%kEBCPSY8HGql=g$tEg*9+^)xFCYIfH z!i{_>%YNVc_kTa&2SJfiMQ`kw*{-N6uCy2uzg2W{B}$bCS+RlPkQB?-!d_6<#U~bs z(oJgdqLwbL$6l$1&PR!ow4-6Nny5D@MXVdzEqz%A<62zxmYUudcQHcqSiUn>dx@hkihYAkMMLh~Dq* z@`+j}Izpe+GBdK0g2rm<%ap!iOmVIAewb5@lHyf$hFPYLlH$MTfFOK;dx(5KQ1RJk!1d60)rds0 zaOZa`(?e^>pO?kioAgPEOB-ey81Gft_%ocZ)Wu|X)_aslz+)yQStff5L5 z8;J!4+6pSC@QYnGX=EvNIzE^fe7*3)TB}aKjzLhw9C`C5i3=K--$0I-7ms~l_^+vn z!hwRt6Z_P0sU=$?4Zkb2(U@p{0#%II$i^4$BFT7-Vk8$(Vq1FK)-;;)vil3W^t;lX z=|;l!i;s`6aOYNU9d|dMVPqElA@2obxj<7-P&V}>)Nq{qS{9><{G=det#@*M-fvbq z?#(LG62l9MGJ=7V1a@J3zWii%hcKuoCHV-e%C3|ed5hK#>6jxrOkDeN4cu~hPSZR; zIp8fkfxKxDcJnUfT_(0&;ZlTmvH~xX;e&$Hm$QtjoY8C;oHktMjQU%!t0%#q{-i@# zd9%i3`g(jTV*>>kY;%3W$(;xM6>q zn`-m3{ZVuti;V(75d9@!h(|Ia=X4|IocVg=t9LDm851Y& z-bf7vR+2t*EPU(dD$ELKjP2%Ym~c^Iq156AU9+bCVna);BnYIeM-|rd8ysFl1kUU` z4rpn4`E`Ajqgy(6&Zak0Qn!qKTwE<^9mVw1?Ks!v?EwpY_ZI=GzMdueYXe?S(U^wa z!iiOVx9lDZw@FE`DOS4ChY$DzC3ym0PCd6HZay1-Cer%<(BsS3i$GyqdmZHLnq@W9 zUKxBjKK5e*?N*^Br17k@3VYwX>DNaUk*Y6l$Y}!S*noSw`PgQOe?iRXe@sNq4BZydHT;mMbD+lc zR!H-MB<~GP?>sp>)zVdkXbdwa~FaGhr~vrtNdtbtr-^P zBos*NqyPEI{h!JA9mm5WqrzRpBcqGDHMe@mb})y(p!GW4!xqVSqi^>)u(N8%RT z%GEGJ|2(GP5|$L7w&&&$1BhiEB3F&iFIA5jS5}_ZiItEP={?-#WWNieo(bTOjZc4# zq(&ZPsu1s@(rhw3c^oHJiDK}kbnMvcPT`%sQ;R?Q#HP4$|Li#;3^XD%xuE{GntzgD zvRHUc&6A2FPweOcMU_;SOedAT8be0ytu5|ekGh)sJ2^RBT==~aAy?WkLwKh| z`%>Yg6Z)|iHV7L*(*yy1khR9H!i{65U*Do3|6@@Ejo@-#f19u}Ok*`op zp-blXE~Db#*ghS#i(d3<#spuo7T|v9*<%uBzi|FhIbi2vRC8Svk{lZ1ZGK!1sk3?~ zl}d!dyLG2>`#RBBBIiWA-^@5zT7zN|9Zc{lcul-`Hb!jhi$wu%TO&6E7~X{;h3gv_dJyFR$ajL|n?9GQB?aTjKR0y|Nof>@ho~#zf-aNRu4=?Slj= ztpJ;)LivbOZd6i{9+u3P^Nv$}7e6nXWj`&D!s9F!4;)Z&RNB%w{{ue$4fpz9g& z{a&f7ZjaC#EYof};PiOW+s;a(UvNn4^SpMbj_bmf^0%+omSdjRlGVVTJ{JvsNy4-h zwj@{K?<$SO&1)!;#Cvccb+E%D5KWTPn?a;%EowFd0ftfdY4T!Di3kk>&@H0Lm50Uc(z)A88#5#54z4fjGg;;(lg zf2}I@*8WOL9_X(=>tbk^L+#;^9A+o+v(Sj3o*!omJa!DmM8ecj)B(iIo{NP&31b}X z(GTUC=HQ_!0A!~7YO+6#*WTjqd_7rcn^?XucyrOaOuVC~TRmuqrQt41`7#Ihs1h$j zFf=kYXfCRvn`M&u3=qt-GnJ`@$gHJ_Mh-1*+zCuRXJnB=P~RAD{U66=^zz=JG&#qf z#P>C_rQqx88kS(EePqUSMYl?=fiTQlrFH{F6Srfaj>FYYHy)&hJMIR#sGS`0{L1a4 z0m}ghk6U-z0yThME*{e!9TR-%SqElBk^mQaj`CNT+DH}2j=9hJ`tSb)?&elvF_3n9 zAJ4=uIj6)rV6k*jfBMcgysMzJiDlZsxD>cH1$U5#N=g0GrsfNy*UW%8NIERAgssvf z&vrb<_nWg&EX(atbon|_=ThCrIG%~pT#S*ByS7(jDYxLgOsLWDJcL5`l03XZu6KZ! zUqM+x<=QZL*>m;z6)FA^kaZpUz8d3i&C5KuwxP*>cW&~S?@qiLB6aK zgqkqjHF}QP3U1<+uF~Q>dY>3Ud+50=-9G7@TChJ@t*Tqg2?KW>Y)S41_xMF2Ew|i(M;u+ioZ5`&G9!R*l0TIr?$e%xX96{U>(z%G`4t}4o$gFb^qND zvvw!JR0HVJm$c(tr``0L{R zWs)Rg@p3)xfc>P=�v8Mdka|XmaCb_0HB80Z|mONNO})jyDY9irYnZQp%sdqLdu$u zzFb>kbZ%=g1d@Y=>TM*uh0mn9y+^(<>~DOUVAy83U6`N!`X3RD7!-Igd%dA9AB}6K z6N-|sN+DTZ?_AlrcFmu{z0=sbI7e;pwC^dy+ZsXlN>`CQITvigqR-N7`9h=UeF?7? z+~YI89pQ5!8o6WM93RQ(@M+NB%n>6rLw(}2Cn|$5R4R4Qk~|2P);}wFjVjGB`@$it zj0;`eyb@e}o2jRfh*g1qxW4<=ZoyBJFbA$+L*qw}paU);O#gpWwr^CFf9<>4r-L^q z4bTze(&Q`9$N4xmL`$R_-_+$N>&>+!S(BtYSvI{95MQrTavzLV;ldhKVkRKJb=+Sv zgzhge<%6CUBq0Y&{+0~g84uzg1`+H*f^el_3BlP`8-?@4vGoMC=-EZ6Xpw(OP*3(# zPl$Z{94fdhZ7lcA<0RXN)UG*75k6i=X~y&#?GG3ERlu4Amz_irSV-E-Z$;UsFf++A z@!8@1Q9IsE!MnSIPd4R8|iaCs-J zSzN+sNX^G!W0X7s`aMKT&sY}@t2MY_o>gjM!!1EX%yB9 zWa~g{l$wq7c^h7NV3yI>ixX1~y7c8!!m~EWx+T=yr$!lq;WZH$^fC)Wjwf#8 z7U6jxSl>vGaX3)AvZc?l3!B{VL7Pyv-Xt>*fPlBivpdvvj31>#O_DU7^N*_DDdU~wy(BPI=q#QaV^P}#9$4pZXV zvArajZ3lfUO<73;aT#fj3uyTU;PcU%mI!-Ei_OX}T&NOmJg_4yUTIf)j>S!E9!tWR zb50?${e~Ei&mJa4Cz!5(N_>#YL?N-VNBIYcq8L)j>>l8E+eNG~WauQx0}uYwUvkz& ziHqN&uw@bj50DqW`NEFmr3vu8Z9Zz&b*aq^aW|=1zsX>}%V2VP^T{Q~noU4pG?y_Z znZ-?yn=7+M)-zz#9&;@m?<4e39oCW6zP8Tpi#2+BF13PYWg^}CJ+mG+`xrqBDQ*0x zYSH`ouzajMS}YGi2St!RL8<>uN>qy6-Y3Ww{JN&|nn*cEljs-0sH zq+(UO>jw$*CL^Xp5#&l7LoD{sspelgap)(jmzrx;vfbtJy4tr8BbzYqj7m`#MzhmG zGzD(R^d&Gmry}_MzzZRse*PIBHf9hPX~?y0+F|K$mFDb*gfikje+Vt-t%nAgCEa7d++~ghv!gbWEm8YE7!ViZ66*&$eymmDU5~ z#mTdZ$vQF%f~1H>S!>Y?*Ryfk^u;M47;Q0IWqY1_H+P=*mf$BC$j@Ki#z{Y&#t1cW zmhi?{u7+>*U0WIL#F{uG;Sovm*BeuOCz40YAZfS4@e)&7!feKmS-pymmjsvm%s=Q2 z2?El)J(4OH@WHLC6K686)0WRPPH~X$HLYALbH__?1aUMiX(LNxSAmOGXD9-}Z;4!&Oyq&n>3@SH1#! zCl@!8ze{jmVblxu0K#RRzjY?_=ry1zm1I}ZcYWW8HoIY*VFOiY$s7|UnG#A(VD_4o z^N=nK;BfTmwnz3pY!!zq-t*yv%K78gKF(hfG`?^b7AhZNr>;SyG_=ri&A)w7$@(`) zuU^PwqglzL9OyEnGO%pG3Q`uAD))Y?3$rjI;95~BSGcUF)-!7#P~LD+5rId7sYtt4 z2AV5ccFeo?8W0f^Z=v}zF$9rod96R5S3<@L$dF9?DO$abxX+F@(5us{q6;%E3*P*e z*sHI9s=3u{9n&e~J|&7xNp~whA%%|??zNhO214=ziS(5m4;s}a(y*4Q1q*1mTEyrG zlRLqVN!oq_`{kCb8LwKnwUn}%S)1;r7ThXx6c3RA9 zxu0U8T2gFcgT?KzgK}X%3nnm5*i|k`67a$<&2M%6B9_8J8LJ{YDLfh7pgGKNaIL10 z*7X7wlPvK8SC3z!sUJ@%qL$mePPesSX0%D{jx{N);Zk~|buVZx!YQ7yuXs>eJQ_Qf zUBBYzJRG4sgKO(Jxs;EZq;if%p&@2(s^?}D4_^g8bFc@)c?oilc|OwEbD}rc<`BD$ za@BY{)AhruMF3T|K`m$+mkISS#K`R*+pc|~s^yjn-Eaz6D)U;I@TVPqIq6q;zW5vx zN(57;lB%e}*_$Cj>EI^>Kfpo~8=1TTy1PTd431G~I zKQFnXtXpG{LhPbC$_OJY`>UT}$+|?Dy9Xk4Ia67RE~@G4qg-FC@kD9$6@;(FTmMp1 zoIXa_fu6IhSa{PB0N9`5pFu_+shRHRK4cxuh39G)n(o=%)??7>`ipC9rAgACgq!eW zQGHqOMenFCFeGg@oJ-FZwELENX5RiY=3Y>QK4HIk|)!!O*gIeWT<^DGeeYfYewe{{dI_3J%^$ z$z(Wt8Kba49e^Xy18xG=9%J5{W_i)GV0sMJRp2#UkujTLMY$@HWmng>D0KNpV0=yv zF^?5!(GB__wp>uc6^m30z8_zx{t-9>TQAj8skPsXMqH+;LUv_@T^2T?bJ7+`DACrf zLXD7*L;kAHVg0zcd}K5ukpCZAorlxNhG6pq#e9>ny>GLS?_6$U6Pjci6G8*ZfSVw2 ztizA`BwJ@NB8HTo-BNHtwF*qRjsdBXBSl8F+n==A{_qoA-}B*&KO@4N>Z2j_^upXb zTH6j%#NHP<4PPf@6pz}((^%XylE>%+XyGpm-sJz0k8WXgFYqm2-H3dy7P2oBc>Ma< zz1^F9Sp@i#4LvSa_l`VHo+u*}_U3Wy6zoU?S*`B;GaYA7Jda7=YO0z8z6A;cWjL>V zY^1Ec8tICowHmO-<#Kl-WNqw^)FIj>?qdAw;MP!xzi`%XnGYU2mpZba5fBe*V(F(+ zjg}DcHLi8j2X2glDOi{e%Yj@mq9_N9B6cGyUla`T?Kd5a{yz)ot+Zq-i}fvjxHb0- z=9bCl7Kfl^Kk4)Dudd2@DM397m@kI(a>&+EFJY){`g2mt{N;U9*|ESYHR+y{^nZ}0 zF$L6+d~SYFg8Wes-2AvFxFQ8AI}*QCCe-|!_GeHx{0KVYY;QO!{lDh*)FiwlGks8 z2E|2YFw*i3WfEl53>u+-e>`kA6VGJlCUR#pR`tV$cUk6?U-Q{ZwD5|MXO@YnRnp&P*aK2zDvt30QuPK`4=Cb+RpeVk@KpTA$}<LG zMny_DnIL>9F`(qDg6%Qh5g10hCXt%IeN=|wcZ5VZHzfCh&F3gw%+j-Dk|7htFKvFE z+WoE^-ranPRK=aa%-@|NFk`R z&xozyc-BgZi}P35@Jj6%ltDW66DUfkTLT>cLQ?}NTM)J+ z1FF0aOcHx~$Ob;$j^jE}YBZFD2#+clwDDSER&G$IE~Hmcg5USBRPPvV={@cQQ3-&9 zjZ9HqUB)FTFb>>^ncO9h7Kq|rc~E$7kVE{J9lpEvAOLb-N;yQqO2Aa9aKctuGpqob z!Ilb1$4@ke7xy8*sUgQZ{JVl$ zJV&=VNZS*x1zUJJ7(CJqst!7p1Qr9XWV&YYUweGm57~6&_O_k&u$F@Ll&arQ6fI-V zcwcC-@ym7o6=}-Tin~8Tm=&ci!e|PlgA~`?(b=zgw2_n=bG3Z*AKtH39*l23)#u^| zK!GR`eHe&^AfN__tdisT_?+84 zB#DKXPA+t;*@!%ZozT551&%de66?7ev3yh}WC?;$eFCV)8BoQBCxm=I67lz7Fk}X7 z4=hwYnv!c!vUa_tbwTHl>h|0rg zEGTQcRvS|b3qZWD8K|w-6Z7H!yJ|E}(oH>5B;WkLk_t2+Lm>ry| z-Msw~Ae!7TgpS2*=vrF+lP1PS`XJBw_+5z%Rji?E%{L6>6pj^l_`m6sa}FNvcI9bQ zEa4ciA1K&LV|*9yS@VZghavr+o^H5irwy#Aw%kqfy3yy6BG{?Zxku(0`q5EE?PWDP zPO0EdKSFiFi897L@l9yAE_$_f71J&!P#^*^DTBj+O$^wwF8u^YZ}jkpyz${d`?;m@ z4+wDaxSnpD%9y@150tRL{r%l%4fi_O+VB?{<^7bo-r?;yYX4E36jTn`Y}6Mn-niQ8 zvAjoBUH8KxG$!oUQcGVB*15C_`AnIA)yTr|D{ZLjYIZBTOJ%=BcfoOZvyVUx_QTgz z)HFChD&y@~tPnGpB{jI51v*x)oxSGWdQf1d$Uc)s!MVG*DMtw%A?A?#RdpB(wTxmZ@TUntN^@1f0D z5DBu2+FiYV@1_v6DEs73`IhsTDQL<$n9?6_J$q@!7Hmzp+Y_DTDQZAW_f~DP0dQrl z0snM-M)lyiu-R9Ziqc(@V z^4jwKL}f2oe+xR}^q4MmTDuXa(fJp=T8iNPw>Z80!|mG+=ge1`SZ0LO#iv}Q?bsF% zzYhBGNH?Wv*3)-&)W>K`8q((;~7EpwOuX08U%O`s( z{>NAi{cmh(xPK`SBV<&&oo(-BPrQ#WG?e8_U?;awpH56|S&qPPQRNfLz}5*?n$I0Z zDRlWXj=XQdjr{#do zc4lJc<43`Ie_vIII!X{rMF8xYkdG>cyi4|Kn zcID(bYV7d9$A^lt3S2#Ko|9f1zPl1~Iyg?KeKV_TB82Iqc;N8=VI3(NWw!_3-gIlB z+kBdC?ACCf%{Cp0dB{V~I#=9CMP%G+a}qYwt;6{ZXEFPY?iY9Um7d{j^vajekBMP8>?q(CF zBy86sy_JWVd#bvXdQP{%Ze*astW8qZbny2*TB?m^GHFTYG4 znl8$}gBd(rL|d4^6#5xF^)TZ+zo>Nq6EQleR;^C^Oz4)MJ~V6=CKuNmrFPGhNH8QX zf_IOuK2sHq)xRh=WVy6rY|n5sBququlkBtUw5~mpltb?L3}JtYtj&DOA&E6TMQ+BV zS|+=>p+)L?YOQ#V$Fau@hgtPmk}P#g16cV&;yqcfdS|B@OfYaa zx0_nTXJ0-q5pr$Ajye3YB|jzZUmNPAVAFBT0`f0>Cb*$x2md6=o4F64QT{04P6)I6 zi`~;~>mO<(e0#OA@`DF)gG+H#qGgtIm6(~}{}gta@kkN>5ZU93&$kf^=lt0v(ALFH z&6uV)MmL=;TxfecU}5xV%eGT0#wYQxYj>E`gOV_W2r$2#FTU6pAzI9YpOPg}&%E4O ze#tGT9!>Uj3d=tnmL_7W5aKducn&`{{BdeWf>~ZZT~W>w%qF_Qr z@yxg;PcRglHP7Z{G$yNRgo=)>xh^-tuB>_ ztrQZg%*gQPYMf~xTx>nbbac7Wq9UM*CSbCJf3#4hm(X|(by2Qp5F0Uk-Su8Y$wo1R zaN?ioZaW95Za~fXC{$}*L+b&+i6YqUAkEQy!6=XtT4F@%S$yZ$ge%3f@B-W-J94r5 zk=@>P0(7dDYd5|v*I8n@6TNITY{uBJHy1}a9Fgd%vrt^|Bs{Vdl$`8K&`8d%rvch4 z%q7U!R#B)zr$+Zq*ONWZmN!A}1khM=vn&aCw{6#_aIrHI{!cX1U;|4(L^SXs_qu7s6@U>coRV80|vo0SSh znpn%11EYPvZSUFNoo-d}llZ+*&rxsA^M2EQez`-O%E6VI@yl|dYVcoH9{W;+c3Afu z4fTJGc;I$wB-FgQ&PvQ^S9`O_(Qen>o0SlP@qr1UAqNFlZrSEgGkDl@c^r8x`clE6 zFY|XMlT&?#0*94R{iXPWqPC`6nqirelV>|WeMm_sW>UFHTc~;6_Sq$;~lSgl%jfNT*X`WArDld++THby{8Jv;?~e)0X9r(&M41^tmum= zU@sNxTjGCNWpftc>4TV?WEabPND$6jRgI-m!%&pYrkMMI`E0%Z3V9B9s^ez&o=QQ5 zB1O@4-fH~;hfjI4L4;GUxy${2?fPF%I{i7b%C)Adc=E3i+kG z;?eo;m{CzN$(sb9rQh%!s}~`ia1tiAf_#Q3E?{6jHN0vFF>A@d;S{rY^G70W4vgy@ z3bDI?wgab-qijFz|Gdezk(l)+k_;K1A27Qjv!gMag(#?G)bbw{(V44=CeY0N&n$?r zw5vuCW090nP8>}3_dIIl=@;hEu{BDrTa7%7{G?RLn`_qFKqgq)Vy$2_AkO&8dKxga*o6=B8Yl&!>!MJ7{ zp?+V>M?h*;a;u3s&sdN~)h60a z5v5KHhXpd>tv}=nHMXwcX-ihxh*fD$%u8P!zp?1|mmMc0wY9W1=YW+2U2bBSGNy^w za7qV+BhH;qCDQG2L1BT8_UKw}4_S)iwr4GJ!fBiaTD<*_mK*I8m+j2%$_U8{Y3J62 z{b}qp*dL2lbU>=4sN)-nE2HKyk}M`RXDrPVj{oOms^Z6fI=OSG1)llDxB9fcoro)! z-*yt$P}GA`O(uclAit;=k#stW{a5i=^cH^e!<|fJ%^Y4ZwiJlc;n|REro^~?eR7&k z4+Pzqn45Ug4)2PpeTgKNihT4;h3EKs*%bPWpC7MdJ@)hlqSV@5S~`%ECEjQv68^I`);`|C zoQQrRcCmyZ@vO+?lC7KEUzcw+DU zd~+J-L){pB21GTXm&wqZPM>!~S7IIso%f&Ex${?Tv%+A292LNBl3Y5AnqN0_i|(3^ z*WPBEwfeL|o=VK6TyDR)G5@&mFxoDtGLW0iorX``#|iQR#Y&QTn4(-i+w<(nQ$D1b zMmAyz3k9OXFF=xIu=;+;!z)9)`AGkY4{^04*UAkX~xZ-;rkj~nD~ z#l9@(6Q-5jVE)1PFwo9`T~&4LzqPeh5KZEW@A~n=+{`jAx>$E- zHm*vb+@`Sj1Z97*WL=G~49D(oW4Wip!K9xdFXHG*-DoC?&Z{16(&!i1L4?RP!d?B^ zBsjwP!4PxiF4XCi4SwXQH9jr{hM|d!Eek?D7wMb5kFn^{P_tn7ylKrBL`Ai3L#WBJWwtx?7D>^i& z)g!MimD!KX2lD3%S74H@8loymn~4YG*S_7rcNE_|mW9lDjpg89qe8a{63i}`4j5j`z7$428{NYysbD{z}7JN~$T|pNOe>y-cKBK33&Uz-~{_p~0e*WuZB9rl>?odW& zM2jadqVQQ)7wbekHRXn-;F=>66<^$s{Wc}0zF}|YkurVl1k}R+kDNrKiXB@oN|bvj zGrD9QvZffEPsPPs7K-GzZ#N#I6pEOsALbTnXI zu`cU!v!@kF8!5xZZQ|f8`UPdz>%hIipOG11@Im1H=ItrvYzq;BW>#X{SyZN>z_gpS zvyMS~yYn~5o)}m^(KS%N&|*9V#SfRMq%@2LlaR=o9SQZd)JGW0lK#-NFE{$Kp^0?z zZTw`&vn=!|+>60qW|5=(N=v|Xsru>g2Dk6F+fM)N;it=<8c!>02*mfMk-8%Z)?dOe zZ#AghITDpm#arV@TYcfJS=rr9@-O`wVdGaK?eoi#z%AYdZ5ny=7to~@?D}b?w8&l`MfLF=$1Mt zM>V~4<{p-}Hmw8;>120#Z{e93>nR&IxuTF1%(S@8WfjhMLCs`Lu!)*lO+_F7W}Nqw zA5BKk+*&Rld|m5C`5c!LTyry=lx-VQs}V1owcTOQYFmP^mWEbI$}&Q=t&bh}?{sz! zf&b`ooO}~~5Z_b=11}Ki95d~W-qMf=dIbB?p)SKG6hS=-AzLfNQwK@5r^{-=B(^(r zFef&pHoK>Ij2X@V1&1h#;WMWHv$ll1gijgB!Lcd!sY#y#6Gqq!aG*w{Zg>Jdr;5>V725?u6I9 z1^7xm}othn)u`bkA#xGi8gCKh9?Q77or|rXbX5-MQB(V z&7hu495Ei8Ry}i1yTp;Tg7bV9EJfevMncFgCzS+M7UI3|kP4vPtdm;}bO*am^(Hmb z|EQomTe_y6_oVjvt!OsKT|{p>tAu@?<#CL8J~XpN+bwGai~Q{vpVF_}=gU7=;VHU! z%*18r|FLI3@cg@M^g#ES$&WH=UZ}(f9-=eBDtxH;W_%)sDlJh9nUv}K!Be|;Pjjff(l2* zJP6bC`&D+hbCFm|e8jSmUw#mKNjAw*@L?m$|J)Ci#D5%YZ;tf=#(;O#b|2Ypdstz> zLg(=P+i8n|nvj+BdaHkup`HM(;9`{a42s+=~m*e-mr!LLnLY_U4)QPAjJqg-i@ z$M4`L*cyhK3<_P46vxJ!ss(7)mKuw&IQR8Xrg1k?w!CDP8P=@(+)(M^xp6S@YD)OZ zbg&f~OTSf%)_gR`+uY)T(mAZs4-Zj$qNLxUUZ|G`W#Xe;^kaVelMnx2C%5$UwIKeh zqsjR!SE<~6%W*gx+Uqz^8R-6iOT??pMEe*6AU*K450;G6`Z zJOe(c-^DIpZ-t!}Mvh!G80VJ=MBcdYMxCV^C4@oTD}^2MejG1t6UwVdJiA)U&IuTL zGvMiiglypnNKcv{iYtFvsdPK`K+{!do4vLa3M{8{?r zNy*W?^-XK^1-#_{CiURG#-UC9xT4kp<%tS*6k25WF(RHcJ8*<{N{0Q(s3I}~O8(m0 zlD6N#zeL<(4+`Z-g3dYcg3`k}w-9udAd)+j^=|7zioNH9 zA{XNm@1swn8TqFT;7!KH1)ReWQH4H0bvvL6@r5#))jgcpUFM|Qbykr236xx4LHi)} zDI09s&6$6nV@j zZ`AnyWE8(KTR>C?U*J_uJjN|z%W_yiAZ;>BpiN*>WHi9^7ZIWmX{%Z~a!Kb353c*eNUhd>)h@uv-DNTsHfcL~<2A3kY8cEw1Q!>HBG5@Q{z?=;fUk4<7_Kh=hEI!vUl$R%sY zPtH;wmPVE#=4AL zrYTqymAq_K9x8J%mO3~m!1()eXxmbpVhRP-9q{=VA}FVj{#jJKM>)%)G&DYu#VV?d z81|}wVtoCv21Ml)D`F8lg{Eaeh=Y(g>qogqDfcevuPWD&$?+^za)HEf%-r>w#C5!p zVdc)Ub5N0_dM~5#dq&r)Ecbh3?xHFTg&xqm2M=j&LAecdjOFIO!s7e+4UMa+oZr`n z?}#uvL?8(_F4on9XUHh^IPdYw~_Tgns6`+))Ns4My6?&l;P} zgSU7kDyNiBL*EZQM*<6SLfIE^8{VW3hc&_M>s|#!_8!< zk$EzMF7%*7=z>rJiL(bL+*`>Ne(@psU6bgzX<0X%OLwnQ{|a(;>Gx1RmVv}L>)R=e zX`gjdQ~VT_@8jhT*E7@#&5Om~kP$hS5n&T97upeujBn*aU9HH|8ibAGw{PWevxEF@ zvGX&J%gt_$M-dw>4q=ViI|6koJw@V;-HS3(YH&sNn-y6=G+Q9c9BJV2F;tRoRmp(I zf=ztbLYbQEocIy#h37TS%dNoI!`sG&ofI!!(|-c|p@dw$SAG&darcqHG2=$szxTll z)E#8^pf*wOV}sVCkdM!5EXY+dxDq{BTE~$*A#0v%o33F7LiPAr)r6m*1h(sS2zNd4 zGU}Pj!r%L$QJ(R(aQ~4=eqrcYSKUb`#>P@i0oM#V&rt_n_~t zq(<+S5zYo1WoW1ep|}*3VdSFK+l#EKgN_SGVf*16wwbeg(^H=`Hs3zoVW)pWJ_;`!U47Z}b3ryR>t z_O&mIib3J3ZOZ3pBSXocXQk(orO<&Ylhbffz8LjBzL;gU@YA^mWuO8MrnzHUYsLF% z4iIqjL(MTjCci&s;VAw(7t5)qRr;_};)6f(G|J+}!Hy>@+(*3MC7sc6PTHXTh1OJy z0nr$vN8~)$crcArj4m6hQ`1h^B9Q+pbewo43V&M53jd!D673y#`f~NTo*h{v>>8(8 zCXyAQ&&1q8sJ9#>UY)&;H4-~v0o0=jIrYl^16~hBcJ2-f<0D&l0H&;UUeuV+iDg$W zEp#kHURc#Daz#J1PqCTD5<5N#5LOcs?;Mpw7!SKA=K12-<+3dt_-*ZBd|N-ppOH&< za>!nZViw_p!IlxkjiGXdpyw(vWF*-1<OFTv;!Q0!r}9#LV}I$G4k)f`_}IwCDILiwtF0yXOPJHYpRNgZFn`DdwHV&aheT%H za2FFDFkC-f-qITM`x94SkF)Mn7C+<)^An*sBiPcP;IElD3AGsYOHTs$D5H=X(+vHN zjl}Kx1wW>H;j@}c(G?S+4y<9~JJC3ntfkqhjrVL#KdrfnbpIKY$)Dm+DuQo$T%TOM zHuu(bB(sf|W5JUUnfUN$Z$*_#Otm=8UN9YPR9{z~Swd_w1DP97NUuCFiB)6OYm1HL zM=v`cvh~rH3U6(Uo{B_AKDHb_)xOwK%#TnyEcU90Bo4L<02W-Acb97O?uO$>7Nf1nx8DQqG?qBh>m-ps0%|)dsQBa{tNj@2+OB!VwmiCqD95|;fl3^h zO-Kk`GG#qX!QKHpZQN5df8(y3C(N7Y62%_!sPm~2X2?@wILTKF&mc{Gam{nqZ%*%3 z%~ki1zTuIxDes?5s2enxcNuOWqCwn*;z&2u3wmh3B^En0T)S~cRSC^JQ*a&AN55*Y z7~U^K=hT(Ar!$-zh9?$;sFzs8=j~@uWd7wJYC1FBo(_r0T$MkpMjlURWn2EUxK%Oc z`o#lvyG>#?|IZ}j<@V7A1=AI1axM$gFIHiI81P#We9B$PO)N%=p1{TrNRi0c41CgX z2+MS>@6bdYKU9c?w6to9I896#Ie1;eU8)tmqt75vZWi9v72!K77PRAo=P|4!@d}Z8 zHx#!uwfls}UjeqjZAQENyu-Poa|~X-jx>=F+f9<3wH( z*C^>F9#@wM1U#P=Al3h%O8pm=A%7#a6L%D~homu8qA$6aMb)JD>#vp4b(j-WCNi|3 zr`-Rek$6G!W~svF;8yFTytuYc0W-$+(Vb_RU#M~G7t~pYBXoDj{U#T&`vJt<&CAPn zYWHKDM&Z`A00h9ev67F3x6eR#_7K)FHk;Qy&uWn8+!vzS(TdZ$(5p7`c3fN`jy2>$ zO1y?dM)icCst(3}8B{1}A@&E~>&`4YtuE(P>-mX1Vs?}UUd=b{3cC+&_L&2%ddYlt zEY=EjY!5qQS+%kRRpSAr`*y>eS!Q`I0?C5sl=iwt%0r$?E!=@Wz=~ZpHa_axNb_m~ zrxh!~qATBO-h3evYlzJY`^9Ol*_KPM+J(!AF`bWyzOv{D){)C5Z^{=IY28|`W}k`TD$jdY7{j$6f@ zI5j*a*J!3uCOEh*DP4X8Rw&tru&UO(^dNpe2Sn{4JvLY`muuL$a=y}^!YvJZh(2YY zckQ;?9hH{dhGX1bJ@x4P7?Ns^eP9yWz9k5zD%$^@TXZ`5jMPeoR4DdK7})RMvt_%{ z)Nq0QlE^)?)<~^0BNB)sN>jwwmAWxAXj>Yl$1;SdcP@f@QAq1)GiMGHh8C1qV18f8 zecX@0@Zyk@uVb%rIOCLjP3uJ=ZJ&f&s{4$Jx9i`7Siji*Tjn(f4*l&)YVj|}g6ul_ z=a*S7hTVEOf#c6CXUTrKGvNHiq< zbv|TDtrLZkYYKdC;S?Gr*IoxHWZXmCRa@>DMH}05vUJy%*ZU2zrlxJHu6#I4{x|ZN z@745f!z!v_1|muN+Zn5%$%sR}|3{#bd{frA*Fr*#v%&Nkua9CX=BD{;&}~U7ZjeF{ z#+6RC=~r0kbLh8}VYx?YUIjL;-J=R%x zKIRvCd;@-CtYN=!a)mWIi4#y#sqr?f=MO-hx8`yX_Iq@R`?p`=IX~!%#!Y5-xVTFY z5b~45mPC$JMa6SRc;wB^R3i^Ew(DnAV%y}w)hf>s_L0=ixyO_Fk7{MvF<;zmer>-< z>}8-z+`9yR^s`NmL zQ&t3ILF!rDPd?HJ@u4m?(hef+Q5&hVZ2q5&)Io|tWW(#>jeHTB@>^+b$D&X8Fns$T zUY*E*6C8Vu4QimjYXnxMLwZ4TVUo8yD*w`0H{b^P8T95NeiBUE3aE4=Z}%49YC69_ zk2yNmsc=P1Gfb-!wp-+`K0?40lSmSTDE=V)MOC(h}jTZ3pf zo0RD!)Z2niEoN|oS8r502{W5UMyYK^UzD8SsX8GP3n5lc=c!wsK5^imqpicQ@ED&$ zvw5ENR~-$HDNa}ne9gm7qCJUJ;B~b4>F@RP((-tFTQUMLP;Js?S*C&W7Z8)JtN)K) z60VhNks!mV`RgA|Xxlua3GxwKU{Z!6D!{uq`~wnplDv*k#yD&tu)5i@K9t|1x6W9E zZM1EN3rwOBP|xYf84`7vXIiee1C9NVnWW>H0JfbJQ$QY`c9+^ur=To3xlKg&=2Pd_ zDua<6u?xS?JL)(i&PtV27pHW7oef(aaok+Mx%A5~WTgnu)~FZEp~Bb>j;;_tcbxn)+(jdz4QFhvoz&msWM0>q? zop@Bho`n4iwhe3W4inqWQ;Z${L0^~0FF67dbiI#V!@AuZ1Ju4fLT?AI#s+t5FIEcV z0Ba)!cIWf1Bym=6?9GE65n*mWLk_{A#8umRWK)R?s=_@o(-e$feVNC`vPPf7x?hV~ zcmKw&P#jctDlG1qtZ}r9V!te==L{6R97RTTe3VXQk9IIl=fXU(W97J_<{sF#Er!n|=+I%&`p`pd>@f=!8ubwiAvS@A7!8@6nex$Niy~yZQdW{7K4tgYrl)Z+U?^&^Q|2j@ zWisT6M*Xp29JHm0{tN)0G6+#bS~`k7 zdk>s4KtI@UFXa^E8I@Amu9q+VWz^o8poVCI$zc(exAS2hTr*a5nZFI~XVL z#XntLK)&9{_1R_SPTB>g?pA_x8k`T;hJE#G^)CeSGs2V{%N7!^V0*Yx!c~F1%(@Sx zW*5bNM~;EYkUWg;x1|lQ_*7k7=a&mz2D2jE9po{b9-d8c(Lol36BH9oS6?)1A9qRP zK0wDDB`FO&M>!Acu7ULA*HBJ_cuM-sTvL-QpQ>xk`FRx|Aa+;~Ti|MYOwv6S+ou zn_YHTFs9MRo{R=Fzs>sCq%jqZ$Q^|a@#jryWltDQxd=?-k5<=umLy6{8uUyI?~h-R zY-TYp4Wm&mWCY@ioO+bPu_jck(;7|e`KCTTcQo4CM*(3}pp94vtR;KyIyDa2B;UC&eS$wBIJ$2o&8-a@~ zbl{up`);X!agp~q#I!OjkheG)7XdnjmL0icQda<{_(6kV z9ZX zfTHMC%bd?WxcbW;#Uihfaz88Qxp1~?16uFLxz8S(DVGFB(Q^fw*zOLyg5y~ih|*D zYgO`Q+lkt6i_uw0yQW@ECehLxa-?2DM1(^$7QcIzGdV)EU6uH=X`iUIQtd~X)&=MB z7c9p?J5++m@#Cj!8}lRD;4$%H7vTtOgVFg4q<=Cg9@JMe=heV@rstIIr-Nj*kc}@r z?Xe42w>b9eQRT%t5i#i$VXzWDX56sy7cNPc8iftguUFOo|&Pyq!iMf(a4)hxB3|zeBf!`)uk8q4K&wj^>w0);J_w(v%so2@ezu;H>~)MBYxR@q+)%mEe4fjSshYd| zMb;8cHW5P8kwfjsd$iH14aL5M7B0+!_xDr{SAiKYn4vD$N+5GjL`b$d!6>@FbH0kOo?(B>}&aj(*fYa@F)gln^?4q;{ zxZ)Ava2w&d=pPsC;en{_EZZT7$4!m8p=#!<%9IwV&(MW9sG>978zff1&YrF*e{!XXfkJQh(ZNZC!obuOci<#B5*UU%+Qf; z+)CyDdlGCtW7*e>^L1H-HAMs`T3UTKtW%ERpZM&bx#vP|4hMAUkqG?#%+3s5 z%)`^mlt=gviSAWIOH8$j829ir_tPJ?yb1y7I{V-t4<>##j*3l$J~CkpMa=p6H&n8$ z&n&R~o1908zS||Jux)@kjY-upI}Gub^7(*^L&_;?7v)E&)xu=Lz{k%cJJ%zT=5X_y zy7g*_mgfP{&N$m$E?E{VgUN}2zgL8Q_qis3+;|MV+pV+U-Jr9duN?@vlYO(mW$6?O znK5ryo5lZwH27FSVK6{>G6poG?`C{mWZD*jZ>Up?6<0jso?=q7O<@Se9BANo+jgz= zdCD&v|9S?2!!+WP4k*pa$I#2rTZ$wrtL%&YvY2sJ1pM|E>PJa+cFhGrWMGM_PZnW+ zp&I!7f^6y_Kngbb0JrtUn3jf>b4~V$o^SuSYe51S1{+StV-(ESl4r4sn~?KAvHKylMaB1*TcfPN*ybH$ z6}!@SxM0Tduw*eT?Txy7q1~qoc-#~{dF?C%%qZj;OzOZXGx_wqFqmXiTwae$N#LC` z?nf!CuRDA~G5w-Y-(D_F(CoVGvO2D4?>ZKngZD%J8WU6p&(}}dG-j`Ry+27M6A$@+76QTLxUpx` zWEU1()J<4`jK8Z|b)raQXfX}p>^G2O(fl3~-3)`SV2KcJP$Jj7Np99A2%NTnPf_o5;Cr#?iwm~b{bcY3;%=i~W~;*1AX(dR)=GkIyN5ZKUgIwB zceY^Rlt3KTMPMPkenNwL(x+$0ZR?|FsI!QDuxfa*F>iSzyHaxyFQEv(UE8m8nKJ8SBVjZbu1>9|(Ti?6or=eNR?>ZPKVgS$oOi!0z5(9!f>%MsVF4x+~kDsdC zR}DpCz*y(iSHoB|yXOD+Bi>{3y-s_grp8o*T#YKL?=@HbDm+3f@Bch@9oy&dT^}y# zWuece!1(bFN#T}pDWsYB(Aln7UM^j=gN~4mCq@^rKDEu`KTq_IEnY7=zGE@@K{EYi`F)0Y3-=z|L}=VTPf&glc?%iyC-~xbpg=nNXnDB zwne6~m}K~q@n=15o|#Vt3)J5TC!nAnR3R{|7&?=}F222C{G6n^ z@+HRN(_hk<*aOAOXjJ5*y>bwO*HI!5}+McqkNs0H$CI@+=N}LL{>auM4 zA~>)4#2S(UALT2Y&S8Q*E^9l5=Euq6}D52A!A=076z5IK*q!BZqS$@ z=B=K}ekg14zlx4-DAZ*~WPV8L@fbte^N?7M_L-6+_Qj~#M{#k(^&rFQ=nG3JMqeo_ zaD`t~sH5o39~bw>a^K<}T?{ueF8&nl?*%CynUcjb1ZpeRzWQPzItP|h!I#_%ASka?B7CSp&* z`(LvOzhv^h%P5~JFq2#zmQaWnVbnJNOl7qVinhgt;F?W_;0RYxZYx`Z+ag;=HQw!3 zziOIkhsgaomdyVC=|QAj<}fgG_N-q7xKxb>A>h(zLiV-lah$4Z)1ifZ+rPZBFvfwfKptX=mNd8UYxk6;HM3#x zIOAJQOgyJ|v5B&nAhA~V!v?cSI$er2G59@o4bfcup=H;=*4QW%8#5#~DK4`htxorc z3+rq8>}B%Y;9csruS!TT3xqma%$T0ou7fd~QXd$g;1hJpoGIe$uyvHN?XewZ_KBu* zKSOBKjn684%M|>oqdWri^kiIM$^dRp#PW6A(ZkatGJ=5dyG{1w7yYHTXzScuP* zYz(6+%HAkFWb*}eTK92JyZQ1{EHKr8Ee*(@(%qqgk{dB}U?BpiKP7h2oq`mSV04v| za(5Wkd4CnvSLFwT8e$`+^WXLigbb*e}bWNy?m=fa6PZQPKVzYrxmJ z50O&{eJoHdJP{bHb@8`cZRue+ZxkSgFLv*sO3SS2d|$_eOp&LXn7fe+Z^}rlnnk(I z{>E%XuHE^qT$X69Ab0B~>4l9H{t~skwc8s*eHBg+V-fv|aw)_-d}Nx5X)PA-h(xpE z$4Hss4+qQze)#=tv zMv8@0k}>A=Z?WBGb^u9b#tFtBg*B_?Qu3DTUfOc_Z%u`Sg4%o?!l~5~M@+oy`#o_2 zQiit8Tx8;DZ1(RbbrL0u6y|GaL6lEsQhsA$5hitCP{Va+@_FhSSC_O?#76pRlE`Wh z%6s1T+nqjK*A^K-&&%>ZWoGSeiM!=g@IAXIZG5}Qt{ac&$rEV|TK8rxAb&sB|Ct@I zH6(KBdE>@}{!ziUZFGIX@6S=3fa2ZkLy%mKANCp3&pDD0I;EDbF=%w&Do9s1IvMlx zGF`j!Xi^d6YMEfT{JtpE)t${ebRHmtP)F_Ya+YdfoViWkrT|3BtoZ!g4L2Izh+{m@ zi~%RC^7^n7owhh^MCzVkyN$FBR|-$>wO41fYOkD>4){2|`Hqzq%P)Oi#^$dks?wu} z&1`iRPvH7#BhDP}k;|-&>GWi!O*|f-+$iD!v7KG%u}yZkmNliVF+_GH$;3^H(o5gu zR|NJG{;D++{QdcIQF!Z(txtSaX7RO*&@H#vI_h@zGYWqqY^qO2Zr1G3y_0oBCzC#m z{9KB+-?M&zaocb!*jWPlg74Sx8c5iJn(@JuN7Z}Q0p<@fiK^D9kn zN0{t&vC|p*F)7|m)S(|Jt}}<;{5YfI@CLe4mx2$XFZd=4m=fg)+dxvX>&L<=>1XX^ zz?V{s36z?rdWr(lxq0NHC6hkv=++=C3JVAJ$e3M;IVD6^|HFCh&B)+{ zhc7K}$bLkQTJ2G!@k%VAQU1Pr(T!#2)^41Gu90m9CrszO$c9nTy%CvV!2Iv}@WPV@ zCtr|!S&C6DdHpMC=5&Zri*+`ES5Xv(QruO#J9C|gj|%D$HnP_N$V>on`F=b|_1?%C zV=W8~9D_lvd>RdTU~b<80lXzcRl2nxfD}hH(sQGyZ=sX@|L}(l$_xe3z>6H)YZXAW z@r2&@$}W!4 zJH7y|a&C#8v;D&xPL2f@ZyZ&nK1B$AXep`a6knn7n&v#$8WCM4jn*|CYqQ1<5s~(; z^^4!2I)u^KHr+D4_0(;VZG(Fw?VMr1lnJI)w&MC=-brTGO)bPGcVRf5SDaw24!>S- zpvbPa5VshlW%O)~P5Kz(r$Gy0jca!W<8HEbKTPXXli-zB1`ClxrMt*&cLuP;)-a^3 zltZbR$>(aUtaPTXvLQs69x3Psl%e5=atk3hP!8B2+cbH?^mk0|2(#P^X%?p`UBE&i za`s;x60N|z`!roS_8LW!k&wK8gce7dNto@6=(Jz~;o!cso^)oq%Qq7Coc$VNmkwTN zDdK>s?Q(JLQ+}#?-JkOolyO1Eg@2SOZt^OPNJDAbsV_UzI|Hxs_>PaIAHu7IfmN~v z1G;kEr#$awUO0>(bnMv+E&-{;XAhk)m2VfBIqRbin|1g$RP(Hi@FVWzXpYOohBB5` zk!mtp8%Jb%+%a{QG}sYAjge2%tQVGCK&wg?I@U#JNq+&E`?)QX*c6uty((x?Qf020 zWYmVBxh4G{@bA@jR|~Dh{SeXdPpAK%XBehga_0@os4<|{K}=eB8BQeXJ=q%p^z4=PF(nXu3@yQ@b^V;x~a zgLXzlt#+?*$H{-;z1V)^v`QkZR4>-w9MWWdVTW>_p7QXP!5X2oO`jh3t0eiiMcKmR z%|QpGxd8CX9v?Z9hm^N38)<4*eg;fgsWIptejitAn@u69G2IN4bnT0f6wm#(a^5dF@V z6uFd`TQ0=+)2t&Iki!<0UGTHWW?3Ap^P!GHzn?7qq#(&YF_s#znh^uz6XUEQ(ssCn z{APYJw)jPGCkW(8a+8WDn-cJ0SLjn5eFk-EHZ|UJlBe1B(I4Vl6fG1V)KF=>U9&xe z#_hqmQKt2A)ynj6D6C{o-bMuRO847Um375wq9~BAK}VgDcT)->8XHbC%-U-F9H9Ea zpd=FA$b6fv=;hCPhFoE1>1&6t0fifHR*8iY!rk}=2G#ZG$??gRogHNi(O|PvMAGTF z7J(YXO}Ox`Azx|pzWI|3lK_v?xyV8NuHYsx z(Tp$rzM_6(tyq$%LfGZNP9{osT#%KeP4-;=YFA&<;rw`aaJOXTWDiW)6x{f~BIC<-SWM3y-GqHxTNEil zpz;YPL@wC%ajrL~fmNK&y4oC$kAvi=zB?E|^+y-rt;N2y8HKTSvArHlZC;^|pgy1t zB5e8~Jk&4|XH5<@Je*M`b(f#V$peiUQ7CTHMG9S`-91egPwxL zR~SxckiGP=$pPJ&vjsufl3y0?Dq$dkky?DM13 z){FVWnV|ilUfTd!i;-m~Nmi$6_$t5oCL>{y{Y%&4E6 zMZap@FZ=%U(RDuC!Sm7@VuB6&O}3#Q^>Mgc^tVSC%@R?k>(>R;`q3kfM9PecPytyn zh2?r*W|Nf}Rt!xK-e6Cu{8?&k+M~B+R?Uui)HEoQ#VH3dEEiax!8h(U6CMh^>lsJ` zuu!?87dZp59e?b`^_hKNT7$JJL@}^hy=JwlvKcUx2tHgIFn)>r4-zy&4^LdGcSbQ? zH80%7_Zb`G>pk#byQuH#>*8@6`S?`s*NZC(apkceZs`AcDNEkiscgSM>e_OjCd*n8 zFZnQIW0l?si!vr*<2YKqz%D&+48fbu45h88%wV{B3|e=bq5fF=%vOMm)B_F|&~ZX1 zeb>+=Syh#2%Fg)A3eS|sta46Aq|P5z;kq2Rk@f(x{xU&|JtU33d^>1borri;80p@GLAvSmhr5Vrw_l;hOPX6i}*vV zp3{{*k$T57Oc%4`KDT5qG7vFAxGcf z4ku=n-DLmWDgjOC6J<|lSpY@EP8`GqcBGAhb6{SbkM1x<`?8bvJ%FE@K2 zJ;FNa6_)=g7rHcx1N-hW=j4etDfuDdIGd!>jNK&48{}HkP;PgPN8ylcpb>v^hy-=0c&e8(mGIGG> z64Lb#K95}vwhHi0@RbI(Ud#ubjV542$i-QULAwzbO(N<`^Po=ScO?sK06cs$8z>ev z&cHXIhgo_meyBB!Yq6nRfRpSy_Gt*bbH{UtxJyzsDKY$cGOM! zHEvq>Qqz-*Oxl_5ii2c{#{x?2uYwdj=j*Wc-PnANA9(m)7FmAJx($cC2{litfY0Zt z#oDqeBQuJCZO9;DL@}IhYAgHbFt$Pjsm9`hl4Q3j`o5O@7*^oqT}Cqtk-v$jgR7=*v=1kMsC>PK9Z=C2hi><`3na2w)Fts!YLSZ&x6 zK>*X3OFM|}*bEk18C|S~af!fW>i-#i_dEiljRIHe{L;LIwTGmeV;r#`Yww5-`!OS{ z`$API^nIr2zT}=v(Jo(Zn2w-sR}rFR>CGt457fatm<7D zSTH}=rkW+_mkiJ9WE3v%g+bDN!Yk_yl%&;Bto0n_y$0RF^h(9Nszf|6MtBxVEVGWx@2Mi3SY4&QL$vLfh5rQAQx z4_A-c7TZYEy=Zm# zO7pEXS8UBs@Mb(sU-6Cv!tzt>7V!D$dA&sDu{M=9boF+aJEHPN(>z%b!36=S{TDrUTI6}v*;ICiE!lQzcN3fwvAix8s0ha!Nx*IS}eu#YC04@HXF?m^Jp~7{`6qEi)jm8?mIW+CZb2WjPwPsEah#}|r zl~}u391My$JCq`$caQS-N&<-a^);XV4)z2=qhLsY6*EbCS&51M@ZPFb2%XhJbo1Cr zNv|H7;1r$z*z;x$Vrb-oO?Wg7sB=MfC&oQP&(d7LSbw`oWXrHDD#LEP5IVR!Tq%P> ziuf+vwO}PocgM7>wz5@<)vS@|S_?{8r5@T1r@g=&uC}Hoj|-IhwY#Vw4NR>}@}40* zeeL1uj$&Bq+1mK)uyY~ag9$WIF{4`Wgo*`!9G%71oInspUkQg<;_l84Xxv?%KK$7w z=Oi;jcU67&jx_l=-~nouY7-b+Y+mgHLwWADuDzApN&An-s`T*|F%G{f_nc4LSils> zA`kIFf;EPCZ$gqOJ|{RE+vO(|x-(IT=8JIHp>M6Uj=5zm5Sq2Si}Hgp!euRe1$!RbD+{xEi>Mv0 z?~)5)8;7EJAcD9;;zYh5zkQ@cy3AS)y~^Hi-}0cF z!w08w`1GdnhmbRMt5U^F%x$SsvL7=#4LC+u5zIwtf#PYh>A_{dB?-Uu2rhH z*gm-HpoNiiHh1A`aUR;m1_am-)^~5dMKAlib6$rr?1K=iV>dF3=kW(RM2lTuc)Mj) z_|bfkdJokX&)*T}swhz_)2-;6`ApOPunY|ZP@l1G@vOMTWlSPhhqqNt z8NH`1`Ek*EFow6f9`lr@00rZyA|PqJh@B)Mal#`r)TO1(}VeS9ezg4n6zI1C|gFjAYGmt4kNk;*T{-Okpn z?3CpguFvyT%-i9IU9e%F7feHY>yNI0fx~<)y=OLGZG2`3P@v@m09!_Ekdxi`EhCBM zXa3M?uIae+95&VX#gVd*$wW%TLyI_c&OW8;<+r;lgmL;5@%H)(5VQ|4inoS7V}rFP z_9-8*8v)G`{I%mwp6%hQglXd;kR0BtT~uiCC%n*d9N$!Mf|bwpgIFWTDSo0s#ov$` zWz&6xhCXyL@=@6`+Je6isw7esc;}3UP6eefP8^KWoiSvj*FacPP+)03D(iaHz-M5+ zVc$^Veoxl>1L-g?jAy2#k{62JS}pbJ-h2BvUU@_0TA0)RjqrN%A_}<;T+RN(TAbOi zs&Q%%eZ&p+ph@#QN}$$kD@07DFNX(ErBtx2$O-QCwN_MmOJA(D#C#Psv{$X$I%6R& z*1>Rms^Ca{e|6P=kMhy>*5K~ao0RS=>BkHuCjYeV+hrsiM-)Q*{OYpWjVR2U zsmB8|p2K1p^uo$=w2a~qi;a?R1htF}encRjO}bK9`WCbuEc(mET+=$CTFB=~BTIXg z^CH-mXOLN=YPx!TR>c{mzjs$xXbBV~5!S8UW4s~g-bAu>?g$*XTT;foUBqNO(HOh& zjd=@suw#v=a3m2TlUY)ZHnC$cQ}s@GdVyK!f8WF1+G!l^Dwje@zP#{#$MeKDZ% zyS3%M#%9v;gRV{645z`!kl}x!_~|&>C?^EL=^XH_QOmw!Q_F82N}G}$tHOS4g0XC# zZwgkb+5#qoHn)8w#!;=HO;8Gs4rcQ#>o>;7;HiAQk8;lXM%{h;W}kiOmfs%=o!v6} z=Y_@357! zyzg&QIA~XjY~yLsN6r1c+<{0k1gOU1h`$*_9*Zxaa6g7{OInXnihIKVdaU5vz=H%n zn&)iu#T9f;MD)Q!3PGLRvK-#E0!*lKgZLUByr7Cgv5zEh`lU{k(YPWkJTbpobK67M zU7h5w1?zHsI{4FuaOv;`@L3I48nmB+A`AE~4Ia5TvcAkngViv6}KUKh<(=&$wfw^F}bGHNphRbxP$7pP*gse-IF41uolMpM7#CCH!99HxC^K$b{WT6gjo6#gfzCW@!YN)i5;h;T)ApPSfweT;6 z*LZlf)=g#XidmR<#Ly^zXHd}+{!Ly}S}nACg3q}qZY%Cg3=fUvJy~ZM^wQ{YM|E{1 zlcBD!5Bc0Q#uimhu+wqExPLe4Q<_nehohuFjVtTH9?g5H%It|HlMJo?n>*z#vmj#6 z?(ZPn&#i*3R9v_lxJUat@B5jHhUY!p2%@R7q1JV9kj!bV;0p@jI4;dATPlF z9Nq@wc5-6y0S|_A*502#{++G#aCbK?Ev(#&hGC!!^kaX85W+h;i7^?MJKc?7s{7eQ z&knAt+tHcmjWvI(Yv^EcYNs)!&RAIQUlMXG3G39p#R*Fy=Pq7iu-(aeh!3tg!e9l0$@rfKQFfJ<(T5 zrBv_wBShN_Wb&DRu!!C@c-6-*6UUIdr5C--T7n&y2KOs!xzE-LWvb?`aUPL@Hh%ABjy-{bEPw!dxILTwwec{|)mFQ#@75#5j;K z)?_&&+U--{`M@maVZ!`VWVSB>xS{&9@z~x#z3!39Z)=?DR3n$26TMvxu|S)2W6C%k z5OS{yVsbrq1j!9AvA1Ert7)Z#r{GTi#C$mmx{H?7bC2tF^I8Xp{o@1HM*Dl6^B~>$ zbJ3I8RR_~&pN$T?;I#5@z?m4nqEsKL#FjAe4V}Vp07o&zw{2vLS$vGablx-i{Y?3) zlp}&wj<{mxa}`zRU6)u)(cm##OF>T-S^Ypds6zG%-Z+~Y0l!6)a3Rlf&iuK2jWfBG zhDNJdEHoE59?;hrmfmO%26R;#3hVbH@M()cKl<}gLyg&ANtr? zH6k)NR6%cq?-T?sv5_#U6RI1g4x{pJ(6s5XvP3+cJ+Y6hPUC1#t zPtKqi>BttNQAa+?f;hAnP0(Q5);bY{e5Xh{>3!0XhBdR)oxo~aXv{P{xx&+*sXs5K z2!Fcj7ddaYzBld)IbMk5Rb5!_`uP;hr{pkrVITq{Yh;?YJ^m?;p;2+ zaZV%o)%<;gYL{;(>4E7QepBZR)P)7b1QR;w8dLFB6=HcB9dx2KGso{^auh+mHLUwo zs1Zi(&_z-kQee^p$v2~udM4<4ix-%M`l{R%uAr_C_Y|sQ8#IcXav?{yJ)=)LdXd@K z7qV7M2|V$;cWiZk)eNJo_M{u_Md)P--QA@jf6Y6mpO(Z;+GHNdteN`4G-?;&XzQ=i z<{HhNbi+Gjg>J)Snr0M;+bnAg(h@{NZu-ArpS$4&#%rrMCl+rmi6(ruk~A{aa*HwK){)OW;#65wJXP!7%T#i>TY7tj zx=}ajW7zLb#wv-(WIjLHRJU35*hy<@<2-J@L^YgjV<&$Fathjsod#xB=})l6A@I!M zgUhWQYD|`|t|W-?8#l4J5d^J}kHu!wFpoh^ zKkAXvIUi(w+>-v*C)1yc<0)f&BkV3j;IVe2UcUgH?bZy?u7AoG&L8L~Vg@s}+Xn(k z5BMX1@k7jI2T8D1Vm7tN@M+@1ga~>ORmm_1ncD5hs%QU+R{9Y!*kk*$fllcsj{8W< zkxP4Z`d+T?>3dBSu8sac?7Nx zg{F4H%=KC=`MK|pT!L+H4<+9+O(yn$Vx){9OA{Dw#qg9GDW;(Lw{!<5(J-D|Lnz2^ z39vD#O&90?kfhzwsC@)wPE;j^8h2mQ)17RXHQCBOAOv^p-xXw)I?~FDrX!P*;q4)h zP4DtA(Ya2+jz;#&91S-b$vVZEyMd`HNXZvrYJW z_@xD#*ALN84BrlYQrf!B;G0hP4spS78QJ;!UV52&H=J;ikX~=gP{iGf@tcbLt4zjl z1^3U-{ht>!Z)&Q#tBGdz6g?4?kl8>O#d7$A*i?G^t}6O5y>ly`%e9ANp7J=>tRDg= zIZ3Kd{2_o$ka@~{>Cte%NbmU@o^UU91_HEI%MtDM2JwM)#Ih!+hDyQAOGiC+4(ng* z)F9cuX-*ICLd=B@Uj1?p69{DvxLUgKP%&)M$;kd=`T>Tm811@nv@yDn27;7jUIgXl>Zk+yP(W zw!lUWCr~{FrZ~he9Z8sn7uDo_4Z)S`{3$;+p)>{OAexF~6*&uV=)1-Zwp+5-f5Pkb z?3R{@tDh45W}^aLAJrY$Mmm_@to>)*=58tS`1kI>YJii5%|pe0;H2_O=d|44vS-|? z+Je-g#;tR9-``Aq1r|x`b!yMGKIsyKqGF-9`4xTOG+d%OXr`Ka4{RdK^R+AWPnmf~ zO6JfCH6C@)vted;XnA&f>P@=#-|uRHi7}JM<&7P)Ph6tS({$)hemfLk<*E3}cFbHq zsN(?y*O=BlByOLvpsV@(_%J?iJv62!B(ift>0XFe!w&_m6)X_RTkaJ3hS9vho&H4V|~jNQXzz8zGL;H*U^MT~fX@-m%mkuGUQ zbO!`bMFRK+Yw;TS`E&FDD-VgcpNbg*5UwPn8^&kK{xd)Oc@q=q2k2qE`P%VAo+(o*%w(TSg)P^_L+Z~3p)1g!lsJY z%oSxbTv+HNg_UF~zOXVJDq2ddW8Fe%9ApOdMWCS$Ll_-Lt<_E`fG!IVMKsyRTchQ= zNN}yyKlx`EQ|i;nkIqbe=n|vx>s0cE$(_Wia+FV4Y2m1JTqzxskq6GF#`VrE=Vy@; zawDA`S=|Xmsw^W>WVDIxZs%h09G&>5W$8`3L(^IeTGcszY~4)-k7S!T? z@xP{Qb6+Q{*K51WCt=FFV-#*T!b)!P4SqV7PH*)|4{jX^?ro@=vbYA8Iq$XZ;w%ZG=Q`vGQLP@|Vs96-G?w9p2NC;<gYU+ zV{5n3J_6MnS+pf_O871;lkHZkY1CX{7$Hwko++zar9I^%&x0HROxk1xp*)+*7hboL z^EDj#S}PH|K!|6xRs-cv9X-dg2|`mp=d%A zM~3!VU{Xc6)A;Y#HnS0(w{K1kL1oo!IGVBi#(90%LXZbt3dICl{G|_nTo74inIp^V z|Gr040UV!HX^*-J&sFcYvE**u^Gdrsxqflsa_Gdl$won($>0#!8D#(+Hutxx9tBhC zekeb6ESR`@Jj!NMp5KSpVoGPKLaE6hN7>Q1i+Ihq z=Ajg{3>b?Ekq@tDRNU^J&!6ENEz$3z*zOrq(XnTw5Ee5D65M4UC9AU~jSNn(qwZv= zWI>Wj8yAv)uFX)!!%R(OvCi2ja`ATvy6+{tH!V1oXJ`}<|8NbWa1Zskn z9ysmYpG|)~amIWVqD9p@ed~6A=1{d^PCZfds)%F~)cDceqLC%iu(WCm8CHR< zv?w^be8ns!2C>uK)?xT8^Ro%TJoW{J8n67Vv-UuiOC_w$PzD3P4o32J1u}>@4QNZjx}u8kuBL3(82&~CplzEkw(Ng^SFu6A`e(ju^nOw*T`UFMHax< zsfuk-6Rjb|Qlh!@>0-q}Pu+m_?`xGZI9oCY!_z-b`8X2`1Nd6O^mqQH;*CaN{esiT zZY_}}&sq+|o-%DI_3hMKl-KgeZ$j0hEN6=3R@ds{@*Os{JJEEo@dBz7e!fE=4*q=C z?`1Fy3O~;U))p%1!4V5@s!;#9! z|InaDHEx5}3wH&~p^~_KShkvR6p@hD_WY|4w(LH?@eLk9zI)bF=d;j2i}yb)G3=gT zvROki^KEcl(3!oy4C&lqfiflICF~#T*;{ZccKU*o%Z6;1i#0Ybf+J@1cY^DqV(1LS zX??XXWDbIym)5zdX2kdeD6Sh4$^Jqr^vp_>hP%ya8wgobyz}sYdE+IsH+z=d-48cQ*eh`9x&ETd));SRb z%I*3*hI$335Sikn!Mn8H9CpiPS}m)s$~iLGr(R!JCH_YpGfYUMF`nL;_2$xeJ$=bJ zG%)-W-j0m)A9z(EOjA&s?CB+n=NB;(CXMtuWmmQHOOFGm+994a*fKP<`+O7en0;vDgx5)g`U zdBnSgeZV!RjI1fA6sjZa!?7!FPgYEaUi3}1*wixOZo34x2Yd$z zEZuvraRPqsGi*2H-=wBN%IkDdpBWtJXk1j#$_0C=ipEQKAv z2~lF$ebfCAf^iVkM)3s}^$H?=SNq^4mF2qZo6ziqIwSn7ujzZD!*hcS?T2p;+7EHZ z3iTC3vw1|M!LeExuRMCUajB;O#7Zp#ro!NcL8f+2qB^- z#P^^5zje-xho_7*s`OL5UfWSDS>}oyPMQ^-pnGej8)Nl=D&C5kz;(5Th zayPQ(H73!3TIL=$sC*`+C0Ha)#yXSE1wB-RsJet<3G@D}(ANbWRK@m-RsAF1}exgBG&T8$#1Ty6@;-Zx>P zWw;j~#YJT$&OiuC)-SzKmiLVDh#j6B{|@3Q(^M60E`(z$a}zSKT%=^3HfhZt%og>F zJtSQ8%#)w6XroBOoiQ8gop8GpX3lX5LzRXe1?^`9J5@pXEhqh2VK`5@>H>h11gPlh zI5VnNLN0C=|0!36$~tzidKWxL*U*u`BPFa?jHf&U_^kUmrP_U_fpXH*x@5(hP1t5H)d~?YV2;W~Vvra{WpN>pB>6kI4H0oVbCn*lmI1XpMec!QRUXD53Y&$IA zgm;CEE9rY8d2*IG+I{KEsx}r}qSaP& z;3&dVxo1~R&G+a;e3i7IRgYOCB(!G&V(Zmo>@ z0tE7oPI5V@(}QrXW3x_K>B+RT=AAmUd7Fv)7)Wq>p+I;pEC(B5q2u!3PYfbV~;{b<6p%?yRI0$Z_PE2eG3fc z3Z-@HhjY|De*Sz|Ntl3ETJ>yW&VAIZ9Qmjzm|99TMnv!Pm8dQlW*_gZ@QsUt`>K&QSk7Xad-a{~JTRzz3~78= zbp5~^$Xz61TkO`W2H`p`+aXjUHr0hyv2T3n3zpw2X{<_|ru~z|NN)18u9gp!8%(3G zt<^P8RA+TJ!6At`L!2BY3krm2_Ya3-Ooa7C9<2!bE9aDjSK^KT&B~A^E5g0(RR_ZPA2-?NF=jUhIsCx zW0~ONf<9%NTy~Zy=sR8JpHy5ySu2fATK_BmDn9WXcH4W>bHkl+(I;w z@Lv2%nAZy2kW}GB@Jz0`Tt8;n4dq)%?>3{U1sVBEWo$sz%N5!nv`c5kA=|&LIBB=Q z`LWyz=x1n5x1dCxYjgf_;4ZYk3)b!&#Zv|e#Gm-hY3ROb;lA_kq|bm(MJwpwsJ!lN z<{$Q!ato*Xor7k)t%Guu*KzN~dnZ-Abf;&2W$uFmp&q+dj+r@xYBo60p7rbHN3=+Z z{hVlgGQOj)jTlpyH+@7)EJ=c=o?(D8d!yEJg9$ezJ~_c;QTh8`jG2veRPyo6}cVSyo-W zLpVzFFmel1gx7f8F`hXsyL_{6;*}7|v2@a0ycx;u*~^&2_pCcVVt__E6lK3k;=F6p zoL+0h`uE%|jTYBFXDLkhO(jWcTmjymJj{~(W*2N~s)>29*87fbeL0y1CM>idqO$D| zU-1H!Sh3Y0&A-#&8}nzgiy~b{7U52&Q15;vibR!1^^q~*>}mv4rzo%?WhAK%$6xiy zOYfQaQCJ1n-7=PoPY0p1F{<(pTzdmS57TY>>;fe__+be6JJ1TN2Bo$JN+4#e>gA}_ zw;^&|#mB3JeGRiuu@p^&XQw=7y$XUutG7-e%-Cvv!^&Rl%pz-fj;;1XYGMxj8h_L$!_v%qwlCZRVJF>r&sN{!( zNV({nfU&Lu3>*|41T7F4@o7?pwMieTq+l8nSv!&wY5nTu`b;ENCNcvT!}VY_6uXCA&EA8ru5+jg`Zd@X6~zAGjYHLiR^0`3BEAQ%rk}Lu zZ=lN7L#qa

yl@k`0H(JIZPN)Oe!dTRpiaGz7l2daT~3ms^UXm7l~xdQvuA?0`L} z!6$V>V!NoGCs~?!Yvh(+v9hUHOkua{9*Y_Ofiff&zXkWC{2HGz=x{oF1zH)lX#}vT z8aBBNM{o3P?Rl15ZDQ+__pKaQaYeT%xa-2&W9pqLd192Rw3Y?<9Lt-`Wi?pSc+2+( z#(AEU=;Kgk6%_KiHkHv5j4@gUGx?(*wseTUw*UA)k8zNU# zbwO&2H+tB8;HTExou=K8j2{R8(DJAZE%D2Mq*S%r?k66Ld5Jsf1nbgy^33$fv|nDD2X65qxWg1T3 z$Dvs2OmQn&ShrICBz8LOnd?wIDh_dD=xUGEn|YY}$_?3wk+rFXi(Qn0avQzDuW3ql zY+5p`4d|4pg`-|Nzxgq^=uCu~7g}H`{d10=v{&jkO1V<=+Zh73x)ms*tmcQbpC6_5 zRJ}z&n^JN7>f+0R8Hs9#dj@*eN|Nm)h@umQH zct?JYF6g!qmb3NCy(*6iD%~u;O9ew}w^u)O z!+W=Emm02chPny(3`#Kw-y~FKRiTRKz)7wiZDxpR- zFn5r386K(xhtPNmbKf;mIx;XGLabe`g8PT0(>ID6)a;4TOfGla5`oCZl^yKcO=3Np z4&xK+THsfXi3yk%1Bh4dR@kt_8I9t%S*x+)+P6~LQ8LvA8#8qiKhx#{xwy4F{1yIL z{f2y6>IzcSV~y52%zDZqvb0VuZL)PN<6S^#n9h8XozA_#Yi<+ZMK}{{KSAUa7lO1o zlltlR#78SER7c5wV=|3jIkn4O|6l??q?FBmh9e!Sqe%GX&>*5@ctgf|=cGEySaeRk z#PJEZcKOUmqeaPZHIKBSuFjGhwH;>Pi9;@~@NAcjA!*|F?==JwbHSV&p$s5COS;3R!`0>&E@yG67M0o*=g+0y0?(6v#0Ljda1~hDb-TR;^TH{h z*;a|dm0j6Ymf)W`M<6Q0FFFhJqCOW0mZC(N4$kCBG!|G#VIeevD2#Sw{2SZCC0;1H zWvsaoKaJl@@r9AfE+3t8P-&Od$HRJN?xtmClul4T`7a7Yj0BVTi4%uI_K?jf#m3JL zU;N2=teo%A;q8hQrrsDfP3crHB|bBT`NZm2g{xtw23=fR@_1`Ex$`ushSAw{&adrnQEAS=k$NVUcm#w2Wn^*X8`;o8kPgEN>i|*wM5Vq*#|w?%>g$9O&h=@>Qqd zI#Wz&U{2Q;8ME(eGrLVbzyFbPh<%;vFZ7vDtAYr~4d13?@pG_p+_T}ycflYt^VwuR zj0Q9d4|UGXn%gY&torZ>Z2MT6x$f2E6ss^_VLo1G{3l^poHK}s+YcvvmQ$(yfyMD_ zK-06t)K|E==JP>_%D*_-)??B%qcZw~S<~S6jnHt?ojj^;y($nU`l@l@o5 z@wIsEwcPsi4H><bLqd7=~IeM`;v27Zi3O)w%_rfx|oJw)er)a&yed+dG@j_NVe|uN)h&&^U z?nGZRyPvqduGzSYgw2{xd3_H!d*4W1W2uhe1V548@Dd^M)$zvRH9SrhqqGusxMb{K zl1+m4eqNB@8q%x6cq-qZ!8~WLRduBe8)@Y1kPd7*alumk(iUijO7--)2}}8WOhe1I zk=Z@DtCMw|ErP@0i_+AtVxHn;m;CXGdZN_NIJ@&Lh#&8V%Z0#f+YwgzrW+ zb#s;u^Q!ZS)?3e=@!R87@%FJ4ZT}*rl3B$kP`+e2TU(!33z4y#`(3>%Cjsz3N$T8| z?PBGrT(OC{a`E$GrtN$xpe_YF>O+1&H=SOBzE>O}wIZtMt^VhK>qnxBU{N0eQx~Yo zwdgb5M9UzP{h{q+>xTmy2Zw;=b$+eWQ?h1cS)phtk1dm}uXNKB2=u!Iwnf`Kx{A0s z&yM(<%)t;It{dA5>b7IWR%6k()6XTLFP=N#>=M?HQzy`r3v`^Bq1|VjTOb=e{LVK@ z*qfx!c8h(od!j^#RP;I69}2?cc3oA2Kf<$K)aP7IXK;}Z%jFg*ObHjTISQq-t#{&R z_$c5jg{)3`UA8r!);I#Z_n@ zw8ZVZzIB$r(F|$nF406K6|D0D9~wyv8@3Dnu?kp9bYoQ|>AI*~;YD9)z04C!Te?WUDKfD}H;qN};?zZ!YyPYeg;#XYFPGPsg{ZMGvg^(fmw zpG$Ml9Z=9OG10qid^flj3B?8d_@*25_66JL7G-ia5pDF(CUI8oVt(x6?&c}4_)Vvm zsFQ9IbLCe8QAC;5O!J~12)qFa+uD%wmbY{TkuvXj5Z})WA5laqk2l4PB_2kUFz#CY z2#4eRIo@xCTC`?%;mDd|Cq8BVQ+6IlQ7h<{==ofShR_cNDVb@zD1i(<<1L$1J<(N< zz0VPxvn-=Z!V?^(lsB>!EEg;jBBZIuCf`1tb`bGf|L!t1M5VsQI(-);eqYVn z)aezR5(o;ia_gvV2>wKjuE#kfN}Xrol%EPEwjnn8`DJL42i`BKfN9y92n%Yq4qLT) zfGM?5lohG&Hm(OygQU)Fq2477&TN!dF2k}u-|1kEKcHvMK!UJ@n%>WLZ5GA@Jm>W1YQQs2kVPhXE`zFqP0K^l=m+LJ@^} zVN|;0Lc4aAE8V(kUpLNTparaG;Gm`OUY^r)<&B?=)9{Sp09+AVisV6C6@~BW#EA}i z=o3}urA%qW5-kl_CoM8p<>r)?k`)+nap&;H13Ip&TuvphCC9<%kd{}m<_C6+^Z=f+ zA1Wf(ork?Sh5H;0MP3Vm=^g!yk6Pu=G+^*|$}c2p^Tjekq| zf~bpt$|w&eq`r*j8{B6*hRzfABj1z z5`9r)v_rE+EJB5pb8i=5K*cKYOSsg05W}eyekZB|-h|JUy?LC4!R6b2NV3L0T7iVe z#X>=M>BsZa7ZnRI@MK_&{5_{2jf_~X3!b6-Z(L01xuuAjmT zWo(q7xm$3z{9&iTlv7`1>MC9}sG%_Yi_}FAEjGUOiIYiPq5+3npf{kOeE5AVf$f~Br<`v?eANs^@A}b6HtJG|U#we&#YIoMtu$$F z{;i5By6$A4f+w-i0cBVLiPfRpSUlZUY-ez!rS4Q~JCAr+ZThURkZgZs+bh1vt@Rt4 zq@z`WC)s)@PJ~sX-+{i_6WK)_`f+{R#B!;wxUED1dZsgTO6XH+a)Q;^BaGpg5vTI0 zrHl{{pvZ`;>xCmb0Y}aO>ly;Mjh~mrj~UIh;AHx;Zq_&8!Q;AFs>Dd2SH zq6LSaQ4+kBAU#7?DwP7Bza1z~`LTfsl}8&rrTG2j$yu7KPKJg+VLSm5s3IzP!|mV)?ca zwzTvZ<^AY{`}$WsO=6i4;>+#z*kx4I_RH8^VqGEp-3Z14xT`@6Vkl#UDOsuQ(vOPNgJdMpywe#c3mJBvrW6$IEYB$HC~D=lzKD z+KKwD7Q*-xX5=!t@tZ7LL#(hrF^*b}KnY@Yw)TRakh8ySg*6cs)?9Vq+DaXRjk}6J z5tuN(u{6#Tt_aznG}JsN7z}3Y;z7ocWM0NFl?ShZp>~7dHh#6eF&nKDACN)omo5gH z;1rRMo}kHAVsGx8^>_CrU6!cbV#|6vw`qJOkbspM%>|=3Y0U)H^+PUN3Z=9Y5YXg~ zBJzVpImzPXD+FyOa8GPAgtG^#m4hH~ff>qk_LI=n60P0Q6OI;h5pjiRA2dJf$_Nu6 zGyh4uxIZ%BUs^b^&tE+iBIMty=|RuA9!RcjxZS0YhxMYZ92>7#G=iK)-?t7m(JmE5 zjnHx&X>zbJJsy6J1Wu&Fu2S<^G7%+MCKHw4t)ENM%SQzIgh`+|N4&aRnwgSL&~kKH z3B`A$(MhFgV$CLn5C5L4B=^c|Z?~(pqHLMpq7+IwRD#;}iD=eo!aL*C#lMO8R5=i2 ztV?vvFXTsbr5SNiuht*pe;X*5uY)HbD-gh*?(vY$mVKK4)fO}|&X z^Wy`_ZJZ9qw8J0kYBTu_b`;tFPF1&Q<%gr1N)={1%&ci$ZQTqGrcFrL9eIILF+vr~ zIUA(ru@3KqMIt3L!Tm~{Y_^AeX53zI=Y_^p{H*~dukV=Bt}$}l9FNWjCN`mz#lH)^ zigi-S89O12je7LRm@9cz=^W~Vyb1LAV#$)pit46zVz0ksoRv?Af0@!q$;HRP<{A{k zUgvO2?8em3Sgo7UkuJ>h(a#G*^y%|v6Y~i*%(Nvm59VoauG7VRq=(t*Rg7)z>(yXm zp?e9z4?Dm>AtU^>cPO!^KzCDUyObi(cKRu$9fjc53#cbHr(hPtl=h-pJ&aeUhWW0Z z@$6paMFQx=iKk687#RscxO-blgiF~_p!wNAE6a}0-&fne%*575mne5#eHw6Nblr(8 zbbeZ^7AM<7F-BGUd*48p?qYRun&3ngWgyrelN}I$H{>mopwyC1V*%+4C>&Tq0%bpIr-#J zQwqvo(V5elHs+l>7zx_=7{qXj;-gO9uA5YEscy=KjMl>FH>NKEX?HYlhKdUNo}e?F@j*i7}+(_eW1ur zZ#+>78MqJc^Gdv!8xM-jD@e9=jhUz3D?BJM-J+ z@1$NMC~{0a@@t`vh-V)2p0d^yV);I>#XaIm!e>QC5DWcA+dS%8hbRq(4ta!J0;TCL z-94<+&@swkW)~nK{3kcp^3!FcpWz4kLPbOxQ$CVI)Z+v; z>aN2nEyGgZK0b;md4;FG5lRv6g(VJqs`sseL9AhS%-P}XuwXVPiZ;9;Q`^?TUP|an z9}aLgy+NEAY2&E*28)L)3eXzmRq$9C_7egwTfT?vCyDNYS=ktCoU;e|C6Y^H1eHaF zx?yZPsiN3viI~6DC4X9V3GmG}Ol{9Y;9#mrX=HRj=P8O|Y@+b5S=Cg)a|z24F;>EF zh?(^7#uge^`s8XXlk_~r*{ywo&(y~YuD6FcG`vc;>5F|5#(!xUQ|q;;2Ut*AKObO3 zpma>3XGEZj!Fx0%btxzvWb#!VTc5&5NCWf|? zP_c&+TBX`vYe|EwI%^rzU+NYXT^s9!t7%-AQt8w!+GkghtBeVeD*Nm;b!6(#IxsNK zJ3%%fEIELB?9WTi4EZF?4WOOnucUvblgJ93>%IL&`_f0{gTE(ohc^c%obd) zf3PyYxy)JRVl$;I*ncteKzD4t!&45&NbJ+?R7gz`aKNH6Ys$TwrU&?#gA;c`psLb8(!x)K z{H}CLdE)zVc28dVx!HM}aAIjOWqa3*_#XXi|4)J#GpLi=E7*e>doH=iCkV=vzmJru zw7`-dI_t+ZK#0ZlpP(&~WQ(Wl%M8A9y}!$wKgA*t4+g>sXjXB|sN&H@MQr;msk~#g z{)0}b3~Ihs(1?1_8m`6NY~~)0&}MLRMso5EM1(?b_cWi@?OC5ad&j+K+?@@olu;{a zDl>%^$h$EFd3GadScc;t`K+PPw+(r>1_KIjSVF`~GvXL$-nv@O+Y`Qqyu{jL~QSB*WU zaC#-PGD0opmxk3f{PYrgMi})W6}R&`>z%D1Ih#fx_MXM3X_DC(SJ&#AGXDt_I*3gu ze6BAaXgfesd&66H{%HD(eOzhH<8}x-Oh9y^L&SEi4r703Rd;sq6Li!MtbVrqqm3Vk z?aAMd;`|-H#MSMUMzDTNVkg#!?;nN_Nvn30&y?2N`T+O6csb>6_mS7!Rg+txhZ@Vp zs)JCRv`?RX{_+Xom$R$hYWvGL32HJcpflywm!VO8lX4kyHM+^l5Z7Ev23bX!YbG># zKxowB@iI0m4j8^4ww>no9dKW)h2zj~bwWWwfuzW>!j?8Pd;18a`ErcH3plQ0Y%Enp zzV64U?~At5775W%uQIzwOzwL1#T`iIKHHf@+y=g6HB+_p=3~BWYgkAuY+Qf(ZMr_$ zSYK{ATQ`Oj=rmB3M_UC0xX7}@Kpj2tXhv-5B|Na(qlXkM0scMR=V075w z(b$ThqPr%(BxSh70Y6yr9IF}@CQ}ulf=(`nIs&32=HP{VDU2=Y?%giqd`fZffHBwB zE+mBF$VMy3gIVFNJQECR?3Wj?yM6`RPPs&5ap5{7iEpjY~!CjCQTX0dJ_>35Fsa_hyZ^gh-(EFFIveeI91R{ac zI(3kb>|)SDH*+|k8KVnCV@UKX&DO=-#-*Q=$U7N$_o7}};FhBJ%9y!t{!kmp8%sNc zi2OTspJ}sF34BuH8fY@Sh9h_L9ZzPxzp8-{RKH_t2!6LwOS8nNznYXz#jmoSup(A? zo34~uk#tH@mZ|2YUq~0onW$2Q=N%{gfJQy$THW%R)OTi|uH_hu_+=`Zf4%6iDjj^V zr_|nL?g1uLVco04+I|6bronP!QNC`v?p)zvUO7nJ!Fp-_La)1|czdDG*JVzqjCW|? z=Q;d4@%bt`UGx?|_CBvZ`ehQ9Y$1tP!%Vuws1Q*bKv|*K2v558ZIW&Ycl=IhbrV>P z%0>_E`28VtC}N?9=@=rA4qr?o?O(zuJ>Ph_%%P9WK;#FD+-#=!keGk83dIAmjump@ zF)cC)a!}CjeK)VzRVJ0)57SvB<-c?rydlgei<7*mqnZOrR)^~h_+HdQu?ZAE>Fk>N zY~7w8#q7$X9&HU3k0MZCV~icE&!X0SiF0;F|6MtKoaBe+k@wk}+=L1Z*rd@5^_c8& z+*##b>$~?0L8HQo0>gIEyWo~r?Fm)T1C@|rfh5B^Ui8&G%>D?qza#73(13|B5}P3> z$!bP1okpLQz%w2|1dK7>J~&*k8$%IfI`RdaHQ7V$PD=D@R-ZQ7UeMo2`(iEe9cW%B zi=v=kd5nHFuKihP0eUzr+!O_dsOV{a^ez@pC)i%ZK~S_=O$trtlJE@H^`?Rj ztC!`Sqb^wpm?Ss%J2390!wg28vW0XsTqanvb3$AIFmO=Y6vnIsP1Kn-K$Zm{;`AOcxDt=eY37 zT7Or>D=BrIK6+{v;&YFdl!c3rp+xe?+1|C!Ui;zXk70n2G$hiy9&+zWvSQ9}S|m}^ zngQXI!qEG+jPr*eb%98Mr^C&09?0JhBz|oxPHU#mzM8b(>>9@c>vt$;gEq$1b~)pE zxoVty8xO;>`GyFq@a{&fLWM}i$!?uJzG=dNe`Xe~FB)d(Cwn7Hh&sPzqMKWo zN~|5tUcjeFCL@fT(PE0D;zbVB*sK?lyHF>Cg;of0ug3_4r~Gxf+__}kB4`qs;?s(u z3xr*0T}2@`e^cu%LZGIbBZwlfaY8HLlPEz|>^i0aeDEpb5WDtavcHi^MgsVP_yob! z?Qnd8-9@+sV=CUM=Fmw&JW{%D6GZUt{=8)w3MKM^xFCd!uufl{+!r zxh){8yXVI7Um*F1fj8rD#tSL|QfpyARNQSh{5?MmM(&7%V4q$LzfWV!{1d3gL(6Vs z{l9a9q@I*47D)uXPX%$*gCQcKQ;ko6dq1G{*^gzNj;@qyv{pRL;1-LMHhjRHx)##e zwE)+dNqEwr*_A6HJJo!nD>zq&IxtnAevyGceNvEWv-5MSi1n{@*|8xNPJOfLIY#oZ zc@^Y=${H{w84=8MYiPnX z(U4k?ze-=40WU7h$*@gzK))B_9=9-SiOjtjip=FSZ3@#N>Q+kY!5!$1=smT85rvZad&n*vJnPCi*B!=UQRid*l4Q-gqP z6^!X#1UmajuOu+}xJrCJ{7UaCZ*k9ezA2{jTshf;T>&H38Z(9(lStRIrKl~WvTv`N z1$A>-STv%FE@Qz8q%=IHHZ|Jc35Mq<6!H%UV-#+!sW(NUXmToWTLoQZe;v%`>%8(^f0 zTL1QNI6b@J62++T=%uWF&%OM&oegArn#41*jlAIxcStDC1!HBhnwI?qMXL4XDhJ=z zFwGt4+@D79P+`|q`rb#fVc@FNG6=a#|szyfZ>i^ts_8)Urs6+tGJmKHD*Qo}(s zQqN%Wc}M6qo}>wvbkamWuhXoZ$Nh{hR`)ar2$^phnz-&v=#(5fi=WX@7{%5@7xF*? zjqv2(svz zkgbk=qn~bc10q<6e8+y|^oGUR{`y0+I*X7=)Srv-rW`#nQS+IeXKGllDv6|py8%jX z4@Yf0vH161_IhN6maqXHfEJRX{0fegBw_rUA*;bjSIT>j+LqgTws@{>(~N*!b!V*b zw~BGBSUTul7r`~AHv;1ZMgVjK|T)6|Cz5Mh<$V6STn=sx+>c+mlNB$|D<7%VX zDe1(v!Y3w!i*O-3Db0$CO=pRlKq)>AB%fY09#*`Ar3F7JTX^j`6|}*79MtmbpqA!sCyx1C_#h;cCvgeLlcvng)uQLwume{ zPK2L6mi0u>ZjKhPy11jNHTe|za-q^R7rXkbr>bv+21o=1+QlPc<||4Qa1h2{z#t&#P@E+5x4 zi#{OGkNy;`6Nc_Ef~lPN^j8%H1e)%18Ywa_0IU)^$K{AQ$B4T}`t^dSzleR`bVfKB z6ez85_CfqZ5?lhe!)~5}mkyk|Fu!B{Gn?+zSm{U-=BQoLkCHI|ks}4eY1S!Sf|afw zKtXa{0g#0=znEZ40Z)h?>@pj6w8~l#>o*~Pkv#)jqd~Oc zH&H(p5c0l)Ww~^zACm;qI){FvEogI~gLEaQWluEUm4Nw7$_B`f@()Xx&zj8JM{mT2!{Wnq_!62Z18!ur zw+0vU4ZO8)a)Y&$(=0PnIUC(s4bt~I`d<@j>9&j5rnVyqQU2l|ay>cj z5wag1#kLw{Y(cHmdmPZBp;MVjw&wn#knhzZ>mZ-mXmf`n5L~LS*$QQ}nT;)-UMof@ zv(0rk=vN4~{LZi0;h3J#bWFa4_YDM8`MseehcL5lpAqh^zL&qCzYHA73ko405qTNm zcfx(Iw(rXZfPcE@UHEwpwMM@n0xg36gj7+_$`E>(=5Q;hJlil;Z{_C)tb)_3D)no6 ze4x8P6zQgDzwo}BQ&PRrUlc+1yg9zy$@FUI>qZ&_V9L7={a=D%dTSy%{#tlfXVnUV-hk^mFJFbLNOdVVT&zSwfg;QBd~Ud+sA$60*|a zXS$``N@30kqs=ni;ufqHWKtr;VGCs(pDF%b?F!AFtcohW8dN}6a8@vMM&l}9hvcu^ z#p>Y{7xwmp@#~8s`j`oxZ9^+dR}7l3J`I0C%Zb|H5kejPn(yC76U^d$5C5hfHCpwT zr~yy(X1j$MxxCZjG^C*ahJ#9A)VY2|VO35CGm`4vXT0(=Hq_95bU>(>WMSl_5N6MV ze?{VV`7{XfSf>lZ_wPnZ(q)vCUTa?hA)sty3JERRLoU*IK2}&^Ew4C z)tH^Xd*#ku-MUovt~c{3uZ8e`chWu%7EdR~EFy|U#mGng%kT!o?K89NC0eeB*e(zJMJZ~88eOT(0tOx9EL_Z}+Fbp9tXtgD7Z}y2T}n0Qi(!}U$#2Fc zWEFjzor2V=(UDvO2UYQHicH~sHB)ee`<>apEQ@y7eea#{kO&5IQMQ#aE$PZqC{R$Y zhGgH+AR_C`&Q*LWGG2dmiy>7-^(=a6;WB+XeqZeHtIy!=29!36N!D8%UAe!zhL)v- zLL+iUubcR7{sVA&>%yf3RbJRB_)Cl9)-usW25s%O_AOq>3V)OT$Mf^PNI8)~;P$)mSnUh(^e zq!Ep`RC8Ob_7!SLP+*SG{C@nN&YF+uE2^JL3sT}oKMgG4tz$lQ$#z^iuY;+QfJwLV z;cgzW!L9H^X+4@k)eZUl2oBVZmN4vYc<}}q7(UAdI*GuJL%~|qNG}NtC4*HWsf3Hn ztE2XGOINjkgN8_4amkbKke*GOU#6c_tR9#vYxzvoRaPCiC53I0&~cLLTr@B6mph|b zQEg)#x}9=ozo`8h$#DoonDuFSV>kY2OpA@|>^(2ui*<1JaJ6)I#pBA(F2bH8naYM& z?mgy_XXHtG>tYWIwOI5ltQ|?n3p~*$Tuu<4?1sWroyox0qlsr8YckfUaej77s`_c` ztx-_ZA?^*Ry2HG56`NY0p!9`Bgw#vR5H``8UrK9xpLW~0v%tZ&ME&s135l(A2bUyX z@RUZ0jr$7HT`?*j;)m<~siSHsQDa-PpJr?NHp%v4(-eg#9P2P|Xd-sU@O*aM??Rq4 zbNbO6RrZcD$$vevZI?Xgl9EX0NUTNG>L82~UK_Jnxk{!}g*=#~qw~7ua2pTREH&!d z+l)SovVDko%_1XR7N#f#xN>JG8Zq;<1u&}Hg?cX`8IMC`WRCn=^Nwy-0hG7@zWsVu zp+L}HoQ)PUqI*lya1cA2u4rHZSpOY)HC9!|5K~k8R#9C?O*?0)KGdc45+m`1B2zt< zEXa9jC9_hvMC^l~InI9WKVv;uX8T?Yk6@3-m+^gu0HOq2@o8c~#T7ZEL4gy2*)=QJ z*<=P^!-NVlp5Vu5A|$r~VW{uK(W@0D3kSfkB9BJgI(6B0SmMFv0sm*IFZ#{@A*CPM zu-rQYoa$O%*a>1^?mYNLG@Cb%Ji=9hj`ESnb+~zF2sLb7EWFeNy2^2!3+_O5mnq#J z*w58dCf|(pvIl|mWi4NmdFKVu>y%8%k{IQ$k(AnqYekK_Gu57TTeH*s>9tocf>;X0 zpo!6+0<1b_bawG0YxY)9uZvu=l$W{1C)$(KILR;-f+$>tElbf+1XK&b(YO}JF)1hI zfkn|na~+|xZ`2tV6|mlx;UQwzgLZMkyHC5Y^tT8fCt*_5Y?wTSRu>KS>!)ET?9{AV z=k_e3h3Slm!6qYN1xuc3m1$uN13)hxhi=MqGTPBbuOZ=UveUx*T|}NGc7wr|5YY%J zFog0MFumHdWn~xH&)}kgF6*MwY-)*|v#0pNf23EX=LT(u6h=60%Y-3qD zcOo8?P0B-fX^ZLNMqPa-0kb*@Rf^wWOinLaEJd|lU0-KrRcvC0o2+b=CujzVkc-r+pmnC094 z1woMD3nFJ^#rT~K2D@E`DklCz_s?^sx(a`((-zs>n>wOV{1g4~o+!jqL(v;!OjNjH z6wUabW{|fc3Ig%t3emRc1y`dZX=lMHLVQfn%QS0C6@Pb!FEu#;8RK{Wf z*;(}&H@@Q!4&!1HgwV^SDDxtSE@>>Ydg|{+#Imos7ecDitOyinT3wP7<@EOQ)g~us zW~=M?TU1}Jywn%4M?3ekmJyQsD0|xf?yh-KgLYy|Lni61+QgHMRFC#lL$s8Rn!khZ zTPreGKC^mVYPhA$8cZFRWtUb(TXue|mgQs2xNnDHn)$xgBpM1ZF^d?*9GmG^WUD%? zzpl6(pMCwr#x7Bj`5jSCLH+ey2lF_`S4e@EwbB3Q$eZ)a3 z>X9+=%LPTrdLriJY+%zkHV%>6**m;a+0XBET&oIeIKY(1mN*U!vn>-G=48=d-Obc1(qD!!BOrps3$R3id! z6=k`wZ{vr|8S9=CS9tK26gBd@Zi(0H`s|Q+pPm%6z1OlGb6uy(I&9kFwMR#!oUD<7lkqI->-O+fhv6Wy3({MS@6gsHKeP&>IbH4IWN#vl3ZOq6)+~?Y@dC?BJ1)$a11amWZz8^wCuq8f(17D_<-K zG(kc#*FU?k{@i|k(j&R1oK>WRF7ex(H(Ffb!^?f)pt`ai^_Phb?C?2wO=$k!w;{oY zJIsAEiyhVoPqByY_A0@RQ5+do9qw~x$7`~M8i#S_(k=T=Z9)RJT+I^zT;0C*qM-H% z4`LqC&x_Z0kb1PYaF>zhSZ}JpMx=p%h0^yuwEmT=i77|!LAvohCsHcl?~G*Z{JXL0 zoc8<^$c|SwchV4Sj%Y({raH&as9wd{x0y~62l?3%CztNM)?67tMuM;So`qFOuEm#i z%tkv>vL{{3hp6N#OK(g~a&kWOW`DsfnLV(|ElDofZrhY@-&dt@4JTgXn*}cynUhwl ztfW7ep`Yys5UhHUk-IosBEM^wXw#e>e$PvMHJv@B={~=@Kl2UK93%fdZ%VRI`;@=jFkHDZ7`+WA zCROFlyN@Qi-Zw3?tZ9EMqhTSBDR&p9e54{X*I)$ttpO-LOp}@PladwP8^h4`iG#_e zb>P`KSIDoVFik}*&AL8t^S}Wl3Dv-E(VGgf(ug&wm^~?Xxru2Q(W0!K%uslXH7b*G zmXZXekx>`b?2Nrc6mHH3*lbr&~|yrWwqZIJonkqkKC=5KhJjej5cMG zlSKn*`D%4#5_et05kX~6^v&stN6_atA)ZvUcCd`0Sm?~SwQm%^CMJ8$e{yG(^b$rHt(dDv@h&R!ZKLRkC{pDUMQ~=n zUq+tZTsvZfBg-J{0*8C}LS1lz!{uW}d2N`QdA^0%-slo1Uea}UK$~hdt@5#PnusTf zq*v?5Bl+JABi*OlZfm65G`QbfC5Y9bWWCV0XxSx03n#%!iXc zAc#xPHLCN6b$pTRC}qoB#Mch_ zN$1~W*FCp1ktkf`f^XakE2kH!incHp18KzhaXn}v$hZ`30)$!WST{4F)2||R;RY*!<{KKpIApfb_%3!|N=9!pS;>Dl zrf>16-k20m53i%-fQV=D@vg)5q>1`W1$Afv``KcHDiBNUdY5>x{IJqTR+qhiB*EF& zrk31(kny)%eGj#YpHxlBCm44qFOik&ESCm0NIHJ%ek6_z(!4@w1xcrR#F-@U8Zl)U zp(w`Z#x5x~UIwAyLVZ7HK9jn6WTK9e9B95oYK2W67UE7j(wy3K@q&#vbvFtFVMy6X zcZ64q?rZ6oV*>OrM2z2ewwg1(OzuG2)Xk*+NLx)u4*tpx{h_Cy$@QNyA4ku`ahd%nk1c*=!V$krRk_ZyGBHygwAF|q=Ajo^~H@G zj9xPjW&%cXuefeH@ww#u{PSvXRb^`fzE%-7~ znLeHgvxr_{txs;p<;8tq?!Y=qaiaU?my^`iYj8fswdaMG8K(A+zgt**g&&^FJ5w0Z zm33Bu#T!kin$V6yBvpP+Ae9w@GKGlFWRV{>_D5%B;$tw#9m}@5<5!$Ku)?Gq)hYmC zMhdwtELRJeT@HQ@_5+$2MoJiJ*`1lyEZcHtJ-3_I8u+Ky!+1HFJj@q1B@6YJj?HFO zDgyO!V@OU6_60*=xv!(tM`y8!y2a=ba!jO25;oly>OpXv&P^$t|LsMANIEABZ zd(Ttjb|1S52E@I@<2}n^cj$e++HIK%)6VOoJS{drXUqTH<%J|{S~G3DY65AX|eXS%zF!E+#CSO`&QI>2>jHLIM+PI;P5LaGW}5;NX{$b)R&V zeOk@(kE63#lN5%+=qur}#NC}8?pj!Q`k`mn)AMx|klg#9LlsFQaZFqXvYpAanD*@C z6WuXv-TYHKb(`NN$m=Vd!KdtUIBM6et8=hrJ5nh}?1;+^a#*V{&Hxp5=;A1oFP%Pc zh^>_VrS+HVky`2c=QUmsESNFNvH-DYaW8k6UfOiGYcN65jhC@Key|f>J>Op{Zh@6T zKZOnpm-vkYqsf|lsyQ`OA&{d)y2y=8UWr3}>DT@o`3H8^J(H$r>I7yykPQXQ$XxZ4 zg(#HV&1h)d<~KpkmVBKj-=&ZrWp+i@OJBnl8rdsn?x3WExv+<1GOXm-U~-As^?NIt1!TZbuHtM!_mR5vdYmf387BHZR|U)b zmgMfTzP$?%Wm{8I#wJksl-!JxXt`Vsl@HVMu%QqaXg>>H(&C`I`Ogv~Lf6R)-W)S4-Mp9S+X-LV9kaV4Zal+dsg9dlP5~z`dG+pvUetJ0r_i z(XypFO~Rbkw2(3{IyF_*UtWIZ18%g54j6W!rSQHENNW^RgE}-mj0G<(xC<7U@hn5U z%#YM(<;Qq?AyKiV1Ya}`J(NE584nq^IrQOSCJRBa6_)fdUcb%p3&~08&62ZRf1x=y z2O>GxMd6K4BBx#QiSYyE`o?=ACS9xw`LF6+-koRDU1s-iqM4a+#H3E>ca75s4Xc2=bmGwfh*dlmz}T{x(of3tS(8Q^=F zDY$B-D2*cd)DS3vJ2WLsE;FEq8v7L&V#zuNgma#e5iwrH!QMmQqqembb!3 z)3Sl3tTzpq`Xvsq4={f2v)skwz?aLsvc?pP_f08ofyfTC81LQ9-<+Vy;w5I1aJt_xIzViN#(|c6Zw{N4$j5XT!D4& z<%(Z{y;$Cj{KLHyg%8H>bc7Y6)%~kQoA{8Sy>w1qTx~Mqw@T+)`>43khc;6v?D;Jj zT@3?+XYAM@DNIm3#1UA6Gvm{=Z}NcWi%NrL?A1TbEu7k{<`UEsi~P1gy}yRY2f;( z*@feso;NEp9WhZ+Xv-{3Cjs6o?4igR>gT}mSZy=OlE6-y zz0@Cd^+lt)NSh*RL!L|vHdtyC0it&!Y$0jqsc{K|atJYw$lXaD{sExZUV97K$j zKvmA8TQ3_t?p2$)&Ce5J5yN3xd*Y_#{lVozShqH`OQnD)+bvEa@H)cBOiYwc*$|w2 z1|pbHdk{|*B9!9TtLRs?e#3`^0$mANJJLilB?GHp40bmPxslkjDBZbNZVN}YEY@aM zCrV<8)2ZNSNdgj;rx7Xy!j5<$sAS7 z>y!?|sgTPC=5x{_{5v#m03*n9D=H(~(BbU3EmK_3k13ca&Ot&7Z&nGT3O?>B;Fps9~y=UMv80SNr{dIN|l+n#= z1cgsd;}CWIg#OtAox4tUuWh_{#tLDg8v*rKHWU(FQih|`p#H5Ml(lAWJ1tdc=;g(B z5Ro)T%R;H@_`&c0Se;&A#OL*Wz|Jt%%Dl>|FD$k z+Ig8CML}kN{W1lX``nYnStCX#Jt)fb{FyMT`{K7k=J5AdQhI&5@O#lVj^Wt5uVYj_ zOty@C%;P+k2S+;VF0K{U;;3--&cE&l>46FGcS|t&&>5@UYH~^78GG8=xOw;x#rB8GM*?fC z0rh0Cv4)yA>3Ilm8C1lDOzMYFYj@>}wH-Uf z@E1h=fcP0T;hnlZ;hcP_v@hwidZil{o6n}zu#5Xdx7zRF{ zRA||1AF11b7xYc)GGQ3@-6qLR2{I}@6E|NYCYl({{A!FN-Ux~9DUA>om2Gnf$qvou zJLSO`N-DT%G8eO#;ipb?x_MyKp=A|i$Fdw0{Ig-WOH40SZHIWPOg`a(C7v;g=Iw}u z6&{B2>MFl<5<}B12x5X8?A(zCU-twAtsht;jHH;cXT!j$bx$5Ug8S>=SfkMa&Og9Q@q2=1HOdvy~j^4l$~B#BdLn# zt@(XJkq!w{|G453dq1e|3N{cU{h{WYmg^hWOL3n$XRD-0-+Ko>ryUDPn1T8@J#n8% zd%iq3qVB{5CDv9oj7DCy3Vu!S(BiCC`*4Hz6^X>jnphGjK8HTd5jUMGWmWG(^4307 zGoVumpZ={=SS(|%%y}D%+djFh8@c8Z3!Zor!7Z2VwqshF{LWE(K`wAr3h&4WWD4o~ ztMbR9T} zvapwFrvBz+HK48!cG}ih%2gEuamS)P2!Q<)l0rV3<5tQNqlP{%D_Pa;1D>ulz=ffC zep?>Eg%8EhsVeQ($hq+I6@%phGX>L68P!HZt+1zFrdIV*t(Tt6-XCK`lPwc(+u}UzKTa zD#RoB;61u6uM?Xk?H;0?UVwXVWi`A-#q0F`r_#vt{rZj-da1v{K3=Pq?6LG6cj@I zVf7s~E*ILXG-75Bauu_$X1_d-Uh>$Gh~i1cs}9=s1GT@eEa54NdfD@oC4Bcd$?vw{6BI8E7Mhfd_0xrJENLIZwgh&k$=veSYR0Fk`Ok5uZ0g^mh|f z9pj%NB|^@0XJcsWg}@^B^2~cf$F%X>!5+dL4@JWvZ4Ea;i>5N8-e0L0#ftd^7Fu{k ziaJL2TW%+=EonG}3AQLK!SZV-+fRRLs1@{qhql&~;b9=LTamO64PK+bj@WX5&uCe) zwh~E1GclJ6SAPw5f$-hq=3bQ!Iv;L5FU)?3ZANPKr=iD}KANEpyDu%=aji|8vrcpi zR!9dA6Gu#6Y|YA-v8L!|bjTz$NFvGlpj#y9&G09HJ=(*g!<*nE3t6|x^wHFeg3MJ- zsGe>&&jzBXw|yWw3(KKq)0(BAEK^Vr*J3YN@XWW3EUA!9LtCS7;6+{pY4i87L9IM2 zB3NjhSBCNYq8{e*;AL13`vvgX1;PP=YN~35t$~Kfvj14^2h#kiX%f{8+2@ekRS*eB zY&uFoSnI%~rnYqy{N(qBQQ|o4Q(%bEi+qzr&2ToJP4JOO7Dx?i4-D^Dq(&%5`(0@! zn3XLrK?LQ=WU~*ok}IHYXHQAl1}Wxd26N6kcB`SP8HXQ+d4)V!hUsDDZI&G=m+8wS`(}q~#AI|5$oh@x*>3Dli=H$(% zTZD&f5Drw=Htec~p_5mUqFpuQ(2mgK81ryGjs*WS4%e8(M6Xg)79d;0%&AendBGrYB#8DekHE zflA2TgmL13n}|efWk2B=b#px8USVqbDNQ*YUdSzQ)+%e%v-nZrltJXyc&s}R$gScc zOD{@+NnbFusfTr~eCHwx#HcGRC-x5&+g&uS^Q>_%>M06;9-}D4!iL0yQ-;XnD9?K7 z>BjJ`xp&`>?Ql~>T5w}Mn@~zcXsBSNS%^G;oj-WzunAF^+h6w|8P1X^U>{#e`+Zcq zLkky7mNn9=F@S7?Te?OZaJ6h*U+)xRc1Hzc(xaXzS_FLBK>-EEmAP~7LC+*Qk_^+H zMUi+N3fJV2-Pm4=!G+2w=EW3ixXF**-{+9}X>%I6Fw$}5nA*hMpdHDc1xI2T`ignX zT9PT@rD{TxpEHAuX^+>9+6^A?uWXVcPmR3b`urF9_)2$es zopVzw!4pkroCDM9I}UZ%D|5OL*%Mi4ZJ8C~?^hW>qXm{IWLq#l85x$Qr-n?-8T8L$;kC(uhfy2Ey9W=*Lab`T{io~q^s~#G)nEM$wwnq6G zix#?Tg6VFb@w`ZuH#{7sb;Sqh9AnUama16$GM@fKRxwIDj_TS!dW*YmWN&eTYGf4f zdqG1~kjO*&3j#clb3PG6Mx4^p7Z`isn?0GnS}%8o$oHT%1}oU+@pIRmO4#CmQt~ZaW6|dmK{+uP@_1oMIU_#LQ$k|CnVaEe zl(U+jSfI-JY9mCE*uf&rV`s4pKz9^(GQ)pQbS#|2Xyq4gmfOr8rG>#yN-Uc)2aqbZ z;0!(=1qeyg_M$pW&Oty6|ntY zW(#5uM-W#j3B$a`3pH7Of@2PStVVJ_`M)eMmcdvE+2C6dZC1JCO9f<2P#3&v=MYM8fWlQPgO6d z6dxFp?T`|nLL(C5lnJh1^D(xeAeSC@dSS?drg>@lmFx9UVajgyKv|c-Lw%@`DUP+) zZ43C(_CH%jy)GDTy6D)|P=kA;1ETTUg(K&o*QKvFZF3P0LTbCUh^uwEN{Elq5eK@@ zhD>WQUk@oodot;^40#f^2Chz>Q38(6?{wF z%XGw}XcOoNFii=p!Ha*K98zT~|Df@e&`8iDa~>_?qY~|B)^ZBpdXQy3U3(th_@E!X z@R+vMjlq{o&_Sp_ z1{0L&YuP~R{jw{953cYrc$_v5`Gswmza=|qUmzGiYJQ#H+T#Yhh)X}t(Xu5@w3PlT^rRxUo%IpVMp4pPlac_4<^|#&FE71{&di_qvoIleK zW*p^%5>;}x1-0+sGGSI%XNozUm$H^3!Bv=jB2?<^||%-_?6N;+I;gl? z$9n>m&IX7&mL3e1Z@kA?dkWVQj|b@abR8#SNWR6d zxe?5G2fi^0qbkR9Go@Bz5(swDxJIULYFnp{Tj6_}eUQz%$%iog&#m|6Pq#=M@P zqmZ%NTa-2m!v$Yy)id>9t)JxGgECXndpx=+|Vl%qpsyx~G#DsFvJ^2HxasJ3y`cHIs6 zkKP`t4NurwQyil|#W{bepL=jOlEZ|qER%>vE=XQG3l*&p_205B3`FJu4&hQL$i8a9 z8K!*hl4{ISK@NGPNEWA*ZBINoaYXzz;ByCVMb)RQL)I``w7qMn|D2Q)n0moCt1=n~ z$zBCSRNc016?)pPr!>6su1PPH`Oi-)?V2so!Ux-1jw#`qIUv4+=bhU#yg-|{khX~@ zkZDd-yiV`>%%y3j!_C5y@IzXd3(8MI&>aF3t9|3>dL2AOMc79{+1TlopW{*~du)@B zIDA;Z*b154hRciNH50<`{i`GM*nzd@#^R~0Hc>io^%n+a;QzcP20a!5mxs~@Ts&fP z_xZ_l=W@!F9vpY8ED2LE5||^25+vtLKW8es&trID*pcvBdRGZftjn-^aB}XKk!Z;Q zJ3X+o?sDUA#gQ(7I3U`4Ji_$+X}NTk1Kqc}O7<9&te!rUTXl~Gjk{q@?cLEK8z)EfMl#P`Ay>XA)KL_~@CASH^-@Qi za()s(fu9%-_qBTn^|h$Q=h+lYbd31D@|a2r;JZl067AohtXC(}yOpLiM^P&nvIpUi zvIuyDhJNM+LinB&{t$EKFbtOFx98MeVFrwI4*`kg4!dN^@ENO`I6$CLOn|Z}PfT)} zIQj6t1`}2I%M;pH2l&^6;TP!Ip3zji0!!tp?A?o>(;dBM+R!pmof^!zlC&U)Sh8xi zgx^;yZ-uplvZ!JM71ol2gd67;=zzN*mZ00;$aeyblim%xsU7kJ`8ml9*YR9sKI=kH zS;u^u$uNQ22n8QkVT1e%T5BywqQlvr4Z}@6Bu#H;9*e(-X>+v%Q)26>6U9#4NqIIq z8l+#rS)%VT0mt^%scwrBZ!X$2>g#+m6PONixQp=WrXV@4uVu2bw_$Xx!Qrm@<%X6Y z2)0iVe<~DH4}SfVi#%8ZCMrBcW8!ci(`6CXr;My9DZ*Q+Px>+MC>JhyIdFE|$KLa~ z?$~HN57TphVFs_y>QzQEme2hUptt>7sHE;8y5i3DR+1 z?4F$>g5yZ60O@7kO9z@0^63QHKQ~kT0wrX55Pn`$1kYMK4Pc4L%JXdjkLJ3s& zwEy0{eSZWFkMsxjJk|tyjzj9g`UcL2=`5#zTa%@714)F~0Sd~A%Imn_(eLT!9!3g< z67lIKU60COOu0}5Dkj$Gy?+)&$MJ2ksC&_>HV_c5A+Augft=8?mTZLKlLuHDGjGh@ zjlL_AMxF9y9Z}FSV9?WO`wbTs*UO1`q@*h-c19h&|3@4W? zBaN;@vncSoqfur4y?j-;As

9NlHRS(Jm-dd=16r^~74{V+};w-fkF4t%=j;#;)vkl6({(0a8W>cw5Z2_dK{0PC*1=`zh)KuVLp zjA@o;4ZNae{dqF`IPN+ikFwPj1|x%AJ5FN|Cu-5DF-_+;4_fI*zn78kxtoyL?u0uv z4jk#172@;m7VM)S8J_&MsiFx6GQIDWw)xO8;D9*(v62Ue7(}0R;jyP1ENJb% z7pMgGa$LcGNSxkraQ!V^?@emvHKU8eYOIv9=Nlo6LS{6PAJgDy&}Jo#Clc0tqTQ1r z)eCAzms8CXEnP?l4iUd#p%cSuDtJ@sSi97)Xq)nKiM$G($V(qOFP&3*gJX{-*sLb; z6w?cexA|}VIZY@rw1JOxtuSM6ng=Gq?E?*7hZ87+g>I#48bz;t2y zGcOO>f5aKLZ;LUDN7FBZMeQ?bJs-)@w)3^>;yz(SZ+B_g9DTebE2;+~8&*)t9;7KQ z5|LVAQ@V!Hk8N8tw(e^fbM3p)DkBA$X2p%q+xhvIso6F%qwv!+snht?;YG;L%j7RH zWqMrrchQcUXHBNSRtZ4&_gntM>$Uy=w9TCoEq=2frEe*`*~>Rc`I!9G$cKHhJ(}E! z+e&_mx+V!jJb2zyF~IXDg$O!e2Ink_tkB6zjAZvGFcn@&^|8ZWandthf4KZ}fQwaB z5H4*hWI^tdnu<#m)5>jC6CiQ~i|euP%2#ipm~v8>6@u`V{=AuZv38S z%I`E(+Fn1b;e_AQ-Lr}jxNPK6V3>PNLG&a1=&l_6VNUdI8&rdm@oemsOZ(Av(iU{_ zJL9#P{Ax;e^z{K<+jt>%qI=FRpk>KpN#67fmo%Mz9L(x*PVAT<6{&Np3ZXVL89f`O zKz{fU+p^|wX|vqud96(&z}v?g2yfr*`qu43u2)?dxXG4WjoF0cS#BOjbhW*={!hpb z2(KcWc-N93g_6ui=PKT;b%h8!IZ5bk*H{tixT$CIOI}oii5aU!voi|X7h`_QIF%8? z=)G|6e7Bo^*iCr;98t-L!^tI5GxPGQFt(EZajbU-0b8H&fpb>sUQY(4bO=E#1h9p& zhy}~ku@DPysa}wXtBVR_jj-6%_Lo>zwe$0WH;i?Iay&J@f_W(bNfl5ew?XgB#HP-3 zQ?1;0Fl_vZs%Um8r+Ur(>y^FJ1(RlBl-I(mu~AO^3jZv9Z|p2LtA-Zb92(N!2*c0u z8o8cslo=ni?4fao(X>z7txF8I^r_j;%-f0(6fEtj1l9D8oSvB8Urob&7KEAdlB^jArgv9XYs4Q$%j&D9mJ5b@K9 zc7{1lU_*F#UD)$_7Q>S6ve@=*IA4w4Qot3-*`p`+cZcH1wnI^=`~F$)G1}| zJ+2c}o4+ymnPk0*4Jp=`5zM_nGABXtIA|=};!JBe1MZ{pBvFt`kfhEU4b4(WXZ$)} zBG&98?)O~S+dwz2RsI*-Muj;gBg)29?2rr!-Txl1>zVK%@!f- z(2WWtrgY)RU%e08Y;t9MgI>5G8WJlp6z zW0Ff$xRq5k1evt5^@O`?F@Ub*d~(iRIyDW?o$?6X8B$VyIK4-VC(_Lfd5bYNey|x2`@XwtUcG2?Hg284RmKP z;ullQ5m}xNYd!9kX|g$N3i&$>c^W>t&fz!h<;dY+u^s{!7b84HMy1CZdTTI{PDR8xNpt%_63~aA~mtx$JEL(@qNO8&b&x+~$Mf`GAI3{A| z%`z-dkdo%B7eCraXa(w}0#(obxXy4hUXqGr*vi^CI!O6z4mCe7)SaL+cf26_j1ce0 zR>*|*SvQ4B~%8#_Pru!=oCh| zF#+=KBFa^E?SgKqmO4gYxLcS|vk^M#xmzHD^Xo243)`aoG#GluC*h|RGu@dK8tzZm zdy_Ex#1tNgDxZu^X2%yZ>ZpZ z`v7B=)WEy>LuAI~A%(&s+e{(O_U@YXKHhPrM9IxXV;kB`H{0n&@rThJEl868MZ+!EyKl|zXKYjG|o|2^qwj)u6lZmoJF#T$HKELpUR9B!2(Z*q^Ol$^70vf+AP!dTSbkTOrcQrP`qbwZ7>Us=YoY)6 zAkx#=Nl_Koryy-4aWC4O;djJM%Oo-lGGl*6zO>^uMe~sdBTElAdDJU^L~NcDtI{lW zTTH`RxRzsdp{9l9lI__H$N3vqx5xQS2`fR&xcQDt$~w7CbaPiCj~`y2HMU^k+GtY< z6E=g80;0@isL5iah)aB3lUr(BncHGPg${Y;deHCI@SgV6NGAr1_=-tQk#OYH0&*tx zlkqf;fZj?fwX|7ysf*PY6{E*zE}=DNV%`oR9LJ|iv~9-9tBc|)G0-`DOXXTFXrO^)8Nsk zRs1dHiQWGgo&c@KsN$;!S?576XOKhR^!FPgf2-Nau1!y#xG}((lvF}OuUgA>sE4Y* zzF~^g5!SKA`Pd?V_oPYIlP2f~G&?QM&h8fOODGT8vMfsxx&Kjq{gJpIjExz$cO~VT zSvoI7%&w>m5p|9QCpCycCQsFzmhsxW{DQTMoc6;8veIGYwxL5T12b$VVynAB#W#Jz zFCiBxloSYz$RAyiLkw}9-lBCuIoBfinn)|is<7&}IYa@li)fQ=BWo+6dAn+kmb{`b z_z2NA$S6ao8R2YyeU0T|2EnhY0zviV^>9TVPc*xjv}rW6@g)`?vv*7?VfbS|58HN{ zC|tWIx>Nhje=G68`q$;pr-N6EJoORw6O4*#vsboY!d1fIT!bIv1@@ah(2sKVkMdWR z-YgOKQ@3>X29JlAvS=fmx7qEPRQ$eb0=-i}BT>WIbVBWppfw=w%F#Pv3n%5vE?ltjNEM(EL0Dn zcTAqEN?};R1Fd1_1xAr{C>kpS!iTPZhlIt#-Ya<15&k6_N@Kj(9}@rJGDN3lIb>6TfKiOR{PP}9|xUi}hb6dHJeW=Z#^bRWH2 zRIWs9y)k7lElsa_Gc3$oF+v+=R?^MNtEJ%^?l^lnY3NcDbH19uWEnRpEig4|Q?B=+ zGYq(I0v%l6j71h1;CT6J;+p#5?CTe=x1mmy2Goj&%{7jdDw%#(c5`KoWvJ*MGVk@} zE;~?*j2c5{q$ji68lg8ia^1+Lwm1Y>ytL-TiUw6%G8%}i=j%^&re3NpLywahz--L0 zfmeaCYfA;0d=n%#EBmM>l3u>>dAb53t!bUEaeTHz>zp$_n?xMm)LyP8FX8pGDA};` zhzN>?ex*NF>_sHg_a%TBoyZ!KEfZ&yldpfS8e*ya9dM3}Sub~@fb&LlgORT{ogp@lcRTUd5F zy2>XAV(gG6sBj`WI8CMcxlRb~$^+3^D;wabd%Z5px7J4TLI}@8XSxoy1^<_2xw2f+ z&1)C@#3*(+90LRg78RZxYr|h8 z5Z|ekOHKYJ1I9ZYi$ZR6nGuuOY>2dzVMv=`Cc?(`-Fb;bqj8lJ|F)bgD3w0B{w%}e z)v7HoHfH91bAFnIR;+ApV=l6yP*@hKV>c=^c}{U+n3bql-tk{AIqW2C*d#z(dYitx zp60qaL=wq@y#;3$3WOktXr2mN|0cT;w8ty`LdTGtcyAB?2+ai~qlFqd;%550U%?of zInV9Ct6oSVjE8Cd#*?&-YSaU(sKvsz=wnjigFhQTf;dA{pmjFYFIdSkQAM%ZM}<)& zSgfATxR-|)y>#RSEqXcFj8U*F1rE?EmVCn@mc=Qd0 z;J#HHA|Fi1>_9b#%Ow4TVy*-I0guQMCYh)8#g6MVllL3*J8@4g^S*hAw!T$yr?LuS zRFeAJ$3z_C*s!%_pJwx5Gp#TD68A6f-zhE1J~b@D6^#DXflxZw<3e*Us7Nd$UY=9(eEgH;LC@CDyEo)Tb=q zH8`k@U+<7AjKcSVXXRPx%jK=M>QmJ}c$I|;{G^aGYD~xG5|ceEAQZJZ*Q)2r#xBu@`Bz-4}JVcJe z?hE-s$>h_3t0@=9QG@u6_yS&Td)9~)%MT;`t|;8Ns}-Vm38oWpt>Dpyd^Fw;2WQz6 z&lES+T294iN~qtS?rQfb&eV5sf&iqj7^aH^N&~G=qnm@qKz{fuQ}-Cr(IFw|H($*w zbcX4|aH>g4(qxSn{rawhA~|ebi-97>{SEz6MKKYMlJ)z7Uj5?Mlp|IdNX4?oXKeH$W(R9>A#ksYQzCUN=gBP%{ zBcJgzhaHV&-0#a~jM!*$aC3i$ye6hArmYhT+agnv-(s?bD7GGxD`3HOr0f&U*pVhV ze|zK0Vnum$jk-SY{fNmhcXLRU(3v*lo;(5Ped1>oQ7Y|2^y@{of9IsD23Fk54Q4Mr z;Ja8f{C**iX$3=pGN?30{ArNrv&y+62IQ|fa742wG1)7r(9Xr!-y#V>x~ zL3`mVMeB@IiEF4Tyx)}Ba($WVDC*{9H*lb@C|5F-4@5EJ1q(*=;ztP9Q~H)Er`x^xrs?Uw0{f#kr7yqE?o2GeI^ zwY#T1f$tM{LlHpmvZ{qV&IS|RbggB+-m_rl`~G&NNNSPq$jtBs2zml*-L49)Nyy*> zHi&us$dd_EQiTGm7*JPs8*lFI+zqAMW$;!F?}-aXI!%h))O86miH?z%qG#&-4~wPB zQ!6v6#HZ^PSYOgb_}TzWkxQr^kgbMNS!N+B{+LcjHFoW=fUZpf^0}&`LmcLfJiZ1| zBQ5{Ff^#+~Nw0}{-%h~$@A6H%cIkI=Y61c$3QK)XkG0X{dFv}z9K3JHpQT^Lz~9&1 z(W@=5*6S?xT(Vuxd~{l36=!XErjoRxi*-;(91JI8~ZQ_#Urh{J<5@M!yITB9> zj&E$$a-%W6OlATR;|m>iKKv?nkFgUT<PN?$8H)A>NQ2y=ymXyw*~7ZnkU%+a0?&5Ocr+nvNT_3$uLD_a1VNkBUNmD7!tzG$ zcFCZfzS^gD$6`0c)Yn6MJ*Ibo9`bVd@X-s_5~Bf*=7J^JN#0z<;W*VU-0$(J3687N z_rHlB5e?bsql><0AY-SgDOUqiYnX63E=*+a(ro!#bbtKpTJE?(&}}iDymYC*BlBLO zTopF-cwp}`(-B8z&CmkZHqJ)TkIusOJnr6^hH%A^d5_e%pBMYl3Ako=)KsBA4_ngP8qZs49#jkNG>2UOnq)*Vq|h9mO>7TueL22DAseaN}rH0VzzNyW#um!+XpO}G;r(=l>Cx!r$J`g=FI zAW5751T2~O)R*4tsdWYWrHXkFgXZ5I>8mZ&NUpjRTe?+ad-P_Xr%J;mKf1L|1OEEf zSVZ!BZ$YA}<^RC)<87u>tO?y1n-vt6vzClPF`6 z4SX8j{1|Q@@6+z<*Sf!}b18r9mj~Cwe-qymYe89QYZWzS+SKSdM~!OaW5m#(^zgw+ zAH97vMPIr`&bxo#0pbs`?J_$ug3h9PZ;D@Y>6ZzzN0Z|TGhO#Bf4lZIs%|W^p8(3hww?T2~xfaB>xQhEvI@Kb&SAzc2EX* zd!Ei~(JcPEJDnx72p$4_+t{4lt(+U zln|MQTLb#gl-hQ?$wRe+*gy=Wn~Tn{d;Ayfi)6?2 zw0qx;esI$`9N?Z+_3y34aq?bBYe6X<5TxX{oLrTc>jCk>IsNieM6_~R<-Ex*Gy zWfKbnHV>8rCbI(NdT!U@ycvE2CuX{5I>gNA@#zgY>-RXdKLyuUOhG25ieO>-Am7|4 zf5G)ubM#mp;>CZSA*p;t<7Ca1zB2p<7)xaTSqF{DR_?b@{;WrW*+7(-wzA^h?E^H2p zKa*CR6i9N7xLM*=)UU(kR7IpY1fk!%fP*jGw-8p$;P&jDw)*7ET0`+(>MZGJ4q7EN zza)$?BAT+L3_W7p>%tk&pCfzbSMoT0dA>czevQ;`Dux%r4z``#6yBRciQaXBv zFcQTz{lUWR95prB%T7TyhBAiklp&!m!38)}wXLdyRCu$%csWj{)|OxDeX|lvf_`9A z4JKi~WW2&&Wk*Nt7CQ0C85$j#d42DlA&?U1-|S;e;=S-k2Mjit&k^T6Huum(fb6)} z!{l9;s}=5gME_=4ISlz#$R=|{B%aNQJ_)(IZcPh5c}V;Dg`6&4)in-5K{Y;VXmTwF zOts46uZVU-)E0o;f=XK%B43Nc{_gO{m(*Mn7kR|x_vH;b-#VRH-&5W@ zo1M+TN?~F#MJGY;wtQe$$xrK}#*Y4zZ5;|luK;r@t3yt+q0!L>Gs|zTPAQ?$P}B4x zxP^Lc5kapqsyM|@_45p`LiIfSNuD8E^sNKx)Y0c_2)9BsbLij33)m2VL!8SGVWl3u zS^vR#o==ynyONEkYx}sJAq$TxMg>tR+l(=IdWY6e^FNf)55Bkop5eOqE*iJeBeX!( zr)%0HOfX;jqdJsqpsfrE{C54jnHj$qGRMavrmoD!NWhmJdakb1R-Q#9n(sVAHIpGDA2nK;JR%`U+ARC2X}5%rxsEHwjT`_Fu|bR)}@Oz3&HKwj=AV|~rQ2x{M? zI&27SXF?wr%|u8_#59a;V}%q3#d`RQ@oEi7Z_o6%mMM^o9^o(zJgm=R-i+=^Is5Zi z)O#T74Riwf1y#Q&8Uu5wC5XhlO6()QOh%fM{Zm{6>M84aZ`yf zbWHu_x~HS9@6?qy*|G)=H1cmf z$(ESBGue~KefL0*ro!op#eG?7JA7{dTB{i^IEhhoLtsu=CU4a_xKzUMBugw- zcB3`C-YLZ*r)^pj_jZiyEYsh2!?EW#l<+w86c;D6<#p9=>HLG(Qur()%MpkSp$noK zcJ;P3pS1fc(UpZhaj0)bo!(2m(3YPM*=senu50Aj_9&LKsoXs8l3mYF+7?msXp7X6 zt)S`3%V+VcUf`)BSJwFHouaZG#8*BGU%mruV-^sFRi{VJM2XYcko#ox<4j7ojQc(V zKQm0wb&0Uk(A;${)S4$P4J#%KaqFttR<8T$qcJ+D6z>XLJcB|Qk*)!a8rOl*C29Qu;qOV9 zAv@UV`vc+!?W+?Q`L?B>z+b$#D1PU)A-Sp?xBGn%j9FJhF8y|Q-09Auqlrq#5CPX) zA;A5-j3QNl+%;uBL-c51VSa55lSIYPitVTgBN>W9QRC2@W9&ChaX)h z`0fHFtJCOB^rRY$Q~W)s32TLb@vfPR1sA@e`-=|S@bJRz5n}t3XT9zzoy*y zd{c6J(xcbt^}@?N^HJk+H~hfxcnKb zH@Z3Sg7VddBUz+mlZ$=ok;kp#85KT0o?9Yn9@ZpqCZvdI_SZQXt6_e>l+i;h ztlA%to-Qn8Pzf!%F%4N1pCPhUF;YJAZ8Nc2e^3%=1R4H*zkFBx5gZEFTFz&AeVmW) zakY~l?jjHdP30KSOL0zZVGK&9g@(T`I9P{PnL5>8ASX#~UL?3+7z)n2TSj_j$allw zf_-Fgn8u;2%wKP+uN_GG4yD>VO-vOK@F1$BH~sO)6V%*0ITFyhwbMHS z&X>*kThu5!h%dZ~n8eFpgOgGaEmi_7WhFK-gD%GowpiH!vs=OD?}C)AomNe+d4<09 zn@7fCYq6+3A)|qCq;Gj?O|5sz zJ3fs?l9Ek1cJ?~tG#;n%VKEW0r0VYyBaVY$t0nGYY0Ead0i~XPL8fpk<5%L1Ux^nY zB@b&{Rw^1N)4*+03Vhx*ZBXw1=HUy#&XL*gjCLn#ED;j_KI3CH_%sk%c;d4Dt(EU<<37k8O zEML%q`l!2alRLOH1T*ZmdnK!@C!%djo_m-ZLaG3jJWj2lG#v~5#9*G53T2XX*_Zi| zlTTefeiF%^BU$~Bm1MTrXWXh^-PAzW@!qs0VVx&NUpE}O8An)$t<9!DgzFQ7%OmOE zyUTaau9&XHk>g^K5e{)SR+);2@-8|JyB0kpM7IUov}aI{$0t>@+~$|L#9>T$F^@ap zC1LDsjX=w2b9@LU9Xy=z&n!D1vmj)*MjKsjDs=*vZ3rgDDh^49XjvAklzgWwHYYa| znU@yt`G#SqPVP618vLFLlUVyZ>&;&OVCtZT7D}IzH^j9`ehBf?#a_awjSn?vrhD&q z$FkIobM^jx9i&$iskFG^hG9C2K#nK(2sC=^+22BsrW6F^_by*iON1oUA`~&Ddl$T# z%2`YK%!g*J@gaTsOcr%>Bu(xJpM4j29V9ighI$@fCTiB>ti7%6y6~9jufF4($-y34 zc*nHdG3Er}W^&^DaHq8Bs~cCuP)jxDcxA!(p`%-lBoC@4m(LV;Uw|)@1la{-tlH4f zLDV!x=}<`+x^W_n+UM)0J2Eazf=Op2AQ}}l=hKkQaZJzOT{vSZxcV1okfV`Hw1llY zc%)yRz1somzKIiDI&73QQ6u{@r28Q*k9`T^kVh0wSdix;zpb^plB-Ngtx}s?O6Ib$ zVcxT#nK)_;t1?Mz8T@Scrl5b805M5NgcPnDTnB9kOEF8k|IG)*B(gU+2%dc$!p`CC`NiIr4Vv%)?jP6>LMDzMw{ z68RakyE)OY4Xtr@s>>Y}a2>h-V4JhEbyp51Si1Ds-n`U+q6-2EFH5S@ZCD^)`bgru z4@h=YK@@e6MQ8WzZWmWpa#9?7&dLPdM72)y+UwJ|&IV4-hGt;Uv2kwPZOXcz2ks5HTX{tW0Jhr9 zy>7!=sKHWX`3--L&AdEGZ;Njtg`A`B%_p*6)_U5)nilJb;z5Ba7+AoCiw?omu zVQI^aiTI_o^@ZatpTMwXA~J6pGlgw#$DOh)C&_M7O!HK@5`fq8jL}im`z%*2W1mRH z9xHFka+r#BQje!haY@4xe}l>hKU%A$<;sgaI<^yA@5?-#EKKK|fs)%oe%D@#`<2=4 zNcD5kV%gVg3AD~153IS9QEAyJJ4HK2RN$=Q!aGADoj_5u{+UBNn8>@5MOVGh-0Qe8 z<7p=?W(=ixz_AVp8YLxILj08jtDrb4>K)<*-MK2Xoo>WfW`68w#c1T+-Z-k$_d{#J z@MUi4nepct<>J=-t>9sEZYzcD)z*-*9$%tSI~>$$v0RA%ELgL}moh%vg5Z)$O5MtL zPInHaC&vcyHe2KT@_wF9wc8#*cIJQ}fs#_M%Nc8}F-ORL@B6bE*G+)@HM(jKT&~2G zBcP8u5R0mqOB4Rw6{C~HO7%DYNj_^8Z{cfU^u{(~d1J~+AI{IpuQkH-8NCvE`ux|I zQdFpnQ=g$05Wj0zK5Yfr6lGU_n6)G z*^b#3AL+FCt(L8*)7gA_EMy=6D3)v&cJ$8zEkz{|^qbz0@{?BycNG$fUG+0DWJL)H zA%kngC(RuNN8xtqf?-%7)M&+><<);ogTr$|DH}bxYFf@Tr1{A*k4rK8#}5bMLQH;a zrgPE)%!8H7iGr6&u6L0vI2102b7PSppuswi*ou;So`dms)TwH zT0GV1ll_wHv^(6Z4WVY=wZQxPAT+k^nLC1rw;ege2%w)!&Q!_2lB*a6omBVlO{wCb z&yo9PxWv*+qptW?mrgNF-Qn@|1mQ|n$vfTH+=WxIHm~(&j<%f297SyS${l+b@31cm zawgmt%+xBhr)8_8BmThIAkt5FG)Z#Xb=FF@zOcs8p^*u$dlUVcaNy&V#3V*yPDl3$ z%khr`9TuuBf>`gKi$Rl!85k}v_bz>#i$&Yzan{pL&_nz6GGDXu+4grhvxnK5r4FXN z8x3Oj`I+qBnyIBfbq|fic(VY~o?NiS_JJKZIbR}I(ZD;XU2 z?ag*aj)#T!gsO}Im2YaEKE@y`w}LYkKwj5}JOa&h78`7o)KauaNOu!4rJ-`;ferzk zwRU>tt2~OIip}U-H;jqiGLE%(__N0~<=EgH;~M_kYBlbl^+M*?{>12~CS8@$I3g1Q zwkTzr!N1omjirPq_`eYl&x4(cDZer8f&_fbQDe#>0v&Z!m5(E zTTj_4aKI=A(9LksYX`$`TkloMV$Jv&{UBGE(5QVI>P+WJk?nd{d|)Pg7uDw%y$w_1 zi{@$P&QpLSf~P7wS|H8de+c`$nM)K?Vdbc^#y-`wAY?gBqNC7-1Fg$!M0oWOp`KZ6 zSi_iPQ=|8Mv8D;O^b;BpH>|4*Ajpcvu?3;-fNsd-?;gCp1eQVHH8x|~vY0H|NTPKX zZvD+Mv~TceQ43^mno8qK0pXz~=l(%$X9aA1H8u!qJGs&=zXT7HS9Tl&9Z(dNgE__Y5pi{P7=y6 z5)!7iS9FUM`e>r( zHJD_V3X}USh^xaVPv1V`%#NSAblCWCnD(x1(l=u;pPymnaP55Y+>dk+i{v56 z2RQA5P&W^MmJQmqZ995@Ybiq;JA;dO`{Ikzwti%S{Pw$f;POPz6K-O%Jw?xT1Y9AW zan)LD^fpN58j3TOpy96&(#?tibPuOdm5;|~iq6{Nv-+FnJMBXCX(#i?&^jc-a4pW5 zC9*T*TTykj%h$&Bx0k@lUW6|MUQ}Q@LwSRE@wzuuiI!q5k_2=Oiu5j8fa&g3TV1f`jlcShmNQl|2m z=9hmUQFnH6>kOnr_Tj*l`jAS<~@kI{>x z$6*r6;1IOmO6bQGiPbih6$#bpZXuReICQer`QC^gkhBd!5GA)1;Md)fkDY2CQLmK! zJCId3*>M*OXJEg9E#7!JB66a%H%XbT!SF&#bQ2jcTZC1_nevB!(eU|uGy_{MR$1a+ z0Kg<$YmV@rd^B8hXqmDF*o!_`oFDu^QS0n*IrPKM@%rUD5o2anbzp9zx2{L8lD(KQ z1=%MAwyO)KeoHOWjEqpG0=%I|2?na8htaJx_H{pFaW`I^YYr>|1M%q(a-jG7S&Gpe zf$vKeQ0o^3_>OhKMzl40a|ajFw3U&#<(c!f%>rNc91+>*HyIB`$RPZXM8ET}9u>+s z8iaba+(l7*jJnHbt_a*x2BgzvR3f{If>Qr-jbuVZ2A0>aPd|)iXgr?V`6b56LkCGI zDh~dg^i#o=PEuqwk=YOX!XWhr-b9o6KCBgx({k4_-eo{IkxV+vyxohT95=2d%j5O2 z8e954_RJ?LU_N#OUdF`wZZf`&_In~?qgg6qcl%eV0v5lhz9@28Pt#<)c@HpO<9>ud zW*h4L5?6n&NA9MXx&D@dd@I6A`YzC@h7eH!`8-Un5~MywTK~E$?m3V!)7QPII;R4y zuj}e8Hi*H=x73pv#cZsXf3A2rKhVmd=XYt*p~>G4R+ciya{c`^OXW~4)s6M5o?4=l zHtVjxg*QmCtsU3cgyNL-8F$~#B`&(=_>DAxhXFd{W0mnzCypAnyWlsT4poDapIR`! z*^yxt>ovEIJ|+-f!|6_A535r=z5&(-OEP8->zmN|fFCsr+H?A2XppPY1bke zR|i9*aHYz;oQk5vc4IDfdoWar^z71}R$eHet2nKnFz^0Y_dY(R@*K=EIr^4%T4Lfq zKTk3-a5?Nd=TGL2YpE!O7O~t>abJOTxstDaodc5Qh!g}sLeI7ihDquXBba_)k#9>=bcXt zc+0eFN3aSS|MreMru-zT^w%D=}-hYJbkAIRpoW+~<;UT0}^*)jce57A?I zVrrbs`dST*Y`U*pp4KSIBPLB5r3$ZR8D%w-gV|VsPAI2d;^4ThcKBoe`*y66&TXaS2NZrHR1VWPITl z{fUvIW|_)~apQWe4NUD?Ic$yZdL(wlIH-7L@%r{tD7Uy{&I|}iev;wa?Gxb(K%9(~ zm%6f-gscZ*ndtZ|JHzzB<~E$Yt2fPGUtv0lkWIjcm30%L=H6!GN6W&2`Ml;J%Fd%* zL+}Ddj8C8~(c!E?Lfxsysm&$S{H~^E`t%#Yh-AeJhxa7~?j5zV#oBNUnQ<1fbIoZc zC5`0UU<<8^ZWBQ95s;k#l*EXKT+a&?!c`=Gn=q~cDNV(esY6Vf@=yd>SDW)pUbSwB z&cua0B`=I6svtKRyf-QHl6wNEtb6LYn|7PTo!^0KpTu?-mBxb<(5K~f*)S&8XD|Z# zOHX?fxyqFj;}t1QK91Ljh)x8rlpU<{qqpW`yYa5tvTa^AkZe_8S|(Tl5(vC~rf}Ou zKWN+(2=9ntJlfc$){BG`P{G-a^^p%pv9X1H+aGN|Di`fN@?|`w zN40;=vX29^XdgF4x+e^oSs5oL*i(Idnu;K!*h)En%q>30A#F9F$+t$d1s#8J#K@i3 zo_OkqT+IxBeH)8jE+Gm+2p8UEJt?b0UQ-lfS8{!rPFrf;syP~-pP-wwT7%A#UapXz zV5;(4O%1&1i~dxRv1Y@~OLF8-`Mfj~)W6UPm_`T6Uv zu$Wb+Lfq-}HetNa@aB4HPkn~NXP{7^#UZGdJohv06k;yHL}M;*{IMABUfNupj!fT! zLw=xHzabofuV)FeNY6#60B+}3$$LD8X5C)>T31C zJGsGr(#TgyZr*@9tU1(rsucEuLyHr&6bv#O8_75Z-Xk_tCSk45FM`88ckJNsh1(2~ ziy27swHGqAZJS5ADI*6G_O;nt9Sro@OB_ODb`>tW$Q+cmI|sIET?-e)jn|79N04q@ zU19w`@u-%R>3SX&GbQUf2+MLU{JB4h)Rt~={Dsa}vzn~9mc~yyrP>rF?CSYE6p-d% z3K*OuJm~!yfNDE14IzjqfyvoM@IP%s2D0pBY-4MXv17C)9Aw5_jkOxBoEr?|aFP=R zC(xt!wbpBVo}kK(q>txxSS9I;=99>F&+7}wtm5A=t+(VhTGP(`p!Vqsa$Vk50v*>` zHgFvjC(+t@LB!IR3do^xAG|Tvd#nu&=Wfy7SsykORfNKlq4BlyC8U1D2Edpwg7}*} zLaXn-tO+o%pfv;#C62UK1#%p--M0lB7GcGM=aJb`dMrMN_dzy1v=7Cra0$Y5;mt{@!?uUNA~hq$UwHBdxe*i<7*zOIODxB4Lt zVUXU)L1^(~Y4&$3Z&pdLYnH%F@b$rbKk=veeKOIV1up2NU@xI~hfbwS$QI>mHMN#} z&{a0I9hNuqAYrdzR!&{eTAsgLl@Wh3v+zs35Fxvlfm- z{DiwMTa^8#M=_RYo6m8!arDM8@S5C-z&zW?(a2;{hvj@7ViP|gH#6lIL-p@JoA$GI^?9x4Bz?6x%1oE;`3&`Nj4=o;^h)!TZ}4T9^Ff2c;jEQa zMk8{3pkELr$Wlq)oVYqP&_4f4>2X8ig>80C7?-NByO19;c^H9dt;eX~@MQKmKbq=| zKIJ>U>oO&1P@}N}(T)I>V$_CA9tTb|z3w_4%9n_LD5g|p zdtSFI3x@zbDttX4)Zz4e6|oR0`_gc(uHQm;OEfB=%DLGJlLg)o7e%EmOt@Fhdgl!@ zG>}WQ_raf^>5o8au^*e za=#1F!afRMaQ^SJxj9Cn#n)wasswU$n}-V%kw;=ATqtwdPgig_wKW9nkU2LU2 zI1D=q5nZdgL(Lnje^bF6{yY^gw~2i;7{BeHT*^zYU0FM5sI4LcI9mmp_P&ph^ zwBW`kL=td@AvnFy@%;X zZzj|vr}z3*#`0@(4z+WUZJS|CCHzTq`2i53<;04SWl=V#Y@2yd5Oryw` zcJ4|Cr|}B$-gfp?M`%N-{Q9#%Z)WxtjHh8Ir&LyGfHZJ{gC92iLVmdvd8@9qDnT@e zXcARPHhBH|$ms1A*~rFn+HX-m@s=O!V`v%PR_^s@3@|WK+S#J^R_Bdyn#J;olV8?8VAU_*vNnP=49G3|{Wy-`wPgrV>UM>?EtrFiW zmF>7tr6m32lRA#g@+?t60YX1KH>%Jj*(4iDp}vwJFNq9pkN4dJn}YgpDIpkf#0M{8 z!dqUpSCDg=fK%QSXPI}vS3QLW4o{f1&k; z4E5%_)=FJyC0no>z6ZT#kdDy6^otUF>u1id6QX)zP^V;#EVV#Z?FsIfQY@Di5v0RM z37-FRUEg?cu^d7Akt}>(TDlQ_Xl{e-da8C;+=lCnbn&H0*!**6Be-8@-1E|Zpe}~0 z`&!bH+~C|OvNtF>x?`VLbj|8XjoTqjPI1Ga`;PHG&J@pzLjj8~6e$r#TC#GD7Rie_BXRx^2#KP;qXhE>H6jUS(Jgn2xO#50KY^90F zY8(q}u{QHEC?;N6iplV48oa!-Td|@+tP<|orbxctE0jF+{Eg)LUAIceKy(NYF|?B5 z1w-nJ8-E*g?dhtMQYDv>eN+xLq{BOf$zDgqR`Iga^~79~q^Q++O_6!VtGzn! z#5vbsu(v~MZCjv`0d=N zj>ueFRMs4V;O2WancPSifr-D!a!aN=Hh%ZLHNh=eoi5Ax?YX|YOi3A$+{fA~e=L0% z9@jnGn3&i#KSd!_9D2-P9cS1;4$3UPmy-z3^7yAAOM~;TEFeQIU*Prr4bQR8*|-H? z*_ZLktAYYMdbHX|(}@C9G-2>5c2$exR6(>nNwB=;JmzHa9QSA`!!cV8H75F8e8wn` z&0r+Xukn%3t@HqKaa~vv+h%58b?hj3GBm2m&IB(cEuz|aucQrYzs%fGZA!SIbM~_3 z5v*@pn3U1dmJEQiFcW8OKfBM1nfYFn48aeEgwKBFhqKP~fQu_!!_NX?3FK6nEx_&{ zDt;=(TX5iSgx(=!3=hOubno(YozxSzEQyYCLG$_4e@Mj^)9}r~frEiLUf)!*w&DCrub(Vpdsry2mbdC?8Bj65cS+$3c?z7ER(;s8ZJy1&;(KOPMX>1E#xUZFURIQO#$ zcG(p6^>f^SY(q|3YBn#AWSrcLVc}@y+_NE4E-*>nhbO&=UBAf!UTZaP`6Ze}XRgCv zj=>6n_>~A(NSkWWCdn~>!e^g%kzseiTqjF^fF^E^grbuRfm_NmN;q5Gqsh0^;&)vT zh&V{RIfHHiiWF8G_tG;CmI(_)Ij3R9@5()xxNT}s5#sr3>|?MHX?v%CY3%C0AG9qo zDzq@8P^;`AQ3s#%?o}gJgXCj9IoX}1(bABLlG}YCi8#h%09)I zyISd<3&b&l4Gv0We;a5k$yYkmbF1ff@ehx2QQj7n!)+c%e39Efm-rgmWigcuhl06~xw@SA8BNw&BhP{M8Jj6|si$g?A z({3v5ur++lvBu4fj9JNuiv8kaGPd*0LunA7^>(r2tlaMcn^9Y|sARBJGi;LE3nCs5 zl;Uj#Ve*c}*5Wc?jUggzc=Wd<7*O2paIYk2f=_6;si&Hx5mjK1E`Te=ZDTT zgzg*^R`gG0vA&Fg1+UBOzWZp9nF<%(vtbU|)jUgPfT-MmqJ-p)>@U&g@@$!{E{hgu zYe}KBnW0dt7Fwp?qDk-$@JqWQ4!y}@V^RZdh;1r^6UVLA+-JJS@GJj<@SnFAJ5Ljz z-ltFI5v%1hnvqmBPG@O)BupJx=ump)FP{WeSqp0}E97}9zzu?DIg&ni9)~3`@4(lMJu#2c6#eMUD13!sZ|`y<>Qx zKxx{1UieyYAGKfzCI50Gaz-L_E`Z$ML32j%i?DZVb>+Ulsli)1+s_IK?TV|Pp$k*e zwKEi@{}hw+Z#k{n;d^>e0y| z)A_kAy6SD)>_~5BmId=sDlKvFa%5Lnd;A_KskN51h%%hMiBkfF>!%VB>v>8rN#-CB z+RuNHmF}7Q;LaJRCF2|>G!Y5Y{&`qwjTSC7(xC>*J*8pC@XbcIA!sIXrRe>TMHf2m zTiXUsHJw%(`UnP(LhXZLneSJ*KC}nny&;~wQoHyg8VRXi_-Iw;3W{o;4AXUJrk7sI8km&K1zVK2WE6_*_snK!e1+4mKO#!R#E62sz4r-q&<_L|&8l=$6f*Lc zbd~4Wh~lNFY$oUT6VK#Xw*)zUPEEegJ6*28EhZ8P#CrgL+lCdmX=5t(!)|zq<9_0* z9GwRwlTDTQwCO1w5sOR~4JSRhnEmG!m$b%RnZa+uA=l4_K;0WaRoR+Gaj_kTuPlp-DHor2beNs(Vw#Tk9L* zWBps+lt}ByLkt3aeea}!=;aLe1mbrvVyiy6Hoe}U_@$*S0*mpKtX@U$3wUasIO6<8 z*(o9nT7G5E%qDR6Hz5-tT!&0Q`7ctF=fN1ca5I(V43Frz3btwJKIE$O!nV<`ChK|? zkNm_-HAwO)J=gAmRFbzPmM;DN>9t^PtZi9Y{+>_>QIci$@P8bgMV6#c6hya#%Mgde zU1qq$$7S_R^|Y%`3j7ywFNydGN9cW{Tz^E-urtdHR1>JJmvIni_LD=zm|MG~g1n_8 zFamv;w+>?rO<3Kulwu#Rt6?F2bwJu^&R5DAymzMm$?gIWmG}{I_NzlQ63Xlzg{JHg-n>fk1LdCKVC#oF-RVtY&$ld? znORV%OifS0q^TY|vG?ag#gO#;_S@SfTZ%T8fqQ0)s;T%`d+#&4ASZeqPt~Ws zq2F$s!^H_dns{@-Ywx|rtFaYAi;k6zwSM0-mKwu*Gu@n(Q?xbUf(Z=^>2MhY4|ri2 zP=;???H8NX3xau?vuLikoQkR+6UvKj6Nwooy}tceZJYr)jJ9wY);;`sAg#{D-pyH5 zVO9;c~B}d+f809?HqR;~> z^1d}XnbFu?(_oeE|V}JuE4- zTGvy!>6AO;5x*9@D9OX~;;CNxs>sEw^(!wby%|qSBSYCGNIW&UCRUc|{6z_2Xvl$hWAH8D5p08Bqk|h-fqRxV|E0g>yUdOomhdB2?A(Gj+G}fnjfz4bLn2R?SCN`A11_cmpR(i5 zX*^Mvh@UDe^x8pb1*GjkDC%o1^b2A9*?sp&JwBUxYb~UoYmtrP-kdC|aNpa+V1>QM zYBb;%ob&GK+?SZz`aU~Q5-1<%?F1hlF*H=LO6G=_L3}<%fJnPvnFg$xl_of&zoosI^&25HW7)*TU#JG>{!~3A#^bm8kx@}8}w$*Fpg6Ov? zpaZUuCO0$nA7BoqPI|#f!a|5^E*gZF*W1{=O1)j`(W+=@6#KWNWC`TfWNKOm?Au|^ zVhN&6mE8%?A$o0&hlAOEUj5dklha#fWM-VNi&kbUq^t$)JM8;+SDuGV$clMZvOv{hfRHi!thLgX0mLv2EDmKvNUN5{1!^0moE1h*-lwI zmY*+{uP^aECX!4al}b_^Kj9jZ=ngjc_yHGk^@uN&k&AG~9Ld-j>Zwa&4g&+^c5xbt zzvdXEnA`@~Vg$lFW*RX;nApsI#=T!Na6vIwyzrY7n7k=Y#B}R$qh8Up<&-j9l3Wlb zQLu(zl>ILjHdEmnqoaKdrK}(d%tdC<;WckJDwGta2qgWxC)xGw*{UCER^RnQ!Gxd; zg>?3ASdprx=ocOTMrax=*utkRT=NDf;9~KetM>_vLwPU`j-t_VYNeQnNB%LkXqG9% zWObPw1t(&;%mu&W9JQ&b!uWYe8&MPY%j+4)*uEW5{-cyPun!lCR(h@P+d|a?jsp^L zP~OC=Z#o_6Gt>saMZubgD!K#4D+1Cc?}BJFEn$MODqr46T_9oEa(vsBm#w;gJ|wn-)ox+_QR1*cR6 z9>a@4bHKakCPZ@lXpCPT?$a&O&ZbcvwLj8WqU*vGa!-HrE?c}B7RS?-d#<;*CmD?K zPy(9}YwTIE@4D71J6Vzp8r7*4nvt|l9e(f;SDVdQVhIB9M6gv!w#8d{4ig^4zV+no??9W)8dh*}to$;) zWz3GH^N?6bf~cu~pN0ohI;X46P!?9TJ`RSA4em+G4CpY`u_Jjp#p0OGIggM+5!qPY z2=A>j>j&Ra7=NRm9<|$$=Hgq4*is7^ycu!9Y~Wl*&W~IX2&tbF*ApYBEs-hzls+3u zX;c8aUPD;_dn{92kh$-S`ru1PEaWB;q1E+b3;X-nVf(?|Gr=4nL1qRC)8qF9sY>cP zR?t2ETnYE8Maar^6|f`yEE$^uKHY_d-Xdu%uR=wM!`uY5-U^9gGxhT}+_ZC#-Sw*3 zjb`C=4apQGoV_Sb-WA+aDdr$6ffqs!=Rjf6Ib20D+*831N$j!S#h0+8?Y5`gDK6YCH8yz zcFMrlH7OnDLthgLp?ums9GX$WCk8nCBNsk~ZPmLkgk)5@O53lM3PYt)o*hUtJPbcx6=G`RE@YGs%dhO0<)r?k$CgKF(9g#58k{!r zLYVsq>QdK?%?Ij&*hwAg95J+fh28~P%cxfLlv-Gy#y!QNEML?v4%O0J%0!!2ldZ|_Bmpth~OkHyMGjHpI$1i3I+WgFr(J$ z#<+-W%VVmbm?awg2pc+RSLoZG5@s7~&Q>T>-E|RvDZ)d1<1G6hc-Fq%0#xKF`iL&5 z!{BKr>*dBX=pjpHVV00BqZ1v$8&7s?|EhdumFw;nIK!$eL*H%OrBRNNi%GaP2b<%T zC#|=Iuh$Fg`q2cY^E zwoUhLDk!_|RxE4Z+tBbcX)A*rwb%kbSOjwNb9Bjh_MTaf@yb8pJkI0?0Vh^2ULf-1a;j{-|tzV1}_f`iyf^dyBlC%a{n&7rf6HHwmj+ z;NovjhaVBD?yvqRp*Ssxi|tLoexkB5zbFWspF2eFEwOtY$W5(yLF6J3jQ5_LJwEj@>lQ=vGOpWA?q-XqM^6<5-=I}+H~{I4`4J|F|`2Bh}0MfuH^j!OB4_hO8Bv)rtw0RP4_kQI3?!JD-)2u zXpo4`oBiIfb=-YUz5Zas9hwjeU}?@(u;#J@V(Yf}%34*1K^)8?jFcH0Nqp?)u1G5y z?Gw!W?_9Q7U$_zRj^uugqx5&YukIzVsJ#;5na)z%lG1W-oJURTYMIR5>pxLt;QuQ`QV+0VBi3Z!|q$=v+3W+c1A-L^2hN0`BeRAx@<&^1b zz^cgX+j2}il=g6ZAYEwj9kpD7T2UxBzz*$JMX!e-yE5x{kXNa!<4c8LeAG)D2NAKr z2Z6lzFb_dqEcJ+4kUcnpAS>hV-XOZhZUr;@Xglgt1Sv1y@i?~~3GU{>{ zdaCoI(q5ifB`kQ5f|Q$VyK$eC(3G~`Chx~#+Wu1e!OSO0q?&gwOzWX9RYNZPYdR$C zJ{K|TNzY~!KkaW=*oe1&)7A$}zpnx+j?`4CeVND>;iy+%vsE64sI$8}pyOzx?r(Ey zX{7=!44ZlLj6gttBv5k&a7ap9K*Qm;zYh1cK&eP7r%twK5;eOFP8r@-Jy5x#-Vv_N z^Ltbnh|^c0SvajZQ&33lS#q>aQ4rJg0b|gv+ZossY=^514RZ*Xn{jt2#kUDQUz4}6i(ZcIRKNeqiybd8>D4CVY z5f83c+9}B!VKqYk8Cz)cqjful)+UHS-ud6fIyEdxdZ|zgltRJgE1l6nYs(Ek+^4Vs z?F~PcNCRG7>Ct@r8uQS1V}5rMXZHBW)MWO8dY`hQ?p!ArwM?Sfhk2yMwBF%v&P7k5 z_B$Ct{6d-124$+l)ep0@k&Piee!L{MV8#+pzhj^sKMBL*Ch3#fJz9+4)c#dde!)LB zGZXiKm5-;fN0cJR9j}4wmp5qm<(_^5Gwq552?=~O#9>R=;bVLAPOw!Vx5W$xZ3>wZ&Fde}FWvC24Piw&ThO5n+F^>WxtmRQTowa9}GvERLBU>CqW{ z=16WJbhtY;GD{7ALOK4eLlbuFX6E6?2@>m`tBIeD+{ZAQ-m7^#0g$WfFm0}3e8i2R zUoCE5Mf4_91)%r|^ZoSW{zSvsUx9Q2JU!?~Bb9=v6?R@yD!~i*xmdE-SHS*eZP!+J zaqdm81$Y18~|8v#VuzHxp-$ZD|a`*XQHIShNImBvjuL!SAWE9%ZsH*&(Q<-UD zN*#Bz!YbtXt9APWY4j*tAAGBgQK%v^a!1u;-?s_2Ep@`6$z7d_RZTY%>d))X-b zHgo+}D5H9Lu;v3^a@Xwzu=OL-5>~C3Gwh0;>(r~LeM9a-|F%q}*L>UTiAD{1=S(RCiCSZ$~(Zsg3R~IPh#+?k0@j;Fl!R}>i@n;g5 z&)tmzm+V~7mV+6&Rxsum9dL@m@g!!J(V5b1J$IE*9+SANv~ciTTvTP2l3q8j5r)O` zV1g7{A})hPn5Iyf{ZVejCKWiljd2`4ekfZyXE+MS-_h`G(4LnRY*Dk5lJnzq{MPN% z{04UwO7z5sd=74y*ial%8MII1T~P;!{^GfUa-wPAjZ?j6yW~hm7>!q6kbO+U@Cde^-VzSpTUKsYwmdF821s7*trF$x4+oZDD=LKM6Nf zzo?qF6S(p9m;tRrNzCL@eChr}^h-TTUCSn%r!PwfC?=BgK zmHiM|gA<2)#+N%OLp;+qzev|SbgP(zmPPpKkJC7+la9ywrU!?0iX5;wpL?$xl=54> z>x7m2hsjiJ@|!0g9m;(RVk=GcD6_94AWv@gyWM0gXi+ z1?Pe0E4+5~LbEvBxn0n9bU)oM>_Peo?Ene@O8g~b*)WalrYUnBeGQih!eP&zFRuPA zeK4P+b+HMAK+W+@YEyfY5qtLSPRcRw74^ttOq?E4b_}tfP}_AL(BQI~DOB`nely~c zK?t<2hLZlO2oozN_BVvtjDfr6ON_FHVUNT3*gm+hs4P9waD)X8Fy5;hbglf}5PFQv z56sGPGmX3{^0FcXpFc~(GHulO!??{ro^^Nn2fkmVr;ssFWbOqEu^(04=gN8g#F_3& z%M4y2L(BAzXc78KguFnal_aXkzpbk$q{dpr?*x=p4rzK*{k41Rm?%H5P0JdO2@t4b z%e~GfJ1YAJBW^HH^1yC}xNN_XL-;xPaHu`JU;NHS0iRiM6p9Ew$R>HW$o<{s3CU6RgXc3wb1VVpQGN!&}eH8!F}@uzq$7{1Ue;Wm&gRByo)^G z*sy*Y_Icr?N;zBmKP^A~IDevg(6{9Y^9L za|~{2&7Q@JJ?TZ;2dGX^Z7_s~A2ZRmRP1Wj4+x~|sS)-7HG`FyP+-64p-+p!ZG-xE zj5GUj?Pe6HqQg77#4u86I(X>e7gqN5nuysBDp{DylLYlA=S8M9LmIf9!wylIkkC1G zo#G$Utc+_I2YMd41IUeUTai3YEl4 zm_@U)lU+f7=t=6f?xM9DVK()5mCabuxXot0ZEv)~4@Dr#_Qw6oRB z!D7haahQ`O84eRT9Vlc&zhY-Jvb+abZzAhecQ zw5GF0c7n!&?B%UwAzAL(=D|Az5o4Mc``(w8rt`pY@^|LwLn=rA!Loo+s=5@rY?2Cc z_n+co(@zl&*w{F7inSysJ{*+oH|WeOstw)#-BPqSZv_)}Cbai7FoT(lB9Z`SVt55^ zQgNb3{W|Q+`obZS)y5$uz4x6G(w9heuBu>`x7ob<+nfcy#S%uaFl2b0=3`qh3dx-q zD|+?d?3Zvi5>Y><2`iXOF~&60r{H9gL*A~r#}~Xzw|6GUp2I4KLE%bFc$A7t^Vhz$ zlP#r7XQ8wgi=J1{0*mW=C9cbPa3s4SbNA4vUGr9{hnFWTd~dTzp~njbEfPz%LpK{z z7J|jozf0nDt=gUN+d@u+Z=piwmlU=eL01!wN(wRh4ouby21$Xu8#VH2X=QxsLWhCg zUwMsG(-So5)nE-A>VE&@X|74B+@<+o?84r8+cb;R*7;_2!lWOUCQY@?Mq^5TkWjx3 zab;OozKLlwdr#F>r68AUd0$TPRl_smJm&myv-gzCctjDt&V9y^OtP?UvznHl%^Sau znT55dvIg#y!O-fpV7IFVZ3$f!>}dt4Aj}RwG`?#vZfOZMO72M}Wmed6ntz9fJhZm4 zo$k%**GK+ugEWIF5Y$N=@v=;ZjYbs8G&B9VCg+jE+l*eWVc*0ynZ5aO=78<8c+@iG zwHB+=*>l9=_E<0#JQVvGpNE*YV5g~}N|anpGFbY!WQD5U`B+U7$<;)Hf_!g&5*1xQ zoC^C%UdZ#Lnw!qMUw_IEhs+oWev#j+aiol)G*?^Bns9%9-VydTChnR(i(0XYOsp*R z_$K*zn?>5KH;$G<5~oM%E@`=ZY|hkT+zk=U>lkM%v*$Jjfj3#kye&?~EAy?2ViCym z_=Ux|XxOwvJhvhP8yVRa{ak8a-Zg}{-|zz*!|}YHt(3eqe3A=SSeYNjR@=(@pJ<7w zkEbqwy{Rf()7WAI*|&DVS>J-LIa$br&QW@K>q|yoq~sVvInz_x$E8bzSR<(s*Vc$p z2^G#*uI@CZpk}i`v~^z;{2?oCde{w3NlUtiyfWC{ye%S6BS$NNR3ML+H8K&NI~0tp z^=;;^LqZ5LMIB^Ib+0S^1`m3uTBl&sBZU@#&UMubCjn}Ro!l#RXz2`=R11;5nTnvT zsi1SyWZF5pLq;>mBR>HX2kh(H6^MrZ<-xgdM61#*Byu^nq$CZmQ*gW1i{uGi%;>1- z>>*bJRRt8!V81bnDjFg(ufMFF1u^8PAy`9W*D))NXh^~wnvRtp9qh*^1&|46PT^>b zi6xgLenNRSYBN+$?);cHda8k`)MQoJebu_2G?8Ho)#s#_in(w^yE3#so_r(lyUq$- zzZll{xgCyHH-KHE&)!&5NJ)d{+3Q3)y_`OX3Pf6$sc9S?qC5`MD_=yQK|v4%`?^vb zw6KO|5{-q9qmor86#X~4&*7f~e#U;Kr zwwClpD8_{3>Uw)PM8UfE4_BrEdWyU(Ah~ZBx<670lpdbJk3pqTfKAz1VMODo@gvL4 z!9P#$CvnHmuwox>=!A06`|4*NL*SA}&pxpeuE{`owoy2@=XOkcy%1wo<>a(ULlwd= zc_mYG82XEuh(1Ymx%Yyr=hyk^_^t%4M%$_&KruV>KVTB+>Iml>;|{iu!D&%S$~h+H zwGiH5hmfZJc@LRf|J^{GfytDDFWRW$p9}NYPDwVAeSg<{TDtcP&&^rlcA_>S&LyXh zW6ApD1;*RLxPL_JerFK9fc=&G`p|l%8bQZ$zJOr@njXz=a9{s13fDl=DeOIfH@Ls? zD5joZ;5QO+?uo%*`sgxLU0w#2p_i^7Qp;D6P>rF3(^w$#-5>Wb(?p3Z1$wEJ9jDrn z$G3QFeSUs7f<8T`o{#c!i#>5$WCE5NspYou06po@;=QfJO~Z5a%bmx{NOqedwqQyz zZw2l%9e~h&CvQ*I9fp$gIpB}od6}A_Y?9i_ss_^If*aQ#B{sy`H>oQ^F7f3tZ9L*9 zk(U|gX`G`>4wzGHte042g2yy=LJ9E9sUPT_lr?zK5d|nix%-{Jv7W^Bq(G*mJ1t$-ZMDt0aZiDCHnL`EBXA;)vs?S=O5VRw=b#x}59D;jQGv)F70UgXd5IA4@~D)glGM}X^q*B!=i<>z0POW>|>uhZr&HQTyi zRV=eoT0VOFHt}xhb;ph_#bqo@KEs^V_;|4tEwZ@jS8fIGyme3LPiNk4<*{AzN{cyS z;&+##0}>IFp5`G79IEoCg%5t&ntJF_;X*zxrYff(Legu|#!$rj4&m{>xNb*)>*CqT z=cjwsuecI_T7mLT<^-oMUF8Oeu7k(&HJ!@>RPVEtbDr=1U|bVT$tKUMtzI98k%_mx zjJU>LLXjYhD8Q?{qTsfC*4`fDt4I7U&9Zk7^&nC{g+(VM_7@$?M}dKm)weE=Dd^FB z_R$dAa(m468_b`=+!Lyd0Wf0A|Fp$o#{9PYNXmO0M$r@YY}V1YRzmO>f@?0(+JRSS z+j^5{l4(^p*1t>USDB2G+`k$C2uwS{^h-4Rq3*L_bQ#x@k7lhUL&UqIhay*;ahlR*v5H~CN2`GM*KDi>)m$ROna>M;sxzY0?zF% z@9R2o=`kN}F*4K*LY;3;!oRoWwXyPBNCHBM1GEC+^RRbhIf41R9bq7bL5A382sqlo zPKK2HEyd2lNKPCG78093;%cTJb|^7EB%%D|r@`{{uh>jiN)au!`dpP{TMGwR?x~m2 zWUGwb-}<%sb=Eht^ROBc!osVq#Z;-O55qXQfso8fFy;XMP-d@n=27qOI}ga6(f2F@ z+m&d~MxaM_4t`5Jt7@FR7kP2w{NiAfvrwbJTPx|4f7rN_Rr&@_&nRn?QG#O{pzeF^ zF^ZWR_v_^8#_6+z{}O1X@R@i&Eb-<7ONCA!#vZ}tJ}{SU zrouzb%bW|v1wW=!j;>(Zvc7d$dmI;By+khQ=^zaK^FCtnH7t`2PcL^Qq|17ibA({o z{wI1vP*pNqN;Te`U0ym9_)=LslW*JC8wbvF&ZCH*IY+d>QaynR$1ZxvF%?iUgg7Pbl(N_2TqWGFwTaunt`kM0A#znEl8r~o zoyYEVkJYP1Gv3Wn?b8<4hOIqXlIZZ}-bVM0TjUVd)D{}8`_0`f9$u#;BNMJ`PS73~ zeB2Dhlj2Q`Ni@;x%A);Lpo-iTLIM)@W!Avb7*z5>mws!K-49w+@;M5OQGev9@MP@A z99inAYSp6eueNNKDDEL71BPHNcoTe(ExDi&KhoRTUv;tjMB&U+cthkC4@@f~E{WCt zFrzS$LO7w@z>~|jR-!RMO%TH&LA++pLTD~_18sYoCNg8ftS4t$g>fm{1@9kb7xezp zr_yWmL{br%orDEQPDB{-0FMk!@)GFf^73$O?H8kIeA#U%Pjru-D8p#_3`uq6!{2>(LSK9Ria%wQUjdC8uI( z(_?QJ`mcsKIV7r7;mn9 zW77~4oh2t$J#2}Kt;9~Ek}i?FVM)kt9+>*Wn4vSe!m15{g&5ebm>(`vwx(8)l+o7_ zTe(74v`WsgrLJqP6~f9Oll0P{r~-ncPQGT*7uR3`v^cDHmC+2-`e1n(oCA(dvktVR z(*^Rd&aBjb#6b%h(R;pUR-3s{UaO>pG8c3?mXGCFH~c5W6tD3Vi@ASyX6%mNb-gk$ zg8Cx#;o(iG;;xMjWXHn&uh*4{Q)EzXFsN>+e_iYc@yVoHTaz41yj=L#Tj-qDD@8f$ z3AZCr$em+|7`D9?RY{0|;K_PY6~N-)YZ4rfJz{fewx{GgIqH$*H#|ff&5V=mCLud0 zV(-`IB7Zo}D*{k*e{)ZCsynVf15oZ1dc*U1L8Xc+u9lkhjyh zdB(-NqTs$I@C%l`0(|$2%w97!J0Lz_+TAJ(=9}oNR_MJ7A?lD`TnhEX_-yPMV~Wz_ zpHZ@Oh`p3XDzT)MP)*P#)Lw(yy}LzD3vW^fyF~;lA~?sH4J+RtL@d~o4c8p9;Iz|eVoMsITDHo}pL!s6hp^xP) zxAxurLdjc$>?vGCcwPN$5|vu8PCu(N`+NEriE?L^r@p!}1ym@lPJd+D>lZLNvlaU` z-q#^u(ep7_KMZFnvAipuudyq2t}|6mQQn;{vQQz91u3;&dbEY=R!@;p6u(+jk{a1E z9B-b3XX5w8?>e|yIX+GP&Pgem%sb@8#wP67s9d;k-MfWZkgE3T~Z%SyKI_OksE{Gj(Q4baSUC~FP*FJRc zsGA{i9GYD|60zUBF7_2qlk=Z)L9_T%1;Y27Q()scp?AyirtY4c!-^oXT({$*Znm8g zSHhF$x4)WdE%x=}{50CHT9ibUV+DmX3t{6>E2In*wPo5&rtml6h1%D_Z>oIc+0$S)(6N!@r6qMxPKT=p=j2 zbt*DUWhJ{nSFh}cxN-5O$2&OI80O-k(hh;L)Wa_ATl{q=?@`(Pw92HL8%m}h2=G29 zX;_2#Fl$sP$VYw8o|Lx@#AyiDUWp_?j*&DVGgo$Mbyv#Q(XHI}NwU?P8sXk{ew;2; zYHVogE2jZ2mAN*DWJ~p?C7r5N$JTahtl+gMHyg|+qIQK}$wP7xuy14FI2CMfy_DBs zISP_dU$Qhd>SzJe_(r`%#(g~0C|Sj*i&?7v|B_J@mhnZo$cA=mdtD5=1Q$hQySz^xO%dI+}1_A zW6sZs^C=pe2y!){Wtxq~cv`u4Li{yd8;6LDXGwoA6R4X|`pFWsT3Unv2y=PL$=KzS zKj+Fw%Y!!z`VkN63_RmLorHMd>VLJ}@otCE%C=w_QAyC8`nM($BDIUB1v1O`sySf} zJM{*D42HD~oBbgUA#NF7w1o-${t?)H4Pbr1a^<>1dn$-FX@s=?iKZZC>&ub9#ecyh z{Bn5h4SC=^A~QaUv&>-Q(;1S^`!eMBeSCePf4I^*C6(A@>#hVvFf2#=cPfR1iKr__$yCF|Ah2Y8DBsMwM9});vi|iQ{b$(}uW>fZqyAvJJah z!QL*zg>0UTOJY=e|f`#}TN4 zPedWzQXKQp7`uGGePZG49YdkJwCE0U2~5xtj=G8Np%MkzSYCf=9tQuv_E-{%3srv% ztu$gPA9I7!F!Vuu8p`BNg6ZZ&1@1PY zTbdq)uHUOjoa?J(ZDH4}S3C`?up8^=Fp?K&PCSIA?U=egLyH>qmycvjA6!6(eKtra z?n#yBHfSLPQTj7lhP+>m$jA(9cqg?WAmko;M#!Xjvo3n>a9cl4$Qu-0 zGS{oi@ms){zO@4#m;DCos>SK4sCY!Nt@GNU2JvKxR{?g#ry*LLxQmMegsFSw?1V!_ z)BJ3X=1UXiAfK|69Gx~e>H$awpV{aZjnp-)nY2^uHTF|}A#%kdEnhi}mZD(LWPJO) zWbRAx6VB#oLr|pYI|vOaHMb(<9rH@&i<7tnNjl#>{yKe&-%7tW);CToaki-wXQkLA zewRiB)*+ol*V}5Qnh9f57d8vJ- zbw6{-F$sK#eTlxrvNdDy`o+MVxw(XLfwzY<(8tCfsKGMtvOle!4vA{e{mqD+4ECqo z*&%{kUq;Ium+=5ao6Hdntws!Yj8LU91x3qIjn3#FEfuI5MiQ^Zak0j2KR5X#w@se( zLet4*sZf^6>CtYxyHn460XqhR06v?Y<0=hOcLke`TK~5MJ)#-IR$OXaKPADN$)iHE zU_X-6V=3^jF8=okVtS3Q7Fq43n=o z6Vp%~1;d%0scRs7^lJI}xY92*$oz-*pn%iwTP8hAp`H4wxI1MiR~dUdJe3L;f)*?d z4MN*)_*G{E?#J5x)T!KF3K+ztPQ4dfg;H*HEaYB&U-X;5)8Yd+)!De}LZ3Eq)mUIb zqgxPVuF>+-vgWO9%R6r5cppXe;rBtmN!N(7vO-lrc1+3A8biu|-q zv_XPi%tqlKoUmm+v{CkReHt}(Fek*GCW z3QksJW*91`qG9{Z0p!Xr#g7GO+5V>>I;GgEra6c|xKv z0m}!NWp{$ny55Wy;l+=TPBmG@N{t!X+rD|AsVNm-nCMnnRw)@!ejm2b+uGgd zrM1P)g%M=Kqek%jjI#-6O7ApsGZvcvpsu`q{a5LzDGa*Xb@rZa_Gy3S8D9&73PSQu zlZEHW&S7tv?N7=@=XAs^kYd^%m|yt2()++hT*$u`-R+ttmOVA0tF0!vJXuoLs$7dெ-b2i{hm=R_c+4bdtryFrwl#=Q*%05e#4LQYaK&l^97eF4c!s)kIlpkXnR`l@5`I4)NGgF~owt z%Ze}`-K?1+f-IMS!Y-30eD-h3WMmPvu}*Sbzqt+BKU;uJ52CohngJ zX+qQhlylsDkj}BXObNU^8qwrTbkj7RHyTK2)@TIud59V}J_*XuwA6aT{ZiRXd$Y=U3%sn*8~&vG+G;n45C1ZBf<0*A6<|cA z;C)C?eX?0jKFv;+p;x(*xxmU_$^XSrY(c5novBxy{n{h5%O-Mep>9c`n*tqWF(uey zizuTEzZoipz=cGT*VN{GBn-D2d8x>cL)3yJ?vo|u(ao{drw?}GBU$`W-L>4RBhhq$ z1X10Pa>~U+yrf(wnfGJq&sosxsKuN5L?~#p=4X@kk0>c+F1yttS=7np4t_V{eD$~C z?O(Y~xuVFAMt|7&Ng{Nw(-QtXj?HOeNDg1Bdz8xc;&@UD#jV2N*)XvxR<}B)HpJ6i z=NR+rBD@7{+3Wz7uGqeX)6Dq`qboIUZ)sj`__e>$e8jlZyx*YzLi0y=xyeB$GTW>C zmF$yI21VW7TF_$J+U*5pHaP=A>l68_ARJri;gWy3RB5w`dv|l)poqS%JKHInQ-Ka5 z#ln;X4fT-D!kmY`(B2rg?6-xuQ*!1sdOkC8(ch>a7Pj@PFvA4X!_Kvk%k!}8U<+g6 zjF&n{l|HUG!4E5>w3M6BvGhoTiwhEf;fL+ zYrxSQe6Y^ziv<j^PB92ybZ+Yqt@eSY`8_03;56PpPc0%a_n zt+7M*Pp^kgfrK|2t$n=7tS-JB=6HM2!gIx`>3A4a|$Sv=p zxE0c@x&$qs;U%ADYeNA8QxCFR6S2D2p>y32O;1<$HX}P3$M1h8&4_*&?hn)qL@?2C z211`ZRn5V+W_|;u$At-XqVJ^NvyT+rMUb@^X6?VfwV`5tbl%v(Gof(2{Fkv;y2CKIa zl88g_Ea7=GA(s4|k?;UC^Jc@`qG-tXr#ludZ^$)FTxE>&@daQ)N>47u^~>pdBy+9P z2hvQ+y(-Z+^I+kV>9THVH4#C z2{y7bheOn7EOqgFp6VSG2odd^tpwWx%N*tTL|4Hr*2%oDEtOfV0-hfss(~hLudXCj zd}!%}GO+K4ml!U5K*7e}9b4D1Nt*D;oV^zgez9h`@V70n28U!cHU7Kn`&cY=-b`xl z;rO-)J4)$qsI>mYTM5wlUFp7F6Hdlfkxr2jpQ^~h-0BZ#Qb z9ZT@08c8QE?6jTrJq%=h^p~%#DB4x7U)$xzpV9f>>>ojtZzJWIY^>uZ@|Ic-^EFLZ z?#pLO3w_9~cb|5wYvn=X$XqhsvBBCO=@gX6HItJ0(^Vp-s8)B!?8%;Dc^zu$+I9Zj z#_usmk#bP0ETsztNb~$;6t&{N$AE_%JN>qq_TeQSMK0}H4>pD)D3r}De$X7JR}6=3 zm}Xp6yQ9dCr1x`h8UD06=k(AeE0ormXrAwHMjW^jrDjlRBP~VjXYcfd#(Wu;OLe}} z5m`cg!5XkXxD1OB)gW(uo;FEQ>iyx+kVjDL;UUeE?% zg{R|JEsWO!B~dcj&d|mN!>Wn`yAq@Vg(j+WHja2om$Wkl%|8g&_+^OPA-)Os)Z|hD zS04KX5DTkP3cr=z(7W8fpOswsDtj=gsb@@o4{|j>W3>~tAM|yVq%fFjBT;h(OQ!t@j6Kcv@rB(MUYX&pVg6V52l;3G-^uQ*74o?XiHP>B}lp zCcvG{QHtOJ=lM!o=g{OWf2}Dv(l(xL8z;fg0bD+aR6}u?*v~3Jn>zQn(_fB=2e@@k zWm#!oYq`VUb&l@$)h|m>nYEtE!zyD4dOk^osqj4?Z=S+^e>YLBY&vkzL5rSk=%XaP zjA87yb}2jmG&nSY;ZL(y)0&u{tin}afmcgi4C*F%x(y?`(fXYdlo7)Wuynpqp7&c; z@0N%1tBNy0D<;m^IhD2QHzaj@0uxTdd13}W(y|RP@~55qL6)7P6@~u9+Vl6pF_Ja| z>GYuhH9*S02rw^Rze2@m?SecNT`pDG!P|&MgszEX*e@USaVp!UWO8x;xRstm3y}KQ zxwjf5*iS|HRZM?f)sNNOz@KkA7n&HzYUPVw2ty!~VX>Oy-WY|tuhW1A{mepx<3d?W zGG2Dlv7OtG%tmsH5V}%@1IZrB>xmQcJ6bbVoSbKFE~Y2{^+(YU0+f&a5T|%862)gO zlQ(R`tNe{X;49F1b=nPApmXRx!_eJ8em+ymMvf5eb6(iAQ zRcqtQ{UmPnMPLj#tbJ{_eo#N4q`T`vM`RQF4U7VKp;YHJ=Fz073|6zaZpJP|JqYQl z(hymU$R?fGyMn8m-`38)48l!dq` z)PO^FyFbQ$>#&)UhiL}x`o_ax{8YH+llJ!sD!=iAA3%L#iz(S|#S~u``DJ$E*MPAJ z^n@=2%E7^6DzJ{jTk<0HVlK6eoUMCX`-&i-?J`eU&F~^<(sy0e5=!>8`&~&VbtcY{ zouujfBCFEHF6of^%+kJmJN3PvMZ3)yOQf*yDafvRa>D%8)L!C}!FY2P%`(nsFHzBv zG$@c9hMJivs$nq~>l(>^2?VxwIGe4)H>~1F3c2vPUB=Gx-(*o1L)g)YkMw--4l8sM z$LpfPN^nfdB&Ak_Q5~hdK#@Zrw!4L?dPRM|p`@H2#xvtOs9FU-;8M7qEO2;(ZRA}6 zMTtoazY90u21fYSyxEjgx2{nGyZ0v@;!1c(bghTd;8WXgm1b11v~sBcjp5~jsoUg3 zrN5+!c{!M*J8wBS05`sD(o^~}%~r($>w|R)ij;d=U!AA7e9NWL8Hwk|H#Vm!EmM4S zQRWCr?yEQtmbnQ0v9pd-7dp!5KqyeKdp;s#qRgZJtWK z<9DY-r)|8jEeMnW31TF9CeC9gY)~W?9?7|V2FCncF)stOa+oTM`#)U%UbgkQC?A(& zG0h6En2~XHf;61_oXsU6-;j@^d}rTY7gK7i0+t-ZnW3=xvhf&VQbf1fG@LM_e>vH{ z5Q~z$cFVawQj1u^Sa118Gt2udAi}S{5YI*uyS81%fTYPQGH8O#n$A>sAVXuwYZk6b z&JtXTfeC$ZI+Kv%EHiZbczm8A6k1K8l01_&u2yL8P5Cem*?v@yKph7UbnR#Gl@M53 zNxdUC=IPGnOFGSY$?bv#UGOAgVb$04jIz3fb%=(3oX&MvW8+ih%$)q3Fx|hrU899m zx)vmth#iU0gHMvzz{o~=*w1q$Mgo7=f8^|_G3Nub>VPJ`mowN4-8z^&D+-`dvOQ;P zn55&`zf1C-(BJlF$(p3E@IAr+O4rZD<4g>ezNy%x?b3XarZ~^5;p4R18SQr>dyd2; zgiBqrmoAEQOSL(yyc5=__F-G(EQ#@5f(fPY2YvtFjp0m?LU^$b<2v7nr|i@=DrJ?4 z$f|nI3$mfav#V+w%KbZ4pz|Dm-!={Lj#q4G@3{kX0$&uyVQaIzHu6`6XkF$cWQndm z%T_++)Q-KUy{jF?8+~O&mE@~LV&~a(vWZ<7#nbRfMfcROIL7Na7xkM@ZmyBba-j7Z zx-7ALAN~rjmz|5V%kxB&AiNu`$212jztc30+zIzy|31`X-6we5C_Vk@`H0BFh$k7_ zYmr?IU8(I{BXlfpGwcX|@t~1e^4>nMRVI6@6u0-=9>*bNUM!9$EI=@L=*ejo% zTRlNig%EXGgt88iy>lz;#Rouh@Nf)6Mn+;$0v1^w#JXlC5ug}h-)1P&=)DicIrCRQ znvhYd8@;%(F}nJE7{1d9x@a@Vjq&YD6T_ZGhN@9-;OAvK6gpub1Yq+y_=K18$LDp+ zo|tGJc4Qk}orVc5*hLPZgezE&hYQiGy^w2;-~QSvFkI*EwyJ35;{Mr|kXO`;S+OCY z>Upe6&e^YQ!C}V9*h&3jYcbbN6*&Ku$2O7-ci2ahS9py1-1rbaA*t|DruSJI>gMmh zJz69!{juORO+ObrYVYTYZrpoTtI*8E8HrDY844f1Z~UZ|`4WMWQicwB z9lKj;rn;($ka>}9XOA4?)17<$UN6$!%RXKxQg8aki?dul{sDxSAprZe*dtdQ=j;eul4L+*W1skACh`nL zmvM}VmyeL4EnH4-HuFK5=$jkTHzc4Zo`IN|ykKXQ!{zwv%@DimA#J+C?=C-mEY=jw z=IDV4WLbqykluCDVYLNQ5{R_{NK~&kOe)g zi6VI~mG1@HLTjq6Aqo1ZxVZo=Juy%bOYI~Zbh^%l6)U`+bx&3z$tW27XL%8JjuV)@ zoRiJSaPj@I?eF1LnWcjiK-(l44vwA6p&G6={l)i~hi>W9!Q2z#bqBdnryPM;9OJiS zUj4RIh_i#S1%*KwIjxPI=qyoNtw52Z5QaraS3}3lQW>JwRZ&Oi&7DYv`X98l@%4Kx zbc~K?i`{di0@{U~f^hfcY7RWTp%C`vF5kU{_iUYkXF0Kmd}UHqWsznfQBFp&q-NrK zvuUd`>==GMWuqLO^9z^5^@u4sH#vqwgVCBfG{mfLl*p+f(Ai-4bxjsa%oa(5Dbk+( zMQJ$Vh)rcEcJL1pH0yXk+(Is6hD!Ro7o#KOTc#B~*T`@qVy7^pACPkj`_@auK@ap} zkjxMh51tJ4WuQglpO|-|RX2fX_2iUi_m>zbkDYI3z8LT>uEJB2EaK$*$qHJDP zc4OnO5nAw7AM5pg<%jw5xF@|&8~MwBnBMt-=K7v8#;3qv)32Karx!tCNg23$QT_l- z&J2IG*22{aEUrb9T^Kyy_{<>b*M`nFOg{4K#gf1!{Mq~rqw3$YhRFEl@?m)~jSg8N zxPm!Br$9aKK$SDpXrPG0KFN;#yTEhY5HJD?`^i-)EBhC^gZMkkd*vx3?NzEXHb0BZnZp(M5oYdrfjdLcl7ICbgM7+oJ-2aDNmYY8DkpW%Ybb|7bN37 zbUvdvG0;v#S=o7Ys{_XK$k7~O3|3uFb8O?dIu~6CC*pO-pBtK?+?BP}g?afX#+CFX z!y~KTlv{QjXab;h612e{`&H|H?TDg;*L}r2vhgnjreQNefenX4pPxa63i6Z`K|0pH zrDyj!PfT_3FAy>j&FS~JjI4WfBqm-5QIW0TLvMDM-(n@-zY1xT!c*7tcBwVBSw({5 z5mE(LaS8h{i^N_{B-3$-^hCp4plU6y`5l*SDe@(uOOCCaLBL15oLehT-QqXrtp#(^9-oN8CLsNa2fLP zasWzt_#{mVT+xy$C2~1z9Yz1FDl%u?w*o|q#xun?kSVCzvm2i`-$P@T6lweFg~v<} zn;3_O7Z(-6l~+o%DfY25jM}@eUzotnk_bEC_DMrNlstW&&nc#knI(Lj8|ACzNW_Lv zE|(`Zvcd&8zRmpy{cKlfYucFpPHrWWPL}dbnr~R)wp}El&%~%l6DgFVc!WUKIOO!r zJ*H`m13UE)6${#xxf&Ot4(06^-ah6_)On)T(*XIqGiEP!snpPfeEo=AHYZXs?bxsi zNNnO4=+mlE@_d&M8>C~kep_&KPBYY>JE!)0eW~4sU%!Io1^%FL-ANqQ&H)+} zaPo*f>?KoFRn|b`8f$WxW;@x2xMhcH@}=5+Z8aSG(nX_s&*W64%ex#1HGA77Thd9W zw_iN$a9fH?6=3jVQP1ypT7~uOdcPlWxUzBx zk4kXEoVAmU0VSKUFxG%VN^%7RfO8#QJ~T9^p7lh#0vj~r?0q~`E(o9ZK*`CWrKYJ}j6>mhdMcb+^ zDa>-y#-Z;^vIC`dIB6}i#PTfA#meaxVjLJB;;51ga60k|P`euqV$u%tm*4Yi$<|zS zqCf3y*__W2dx-S9>*RoiUz`X7pGbBZfgz&ZaieTQJ-$H{r44WGtgjprZE6k9wsVZ9 zfcCoeRlce7wj4)g0_vF(l>UNc?8#h`17CS%p=EP!?HFa@R;Nc=n{r$QVXzhAS-(#c zd@1EU^3X}g-GDM5A9kv(xtBGzT5Z)Rf|((;Yj3kvgk#F0-ogW&pWi$jDObbg34YJM zE83o3;ngU%8qw1BFTUleL>YBH){zaAPf+Go^3mEP-Oj-oZHSw#cA`K(k9lkk?Y8X5h%GKkHE zYw+@3$JwS0o>*d)DCA>g$#$i^+I(H5@EPn)@!q*}xUYWw4#~e2P+dL6viJd(t+gIi zu}M!-5{4q90HpBC{!GmV;i*XCD);(mffsD8ePI#0HeN8Kg$eBTNYLL@-FB_K)7zYd z5j0jOBd{qY;&W|aoWl}{fo%r_#k1EZ*~SmGEWwY91sm)JxJ<_stWtfxY_)Y3jC>UQ z3Pgz9y+M}2Hkc)SLqG7w#b22UI}_jC3Cdq6YNhJdf_3HdQuanDdMnbKRatf^4H){t z4^qu$19J4tf5kFIia&V0k>#(7Qe_##0+kY<+SB-@(LApA#?(66W!BwfdCg>*e%u6|DU#9P`rpwM^O? z2$&$Ay?ObHyBv5ZP`d&q)fr%kvvi8Wp+WS`YR)hu zfvR7Y?3a+k#;~nb=ZomdirYF=+F}~kRSLJCFbxa!>^BYghn}P(S+I~$1_bAcGQ>~4 zTZ9Mwd<-Xc-m?mMG3MQhgH>!Uc|UUuU66I}=hCUp!ywjF`r!I{e%(9pooQB@%*fD^ zc(3Ey5>LjHXmw8;(LvBiq9&0evr6e6|UOjz}sLL2dS-J6HK`o@EE z&Ltg+Vcs;5p%IbDVt^-PIF3(hxM>tdauDN|Fof% zZ-Z6{qO`H|{3+YhI#rBh*y2hBpQ|H`$UmMdWyNj@NP1XA){*X7Hq+|u55#Ll@K`{TEK~{N zLc7ZusdhXaI}!}r#$Sl`z4U5~FcImGoP6AIrp=dQn%ALeKgR7VF#sErGgBA#EA@GADrFJyEq29s_Mqg;(=}{ z6XLe{BmIT#s^9~dMOEpB)8=oHOr3e`V4iJmZW;R-2m#CKuU2{4HV0HY!fHH(I9rm@ zxuQNJ(h${}K=l0qAAhu8Tl60M49gg>w|tV{T-GvkXMEWjSq4|za%bCUtH95~X zJ|a2N)2@CZmI4gfu8jI>?NqfvV=^`TuEY^xQZCobj% zCMovIiDy5ZBC)2%d&~~d`}dKE`zF6DT0KcPBIN@w@f?NiZB;)GEizqjWr_N=cFFjV zWJg$}?7*qy$ws6t#sMr41Ye}}=!6Yn~JnH^^T3;f;O1r}d!$15tZIZDm zc!@z!qKeB`)Nr;SX}@JI-&3zoshX7r5T~YHAhTO%tW+@k;Ec`|(f*>n<`5!b%QA$< z7KK>z6gD1@euW;078^tOcHJ6XS;-dL%nGCI?GRBu@*<_ju!JQz>_YN+MPO{n^HdAV zvYH9U1g1g2q?P7=GMO919B?XsqSeRs`HmVYum7Rv%c{oc^|pj4Y$|TXwS=pyc^2LX z@R$;*s~2YEFlWZOk>6-)ccfMCBQZIM?LmH^GZ_>S`86PKm`=xOSsU!9Ke*lqF@3OW zT*_Vrs?H_DK&Ob7uV!Q!kZ@Q{Y?DK{X|24F?jKY?LF_YD7w$} zjkK8o80x#46u;Mz5gK(WXkUbs)$^=6Ez|8UIrRFn$vWEz=a2w$STn*N1bYouCUCS@ z#hK1Yl~Y&0+FFjL3j&NYu_|Vngv2Rk*MBa|sBX7;^~twX30sZ|+0{~Yx!6HpC&WSu z!>shBS|qCCW~bOQNraf47}hKhklZaTkoSg4G(aVa6AD}UQHKofW8Vp0W(gUZ!%A+bZTQ0luBk88e<|k-0@C18U&93ML+?Yz+^J!9W(wx`V{6HK5;dP>l z9Bb&{zu7BNIxJaI&`e`dlLmPBjeh{{QIk0Y5lwQ{tygcWNS!j_49a7*bHdKk#xi3H zm7;b5xcaaEwzfJYrC5s&g|*m@pej1V8qkMbWk`YySJL2(BqLtyY2mTYaYKjeTnsi% zi_3>wTR@+`V{Io&yF?R?G~mrv*k|5qACg*3imT!e(OlJy9(bgxX7AvLdh5M{GqH1h z;~208OiSINJb_pDElPtV=6ZJ>gPvB9A|G>v{;8=K>QzsQa=k@55%Z2oFDk!1Qi?fO z*t)Ne;B9@-$1h{73$e}--L)ZN!{r#EqJV=pWv{BqUIl}18bx>T@31jbV1BHh^6e3^J;`8V(R?ehprI2$eP_ka=H|v$((qz2Np@vNK$?)f(O=7S!lS-e2C_hPzTB&hmis4R={}q4` zO%FFjmz9CMwZeK9b^N`?m!5+w89NjQzuOmLzO(RA7)kEX9? z8LJ$>V^=rJ9_nScY0e77_GD2~6#9hju$;@sgG0dBVbJV623f#qrC=thJfo?0-{a#!#acU#?WfpjX7)$)NC)2b}cs(K2 zN{E>m_iKpC@xN?XOv7q9TxL9F6$E%4rQLvZ_U z%W86DdX0&D6?P192(?bpyFIRsD1Z0+Hk6Nhu^{>OrJ=c=6SlA%uTh5X^ZLE9cH&xQ zoonG#SpSqWRka!Lx^iJcEzFu4jo_oTXPTE?O{a!90L*_iR-%vdnakRV* z55maypXV^6atH!IcFEi!#yT5Bja^f*NRo(t?m~IhHuV-n>ehJNpMlaCYtI>%Lmrju zBM~!o%tq4uSv(Pr##kQjUa06AqSPczg*UpjO6AnzvK-Ry_32XO?r&aqq21JjvQwJ# zOIv^YyOFP|`K2*eSXAPfoDS1SNAod_U|hZxtzcJIJXbuyBuJS4nxN#jk^yyl?-5PI zX*n~tYRAm6q6Ft>wo8fg0v#BQg>093M8!PfRGNCampi`vTvcB<;A@iY17V02^0=xd zvY42PSLKMeb34bR2YtY>1=i1EsWz-DHkMn`H8J~=Nw1vjvIGY9G*cx-OW&Qc;@Ukb z?{3abiQImlWLj==lM6WE-jZe>9tt9x=7w4Gt?PrFOKSAWJ_-At^QDvl*X4clONLY5 z9LOBtM;Ch16hl&<#o`tPrA6@rS@epcVvexdEz(kd%(z45-SU$|PxVtv>NRam+(!90 zm@8*4Fec_c%1)O(csXR7ul9Mz(FpZEeCP6Ynn1q`%Ox`i_K9BA7Pxq+20ds<)*+=& zAkPBjTY2c-H)o05^E-1AEp7H>+WR+=@|%UwyHo$d4O-avq#lYEBW0o~*xpTlwNo?A zNsS^?8Xrp|k&-Q%x#YB`#s!R_8#DuHP&`1YsykbnGGhswaJ=PY@wFJtG}GR@^O9BgRZuZtxXTfrwV z@+|WgE=@s{3a6DN|9^bedY(!Tl2E8M^EzgZlIUCZqJxk||92^L{+LHxGgvEBJH2Uf zxl$hsI~OL7xpO>Yi#2*;a3t(w^=J83Poq++{FqZPTC|MtN6s0;%-F^g%4$Wd22y8Tprz09yS&8IH+9@@Eq2$1CBH~}^|OtD9G zSUk+SF!*~%KWh2mCujQugPW5zOycMMmr#o&N%cf(D&GV;96oL-98YbL+^9UmXocHs zBra@z(Ql*Givx(W?KxT~{U?H~=Cp=&U3BO>Xv4#l^;<^`+jE=JWFm(5b*J~e$77z$ z4IDvL9t*mIFHeZHui9O|s9k6Z7CfN#4hUbW8O+LrCS(zE_nd+Nx7WP@zJke2F1C9J zsRMmwq(6oTEkZ1f3lQT@N9&%87o=`eD#5Y!BJxWJ)z)YVx4k$tI>YDqkY%4_Cy}y5 z6)3{CM{tC&XYMA9l$G7Kd zx=@8z<1G{K={g6>O6GfTl~m z@R6qUP6~8)OC1p;>&r3PU~uRbe1I=(DMu2G3A$|!+rFC_ZQ4%D0ZcGe0Gir+Saszj z)o?h2uNeI(aK)~eay+P3oh|3@sfT&`%%yXiNLt)V5|L&2lRO$PZQjUW0SE4JHo15%=(W zZX-hIPnP>nF40|pD>cN^^tWL@Y7KUDG%iNr1@`D!^6+`!rsN<9%xpW~qH5|VnJnp>CcyxwG5ZMz& zQ<^uX^;)~6*Q>9r!-Iors=Ni+nbMBZrP;xH2h`*W^1pf|j!| zJwAl%utq?`)i-X%Vr7@b=?cxu%v$MD%Cyy8oZ2qGU+icISKKQmQ901iSHGd$`3N9q zvvYKqEed-|g4)(RMU4^vH6>Jm&eX?xK3~A$)cZzh1rx`CZsfZptd_ zE%2RGnO>gMe*UPtnu)xF9GzV6h2-?nLzXfik$jbBtRI7RRO5_+f$8o}A;B-t1O%b@ zE+~*rxd=N|?9Opcj{mAJ`)Xb*gIw@}^Ze#6V2b*Z<4kfIctF=%$&(h2*L5ezW&F8# zwx2h%(`P}oBO4kMog1o*3z2#DmD(8U-QYN`lD`hIbcWbp3b*t_AP_kExW~Q|U1b&m zq{)tvq?Xc9`*#aSHt6uJ{LOuY%5=sVOldz^zwl-G6tIXUXg0v34~{J#P=QYxQpr=( zD;ul?=HVKsY?E{+1`6z!v#EC#e3)BR-Eu1ktGJ+`7W~x2Qa86>m!Yll(Ou7i6%O|o z(k~`;Wu!pyIo27KTEJGKps?DkOr_^FTJ!zdb}hY^j16Z?pyV`s4rqtN57_SG>eY^; zb5Nyi3{v#4oOqL_kJIvoocfs*X_lwouQ-zvBWBh`)?!hm;?wIlBibaW2aqxlrJ>6{%Omc{2 z0%~WiTMy9rSW!Ih(XHorKBN2TRRgNx=;OLh0sRoK@e4^vvbm0I;4FRHESx3;S42(M zlYRKWUO$LhHnOmKBMQG4038JW{Vn3INlcR2)qqmOr)SAz#^(-($`9@!+e~Q9j@=HH z8*p(=J5f=zM}J>sW1h4oE?z?uWi+(8`MLST&UnVaqDppL9n>~~)Q#+kOEV;e2wP*( zRoI)Esf^4LK58w0f83#nZldtcR!k9ZPVS0t6>487C!GuODKT<$F=Nss?v|A+plgk^ zw)mi}lD@g_e8yOcyEmwa)dZ!uWoDRfXsU^|a}S0kEm4s}@;E&)wpzeKu$tOFEtHq_ z8Bj9HZ(+2LUe7wxoSNG4_NK2z%4KBZDf#tG?Qi?Xs-o%kocG)2-+-f%_4*O!Re^Isyp~HxdH^kXM1`b!Svb(m* z5pEQNR-_>nhm^Zl21PoHpA<%?dsM?+hwJG)G{}5e+Op=28b>{JZ^_DEk%(VWR%)Be z=VB32?eat+G`!T&mP&k(T%OVwedfolBy)BgMkN!Ecm2ezo9=fWsCj$6`=ucKN1OLN z@_IHgLkyi&T?Z;7jL&?Q?A71=pvaHSel4xSQz5F$Bx4XCITmR4(_XX%)uN-9Ysm8k zcZCv0pBHz~R-T%5$KnMkDq^}x5bSOeZdcCYV&W?=?(VMecZ}?_>+@4?fiycMJV8s% zRLw7lo$BqN73XvaWHs{aRh<_;( zo*_J)805Hm5)hywQIo?kXxqQgHnG!UpN2`HMr^~nuGDCl z`dIs#2Mawd=<*mp?eCE%KcPjxPxj)$TwqZgEsUI}QI87_p0ZCDnbqf?bQhcrn>Vq# z21#ydPhxUPTsS97VdQOKu7Y@O=`9-wE(k$ZTDTB{rk(_+zU3x22vy7WMH~bBgHOVL zBQV+A+H%;TdQ-oLRn}TFV!Rded-J@%*AF=R-iwH*!nk$Kd!|)r+s~VlxE2mU?3UFK z`|niod_m??FM^vaS*FE~kkr`WF}4j6DsPQvea1cOsoStR;x=U9dATQFPuU+c6UgR> z+S#Oh){=v7TkYv)u*{{0kw?yuTZnt_iz^#5(nfXXiRG)LRhFYt%h-;-^i~Gi2PRdEF`FF+vZJ;7Dz90p$Pa8a{rM!yt~h)IiNg16W8U$4hr|-Sv?7)}5}NufC4qnP5YTCzt)%Mq zK7a@QGv5zI7ak9B#Ydm(8=gC*{q{AWeY9c!`jsTv&Al|FW_4M3wqZd%R^706dlYM% zNvF%r`}#>(i#%g0IVcQ%9j()J@eO zzdk|OCH*^CL4#PFrNK5PI%AI{WTI|aMb<|t7S2-1U^2>`2fwE3Fn(E5j_Ez?;_f; z$|xnldDP==43{Uflc4l_t?b>G56mgZno$J!8cRbpX>akNMDAi}3g)A(gGK{^@zJW) zZB3XFGbNl!L!O>fTrbU-jf{XbQk5+BM60uqJICI@wjq~iV<#P2e~DJQ#f#>oAv5X0 z*H{3GCb3F~nU2=ps`uciB0+9{0=I{atT%D>yGi`G8!}qqNjCO4sE@b9!tjX$B0PhU zmn1;U^NUT~kJ7QQ(VCu8P$;sIdEAvcs?H(z@81$Lu{T@8Q8&65^keaBp_@T=3aP* z!Y^me3}DzoVvlfiXm0+pIgUP=^M@u*_uRSK8@3^uu$Za&1(}h%;~}pVnC6`x{20q6 z@^3rHrF5mXaefDvs82?#Fw1D7^1TVS(4cpEWSxPGy|3Yn7IqL+z_m-6;Cdesf|O%K zFH9R5FwN1O=M}h>DI837Rd5<1CQM(tlTq0izSgjiy8Lkq+j*(-9EI@eSOCh{C-K5( z@Z~mM=3vVNxO|+AS=Pd?q;Z_Mh1JC(+5S6e{ej!V%D-MFqVx|y-c#3b{LKNKYS)r_ z{4V2`sU?b~+?5VZ1Z3%s8_y-~HZ~;p1R{6p#wsk|~= zmcEIE#W3^1(csTWanz~GmPcGxbqb0a$gpN@;1pbgedFgK1e-Dyqj5iYQh8&*4JsG9 zo%Tf-{X+fgcK)SR!!dnnzZ{G#mtOj#%?dWROd?K5G9-ttdY^NsvJIPs+i(dCeszNZ zJbSNcb7|ULbDWr@OM}u$%335E_jR}<&VGA0pTlqb)t%FL zb3&p{ZvmIYSnQ}v%I_PjM%yLiY*JF}7rBU!Sm|gdI~#vAz9>8wfmhn<>cc2@hKG?5 zpt4Vdb4qO!@0sd$3YjSIR-PQ5?NgOAYeXmt3DdqHuv55H7-|dg(K_V?OguL_DMH*v zlw4LXh*QJpNF|ch;cN2Hv82chYVdOu=5+YWxf)S>ue?c)fa)*v>WFCrcE_L^K+}G7vkV36tciK*t_o=+&(HN^Zj0fPGall||9? zB zXDK~8sO^?QHuDmhIf~_S6)h_Iu@jN?5~`4ukzttA&8Qe45}7D^wBHP2P%PudJJ;Tm zCY+f01a3D`D(%iGM}22X>#NF`X|c{UM@>AZm@@)9b>E_Ym5tg>PR3aS_*)qY(cdrK zAk=<;RR*PDJ8U85Yd<30I}SQatVwk<(3JBB!XY_TQ@gjx?f}A^s$Osp1yaQC3WiOf zoNCyu8Tr+w(v^bHw)L3?`xdmmI=OXZ&$^H6d5g-MH+7tp?$0Mx95qB4fvx#oXOD-9 zr2m>z_T#qu+?h|gbBg&&V_)A_efe~EYxKQ|0w%6ul2?)CjnN`wUQ|IW4yKC!IO3W2 zR?*{Yb>GIP^@A-6y!!#xbklOUMm&W=5@kq#+WgG-NWM;InlV>H#)JLyC1t|yA*8V} zHYRZ^NPqFSH-YZNQ3}Ia+eXP4M3N^7uEz+LQdFP4k5EM^#lHsdI=Ov~@?^22;n(5L zQdY`;Z`T(b{i`~DW;xFVbm_B^e25_mU&smpe&jKI!^?D|jK+M0hKR$!ewGA9&uA_h z@raj7cMeyH-J*A@-OAy&X{lfS^tq0+$vTwWNp|PkNB^PTb%MmW@_LkSzew|XSxxP+&3t>lkCfq+2g40Ut+=f z#_m45sXuM)9^F!Z1uc1TE;+mu4~ngJCSE+k_>+pBTCbp?``=PNBdC8zh@$X`b-GIl zdP+rl4b=(5BY^r8_nS$pj|%V!oDYxW2V7F$n%?qjQTlcy66x4Hinh*G_%f^UF8L+M zQCBz>1_Rn@_V}og5>!o+35Wn-kpBdxroGh}BbnUf7>437mxnztrsRu9es`(A^Oi;N zO~jrY$#9=d&Nn`l^QX*r6^LV0>G)QKyev82Z4piv@8fxJ)?i7hi!A~(^ecOJn4lJP z+gw25?7O5=Q&)&&UM(^FGCN50>!yA7-Qx0W#qV!r&sf{Ts2UE&+|TsCI0-vP@v_8# zHX(m8y9yx5y#`*|nJ|P8Ocrd}Dk!hCts`25E0BrNJnJ7&n;)NFp~xaoadSiwmsV)8 zoIv87kMoz2Ip?+!6HYvivAF$?^~O1aWcAJNj-RN~~@IDHF3Fjg;q4 zg)5f)Q3B4p!Fi)>BnIC^!VSv)^ z6iw@SMgtL5pm4MMnn90PsrF+@`}`}eESuW8ru?E-;oDRl^_Ej$pji&DOF+UFlfoFm z4Z#;zo;YGIR>xG-qX(AKQ(rNCvTA$2LfEEHY_S}Y;~yv3_(`b?m8FC@lavqwz~g-|#ES>VfbPp|_tgkL zSg23=(+=SuKF}YH-0`6|PJZr+Hehoj$`ICaZ#Q^Y{C(!s^XRt~R;7bj*Qj;>D!oS< z72f*|YS=<}dkEf=Fh2Td6cbTVuF_0(7I7z0K<48E6 z$g1@}J>2{Tx%j_5@i_5^CsN7TZ3r>HwsesAY#ZtvG#S?jLvrJmEN=L|2lc+qOFm8^ zsRbM!PbSb$)Tq4DdbD3kylFjofh|aMW;*7u1HT z$WdVq%zcJ;E&lu*!5(&FV}fRujJPXY5F;KIw)sFh4Y^QwvIUoJb~!n;43pB?SfY|i zJvD|1o;G>}KWT-TiOo-?frKe#erSnNkeUna4*j&Sou2tTngS1VtXoLCo7UJt-{#qT z!k+-@DS0+^MN#p7wN5Qg2wUH@Mjp*VdSutYT^!O8$s4I-P_WMLrG{|6LtM~H!hte$ zoLtH_8*(4Kx0W4ze;s(1IHZqc4Tm^QfanchJMU+@IQ?7LSh+q{NPd<8cKwN^&6F}_ z&WWb&14ciwD!$dt7Rz&dh_({6wsk1QrdM4qGCtaOeJb6L;OBBGzN9v%Oli5yXe%iS zIDr#&`^NF`dJTZy63|VVg|;TfH!PN7J{HHFjfe<09nP)sAeleMN$i;T9Re99^Lx=CAf43PsWtO-P&L>GJ-$@+Hp%mhc5X zjE6~U$(eKjhxWM^+km58ZBwX)7~}}tA^n6sS!i$b9{Y9^$Hh^8{!0}z$Jc8h3as3&`=I zW{CFNz^YUFsMKKjm5VvCf2ei> zmoog|I=(VUX?MXJf+w;ZZQAR8jbH6`RfdK8HikWu2ne0|)d{kygxu;!3mlhb&?>c| z_3gXgxtslL;g|m2L=17);TlbTQFcQjAX65PecF9|>v}mNWp$I)-M`MO}rIncB1@^$CGi_xm^;EKk9D?`v~E zekGuhf;eV7{^1Z_o7`v;@It%^Eif?pdPrY8#CatICDSd_ngsB*H5qHaAct~Bmbm@- zMdD|fY-}%YcX=6Xsheq@$2ieUt}o+j4}!*9dive!1ezx64!S zLB~udh3O*Y(Frb9+p+KR*MEBHDsj{5OaYT%lXI$`iPrf08#c{8*nP$CdYaCx^{!lW z)z$Oz6XYiQYfEfwv!f1DR9v({#pMp?QG(LsN#gh7Tj zkCsvMbsH%Weu0TF9%e#Ax6Sk_R%C>hF5<^ku3_ry`M-U@TX|~2T0*{s9{MYe&6G5S z^ADacagG;JbH3U0KU@hUw8*9TY#8NcK9;=dV3SD?@f9OFbPiNFi_H(VdZDXul=GMftqeu&|_INy}n1vbbFuq_$1;l`D8o=|3h`Vk}U< ztks$A^kri~kHgLsob^7Hxo>?%#3B9t)roO=Q$sRlyXF6-iuW@iIy`2lZKTnuIf36>xj7TR&>RyZ_Q4EZWxds0e4l*3yL6s z*S7)cMyJrleC8ps52>Rj)RaNT%ve=kp>F!ag@({Qr7>o|(A@_;tQu80KJ)f(lp1Y$ z9Ntc>RxDE8Xk~0BHV;E9|Mte+5Zs3mqp{jf^01F%Vs;P@xNs{%FUiVqQ#us8aY*4R znSs!LA^vmszM`O)=q6HTy0IHCjl!aj>9&TK>#hqAwyb1&Ja&55jw5+mx3owJ?+Y*Y zR-ODAf*2MQt}pE&*HY~8GYk5es6w~NF_N&E$*H2}NMVS}8$%yQ5)#OZW(&BG$zLdP z3<=-HXNJ{2wL3}AR6hxV@W?Ez?q9;VjL7RD2>!6t`vOOr5qCLgf=-;EgN;G>bzjvI~FKpMATpX>#B0GTYMfTRkGyQYStkSFOcb6|cCiy$g zMEf!vq}zs$WupUJN{MO*JECe3JA8IwG3)9zZM=+8iHuyMlnup5svihY~?N5IqXiacP4%DbuIb|%PP<2eDfbYuJQcxUB z+XY;xW_ORa1AO1I$Qln$5#ToS1JeeEDZ>b-W2*N8YUa*=&B^T$Qwkd3EFfck}7>Q;!t5wcyWhTDr^po+=U?vbcKZZ_wTn}R293o#v3*Owp%RgP^GSr3V}PntZuLdV^W{R=tR;{T9D8Hw06| z)mb!26J>RMekLJV(OVGagl)gKc|X9G_}ffY%kVG3`XSo!LoHtm&SA#HX)fdJkJ+vx3k58#8B&3~9?ixnnJ;`~~kvTWFvb&&dKW?q?`}Dn+Cs zTll%;&YK=t-R6kLF;3w?Rx)jnnKn1FSn7&s0-vY)L}JXS?#De1 zW2s%jw4a|(kcSCo5ukLjxF?nnTcc?cJDs@}qCB9NYeHyMqVqXjBf2RoFTdo=2>V7R zw9z30cfekb%gkp!ikxm`_BGf%ljPG(N_-9=F%`MT^-<#VM@2K`Swhx{NK=|>^B~ag z9Ql<~{v&2~;H*`^RZbPy^=UmhcWc~*MATMLxiT_@Np$HJzIEpXq3=D>9p((@u~9}# zi0j2T5#9$?Gf{K$Op?0ZonKpT8_JbK7fbJG%w!`#+~*xnNz^90yi)=~+c*L$P)xx+ z{GKQHE#2XOn~a}j!ryAs;g&g(3ggUVPKMvtJqxb;>Uz^1_^6VW5wIVfzHO`+#<)h9 zXeZMCK6WdR816M;&{$c9{{$-|Tn8X%+wxTdz`G!sKRIFM@^7a;?F8AzJ0mZeMXM^U z*R5gM^Z4A~^I&*Q`upYq)WJQy`>@>9BKWk$FNV9J!pfPBA^LYp{b04k?_{Bt1n5k` zB?|d$aMux1jYqA(ynKJY5>kF)@bM6PZXN3g;*BV}wnz#8Jyz@*-A%Pmkx~R?)f0qb z;q{9cXx7`mBlB41i(^FwIcRr@>aXNF2?ixA2+;oi!76teZSp(PiqZ+Vb{Hx$hdm*a zy}^R)FkCkkX3TUp1xDRHr5!u#T*>~S)ogFC9uLZ(K74zm-ovc?<^(!VRy8_TKLiI? zcd{J~?IH^W7?6HN+8dlZoSD2}rWyYpRoKX^g!;|;;w9nB2c(?=R-&@jo)4XFC2mZM zVN)G2g?v+hjITUnt-h<(FX+PVP-3Z$YxW+illSr|uR5W_n#`asKyjwHd!qO}K z!Rp+bhsRZG5weo&v6w`6Xge-?vqI-;{n_bV`_mkJi6))HY8LEvp4ehQGpx$NOeuLL zn#Fklky9r2pcKf=HWCcaBSbCcu6N3eu7FNT9sn;B=@a9V`WxK;ruPf zX2FQdFuz>N|B0W0qQD$od{z6NNQ02}8py&H}6xXCepC3iPD76HDOTh*AJqWQGK z6auU5e}>bkEG30MlT=X8rAGbU1h&eCD?=7&cd#z46Q|VDEt+v8JaDobh^BY<8%U4M!5_A5Oi`El~8WiH@|;Ap;l zJs4xbud-SaLDQy5^NK`3&qeLp@%~G_Vz$AS;;b0Xp>AOg-N%>FuPq1tNzV{TKUGth zggq0vV9(Jbs0Z<=7oC12Geme(wU6QZ85i}P7AZ-%zCf14cj0|A45~BrSEO2|GAo`V zK#rl`=5j%xI;8Gnm%QiluKfbjg7oil>k|TZZ=jRNo0RmJDGrwtZJO?+wH8p3ETxm< zTiaieB1@{DMvs=$PZzXZdpov?BRh-(?V_D>MRtWdUAP{j5enJX>zS=b5)SchOZVxfwdcv0QFrKt{Q z@d?kY#hE95Jdx2(AsD%)7Oix=MVHA7QEfy=^<-bHUY=7!$jsda3Jk$F^(c8Oa0h?0 znyRzMkVq#Ao;9<^6}!jR0&b@==`w_LzO5cB`+1K2E0G;`o5i=d-wrL}_a9!VOqgO? zd+z91&EufpWbQ+k*w1pwmr#f(tN%ZbFk`1-k+P0TtFlFM2F==r7)w_6O13{fxI$Tx zWrUH!^KKRe-F^ob-5oA+IbBjTIE93J`rW*=-kTU-*D?$y$TCS&5nFp|IO zYHZxEE5wXf9kXUJ5iJ%QD|AZw_dK9Umi^JY_9-Io2V*pD4aRCJZ}jwHSB(W2pZ(Jw z%7@Z;a5u7f6S4E|7GOMiXj42I03Cx#`R zvyYRLednB2#qYraTpo3g!rF=Xlg3nE@s>aIBHrAIlS@1kwllteuE_qz#cSM^XfZq6 zPgVhvxU!S&=0tM|m~~B7e_5ChKQNrT1hOo7f4qJ;e_!D*oTffh+St|2u3aZ~Y~f7(fTRYZ z==(&YE=;rCMXN&{H?Y=tUVBAExSb1eT#K=MX-T5r?7_ESu@OP-3-`4eQsUxN+EELs zu69lyv$4`FOmQNW7HdG~C#32vR-t6=dV@`&YupzDNV_O#G?Gt6*16^~e|IHy&Iv_k zXfg}3&8m7uixUL7#@-+y44WFnQ_)>4R)z&{k+1_xvSb3L?`4{8IW4^Ml z9OpQ=@z3&Y5IzGVB~6%aS$}o7Qtt4a@Q@ZQ$QbCaQMO{C;I*~P^BFA9`x))w&dW}; zzsg~Wj&E2cNe8z;F%YR%s2IP9=Fj*zVMDa3+Di@yFfJ10#nJK`30u_~`*=TKZfBPE zNp~enGWpku{fKlI;G0mNM4{q`-&kMLSkfzzI=C2F74){!KFoH(e>eaHm{J1vh?%`LsD zc)Pb^;u~OX8zb3|vA*0QF*3{#JRgQ4i zP*vs8o`}miPsezV>c~YC0Gs+@`E_Mf_v8ezP(K)fty@zADBF&RYp)Tt$26uX$Fq`!+#g4Z!H0qjOOua!4bCirKKF6tc zb>i-==kjogAI%&b_%Wr<&MdIEC32jbgQL6+s>2VFazlwnUk?wJaX|U$R~O{6Yt!Wz z(3POA%{e?^?~6?y*-C$#ptA4Tie^fx;ARep#t>>oc0;R2vFICE9s6(WFmY33z*|f2 zreV`WZgQtA8zjED%8vDGD(XUGWdelMB}sNgr()Cd{Vh!1mX&KA+j6QTg7$Z3DqlIV z=jsKRY73!^gLrJz?JB1Cc@UTm$;-!xHfY`Hr%m_U&3VJpW7ByZPp%{l#iB=Zlq4le zGC^GKMeOZt%+@Dzpkz_1=EuvLpZF^w&Uuk`#Y2gJlrz!B+}PJcw|%gRk1{e^gs$qC z{4u7eJ`d+dB)P585Z5CsEPuo{{-K%1Im^egf%W(6U<0vsQ@dOsJai7(Y5|A1IF!>Gh6!;L@ixGz3PSLCzAN zW|Rbl9t#eSxRpU?^)5f2g@zYa6oVEG_?klAR9Z;9kJi$jnJbXK=$5w7U|)JYGP+B5 zD`%?*qt3mv!P2o|-J!K{v#yet_L9^g+W0s`LpsUZ*Lvom_aky!VC!s}qzA%*)VTqUi;Udr# zFn6vCbs``S+G@}3LSQHopIjd*2vfJ4CVugXbI{v3AZmdMDgAg$b(A%$_R`&lVL#q2 zn3=V(SXIWKF~LV-wmi877l+(@R2@qG5vNIg>GnW1H;xI)#+QyMaM%tyGU_%FMT?P> z0hc+t8^>IPlJLM8Dg1DQER9B)5vR{V;a5HDai~3smc=BOWNLm5(=flbG`ey)(j>$^Nkl+kO%bJ+JcPBx5=CGF?>@I}7$Vf(@3 zS{NBo2o6^5?rSQtn1}J%N%G)R-gw)@RL@I@k#7h+v=w$%{9J5*-H~YVCqmWX+`_~{ z40dmL>|(AWRg{gava|Xa^>8+ufF2=3@_RY(@kCe#igdGos*fyY+-n}=(g*~M-G963 zc$5CO1!rN;3!|alLCL?@Hc33iYU0Uzt-+)v%oaz>8V+gqIYMv4D1v-#ehmbkEYSq$ zTSam5t7K2K{5m30?Rv9L!80><4+T*>a7WwtK`^5F zIhEr8Tp2vLIsLacRdvLcA%02eNUsz7ohcQet{$?D)DO3!RLMpePkN3i55MuW4_9@u zh|u$frs`+Yg)sI8CT(ilAo)-SZQ{<&(ZKtj-XB(GE*S_=>j&JhmLc9I)^~#vWvq|kx$b?2F^?Nse z&##uz+kL+DxB70+m>V?$&spn}HuoQS49V`w19LLchIP~Lcsxn{dg?9bI9xON#`Y7joAs7i4MzyM^YQu??a} zhdB@!&XlwV6g4ERkqviI&9IIXiN~6g)4MbfpWdcKM+btmf|=O;I*oP%cQmkMtWfQp zpPJ`soQ5sQd!-8EKV*U@{TYrKBOYa?>S`oU3@i$tN&BIz_c$n+wl`ew>2>k);JxLG z`{*DM)=@?jQrn%~rceS8$FrM0g!ZjezsB8y5O_7iJ#b!~BE@JR9K}T&l zdB&zQ#l;t6XhA_mKiVqVMPu>YA=k6a>R0dHBmie3GIm}M@@p}PN~v&0x(7Cet9;tbIin`wP-;m;wKAUlJQsLX=D*pKxi?sdEI!|Dxk5;i#A0U_Vp4 z?;l3}dHZkeSPVc>aup+Vh1rWrunTYLOu96*^IvHa3I}C;Frj3BeV1%re4f?y^c6_1 zsHNZ#V3FNyJ@yNA5nl7WsIz?E7)AEympX#l=?YtLm<-fY$+WOgoWj}ag(uo1oj5zC?c&JLNl5JGJ znR0OLceFhII}9$KN5#n!&%HXC9WmS*M*g1G^I)h~=e!W&+wN=0t|x`{7JwO-zTCtm zD;9KEcAWGx2`os+{mBMpjcdO#^Tr8Pi|CO_b}8E;4yx@<>Zg@Ps(I{)euasmX*mfS z=qly<$`=3H%!DBYnhz(=W9o0S;2@$oyOJcAb@60MRryG+2h)U2`2NtJZ5Bw4u)I%@ zBqgmzH;esFt~{DUQWIPj+X#?PJ7&tDxVg__1=GkP$~>Hy)6|1v8j1F>O7$}~1vOFj zU*9;-UeeyiWo5ad5Wm)Q>0ZbJy%%Poi*$IPEJu4IbtPGuMvnA?*qfw2kC>%IrKezh zMM^2jDu^Z1^!kSjY0$DQ^IIg`AEJ2LU$)Xl)kRhdwPBp1{IXq9gOca3IhsSANm`?A z%-v?G_R0OmO`H)}Wjl6GcBx2+#5~Axk9qf`tubN_SzRrLQA-oH4SS&*X#0_??al|Q zgDUixR+l|PCy|PsBYKq8$b2tUJ~Mqp&b*ut)HY9Tqb1F=L@NSn7ACoAhT5puhWP%( zT8%j}k=&EmGA-4IOFx$&1l(PU%Yw_y7-!?^K@ z5GM*XBqt5*dlgiUW|`h8^ZbGA;Iyj^4(Ha>uQEn)EdZy;I@eTa@0x;=On40mF-#Af z#-ElF45PDX_Gv#}t+2#l)80!+DG@GKXQ0{Qccr^1kQ`Y?3=sn7O^2KZd>Yku_7jC` zX0+X5ok%Q3B&f5GV+=09=c?Kuy1B^f5bWy2Go?fz)_&f5b=$V@ z3DZ!XIS#AaA$8YFK~wx~!rizv(hkozF>csbTP-9O`m$2%Hja{i32j{1N5s?4C0oC2 zP=sMDgXg-5&sKV7%D$oZQMnvbcumtn1(ki{_4or$Pc>34#&0=e&9rsn$+;XggDmfI zF>Ns0LWQcUeMVt(M~&q%Q!~OqxV((7}w161C7y8@@7TZ#FrSP#-KE4p@ z`hoj+Cwq4ir_F?O3@T4PcQ$#rbvt6BLKzjijjrs0*85R z5%;k`gKP?0vEkIE1RTCaKvbH~P>Ry2GSEe~h2db;u*46cbv`mFKF}9LX^UNCRvZA7 z%kfsesSPR5xArjANur%@k{ru$h(B(zOvmkHrv4u|sSJ{+S47sis z=e?i%@y4Ik)m?T#{WKzv@W$mw^bcnH5A=K2B4jpmN$7A*JHHYtD^4h^gf% zQZ+fJHGyKSq6f&S-n#Q}Id2!~E-DoM9nr?EGI^Il4(pe>3a!Tk88Un6(w-(B@3-UF zTfm-I@sXB=XqwIGUD0Wv8!P9t`O|;~MrGjh<2iqLrWN&VX`L+uj^sT-DQfdI^Ojyj ztM=v`<4vy(?FoVQI*rW6)DtiMXb^1L8!+4*;br6T7}r%htWa31`ukEta&v=>@znvJ@nBRm5~*9?JuI$Xwx;J~T@!QVtS$vcI;P3R3% zRHgb*JzuZ-2r(>QkZr^d9yqKrF+E7C*IKQ@G=>|Sgq^-@T5-~a32^~~IuZgC&*ZxE zK}(<;JKI&yAx!>6HoE#_%&04Trj|T^G0~*Gm-0g0xhuG25vIQwV2FJ*nncs#<5E|j zS)a)7x#(|C=k;`Xz7Bay<)aZGh_EK@JNf6vjxuIOlHRN9DB=fY=SpPjDX*GRuh!V^ zG_=dyfkn*t22;{YeO z!mY^CJR<-k7%=QX52IXa(@||*NoSxQ$v7c_mBk0KI(3EFn~Zf_b-xBO%-(vmw2!-C z*kIl8Z%L(@qx-k3rASw6m^>6?|C`D?p$ zR{MGF33&cy*fGje-LcXM@Qg95zuQ;GWhHLT#2*JRzD`EH84PO|Z0tQ(Y&JAawFlxt zYi3`e_{KbXoQ%f4;iO@>ibZPc$kg{6d{$d#q^&oGwGiGkF29?Nqe=F&`6;+r<%@QG z#N{{87WEsP(h`B)gNLViN{6P6HQ%Ei^AVcfudC3$1B&L*5Ipri@8FC?5*+vC%oA{8 z&kmP;S7S^*4RbU1;_kn*CrL2L0X zv*j*Bvv;*@pg>p(N%O#Fdp^r2c47Bp!=!>4=QC|GloErs{S|5KU?SmG9w$TZ2(3+4 z@sQ*AQSB)0*9ZcT5Ii{9?Mhmj`5it&8cxCg;#0ie%(T*G9= z*`HW97zKspFW20_nU&GSx7hA&llWa62QOw8(^dKGVebVM)k-2?(;`K>`>el~sxA^l zwIdqKC2J`fVteqWRdL-@7#$b{-3Gy)X_Tu#nf)9*7PyNx8p%S?|!#ZV+ zUptS?dI~;iPrjMD-mFn21LRhb&pC(v@V>WU@x<*T-4yQLco$ngv$+BFfVe!2Q(;7K zTh8k1erbUVp47An}s4#qaRWpgL5#z*4cWvW3ItvK+Y_~IxN_8^QJIolJTI7 zn}^vZVsebKl)ML{3>zYHj#g0%MB-5?@>d;m%uXk%L2D)zG|i+oSFRCWx!w6x0K-QZ8VZ6lbz!VI*6VEJ9u9lOx8n_&?2Ld z?H#dMWH7VTsi~!azA@w+1yg0u^G3Tfz?g{QBTh_g&Np~#BfIAqTU%}V$Dq_k`i*OddQaCoU;Lb2<0?vC0#U!cFz&E8!DArSGw%6 zg>yNK_Tx~`Q$gI5tr;PicHktFb})xEhu%b9nemH+eKFIXbD2-)(1EK6qaUQ%!F>Tg z&9$e*ZCp=H)(Xdu%94J|=9*@k55anCP3Y7%F`#TazBev*+Koh9Y^iqepg z0m1ghtm)~_@S{YpUsH9~Z1g&#$HGi3n|n%gZoS^x$L6kch8Xv5Mh3rGwpE79Q}397 zX+xf&;JPI&{;`CPi`igY@!5y#ufDJk;mwxlN3T&nHm@|VoM{u>v4`uD;`?vwVG#n| zwnlaANORBX9L#?C$6O`P!V#QYlSUB$YXd_s6Kd?LeS2 zVRDh>5u#Rr$Lc+0*EQoM-MS|{5ETJY;v5xvV2<0M9f0t}!vd$%EljQJmmhg>L6J(Vb931102-+%zTv9Tr_fc6 z*_LISzk=`As!ODCNlqUPhmX)t_ z*Dr^7CZ(%qRIz{wr-s)gz$F=6VxHV+0gmV6r1CMYkU{@DfZ%FjEO-2Dl*zG|E?Dv0 zZX4g5oR4oh_BSUYkqAHU8tM#^%(02`xp0<>Ga;R6hc5AMqF@0#%boozx#f0Flsbrn!n$; z8yDGryj9Bkwr-&db-1sxR@-!zQjIE7y>S<{Z4)Lj1yCW*nfHBMz&;D|G)`SX6m{Cp zGJE(8X}$c82-90|UZf`EA5>+|9wkVh525dJYM>nsv;f~)>p4;W{Bt!wy3i$^mEFi3 zRGpK1b^7r^*bY#adkM*7A;Ljxj(F8EjrCUq> zRtr9hgYHK5i+^&Rn|)mnJvNt2p4l3lQ6Vj|*p0-wBx{y<_fK&ln&B_H)%++-t|2n2 zT~g*6B0-L`8xrl8VUGv$PrIU3367pZ7#B%xkkGcty3ZEuf%5VP4jBdwi7X8|M4Wxl zbiWsc%e`tIj~H-0mGw}q_<*fUJ*pGaI;(XRhc;-2lTa^*_;`Gv zaB=;lLc%Av@Ho3M-juDQFlQDu(sBH4I|O6g~DBhbaFQh%&>@Zqq;yUQ1LI;MF{rz zf$Jl}g$v0<+Kz0A@W3e7>U9`u(DaIHVX4_#Vb>|U4i{aCLN18b^Xjf?J}BY$&DxC| zq!)-t8%rhf!J#8gM??k+HjNpW(&2iV|zvF;*wzXc8GMagQ@x$3W#9 z8tpL`2sOnx(j?UK>VVe$sRo9g-|{X6c?%7`__ zwPHn~PB?#r3I+KCpuC60doc4BRLQUWLaoR4ge!!(CPk>-p5lIdjP%22>(T2X>DLN> zI2A!OU@kS)froFH+f4U~JWz4IQkXrJG*U@`qb7F4O|j3rn0m-hP7!}c$&$3Vn^F`B zE7^X5bDiVfA8$;j{#e`Rm_K}9d*8m{ko;g^I`Z8HD}+yK#w2OszQl&HWfPggo}Ia; zB70{h#DQy15v4R!v{wnyBj%5srtCTi&>E4ZJ@2~i(~9kHyKteuDjxB<85%gG@ntFw zIchc)`L#Hdk=mPhf0PHuB9^fuzX^WZ1-dT3Fpu#`H?m^OqTeXOQn!}T$RrvtibR-aICO=m^3=>M5B4~20Z=${qA3v`MUqGVTTss8H z=SWUWlGqB+I7K+BXFx(qRR^JYCOU68+I9!`OH~?^GG3weU2~bH>T>ho0yT1sPbGRD9t=lxIRG9?K zY1rFN`;t}EGedOC9v992d0>h~!4B;OD$-o>Lan;wOS;W-9r30t`v7;p_(c4z`f_=Q z_e)2A&%q$izB|%<<)Md-i}4_LqFg`DQhjGEcdBV9vk}H*6nsoW-RY`Xt9%&Ai&aK5Z7F4mRy9=w zHIZQ_4=XMqbEKa5WPmR_E$3$J4lQ-?k3_Hk}5fYWj?CQwIsT|VPmuc zca~Pu5BktprGTeYqT62J=)HSRKZxAD(YRLY`THHUYMm!r+_IcjTu(Uu`4l!$tn2Qn zlIf5f3@D$(nZXisoH9(ToNysbb^H04F6}($`h{j9pkb2YGhsaKfe)B)nT_6B6}C?0 zmJ+)JpA%@W_|_cXLWCf&<_6kEP=sG-`ZX7oHQ8dT*t4p~c>7V8zeDiJY<0a+R3>{{ zL!Vrrpl_ug2FW+r-noi98Cx?qI&oI<6$p0Q6WKM<`lV^hlpiYj0<2i_rfpG9FXT|R zm`Srw@$v^rQf72m6?fm$<<`dQf$j*{12EfiFZ;N)&mt9l~ zzn^;{C=M{Ha1M;EMJL%vW4Y<3?b zUXS9t9?A3I6pHsUEKdrCAbn1WmkZ}O#4mN^3umgo_@P3PJ}uAG4Elpsn3glY-^Zbp z#RFY2DwF8nVFtK`BhYC8RuG-Ziu%iare9?7|0w90}$sddYQ^4g18 zD#By>nKj1OZo=NiK%dJ;O`%4%P}v%7QUs=f{cXCdka?#!UhW7zsG_yL^_N3Q|6?`` zJ3_d0@ym~VWazU?)csae_i>nu%jh9}dSSJt`&*8TsnXX(*)P*+?0&f1tC{0~ zQePVUK~k5RuXU_Zll83ZRHbkBNeM%BFd&<&rN!jXVV0$q7U-;GL&jetbqtfngQ&|1 z)BdW!euk?zT?@m@Z1FMJn-bp?@SSChJU|2WFhgBdipFh=H=|Y?&Ms9x%P1C*-hYX3 z&5eh@Xv=sr&E1cv5F_I@h*+M1H{&B$J$-+|Q#<4Zno!soQTqzQUbUPKL1I1u6nZa5 zgRw_ZeX6Ji@1x7m_*P16m-1p*jM7MC0g@CHbYY&k5?$I;7S0u;h&cEGP9k5QTCyo(jaPPMhGD25*)MlHp-WC^pGuUcvUtV==fVzDY}XtG`UFwS!5*f41x)H`}h zq^MlQM~ehH&)&sG z7Pl<3^Lq0QOR}GJGT}m698ra!+Od@vhj;ZVaV49$DHGDlxf1nKX$kM~j5E5*hs1BP zGsI)1s5vW?m!)fZ@tGQWbP}C=S-gh}_e~KA{(uzMK-4s-v4=8tH-qt@z>bO7pXgeK zX~K7zd){_D-O7*ci3Zl)(Cq511!=1({!NKU10)s~rLnyJjkrB>5Vp3?mn1Jp(y@xP z@b-F((1SCa^}{=?eaP;K<~YzRp%t9wXg{pnLJCoE;YYimBJ9kjt5MjbVrHfB z0S<+aYrH?0--*3kRVZxggYV2n`T4%|gxY5t+m36#I42w+9adiuYZ_eGgV7ZDR4^Tl zsJDjpRbC(tZk&H(lE>5xSB2>4w`>KX^8HI>OVbvIM^E|QuunAoeI#8xT!$ts-UlAf zBex$K!zZ6lLUTp2Frs;Fxx1WT^;gwo{xQl&BqKq4azNVdget!? z2;$&i#$ci>)+8kVx~!k!lH=!Kgyfw)K4a(-56$`YRyFTIFV?%i{eUVVxGc zZg-;D4<$m1^ryjT?n?3u3%EK7Iutu+ELn&Ny;}vZzE!Y^cFKg8P0a^V(5E0QcNL@Y z%Db5G2}1v!W}3&)rl}r1!YZ@_*Zd_GRu#M_$vae)XSfR}j|;L_plSWlzKj#lrZz_+ zpLt_VRc>5Zmn$IxauJI#I4RB_UYuuk7FEW-;3}p?A)&CWhC2x(Bq{;|KePE-Jc{FB zZeTb0Oa*@qyov{3a|mdujJ&C;V86PBZGAgUO35UBHwasI{H3ZK%KlE;1fixb6L*A@ zWO+H>B_lX;WIx8v)4a|hzjoKNncRnk&Pbsor5^+sYHS~=ME_D|R=$w8iBE zI;$Ef*W5R?u`^kLlNFE-gJ4b%(z)d~I-{5Mn-=g!oM*&?;z1trynlB5YYO6>L*^9@ zqQaoe5ijPE#9J4TmKIA~=8X+pK@LC@iPDTrM{~awkN=#j!=tMKKCbt|Jfqix{d{7k!*+BEB%`gDX9V)8 z_;Yd3I0J@mIun&WQ>D}TlAxZP1IAW)uoS3(v99(pZ6G7{*JT*YP`+*@`bmAEp%SMr zYm&~uo3<1~kHe=eB;BFrYKx0!kC8-d7aKmVgyp2Gj{YcT_EMMjz1g;8huuUxk;bNw zIOb!ncXaJq`vVDbA@GcYDI38k$N1+mi%r4kZS2gG)FJ!NyCwj-Zv4D6 zmgx(_O(v`}&V(XY+X9NxDt@G2BWsvbTI3fTyQj@3&A2xNRYXZ@o_QOi$HWt~zd91v zV~Q7xxoF;;c^C7WY>{2EeanhpW5044pR`vCpaX;**hXXmoL|TkrzUT} z^D0k8dJ3AxOL3HRW+GF6yWkVs?aNiI$02;o@RTyBfiHymImrx@h?uhyv-d8Rt_U9S zd>`Fwt&x*oYyv_?4)F?smfGrY0DMenU`K;*Vcd?EbMfJA4`8G|$jD^aP_&$s=^-tqK!(hs&yXcwR3au&4D=%KB< z$kRyk>Z<7-N^`D1LQmxG4>^)C>8qbWhvm$ty(hP+@C)`xyN4#}v8h;8_$()pgN2m; zv{!esIhOKkn0?anDU@~CO3v9`OT#9X+@o3==QbIIZ(*&}OoN!Qh{zR-LWr>1642! znrfq;`*iG9PbYNFI4CeM6H6fVQ`(a|pJ1C;@c(civTcHhu89w4;Az_B{p{tq01?F8O39$0=k zw%VuwrW!;{XS~<)P8xu=YQr~2 zH{aekc~E)}*iTZ9@Xa<9XWV#>GRfyt$aDo%d7x>l zx~4RL*yiAMY~hqo$H`6dn@Uk}QI6{L@X~qogy6O^%m4d+F-Km#^SvuP(*%^eK;HIN z(#MB18k)j%WfQVn|5Z}=7NO)hmyD2%`D zBj=y+;O*{yn&Y0$=<$Bt>uTXFVmdXhF7R1!g%>s<&uFo_7rtnD8tffPD6pn;{Paqx zq~C=!y(wo>esHd_eQAc>h;0%R3LR_1O;d!$eXII-@|>?;z74AGZqQxIKk#A|X%*;+ z@#LT8`Rr^F81&;5a0d{dSLlAw%{3GdZ);*F)9j|}6%sr&8=ihEJ?4UTWh=6WV5<Uqq;3|MoYOxa{4})QvGA75I&t3=k;n%mGtZW_ir#?{_j@* z7eSTs>zz$MueMV`h5%MTslRu&fm!kbBO&CMm;weV&stDHMal;v z|L9e#S^HPHI0m@a7nrlj_dA}fnbK!p;OmkJ^8^w%TjpcP|Fx=NIuH9U5GkC*P%jy$ zM)#v?3caV`I}_-kq`S3BK4RQPJqOizseLxeYm|I{ENQix`H3<$+W9 z2&i{h`)vBtR9xxVrt5Hu!#_(ex)?KYd$kP~?U!g0X4C<@ZYztu283@b7Vt`(6C-ya zS>Hj9+{rgowH@M#O^HDjfvAwd%8#bd$jAz7`<4!;2a~FU`tfwWg1iXjiyO^?1R7iO zA+q@!?Lh9b@a&gL=&;rB2lgR0gC424dNGc}#^4;45lxG3oVI6nb3LXpLRsTm4jgEI z+VB5qB{yKjv7R4WJNYF^@I?YXkAv_G4 zHY-JrfK0P_zv7}{*+MwpyPt@|6eL91z`9sbP8O6ipn1Di$x{zMQ7!v#ajYLVL}{9Y z8EFn^0B{D>uQZ)}PjLF#))fzP6o+tvjdH9BRl-QAIi$|&J471!HJ5=fjfJ(>-JzYX z?Hm47?jl-4Da-HIMyHK7ITX6LYu^fn((5AIAI2p*a#2lH6N#}4gix@KTZq^cZL!Jl zE~8%rz&9@d#;Ut;EWR3p`lO0|bBpXFenCnkai098Qo|^NDY7?B>S1C1of{Dhox4tu z8A(z4AfovztcGM+6sk!5fiorEy?p-FilEbBz)bve-xwD(DhG<5BCl%vJ4cYyc?0PL zqx~_8tY4_XbTwY0CWAd;wCZDLt9VFGSSqe4;q+5l%@h~!e2Jd?FJc0g<8Y_qz72*TRJFME+(BruF29lKBn!xGjq{l!;t@NdHX z^auD@6xH_MtysyT+T_@D43oMmKFQfmc#jg~Ff&R=R98rMrXnxTps8$9*+YsqEq!8= zVP6?JO-mo`j@Om0pK6Dl5hn_zxKR(HFC~~Eug?Q!b145Aip&&|jJN(%{4f>k^y2TC z!^h`x+)A-u_A};|or5C#LE+lY3jc*5C?a=f+Y8L3S*EjrfzKJ`AN@}A!z z>(N2?`~4O#tPp!p3>9UYSM76Qi~CCh=Yg>@sa~?GPZgb$M82j!H(QJ4h~vo~#ikK^ zNtAQiZds?7F4375n>zq)_6g~tH3*Yk(_}l*;E(Q2UJhquUh;CnG&aWzi$RymTycE_ zN|*G-GxNr{TW@vy>G;DwA``YBm$j?H1IL0L!*Ye!CRDw5)_cTY!c)H1x1dbJI_(@@ z5kBd~Sk(5FD-yCG`Ym0OnhUn*zaAmu6E|HpR$)u9~mJ@BUH+b3W(rwYX5(+smyn zYx>H)(Q~?gYt}^rk-)$3w$o^exE3O~?Y;AWZ+#`BuY@o9-FELNYks(Q!)>nt!Jn1_ zOBSowjc|KkmxO*mz_`WY(-GyD7g)OE)tv8glths~dooH!@tt3={^o@Cey_7zZaVqz z9{s9RvuxC!(Y}j*2C*biCbaVtMC(gpYK3(BK8ek{15hBA#Xnx``&x##&y2uu(1`DB zFOA_mqV(&VOfQUa32;;rgQH=%S%X`(i@Sd9?2x*1({Xzn7iK%}FIo|X+ zPmz1l>gV&r;>c|l9U-TQAYga(Wjfm{5prpV5%bUl<3xc{X)nW);x0D0Y9?&c_D0Bw zVy_&-^L&D$mV24XwAAARKpxbeJ`*O1Xw6BJ2S3ejo5xt8T^PBwixAHOpA4NbGI`9} z!(+K$+7VX~mBsUrty&?2K4Gd;JSuM{??y4-)Vc|JSxk{4zhcT3|K}z8XOc=!&t$bX zt7&ywiNYla@vjwKS}~(C zb@L_os9uM$J|1!)``FGls^st#ai8gjLz+g5A$=K%l*@K3hUU_s6>L+kQiEjS z;)x_ziwjC5k-KjtAzWU%yt8>tuZVa#0v*1Sb0U!Ux@ED5Z&Lc5m_59Zk zpM_BZ=b|v#rTutPIkY7UBzPegHEHlb zPmlB;UL>lC$+d?GmxZr45?Bwt53B$_^OnTM$TAuU!wnadWPwJnbhVc1v@OCbt_-;o z)!{2<#zP!JP(2VP4R|-i=~Vv4pmI&Shq~VaXYJDw{k2tWkB!;f(nVuZ6_W`Q4>J`Z zVHG$nA9Q_Wa2fw)yAee3>i800rj+rCL4eI>G4G`4qieS}vKD_GQ#$g7=Bt|!}O?kDIm}wT%3nMIPJWN?$pGr-V#yAjI*GmLY6Keq6>neC{VH})^uGL zr{#$3zWSh&H-7OdHW1>F%wM}4g)J$$HlqqSEGVkL*bQS{FUNgb7cXC*(P$6{fq~b{ zhnAysxo+nWDIcPPft1$?4|ydQ*9p1Jco&n~Sm<$<3ulhr?OzqO+92bMI^T%;vqYhY zPGXz%;mtf0)a37TBu#KK7?~_>o7bp1CkMP(-beGOMUb8b7ma6e5*Lmx44%=W%{}-9Xpb>!$PVIg>vdC5f9ew zo;9VOsw`S%E`eSCL4~sQ6#WZ&CLNSdqlTG$D(c$kcMfr-LFMr&++UVzNcJ;*QCq2& z0EzMIIv6K&H{Ul3=bCM^ui%zxTY4WJc;r3jGZe?YCBL8b!DLmtP`-L)XI`7XWu-Fb zhW&Q{Pjvj+_|SN4Y9iF}je-h=Im}FxXmr7pZRsj`gB z>ggRS1v)i-Z)KMp?llM$PPM>1i#=()Nte3?{cG4Q^AB*bEby9}K* z4A<#`;-@5zsy46ZTTx``XIWoS8jV-85smBGg|0Q($Z=vQnb|JZx%Le;sZM3-ss3$J zg=F3*G&w5OZkPd&Zz7BM#(-P`dX%EecB%13B(%OGN;PUUJXL1!DukMP4)#Wp;i@0H zkM6c(Sjt9NxmhqZA>tKp>~l<>v;yZcQsLRdV~yDUx1AcXN8M}y?*UbnS!WS8lb>37 zFKbT(P2=lPS)x^OFqU?>l?9aRPn~_e(POPjvtfA4uDu?HW{(}4K_DniF0rpUD7wW~ z?LM5_$9_ATS+C{tu{hY7v(ma>ha*W>hOa!s>Zbb`io3F)qv2WsKo;vwh*##Kjd87) zISYA|O`^%SQ2(}bwPw}KJh5G1LEltk<=o`+3A9wiTcr+zF1Xf%&!oDu?RQD&L(!>e z{tg@UtC}?P8=;l2k--hrDenX%#ndV^&7y(F{iD!9`ttf0bhk39X_X%au|4_^AvkI3 z51YpwI8~A<6&#r0tSew5#(^yvv}pF0wBBV45Q!p>nIrr9*oB0qKWOhA_9^9|7=QZHrqN*!YRtP(j!EbH-~K&xH}Y>hV_aeXmXs$hX*4(mZ}f&YmUMK` z65)HdZ%jhTzpB5F4HOJULhOZwEg{r{wLQB zRqmKtc*$$gla#_pNESHP+}s5rTI{Ip{LohQtMD>74H7^gtL~q<5(}-11BVO<-TTv6 zZmog&Vhraatb|&X7QgYQ+;H7~X{2*Y|7#FG*4PTbDdN%BXy@$IE$`Emk-Ak~7WR{N;_=d9j9A)*evzBC+1n;x1(*Jh1DULxR z&Aa-2w|N9ovyqjNM~F!il?M9W2(lx&DM29EoHnb^mZT5H3hQuZorXl7juE$CT~5RN zYBb4oL$clVMCo{}#s=hL*pkRg8kkz%isSFlbDeazj+Rctyl8IRFG#DE*6osb&LzPv zs}M~7j^X)wS9(85^C&W@CrQ+zK2f={6h^N?!rFTlq}A?AmwK_NDm8qMm6yR!pA0*T z4arg1b*|4kHz)?k)JHN)mgwE2-Wj444Ep)r?(_4|dtn&2mDwW8TQ` zIYBa;AXqD7e9!lRCLJQ0xthWUdGsRYbja#)I!skA3HvbI(SzYxK-L>3meApBH0VaJ#?^Xmb=Wqr0I=* z71x|QigClbpnaFq1o1$D2_u9$;PZU6OjnACqw;m;4j~p88$<;s>Bi7d4_=*|k?9B7 zh;Vur(dxT7g;L$)FI23z252tqn7yDpbpWUz7YPF%n7?KZ7L*n{qDpP__-TW zN{b1LDo4EE!W|j;<>G<%H9pH5(QHlVXVPYNRM84x5;iGmpIoTuiGgXhV*fzqc|s8< z&(RWVvS+Gvrt~%ZyjV;nxnKR~>FTn5)Rj^^W_XGj>%JnXM%QTMGZ8RHg84cYQH6h- zMr^$$%KtZ$^mcrT%hezY01*&3;eIQwxWZJpvdyXgC8~hut-{w((#&plR#8j1H4^uG z%dsL&u-U4A^YDqRpEF|O@&XT8NT#<3U1N+wX)6+t2vayv({?D^i>Szn!7$S?zM%v| zdS$r&8dtE?pRV&2Uek2;w$P}8{sLi5yQRC=lJ^GmdRZt|t_Ptx?*6QHG0E3_%;qn* zVv`k}&2s#%zB`6hXXo5vj5ue5=p z2%aa6c(|VmQ5Y|CkbZF;-EEgDNjelo6^RbA$2{qNcq^vW~u6yFRJq=c`=0vjoMl8^$=16j5)VSY-TEG|{=eTzp4? zIR6UNCx{bWmKDzxv>~MiS zuPU}ooy4M#m4eXIqb*SFllp6bbRxv?rJp0o!On5D;xGGznl~3th4?j@uJYMnRRvVp zR}=C)h4-e`b5FL9m^aCmeE6p{_*n}_%1fR_iO_16?i%eQfMsLog%W+smpe4*up7ir z}dJKV&L5Gec_D8tZ;4QW0>N9Oe%2n`HnODp|{ImN9G8L4!4&U4z5+w3}P zvRdGpu(7)&xCGoU-NY?H<7k?}tw!uZckzj3Ml1tnElo_AQOW!knR>bWQ6GXIcChyj zR^_B9!dcd5#Q_c9fjE-v{h(Wt3Qcx8we2WwVHITz)1aiy@CD0H8(7BMql>Ed4l8Vw!^M4N;Sm6^L_ObXImnB`j;XmmYRRTZN8$c2nuF&*Kwb>z5n8)(x1a z2YX*>e#PBIa+`8_jV?C1=;k2MKTEf@9A9dY<1?I*%NjR(w_P|g7`hODerqqn4$*2# z%>!%YT!qODzAi7|HZzIg@eR%Kl?Yy*!WPt1FRu{R#*SPtRWegyNc8)<+D!%3-HD6u zWf)F4ea7WQj=15i-TYGvl<0de;Y~_#*|Xh_r|Isp<7-?QD^qH0{wEFF^QFL5Y!RBx zXG$C$EYZ1fYwGy=RzoOamv5qx|=s<29ed`;aTjgE+JO2_f#&NdJ)7vH>ujvgoV6hlQd zt+kL3Ea#0X;-&c&=;Wvbg$50UYyvzTb7YpDohYwW6=zMz$iS-65CRO1jzmy( z=jlRPkWU{TT*gH^b9&F&-d^-3hw3hvVit(R>i&c#oWGEY6I?<}aO(R^5|%~=T8#iE6x8_ef}5H+P{f7c*X41IhQ#vY;BEtMss$GWke*)iqq&For&R==q}znzbT;z znUFGU!?y8ZVi<^*qqHZ&*cjy48Nnpatv3ujnmf@q`XRo&&%(e}!l58{T*s!&vv>Hs zj23siKp%)K+kC!2*sx8@1&Q$kD^her7dbVIrCWGDkefLeW|ekIe=4TY;gww`%yaTL z`A|?7SW_&rz1zsz=ABZN=uA;YvW0}Lk>m@JPtChPgJav+o>;5}L_X?@bnMdbGAd)L zQockcjsAYyusKs4lhlO-68`Tks3=xB<^VfjO9(x6Rw$ITY)`y&q?{QcOZ?6l(8D|5 zObi1`b<)>Tg#B6+rf{Ru9O_$}$ATxfC^@)OPPXYZBiR3G!NWOA-!)-2YfHVHX=JFP zxlz1sSLFJJMG^U(6?Vu?YcXdcn5cUnJNk|Z!0emP2M1;NA?}nl!nJ8t29_#Nu|URP zf#Sz#aah-PP`WZl>VJchp9X#B<@o4j8(Xoimaf=WLztB%0jqpoqWn zZx~?$0|gtTDhN%7W;#Ib_$w45MNc3!KFg1-K~H!uXyw@|MRzzUO;B>u2!;Cgpq!|#S7>4TYq9b@;^@+*K@mInR(+UtC3Z!bThD{ zs&&aae4-YtB$e!UarP)jt7=x0(d#bk7qkAG1$Je-ovu}Q_PLOkMXf2|<@sPt>ca|! zjMw!!oO8E2So{Mn_l?o(R>E5=cpCD?YioEr;s+ZrpVQFr$rks$-jw!&%)0r+G+sK+MApwb!XNq-^%tN;CDM;0X%pv^X+x`a zsqjNNnBJpS;u*T)ny-~joYi@?jPN?4t+4&2tBc`>dz@^%Djhxm+?W^rnERLw6L4Iv zeP2U>Efia59#Aa)CX85c7%L+BLsG`*?rY(yYDX@wVbJr*S$f+6XmyNu>y?m z{po-zeVkpNJH)M0tyjDZ(_gEe8EtIPR}QCaKthRi$L=&HUVA4mo%=uebde*%zbLIf z=LC;S;84c>U%Ds2ga;vu1CnNt&u`>8T{{vBnl>SZt9L)m!3{@18I~fnoOeXNx@}m* z?b7p7`?!oTZh%O8E;98p1B~MK^d4ca3BtDXcN_Q2IA4}kv5NNI7fTy1gtEqIiWCRl z3$qo+FdI-SHE%0QphU`U*8j%xh!SI=Uy#!X?;f(|v15fCO#J6ME&c)&^Xs~*SD|HC zmbPD~?{=#`6`f^t1vJ>O9Jy^fGyB$3n>uc8Kw^(rYQA3{@|KsDbwJDDD5>wx34;P+ z(-C2yymx=95RO(+aDulH)2dRwx%h;Ykq&elj2wO#1OMJK1}(9k#>~111)X(GF>5P# zIK{fWjz&=XBZo$vj}k)S2T9= zX(^VO=qOwH?NO{G^(+#<{TnfJnM2oNe5H4bFwY5mTqPMQSW4)ZgAjA9VT;F!?g@^F z*g@DnDRNf{K{Qp9;?I<^B(vo(8ZqYiDz?BN8P%h!VNjz&CziUh!F>X@!fIy1gEfGQ zh*0RO_Ne(sX^1lz&*%kYLOe86>r?(imnZL}R3nq4l6ZY4D5TP~b4<_`BKxV0J|@X& z`4<7{VSRqO9fNszRDXaN(wx_cc=<&o;bm%&k0lBfrIAKzlucQnAhBX!q#M7^@Y71~ z!xyp}rL|9bQ(CusXZF|AU8(48<;=3qL>LTOFzA<0l`_{*_@I^M`{bXv1T_a5ujU_MO4uySj6*Oaoj+GU3dE?Q6|uOIlW3j|L+3W z1*HyCOw~Yzqm2_X_67UY)OFa!cP!dznV~C+S}_2}haCY1q5A^-Q3& za;bOF*D4W5*eRFs3bOR&KC|(5xy8nLrJenVY*_pH5ZuelCmSiI@F$c^ZAH2x<4qom zwC70$@97y&1*2K0*In7Z$rg=6!)x|Hc0k5x70%o4W4Xm~C49QZ-oujun=X3$f~|8s zqmsgAWHl=@=PhoHyZ%{HxK?|!gc@WDkQ-baXq+0lGpm(9qfP)&O)MUBDbuIg2 zPeStr61(wTLMWq2i_KeyWM6{-w6=k5v1Nec<@PvQIm8D)OFr6gO`oQ~kMV zg7h>Mep<=BKp$#2rRB>uj{RCl)*OM7qK*+B3!XeQs@&NL9S4tH6RmgVil7Lytn7@mR(wjV z@K)~!!`jL4N3&lw6I@qc?n_S35+=yFuZ=uTYrR;Fz0V55Nl~x4%A3Ld97z@F0-|T3foAr0t)puDU72$TrE+C_aq81JTp)21}^$|oQy5+x@K2BiR*aik7zC#GuvPiuM4>^C!KlveLl~c^h(j#ep5}@Bm5wKZhKM18G@=tFD@^nNv z_&loHFoq8M)zWFpKP^r%?jzXhp>Uoc3}<`bl_lyX?CcP5p+6hVtK|ywbrG+3V=AFQ&-@Jm7lX0JmlF5Evva)n^`{ZPYG zNj$dK9szQ7{%=*+%#w9WvPt)%+l_M{x`{T6A2D2Mw%`S4`yG(kx_aN==v*i5oYd&f zX~|tX>oew1QRHn>7dxuy4+|J>=a2E@eol+gO%91ADL5Tgd37KVe=JEth7Y@FZ2~(e zyFKm&&L4JCI?d+;|Ei=lk=*;k&8<)kcmD?Ni@TAw5>%+!#r4c$4MEed<}LTU5}kB* zmF92VI_0MXD;LtyA!+ZAXkaU9dLwm($xjoYSgjj4zD?Lr)b9c-)`w`sKn zYa&^Wo2dupb4b?Ynm>UvfH(v#vq0hF8)j5xxP@0jYT+3H9XQsN{YjnYC!)~8Ub5>S z#mmFm91*gv9N%0Bc&RXGI6b=Ef%WyEQ0zB&jm;6mYI-bWY@gnmmB1E={PLOUW}Mz5 z$Ha}i5X%BNSsxE{GqoS!I;2O;WXg{4);TD`BI9LtZfd(qBvhYy5gf2`NcYoO-nX;{=G?T+}Rh(tF{`yIm9xff#2ORt<`MyP4OUhRVg z#dhllyjqt3YYJTqNr)LI0hySjIf}0l{B8XKL#?U;xv)|i@M|#>2CPy_ZW0zB0Xz=j z1nsb$q3-=65U+8sy-#tGn_HlGOZQXr@ss&fsuoJ0<8u=IB_%>o=0;FaH0X&k-t4`qwFgbCz}*`9mF*sc($GorYgV`i;cKxf#}P`V#lv@2&Z& zmmNP*w{IQvqhg)D_kC%cwsYkPudF%9G^6NzfA}IET=F3}ccQbwygMIm3T;pNa7w(B z&Qim9av>F|?X|LzpuWl1IBqj;awjcpdNia9q`z{t&=SAud4+eurUrVhqP$Db*m3je z_;wzBXajf?c{IuG5iV&1juN|E2Pm7|a>qTjq&4X$ISQJ<^FWJaGW<62vI_&&teMv= zS++oD|39K^7Jp@I`pEC2eFLgDqH^P=g+pWxPe@#%KBwEEsF|?TXnxRCdq+*p5ER%j zoq^N9JOuqA(9|2UIbh@(!%P9Tghy&T&QqKm6%W-|;|zlGdZ<{&0dV(RuGqCqseB#o z#Gh_e>78UTq2~l_KAdtZ#DIemUG7a3=g{N%Rf!?c8Oj&=$33+sdZZ6y8qAq#WltGl z8=1?IU=H)Sn5NI;PF4^YY2wBSEL99^VMDesT%ztY+nL`;F3` zhP@(%U0=LF$<3Cxt=s5OJj(2H%4IOl?rlD9F)ZR0Hhd*ug&xbsMlwucqvPBJx~kymy!*qH$+Kwkl#?-t1EZ z!-qvlxM8a^^h=rwi~FkIsN?J_dgGrMgSPm19NW-CrdxdL+mz+rw32~iqOMM>{aDs` z8InaX-=O8Eljmj*@=nmX@K;Sk#&eO6e0gzJy|=0${0;(}$?e>unRA#C;`` zGNE(=(nAstZFFZ@5xjHp4}uNWgO>ZAF;X3pjdTjqbgxCx_?(I zYV4-tReT#|X`qIV0iIf=ij3(vX!~Eb9iHAfUM|3-reo2-*Ir7TJp_6ppPNNUX7!Ma zhb6}|bjJluF9X;W)?larG+2cVHd%UA_50U*yI{n861oWVM5RcibFRH2&aa=(40x;> z5hTUS6`P?r$oWkRa(-Dv=2lI)>5$12EGB^Xsk%+=iXVwKiY6w;oj}Wkl#82oBWUvJ z5>x}MqKW?zc!?w_TyZ6-oKUW*F$6(xa1%m%s;P3a%Mby$4 z5`8Wrg~f??Ht9y#SYG->qC17)|5hfC6!iH;yJyy4 zxf0l?gbzlihSwu_Z~ZlkSJ1qhIrBJ0n84O6fw4fw9y@X?@LBlKkv~@6GKQAa zI8%gMsv((4|IF4iyP7V<_MpGwkqEkQs!o+Ic65<3Rws26JeR^A{1$Y2I;~RE_AKyg z{Wd=y^LxkKP)jSmDgH|gO4G%Ims(_O&6lH16NX5HRYdDIpK`XeHUEzMa!;nj2(Mf;-`Ti0K7RXZ$kT{v`i&XV z>an|iN*>~wU*CLl^Zxm{mb%IM1SRL@EgBgbQT`@vh*rG^`$*m~3ON+&PbQ|OnL%Rg zqsHei7UY}BA2quWH<3^WLbn0dgFX0&f=G>tzEAZ^3GBR|(P3P#cX2FQi7E}rm_*D^ ztOJT_1wuRVbrg{iK(8Z0^-pb5TU`Ao7sx`4YuKeQffA-2nbhYXWTp|cuAo#AWExt@vrL1Y z5mrMsWS5(X=cs-nqrQk5?%SQax}Psj>08BzZ{b`c*Zi&?jiMHc6&85$sm`fgonXnB zYyw&cu-(?@!NJ!I$;y9uno@|xuy}hwQ&C71Pi7(iv}QI5P}v0@%(;Ag#lE$QcCic} z#u-um9EpaNw_o+oM37(f)3846p{X~nHkU;}HT@J6L)~l!^Lsf0QHRJ7r4{RVecXAg zY(_YTS}PJ7e7#C~$$rLXK=`{rS@%;;Tf$a<{1hAYKZWF8R&b?E8QV~DCC`h!_-9qj zMqx%wQr&_kIH2d06J2=J%=Bib3xh;i^1ezm)~lgIC=d#vQTXRncWix;^@SQI9W1va z_q8t+A-7YfnzA;#i}A-i5Q_6%q-Dc{?Illip*df=8^G5rfDbCZt}L1-?w3;=>*(DxU>n)EGi~R6s%)(W2TL?|Y%9CSYEb`*7%q z9Bt92k7H3MAfj=QEmqIu1q%)JH&3I6;5B8mCo0JGVByIB-t&3Q7w-pQ8Ah1mf+ccL z6O6fd=B9FG?ZljwrJY?w7ctKgg~TD2ro!1OGSigS#ym)8c)y!f^g^6WIXB8vZS)4( zEfZTdC5U6%fO$SfAFDE@bU!-Sf;a0^;Q>8CP^=_kRYQ3E?^v|xftLi_8-N1S8;)yr zk!xx;H)yW#8SuKJ9~CTs%Q05whGH0|swnGN((tRU+n16qjQiJ?5m(-+7p^Yp_>!1& zsnXqMs{sj{&wSX%sVu*ZjE4l2V+Bg#TV5o?+?{#l$g(ytKbhLYZPKWwrGk}J*&8DT zOPw*uzARA`94ko!;*HdRoGUbbcxcql*;^Xpz2UjQjB@W4ib;A$k~A{UvT2TS)6ssZ z6YEfFHih<04BUnF?N;ABeAI60GR~zft5T}$Jb!`E(^?c%shaGf3>9h835Co|HD!0X zoOxnHVEv=VXWkNg=lry0v9)OkGqJ@|``XLFh9dI4>*UZMy>Oh8-JV2_&JI0&TcUD? zw~Npg3qiyavlsS{EE$6%f+#-SGuod=dsgZK`Qq_k8Mg-rao-Jk5IC6Y4ac}vP+E7R z(7G5}ckVpIly=}!q0bhqA8CnYzjOWnq2m_*I31cCHu!POP8YMI*G?^R` zr47e!2+np%a*%zZ-oe$d(MW|ip_~%Szx?N`*<1f_wdL}Fg_3>75M`xV#eB;*xlEx* zO*p2%ybPe>5r%edw+pKEMepvkZzu7gzC{hmr0R;U*nMO{gJyn8D;I^dbUo4@)avet zB92w**RSv8Q#+`SJjfF1|KVe~iPi>{?~Rh%-K9ryowO_)=A;z>7(jJlQvR2v#9$H zG3^;Dx|y6)>IXf+#@sV91>pf;uKA(nSgNX~b*cyH1Sn!5;X4KDelJ1hq?iV&pyr=1 z`duProiZ~>uORs8bHW7a6_x}9>f(93)l2KaqPoPGjpbJO!$v1@`66+VrqQsL{8NZ;c z(i&x)|18iWu;nF8w(yW^gPN zB{ztxcvuSp+XU?FZnC%r#T0~tT-`|x!5!Oh^gxvI2p3=?Y@IqPRB>bU-1@-^PAcp* z2kl8bcV|;rrY}6SlVTFIPewL)t4qJ=EQc4@>UGlM!u?KIenP#}Ze`oGs#ms5z<+X~ z?T45tyM2thQvy57YP8Q4E`c92t>vlgt9TcO{p3J#PN`4x>{*S832xhs(Ei-3!HK}= z%G7(Jl3k$BC$|9=CI*>LY$W^KXYmz^Qz&MlEWumw+qE(ygq0Ymik-7p?)IoZdg!&E z>E|DI_CvKUnGu<~%IXQopc$-=6`FEf-pZoXNNNwA9zL|PE_&TpwFqo8qp&xtHKBn$ zyMS_^m#anf{f93CbwZ;7XV)+)u@UQuhY|X~F|i1{LhJ;gud4Ps5?z znW&iRZ-6b1-&?^n=A6UBO?P+n$YpGU-Z7KDU;J_^6yFXQw3ons6>f{a%GUU;Gl^N$ z7l-;y7Ia4nR(mxgPSZFl&B;6zi*^tH^-QTLg0|nN46}JbxanBF54lR${%((L9;{JN zPbR3LJ&kURtWQgNtbyh=-AE#%OB?ifYuh%}`qO%7?rN6m$xZQleF#G9!VdzCAwM=R z>_7%zCpl~V>62j&4mn>k#6BA!*CrdtGS-`-JF^C}6CLx-9bm5ijp7%6EDUyn5~ZFU z`e8NW)3<8$M>+_C9)1i|Mv*=kEY}DmN~@Xkw6kVtUjjAB7kTjh7C>dT=f03NcV7R_ zaIT$E2aMs6y6=5t5fdqKk;i@R0!q{d%jVIckmz=o(Wj=BGo-a{S@7h|6q^6K-j<)a zY)wz?V|S9|{S3n)Aszo-M+!kr=k)IEgP-)Ym14RF$s`2)7&8)@Z?4j+)9-urcc7Dw zAy0h!fN)jhYh-VyC#Rn z!o!$dgpJH{jXiY4J3a!&IV)gt24yPss`8X&OevTBLRe9cZ`3w7xI0-IArn_W8 z4MWSqqL$b?5CcNC&~l{`>8+A%eKu!>M=Ph#Wr-E6SDy|Squ8WR`&HB#aoE~j- z24m~O&n%yI(WP1S^DlW=Ae3jWk4n|x6OFNE7~7j2nRT~ee1r8fU2`?g&1%u|r>t>X zRp^|PCkW@S{E=>XzBuSV$6`?;QbYN^t2AR?N+sLEnK|cL`Y7-)LooRXd@xV~VXBMHuFi_<;b3S5x#n_*s`5Wtt&rml) z>%Fves6G1dXlm$#{wrIFWW^>5=<7ldR``l;{^X!j zbGU2kAhMPpfEV#ySQ@vfP)s{Yj-#a9;RoSD$gk*mO|x!8qX70yNH3p7;NDdY`3qln z99k>w8@0zsFY#BkLr_of9_ryKIn6dw9MU-cB8%&N!H3D9MM%rPVUwfMU9cC z{zkSK!uP4WxRR@}XrX!^*EzxnbcJGLrsiK(gtpai#ALFPN^aCNTf0nDy7_98-xd=x zN^;|HzsrD{0Xpxdb1m+1=!0i-*o_ya;$MewH^u@u|LI~dM@y#6Js8h|c7z@?_Q{fp zdG~qyA^R$-#}dZa4r36TjW-sjS|JUh0X$=niLaY=q3tHh zWPenT&RqG^;AtZbN!}D9GxtmB?bwW?C(|S^C01oHsJ7b%(Hwt;962>6-i*;b6mn&%AK76oSTRT@2Y5)UGC1v99+; z%D!;s6Fxj7?{i@4lMIl8XP~H(Y}Ct-tivbrvARtUBV;f^=843IUWgNio|X*Gn8k^G zPD*VADSTl{R5dDNf9mNoaiO)60AT~jrlQyf7SlzQ3o&ulLmR&;OXTDl8E42O zN9utBQ|^;^)VNn35fQ(m(|+iF?q9?Ic6;0PX>GGxjzHRZ5$jRb415OdO+%QKFVF4t zWu9A2J|fm%%gPOaVrY6Fji2N=w&b2B%e+?qwA7=&<+iE|TaqfhB-L zy4pet;aoWckTeBkn>1bBt7XAlspy9e`LiHu%efog6YK5~d|K3>oa6y98;PrW9z?<^ zqvMqOduxs;!U;Q>(z$f$0bAB2R_uKz$|~Dqa%I00SCRtBI3T-2-h39jt!rzx2f2f z1;tFVL)tL^(#$qmOXYp3l^;H)XdG@cDh_%Y%22IODnh3Y2+79?6yki~uNP~DMiEFq z$|{dWY^s>>#zfVm&&|6@OviII^fs_8kmG2N>=!khI6;?CQTEwTb?tWw?GHTNXs_wk z>I^;!VVq0CtejWj?+uqwU4KfbFPugU?=Qo5EI^&1)tl!cMG&4%Ugtc-!yD!*KgIl@ zzp3t|jvpvr?GNxzTRS2jblI#j(;Ugjw`2rc$RTDqcE{ru1woq6f=ZU88rc;uU8=l= zTldc8PG(C;V|1WUU!X>jd~N}27C|y{sv1ycS@lmAUf9{6?U65HqM=%m*Ulm!I@JZ$ z?7>$F5i2ywZ2%();&ng3#?5n%Y=}kzV271=C;ocF%O1GV)tG&Z{y((I@`c_I;Z(|P zHIsbg;8>C{Im$R9Lg^e+cWgL z(Jc+%imQlY7&k?V;8Ws;42bzsIT8!CX;1E<$6eBQ(PB6n`EeUjFJcYqDk-XH9V^qn zuK&$TvvUKF6|V&6^!jfcyjZ>AUj@ZGLiVnivgdlBhZUoqP43~NSSqHAU*TJWT`~?E zAJleubF6H_bh4gN!85pijc%%B;zaO5KQ*Ht1z(60r|6d;L9}R^(%KU!+2ezqG!Nj2 z=OO=}7ya^_EpSC|(0S-95dm72JGlC}a z`*m#tyKJ!Y^&OLG*g)5I(bGF9!|-)^;D^}NC=lekAwI}D8Ef`CJ9d?pAYk<&_21AB zb6U{w)f0zvcy=YB4-Cxs=1^Ja<4i%!iv3Ajen~5D8^?o;52jIxI|hKxSJ7yK{fGEO zhIN?1>9!1PA``cll289GhHWdFxN{W-%%)$z^|T}h^CV-9ir(pPeNaTXbo746G{1>A zQZ{~h^oI&OLn5B*(n~QbXAYBFJ)ejcsO!m0{qsVuhVVY=vdT%X<1u;@`zwcTxW!AP zr$b$A7HOkwvHbeSphn-g!d!w6w;9WC9UPGc7@}EkCPPl{y{BEJ$%a!ipBv2~kcSJ1syEd^x9BaI&z0N%J}(TCg9BS?O&=5M}>l#94((MqHBt(C}z2Unb4iRGEO zFkZNNeTdrhgD!yNPYd~=Pu)7fqC-3?UIqqIp_-RCD%dGJrFqG7U)M**{Tg9D>Ygy~ zw-Dm+i>_(ju%NXMZ=f09Q}_lLup$z8ctt%&MpU?2X;&hwr>9lNBcsN76=XIi^$X({ zn_f2&3{vU1ylJ5qZfgCPSgI8XJJ2Y*{bovlf(KY-9dJ694jCHi1M3jAHbEHn})KcKvk-NzgVni`=F3d=@CK5^-v7U;@Z%S`wJQPzM zESAx;cUkJQcJDw>hn3@be`i^5tlouR%^Q#bM-EX8b(JJx?SaEp>e&nD3eFXPN6U-0p*uKOpyA+&n@63oPJCE zi(IxT++R?2!y}rrh#E++Ii_7eZJ^2bxR6+vSi|HwUa|9EiXhS1cB09Ky{l7>4gH+Kw8f(JqKANivh3Gv z;$7P9*B5a_8KK9>tApos=3p#lEOoxrqg-CYe4bSU*u`G*2f~tk*vQb`@yVQ!FrpHTmxD;w68o! zs{}jMlaF>S0XM{Y5T8jj*>CdhHoCd*i0|D&KxQ}ldM%4+{$;|Awmfb+-kj|c4=^q| z^^We*dKO3G9iSrN5}Eqm^4dMKP`0Bo>DAUFKVyK4N}K`iFquy;hq!TqbGpYvIyJX= z@C$2hk=2T*Yke=|yna2XXS{_7n?0vdP1e0_@r|;O-8&aLKY^K;U;PN>%%uv)_J~!8 zQLdjkL$vs3*4b#==1+*gP~<$2?}Xnp@`iIrm#QDVO~y$q&IM~Ko89N8%k!*A?;xph zC=95icUc}a7OB3y#Nb&bt*aAq+J?TfrYSW$FaZ(KVPhBx^K$c%R9!Grq~``{9K+K+e{}Pw--`IPhNi80Rbb+X1J2gpG$u@ehrDf zfdH}M6XVk(^i%`ZDjL>|d!w)RR4-l=_tP{d*Dux$Gp$f$608q={?XU<#7d8S#CRmV z$GzNZ_H3hT#ukt1Y}G${uYLdQXIeX-IIDq3c9HgeRG1UPyEqEk>xZx z@)U(ioh3c_MZRtkWSZPvaC_M{G%x-xHYZKoi6!33FC~;@nE@rg6rj966&8G0`*RG9 zAY?NESxikYB8sP5k;vepGbZmo`WI!wm|GCNi#wG;-*&i+r~?1B+(Gc@5LVd|HSH6n zdS5LimiRxFv2FHdfs(H)g zqKoy=3_ygl;SD4x0$y+UcVNX3%k2fFhAC*=Ze z*4|fZKgLX46ok>MHS_XiTUR$RD>`OGYYGQ8+o=(a)T+n^=Q+`(?m(XS%q-mLFbh=Z zr*i%gE7+o-Iz?>p-IS>MAvEu!W^G9OAcp!Fto&XQ2VyW8PVC)8wnYGQlz@2_7#3_` zYq^Z%(0RfhKke0?FKQuI+E$)QVG_Xk}KvQK|>zivRV;kmhldz(_<7L1- zxRpS*=nHWS0>FSzp9b$Bx7R=756~mm562$o)Dx{Ja$fjt@Rbpz8PgsMk;v=w><}Dk z_e#9`a_|>Yd~#$~n^b5hY3CE%PXttV5G~jKF3F>8w*(3U`VrsD{RV1U9~10Typ-Z!NpL&05BVZ+TBZE~oJ2Gb~Lhu$p@idl2^pGsm$q{xlCu z3I|XPMtQM}zpIF@nwvnpM-Q0rBYNbnb>G@eG6UT#a*d#tWl#`D@=Gp}k0r>#=$sJnCunh-qdq{bY&^!>I4a-XZ` z+Rt8j;Jy50vF9OdAbv|r5cZ@sS_hSW%MXkzU-vZr4nnZxxBUu4+mnx=Mi#dV{O6%9E{<$bzof+Bg|GbJqR5t_ z%gNSN?2{|`ABA>0qg!sk!E4lYdL|OhPt$|aY-F=^(q}2!6$=}+QD9>2>?OJmb4kX= zo`d8UI_obMMf3k>&q6gzuf&=SgKm5;w6YR*my(nGg`0w?AvK#0P~un$fuIq5P=&ao zA8pdJ>Po^ak`V?yAsQepBrXB!``6M@09Q#;9P2r#h`zyem?G(+5ocBbypu9WI#^zZ zG6wt!RG5k>yI4Kbtm~djnChj&t|zs#<<&QPq|qGiMDV{eG4RU6}wj21fX6v^+OEYIavpl=l~*ejFKKs%QY_vWR&=|V`72t z<*)sccrm7OO@A{?L2oqQH}Qz}n|CfgsXGK>OFdq?CYcCt__%mpo>&cITI&#{iUwA& z5j+u-;Hq8vjX4V~JFs7*Rl<#?@7mDWsLBX7mr=LitOklf23H>zZpUHT-TqIJEJ)Ot zxaF1RM|chtpvDgrHO5czqU1QCtDmxj)^K)gMku^7c_^c|4oyDcCcM;5co1vCqJ$kJ zx|3-SOnT|RnR*Nelg0`Cu;%fE5cTt%K{&AqG;DuU_q$OwliWB`fT7AZ+2AtshmOAn zt8d&$CxkZk=S^#O^x@0t^a+cy-|YlTy6!uKX+vGL*3^{L80NcqB#<`OY@Je{zo~#g zczj2lRL-n5@zI_t2;%p-f8jF`QTCh%=CPS?%zO3Ry`qJyeQ2r{>E+(w3Tpuv*9l)%%Up>L*K!B73;$DW)NGjORv?HPP0A?enS_OiK-rL-1;n7(BhdsG(FiRGI~~NnMPd#Bl4@$Z0*fQp%dDgN6Mt zR}{bHm*p8oT3a{n8}r?*Ze1I$HnXaMUiNRB1>w}v3QE9wGnDNsE-m&lLDs5d@NR0J zos8LQ(*twcJz(OVGM#2JwP9Q0ahr>I$A&ZsDqWQCRueo=MoX~JmZEyU#(ZjYNw8rF z{kE^ewy;!jB8}kLVU9E}86l@tCdnTbgW+pjU<6aI<}sp(mNe(H9+fWRL{>yhn;yi? zt!yi5A;Bb$TK0m78Qn6ksr7*XdyD%DaF}<~4ZovBcVF`Va>nsSIP>n8UbuRiC2niR zl-A2WhI_JU)3|$P_uhj_Lt0a-^pFcHaN@xcgPI6BkAFC?)<>KO$)AuVn`7I>eb$pyQDz8A zfgZ7TG8!08Qg!KVl1z-!1Uazwu0UNxbP~Sia(ZEf>Ojy@ zpIxvZ@--inbTU7mz}0}8V38fu>a5(5svQA3F%i|abe!6*v4n9PLkLQPN>)rmGLAu4 zm34l>oRQOJ7O&$hdubxz9UX2_$<5R%R)*PbONg%N@J8kfClf0cGI5QLO4@;AAVhX45Lku@l#{l?sKVMlk%}D+QP6Fi^8qQ3TC~H1NHi zUN+bBr*J$+R<)91WhvY+qbt29>g#FMbxj1?`we!OYV&zoD<$6;-I<=V$V0sT@;JBJ zB0G;hj<6F+2+ptx6rViQJELUKV$`{QmeKk~8g$mA?YVk8$Qf^5diJG*-Pdph6;sk# zuhYeadKe)k<wFi?gaLwv zj=?Le8b0>ozl#@(Ra^lLz8lnbY|x9C3P|Ol!LDKwccWiGZSog7#a3W^e%$d{4E8|rmnmiy{mpX>dK zsuFK0L1RaL^RSZhvRnTD$++xJ+qbB?6rN{h`#2s=N#vJj>!g5t>B zBZqb@Zuyi2^0~a3FeEKb%+dT-%rckTeieFK8xi&w`&$on64u~HW+w)Q z9yK4@Lab;m7j&^$Jt@VM95a~@UeT>x5KNo9Xb0mJRy_uS2koyu?xW2}FImx61d@yY zsU{boxD>6lr+xl7-e}>OmfjoWh4aB>la@1`Lg$E{flZz*Hm2|kURQ3bYi*;g%!*tXkwr3G<-gggtkD71Y^WXV{W= zW^Njs6o@d^aF%+ms4m{DGoGJyCrWjV#wK0jxtqngaCO_Iz3JklM zskoQ(8Oga$$$iz+Eg<@se<;wRp<{Jyk~D<@wwH8+g3SueKJx>g$XiXfZ~AULXoU(K zY}GBdLnJ()A3sGF@<@X?6yidW?QSze4L+M)hv&~gYiuf>^|{BN3SohrDyh>fzXT7B z&!Z43u*RR>7j3GMgY|Vomy3Z<-npfcx6@h59aqeg)`6Yu%JWqEq&&!B(+>S*E;T|~ zEO!Y9BlW^ia!9!YK1aqmam~;8MN$6s(JqW#*Iy9f_aC#f%Ch)uJZRBW@4*shkF(~D zoiPWcA{w6TDtc>=BC#d13!#4rO!mfPd*|QTd>Nx755Cmyae)tz7%pKX#nkyG+V*0m zbacvRy3V=A|MgQ0U$z%n7!>>I`{*}7-o8F+dbcR8P8dWughevHxu4b``CVl#ysilF za-B#;Yf5jd=w9Dc%odgqaj)J5i422qPaFI3Mki=VA36IQZ5bTxx{#LXygG2US7m%1 z6LnlHpTn-J-RMJM=S_qgjUuzfh{QbwoV%#!89b5A6QoYBKXO1(4=&2pZm*@-Q(D4b z?fuPUe+5~2oZRlifEN1O_ZTMHlH;<@to-J|5MsMsH)ylZUpfC?-BLOgHsv&ArQf?n zx%F5Te|kFN)=XxsE`eCm@MfU^n;>L(W`(dXV&<+IB>x%2e%LV8vd6WcnA8dxeUT*N z3X)6~X!U0(XOTSR9JyFlHsH+Cu|BJ$s!~Vz*l9B2r>_?Yq_t2>h3UCxe`8%RURBTA z6#pcVTjl9JTW8>oCNd=`u79WS+=7tObEd6O;}-4y>y}}I1G*ALOYjhN0G@bV zub7Qe39|wk=bhGmQ!=Z%gddR<9?wm8C>N%Ckn{{!KwEy$_Ny^cfWMnCuiloxxJB|v zOhlzk8?4}#{@u5DK~@BGwh$shSJiCdXo>=}aV^G{Pq%h+Q1}CNWrVuF&d-^3X-6=fhAb^=c@YJ{a%XJ)|7qMmKA2>2)}a@T>BBx zwJ-M4_0){Y%vaW;5MS#yV@vw4_-#Y_6}js$?wvHv|2+&XOJrGnnA2VLoIJV{H8Z2c z$|O_`Hj`NSTb%@+9CBP4v&BAqHN)qMXeHaq!f$NE?l{V6G^yy6H_rzKecF9{Vhy>b z>a8(!1)8Qgc~a9=mJ7(Q1a}Q+O-({P(qQ8E7YdyBut5OkHz&$IMG^Gdar0V>S!bhm zEACzHaBqvqG_ScIMLyNBOjeDTj(ZjUJ+_N+Ce4e?9_N{DAirJf8p5kRUonwpX9#vmP`Uw!ZgCP1;Z0& zdclNEqZZ9L4@bYTeE)yOMt>-McRqWB!{|^CGr=?dODtU3mEO}wPUDvxAHYm(g&eeE zE#DEz<^qiAl%igQ%dnPT4W&OM@nhi@{waoSxcmFkuvc;+n^XV;oaey23P2o~}UqL6F zHODA3)A*}8mZUN6snD7D3ZavDcg}bSzu)6|2IZ(<(qW;ah3=x@#m}R6gI+#Msd6KN zuZ#<*mRUma9Gt*^(Au*&XubTvrk?3`ZdZ9~U6a2yzc$))U&wDWK85kBZiCkGH&M)) z-y>3(zFX&WE`j_r9M4Lzme0ftb!1nZk7K@D@3UyyWiPZw?n-mBNQwb3x6YoxCwF7z zGB0E1&+0gM&O;XCb1jhtQi?o@F`!EXOxk;87^0q zSxLnfX56#O5cRvAenR;9`p-MCglP4#Q0KQU3LoLyfWZ*%g_($Aj!b(kO3DMO)=H-J zuC<}~g*y+&_tQ*)o5&k_aAR3CutI|FEW0dAXH{vh zbNP>h42z{L{CKZiQn`_U=c&}84rQ%(sE5y+gSw5-1mL?1sXyDjDOyT`NFvkb^sEHY zTbEf<0ew2q1Vs-r@J#W~q#$(hEPq1p2qTs&5a%V$Y_t9#v1%j-igzvbQ(QWj!Ip+x6ih*;&hFoex-7MZ$ypbi1lxgCGVr z_-UK+TYi1SpoA9*E~K-5Z885|PLb5eJoP(`Y@7a&k^1&+^6ux*J*G7CR zrr{_hZ(X3bi&E%LS{pw%dnN8y%r@t0C4$a}7rELTi?!$GJLCR0$Sa64|v{)4&ox)Q*>R&zyR zsQLv(9vPQf&|P7;CYzS#s~Xh|M!;ecv-sa+Q6%cAE2<61dcTkD@Yf8Oa-J|kAich{ z-XeEdLud)3>x;tnTfPNV2r~i*H_l)-20j*P#S50i{#y>b(mlConK)4K3E*j=!e~t~7inN8?d3x_t|`Ms6N_o`&*@8Y zTxeW%5dQz7*}-n;4#tDw?w!YA`CvWKr1tgAP~k$0_tDI%(7u18mymK4G|yoyZBfW0 zW@6u$J_@nOQ!>)XY9nd?o5f4jqWZ?xD;I#YopnV9wrzxAXJ2&!a#`WS+l<0DSf+_2 zkPBXAemoAW>NvS0DVxpugMcJhhv{h%ig`cVL&<&25bheH#eFFs^Fjt900_xjG)C4< z9o==754?`Z9ZCaTq!JH`mvhqPdtV9m;sSpT@=iVn(UAN>qoIZ2>bfnutXP!}Tgb1} zMV6Y`fRFfLr%0ZJS2uKz#}GCfALcPGf2T*ZC$WwvT~SoZA#7$$Y_9XDnscPT4hc#P zzmUMGi-D?6*5Yy7st<3>Sqp6b^T@sH#^Y2t*JtzpVSS~*HRFIi_8%rotDF=_$5L3( zL#**x{9$HTn@7&vn$#=~B{V^Sa*p8mf=RfHVi_uQLMY3+X0f4c{@l4A{`Kb>M85Ym zNq9}81ZwygM5+IU6NBmn1*g(4OmIm>TYF>FFuimD=p^k`)oJB|A5CZCFXEozhuIO^ zO18lEv=~Vj`qp1o7akq&#jX&FC*3cI(vwxoPi@S>r`7g>QV+Fe*>y=_`U?UQ)Bb*s5&OtC;Y^B^9)2fK%CUZW74ql-Vvmr zc#U})sY`Lj3TDvi*u8!ue;m=}P-sq~%BNG3vEDb$?1UiQGkS?>amvF8;TW~&%+6GV z9bFJQB=vtIuQxMB5yLW13E_zbhcGRQseKl{j|+Md%AD1wTQat|dEju+oAJ(hhE>9C z7mVBlB{vY1>wBAm zljMj7PAZqs1?rv_&PKdZ zagk6JC1FqJ%c?gDL%YGDpVN-N54kKv8+kNw zG>G`QFvMo)#ZYl9`!lJGi@tbE`7FxYlnB9RZw`1GD5%y}xsST+zUA0nbS&1DyE9d*bXKTZE{5byttUCQu>I`Q z)YYi{mLBqOxCGW%7u#Nm`sQAff;jonnDnRPT#|0IM8JJZB}ed$#TANZg#`%T@pXnF z7Co$~9@(p_=%3O2@ay-rB_%W;BTHxP7F{#|)CDe+PnA{y5%2H0bdFcmQm9C(SHXY2 z7?3dAhhCV~la$MMP57wOcZi`M>^lss{`1e+x3_DmIMw2x8m>r}oQiK!nTVr_xeO)! z2IQhOCFwW=pGux_>xUYB5JH zzS})fI|@J4Q3ba+08D#L!YvlQJk>mV2-Jz%W)f>mKvx^8O215-fmdBfjQSgELi1(Z zC1dxY3h)dE+t2Xg`6vD7WjbAm2kWIq_;7H*2#Bvk7cRu~GnOy4Y52aUR!6UsCnpCB zJy)nV)sN$@(@kc_6ydzKCEB~Ap%DIh-i+l3`p&kZzqJ1$!#zQ-1n1m4z(zM;kb5W^ zJ@3!mG;QpvXcIXZ0&f}YIatZPIXqrW5IsghI{PQd9GpU`DAxzCg;&*|*#A$fc$x6+ znFiTUIB#57?9++PiR%@7)yD3FN|5wYgIL2)tFV{3JVq4XQ_~bi;n(EiRZWOyFmtFZVU7>A6`KZwxW^b!SZ_&QP z`FPE%K6ab#WS(W}z2>tIG*s`ev(zF|p#J3JN7U3(;DlvRN)juLsPTO+G)ENwy~;1D ze&TBzU_VmWiR6$l<5w^QG$BLSvch#V#&$Md9a~IL_2`>?nC3n}{bcF%H;M==(@Uo5 z;ruTqondymO<}seUOX?pvurb>XVU7M5oFki&RNf>F2#s-fMMs_qcS2;i{qxBR6SE= zJft;pdu{I6l|iGtQpj9TZaS&2pVB$7q#|AB=hjFw#+(w@*p{upbIY9 zggt~iOl8C-mXlfcwbytA`a=8jl+lcI8K~&?W+hl6_vKddK4)6tR@DP4sOTX0rk9Ld zqk1hmWlnP!Rz_tn9OXMY{YG_B^~AbTg0#Kb0rf!VhxLKqmlx98TF5uF-c6|$DI6NH zZ&I6>31s&)N6&G$va1bvj$<%J*ugyYyutiOj1f8Ajz98hN4ue5$KbIa;y&#(6|A8-+lBATPNPZU zhS=-w9j*>y76p4V>icL=HkCiwp6I)s1Q?^QTvas|rqY+YX#?VSTinxZhsI?C^}YWa zX=TwdCgW|Ok?OGG0xxGyD(>r!O()#Ob@ z?7%a}GQG<2Vq_Ds*+VWWOjx-$1o{79$mcdc=$Zwi^{$_hu5ttwDYPz7#Cd$|b}$;Y zO$blcE>B1fAPgTKm$p=U9TS90y#%-g@Lo6Mr^(7J9VN%%r}Li^Fb3sb?hiG2-OenN z!UzyurO-98@ydmF#MnJ?zn(}FO+$nTuNMC^i|DBU-l6lwvo;U-PBW2meUT_Xe!2B) z!fs=tv1{kL$dtmuEHZp|@ro1ONX``nr%{>jjCo<#rSeK%P%2x`5(!EGxJu;+1L_)nE;XywCO1+pEtNUl)^r`9^2VFnrz$PZmyn1iU~zq6$)!IT3vpja zPKlD!xBE%Hsg&@EHxxc@p8Tim*6n%cmQWqaqBlJbp(l-_&3kXKTK&n|I-q#&mMYLj|856iZ(y;pO}mZAjKpm&gzsrM>A|13p*%XL3!ip&fKXlH~&j zEhl@InM^ml=>sO*^?neFVJhx7`%!qOJDMG5AUN0M^0WT@17^ba%DG7tCh*wl(`Kk; zMT@qy(tqElS@KO>tr)I5%@Il+un|1zkowJ zzjQ=Vd-F|L1Gb0k}H9&E+*w#96Q&d98zpGBG&YuPq%g{0rs%C99oMf^%umDrzgXKt6~;ngTZ9>U@{Z!Rr=1GDr|w~!V`#$$SY<(KeVhe+5Z z1lXSTRZ+|6K(^$)vz+B+cnvm=TDHa$ksaRNy7K*2g4lsIrT$nfIxVD#mnVOHp9_yW z0W;8{yqXbLr0{bkM>((w8qC%S-~9W!FVo*gb(PMJ>m>Z0l$Z9s8s3sq3-7oft@vA3 znC;S}dlqU05Nu?-?xz`zzBNS(j!Edju2GABmr<_=l;2`_GgbY|hEvIOdnU{1%B5v1|Kzid_nAnGR6H~3Z^|MLFo-P!egU}(I%P2hjhFHXRr^}Be zD^YswLfvhi!-8!uhXvFg3FqpA6W-0_ zdltDf`6#JXNR~qXdMZF4lRMt2v61NPp$1Q>T^$yK5o)?;`L+4!xfn=wZqR2Bs+Woj z8sivDknl1ItbDi~ER8*IT2L6#dx$mx8NZ2B`ss;sWM8UZQ0OHrL6zOdI}R|*~{FBgk# z7LrI80rhthBkMzFl(*#~!%hSc!^u~A#%A?9wH-d=8QeB5htLeNk9MvqXbr^J{wSzL zEUEM_L2Ad-<54We%8Y_5Y!uY`fn>}fa@$G-?d&!}wI9#1&AlF&1A5^D)&J*MLAWs; zzh#L&#PU+%>?&$2vGZ%gp`TWEPq!kV?K*qu~6P%%|nblNTw8FHsb{?&*-9ojbb zfW?3it=r?NEY1@9v8GKL?1rlX7qw+=OWWw+;l=&%A~o|y-vkWOfIX@JpH8;3Oi_fl z4Ng1Hcdyn~rHF~dIv=WrZ%vtZo1=^@RsNt&1cN`=d4OVYpedxerhsr>F&5@F@%q+a z6pcwHw+ldf(FNeTHViti37eH9-;%P}%}+fg z$%)JSV7v~2&&6*7a#sv%>DS8CAs5jUO}XhgjgfpLOP*}FZ&o@$iIE0ItF|30_D)ZmzN)?+@&HiTFkGD zZ@D~kBbu7HQYSK>g5szHzVAc`HE{@HCG`ec%DR+FX}~(i519o;qLo7AFC#jf=P7dY zhyN0g=>drS?kazZU5%moz7K@;#~JW$TKlz3V{O15<=Gn+9aD|r%y0Jw%L9u_l$eYb zb5tJg9z)qm+qGmkPgt-fUFa2o!%PhtX7>dh&27))&SVZXXzx&f^?$4T<>imSFhpan zoh_6wYR0NVeX9We8EfUXQJC8dtsrbm9$UN~^($+MH^WikSK{roH`C2HQ7xC1xI;;W zXk>Og15jh83n>G)&va?55?Da|#h;ag!SZjYEZK8VklxSE7QyC7*;tBg;wIi-3)s@~ z+%#DZ958gCN4xV$2rLe+8eS3l-nx0A&u$V1S;eKRKDtQQp+$S8@?m^U9PL%v?2U#U zMPX^_32)7w!Ib;PJlBV&YS}z|~KD0f~u^N>6SFr84K9Qe;iQZ1@i)UL({^%yILLA*)QSEl<8X6BoeQ--1Sy2lb zM|J8jY&oAJ*|z<@7G8C&j^2xba#hPSiHH1Z4j=fkuwzenzC7R8RBUtJ_tbH=2Whu> zUS$Zq@|?VSeYH?+Cgoye=PvEGLgC zRH};kJc`)}1vkyk#5Usjk{wIiZpJ!&rJOJ8ZS<7acSe`^HyDr?`4h{>)avs>mj-(J z$<0+LW5cu&Cd}KuRixuIz3M3F{*+8&vc}fJJd|f9Gp+pjR*9UJ4`=&Avacvf!cesQh_aedc;|fkD~+0>t#cfw$@{)3^iB zAQi686twkYL!)ViN}yQdGHBr<3k_L-H4a0{U-)ptYK;82LHNKzUDd-|8yRq=v3gvJKXuy5!Ur5^n@p^V2} z4vzhr5^6bM?89?bTeI~#2e8MNZJDL_<5*^HlkGsiy6c8Pf_tjF$Z)JSoMrJb0P>I58jL)wDMQ5Zb7MFS(aC=ea|^z`#wW=!Wv-ghr<-7K4drw-@~I zR7+@Oj2kax>g_~6+-P_K&WMKbvsI!|Woh>i``LKH4kHsw!=|hE|DwnDxIfAu>E5fI zoVP>doS3LNvd6+K<^ZIcixd)D_gp^3YVW`>hc)qBP4>&Xlg$usdymy;yY?9{pFMq?Uu10K{34aivnbcD?Jt_;s~_`DDI^;kMM;mI5M^h28UUfkn`x%AZUyIGuL`2gO#AdD8u=hurzREr`vKO{V{jpvbH#P&K$ z2~=*K8-)37$8KrOKt9h!SV)z07&?315?K|uHpFitz1)AKX7rHA@PD0IB+ncLIyUrG zb&MkKv_&c%0Hkg^eY2&6+pk5s6;v(mqhMK1pOMPYmztdcB?!{pK)>lFF`M3s8Q)yz zs$iRpF5-Hg((tP3kqY?XSRsERK27>eynbE!EG7+wEj1TU_sh6I_6w1flIuZR4)Wx% zp(t^LuXa_HOrH5y9Z)2)esYd2M)?1@K~csLZKFx#D@NSA;p#PKIO(T_;?<1nWP|R! z$}@dBukzN?MuFKKOyuiDk8>Ab=tz8teG7*Xh$znVi4q@jzFPb_t%F24kW=&6(rBud zZ6Z#23ztJf+t&~hMOh&B z2lX~5<>#_SnJ5^`H4f|XzC^8d%_i(w+@0wUX_T(G&ur{=EghtRW<55K;<#I<2b-*!*+3=Da6|tZ zpF>`YWNoPrI`7h{Moyr1LwCA`j8-(T)jNZb#>TCF8dOeVP#~f%BOc6ReR)mN^Z(cd ziPPxG?iWUf+(uGUc7uw(xQ}3E0^`u&8grku#H%;;yliEovtdrXje)v4W|4WbbEi(| zCMG0=;fYM~L7}_zv3@vMkP6?W9+HqHBY(?Cx*ByjmzrdKRnkvwN-Wd2so1Xsu8PE7 z7tZxWO@gd(?NXt{+qUAXJgfTAb=XHLZ|f9>lM)57!iRPW?9&GcmWdby)#$&oof>v5 z@K;Qkou@tg+v%nRue~aIsgT?I&chQHwHGh4lT6c?fv-R5w7Axy=!#Iu!_4UMY>So6OX!`f{!}7$&kXUQySfO zv-&YOV(H_fAq^WXJMA~K#5$2PZQ|?;gF9~!ktW||9NIL%-UZm57;~3ewU+~Ixsi?& zdkK4qRJTH7qIaIQ6{3$QH(0a+`zwgIDA*-{R^-kNWc-}m77Ns457-U&BUg1 zx%^Gs4$=%|-;RoT5=fi{ogTBNR|Y0Lq3Ka_J;f5ZA{nh;H zT{~m~SrppAxy3Sq+na8>=~++;q1uGyR|j*+i(nBpgi_0)^Z?@&>6dpZZAt!C*RRj~ zJon^AWAz+amo5S`@mP*A6*0N7_qafxlgnb3-d}RkH)BU*q@ARtExzKc=#?G5w*^k* zd}kJm?BJZiyy8}432>z2g`ruTeSA5h&f#&0uB7|iy1)}C>as^$El#Wa>S-1qVh9g~Yrv7dD z?{X=PM_jqqW-L$4Uzr6-m}U^6>d4`n7aD#54#2NOT=T$vJ4mnP?qDdqxf%8 zm;HB3)19j`|5jgg3QE>5hzxt>_%WQfmoJ}y@+O9Sfz*@XfE~rlUoWz_{1K1qw$u8i zt{M2huaknVsGT!~KeWvCcU z-E9XmKX62!5|{pWvgj^2?fW`mH?dzXgb;AS_OxM*EUmi&8sgz3=rNkv$1%)&PT%wt zDqaWu4Bei=T4N_l+7If;Y&=Y^9U~g5b-#oIyxDlN3E>Vv(A@-h4u72yt z+Gzu8BVru{snqnWB6w|gixdm2c|3CS( z3!>%<%`urxsWyp#2b)UOk>ZxLOOOj8wmIpyRZx;@v&^*k3;qC)cnRf7-SoJa zVhT{VahxluIO`c+Z$|EyelKT?GY4#2`J@fZumbKTrNq4Yn=H--z{6iyBB-uQ0 z+0Iz5PEB%S_?li1*}IXufvh`zsl>)VS{^Ai2SsY;N9nL3c zp^_MjixR1r3EzTkG~{}@qz8wM72f%MZ^M#-KANGqX9&oOr0K$~$H&7&PMN&tL7?=E z7~Lf3mv=VhGY2m*8(c8LxK!tN)C8-J7l;seqZQCBYC;RDKmNW8OZ{K|1(_~K83t;B zxBpG1<-w?8Z-ZiWSn_U$r1iIJ_X}~g(k+AV+9TySxof39YM&pR(39w_4w2*&)AhX4 zN==2XvJxNt@a!995Hk z^h?}4{N*(WiO1xP=fd+Q<6G%>ZqLRTlK8a_+#^a=MnRZIUs*%Cx`BS40GhzdL1D;EoN0oD#ak-Xftb~r#+!QgOEhU%SCtzrZ=BTWT_ zs=HcsZ}x*V`lj%jlQzXbU~QJWIa0=y*jSab)PbnlpqG6krs#k-88-2eNTDNkuZtbY z^Y?|iSxbJ+n4Yps5@W(F`nc0oo2Kbt1K_=As`E61pPj?+B}Z!V@3a^no2pgSO@6`_ zqRGNlwe|ZAJUSI+hJvG{u1F!VNy*z$nOlZD8;PJ;276l)NUrjDYNLV*URPw6!RT+e zvL@rj@%cx16XE>o7sG#h!<(0fM((8xT6G4+tsK-D2u;i?H}-4A$F+8uBajR)bKH)g zf{8*o_Q=-OUS6#+UTE(4UILdpbL!R&z(aBATNhp2{4xs+7p4(J(*GN@k1JkCqxh4_ zo=MM@e|5ZaO78EbwE$vpU>kSN4Sziw`z3yP1iTS1=3Ih5F--W9f123!8=|g7Z)* zJ^Jw?*p$RHUQKEtzn7hJJvqs;G%Rm~Fd8jK{-YaTFf?-S>DV|4Psj4h<)Srh>aW>4 z6kd|_udU%(oP7UURe$^qFB=}Yfic5pzQQ+`d!_*;C8}2Ha`E)B0Y`oOcAdej(JKH* zTiEZt*4Hhom)Dyet|9Y&rA6lZ3h@IL++&_jtvt$Vnuq=(u7D-V{GD~J=t&QXQJrpc z6@hDWgAAFNf7@g#V`%Su&B?2CU)U}x`Lv7MnqF%yzDk?;8(xv1gcuC=X@+ zz6!58gkW@c2dVXYi$1HfTr(+wTRlCS8dC^eDoqwFRGM?Y)hd0G?qjm{_S3TUCNUSY(LLZcQMve4s9>A3 zP;pI1<#}4dl^q9}0Na({U=dFyz&_xJQ&~m)+ON~*g%m8SOC4OssFr?}Vt!0BQa`o1 zxc_f@p^C&0*+QAs*hm*%9`49vwG3&Sk5p$)k;Gb?)Vtc=@ImWjA>=Fz;$cVvK zBRd7ZMtH=s^(}la?Ft5Mpwv`qWJ1C2FJ5&p9C>VImN*rmE_F@LZ>jJ%a4c-w5pS*r!>-gMitM!4_3ctcDTxZgzAEa{O5KA0bkBhm0?=TK(_XDAli zj7_O%yl5^dWFhGks9Ix_>XOm_PW_02=ECo)4kZsz}d()esH|QU2 zn&1KSj}?TlLbNLCC2WzeQU5RjHvZ9wAHDX%oOOt2m0r_JRA?ias7CZ+BR8Q`1)vsR zTveO*o|{^t9IbM3^5uDm;8qh1nO~+8?9Aqua8py$qvH$T(6OqHtq%A)MZ(KkAR4T< zGe5eF8dtg2bS4Gs4YgL2aNl(m44;J+md#$a3z0aaFel$s@0=}pt)Epk)y#*f8Zkc{ zE-$`S@p^tDWb>ktq!kZ8^WTxjEfpS^@8$G?!Jr^W-G>g+Xp0$osM;ZeIJ@v)(n3A@ z1gqU$WCx32k?F}l+-JqhGg|65XZmL?to(7!gaS#+6Tn03zP|iSo z&?l79ri9s)G4b`d97!>mN*f9-{HZ__zPqSw+vvr)BAlmm9u%BJV0tR|m3@eiZZ7UH zQKJkk=J)5WOI)sd@P)XEh(CwX@*qq~Q7JB&2TvJfD|6K_nVtO%y}}SYNrm?B^1WTR z_SZ;rt|-W=V03{Yy5OxoYt@T*AR-u;9k(x`sT9q=sUHY z%NmFGvEhwsta-Vvc{=32t|Y_vlDkI?(3G#*>^IE*C+JnoA#bdBYZL>^v{7va9hd(^ zPt7XHgmK-V^5p4nkPi*-gocqMsDGHsNfj)j2U9p6pT$jmfYd+h{$HwI-P&+KB_aV94eCPzRoR!nmSeM1@Vp$VgAi2Q>)K&$8<}eGVH>&Szw!RB$Ya zI%=ZM?nqLxKH5c~GU~vQF&j^hY1WmvjA!K5E1E*`EiCHkzRlK8x*`YZc}!~qsSU|r zn>6Pww!?mKB=0cgldv%SlbmLF`*0IhhLsj}pYa#v8zX&2v4j>Bkh_St4$9P9CaqkD zb7FLM_%JF$hV0l=+o3V8?^6VuMWfRjQlwboUMFm!F4)pej)tn0*d6QJ*$VF`XM3~G zL;c&+su;5cN8NLieTkX{QCsC#iRW5hv~v@Dnc&RPu_*Ngd@>}$FKl7=l(MazbJHcO z4?V9&Oz203_XAlBqlV42xQ*BLKkePi7KSaI3gx$&3we`i7Dkf)&-zwnLXizW*kOBe zZRq*9HJ|%;F!v?_M?eWyE-(+Mplv-J@U2IX*k;EcK_+923BPvC!9(reJZ51D!YWmu zmuN{LETGsvzWRO~MmNF^tc;xU2lsUeHo>$9pM z%ngF-(M}j+a%~dknc(NhT5(OqvW52u5MlJ-6YnnbCJna1+`kD+>iT?xOs8S7pyNfj zzb0MwXX$<1oBy;*#P(TEd&BlB=le-Xt?Kg7Vdq7{5t*q;XF}sW@Lb`eenc?($+%78 zLKBIcX){uoXc5QcRk7b|73emDUXJ{PI=rj&BTUU~=B98XzYt(GQ4k*whB}Fq*dAC< zM*CA2rqtg>%~ihlt4bz8$=7SUM6NiUh(2&P%G!QDHd>71QiDbu&Jvx=Jq{=Rm;D&j{or;) zrxOYDx)WP;GFjebL9@CdCyA<wFmjFSx&O#ldmC>uG~#;cW?~_hbsjR> zh5>xY>y{&y$`B62&GlV(B<+gs1y&@BANTw^c{!_l6rEPb(r_z<{)yo zq1;haYbQB=ayQ28OUN6kT*G`vkf}27+7NQspshP#=&8d@N3}zy>T@y@BXNH23@%N7 zfeN#ve*)WL09#-)rh$)IHIdUqp&|)VHDy09-fcKU2 z=(X5T9<;oWCIXyM6Y0&@r6C9o+d@ua548s9uWx9ZqLk1y3{!S8G|;aGwJ!_H1(Zz5 zK91^??qBIj?>REyq9ku6iO0PO?VdS{Cf&gyL^f$pN3_gAxIxb#3z{6*XKli5Z_V=w zL|MWdnR%Mf(k7#I^--9ZbfXf6-+85cB;vTpcJ*^06K#dwDn_k^p}{c2^Xt;XQLwkO zDZ~>Rdc1=fF0E4~*Ks$L=P!z|>st5o8c=9T@*-E&mgBT4l zj-(}d`1Y)1+oHbNxrhs(VfvTB=UW+PbRBL=ABQ-oXWg81F%)8St`|*>k~*z-W?Sa; z>jE{#=XbVEC;8v9krp6p6=+t3ObL@?Vk5>fmK_WO@+6AHryw+(p%^^hl%;ZuI1F;o!73kpPn9e!`7H&)wVgH24a|SUG{iwh&1`mpUx~iUGLhZ zvo0%txbgnbs@03^jlLPw1)w$GTNjtg)T`OWxnWvSqeFjuht)uSBCWj*8NlzT$T#s* zNvrb2Qpo~ma7L;=QoNl{H{#C2t?bqf-8gz>-urq&9Qt+1|M2r*@|Qv1XP(vnkC|xG zEvHu%-(cN$B&ZUp- z6#CgLJ0e&IdtFeK%{0Qexq z&vK=F-KK3vCOtEV$ex@nqcj;v{=W!M)bQFbn~1O|@^YNJ2h5#52SDrM-J*_KWl)B6 zkZirol-kCWY8lF;Rs|{gKn#O))pHyyZ#;3EZfLlRo`B<~nLr{u<9mB{=jK)Nn&d46 z!84|Yh{1iu=kQPT6@bz=Z3nggs1g;e`gY1${=CFzF`H18+*z^`l7M_NMMD;k^uO#+ z#73A`9xbTxmyPzxmQ{o6RRLWQzj~yBr(sL#Sftl>+B+w@}N} zjC+>OON;IN!<~@;anZQrpvW(;Svb!QSHDcjp7J!qa)jq#GCUg*MJD75gk$RBXtV5! z4_i-21Y1mh%qwzP`y+0ccg_*jsKrz#=)z_59Uq+^)|jUa97|I(4&MK#Spe97LY#Nm zA+_`Le%<*O(t5RPqv77GJv~{4;e33#%CtoGyKg^R zzzYWgb(EnO3V!|5o5Xp;`G&b9!z=NqkP2i7o@WXme&1IC3gQRh>vyaP8EfRXq zi|L$ZfvWs>@ZKdI(?^jFHJH`-iEcUo={}AJ7nxq6&WCN}6RArR^jQ^4NS|yjVgrJu z;ITWi#lK0_9hyLi*6|^oq%wpXlT$k7Tq~C3yx`NiP2M@>NUN)$UD!yTSteRzDu~*G z1;{RaHk%7C6mL3W4r@2c4}O&tnD#7=3=d@CJC*GGPvVIyECl#_Yl5m-x)&xu+Dp zCsPi6qJ9?>O-~}VbR@FRjnwQ`SIup;t@L-OdR3`O0IP!72=}<^ES;qd0ieN57!_qv zq8N;ju3OitUwnI9n+M;$S%xc~-R(>1 zz2<1jFAz#@ZwapX`Y-#tgQMHxuGqt{x}>aA`ic-B@fh{YQYV!2M-kF;B8PHq(;4cHv6K^pokD4hc}stm@ix@`d4$$MX%QLeJ_s zNWa8#^8`e0Q_HyR+nP6c=No97**G!Xzc1a%uEL}!+zKj?)9|SOW z^d1@gR)9w@x%D=ok!K&1X5N@Xa6V0%A}kT3;XiMkf?WxtK5@z%*`{9tJ8WRf%BPNE z++lI;CD;mRXDrcj!X_yv7?MdH2?iyf$!_8espe%!aOt02@R4&tlv&G9HX!)Nokb^6 zq}}BxxyqzMz@%0qNQE^cO=M3d*)=zfzY&~5E9a`4NiNx6Ja9S(@*o$$gdo-1*V4H4 z3g#UUE&|(g&TQ9O2Dpm4TD76tkIR=)Y#G-GiAl?Xj#S{HQkN81l9uIh@GD(n8`0g+ z19qKqM2T+<^6?oRKQvN#IOFY|-o#wVf7CUc-c{JLe7x3d4RP=sGkgk!)|vWB|H8UD zjPF1g-)T0iW(bwCNme-S9m#%rHKs1jD&dd*4Z)69=649kUNwnE2E-(V8r!{NfjcA% zc{RkbvxznY8k~6gA#QrjjGS!;O8}$)omC^DA%{oWH{tlH`|X%X_VxsE3VI0=(&^{A zd~;Z!vp&(YPiYg>q2{+EUvsq%qvzkG-vG;(0|M55_)duJ#~^q;Ii9svHU3-m)J{Rq zzGu9>MZB{m&MX8r>nq`g9kvyTl{jg*1Y(&iq>%nH<=pAMSV4Tqdf~@E06;E(?YDd z>{4k7dFk+?dTFF$&%KC5%FrIKL*r>uvDEB&;XDT(!gj>v%JAd!b7Ke3Pn!hEkk!ov>J zBYH?W*Gb$7J`lb!Z?B&mtsdsbslAvX35;KP<&!}T>63_i%KgMUWkX?M7_&^w7=KrI z!^asJFO&gMMUt%h`@g?_`*eFzC<3W5CgAUiqp0obLsmlf=ZY^ z+cvBk-_nQqQJ-}WajUYsn(Rf`Q_0kAhX3(8^!b`9l;+xE=>6PS4Yi97jJNy2=ZJ7* z{^Libmd^img|3dTZw%#_yuk2+bRdZ~dO9g99Co85(6j)NPgeq2d?n|+34#HIwJYR8 zoz_RF2ueup+>m=S*G_|mQ?PleKON8A!xpOyZTd4nEBBK3Lmm6BB2FWy(@NR1&Q$7J zF9W~wz7`PI@v0_BPGxdy!mIQ)_VAQM!!AM70NJ=X(hjywVmYK%}}u$-=R_J=x=+pIZdqU zSX=}B6S-|J+MBBQtPY%w|6(6)JDqFxj{4}3fzXL0;U+)Yr)Iv_7APXMG!Z%I^8c2& zAMO66cWQ7Yzi!p7G{qrY`L|?Ng2Z`YEgAAU>@|&bw3PWV;|VQ!j>{(f?IC8txlD6s z{rXmfXLgg-TM)8T6SpZRF{QZfMEUocx+rl1@d9Uv*P8QI6xQ8EKR@1@r7MHi3)%)2 z8Qk0Esh^MY-bifn=%ral;1sD1K%j}qt8)qj*`7$luJ8NBX|S?d%D5R?bdK2g`%FMs z#HiZwA?s`8-J_wx5TNeh^x6(b`*WE#mN`y1t0lcVXgEm7ve{XdvuwU7*LzjAX~bPb zv`I_SF=Mmix*2TJS@wowC%~OzXdi%2qxX zxvke9Iktb;2jfON8pVl2Ji)5!!nEF=>^s#dE!}s!k0(5=(Jz-~*fZw8RntacIJV_+ zr?Lo|Izk*BNqFaO{61>&rHE!+8{G@+4awcSn)Au`{*Z{jLPdASW-6%E9-mQsFj+2KQHgJ^CbVkC~ z6I1Ub5YFWN>#SwF+FI8;V2Ii8NU67$=jXjm&ifU*$LofN8Z%{!z6oFT1Vx@Lu?rn7 zi(Xp}mI{Cb_sRT{+YU7a*C6c&PD@d)-4PTuhbaG*3OIj2&VqyNP)DXW(;U&wkvMGh ze{^9kvh6*cq}MTwas>rpWIH2;bQsCV95Q$ zJZWVFYIXlNC%!fWCpZJT0xX2<$&+@mD5W10X@eF%Vv-*ye&yt~JzA2@bRMqZmcTMi z1g8){xdd?He6Cb05XCHuZHWo`INPtoGUEUT$_|H!Zy6tB9yCeG%id55?W#Tm9IzwY zSB}{h(DHm>drHPpftdz8XsY?W5BYH$?M!MGDE~J|vx`gPP@!H749hpuvjfKWX<&TK zBKpwUx_b&x@+|C3G}W_{&iHN)oG>A`t>!xNOWYq@d!p|7-3=_~m32*0C!O+(L3@8e z8uYyqrg$$^%0%X!C_=r>J3TEQVm|2OrO^|@ZxKd{p@kT7a=%*C7qM05XDZE8zHEXA>4Uy0#8 z4I{L3?$~}h2t!E`-|>V|jELSx?RyP|o_V%WFPjQjgGAo9!;R~;CUUuIyk^(du=;F> z#9pT8&tvSYTMh`LnMovvHlNVF0aMpxXT}~6f+`>Q)Sgg}L8~iD zm{==wnLkMc5molAs{-3bQ`iur?8ZP~WGs$*#O7a&g$oD?td2iwsr%>|K8%u9eCKc1 zVUOg9!$wq3HR82Fj>u0P^hqI>k5^aubY{ufvSLDCKQQV*OR%ow6p2{`S#xuWvN%(~ zuR=02TsSA8Srut8$i{-|gB^hbLsg-;ce|%FMa0dle3<2__(3F>YlFHbd&774fKG`Y z;|s}y)6v%&#jN7*?Dwf*Bja2f(Xs2w>GHM(`6xv&SdK!Y8YYLEN~I2!JQ_l*8p3E) z*GLxT=B4m8y>A!;GrZavM+)fSDU)QoQ^h5m_O$(qeNTLAy_XXGOk zLzE2|KF?pjPPZR)qz*HR?Tuigaf*NT95U;o;MEdEd}Q zY?iy=5fNclVT4DkVrKd}xFDIW4bt)J1pU7YkRx`sV8od0|kthza4uCJV0w_I3mMSTw!e8uC~d{##(Z+RF8 zoG{BB(Cur_9&AXgHk$ zJGh4$$sDgb6%0A;ti!Q=#UrP7qq5AXAU4(iyP)U!dgLjWkEA;IoL5Xa0bWBQzx{K> zK=g{x@$)RrRXp$-hEW8o2VQ%e6-0@~N4^GrovU-c0DbmlNyo98NN<6P_{E+$HVb&W z{IcZv{3&TiLSMQa8PDpkhY$5q$Kb;d9^HEGGO~K&W*4Ae*>g`fbxU}(M8BoC_cLA} z++Cq{1u|R0>WpA?k}OqIgoGQ?s=LfOss|GR8{*Uwl}x~sBte|MQ1MuJ4f2+?8B!%r!n_>1@T&|@HaS zl6J-oxiV(1XmA&0*>FQO&g9mK671M!cqrk`08n>LzaM03L<738e_ch zkrWb+thcAdKtX4j#h&m-oHk`<_x^N>E^;Y7YR5dAhX(M)K4D%-`rn8lJN6kJli-jM zbUHlsv!qc=a@0hBUvY%yawohUu}Ea&Gf7?C^Z#Y6b;n@lCL~*W{{z`JnSEDCQF^>) z2BZe_4T{Y_@Q~8T+7Z+?a3``XIaL43z+^)=kteDsqC`pwBR|IRbvK+pE5z+7>r28$ zb6Uxhv8yNXYud=;4R`mLKK?ghB%O&M{j8}{bwbFf@|4Wu&P};uaua#-|62>P5POhN zo`gB1It}{2@0k9ucIlq%K}2I%S%0}(TV(@1xP$s>9>3(H6CQs9(0v?qh)OL|Zph z>N!et9%S}m(P1yTm}6Q2jLm8V$GqCC_T3V?>$ilVvqDW_<<|23m*=0&U+;%zKdF$1 zzG~fV?15$u;gD*E%b>{!a4|el@C4jQB4oN^Y}*Q2G0{%%4l<$Vl8h**x!S1G4@MXI zgV)+FJ14(#6G~);vmv{pm)7jmN2z|G>}ZJ7)Fz@|Hu3z+*HLk?^0{){fa7}}M;mDl z{$%c)$jdUUk7mP;%2&O>-z+$tMn*KTGj(YA`Zh|V_q|1EU?VVxqni}C&JsJCdWS9_C_P|Q+#=mN;=Wn zD?PP_dO+h`lx^3;!&dU~D3`-rBc1L^o3#hTJ;Bm#cajd``YmUPnf zo5Thph6`Yo_r&zi`uU1$f>Bna2h^$C6af#iJov?-WKpB174cEKrFs{7nme*F-X{1% zatV2RXmgNzTTR5IQjyGxlo0ehZ$)KAvwO9g9p%xZ2@M@)EEu-6wIw!(!qnWN2aPPG zVqdL6PIXiB-VM}L7I8PP zBdLFuTF|B^vB!28Bp^rB-$Lc41|uu1mE%j~!)L#Rt%x*k%7-LF@!{Oj#S$otY74*A zALtI!;*`Ha=p>{c=X{sT8VB3Jl-rDL@(2UJQxVtNm}!3^mRYZK1MK`GW8j+jsF~t- zKG7s(qcLz2hXgW?aeCZka@PqiGl)$@`1Vh+>@f|iyOYP%b%LuEvZE5l#V2{KuQhrj zQ)y)Eh0RM1?-9!$@PwBv^*dzw$-$P#mnpf1qO3bF>bJoCoUvgW{6XpRRVUc6nau5O zi(f~q)7ywV5#@NC&+a~jc$q-ocD3woenx&Rr%+xU!hH{oJ0r-c*EC`|c6N^qRNI%) zqQrgi4XDT4^IlbUgbZa;99eob5_LlAy*_HYjQ`*MCaW05k|!#PGri&0Le*!LTs~tV zR+D~{c*v0GAy1ow5=jo%o5dUI@${DN0}>H5O_USc4Ljecc1f(J9VdxAl*B-XYcv;k z$+g{b#CbLkLt{X2ONo<*bL>(j=Fg@$neye<7uk$Yt(JBHi($CXnQrnaFmT<-&^&9B z)ipp>=)AiMg;Wa`f3FWfYn(!8GIu^Xw}z+jEVPx6m3;%bACPP2w1dLV4n?qtz)B^ zm|3FXnMYV~Yw%@xlCa|Lmr)Hbi>wG?c?LSRuliZg_-4DT%a{L!g!$tr98(s9tL6`H zGC%7_OpFU3ci5l(#5(ZoGh}&+ysbT*uhhC=fG4{tkt1dqGfcbli`Pv6^V~I2qw~$% zaLVeh&Y;zze8cY?`IEtMyG&QLXvi&g)lisKNZQ)|UPwpyy4s*0M|oR9*XBdT5k<9A zR|*+T@FrN;kN{9!^THd2cnE%p; zvZo4@d(H6~o#%FT)mWu_83jKzwb!Cmx@37PnKKRD45Bm8+#r#a(hwquc^nr6y6+XgukD&rq$61O|EG9u9 zF|QJ^F)C^u#Tj!P=USMUU#jm}jjX74y$S5SI5VbW<#GtF{hrwPrY@Xx&7BmOS?4+& zp4wqO@^jb_Z(93Eh2Uv*l5d5Myv<(42e=>i+l_8+aUMDy7NkmTwQtyb=yPGG+|1}V zr_Tj<@uXELYFW{#w_g>SEuudzff>_SO;hX}eCM4^XSAeW zjq6@etccJ2`5_i*Ci4Dj!}>5i84JSST?DC(`@v?E%EVjv5<>BFN9eN`So{}UNXoawxT<4 zl(|LDEzwHvjJ!mWb%NMA(_BK+AX0vX%69+%GIa~J{iliUi{dn*Z?C_NX09FDBcT{% zMDYi$_%_AwDw1wVa9pEG7xgo77lI-%7J(8x;(NG0g&x11s=n+b1st=*zDULip+A2z zb^%Hg9uyUcGDf(jFQO3Bd=hsXacsZUGACybd&kKwQul*Rzsc(Ktd(2zALwlz`H_(G zn>EjpHQSFjz?TKurAPtM%VSgOtX}l!-=uT-Sp``H@-}MHp)7ycep7up&`VIbH43&B z8f#Rj=twqQtKVsFY=m~RVU>1Z@-u4aHy2;snq&D(qlD45uqRv#uwL;vWUL}@=oYZ+ zLXd%}nQZBjaNr_O$@4@7)yu^LgmwvZS-zg}dUgSzGR(`*4wH;3%lR8(;h}jedKL8?6j)u;@`SW7)kA zoJ*5tF3d+*ygy8ylGNpo^)_cj8u>d1by6pdiESY_6;;7^pA*ug7^=C&V&jDn+o?1- z;a@n2KimIi#q;2Np^-r#TGP4EdMIuI}mQthLh4-t00f0SS zMBm>GHe}M0ev$#_wBSjBKN!czANwHW_UwCy-2W!LNv27=)XS@b5@z11*^g(qtq$fZ zX~HzLRQr(O57wBinHJ>FjUhnziIQ+*hfO*1%CrA3!rNmO_5$5&n4S9%4nUdN83)G@i@B*{ON=_-2!!xbbbHR@zWtln)w zz4{f1t0ORnoxdi9Q?^EpxF(bwvzt-+61*<+(oCV7N6m6NoL^2gzuLjb8w)u2!vmom zlkNL6dtwQrnuhbcN<9^yFSTxmDt) z#lfdMRI}X5z}Z5qo3sI+_3y&gA`ti{ilZ9cO~Z5`aKrms2C<-DvuPnXx+|oHq+MO2 z-7!cVl~1}{F|#S`IhEP+hwDtwHXeshZ#hMw)_7z0#YzPabw||i+-K4|xM&B9N@1u! zAtIYl?lOq%AN!Z7ivNUBeX<$Gr^?`g@R5w|K51eWd87HlMxsZXbR4qCcA^OLo-(;%dLLfJ-48JSh&piUsXb>b`z}8Bk z1r_1xw~hMhr}?Dq1p1Unhsz^$fmv*LhU1w;S{hNu{VbWV@ugxmmkkNo{t9akCAP>( zy#4Wg-kbbWi$t=l!#&DMbC>EcDzJ3$LR&^zPMr~00qV-GN#+-0qiWRmlr(Pv*(`1z zeWp|+#DS*X7~%~6vBi%xL4m(eZ4E|ZeQvyJM_P&*FB)W!&V)%fMaH<}RTr^(piZP( ztF=S4`F2BcRvmdj5neb2tb5fPu1Ym+nKC-bytMw{$NzfWYt>of!$z)Frv0nIwp>dm&++Sp z{+n|OsM8<9KMyW;i3}Cdo@1>0>~cw0A9l~4Za^K{VV);^R?gokTK*d z4?Qwf!DP}ZEMxyP(Z{9toAHPk?TG~(Qdi;Cwtnsd6=!HQ9UQq|FcLOMx|U-Hj}kh)v?XM~B(Rjjku94WkAp>w@dm{&O`2yUB67fxQi9VBe2 zTK=t7%0)N68LUPqTMMQ-&!q8~>b@a_^=rJBy~K1pR(#}}1ed92`MPPIw+9<%kuup9 z8m{NH$8=RnZmJoXt>G}6Blf^Hye=)xtn@}L6Ks3t6=grZygbMv@r1b9RQgnrsKncR zWX?&n**lW)#L70&#@|K&7QaMaMikJx|E4wQX^^?1R`!SrG3JoOC|r~pu$ZvSH6wW8 z=#DhISs24sO_4w;9Covm*97YvXg^BL+3v_(hjbB19|{)tMLB0720inWiHNBHDN`T9 z9_MI-0O{K;<04c`o}ZPBf@(d7BD6_OeR8iO(+`1yuPPaeFAw9hxn%D%BNQL*BqLlW zFA-V#oF@XZOz~Tg7f7fALGfA9gnzkIsVU#9@dD8P`pF&Jz+ahB-q&b5vXs14KL*pc z6CmG`ZN-pYID1S^Az78!7$F`XIB|A7{u_(hYdA{VBQK5>CaxOv4>>U&DzOIcZS+_Y z+w4TiRY;qq;3K+a2`J*-}*uE5(@n5TuAbbSm8OU?-Mmb;-C ze2>NMmrO8wxZL@6NVJU)q`PdEE|R*wf>?tf25zjI`_7!b=HrJ)&w8c@y!ET}BE;Ca zES(_Ua4HZ~f`;Qpa@_SYWsM&`&B#FXmY$%<#b7peW%?QJ4uqcAQvZ(Qi7yBD)5+~; zp-EJ1MfR488|eZcCFS=<2sGBqLfy1rzGa;l_X}-l-WfMhmET$8L3G3-!65;b+uIn&Vn!fs=V9fo>Sz2o zgN{wQvx#9%IK;LK7^L$(3(W|TsaD5xv}y3CqlalL1R}nzta6hJ2VTIJOr?* zc*qx}>Ra29QxD>Kxk+?Rc6clOv65uZDN^})Y7qUIrp7tKJeMFwQ<*Uvtb zXE)+cf3bX0O?Z-<|GRk|)F<$vTPJ~?Mh^MGkzP|JX;E~mIO$ZUCMGY;jd-$1-rd&+ z#MNdkivPOU7x%d&6-ghU4sRU33BMdV{OX&kz5~#sE;#lCiGiyKr727pc2qU=i5EV3 zY{A)@2l4yry=6K zM~j;1H*SWUT_dDBgUYK75#3xC+!E}G3@FW)x4_5-nP!}x?o5h3q*`zn6Lg?a!ZnX~ zhD@`VJFL6+xNkIO=t{|etA*g6Eh*SSDV!6+S3MjI!*T&Avo4i==sUQvgBkq@xwG5^ z#!8qDefWVp3FzWTr@yEaehA*zML(+6?xT%Ecge-Mu&t9*Xn{NE-+^wj*Z_PIBJD{o zjO}`)ZR2^QB7P}TSjxrR)Ef&p+e6jS!#(i+7swdB*bk z=}h)e;!eTu5rW0n=rm0FLWWaW*un5?8B;PBu$6%tYGBfa3b0*ir)=wUL@XBjE zDq_9Pt__Rj*ZzoksYQ1N(v?5hxai*LACn0;WgT`V9st+pvPd8wL>$p(9~_A@8A0b? z3`xcPt_w>O0>^l)!9TZmCi0U=vNy2pIwSWA*5tmws#)C(bdMG>l<|hxe$NLPsqq?2 z_WOjIX%T71wiU`r*&^tjPA(ExHKS?t<2e~@Q67>sG3@cef+NiQi6DeJ^C60*0WCVS zRd_M%w+#Lz38BYDfC}YMp=RW-=qhq0OrT9^K9D9~{u{_9SIW z8xR<;W?O7M=ZU1#p$RLLggubfdfzy;^C}}ugxM6|JvC=B>$WaJU*dDTEFAgSJDy$pzuuaf+^G+Y3i_WHsR)fCayif`_4cqU>IORMN5{b4C>5Q4GF)ZZhwKcrhBUN`@6`K9r*|Zf z-5@*`hSAf8ra-m3{HzhrP6(~>b|ve(06##$zyBLmcwCOMjhIk$h=LB>U7KPKb=jvR z3^HR1w*Fi_-WURxke1bVR^vrJag!y{)mI;RIT0$k)1kaSma#Gg!^8skB?RI?u35N!~Mi1l%T?RF&*(HT=R! z!_)DtazB0QfFJLR+;+&B3o$KIiaoPe2KuFEqyglVw~a$en6-Br>qSD@&A{3gm2gU~ zv{$wy5Re`r&x7Mt4WIjIKgR)Pm(H2rqCxcQ{Kx{wn((fLtBrqw>ncTuDz*=fZ2jN9 zJXTCrZphy=*Q;*oOFX3<4r2`$&t7lyH5IKZcCR` zDK(Go2W$@I#dxqBw@P`;2VIBGvNGm=wpuiJ_7agZ3T+GVHJf1uZ8#rz%WdY1n2^m= z%EvFd_IpA@9X{q~<i41*Y@%&J5Rp2e|&vrA?b zKmOkc`Th9MA!73jR=Jh+Ht7w=a@=s*3}yabC>P_t@=_Oq1+t`_#KCBvLy7Orb1^4P zq)>f&hh1qe3MU-Sk7JIUo$FzZYACm9*<#%{D#E{$^hsB_UQuTbk6hd#_xD<`l3{fT z&}<0h6yF%M|2HH3gbw6JWT^CWC9QTKYev|K3vr%_Zvhb}#|_5#U0OJlXl~XHK3>BD zUalNQugzms4ZE%Wqsssew9prFL6%_-+J9ENHFIvf^k+>lefDUSyskvdk_XeQ8S%Q8$IDWoZc7%)Ub#z6rDK~+(OBOLQDVu06;#J%UNSYNOqx`m2j@pjGmCEY*T zpc3BSrU@y{YTdDgt~3?-%uuwRExRi*i(MCqvPt3J(z9N*b1K(D40m1+bs52+WDe_8 zO3hvtY8pt4bZ{OMj+N#4`r`{dUuG%qn5R-R2lJzf?qKMm&T%G*vfXf>fnca#4LM+& zD!>z=*UF$hmHVP*o_^3x{LOj|ZpUS^j*&5W+tf{DUsWFwlN^?=g%sCqzAW=37#}*g!JgLD zP|^$?2?kx4J?Ch4GymHHFjb>_T$FDzlR7Eb+h{~$+n_XqN zZGQn%%k@@UOOO_vYFh-;U*YN=iTBp8zAut95p=a=VyXoJItv zhMQ%?hgagruw?lp^obAcnQ>8N7C=@%_r-|3c8B0gARCX|16z#E&A=yG%6wx-ZZuyi zGulWQ;A}j#UTkF7R?&GE^wm$v(ynHf!1HN5uS5tPNKfK`dO{YM8X->UpmTXs9)=)m zyZpOK19Pk{GP?~^5yt-_C(_Tw4LDTEP_008#`>&iCH<1U*$vT&(;Ufu(jnBC(^6P3 zwpEvBSy@;5;f9P#=PPcZ*F@c^i-#o{7r+#l3_K+W?Q#PQdQdJ~q=Xppja7$pey2Zl zTosKXJfwh&tY@M`f2(hXN4@I(%%RvMH1yV$OEAo?w<3FZL(`BV!g}AaVhnwTN{!xS zP`dbHM8Rq5(M!fohaq&{In;7(=d6S+9Tx_izrfzGgd$-(3ao>;H!jB--uEnXXo(P)=vrmv_^}*`7)-TD#Ney6} za7a;hKatet1}1vDX|JyEI`nN$b^V!k3Xedd4-&!FBoeDO-=BachKE=3Eoa4c(lKXyQ&Rmf(BI4Zi@xSBZXxkD8*(DvE{Un-$ zMB~)g)fq*vZrZ}v*+VEzej1-QjcC}ztnVb6h9=YUr z?K}I+xFq*F!d|#1qL!vUG8K-cOGxmmWYk;rkl^=aO|1lqp0|Ua^gk8|3fVR%k}FEI zS0rNIj@0TXe2urx+K$lTE6!krkB2HJa>Xg*VL$zTv82D^NS3QNqU6d>>66VAS5SMZ)a%{8CtrvkW&0QgxVvyZP`9}T-2k0ad!>kL(BawxN zINT5i513(>Az-BPLT1oDFyj1(-u)Jo9=9O~k40p-uZ_61z`AaXMENT9l^SavXgLnX zUes}~L18^)h(#9Opm(8He(_^_-BX1#$&%ON712La(E@8nq`{91UYQ(K-2wryUkv!UwGSTAyxls$0Pi_alP! zMjH(Rxwt1m&hOjN_szcS{AicR1k5vhHjX3cO`CF65$`zP+o**Esjg+e)GupTQfK|_ zgFY-sq!VZ=Dx7^Fl2Q!H3FyFXu{T1CU+#s$RU&AF3p>EasGh|8_; zoJFzt8iB=)EYao)@1q*Pc-LQJDdCq^GxpI;I`76CbMCuF+kRSLPs-C8BnM3g5(#-Q zy;if^E1S)=8XL9YVbyOHZIVNTh9lu?jssk+otGCGj~`5)QdurWf@U1uvvw-~>TPM# z%`e8UxfDnXmn3nXjOL8}*4#145UZidi-OX<3{w^ymvJHAxFsjmJI9?u%TN<_wv>WR zAl;ii2n!$OyaC}x2V0Z9@r6G==A=6MPs@5NL@%Q7=DJoaH>6OQ&f?XEQJ4EIJdl$C z)N1Xw)Hl|@cgP-x1kti#`28ci0g{zu9&7pnM^Wr~R{c^Me{CI%U;Hcwu9(pDv(?1ps@0;oP=?DY@fDsJvzF-eni zi$4E@dM$oqGobwmtx746Sn-WllMPy}1qFhhKWkMy+M;cS*u{ZAt@TVn6J}Fj1`Q8x zeBHVTX+vW%h{9RcWm6|2hbPLA_H<@%euXo%%4%^yJOnL|93cg_e&tVV6nvNBSF*s0 zn8Btgy@}<_Gz^ab4)LNNX1pV=cHJ&CAMD;F894zpY(|;l%MR83G%>oxDs<8zi-4dc zR^@UL-|~*1_~|1OLnX98?+GPA^J2vEkmhu%Yp(Bj`@ZJ0m%%cZ{RrJI_%Jt7l({^I z8kMnq8gMy3>Bb*@%Y*9`d1(M`{l)z#EWhHnClK!acNf zr1Zq`%O%Z#I}>&2iVmD6y27Ak)lo0zO=fDuh6Q(0bOr>j48iq!5Y;tO3xAu>b8s?> zM5=C(yDK2+fK#hQQ@iFn<|X<@E;8dJl!{`Pt4s`YX^}rQ7d5L5<(vKa? zcD&S6@Nx0Ci))hMW3w+HJ(3VDb(QnR158q#qo@{B(VTkLFEv;z1=$9*#u7TG@{vG z829or2o#NQkrumuB^Oyx!60dJ`xqH;zcBafUFMcg^>%?imLRI*;@Y3uvtfS0xaJFQ z!(fY}=H=!`tzVN?(W9(ZQc9&HZF%}9&DUs3LaOk6tez*pX(V6Kb-BJ!&*)S3IfIe& zIeiqUFgUk7GD7}hC4&#Nw*p^yoe`f@eFE_)oQfu6Mg2i|o`n19kDa=p7J;n0TO>&f zbsOcbsypaEpUF}pztDEjO_xiFeKJziozk_W&JK!vUW7EnhN^-4!TrhgdxlgTPBo!4 zVmY=Iep_?R!S;pDP3JAy{hLNa^1D`pA+K~ha4W{yg~iLJ&LY*14>rj&_Z8u%vLG3M z)?XGtSrOjsas*N+d{;>82ccI7S^}=`>K{}1{X%!Zv013Z)iz?}0;wwUmk7VLgz;@( zjqm?_RBDUVPDI`2w>zB3a|+LBD0Z2Xd(*BjlqPuKdT5#&JgY-J@=h{sn*gqhn9mrs z$*EOF=DxNVtcw+l@^${XVQmWQ?v5;lP{8#xZ{(>K`P#TC9qz5`v`IsQ^+*;1A;<(5 z*jgiwSo|-tV+g?*WepqtnuB4Ghw6XGM9rF?qDt_I!l|Dw{=!WRcP~N24M|MycVRQn zAJ#Sjq1tP!yOp^#K!S0|Si-*pGuQ;EX6_pzY&5M8h1l5CZcHugF9xv zM!VdiPoMe8Y+Uz#b{T`TL!Q9xDWWBwO77w#do-cMZ2Aq5KOZ|$#3V#h=kbMs$Mgo| zhGooI+1xk_oU2DQ>u~I+^lFzMZ5Ro*A)(k<7`_(~gqNVHy0y}g)|q}m@Ec&EJXrT@ z`lLBQrwexyT3I)=%v!;dPk>TnrxkQH`k}dHpZ2fDfs^^p0_AfWVwPd*8rc zsslAY=dvQrDP5UQlCdDw;CfD-7(Z|cf~Hj%9f#K>9~MpozX#HCWx6EC6YpX9erMOe z1hi0wzwk1LX$T6p-)_cT0!p<183zi6qV%qnithAw-Ez{rS2!JoRD+$}pf3ZFb^$qdIE$s5L(z%&TWL-LwFC_&w)&mZ`4!sWD7!k+!ZJ(k z3KZk+;mlVmhFbD9>&fdF34~ki|8AUl5SPE;Vw@^MvQH_AWaxc;8Nn%Am?+JtPBWuUGJUjL%b zd}bnosA=_X6NA?1oOb5+Mk*aO`=P;S&%8=LRd)VPf}LWqn9vPL57}S%-t#VYi)u_} z?JFN+J{WIZh2;QlL2i9Q4fvz*=fRmro%cLmI(v(3z-&#Sjvf{oGLHOEeG<2|@z%On z>v-SZzReJgxE8pgPDdO)q2cNBUwSi)O-+ zzHlC8+lcDUIvMI~KfsTS6el~t+G;=Y2Q-jt`|b)#PUQY|!^C=g2{YDKC1hbb2LUcL zcLN>E&n2Gol3R_8W-+g?rAX9fm^;O1mKy40QssebZWr@ciWOwhJ$Wq2B8y+3tXSGR z1w|gsXdHqq5$8c{>pRE~G3aHNz9z46<;q=yz)L1b&k~OHlkoR@9j}A6`ZN^k<~6;J zx(`hyPma9q>`;^%oe3Zxx*&Ra#d%-!S4bd)sfwcw_XEWLrMnm*1%4 zcN9A&%jZyE7qH8WOU|wC{AIF`X3w>dz4QTxE7Q(6NKYq&#{bzD1{%_$2(4khrh%<5iJHF;6*wrZ=j!uhuDPE$0sE zj2GgM0^0Xy(WW&Cl(L!h*=Wp#oLp&>dRbyi;$~dYss~}xeVl=puZNFvL83iri2NT% zXR#zH5Jk~n!evR}PU5h`-JP!=dNva=9Tg?;^4@c*8?p!haJ2Xe=*yVf_;95GTKemJ zX=}dMgU0d*0jLOaclywuixvbIblGzERP zeDW~{cY;AA=g?aU-8Ue0<>4sgP_ld-+n+h~ZsgmXdVi&CXVP)BU$!K<(q;`!SWlcZ z631y9%&M}3ICCOT;g3$XtR+hCJ^d$_jJ7{8`|eBzKV=F`O0Z^ybxiQnMmr}`*3hX9 zwQkl|QHVc=Ae<1tu$HW?w;2RyJ%?ag3#F z;o{AsRS_hhyVj*e6Hdo!%a*Fi##Z4c$1_965v-s)QAjyiA^c+#=#IJgRIlyeauAoKVA&8(p3NEX2&5ij z8H!D#87n9Q6xN6EcI`Ffz632E$K6POWRJjPR2@}o5$JBj2jUu{$FB6&Dp5h`{lqaH zy~S~kZu5{xZAtrE%kRghgD`K+(3sG#P)yFL=_1zfm_YWWN1)rA{cFRKRN1>BbJI zJudb$jNLj3SOa*xQFUlo7MS&OdcIp;Gt| z^70)^KK=i}rAYQq6wbvkrPjvmYL^=Ua{;Hn`&u2OE6I2LY^io7VD1~Z4z;3m$zDoC zL`V$~-dv8xS6pkH`$ zZz!W{}Djt97 zrFFXT7U-HX_G{S;8;zpUZ|{vh88p<9Q1;+|^C}-UZ6tNVSQ^Rp_1AG2dabuVipn}pQ2Q9YBu$ycgp$r&4XQYgVPl<7Lws8jMOERv+$zE3`Pm~o zwxOKDGm%m`2kS)d6qf6k+(4YH@kZF?Qml;*eYlgMfYs)MGTYA=>=cJFHw&2Dg`Xpf zaZGk>-C9vhNQeOrf(9l_jP^wbHBzQpt-hIp|fl(LKydX|~Qg8^MZlsGO1 zUV8yyneOrCm*-@xsg1+t_3zKRhjk}3?yGY>v43G~B^2T-^&UZX`hn!iE2Q_~uRdFc z|Dx@QSt_~+Khk4V?w+Ol5iCusbm%tfOilI#d78`Eh+mI=Fz?KhQE@O0kS)==?ER zrBEBRtYIiWR8U-gVKqAk;4kNQ{E)$WQ6`5ChUf$sB%_mh`WH=gogH!C%FqoqJ@l&~ zy#*%igzhe?NQSFRvF~$@;lR^q!2~z?Lr=8!x)Hp|4`RF@!DSK=I@fu>qA?oeiuauU zyLCC-B=}d(byT#)5kjMTHDfg}dfw!V>J3{h@;}M_5y{f6CAFFlB%RJQ7R}H7?44aA z1~A#*C0TyMHM1bA?>xjG3np$!mt=8hPJyTChl*1S; zQY)*NWrOI67NnY_Ns&N0=@cn_-1h#SVXPLo*nZuq*(p4TnsYf~Le9VJ1Wx38=ePI^ zkPw_(I0}Thim@L*Rr3qCeZ?&ox1Xr@X4><^q`03yVbUi((;0aZa)wY)vLT-LHx+Oj zg}tgR-o#k`u&qO=ccE#Ey9;kH1$;VfoTC$h(!MNNKH0yvOM%Y$rBC`yG-4jO-cD5_ zn#_e_$Z2uj4<^!<5_dB13`~yOQH5fKn%O=PVCUkAJrb2F;dp#_N+>;f1js+XdFEGQ z4!Ein(k;^J_a+)1LLyxYW)D-qnIyezdD77r>Yho z3OlP~Ln7OYt(YccXSJ%EGGOHO8&B2bdV}78sjXH2jil~TNp!UXyO2VXWd6Aiih+Mm zmP*9&JNZ>+fU$GQb|jt+tAi8Dv}k-{9>U}|;j7DM^nyf*{|@G&Jd7|<`y{5ih)@^7 zJGZbl?@3C1^5X^O7jkp_3$jGs+3tGXi91O5gDKPR0q9lDt#ite2^89OSl#0)Z?*E8(qx89(M-@O zKGfx6_nB#}evJ*j+3*m(g}QHMYRTpf|Feih@{*=7s15ICA;^=%yC&FZ(+4K1Zs%0B zAKZPvgK#00(-9=vq7XNT*pb^ck@c0#VukeQm1k5o2)s1D1YwjCpgQ&ChC&#hMbGTt{Jkp1F7dUq8Ef;UPL_MuP$<|;YXX0<^q*=L8jPTfZ~Cn z8~xVv?xeV-CT{40h1T$j%0}|SL3PH^Q*>OE53=0-QP_w5o!fevvtDa!@Y@q-_?E|@ z+-^G;o^oD(=30_Wc0~fh;6Q7#I@*JoLa(r{EG7jtBl*G`?d9y*i6{K%I%{9&k3dhj zdl04@8xa&zND(PmFhPT07(FFe)wXY6rzl3`hDI{Ei+Tqp(V};qv`h2izeR)AUh?j8 z?;Rg2=~gY3D-uK>u3DK~$qV&KH#Xc^nxoSyisK7t#slc^e}@@u8i4ol$nN`L zW*pS~6>e@S1%f5EJK(s|5)}0Lw47e?uL#B^8%QBoL_m|0Fhz>r9M8!jetGE~+1P)yDC53_z18L82HF*A5ljaGp`GxE05hvN}=mX6TF zczxMgFfMNgdP5oX>K>{SoA!r;{G5MlSzn+sjX9U*gK-MoH|I=%#o{*L#U~`vMU0{K zIPn$v^GF7Hw9+l-Yd^$(*7qkwWCUhhzlghJO=yG9C94fs4!Oej_G1Z8Nasx4FR`SZ zzd5SCY_j4vLwENOxm_AyVMn_$T>T9u3UO)5Fw~JQ>9Xcp02ohaW;HDvuN-ZOqR@B! z;_2GXss=qPH&#TsYnjZfvtbf7gbVw2b4v|;2z7Umw!D6A9vn1NKSiM-Mk@CKYidJS zQt-%>-dT4&;f$1bWnNo)K2!OW!c%?tiqmC`t5DiGC$Msm9Whw4NLVIGwgLq~)rnPA{g35FbE(?1p4+6hg3enrDk=UxgTDpbhXe z(O%+aLhpK6@sgoUH`QqE$8fK=_ClG1$J5q|bs66q{S_nUh`(A)-MMKmF~(OFNh4KE zyZkja`iJ7&kF~eimb&`-fVU>MmTrRTvv|9xe{}!~N;3?viO-y8_S~!R-?6pUxa@LF z@YB!(H+{`{+nLlgOJY$g&6v{d7nmTC!GY}3ByXn7Zdt8Caz}X)l7l-qcFPl#Yf9+` zKIlt=d>-6Jx6ivQw}|0T!WX>63JnZnl4)e%c?ZAwyE_MyZV8EM?W;PLS3qO}Gu2># zePO+qI@9xPR-v~^eNMwQ+A{82A46(ZntmrTfxV+Mc(YzaBb2Ivm3TZYnk)BOUE)eZ zSfT4`3Sl~hg+md}xfWC0W!heGPGNauKDcww$jscn?&p4m$ucH}$opkI(~26}R`1S_ zXOlxM6yt#zDd2!>({kc1YkJL97$kVPfgIPayK2#jSe{VN-;vE9iL>R3`8>#ecUhVA zsSbula}Q2q`d6x%1P@#*(e^HGB1ahnIR4n%6uDp2oT@3>w~XIcZdN3QD`=mzr)Ox4 z9>)xa1+T@T4_iNjLe{D6>h>2j6pCayE$$mm(la5TeG>$aHrJfe7k__-WS`K}LV@w`2Kq)Xlhd!pa4HlgOVR16 zTBF%(m2cg}gbiu$KFydPK8pO(b(rEZ+wOvc#U`8iiLXr?E zPn>uHgd~0eRytQ=A*D}RYUo7dTl1RK;_P9e1W(aEyrwnz684{%AQ?pjY z|5o9r0rRKp%3zk~&^Rom`b4MX=h)w|F3EtGi)Tr5!4f7R5sa@!Ts0Eoi1H7^oWbhZ zGSV2^G>EPoBXUZ{22#gNT;-SQVDgz@6NuR-7ox8wlVEzt@<7g&4 z^C||Tqi%?VD~l;(U(?@jHwS{oLOjt+?rB(xlq5Yh;$ebG!}aer>k{os1-H~4bb^~=5?9~$4WOoCGt z#FnIElM}E>`WxC)Y1i{1GdRYJD2&ct)-G17(4$)7NkkeOqI9NK$=Sk-f{6S2U!Y(&hx*O=mg>w{N^gC^2BH>a~4 zjaNT7xxw}*Bp;&8Cp`CqN>lR_{svNA)I%zg+lw|U(Yu{YV{ddA#M9p>Uq<*|Iv*wCrA~#QhX9{)^B0k~v+Z~}bWBKEIi$}b!49Pr| zN(RL#lQw9#6}1|Of2!Z~l96>KUb3iuX^Fnk4fSKDDbo41y-X#<2~FYS$e!;py@_QD zP>%|Pfo3QJy^VE%5u*e|iYk7L-AIl_g=2f(ajH(Lql%kJ;~-%y=SmL}ifWqKrg5t}hA#n#WK*;v)U=SE1Do zuPMbApI>IR)dJ-ew&kCvUQLS^d*r7e1uj$TP1Kok4RbX*=+tLaT&}xdPQezf-}2Hj z&DiZCXS(+`n*PW~Xn;`*S8T&jggg_a^G62!me0|-KABwj7HMJa# zb;|wU?{X_%@9U^c&x;~h8`Hl2q zR4}5Jru$0X%%FHXO!~%XAMyOWgW`F@NoHUa5v^G}{pDMQ@Zck|Rna5j77XR{Gu(`x z;S;#Y(wQzEDyPB2{?nq$pDRuLC9*zYv(=!e9LC#S_YH<4m(^s8ONK-BmBhqUWJm0o77WM z5Uxs`tIAsxjuVENiEP-{D7t(p9SLr3cgyqMz3dh;^%k*w7@ zM5Sq0TETJVRG(^Hk~nk;YoUS$po=8Md>3#62{+;XE3WXp5>~z6BQu;d_KQF=F***` zLdgRng!eX5R*=|2Jg(C9m7hn z`57HL`L^&__ftgQwjb`<$lX_uH+kdONCXTDg^ogAsI*UQz+eUUY`BmIaVnF9=)LZe zle*G<+q*zei*J*L1KOaDEK}m=XENLUfw}v}?QY9L!?U4psYYW$0K5yksA}#|YlI&` z!vyjm9QNabB2~YYJPBfl|2TY^m7bp)(MGJ8NbpCuYY$m*Swq=jjy!piR|l?GTUEZr zXtEu~Kv-t1d~DRs@iIDZxKDe(4C2E&S~g-`3$2vmIWW*gQ+~;?L=_XY**?4tx9}I+ z1F0eNZ|Q}V>Xk=dtQ83g%};I%ZCH42PW#|9fcK{L9aT22?<~kSSm`{&UPh))($o}j zqUyvN;zNAG;LUS{vWC!Hgsp=8wRAheiXWj)O*QNHxKNl+SEZm{4e+CzqR*fVv=ytx zi7gpV^AXp73C0NBkC-{I@su(L(Q_vR<(cYU5xz|5xStRw^f@WD)&3z)bbMEge4#lt zPXB~z|5!`gfn*&99E(9$aGoGa(z1e zZTlu(G;Ll74U>e3;@ewtFcdn~NVl+D=jrm!GXx{FA}NP65w;?uw~^9JGvv9f;f?a= zjg0Z8inHOo(c#PJV|Cw+XzoNaz+P1|xs#>-LJaq1UWYF&_S;xhyN6{En{-ebP*ZEGfVcZoznQ8BoCw zTCj4)!(D052aXWmNl)fXb|Kme`<}G7;C%?v#<}aK9#8VE{$$$Giuna(JGr-mTw}+J z0|t4`@be{)6t!0~LXeJG4tIvWAh$UL)~X9p>h*vsFRiE6gNY@h0vv~+;`;rVwPcGz z##FB9R(OutOj}T?P1~wEONXXlqet8Aa=0=mV^P!Jg|^|PNOa8vo|rxDwQROhV1QU4 zTNN&99b0yTC?X4&WFpifsxAJj^TS;4)}Wc^+J@{?K?RqEi$V6}2NgDdLaiTf`er~+ zS?Z;{#>g;pSInT5Z}NNB#eG_8lz+SBPu6Lt`10Khep^>{?9{uw;+Mb@9c8@bE698F ztAsAS2Yy5)i)R7JcfM)xCYYou{4yBnu?8)*G$^bf5p+u~e_cW({T*&h(X_)UL`KUC z1`>VF?K`Y9PS~dG2gBq}8)MFwGm2T*?AaZoyE*O4QKhUAG?F4P(x_|YS6=)?SUPUn zZp;|(h=@^rh6#tF3chBPAcK_&viV`5$AdtFjy0Su`3IJbw0%_$ z0GqKgbaHe|N7MWje1L_qd%Y86@o&PV9qe|Gx3M~=ls!DPCoWc-HD!r9xAj6UMnjs` z+A@{S44#yT)74PpYj!L{48 z1q~~1YYwWEr}XTZkTsfw+$=w+P?=68u3u>Vd{&9f)`Hu+@Eq5gtt7arBtA>1!-`xT z1SbOW8R1}h00!Ju^{0l;H>kBGug^1%RtGF#GxZd9Sw`KK{78k(oN>>Wb3(reJJlSK z(9~ClsZ#{!N5rVOt_Y^_d&43ONJ+|iFb z5Y}T7{H5MXDQ=hkpjJ;JQ04qi*SI!uARwdz=YCoSDWZ@p3snwxB&T1~^-mYKq>W@> zT~!jNQ{^&qCHJQtK>@;DT;wBeIGi&Pzj+xqaM@o#NDe)}Df~$MFjCPDYJptrxVZGvv| zMv9S5rB$A4U2td_0*w(sGFR*m`*pPkBO7(4CY?Dj-o8j}#?8h3GXH(&a(Ka8ixrC`YbY@RunFpDwgKzwwUxNPIPDD|2Cla9X){_g1Sc3Ig$G1%XptPjJk`B!SkFv0<7)9mUmI3~R`R0Pc^QFl!UQm^0u z-e)7%rR0!4vd1S9k%DrRWF)MANHALBi71|X@+o~450VF{>`GRKV{&QsPD@eAfEGUg zOQJY4DD8)z=`F;|*@1q(VzFoX;qiaf{BFtvXhT9yFp%?45ZL9HP3e4=yLPi^m|Bj& zUQZkfZ^ZJ!K%vtlc5X;AtL6z$*amzJk-sNyPC>lcAMbR9ZZJj1L(|^CaxJ`*7$OVmIC5uJ3 z&FTjr&3J+JlBcAedD)=uE9D6^0A4jYzQ@n_s|j6>l%;tf+ff?6lDd`f@6 zKmgJ;!84>P?fg+vY%OuiwoWV5hcNwJ!(hE}c9T|9E5Vak$d@a3~EdeUzqxBXakaye4)>Edse@`CqK}&sx>Z zN@D&3DGa^$7)-OOW{Dou}( zxA&CL>F)~K@_wN5N49o}<>tl1;{oxjnAJ|?>)bN$0YxFiYX5RcBxMT61I5A!^I4)R z^4N>GQahISV)|Hk%|?-U-ssckae3@IPQi`hInobriX_r~^@xBgSNjC%Pe>WV-Duye zDXTkLKI!-IBOOd9nRJ^VHBN{HBw(4eFL0STWA!{wZ=LQFHCNy8@WI0BtrJm?w)mEc zqqx_Y)m4hsdo>|~>aKIi^pIyuOk(1bqY!h=9>I_w zAbn+Zg7GNT%Y^UCdHYfOU0wfDZKtOe8_IS!TXs53Q*4}d+c;Zz|BRE{cA542MuSyQXjsCFcuhF(n_&!hQu7duG+Uau+vys@##X4HPw7_l6z?u z8iNdMq7xKRuL$^9n2n%nY5iPXGlilnPdZsMPBR&{-9AiTwh~9gVvQA?J(Z4p z_VFh6-`t_yuahMXqvpM*{Ut=C3v0>aR_ThXG(5_b`sok!Vivk4C``BB~)mVIl5z2P%8>UyhbGIV>QuCyjuC4=rVg)GxUY1dW~={fN0x|w~-WBkh7=GNeX9aNrNpi z!wA10=gkmeZK_i0LMNeIBngv_)2tuu{?vbf}p-{s}!p_&U zeu>SR{!{d<()6xSWW??oQR=JboeVFw*mM=3(a-ipDOE2*dYq$mzbaxMQqDXsje|4R z^GF+lBNGY9RyRrnyw(mi%G%VbCROSsapfIB{-x!CKS!BAnj#UhEOaefv@;u3d%4@t z;b}o7*S^MM6@l|IBRwdeuX||JzDx)ht=-a)2}}unf`|l>3sUh)eF+Vir#R+{FN?i% z2{!0h$qsic>D^966C_#Jx-F54za2fy4GOtXjM=#BEYSGLK6(KoX*yX%>K&f{G|16B z#|RP_sVf+NRJnQ$)GGYs8~e$28^Fr1^1ZJ3v_|ty?Oxj}zvYB0DxxY4+QP7mz^Pp+ zUmv*9p7@;;p8f%Kebg$<5Ou~0h=nnt$+&Ge3JVdgf(eF%qzV(hXrnI~1I==*t5@2$ zuUUi|qx;fXfNyo|F}JDVqmE6&`2sVqj5UCUJ5UoI(v^m<)Oj>jJBzO6b(@Vs^V9ed zJ;U$3aQNl>D+FaKo`Lo{J4w;lXp%{sDxIo0Ig#Ue4qjU0FkvuHZu9u(t2FN(YN^)Q zj(ZYGQqbpDIdt2tnMJ(qX|^`g>JQzFxH%xZzQ%sZvd`LtTn4?;ufN@F3>dRx1f#Un zl~~gV8diDW^SoKxG~iD7)D-sm%MQG;6z|baTQr0Q**tO*FKdmQY{0)$;W{Vn0fOm^ zQ<4^Rc*tZ%Ini@}cGRlUGhFzAg`FG)j?7Y(fPZ6AS}4LB&OJ}-)wvR-X!%khkxUpm zWSOv2Qk53?%%mLa9G6oGmv$WIb&Eo(bnHVnvj)B^rt*w-5=Bs0a{{`D<-+tBhF^65 z^$-(}&zB@V<`8YQZFI9pb3$rwNGDdlZ0YFx3q40)OP9H^%b!Vy*&{SLqTpsAa8(^z?h$f8C5jQ7jA>t64=gR&XbX@M=oeVF*YOk2@GYcI zvheuXPee3XZo}De=icmN)&k%Y=0>Wuw+qG1!~m8*&-jZtzQ^nHH9uE&$Z#n#+syLM z4RfP3G`5ExT=U~wkM>0p9|%qD<%qcBTa+Z9c$1OmIQMP{N5=E{V9;|!1@njt|B3O@ zC(_T8KdAc=#)A}Xru*rT;#&0m|2Nv)q-t_}bC%g-k;7?Wy7xRr#?+@XRX*pitG3=H z!x--oh-Y0`qb5u@EwBCht*5N+sXCDi@eF>(bl&qwkMW3}+zb0bNs#DOkiMx9v9Mn) zMh#Nm9mbUoo%~u=9f*)D^bd@);X#jzbDnli^c@TjxWy-<-0e<=0UPn{tT6DmYnGS1IpW{ZGyu5vSkgmjTVx{(vHsoY6?)urW zuc4H4+_G|tFMYic1%eQX!+Kjvu^JHPcd{{?EV`3uTwU^DBZA;rq+|2#pq8kUx-HLT zwd=%yQN;<#e!B*J2EuwxSv5!M;A?%T#ocL^E6$T=`??4=+v4M2EN$8pk6y{m)YaXa zcKGVzoS`shnINKtAgX>wF56)x8Pr>SlZS#Z2LTs*$)Yaex>tcX^p}8X7( zE=EWxWrz^T5i!|x_g?yYivjxtm!1jTMR(#CK&W_%3Pl&rm&&Wk^67?9lhHq@ju*Zz zytZD}e0i&Sb4zL2k8OwxF<+8}fF)ka2J;)hohhZpsyk^CQ}x?*_l{`qKqhMzXV1` z?V=Uwo~ufIz>MaN!sUh=}m@WRxadA263XRX&qn2d}Oma{U z4S`LnRyt%^R9yZJQV>=F+8XAt&7^ig5GNI31QykL3p}h4h+Qi0YQVH206jp$zky2G z7`e-61~mbrz4ewV`t2I^{%hFCLE4fNj6u?U)9Es}22|my{H{aQ>@}htmEmHBY*)Xn zdO@csJ=q7oQ{ls3Ar;7iDE62Chk>_o{*-dT(T8s>4U=3@&GY)%al=+YW#i%z6h}Dj zjSd_4J=mUYqF^*n@{ZEitEONYyKTWG4ZyCyyioH(xnEUr19k`$c)i+$C7{nrVdd;2 z70Rrcc31J&eTdLtM;A*;4|5~s=XbGDWDjF~k~2&aGMl3%RuVj7jP!^&aJGvbh|lhbpMxCh zu`)W>*&Cy3N{U?4>~;g=jivrhA44C*Kz7|QI}6K*db!f=d|w-iVygFDFjW!+QDu06BjzXjYVnN5NxNe zi+)m}AJ2Z6aL{dIf*EWb=$H*9#UE&@??+Q@{oS4ccaZi| zv~CX*{5PYxqBSr|mBBduIE22Vl|P(MI0h10$+oG!L`Ae<3i(ha3{Bk7_3&BC!LR z;fnehU!U-zKoOS-XN->@Z$f-fWEwFTSaX6g^{q^sd6Lw0?_zQ!+YIu32_cggI=(Uz z)*G&pu=+678$qoU<5-tAuCU>*eUpNT-CQi;RPU_RU@Z0z!vY~df*PFr`XdEt3g{Lo z$P)JG%|fQxaGL&9=k$*IrNjm&M1O%0bq&I!C5n%O0b@;xR(ksK70!4rR7kBemLy}{ zqfgczIK1pf$zlDlhw(kS#lRH6u9p8V*)z;2p;4`|Kpf`lq1Cmg5Efksb(-(Q%C?7}(`La%y8-NJyM zZpwf@W&7uWeJj6EJ8~o|fFiJxj~%7xXxF9r`uZJRnrE;V6e{8SqsosBr#o-r=rKCF zfIEs!xFhllXr*7JMqmTW%)1c7!@^4P{{3wM*MJ3i_hQT%hfkADevLS=WLAQ@Il_p) zZ;H%t%myDqJn zoz^XDw~R#VyJUUtXd0cu4U*8eZ!LE;oASb24n$h)3vpN`&V3*f%Vx5*NQe~3XYsKu zrURTP7W%v~m8Q~=VT%3$+6wmR#j@2XnmbeA@d-e|k(sD8^#--O3!^2rY)0N^y2V5n{35Gh*203^uyc2ckvvIV(`i6`2B29Xp5P`AR0AqWNP?zUM+69PG{ar>s&dX1U#U*Yfz^3x@89|b@3r=gOBjK!-pNEd1tJ7^jOlYm zO%HJe2t4B6Pa}o;Bk4P?nkg3xdR;1ACJJvi)~T!9F()S1 z_MGop>^hbsYRkB*d@1hZ$4wB`ibbiEdHo=W1z?I9zA+9^JbvZJ%YSE4YYwAv)912K z+>rZ@{Sck~6b>WNP-d!oKqQysD&5x!56b)uWn!&cWxHe*xo)UXUnW~Ni0x`w#cVBW zyz>ui%26@JF&k!r+<9E?i8@_4f6Rs6oaN4K(hjn3j{YL5E%Q{f%3lrkHXOMcGil%c zQB04&cDvqqfGsW1IT&+r$e6`8z(WY<)g@Zg;(Q_vvlPaBKEY+P)|y>t6Dlb%5{!ui zg%=@rS>QM={DNh}$G%kNMlDB2tt)U7AUexibEXQcl_7lb-kcA1FjWbMYk||#D07K$V8p^REN?YZcMSi)x5mHO> zdY>eF>J^Ud*+9sWnrje)cTj+iO@xNb`_$NhA%rkK3z z8;_1E#BiwY&GgWiGYn8L*Yh_S!?0$aqK_t_dSIAf{K_Pemr|Q|y6%zPG|Snao8@#9 zqn%?&xKgQ$uBzNzKs=cnubT<9NAM@kj@lfe{@K9ZFk=hk#OGm4821jAJ9;YO|L|MD zx>KGAf*EP8v}yQyqG$uHwHO87^%ks1G-poX51RKC!}LCc*Bt-B&Qu&HYN0+B-FAs! zrl) zs6_q9&3%cAt@fiekqRsBa7POmcN0f}#>!IdmNzM&q9=O?rsn%l0Vxm(&-p#>;2Qqx zxm6mioMB~+ogI~S>L4po{Pqx%=C|%FNiEN%oF_y zOOFe_T_a|q#&d5!&-Q~XUh703zqE~f%DAP8gX^d$A!V|dBz~A9-rQj5@)V$ld)lRe zT-a{z+@ThJAetL80pR?0>I-GX#L%vJOXQu7&{*whkV;R%%QsEWTjeC=C%&XJ2pKgB zdTNrF78bWw;Fxc;pRf+J?n5Q%8lEkf`%L!gD7u$7wYeds`O9J8d*wJezt>)P_*zc$ zv|4;>l|UZOsUm83Bxe3z_?v8pNRMU_ctbXaO%Q)pz4)Ynsx!6PWf!N* z@$2lpw?g{h2O7i?Gc3pJ|Lq<9MZXRhGF@K11#1 zApke0SjF0T@D`B~VZ6d&@S>MPvyP!;$`r~-pvx8x;B^MV}E2XW)QCXsP`KsWv$j4%E%<2sXZ$G%=-1BWn5{$6|xRU@Ko-oX8jfrb0 zrh+4z#MUVBR8>^(1yVK_FC|Zr-C(rBnc6Cfw>BkI(wTVj5u*ou#OI0=4p{0Z!gk4< z)1{(0!dB|)FS%;|)MyE&>8r<*Y>79*hwL!>3dqmcBz=JA53_uxkP!miuCmlx9*2lI z$;E+~2}f_x?&+Fx`ZAdb8^<$0P)}D^xYca(gY@*2NQPUtV?1?vNnkM(Db3oSAY;=V z#_jh)0sKYMc1zLpaHGRi0BFqqD{lkr1oT!@3DUO|Llc@WRUBTWar%`4er5-h>l$Lm z9yp_;p@sguy;M*4xnTMdoF@7Er6M$PSN)InKW1Jo8mL;s2b zK+5zg#P{!&O?V<;c!saSC(YU4<)UiQy+$&H3Ss>^mUJMkSJa2=r*&_CAgOvaqqNiE zS4ZgMsYm;U?XununI9klMh1F9g+9=+R|16-Chjvl);ea-)P1w)o80o^i)?r$%xzka zs57I9w2|k(uIZgX@kI~lzFV7@ca&jT^q=m1OJDo~ru-(xn+jb8O%^37POY(BDHN=w zRu0qiU5;yg4^H)STvoJj3TzW}olyXEm|X9$N}SWL5M{Y$`Ay$C!seV!{0(26luu*^ z55p>j!CcC@f;=q0_6}{$Dn6#-jBEj9D%|nVBq#j>D#=U2%vLd_rHo*6zO}EQK>UO! z;+q<>pxgK7>po85H!SCiO%_etO!`oIKMQ0@LE5yE*hEVaTrLs}UPe$NyNEe_!`Fbw zMq1pfnI5}}KGmT5A`s44b@iFAX{H~lR%e0rI84vbTYd_9rHVrQwgN8-`idywvB&c@ z?_9*s>2x@FG(~P1l#5zw@@}VC4X0&yoh4$tsw5;%Xwa3QTm&}LwK@coXg%{aG^R*1 zs8_|W8-*-mo;Y-8l78JqL@tbmy+e(*1_LpvuLZs-^uE?l;}p1drGtD>Ddti_(^)It zjIZd$17tNC5Bh*Ux4mEU0ll;Enf+CrK@Wz&T%fxB{?JUiE@$By2lS(H{VAqZ@EI^5 zrskh0|6|AjpHRSeQ-r%HEj<|0az%rw5e^gK`xBD9mX_8mlOljab8pQ##J8oiuVa*a z#0(JgC?ymm%_XNEZG8Zi01;)vdVcY(J@|396O`+Q*)S!?B`LXBUwf6}5Vt<{OwSWv zUPC!E$T2X(dE1hw-D)u+#& zj=2{Hw_UMIcc_nz}uEefNs?aQa_>2-8Iqff1H)&Pocl zeKJ)E^Jx-E#@#r`=yYUVxZ30^8cg>-vU{evWeeZZkptvv;$lS>zjJq<+|od` z4~jIE8hZUA;4PzP*r4uDSQQV*A)hiV6-5-v%t+pwD%sLY5sqpgR~Hd!riY@b?m)}p zPGhk}Wo%q_heV^$)MLB6ie6_1Rl*NV*#;bPyoGRK6lnvb{eJG}vN3j}S|zNKQxd^< z#8A1{=f;`VrTyk%3{SZ)`%v{QU4TkAlVSoH;QJtbgFKXK(yNEqUt_1=t*a=7X=_Fg zf41@yr{^K*nV-WJJ+PL-B*SPogYqU4Ks-phK70X|WZ+a(TK<;;GIqYVk^=O^lZgT-xWrrr2n4U^I#VY$%oxoc=H{B4De?$7HpI3kVuA(^-HyhL(th~W=q?fHHlkwQWHw; z3JTAC5<=>`i(gu(e{IQRUdmqU!?2KMU3xR|!#dZ6WmBGpL7Fk*A9Nmli)e-@jSqG* zSC1~#_;=4ElaUFEy=AB1NNgDm_e{`_9T(`vG)ATZr_u#nmb1VV6eo5~OJ)Us-uxh0 zBD_h`vt*lhvZNGAD!z_KT{28cmiD&Zx;Gu&oM0v(_c^6(TWl?pR1MUH*1Z?pEnWoW z!*MD~x^*4JwXLq3c{2w2w?VE~W91%Zp``G|*_>CAv3bBr?HfRXaf9d~qw~r(fL!sE zQn$U4h_}3&f>B%aGmyS&8@H@avLV0J=OErEGkBR&tLHLn@HN|QhLJ|nF@L_+Z+4zP zsZPfyV=`e!)^Yf239~7`tUtx^_w((v361Wpo*~blNG-2bJ*&H=I@05Atz{GANxuZI;36_#`pMXGdr$SuF0z$K4jTRJ582N61G zx3tEXo!Z}b5y4q*xXeHWYM0yxZKB8jBq@|wg^US96sjumXC4)OZiC2CgRs=<8ra$H5I2aOi`QiPEdpIm{G zRk`BzB7lB##Q%I(_w7+2r0OtdaSR*`8wK2^xW9cy9xgucPT1q+azpWmdF6RbQ;|y~ zcoY6;!WEP&H=&ojSD~5Pl+>I}6A&;H>#?$ns%hWFItt)-7a9a>y+mP}Ro$U9fkk}G z=f?*acMJIS(ZF^?q(WlFA&42>DD~zUq>huE_l+aH;A1sv>MfQ{4{oBTsU`$GJg)ZfGWKp33lctKY;5Am;o&_Ag zFW#}i=6PDHb}X}`(Q@Ort60i}LBxMs+B-{`5lk&_z8c=gCEFKH2VI3{P+2@~%GN4w zHwWnvoL~12bftx##x~YIQZB8-kA#THHA_r3F&ZgQ(U7GeFH`Svv!jJHiqF0y%>dtl zubQ8n&*DI`_Rm4RW567YAS7}#ZwQ}68gPcr5y;Pm5EP27i)fR?BaY_-J~Sni3Y<=p zhCMulm7v`qW1oV>Ur@nkL zy*hrx5ye5OLSf|Oa+r^;c{R*Oh3e%?5U+)(;t0beBE!#vi4){^Q~&biJ8++5WiRgz zUp;b}Oh}up_Gj!@bueCR$5 z>T(QyQXb(}%Rk=btbD@9o$DCTuWW?iZ1{d4*Ceq4JAEg}w#N8m5F2vY{6Qg0?M>@> zHcS*y+%|Mkoj%J!!OCxnwDcmPotYs}NelJ!sm?q5OA4rb*}pJF^@w)l+}8+my<%*=+>C}AQKrd~4o0s*@zeahHETL$qY=C% zfI1K0oMMu!n3~AF>U*a?qtf~}#~uM#2U#)t7j5N}dL1Lj>567B2Sj_Rz_Lm$eQiRQ zgldBWA0=#j5snoFh4JU&ttc_jG7w+&DwVl!QG{(Ns;1QILDx~+Zui6_>IY2qFkfsH zX0c@(3>V}MGkEJ5bfc)!zn!k!EbiEqX%6uy`#G%1Tf~2UgtP)trUtih9{`gEviXwG z>^FDbE{;^U(jNuJJM)_OJ0ob0kY*~u^Bwhz zraolbIy#W&m31c#5b-HZ2@&_a@okw@#sl^=eOO|ve+TM`S;k*~F-qPJc2B@NH0+G| z<3}r{4E;L!II*Ao*uLrCAC>np`mTd=t?45{0V`omf;sMdajGfxMG4_5Ologbz8vqV z3#JtJ4ynJvo22qlOW106q5xXjT_0+B>!F4G;G*`+u}YJaRB>rku3CCa3cu}lr9mdS zv@VB=9WPb>OQVwahbo&Sq05UcRAH}zeVizcTv}vQOsPov9Sfr)5ypw>5BQH-34~$C z(rJ=QI|&&}(e3Nsk?jBAYgrg7LFq?t-|Xn-F{jzg3O8q%4mlot)wUus?w8*%W8u_E z4A=+@KWO|O_5d4E$O!SC0E&^PMM?b zZ8n{7QT`=^HJ&HhwlyC%^DrXZ>IgEH)Z#;*BZ=5g_XFx5QiLhdvlu0u{zU;7%*0Qh z;(T81u_5*_R+m;6Wkm(!yjzt@#z+jbi;twGCT?ES`T&I6{@i3o8o*Zy%}VmTd3T0`|zFCz_&5p1aaz>7}= z@1+gSc~@zd-w6>M=X&zz0#z8QO)LKjTt=((V3kSly*Tgpismqj3TkP{_rGNW8jeSw zyD)vDhQ8RM;;|H-GQNF1@)>z~zjy!&8)<9C(QqeZ^%Q>qGwmJ{YpOS(Jyv1FukH&X z+a$j{--CUQ5z-xkJ{M%Nr;XU1s>9ZzE2NDy95i7!-gVfuxUriU&(8a04U`*7My-jf zHgkt|C}0f!$*pVZg^%)n_tf=2TPq%2G1%mgPr}YzN&&_0j8c9ztq@Jm`;wuBX8pdqkm%+{H?amOU7f;PLFjC=pwaBDNf9@Yiio)eMxX{aK zs{Houeruo29XQ}3ccROSTeO~;cmh)U)+T@>ObUP;&0V76XRW zIGy{|#Y@&9;Jzo-NM5M$$+r5K{m!C^Lrmj(3}}Sk>Yazf85(k@?;;w+O&zz65GQZn zU#x8t`*OC89^4)RXJbEWJF1?AD|HQBWI*2<`kYu>znFSU%Be1BJMX%I_JbDvViaO> zTLzs&KV?8eQ7ABNvbw}!7D;^^@NPRAWb(8?Kc@^%g+Vx31<;kXGCjjrr4ROy9!OZ$ zf~>HE4T^!?iAcslQH&wKwub{*0tMX-(hfntRBJ^$rRK^BYz= zOch<~4_BTUCB}PzD`P~GVrVn=?fD(YewQn1g$41Fw>kNZSZo}P2Dck%Ih2xqMLYVI zSKIYUk%V|B9Q^b$?5A);T#56RmLBh^Pj+D?9t`<8CyhBkHsfZ&9byRSg^9vtEmt{Qa12Ze3Ho)M~e(q(Wy(91QHxumU7J14ZsuY zcof%g1myea;~`e(1S--()umWOIx?9*6M`JEbg@?XM#PXl4GhMS3KXg98HWyFnOuFq z30YN)gOr7kcr!>9gZa1XuU3#fQAAV{%ay{I+iRkt{^=)JL;8Rw0ub1wOXeUTngJHW zWnaTe-|i+m);4inEaClGQ5rM`i`L9---Q0`z{t1$_uepO#+!z$1z*Slv)J3%1tIz| zRafl-Hiubr@w}b=zuOtg#qz)ekWavzQ$7K7>oRNU^`^USu|gSoxVT}!wZe%(3rfjD zh^lvfm%zlEDdvuxI*%)Prh?PYQW`QGhm$fSY&At?#U*%<4j0576LeR5^aQ2|w<*fZ zp2WB01-ZGph{Ec{KkgDurfED`S$?n$yGP}{$1e<;eYs`TO7l!5Cz^W48 z>QxAXLwGex`CTaHn4HcsY2*!hp7Of~TAg4u*2EKc>#+HhHGxZF=q=npFT^X|b>Ruq zeDlMMmcO8?s2epOX?`M`SA)K5nS90ed{tb-n_1(>CDDlNcH^nUXT{mnkc?R0*WA3_ zaNn?VVA|kZERnT*D z{zMlOY#7 zf8H)j1xv^v^2!2s=tYf2wRObBqUD1^Q^ zA(P-~IpL&BbE>8~u=qi7k+q2@HW!nniRIYdiNR{RsXuayvanP-hhU>5XuF!E74?9C zD({V!D*Ir1i$+2(5^hi+Wl#07UO6UV_$x6M_&TAwFl%BNM>LnFTx0X!EI6=1kL{GUJa?e9vQa=UwjR!E>3G@Vgc{7tE4=>$u@p@X9Tj-R6 z?J^s-Sp(WCv?+qwD0XPa?E2cj|8C*T71D?sKi5h%du+8z=4=X+UWa+S9KD1ReCwSh~g8{wa38<#>wrQ zM>it?eC+LUADL}*3Us2Qc^w9Cje1LlZ|vZHmP^u}oF@xh{^rp)s2g zewBm^nhjs3*4+suo0UfPzsQDTfjieQ)!9mdUgZsD zH>xy}Rms8l($b=9kgchaAwMNelhIA&VPLj-?oGm)XJ4`+ZcQ({MVEKl z)`1mROu*qCs=VJ{!s()!|gCdxnTFvEL*^lkqqS(+@3<{`fE8`sTgSvrRhyT=tO_TaLY}}`ktQ_ zc97I4(0B!GD;o;+^4nM%8@UOt;Iq0Z|Y}<*vtWJibiUuWrwr7NSxpTcA~;;>vR1WJwOgjax0@bDN%v6*OIV zPtogzoH`W<*L)>`%fNy;cAR)XN$EwgPJ=1Xii3B7@CDQw*`k?JkIaGmLd~2= zEfq%9K7xF$-OuIcYZ3U{O=-9)sTk3TIv0ypk%*9?Rs^aiVYyk1_a((LtMcQ{KyR+v z$0LtZ(36Siu+9DHw^c04s5suM>LqeM))60+@-9>IGx*Lrjcf0AW=*-=p)vuolmoxO*Ds{WwHP|}9@d_WqMnpieW#!UmpWWJ+#R!(%!oLD)WI~o(OLQT z2vV%sO-FEkys9Z*qsafmXIetC18|G5aZ~J$A0>^Llt%ca+m!qgB{uwc-y1Qsg+Dmu zq@k4Ck^j2{buP38OxqZBGW*<2!F);9_#Hs289ZV_RTnWJR!6}jodBjm{ZKZ8=M~YU zjjK{enS63w-$#4`8nUiZC-oiJu@DVpFg9H7ZZ6=(fhjMYGXDt5?+s$Rq)xJdALz3F}`ps*?x*w4z2dHT!Vd2cnfTT^a%(f2q{xtA~rjr8ABa29HVpAJ6 zX$R4t9KRF`_W8TyGwaF078X55x%YWs7YKKwD+{Zz;!jYx4L z{vb%D7nAtA3{8aYWTcMg9?~foFxKs|U|g*>iH+k2Q|d#}EcMXeam;bAeT5lN12}0~ z@Y~MODJL<0#a`7vgS@mXZhNt8t$nFoIK^su<9FqY2KX^F(zy?BMEPjnOoUX1^D&@s zh(w9UQ(QWXqudgM#EcSEs7Eq$|BGlkNqH1rDG4uo^=b7-t4a|c(cxm@aS)xwk;~~p zIJKibEQ4LHL%da7U8&bI*D;8Jdy3_raan$JlV1Cj1H8xvfu)5lL_F4_1O0+z$e4yz zNk@!Ua#l+NPRYJn{;A+H4&e^kk$|<#2WA#1f$UhY=oC=jxfXG|Own6QTvERm4$pdm z1#hfb1?<bju);CFaTCI=Ot9ANNn1MKvKHs^TO4CvqDai@go2 zkPU*`h=~`S;L<&+Lc+av*M!HY&>9G@^-LZ)`DyI}jzY9$Ktz)Zq^A_QYQ}ip>aa_j z&A0+*o?6Rhc-^-GneX>*$i_X6tudmsD?*~p_Cg; z{>0Jd-!Zjo3+qgMi+)wkiWDQkO{&g?4)Nqv?Hb0|f&C20#ZBu-a(98!GM2bL;&zqT z@tze>e|I22Pu`sJ=GPG!9;PH)w;)aNj!7^Pu_k*NJwXbZxXM!{t~}(4?>H1Va_pk{ z(H`j-(?{LpB0OUhmo7IlKSD>OZ24Bx5esmnl`<7G_s^{rAxF92TnZht#)7g@lw-0~ zzZ{??6_oe{XY7nF=saIhEXXR~K)0+IshI?rU;y~aSHUD-`kMa53jKG3OD25Sb=eTY zoAku#i5#;Kk6#~J9Bbl$i`Jyz4IWz8qT&ym#@gD)c_-VZt^_KMFNon*M2-9XZpIQa zTKQWysecQpJ^K0%^4{eMNtU%SDP9r6`eCreP=_T``}^W^=}Y;Y?Gn;|_n?9wdr7i1 z`2D^o(C~t+-|$e**RCL=G^DvtRWr7bpLp|*@dL+A5gp5YU>;pc@u+qr+}FqvmR9|o zI=2+Q^K+)GeFj1$_JIb-ry$^_TfHkypU6(woALGIdcHbMHn;m;_9=YCF1vO|XM8($Y z>t~TZXY5Y*LNnJtQKU@jPfPg(%Wc&QzGrf$Ih65SBx(hNc#D+9rp1%Wj^_lML4Vph zNHoW46kHD`m(BF8^>E@*WJM`K_Qmx4i^mxa@3QUPd7U_IRuky4{dP7fp$$46%}q(Q z*Yw_a{guU{k5tRX;~~qNLeU8&oSX3Pe+6c7?%2mmGk;jvO7m4?Q=NsQSm(cRCWV7- z{3Sf$DJC++$_hSC>mcZFi_38z1>961K??0J?nr|bEnE@(c*)+Htvgw=r72th8Po{6 z3(L@8bMg-+PN=iPaKS!rX( zxOA$(s#0nT_FMe44b1Wpm!yy}XbxQQ%v$9*BRZ1;pPh%=q6jFrZWY3eLIfDQxx1H+ zv8CHn+L0c{c5Hb^9UldK<4-9Mg%Z5(f^d$_JimxShRlD>dlfJy`Eu*&kJt=0e+pbhV2(AlX7u0sj52$TCnG(LK%~&!5 z1NPA~#Q3zZ5}cILj^Sc(b0Fo|sp}D_2A`5vk zRxb#gmf0)KT4Cnda#Y)`GI1N@tobAr<0Bz47VBKs#|6ehy-{AS!TQ#MialKYN#I=~ z6=RBm6|zUbJrf=1xh;H8=gTH1{DMuiEBvN!#zEu9sEKzM#|~%Y!XLt|+%TU;e)5|m zzfkZ!P9(bsV?$tJyhMHpF0bZg_~ZjZ)0k}Em_-^Gl%QV9AdAu$RU0k$AITn;)+oMI zVtuHdg>t0VkjVdcBixFKiQi>b?6b%djoAauY!Ui^0(5%fOLXwSKk?|PZ%{U4Ci2tAi1`aP80~kNixetW_yVI6 zldI|UKoL;v`Y0E+RG+dOM+~Aq>EzrnOup*ozRZ=XR7oH#gX8Q%dhzR6DsbF*ZIHJ# z6Vjb)8})RHx5Y8CkKcrx2FUFh5kW%_Y%XE=vc>eUM7gsN#9eVaXIN(2WKSaKIHH-J zd|slMs)(9Uj%Ce+U@#7|s5SF0F~gfm)cT}YAqzN8=W-U#~ z;g;xasY4inFbF^O~daR=K7w}BG<)l8#Av(MdCDi?dBOje&(fE@A zj&>fKL`cQ?Ocg>{#EI==!+yibUlry=q9>*U=qY~`9<)?1GI&KjRjn~!bhHj1RJl=8 zUy(h=BlBUNE9qq#h6#|}IOGzt^Hh8RMa8dA4l6s<#@WMBmf;|ZRj>;K$>>;(Z29#x zZoQ8tsP)p46Ap_4TqEuT$46GZyX#4`?IX#7m@1}A&~wf`GS0H2W(c@lm;pCb&B6|E zBkq#>oyVw!L@iWlxW|xB)<~mo->FA5n7-HO2!EX=dr><4bPAuZWK6FRKqI3%vycxE zc|zgxDu?jYR4d$$0%q(@k?l^M3vq^6jIsE(>uSx4ODo0n7D=qP>R(*au_5`Q2?6{R zX}z~USklJlAh&Da@cUJl5gUqdTooBYJelp_`RZaT!WZ!=Htme<*!Kpi9{H--`yoHv z66fP0-8+R(EjM&#I4;X-#<6LScnV6nUSMd1f>0MSdvP#f&qvj@ag^PyQbo}H7J+rQRm<@PybY5`N8DB|P1OR5%Ux zXee7Ykc}pav<&73Q$#VbvJc7a|IqYXWT$7flOs^MJ$zFl&Or-D=>2wvHtd|wAp^24 zhk+rdq`xO?9HC2^!Nbk@-wmkj@vs#%QXDJnY$SEn;n|Y^U)N&|_VpD$lZum$7UR-( z8RQLPe<>s=k4*?YL!=))&|Yqu|Dd$9RIcS_F($L@i-m}(&s8C*Vx34gT}HnNE>Q@7at;4Z zLwPBYCH(M$BD>#~VSppJ9C!`|NvoilS(Rar5m8Ixj_H1q<4{A{02iE$}R6hj#4KlHXHJ!SgUzQcl9d=9qe+F?pi173Fs+mZhUZj56|^S5|_21UoyBo&B0B>Aw(?HO;+u&2J3j0L%)Q0mjrn*T3HsU(V>vA z_MK4~Ifz~tRe@wae>xEd`8D(5=H&3U$=7^d*6Ap=^~WB^zBZYXe}tJ)p$#~?x`OAC z<<36B)b^f_ad<>Bt#(48%Le?gY`?VNl}k{QWo3w2K&K3s(g`I%IsP+&vi!1Zv%71n zjP-~*PL80uQylx{qY;EC?5RjM30Kq_IGnU2mTlJ%lG@gn-dkGjqfjx0c~gVrkpfjI z&uB+i^?*5r+L*`cmVXnnEV-wJ+Rj0+dKMP%j@3R()UX{szX%JkxKR>!J3+@yW^ zg>;v_G7Yb64f%K9)I|W3L zba_)cs5rLR){XCmMdR!4E+c;dQ{91)#W1#P<~2}-Zijb1XQp|uE}F;2r+H0}U2ni+ zwD99X)W=kq^fp(KXU*&Oq8CuJp*SaVx)iQjua^Xu3xEBj{UTP*_um~tbHPo24|D>3 zG8f@S>Pz=NGE-v4+L&W)Q}>VNF0eaZ{qovSvxx`P+E5aLfe`O)2@~B|-zFIec;c@U zQxZ`vSa_3``C@BhTrp@0L49K~e@D?NopL^`a3&yRV=6J`1?tt<}?tI|8bY(pQvw;_QrU!7i3e_aOEIi3bs)KXECTVbgkQAp)XbizfKsrjF4eC3~MO0 z^*Zo-<6n!Qi<}+m!gC8{4r2xvY`Ui?H*ErG1uPwUGg|);zvF5L3)-0V!=({JgnX<6 zYQ6P7`%JW5k~K9v0bOR;8nr_^Ug9f6sy}+r&LnNm#MN$Wg}x&cx{J=QqNA`q)qqgL}(F32N)NI zIbtVUyvVF?{B6f#54m7zpKbR~%%{Mu z_Luu2%sbFlJ;!HWN*}7nhkR>>)4R8-Jq2gV>Py_W4W;&?P;&HdQ6d-r2O~rY;_1fGC6$+sMXZCZm{*4-S2ky9Tru(kZ$H> zw{F5^PH#_DYj0bL-&vxod@R;czIqL0{Xr-c;QgXRhAw^_u_#^xb!sEyyCPo+j-OC< zSJx|#nWHY|&;_Z$4f=Kx+`D7_@ zFK~RBz!RZ*tqGY+FDiJc8!eiQ` z%fVG|P}euEmgLJG95FT4Jh?>qNj-47AEBs>uYs$dk!XpuAXUVD!`iSd$=hKKbaB(p zJ{h=WY1}CxN{;T-((D_;4Of`dMA0Gd=K36c4Fh~0L}>G)+tKYQ$P8mXBgAy-b^7^@ z@SK`7)<>4Wz%6{-^yqMqFe_hB{_2E7_VPEG@26Ixv-e4WMi1n=WzA||?e%~i@nD}$giNA_qx=0gwLpu%)@atUVz->k{c`EF17OkNH% z;f~%knqj!Fj`|G!u3Cy4Kr6|TTvdexaIMhDA26M`nZhp%UjeVZLeKT4>#H*tXWiz` z9oF*Ne4m8_H~ne>eo!2Zl#Qu&+sUu^&DXsfHZJ{zIC{c5jT)q; zsP)nXCXf;^N>u8&wvFoP{TSdB*HlZ8<0a#A2#&?s~L;R{zHSPr_|3GkS%S!;s!$6&6j$5vh2j>NXJ zdBdk!5Z~vs$6uV%l8XeUZ(GbOI`8H-gT@$e(vxDJ<|>~kN7|yQ=(De%SDRM`zBA1u z*GA2ehTMr74GQwJ)@2U))t86_+NTj>0KbF(Mawo#izv*Nv_vJCQs@^AV}oR9by8BT zGfhbK=DB@Kuc}4AY(4^2h^O5!PKxEB(Xh6C+Q^PtFg-CjGbVi)*N>Q25{IJuJPei2 z2qH<=`_PYP8SkuGZfhGS;DliUPI!onbTQSw(ih&G!mDp|u5JwGc|TPh$>e2>A?+Y3 z+B`QZEzSFh<>O#~EjZU2WniWMuqLug7ZFTaH6h=zv&|H*$O@8gyE0XYQ~mbw3DcEl ziTDL<3i^~FpRe}L)r|yQqk?>zp1I6E1kOz=qC$+36;DtJ_jRhppu~R#?Sk_)b<^sCgz}{co`IHO?{n zH%MCDH>Q4+%G^*vx}p(5VbyOu#8>@{F4RVKx%_z%X@OHmo`CUZBL!H~p2!aDFZ&kS z%!f$$QPia>ieUdlhA+7_kFXVaq{xb_h6Mx;#kV!9?K>($kyidniNUkAyAIo9xZBjl z8mRMv(V%f{{cs@WnumZ(UkyyNZ=>JmXLh-aH6x}3G}?p=m+r-5=k6yYAdPwjj&d~l z%X(TV5!MXV`3!FxjgF-v8`$pP`emO=7Y=4(RZD=?&FF{5TA|7!7f~!5JJe9DfT>2(pXkWFOYQs zTG}q9TK{&NXIw<5rSf6VHgqu+^ZZ70hpC?JDxDjY$Gj)89ucI%n5_|dv2r>+H1{L{ zQ+a*T`@*M-x9F?bCz9iMU(}OuK9Vx7?RojNoO;C!I-y?~m z)VpXY%FMyks=00Tb#w%c_HF(1V`ERI^a9QVMI^4aDcoK}%*lb=#4Z$~DHpenI@6ekm&D9h%( zAmxiXOTu&hZ=CjI{zLMaJWp<0_ z`ZG?-=LxMy6L-bo=Ly>`MoBLczC(uZS3OC#?G=#X!7GYu-!+Y~Ufv#%aKhVjg!nID zuYl>bX`SO3mmeMTACOWSnXhnic&OCp5M^gFAEesd6G=;EDIM*vD7sw7N5ri9DGV+Q zO-!HCn!8DS_z+4dj%BeiiRAIW^idf{?j^=vOGw2Phz&inb74Z#=!tq`;zNzeJY>d* zPJxgOT`EoK8m7uVvr~_o&rCl3E_IkNnSP;HDp5-MkiYG8KWRAoVyIPFDOc&&@8u*; z6k?-^`xA#<_O&ao4dbW2?=|2gz2ur|O3j$L&Pn`?9P=!Au*aY^RYQhDzXmlsnND(~ z4nKKN(PWcqKgd(u>3$c)V*ZfcR0Bvob!^VRM$6ElLiQnBa}ScPdac$}_~EbavW*lN zXMaw+$IV&c8VDGZn?6h2ROZbzHjW-k^xMpnZi>GfX1x~S4J5y=H|?IoZN>K9k0$cx zfPtC6&+8*MCfH$ONorCsK3SY2QBQQ-M-Dq%ePlQhGu>A|^-`a@mdNNI&%E5L6*afm zXi!v?8{7?+Og*9p`%;tGmC9=lfuTi{GnAshTljw1ZhBqo&T(SWPjjR?m<4BEKebKtJ92xV|b& z=+P4x<;qHwcg^2Mc7;|M=$~v0KFgl{%PDbShtwxFC`cJpO}_+9f;jU~j$6lai|I16 zee&VIa^INE>H^ekC35K3^`%#1B|Wkdm>iAXXII{@>h|0oA@x`6(M?qYH(z;}#6kR2 zmCjb|KA&Sk%t`n4iSnbpG3s2Wu2RURCEtH|tY9zHosQm44-S~y^0j#}(zwL^2 z%=1$pMv=<_uDXl3-GgO)enUT$>8(nmV?(03X;uxREhr$*ivF9B(bg7JQ=b4^dJLg& zc-uYV{bhdoR)?5!eT{;oV^dIB}nusHN#eJ z{gB>VUo=`I;24G5s*2tgH~afCxGKkR0ZUZt4C@#_m%9@02XG`_LQy(u zu)55zTn?0P^AJi1!PeBNk?i)V$Uz%gX<`>`s`tFFxGh0%fx4IU>w)czRj1l}DbI9G z*>d1}WYDgj222YNJSgQ)_LB?u03X&68lD81fpT0d-YVb`8Vq){#OBCJkI3Pq^3PlF z^?b2jUO64)n}B8jaw&s$`blb(hcQ8ZQ&Y^YsXNkXXVXgdIA8kTztLzhb>VOy?(ai~ zwO_=M3!0;(1`Rn++<;mC-Y>SHzfhL$WX-?!vTwQaGw>UwXY{Rcva~qeW4Aava6&sH zd7_WQZKzal0&~q_8$srp(D6o?m74KdnJ2SGg05XSEv1+f)9Om+^W9i11H$724rgh- zx@s$zcck15$+y(bct%{_6iF^iz2|NA)y?=S~@Sc9ODis%zPfMVT#5*pdB6dJ_*QAQd)7C~1m2hQ04(9yB)K zLR*~Q7plz$N>mN{x%zdsFjd`l9?5Gm1;eT188%+-hYe z{SFzP6CkM7%F!@!E#+-iT*S%~AUq2S_6`yJnKfcBtj^{aix-dcCYxZX=>D=K(|${N zh`*{rO#ithW--XjOcib@5MeRnf%<+#5{^ACCnUbRy$fTuY)y62?lM>%RV~Q{O2;3Fis#TY z-rjU@dLRx6y^xbm#f8P{w0y%d2s5=C3#KHaMJm#zV~|~vPqmyA6PT?Xmj&>{X5&MA zrW_N4*%JG}5DmUOzlB{0uJp}nzqqaohV5>*q!L`k9-hxWB`R*aJ>}^xo@mAVB&niC zmUR5yK)o6*oM~(-i2M<7_CPiiMWh}(!eA$9qM8O$tj}=IF?09Jg)HCm9B)i1pnvxJe$;RzB*AAbPKiayY0HDuG7cNnfWPm% zST1|*Tu{G@3Uze&b7z#LlD{i+G7nC^uONe69YV?6nfUPszbnzgAUFz>Cz>nR0!>=_ zTTSWHOpHX5%OkIs${@`pDD$!0(OxI&?{41N%DSEGd0iq2%-7gZu_bD{%!J%&*~(kF zuRwGSA8l?K;?A$c8=5-8UeHJ)d{!gw`mu(TP;Cyu_?}Q<_V=*e4bh&!TT#Bg^77tb z5{oWeR3J5+f1xk=$BUpw``c?UeE>+Fo&D3BU+K6HA;Bg2l(wr>+FqNGWvq!CrwE;J zh$XbR5G4`>1BX(wPwOTl3IxO_%E!u5OYAkxgS;02Po;Tb~ zq0|(4CMaqPLUh7|Wqj->sO%F2F4i%sE(m@E#ATqau=JAYFCB+T?=v?^#Mgk6HuDEs zV|#rPZ$j0wq-Vtu>6CdqB z?AYArM#a|rQ7Yk*4SL^QBuQ|%2&Rh0o@90;$r|BX$g583ywjbXmgqk4*Q%OSGBQu7 z3;k5-NmlqfH4zm>8##@nk8PtT@?+Sd&x?+m ztG}tYc5qUNmaa@yMrJ58qM4;+dD08Sm9Utd;eNfk_r0hy-dOjRlh~CE<8`uL?{XDc zLI1`C3CM<|X~l}*6EVX3hl-L{=|XHS+`KZwsvF`QCK;~>cQ(FUxv8ln^HjUd`XC;G z!+&*Ps23)tqhTy~@O%8ZFTne5@$*psnu zgmX^2v#x#gwqz}VfQ$y;?L#V&eYSN-lS_~3j`AE_;$$D9Q$?sO!p@UwUNOFpy-Zd6 zk^)ugb}{yq_D1x{;oB=?&4KtPyysd@8+&zzag_Ypua?IlTqVlFDzWs1AQt~lX&LrNi)EZ%%@!Vsw;RWGTPvd9V z)=S5?ei7FR$n#_4KCK%VbDKnO9}divoX-U8Bev4`@nkCkp~?GAgpY}bW!&tPW?VW* z@&{27BqZ(hfI{EK1BDUbu2)A+_sQjV)wZS}hKfSSC4|GU#Ct9%k9Na+aNaH{;O5E= ztZ+B)D6{Yw!1j<#Rj?JoAyb`< zh8Q2C+`ci0;`dtS&$I+fqQJYMa(&JS|25e&?T${uPft=U6F#M62hL+=C|WRM25#NGOL=tKYjTBSyxENeVIch-CmSrO`Lozg$_J z5=N~AV0Jg2;V9U?iNQux;{I;SAEOb$>naPm#@6-6s;zfGXtmL>D1gm3|A;#GUii1g zxJnqv!W z8_mDJyWTS0(Y+YhnsnNPu7yn9f-bg#_ynb4C_*!~jw=9ZBj~0O_zG^D@@G|Tkm-iA zvq^T8cPaC0s=O?Q8((=I6n85zqCc1O__Q_s%u0X4W)Jv`NXqjd(_|bnqDl5Gx*xGn^Ma^ zv#9I*6hp?{&ptl+O$?xD*}T`V&H~npi;El2iB9{6t<>azFsd;d1e3wcbnB-o=YZ#% z$|;M)p_Hm^+oi+23&>%$`ez6(2!15}W0q;OK2yO85=--PL`XAq-(TJZ;X@r?y6c_n zQe;-^k8UT>`C#5p?ycK%tXv!;vicRfofn8Y1n*t<8kt;i1S=!uTO4;7vBh?7;gwq$ zyr0#rPE96?TBhbDrjxt8Je8sajaSu)SUGA*-+!Qj**>*63Kj&L$VrJ|)su*{i|Mcb zEuC3cu)d6RsW7Ifsxmm5Rvn_#kkb6PWMZ_fU}9xOCgX7|m08q$YjhdE@l@3DVoe^t z`ZI*zmW^#L;OQMwM$Yo zR{BJrzSAeP0+i)_Q91J8eWlwq;?BcCn)7pgp&kq@H^Rz1y`nMbQjHriM`itHm6whT zj7J*J>wlBbpSTAkMAQdsa@$uo)* zX?5@J!TdQ%^U!CZY1ekgq#jDbWo5B0<9-?6Vx^MI?)TjkKP-yye!{Sb&(%btP&pqm z_kEzWLz%~FFX=S)o>94xvG1L|i7%;fd5|rQQAoboEVj(>`cD~%X==wC-L#oN=7@n8 ztIe@#{K7zU|E`N+CTo98PsI7K3YfrUPB&(iAwgI)2vv9-PJdKm>=3U z&nY;Y9-`Qse+9Xfcb!kl)el%RacPU&ef>h2&J$~zWAp8Xi&)I6uMJID!|;rl=u#fq zwQtz%3j{*Fr$q!bC9@)mw$dsl7-`fwnJ^yLhQUaR_Oa$4<~u5#bQ@%)#isRSS(nrk zrk@L4k_#Lxgv2aHs{&Yj2$F1=$^vRY^0q0Zu690&-Zio@K!B`z`(wYm6l!qz|(z3&n5)+FQlRF`^Y~i#w^TW)^m`hG;Lx%^)h8 zTBIaZikGyt*54)C&jCjhY1d$OA%GN@5RXnqz$rda*0d0!julTk zs_W2-GQd7*uTc2hvL*3tq7sKefJ4(P)JW@T?@ALXX-gM&b4ee@ip6u&nM7$Z`)u*s zUZ2_wh8yJ)^k1x~qWNOy+?Qfk7+IeN&b%)+urE83cL`82Hv1G;sQecnDvbN4mKeWW z6sfoL_1pTX$M&7oKuNcAx4}``>93(Ly9iNQ!sHksOj=x;ejo*y+g-^RXg|upxY|yE z`c9AbfWI)e3?QQpStFC&gKi(KQf{5OVbNYc&sSn8jbc~8d>r|L6|ugKX7Z8uJH7e^ z3FV+}jF~pRCXj>afQeESD^%`GP@Q>F`a0LNH)UoOh-0^LqdEoa1VkV9h=DcBXG??d zaQirC)G=#*GMC6JDFP^>HOcbW_%RWNvp<}fdc$>b`8UsR*(^1(Q2xX^hJ>K;0nk8c3Bt| zBYpZ+!DFuYnC`pek5ygvu|`F!(Re3WPW2COIi4nFBP_#sMGw6W6PLn`4@-`CRD9P2 zxYcJ*g=tqUX1?6*#e)c1~mH#B8S6FsdlR|{VIKEbT) zUdCl|-?hA4fYc^;h!JPf*Z`6y!BoyUvC@c!y{w9b@CvuK%A`kjhT`qym`Rwq(Dx0%|YAV7A!B8e8npKM9mWG4nyL^ z&QM{>vzO^&6y*p(I5YBV=^K?;H^%zNN-~rxr5BWw*%A^aRU{0pOzgLEA|CA&?3IZ) z!wB>N{ht1}1*_D|HSD##n+NoAU_z}FJ)^GQzT(B-qZ+YWDenBpNWCj=9bNQN9DVcWxQ`V41gVhxlAg z4APCsxKCC`^sDxiX%wg=-hmCv9n-3+3T_P772cB~IY|yB8mY@gRSd*20zTXF2d(1P zmGl$rniLkZFFD~kroRT=>fV-u>Hh4O^-VH?kXu-@hLGRJuh6bHW))Al~K zvQiJoukNWlWH?Rq#lPLf?EQ?(3cYsVmQeDE(&3`#CZeNm=Td=|J&5{aM(v6=Q;$Mv z@0;4&X*Sr~<7R7--cRfajxvK3p!)AUsKqR-q8LT9e2X2 zwhYuX@9PEIq`Dw24(Ie@67yKdT_EMz8ijHf90B*4vhylwzu3I+1WDhHEebcNW1!JH z2`;{W|3?%B820$rPWc0k@1qh2P0+t4%~%ok*|Tu;w715dDo0Nb5BA9$V=jo8XsI6j zpirH3ysRMRE17=Sy{(t;0biyIUrB{u?_Zr8ikF`PJ~#D-^=759Lf*S_fKqfOLt6d_ zhC`n`lS&kI%R|>L?pq9N6I~trAZYlxAaAG8zTgXdD%lswn)OUV>6KzRhl~AQht$4R z)3|nf=|}p^qjs1-s|t!ih2>dhaAvW)Di~u(GX<}=;Vdj3yc{c1m)3v-z?S&u)jx{1}thBct&ffk#3H)I1nlRkW*S#wJ z0wnaV?PHe3t(m$Joks9B97jd#;t$aJor9k4GarZcYTRPl=~1zy|02#yO#!ny$?1_u zbDR(vP3Ul_>@Av#@-{GiZwVMF15tD{UAG& zge##tZFi97rsaeBp(6=0uv(nlCQ%~n_`cPvf9LSqgZX_S`G(|jiGoh0P|V()LzY$; zj$}9YSw}Wi6;dP>dUjoM0fjb7Ci+MeknS5*hpToOlL$6As98MOkPNj46<=FDoAtbb`C$c7vy#KOh;4AE1AaWKo7W93*dJ5kTfP?AZdf#3Sp_S(l~-^pM3m~h<3l_b4rN&6SY;V==X1JdnB zS12FITkFtBTBZrpY<=D~e$3qAff-qn)n4ystA#2YmB3W9z$;wrDl1X?L}n8u(k`~Q z9947^&e&npByH;a-T^#EY#IbYDz!-EPU2uLllzBnB#TQ3`bQtuqbB&kE~P=eu8tgw zxCoAYc7@WMBmL?dK=(8%86EK^f^XEfq9A&$M5N?|=?1I3={BIFz|ntB29KMjlA-Jy z07Jye=lcl#7~*{54K9_vT0k=IZMAS}x zVc5K;zqhk7f3yO~RWd+{z>HS67RV#1S zq;IIMnT@tv=g6LP=+p{i^Eq~^)Fa6krOA)yuZYF~nNw%ADPl^76n;GIW2vzp;qNeR z`q(70A$&=Ce&a)NZeIT#gBDt2O)p-s09bfTd|!8rcrdsh(_2!Q?LqZ7gUH3PmdNsp zM_b0BOeTO2@#B6-RTlQ&FN0Z-DVe8}R8i*yKV0i1GH2OFl15|vbeEf`+rQLIm^KsG z(ok6*b~J5|?CrM|s`|q>DiQ5Fk-j}s`o9FH)ww7SxL~VE_D|(GXE*cF&|93;&fl#% zk%o$XR`E4|a>1E1UM8^=$sLL?Mm-4DdPv{lM^B7MWohuHXNfm=Zs1jvkCqs#3-fw} z;?eD$g_^v=&ZG=F^EAnc49a&^MCK89qJ>b#yPcSGBfNZZh355M>P>eS95&$15SO?v zdn<4El@N!Mi&O{2-|{@-DmC|s-HC(DdR(VHOiQtxhmV$C8FTKY6@4)nASm#nE^D1f z>Ht;P^Vue!tc?buJ(8W8Do{8cC%fCcCBo;Et(U`a4!}o;Ro85ygGZZ#_}rnKxtr_P zu-O2CQW%0VRM^~nAeD59ZfE!1h!(s{J_RIFa?)e*cbEb!4W}0(%SWXJ9NX7aQ;0ji zF>H`&z7iRvT$OVgYD1Z?nJ-Y`_per6DL@{*{pi<*psLH{;F*3-e$+VGD;vW;OpH^F zBKke?gXScSp~=+FoA(f5Sc^TVsdEd82vHJOxAsfOw{7{(j*{{xGQOSH-_%bJC2fX_ z(Nf;6Zf#S!w>Qkm?>GaLp#Gi@k}_0E#c3YRIEPaEvIoyEk*mKeWWFy_zEQc1ZqutQ zFeA?qhaM7z>*v$mLc#M~WZ}uW7eZ&^(49Ej>N<7TU1@N`FU9q>^d?U<$ZdkGL@|v7 zE-R*k^I-F+l9lX0MKclQ9;5iaC#rAPZOi80I}38%C=665b?a_9$x)(b5N(xKG|!0& zd_nQT3|To6&nfzF(Pk-%bcTiI6!mcFZ1jg=d@FkNg(X!_v>z9mBK>`fqQ^09L$#y^ zSdVI*b;C^&3VMotJ)Aj1otN`c6?A-4;>G?b4(cYl*3Qv@0;a-Z-Q*UZrtYM;!kLL@ zHmeAMAbkYYAWTmy$o>0V)Ufd{%<7>yHbeVT$Min)9cbF%|K9YziI9BqB<};$J*EPO zR0ROUYkOc+01)3|Gz1f5fiXIMSbbGB5;nnZvvP1YUhsf2Gu?oMifE{Olu{OAQ~ODY zwvc%_;G#m4=>4ojd0FTN+0p%`GR=VOqCWvd$UpXi@m|ZqNL~UQM@%%E@9H?vi!>=- zyJ5n?khr6xb^*Tm_cX_u-M?)h;K(P}ZSUrh2o6>e(P!f7VCU z$`y@2F*WUH3nDAyk7udV?RFq1Ze+?KfE4jr?7&K}?3$q`gac*IDj0$kjtikFmoXEj z9;;~_Rs^HeH(Gwcf-(7N8)B?C2B@l`k573PqHs6^l5)-$9wUr5P}D(7#*$s#1SVcO z2noAyC2c29sP`NFM9Rl1AX0R9eRN5R!-;E^l2_xn6IVZ0mBacVio&BeKMdD&ax)?a z9i$VY`l`U}=c6Vp*L8=mAQ zXy#vu{oFHCr;$yJcMOI1b`+DU5|~x>2kNii8JVs!_q{Cm@|SH@KybE++BrI4VAn&2 z#GBcFu+%41LXXn@=(DRQr3<5YDLNoyShI&SZ^X(I+FCm{EvHk?q#h=GWBm@lEcqRO zM)_g5r$hIK+yWBUD_not#NK&cygQ9nWVTZ7RV}j9Bfi=6CDf)5_RxYWrwf_Y^EhkF z7H`Wp=!&Iz#YarHj1M^{`HD~q z^MDF5D>vf?zGP5t@EuLqj(XwMyba+kdt)i8SdHBjZ zg=2xZcsYry7r8R~rUy_V$pB-h4L&`c>9`*u7fgP+bcW{98fdk9siYKme@g4iaoJ()OiK9+fg?bYz;&c{{j>77Sh)bFV z%3sVAK-CW5qEw`Sg$UI~_S79M(tc}nJXAf|zwrd=P6zqGeMG3@fPj-@_???gU zmok0~Co(y)HfZLus258V0_&R?rv@)~Utx;ToB-nSO6GjdVNy`=thLH*k9b^$l-P@*-h zHP9AY9F^<5Y9{~>DrjL%SIgCre?qsnLQ|3puuZ6E^fXp|K)JDk^#Le- zYoXmnN`@}4kb(metkkk7LV&tn;+UKpIP1-)hc30(d(m?ZccQm0i%LcTQ3r)mN99iu zWy=}2vTJ2|wIV%2if_I&{TdQ(N1>!Fb=Br$>T=9nKVW7o!#4=ht{1nBYQHHjNjiWm zY94ZevTFX89a5JCnk;&WK+tWvf>0tY>R3a?4Mxd`k4A?OtI@<{xZCC?hgacUaY@Fe zQ^q;HTb#k}Pg22^TC1t5nldBdZlcXV6_BQtIyZUn?a`mMs6!u4(YLQh{)>4`?!$DR zL{shDP9dXQ6|nU6Ny=@y_XfJU-~-o9zf>Mw31fh_dVz0vnIv3S9vTD;KPm~eeT)q= zyY&_>z0jjc=1*wAYcb|lt6eB~C+(be-TXvmmKY%%YBn~}Wi{F0$=e&^cGhTHA|6&wR?~C%eR6xl$h0KhTO?1)*N97YNbzdscNlax@WUsZD! z91g$DPHwk8c-k=Kp6WTHB#M@d4Ez8t`=Vd+<=XM(RfsTg4qk}4qehsIaSTqUCtb$q zlrEHfoy#J_xJv4>`_kO%GY1aC!EW!p=9Z)CtA}~&Oh1i!7Y%uggG9hiTIO-wTEbvx zgYp(acFI|krJZFEA*$weQ$1NK#StoP0oOD%a8)SK)w-qg4$4DKt9h+m8Dh_(t~h|6 z1bxjuZlJX-Iadmx=6P)&ITLA3Nj|B8iVYO-iLJPDG;Dn)_wL1QP_S>%{_fsTvp1Zt z^#zfoKQ83BJ`;H~y3V+f1vLCw-tI!03U)RlKLxQ5E|&Uh=@hz@eghIk`q)Plj1aC- zYu-4)$Yz{8C&PB9o`S*S$cHZZ#9{7Y@7^A1QcsaN_F8Oar1n;)iBCAccFwP2^ygq7 z6r~{MV zLKg}2z#*_5@i`7N!k4{0#SD~AqCO~Wbq0ihuTL)h$N%AEq+|$L6GS}J+ab_RlPR{R zG+2Bh+aY(Y5wU54gX>LQdc$nZU2P6Bf6ztvee`U}1WSqe`0%_xJllQY^*#Nb`W znB2&74>a*>G+#cOZeT)!aO-kKmlNy9&&38Wq-<8D3{B&KY}yl=is(GHJ9pa>Pfmw& zzZc7=GUS@VF~+2XXS2suAhV8f?6UC-{avHyo74?8o`|wVSx0q8BF-jh!R?z@f1+}3 zcN{RO`r}t93?{4*O{d#@VBNacS4o8+~FveOjzKI@~E)0av zGcInJ1kUfqihX+i9klw_);?HK2&JZRNgXf7_F3=UFUl!q$%p$lH@Pl{pv}NjZfako zaA_#6|9K9AvI~iXk6)(mFNVN)NTwYYAV!vrCY-CJ*}%l7iK5`HZZG)MdF>ZYf(lEJ zmQO~?G8`JcrblhiPs?kv+xPBgBc^DmO=g)wz!3z$uOGn~mG&k-l1CNDcURLlK+x|& z{E3-d)-6Qi%sOA%aYmEc9M?~S&L!NRso#{R6q0utjc1rw``vqUv@)#^$dg}{U-j0U zDz$Wb1@lQ@I;a2L?hQH`H_urvm}M>xZIo)$?03(5EoD|R)S(N9xE!ys(n~bj;2D*} zjoDnd(va6H{qD++h|teYLnbl5ldN)>$K{>+bS;F5sR}qYu&TZ#eB*feEH4y!8=8TR z-!>A$7om0iwJ;(~sBF$zeA;(>F z2@7M6!8}AeLT)T?1Q-cDBXYTJtA?zkY<6}`B?_39m^sHIKpYJ2FA^s?bn?4+&>gdx zyj?V*uLklpX2q1w<4L7>j4Qp*04po!LPU;{v{u52W3k;eV;Qp=r?wVTg3$1bJUx_|(~z^#)Y7#Q4C*^cPFwf-h_|Q#YnsLHVv~ga zgD|P#tM|Y*avwKrrnR_Q{A%Tv2{P>&=CMeokJ}) zcQL?e%##b6B>~t2?q@L-HUyG&=B9|(#TGxTpJRzzar@dA&XhI!j?n&=!JRkQ9Uu znntT&TjVyI}}bzm2rPrYpJB{dKjD7I_7Z`pXQs$^rtDT57J{tEPWg| zwVO91-<;uik=NEsi!6pxD|O4$Vn||ZYyQXiltIE|qp$2fl1#$d-+tN3qM~jr74WqM zbLL9VguhXW%}K3DhJf?W^7YR?iLiRHAvwioapYV?+Hz6Ip<09)5Gj^wflj#J);M}e z+q2-kO3n7-)D*NFQUgD8D@&DxH9DW=)zuvTRXgHoi}x`Egn8$_$N9B2`= zrsgt~l7awkhU=u<=s>;Jkzk)`wfEW)WOzA5XQcR$dDHcGi3aK1J4(W#Eb(;81oA`PD6W}Y6MI!2!Yvf5*J{q5k1 zKoMbgDZ*AMFs*yjW)^1fiw6l|p(xj;X>T?%Ty5QG?G&giUj9U|Jh3^2OL8=EBp1B? z;9p`Cw>p(eav84=AP;S-O`Wux_wf(Et4pB}sibN|#4@ZfnBfL{h)6taN5Y^QivO7b z*DeKt-I5OU`#Xj1{X@e_(*`*%ma_kON0O3o!g(ORJ3uc|9)hY@Jh*L{twZ&LVSO0* z8PU`~OL1M+=^|RLL#aK+#BHy%@wH7WoK5<6USukq{3BI|LpbeKOqkJsWSfzI)g&@4 zUmm);k>tJ6#qzFUoa0bJ53`->)*S-gdZu&qLImLTTs7ti<#^_z_b?$>O`8jWw+J$JNsv z+V`DW_!_MiWR_Igye>BM_NK6utV<|&YtCJY@*Ef`&Yrl}hUmnaz6WDO9^7YFBGJ?M*xte!Zpgtjyn&$)b!2k}m17+_Dt0V`|ADLV2LZo~8aC2%5HC zWlGhcolx4<=Lljh>z5@yvl||(=Xi^+d84^+D>>GGZ0FzhU6PJwR=;8ij-klp69YT+ zw36cEJ)ybtkk%XIT>RVNqIW!c1GgnmY6;qm=GVF`oqB5r;y#y^Ofl4P#x(85Ag7e) z(P21}GdWT5ROW+qyH(m|@APKt;|i@gF55ho%tx=Z#7A&CDdQ? z2S6VOUQ4Dd7Oz)5z>PP>meS0m{a{lu%#>0OEhgKF2msVi>GR~Rgkon3nc7SJ$Pr}G z`}91wNsfI6%{;psfA$a>lcLD^-);UvKeDAeEo_PNe<)&<1r)Q|&q5lz2PvYKYUZ>~ zZiq&~5xF=uQV%2LUbA6(xM$&spG_Q)v%g={e>9zq6M6U#_ZlZ@UH4cYN1_N`RvuQ^ zh(FKH;sgVuLp~wRpVFe?7wiz4it}}tysV-(?TbQrQVCJnOrQG>5*B#~ZkEmmy|fuB zXf4~QdQizz+Fn@EU6WGrb?qg?TN>v0SRnx4=Fqg&)ZZd#NDhhMqH$hW7VXmnsYvj} zhR(d>Iy!IKT@gcx+AE@VsED~r&X+umK`?$FZwY)hjd8nVn}Mp2b8pr3RmSy$t}?W; z=KyiHc`}E{ltf&KF1bmN^?k>6H}vk3W?7F4iTjx}ybhV|oy*x<7`R4$LX4*^r#*)x zbys?l7xkEHnLWz!5H7zS6P>+ON{^VDyVg41F;)OqJ%r1J4H5CeVpjN+`tU6*abfl^ z7hJk>D1QqeM!?{TG2X6!8-|`?frBE$z0t<|Z24#*v2QUAb#Hj0q-5bDOvAB8>7x$r z*0BcUlF?9mExmqvU5hP?_wMEr>kt;=(UTv~EQUo7Hp$=R5Q20)TC#GGA`lKOISMS@ z%%GM9Y$q~wB5}M6tn*j)jIrw9Z2PVJ-qnM6kF%cZ7{u0&M7t>QRp44Nrik1HsKP*g zwb;D_BTpE#7KL`&KI?Gi^7g8qE>O%UbNZ~P*dHJx3pzxLe z)Im05cvqLRelcYK`=}{l&T+b#VhTq<>FqJph z$L)Re5wQ2x^9zcRc*#>{gdyp**yhN{SA-$EmxMwCO(b61@1S)lFIj%Yu2x5cM78st z6WqUZ_-!6*knntTy8#SF%zHI=P9J2x`zo1UM7Ql!wU42r!GwRg6*fKNdj(+*dcQ{} z-3?cF3z5;1!(%ue8{4zQks>ZTC95D)g-ff%TDE9bJO64&Yfja5yd?|er#$lcsQnv&~*cJw2Q3%^JZ*G(%pttw*$MCcMrfKo& zbMkrgsA@2#eje8vzD@~xQLk(nLAW1kSTEpE?>?l=DNBJ-3Pr}ZU0NdZR5JuLLULnM z{yY85!hYPP)ruCfIR@#A21}1r=(Zg~mGUre5wevd^6cqHtOGmvF2*lXjjl+$$QGz? zjHZA=uY~yb!CC{o*De|#ks}s?1m6A;0F#I(P80BgQ$-n{>f|8@KbRyORa;*lW>Sv$ zs(t4g&;m$Lu{5l*QS>Q#3ZPV*GsS5BVVP%IJhdOcI<=UB5gt< zdg%J0=Jp^*T#=t@cgV)Pb=#P4uU@LDVVq;^S=Lj&_+;sKaO%blayJN!Hr}|R8|nygVSGOQZAFWG6$~??Bg;1=y7g9F%K9i3rAi5Lc;H34B~l``cTl zwqKQ`Fs$0FLN9rVl9<0GN9(F#F1pCVD0{Ug-7Acbn{-MviHKUk)0g0Rg^OYhR@L6+ zay`@Lyf<+4a)$)c^HUTC=~d-OYajqeK)Ampr`KU@btcubMYQSy@*^pQ;lw4Rl)w@L zW6TLV?Q$13sFeoO)^G9`FeaX+81YTegdj6S@vy;<6hyoj_l3_F7sMTJ@~o&=7Tz~h zk;?8WY``uJJ_^Cy$|dKFeRUODWU(VCyhVcS;uZcr4!2aph8x<7Sa6mE`5=dQmfh`UmH-9ECMJRu zG+GN{?Zxu#Ye4I1dF0LVqH?ZsUriK;uh%*>!FD#5S44t04Wd*89)%YbA!H_I*N@$0 zTn#77&r+eT897-TVDD@0uoy8^&CAz5jj_vsM&m0YVk#_kDwKy+4%S;$!-@WYkVd*{ zoRgxlR$r)G8Zv%y&!^U|Kj)ys6!zrC7>5&Y>xim04za5G6#sk92aH7Jz`f_3_br#| zaH&&7N|tGPJP)``3#Y8&lZm*ibIlATC!0k=#`sV-(6SDKoC(*ik?!fY&v_sztwcJU zZtz*b+U!X6i|ACXFCN$$l#SemeJ0q7OUODx;}68RngbM~d-$A@|gB5dMBZP?9DW%ykW3oKQ2AjX)_upWs6!eKLL z(=>Fl$D6fUgC1#$2u@uwQEG5<%CX5;XVen;VR?RqYaHa3T$b}VaL!hAJ;h_KgIIh` zQVwx*@YW8PfmqrN2io(XoZzlOPT=P9i;1%|;@z-g+N4a<7i`u29Y823N%4N-hdfXK zIvUQdG#^eOq%YrmA4kYC?SY@R8quAl#qFs)*ld5Cz-B+qPzykd_HP&@yQ;h;YwvJF zn4xWjy86*ZPvPq)afou5lVmAQnWxs1tc3jG){kpoBgqe2`<>W*C|A|6E;G;XI|R+w zZ$_oDxmp$<@GT)iMK}$mva73g=^A8&+e9|knp%+DHy)0DSEox%r$pfq9PH!Jr-E6n z7fWfQpb^S%G<|J_57FpN4oaiT5R6Au;BkraPPt~FA3-NuMssubI8H$r_-blv7IuW6 zJ@wPG&!I>s{n14oW}50pl{G}j&qwm#cWJEH5U?P_rrkXT?-||m{e8|AGiD+G(m?a6 zd^5iUWGqt&o*F_y=NGd86)g0Lb-pDLNz2i%iwX{-SagoKYCDeEZ*{`Of7LdF0N}0Q zYi?%=jq#fYM|^^yd`w}LJDyLk%tVgeC&+E!D*h%lI{DF3R3hQ#x^}WFPbLuj0j{F5 zA&4$t5VU4rkf{=rY`G5%-9rApL~Bt z3#{9%vpwRTG~6z!hH^VkyOnv3BjRyWM%XG%-$C2L@AQ+QhxfZ;n~L*7-6tSD+Px~< zvOwoU*xT)9yz5ENr@4$E7H+6)`&7dGXl%s(9=QTBd_+S^G0ks_UYj2bpvppkGznT1 zEERKkre=@pS4G?f{O{D-CyQ=KM;JB^pwX4L;3ul_f%I;8=owx0bnLM#kV~@U+o;b6 zC|*T}PNH@l>b6c-Ki}u1rTf~{*@Jl&t{T1+jPW|@f0v4Mvz{l@AQAR7oie5954nTO z&|!CG9H)~J4;c}0A=l)?%#62U|5vhj*GDLrCOj9|)eSz-xU=_kLhBMZ4fmkPv4)Z7 z9VINxnVu=|_eS?{@0#SrWMT*t1YDtLgnM-m2so-(;N2fzhM+7^){C**u;xey0nH{!f7Js~5 zL))~xXC*;&YKEc2VXBfTPrO@BoVYB9wA6?il=U!p*v{9Csb+M(^N$Oq3xeC8!>$WW z6_X4F74$M6H&pRjN%<9Dg_X zXlx*I>hsguTB(eL7T@URJx@raHBy+d|8YwiDOn?GsH)j;uvq+ za*;uF%;A>`-`+|9M_z6VttW*TT11jfhn8ldpPYkeMiX6Bv94?+vEx>}`A0>9Su1o= z{#?x2yxOl-OTfjY$w+pK>Dayy`i^*IFPqBsK0LrI$-QtnvhG?Yi6|KHHD!-1 zXY!G8fn9D2j!M?$9h0{Q+->dB`%LD{45GiT`jSU zmtVofW6CX_g-wHB(h<>Q!SY+v)1Ccmxm%`x*CIGow|<=zoeGCLw^yqVv=VN*n~)GS zZ6lib4euoW=2Nf6qp#j|ke9(iRT)>dO~Yw74w85~;9`khgpB@ z4OFTF*WB%{`D6G1XrsG{NAxa^Z0Rrw7W350v;MI&t(~+AsRwOkkQBO8*y=zxzn`tU z!_D+L_xN|4CK%#MZ+YMrj6Lg+FvN0=JBq?I9m-V`l?)o323cGx+?8r!KMl2 z2i*h2hOom1NyPQ0)ZCap8H1EjQU+wnph(8rJ}VS%I#WKMrvv3@NU{F_`DUEUs` z0wWs_^`kF2NjOSg5Lp_lW>iamP$<&&#~`_`(V~V-N zYKJlDOFFC?I_n8g;lZD-?co|sug-O+4@1aURfYCKP`-kNV~=LgDQW=X7g4GM~LvV2z&#ySMpL)cpNO$&_e<*hUnA4lIUqa6UF=RXmY$h0t1EY)4*9$H7~{Bq!Pe0@=FfD<<@4401JcHFP;(j%>mH6?JOdHZWZ19u}mZsPQ4=(6{ z?t1i|SLtRx%d=?5|1p}(6QL1|tf0g2ggrXMt3UJHcYRI0y=_UGgA?3Io!4Smvr*z9 zht1*bZbl1N#?+f$79|bEX{2zayQ^=XBAkY%T(f=!YsspKHw+aE`Sm6p*Cw?~H_V@} zc7SVqJy55d>7B_5l+!gT5V&ycLRboe!wviJ%c;pHc+--?#bcz2RD6l)a?LXoiac(d zMZ1ga)5{!6_MNpgmasef)-((A@7PA8GqkMk4vHXQjvv&v+zWE;7;NwQ&^>X_oY2#Iw$fDiZw=z-uQC*X6)0yMOGT%X=4Brwe z)RfAYjZ`|mJ0+vXouC=>lqq2gY~ow^BIDYLa-Cmwc!Hb@(aRhQ*jDtSD6US%uAIvK z_id~wEM{tbxJ2!`(rr>@s7G_k*vOdUM|K+u;Yj~!%%g>0$?xxrBpku2O&lL&;Vk;r zHAz9QXY=fXcBePUT0>DoLa8pHzH)`9RQlp2AKnAa?pXX6$IRdH59lvEmV#b!@ zyNAe}#a(nFA(NV;b2mW=!3)fMiv%+LK492KND4xiCzR^=Nk{)nC3aj(p_=8fz@6s@ z0{+Fb!(uuE#%}|~RNdy#xsGUlV7>}mB85Ca3wt6_&Rg(wr}BYQ8@WDUdYawFQ_=SY>uV^FI*avMxZ%iIb4LtL&} zX82lZ8HS!Sh&`nlxbdVP3!>Xz3!)^IvZf~AP*k=q#xpkbeb1h`nmcaDb}5TkO)3n&19;5)SCILyVnoE8DlGEt_`&+w25<% z+Rp$UN=24_cQ1+-b?TM1e3DU0E9^p(EQa^TC1$Vnt?fl>L%NO-n-l?hHJRt0j`sY5 z(!^4d-!B`vCF9i%$_=b@?c)?t4-5Hu=gswY`*K+u(@|cEhKu3h{IAFo;m`)eOAJz zGI-+wj6&l4V!wkfp4Qa~#kdr!IYu5W&7&HnJBhlA5g8XZN%VZn;~RvJ!vftMG=YCh zSc_TN1jJ6g!*&Z*LY1;kJw<9=EWC~^5pZ0g1;tU?UU}25H2$LCQ&*#`O`C=eWJ!*7 zTqc=6zQxziJ*ytXlnE1|6PYegEh^qtBx@918_m5c2X zwcZ+p5k*DKg*7IkwWgei0xn(#{&4f+9lX9#h zxQhDe!)e;AKB&M78&4SLZU*&|AV{50@8vo)#@4kmD$#icqVIQHN|>k6jhW}bQiwcQ zFXA#c)PzP6L7=J59``9cJbE6++w%#~-fzE_>4_X-V(O{vDs$<9JgmMTg1Nd}qmDEp zZ)_db`cz#o?0jiCN(~wf7p92PG6`;R#+REpd)?SZ_tr{wM6B_12umT@clq`#OZE0g zqWSiA82b@(73asVY)2paB8xwA_9HEXoh1jQ&}j?eEe9$twk89-m4T|(h8P)Ra(1hP*A>IprZC7U5I zkWYDlSI&y4vErdc`7#?;K|zsP#hh z6Ba?eZ;PjA;oX3n9BIc!_aDgo5@IE03l(V(-us#8L-V5eHIXfgni4JGdgAi^VnM2=Xaxo2@0q)k@$PT%5Zc_ zckldHk>t^N_Zc(fTg|CgX^C_FWZ(9(=(*8gA+@y}K-Mipj!t!0@OrCNdJot7xG^_$ znvEy;7^EH0jbxlJ-e0-wn-<6l8;ES99^A_qSC(c2wKZ|?CUEXiP+FNuU!>YQwNLwP zu)c;;NU8`e@|oe=Z?d30?af5?LAE5NGZB9@5$%@&qB8Go(;Aam>}z?^?jzU-g3(K6 zaA5d#wQh%{g1QER$RutLU(SJ~7W+J^CjRIwnw`67zMxvNbDRX3TsCBpK8b0U3}wLO&nr-I@e`7*ovhGbW*g{TB8uV~b5e3! z=Bj^RMHlPf2Fn#c%%LoooJ+5qu^Pyvg;7;*`aF=RlCQ(X>|y$_y|%yK~(B z>ucPGe0>)<6$Wmm3Xg}Asu&xG_u*C{Un z_U!wxSeqy+Ifr;F4R6CMT#@iQW~3#m_XCAeqBvsh%$U+v9fjg&qY-e{TZdAMk7cCx z1UJfYNmBTd3>ic=na3m24ZmnI)b6Dq>DUM(P1zg4;B+r8`p~<}kL?~oEz1%Wi?|4_ zKTC$<)#Ll7og2l=CTMzCC1!J6cno@BxgokV0WIe>OH<4y%hRBtTRW3{zCP{jF17v9 zuc)thr`@?hyBJj`UZ!|Ly_%BPA+agtOs1x-of(Z1>1U_)OHA9!z7)stK0KC}LCr-% zA4WO|&ox)B;UwfSDQAB zzbmR%aah%>8U#C6H}zbLj3f^%gH_XFg|@#N=|zPk9@?7yl(v(f!e|enOoRcmV=>6w zvAiy6Iuj(}LL6*-8(lxr>fZfY6>cx)@I?=|nuBUTsau+d9@Ie^lH=a9S9s-jxohYn z3JkZT?^TL@9=~75V+p&y+0RVKnSk835c-s-{mR`@4{o5PkBxN82bhn=+TTyOAV%Dq z9pBYTw6*DJEh9NTyx+*faTq>Kgq8_uOZviBS#ukCO>fr8^DV!iBBmgj^JLVeYi;bo zMXGL_75-gT0vD~-AJd**YyQkv^4ktCRiDU=nesBQX)1xcr8eFuCkGinU*x8+%cibq zn~%0(l0IJ21i-}Uwa{)Qs=wDj+ui%2Kq$sL6FL`sk(9xvxuZzeQ|I8AS;B@D2xXDB z#BxYY`L_Iu7<-Jvj>0&cpes6vYxFo+AD63Yo@$PpCf}p3bHmr5cPP_UIMEV3*M7Ky zn~w#Jih~dvy`1N@lXb|w>1%i~O%2Q?7`L*2K@|m7_YIiKdC1Q)P#(r?cO+ zPqixqAk)glKr?f4wH&ej_-!5=D0CxJ4V%1F8IS6zNqJ1F4qW{HRx>WnbBvp{_Bqr3 zh(xA{2(2H9)G9Al=lSuhZ_~5rA;x9Obh9qvd^J0!M<(+_&N3}%USsk4FI!R)!NS)K z#8TCB6u?UDH(}DcrGX7)MIDA*ZdQ!Hf{nIFPu`4b*@-$P?7(e$^Ld6p=t__<%w?Q- z^6{&k^@dGTM~l3tV*5w3kb`+Ya9C6!1xsxR88Vfy!QaY~x6@p_>}00yb4_r*)ePnK_m8 zujWe};CR_Od9-QEscA9F>?-6agxsV3m5D6{y^%(ia+B$IYd@N=+s_#o#Xg`Hk1iXz z+r|8|EJAj@V1W8wuslR_QM`n9@&N-@UzWon@!H}zvGP;L1jLe;ML0YGM}t>I781D0gr^KQoPQ-+nf%|Moax&U~Qr^bW^bT#j^}qht3so-lI1r;ch#+PY-lQ$Zp~`% zawuZ}bqf4UVd%E6^uesRpY7&x)oPOFMBl?;a>a4F0;AL!G_^Q=S~v}d1K!&%dURS7 zYH9m<2`~J<>E;(wCEvUA43b!Bde|}oqFTm8p(mn3-4&H~p2ntVrS7a^oy&$)mDI>o zMqtC}9Ug>JlAKqO)*_qR^<>K+J^0$>d~($+M~@G}vr~XfQNifLOAD1pZ(+bKaXDQ) zc9}mqM$?_VNgj?~jBF+hQeKf_VrCtumAjEWcP3n~8FC=Tn5YNRAIFed?-(W1u73Qf zN&1+Y^27VEYXZ#^_j0S&@3Qr>d22}5(;wGb$Y7>`+XlCf@_iEVa?2=72h<{(IWjxQ zwIi5t%3v$yhM*kcx0Os4O9(&q_|MS3c~g@@WN1bN7YO0I&Q7?Z*SGkX&{ z03qy^Y1?=rAcr+iWGdv205hLr}is`bsOcnG?IHsVuSQPMU2Ab zt&9yrh&|#3d_!)L!uaQx`Jy%1Kc?_hKW#2-)tDiP%}>11qH6=_5Pv8aH%B*x#&+DA zO0;OB=j`nS@t{h_HGC!r*klB%*0JXJJD;&ffBw-LmNHzw#jbb;I1arlxjTv-<+N^F zk>)Y6fk&zoG67vgq`YtMXxf>Es_pZ!-k5mgb%i~mxm*votpu-=INuIb)O=t+f6we~ zgN)-hq>${n{WzdtCTbe}!uy_9YZ9n5cCDjUkfx$ONf|8OB`az~KTCrYj#)L{=qYSw0*5i9 zUbmqqL+i4F_1E;|RORrU)T>fu9mF{a=ZgK#AKI+9+#8D#h>u9}9BWX;aqyffXiH6>nB zP=u%isesSTkJ27b`S3_U_t56uYYeV zSB5P$3TpN~^#a7kvPl%BzYhO;v&+tBQ#@{vS$W2h=pU^(Nz1iD*_)LZ>i3-_>e>5tp7%r}%7QwnD-MVGZ%<0m@FG51nZa}?U51Mz0 zoj36KR=-gJB^oGK5+A3*ok+BRTCpjr=&9z7)Z6#wN5#&?Jf8-JP>;RDWy_@;)n^kC zr+7zRf}d`dLsyq5u@PdyUKzLB_2r>`1t79=irpd~?jsvjKGMZ&sSkS=UE6WMIWuXt zgmEF!8{LGM#n92DX?J{Z4RWL+eqY`y3p$@o@Z;WoBHD_%_Q&G8DLoZ`H#@4ov!Mx! zF6_t9-;1e3OFhc1Qr^EgvbJi_D&eT>FhBP}hdDQM#$?1fcfvfUrGQ<&v}I;X(j$N8 zX7A-p{XVUTB2%a&d(8S8pt+L6wWj=xFg|{ZDH9QWjs!gy@OD6!rvz-vZ{rW&UM7WI zZ+d$tu*00lTlAn*`fPtGhG-D27fuV~AI%sduV;?H>cr2qXdyO}-xLh49K=wP#qHRd zs$m0)?5Jb$8RO!_+i2AckTJz*39^7VQnHTJyzs$~Q6mX!V{HgC%Cu)+nfOp&(Kx}mub?&R zpd`hjc|5ldQz7-U?ue>=pr5P7%)>k4E+-KIoop?p@`U6o87IBP%fY6ng-07w#ikiW zEPisL=Bd1+V}a4*=TatfVqK#euX1+9>dbv-{V5v1T0~gma`hrOcYlc@F00xo!%*Z_ zED_$h=j(oMjUeey#gHFa{Qnwyv#AIB)cx8v4!Rv zVgVUp4##vzMyE{JBcMfZ0P8!|c2% zT?=(n`XOwv+Iogd6U{19;JUBJihB~8LfM!3uU44f8P6lhXd#3+cF3t!1tAON+OajJ z$NsERC<75ZZp>;eoIEEhhVx-!HRrQx046>)9ymTvMg{Hg5 zMAA~(kv+_SlvpHxeEEyY)OfI4y5k^AqD1^KB?g%}^%Q|IV#do4s9a8NL)^PNHcBl#dQp@UTj#j;;9^8r+huM36 z29CUDamA|}d&Uvi$68EAduD64o!%C!(oU@U#u6Yx>_Hhks_dFC#5{x>$^5>#OR8O^j2jB+QPzx+l?ZQ{&={&%!Ynbc!iSDtYIs>-q2a zEN5a)FWNz=W*LXpZbD(NFn3G%=2;CLs7b5h^BlxgEhS{^mE~b@=E`tp^Y&}4I|$MR zD1JvWR9$p^Q;~%voL}pSKBG%@y>F~I_LpcTlr9rp2~Qua@Z1B6Y^TU+&-S>sb$(w9 z|2Hh|+!~3<4CAOvjVP0xBVF+7lS=lnA6WzSnHy<_~noYpve9}h};Y*8S(DN0Z1 zRAH|rS;T6P1Y(!x1`^;00`I-z^-xPcNXUk8G32+BDyb?#a@iGX2hFhDYp?8hMm=4f zt-k&g$I-jqNA%wdhEJEls6Ls4aD3u#msgmrXr{vfWg0Z;7Ycqaq^QIfreZshM$rl9 zd%xoB+?b)CmBs*>`r4TGQ=3PaSy-NOmlsh#{frRC@;>~?MJU6`i*@^;9XEb>$Hl#h z1Y?Mc2beQItUcx;V=Yugg8oo70tq2AiXt#mxKC&`XjkMvdQmE3a#6*qWvFH0fuOIR zR;myCV`7q?+`4Y4qNMpxll)WLQmSEsc_)6TstnZJqz~UeI9!sC;3mE{!gen@562{G zEw}u$nqdnC2-+;ub!M39LU9%HpOBF6Jd@><+Y;&CM^y>olPp_v1wI$f=HFXGReo+$>QqprF0hxlKa^%Fz|k%yCl zgz=P7l~!f9%!v~-!bcW5da`BYRE1d@=^?% zrxV&5ClTyL+ZcULNbKPo#rts^0EWJ{4zkK+wBl?x6`(orl)d&ggW4uD^T}hHs+EFt zm(&6teCQ_!eDLD#VA9Wzk<+N^4JAxMpmSU}6hyfs1ML2EA4(I*Wr)nXVe{oo`JMa> zt-@!&>HGCEjhtaV&_WPdjEVNB_yyG2Y3mVN?+H3xv6osiOAa=~&_#UKuFVba^SHKb zq>TrGuhUyEDR~o$zAG%J-rD&U(20xhTBfO{g}$)RLJV%|)8AcuyM=~H*|GK!qo<2a zdCE_d!cOjo1YfW+(YNhkfdGmebZT0p$eTIM0yWExP+`X zlU0*Myd%mv6HgXq#G!umg(&Sy*;TmN?nY_8A2WaLB5BAdHIgixI^J01&t&fxM?6Q! zQ*3)MGw3CTnX1bZN`_#&we)Lc-(r8Oy z)UuUUgka%wwpG)-PKT`dXUHi={O;Aef4&%SjAt|9q6yWl?pVg~vX6~6pDv(NOHq|I5C+W-p@vGbvkQjh};b)Smqm<6Shf#RaL z@-Go{(Ocgk9uLerIh%%+5nOH8AN49Vr#|o$6TVgOQP?qscX2Q8glq)oAMzspYVxxC zC3dL61}F}bJj&CTs=IV}xe)fP8zozIk*g>TlVdrZoWoLfYGA&BS3l9TKYc_maNi;OUQwS{OU*2GA*GO1{a zBIW_4=vMpwrNq3Ll==Nod(JUfXnJJ9lfB%|C=0HMw*rTY=@B?XKB*}hcQ;!;lJ*qI z=?}+1tGjHXdaheo!$rbqjZ!Uds8ixLVy3&rug4G!-T~wnrhu`k0 ziHRQfmm|pEjK*Y^4V&2R38;gS(I2Zw(J0G=6I@u|({H?jjKYek0!Ad*+Uym=WgW_g z@w002<~Y*XL|@ib&-0P>x*;KXppfzAWxC^7mENl zMofK*5Ld&u6wcFCUlHy?Z)-?&>ZA#!fs>vpvh&^1y$F+zH)I@=JEg|4yU<|w%-|+- zyR}W^9)(HB=i&8qB1*~dz$al}`AX-^tnq0q6Y^a>Bm6)OZY3A)nBm<|DLBbi{zg}x zEw%vX?wN+|XmT6x4is@gTnTrT<>T$us97LUo#wpd zK4rWXRf`2-4OR{%|NW4BkEjBt^qo{-GIh=P)QAV^SE3qb)4rZ2b@GtA;lznaO#Fue*eRy&ATpn~|9KGd}Es&vTA} z!L2jBie(D$*%e2It|n$a^zgnJB2ijh2Un-d3}j_PyfgB+ zw^5i^-=H=Ejax6S0yVIkTgzag**KnCnTD108#M~o=weT77{Z_%Yj_p~Y&M$`@f$Sf z3bwh{e`?0yw59aZTpX5_Wi&e7)L09h*hs{2zjbn6cwKq%Ax9-3e|j%ybnFCspH_WL ze=R|_uR>R$Q4^;e**B+PKZ-2~a*Ix^^u>I+#CR2jQQQtIOp+?qXq9#eQbZwi_bPZe z=pg72mFSMY6L&g2+cZXDB20JtSAi!V2O?uJxj7$NhA|>dcCxvY z`Lx1V|0@MXK{;}QeXVKy=Fq;L)AZk2HRQ$?lG1VvoZC;<)Dz=;u5ek!sP3cyWKZ%< z_)@|w#$)n5KqN*dOyvQ)+%q|}AOm;$`16tu3&&BFoVH3pJ=|P45BBvx+9B=idPlD- z7K^{B*bWk?@fu-}O;{XqQzKO8KkT%hJMRS4KRuJW54t5?bwj1m;VXLz>=x$}ROWw_ zFUytXv%k7sgsq@>&R1^1O&M>Cwj%ntQos`%;&wi)_3Xl&(r}uwO)i?~y`Q!1zkT5g zvB_w0#Tis<3byq$v~WY5>{eA@4Fl@b_~hb zWbevQ7-82rolclX;L0}%dEy70P5JDQK(G1O4&E2d_Mx$b8$^3dj^Cz<234;JMJ&>w zz0UYiRi~th%KA$XkYzs^B^uSAVTMuX-UYK4H}T-nb_(-?F0tA;NQ!|a0srw0`s*DOpP&@bdhNjkb8MI1MVc*NArKegV$YFR6g3iheUK69XQAaI zGb4=X-JtSAbYunxixdvo8iO-LNQikj6TY^iQ+`6BR0iGTRTeqDn1@LsiA6GjeYxcd z5V5oQ`Vi)xZ38I`EOl)zyo%f7a`3*}5Xp#$(l2|r*CyAy#=R2?WNvd%_UKnmooxv5 z_0-$0nTUah$heza^y)5y!sg$zQ9iR8{u8aY*(z7k@JD~rd1`qTJk zVI&mQxva}8gD7~(wM}OsvgDpZWhiajOV3C%N@Wuwb^JkjNw@LF>!fBFXU-xTF4m|hbNQY@m-3mXaz1BE9z!9N){+s# zfqx!)vGDJ>=?j)n*c!QEOXj}qj6JyrIxXU|LEi{IvyBOs6QygCqNdSqX>)SYzXO42 zs1RB~dBeNg&qS}J3u#9B@0@!^BF8$pN{GE@E-1%_lGcfW!@2ZZ%-<&`N1u5rJ&bkY z0fC4(Q(K6M5DNiqE1rs>i9>yx2~ z>JMcCIWxDF>wuFJ8n0XO!=#jI<&m0Q%UtjF`&PyqUKI)YXrp75r+;eHDVEMW+zmN| zP!@P?ovwRb8k>-?e`pgLBKhM_O9Khvj~F`AtS&+0zcxGdQ$xynT)p}QsBXLxFS4&m zO%J*g_~M|}qIoNdS=x_C6_?IsILiv%JlLk9d&On*T)|4zkx6R`)dlrxdi|~GZ>v=^ zn)fnu@dSgx$>l8R@%HWQW52HClzVL{dnCRXEjrp|| zU17Z{$a$D{1r)vK1Wh}-bbm{l%%US3e@B14zSdI|_`da97=i()bgVaKm4c11VG$;r zS!Zyzh?R<-7Efk|RemmB61y7jPV6BIRZ|L8X_6I4u+kpfWxt;-a^LWv4?Fb1&D?O~v9n*|uoC{e6$0mrI#sR%<2$z_pD;+jPC8d2Cq6KV zboz78c0OVptFBLMe{E38s(U*@UrlnYJ+D;y6F=3+w8ThjTI~V+ZjsK4{s=%JMXUGz z?sPxK4B@XS9_MpnV=P#&2lWLF>Uq}@6trrJ`7Nls!=CxaG8>73_{ogAd0Q@l0gY{* z3C3phPW7nvi687m;-3R!%~Gb(Z^P;#MEh7EPhMt1z>O@%c{yeBbBeWiuPkEcH%498 zBEwR1OGmgQQ|vqAPCGpWcQlWjFa9iD+3w5Gk{84JaUrJvrcr%=Z%jy>K@dLvI*#Y( z=M*H+`U38$K4kW#P2Viqn%0;Fnmh(lwqoNt8lnAmW0mwDr?2=L9KR~-=8IzosKqs% zbl4wZ)K!%1Me6WE=Fi^3#ll^EpQ4Cl{js1|!&4)D3to2!6C~N~H3HT%^Jj_hWhjC9 zV3G^Zh&Y6_tCaUx8L2PW3t{jb%46~FY06ul-vyk%&x2Y;#W*u4&3o$*8Vs_+R|90m zy@xEOJYR^%9eH3ZKCK5(Rq{q5Ek1A0g*z8gO5$SNC&aK3%pkzYrpc?94wn>mZ_+H* z1(UcOzk~X#Ly9Jonqs{uifX{yqa=?Tw`j4UiblNUp4rl)I(yWU2#V!8@7Ly0dkRbm zsAVfe+7sx;F+@z*uHV_)eU~C7{o53v-yLqwcemsVFFHQb7-<61L^e|9otiGY#;NJM zHlmW1SL_8n+~42nw0xkxM|M75ZF;4HwdhvqZ*#bUgN-0C``n|GE$-CER|t*phXA|p{>LtOV#h;*ndq9sD9Yist58eOo^wk{{!FvwuxL_h*s|sHi34 zV%?Sm1n(|fa3#oE>(0gQ-7=|t^!WC&y!GQskdIPoon6^hsib*(Um#SSu*!VpDD^N( zCx-34PKHYS=Hg{vYdKGf=du17evWv4ZafnuYww1&LlFlah$a{^8d&dq9|Rkl!=HX2 zRGq>whU3R*&@Z1GixV61t5?d%+b>;3-W5&kV+dwz&%UKVbI*}r=&+5~4k3oocm>?F zkiA`YZ9>KS95H?nsthecGuDTVhoF~ysj#3fdfT;yw^|TAeN+~C=qG{z^k&o{<6SqL z4^^qbxfDF03NOT;11}$A#fM^_LC6Odss0*@kh6uyPE<>}79wWW9ZdJPxgW&WseUf= zWOu)te$wd5z0~RTJPc$#_*rX(XecOunzS)~{vvjEyi2jbmGJ|I6*)N^Ar#}P%l+E6 z5SJVC90H`g7O{M77?agT96QRJ)qt#@`-b}}${x?)q@xGKetTX2-th17a={uKN5O3c z`z|McU~ftk@=G1f5+8+gqsa8B@ZvgXa{J2(r>) zmUQKqCg~8I3j>(1e@VW?hwR;_1Tpk1YXaC=ua6J z|FRmHJHI6PEn7ty?aV;D@_b98$%j1u#4WsF!8gn4SpmD2#jJhODF3j7O|jZ+pmR-& zA2Vwz#bFk#%KfHJnlWHL^1PXvvnTdNMW;>gK9XG6}$H>6@=nhz}n-1hAr6Ji)gLm6Goqh9Kf5kBML z%df|o0SziMh4#mIEVQF~pBi7Q!`^%Da%K}-mVZ-0vV0&33nEfc-j|q0oH(twnUCmWUTTHD-rJcN(fF z9x}s4q>>JGWSxr~*czPF)|`G&6RGk2{uFZNk*TTpz#vC8W*Lu2y%sbtw}Mcy-X1*u z!yr^U_;f@Ix{Demb&Huyd9ZFvxrChV<;(z^s*v2&hkM&v7;~npwG?>Nj*rqWR||tR zd_VMFnf##wWysc^LBNDiV?);p-z@tJ@eh1Ef3wp**H1id68RZmtEw`HWFH-i=y{4vslt64>*fYn%E#HP1pK;dnS31Gs{I0diPxU=#T&s{;7Pmra=_^TRP=a=~W9P}W zChGux<1Uc3Dn0x;$qt7=d(_}XMGH2YrgEiqrEP%9@Es0t7D&d`ryeMLNDb0+pJ$LZt{o#w{0 zef>!Wne>i&9TiOeeIy9$<|m~5D`M}be5>8y`J3qcahoiT?tP=UjJ+}kn($Yjg~%OQ zK7Fwr6cK!U9_cr3KGGy0ve8J5IV? zgz%jnrl<{n(}iabPYT3Xty;IdC+4v=P01zipE!UmtFWBO8j7Owb~l?SB+kJyr7BrV z*K9L#^(X|A2KFwh9|rqoYZ=Y?j1J*$Z7&;NwimVJSI2cBUzS}k20Grmr z^9;4#M6CByFfxc2DfKX5=R-s8PMKP9B(Prf&C#X|1BKk1vw^>c#xy z6!vE}lJJ4OAB%qJuP57<;HwWTZvsVrf}=09#@%+=ZtUO zf;Sto{s|O{q2YKPLTi(y^fO3$nOlYPxgS?5w9J}Vs7h|a^vPh^tgfIm&XiBr@&yH3 zaWETfR0wfF$(x#Z*8D4ya1$o!>!0lFev*}O)xIJx!REmWl&TPBIc${Xe!6X95u{;V z=q+RthU0)-AO{`nn#5h0Tg( zSJCJrX1Sc74*E22TCXGHC)x%DyA-<->$y{g%b2pxd&S}%RGK;qO01gDdIA2D$Pf;X zeVmt3oBR?{1TBo_1+VcjexZ(ya|3|&*o-f>tSH#Yac)9CAq5t{pFqP9ub0xM=Nt^a5!(?5Ky(H~Z5Gn($|4yt6AAAbvBIYks=k_5Anlbm?f?j^YHwP3Z<`3;Qs<9J77^IS%1x4l* zDlrAoPkqo*Pu=ih}o)z9=>3EKNuQ!GNhADnb=|1 z&6vu94ro$N4$JKFzGvs#Mt`pY)9k?KwJ-D%&WMuO`aZhs)`VJ(-uxweL_g6~1fQ8K zPyF}1*U)-erNbzH1pH`W%&=;y_2g;97Ek}_KOR9!!N{!Ef<@3p6}&FdqTS9%xV1fq z8t&T*^a_r+k#pv^lKYoBvLE$$sm z`?N z*(I9sE!;6a<|i7iDIW^76<|0NGg|%AG)CqU7seuBFR=ZyIZbt3Lpb@WDAg&H$&JrC`LMI}CjT?nr&iKi}*AGDZqMuaMH=i$(GyQ^O3EmHJdAK283bZ4)aZM(lj?i$Xg^BD%LbrKc+XpgHfcwk2nz=iCIv^DoW9elmM!B{x27S|GGOz7H|l z5A}Kt6r6?0DkL^0b?!dH>o}rr@jpd4H}xAUH#a8U*b7YR2V@D0Pen=A(sSWgB4>u} zDU{Xvj7i3W!vIr4B+;k>I=mFIf#|zy(53HJ>PB3^cPL6|(1j^v()DC510WMN$l^~& z_n9IN!fKxh$$FAdXQ_o*OUmV4qd;VEC#=KqHDze@_nI#RCh;F(TxfZriBAXY&|eSz zk)mJEwN4BDST+-XoG+qt(Cc^9$4~9+~NqCP+qdj87E` zk4#hB3a@j6wNbuu+auhqvl{6dKwVECWc%@~&@@B4-O`9(HwAr@;ZmaW?cit=9N*wb z6 z$zGPC7+?DQ`6d{)Q-!l(u)I>Vxo;w*yK>=0$_@PvuTR*rzc$sr(a21kck&OqpZv&K z0g_>Bj9PqBOtU;XncqTcrcfm5wR??V(t^0m7mJlF>?-&dB?CSYemOCyXU|OXReArGNh{8(lm|#i4!g14&%nY&6bt zP(=KD*e-H{+X^j>&}JTHv0q%+9&~E;@b^f@^brBqQS`6b4%1jX>YN$v!bzll&esTv z#5ZksIy$$Gm(*IQUv@&bpk34`*;LF# zqm4*Ep9ukmhvsz#v%GZ?TqCKpHICNEwCJez+B8x%`Ti77KjBlkNBeu;`YG%9a7%Y` zTx+zunA}cyfsir}a@l-)Xpi!{;QSd;suj|{&Gm<$k4rnvv0tJm9PF1_IrD4r7+Qct z!%R>~2kym4+V~Sl-i8g5!f02YNf&3W<9BgwwYEDUO^gS>cqeSfkMFMWXJh> z_pL2Swn$O;QgR2l>=$O+JX@?6qip57eMWIG=|?Z9v4-4GqNp`YNbL1n8WUMkQ1Q^b zRmVJ&6FGzxS%!2f%TiU!7)s!L47%&@@qtIe@KbLU@=M=m--qbK?gbTq`T#d>`PuW#`!Q$D} z@b5-s28pT+mE#-of;twyEq5|mSwuZ}xN7V>^$nuMl7A1z>)dPV9myvOF2T>0~@S`{8h0|%4D#vF1!dSHJ0HvtO;5MF$9d0aO(^;>PR#Qt) zCQDO5q-aK6v+=nAlNe%eD9paz_GW$6+_W`y>kJ*3IffZa%+f2pdl{X7Cp@PR=kz^0 zOy0>+>!lPcD`%r8db7nsC+|9RvYrA;#g-&;r>HAIQN%yQnla@~>0{533|SjNI9sJI zpgW*6wzo?r&v@t7TvA9ivzJ=+nw|9gxq*3%Kzfm|%3Yp5iDN(C1X52`oac#)7Svxy zt^6Hg7Cooa4cl0ox5n}WlQ2@r5;bdhhnqau#ep)o-TBN=ud7ty zFjO_IA4W1of=J(Q4e{IMSfjTT90!AjGQ;9pbv-Zjvg+sa>K)NX+wW)%ScwoPV}uVY?lZg|ovzo6@!id$)V5a;P1pI^C|BRR35E5%YqOLSaM(pd zm9nP>_$*X{wCD17)<0Qq5gX5XHIu!3A^l*<(K3_nM@d&2@l&J(nnA z86>(DWx)h93Jc?V=4=*PN=wT(?KukUJmxl%BN``HFHbL4`?19It`glVpDjMS;SGg! zN?jt-aK8&AF_m`sBhPUkO7HnN>D~?zL8W-7CqRfZW!qdaJA5oH2@{-l(79S^n}?oh zYywY<(!5_xoqbhhAmD$Q8l=44c~7=tg3N1GV6r$TG#`ja+=@slfJgCVq8wF`+uW+a zM>5s(8nn;%93mSQtZceY%D#t3`$`DD5oKm9EKD?4yZiCSPQ+J6qjuRL=rsGoNYl1lB&Gpg+W()FYUyhU6{ERf$ zQ#{`;+RKBW`~=DBU`0ieglQgu}U?J*(crs*)~GS-UcyC*FRk5+}6_ zm76$wypuwMjb@fvIF2A&KBwc_GtU(N*XoaZ?)=4zE)nka zANKB&;a0MS&e|%#`h!qP;HEq+h*W7EB(K(+kyj+r*}R^d!0TCm?%3-0 z)l*BwvH$}-OkAubv%JUA?$)DkIEC(rV=Uv8wH*0&{ zl-?7nSf}@G7ZGm0?^I|MLCE=np7&>~&`=+JXv$Yu872qM878Vv{nHBKaAOpqZQBab zxkg==86wsSwWjP&G3SgOP7~!bqL*J^udQELfv-io6KV=OQBA}fio90}@>gTRf$imq zJNVn~v}8&qx-*!=qj4?iaKVM$t^9=z<(6%7%NFQsDP}vB$`ZSAsHC5gn}fVP>!0rJ z{lpPV-v;&U-Qc92UJSL?Iv4K7$6Y7*k~0M!v2$|na9agGcXu2^KgMu(p(diXw~M9U z%{AJgJgAbHp~*>9L-AM{+E;5vt2%$JlP1-nTX~-Y7_LcJGrsoZ4DTwkv#{|>q?O$C2R{@bxkO~t?d4rdM8NFx;`a-A z8=;@L5ZzkScm&(tiZaurue{WlxeVC3M9x4C_G zl*BNNe*YX!LOtT#P(!?K z+gDsDn3&PMm}`Ml56dB!D1LNN(lhQ_2mSByJ7PxYUusa+v;nEKpn&@QT_;Asx zQ&t*P&ciLFhs|2ZsY;l8od~SNK z-7h2Z(+`!1Pii^jx`~1w)95A)I<1rVXs~1r&))OzEFmBSr5q3ypVw{q^Qp{E6HBs; z^S1C3g>$#{((LVeIee9hG^5s5FpdsOp^0`4<&g|`Cmmt;pnkGSj)JL&e+^Y-H5snM z_&EEW&Mv@NpX4g^8N|U!UXbV0*T%|_E@UoE#B2@rG^Cpzx(Kk`I*wK?CWYqb(1!2k zhf;g{r);!%pDZ7n$05X3TtevS>zM(>-h5LwiRt2%U}D;d`< zJS{$$yx?gff{8nASDHz$X$*2avd=4+tQ=d)O$+g;q;goKLyHgmjM5h=m`5w-ED`ub zXdlHL_zKr+rV%k6uzy;4UnP2BziOBU&VS-c+6ajk%DSO#)|yqGXL0goZyhKG4p}LDEd1P`aDL6ZQ zwSV>I`CKJuuR?SrE3G#O%SctN+N`@oOJ`OVa?OxEs?OqXsv`ZeAeUjIPj*oC=p@Ky zRr^-3kE1HZEsVnIoe(tlt8p~x2~o0MkX0o$Y&;$hmPbC1z&q~_L?Ae2ANLQ6MjxWM zy~8uuDjQoS77PBF9to8<*&uFHbp(MP&p%x;+8gYe|9v!~L=C-T| zaA?ahNeDrFoK%XiN`FHuU0Mly4Hpv_&+fWqO|7`oHMWf_^yJMkUcvbHC}ybp=LE#~ zgJ7I_sSSyUi`9$C;u*ig7tXOx_k+Rc;-uj|icQ7y>=XVrOwIs{hcs33m62+W zrfEmFGqocofe!ROJ+s`cmbwJNcR*`wKi_=bbzj$;aw2%fh@(G>J>>!zALsGoFSMjp zao>q0E{iy<(HhFHz#V1Ft_<~Of9AbhTz&yF8f(`kO12X<&iQ-kLLS#^*dU344Klj; zjbzBFc9b+V&U)i8j}5pxmN3$NUS8+nhP6(P{F(PuzqQ#2S=c>D96Ui4gVTmIVSVL2 z<2gywbkFXlz*ofsLx*w<>c}HGs0Iy=^0z64sa16Yg%+d($v0lgM}NAI`-+HLtXNtY zZ2Zx#+?0ydNBCPbb)p$-gmn_NtJjFf641ZBAauDfgt~Gc9uQV%b2r#73E?-TM)d2! z4<8+#!+5pZ!!IUde~RWjt)1-3;!Gb6CXyUR<`!uXjNOep7}1Gz)xZk;UMI{TsWU%u zKe>@vA%afkXUcSkY654gUyvAb_kKT!Rl;Bez3lL&3Bh=Vm_YE4)%eDailQ_}*2tJx z`ki(@|Mcc{_L^#Jd5+e3GbXuachT(Jpl+1Fc^O|WJZ+!N0(=xf^b<4CC>7(}bbW*M z;L7bXFZ5#w%a5;pov*l1e)(oyBva9AdIpxKZp7)IRC%!M=%4-5n#LR?i)XQtAaxMS zDhAaeX+XCC;tRm8mL|=AFEDArE_zy>3ho?vmO;oiO*~sS#_2U)6{+UClygFzB83|n z3tWsHU|j-_)u1yhOdsG+CZ6LtO?P}c^Ik!A_gq>b^#Grj$uO;0{74r3$Ov#5mU)Ls zn{EPv`eee%D?75@<%Klzi;g%jNI&Ain$8gG`U&hx=+Pf~4Kb>4W!d$x+y%WlvMCU^ z%7cyh+#qdyo+f1g_Oh3inU=Y9Cw?{L5+LhWHpw6wKhGCnlEL@#a3IT!b@6;=0Ti}{ zM(ZdXVs|`=5x=zj`x_4{_Z1cNp!QRUhC>rx_Tn?KHowBeJ-M%ID!w)mSKLcpRz93Z zbuNp7C))UGgXOd|e~ad>Rh>we4hNHnDc7#z(0-l-TKik4?6aj$X`9*4lBD$t^pnEU z;+q5sjluQwYHwoPyYwkt`nPvYW19(C*H(cb;0Wnu&o|QU(fhEeG)p{MiLnB|YF9yy{)}hNK=VkBEpBD1!LGB!Nj59(;w1?8F zEz8Vajz%fRGYW0P3o~*VSU6K&#y#r3Nn|Z<^F9~3u-tnQeU7_s@u-MXN-=)} zV-Y`Rd?bS$ZVn(3E-cBkA7wvSFdO*KObkybT>Wb-NB)bxvQo(Jvut!tG!5H(bLjXr z(eovVVbTb0edQX#5(AD8wfl>S#~ALyHKsa|62&VH*t%R^(UHRBt~B!`cSG9Iz}T9< zy{Lke{_Yie*B%}{0uxCDZy+R9pwAIGTJE>GI}rhab3!oSmXl!8++4J{)xT1axt=Yg zy=Em17Y$fl0`ixM9;eE;N~A3wQm0M8zYo%0Rumw~#onJaYWR7)f6f9K#>bT|KMsF~ zzfJP!55ByVcKMV){Oz8DXD$;!ijB$HFn0U7ZBvjhYd9zxVCScGC6u={5nUm*6W)jm zMq-+D#yBxd5H&5zLmRa+;pT1n>*tOk%O>K8@%EW-&rKHYq{t3QD5kCcw0t=b?He2O zr341RPRy!)0mE4m-kjteQ_Z;#d0Ykpz+4B)^1@}_n4mc$#oRF3i`<{*EGdrR2zGI(iW zqBvk`)bTIHZ0X`DP4?5AkV~+YLb0;zQbOln0dHI~a-&;me%R6{@wo?(z^TeX&Ug`W z9%eW;m%xy8iQ9g(m;4n-l<~t%`1e7dTd@=rigV|o5mFaB43?lJZbfoSgPG^UDT>Jx zy740~AvTTFyKu4tDPt8gybtw~N2o1Clo>VaQTCVGA&$JLb7rykcS`6~C~xd&W}Qhv zSxksfAlO~GuQxFEPjTRHA}OTa&20$t{_Q_taJ^Os!76F;&#P>5ebtzG&A;531fC4;7wd-9j<< zdWLY)>LccSDfJ|@j`+1rQD%-Ws9X_pi>jW&0kkSaO0cyv{+(^Rx0kL$(Sgs})b)|M9a?I2MZ1eRT(kHdKc zlBXf*uIEML3I&tf?7kHB;8?cz!|Z~}(SkZM4Vu3}Mq#gp5b+(aSJ%YD#Oxr^vNRd& zbGcHfPuW|9OV$vGL=&0z7p&U0h;#}N&tnnTWY0~+L<-WeT)NHLHC!s76IWAYSGLU@ z^DqIdj^8Z03`HTbOW?A5Ha4cXYM3|{`R^=L@7_4(*rl@*siQNjjgNIfR?NxTUNT>H3DY8FH9mhGT#&AhtdY=WFX>8ijS?U$-+@ z7z)9hniL|27;_PqSlWO0WT45nBp+wjMbaAKyxr?W%G=rII?x{@lQu4$uK0}^l595J z8+z5KU^CPv^f<|g0KS^oYSR&S^gwFSz`i(g&3ano=qz1qwo6LB;fpHKJ)>MD2TNhI z_*e9SX+}d$2Ho}~PN+n;VuBVko}ssCW*Ug92%{aNlu@BAaVAlsyQeZ0 zy34)+hMhgms1|c)G8K=*D5GPKdL5x_AaQCcAe+`jLrt=uF4pRflj6#_nm527q++F81)>fvf3gbR&>7Uq3Vd)q78YUptRU$~2m5X4to*FrpY^r+a{bmc&YTW`IeVuw3#_IaL`mF>dJfoVRy!eF z;(S8G4`V`FPM(hqMt2IpEfcOXmBgWMt88^p)j3^@O}{!ZSyWgtnz$oC1oQd-m)J)kb)a0S|@n;8#%6i1QoSstTME>cyS#RbdZa(=%27=u`rFp>pbLt7y#k}`h49K*&TKj2(x#O~9+o_syQ}Is5 zWS%@#{n$rltJ{_zT)W)?E0n+_$EEt6J}-?su}r3F(3R(|0 z3oXv})XB!2_Nx4@UPs!W*rB&loJ?Z)VtY()G1(DLgqa{^r52BCnO##{cxjJL+eTzL z9b7apbZM?X%%X9sPPdZrt=#W4bx7j@A@hv)NCI_5{&y$~&Ib%x$H+qp5rvTSy^7Po zoO%@1$mo&u```eG4o!Pk&yotdx8^b1<4mYixc6XAtQQ-NlT%= z3R4eE0LespEg7Evh$Rs5JZMyEJFyENbniUp3}Nt}zB~^h5$~D? zRXD%kcH*kRM#m%YFMd{tSAIq!Jb_nbs}VVG>u5Qah-3WCNbdcVKMu7-?GH5_Yo~gKTz-9Dn?!(nUOa7C~hQ_Ku$d^ zK=&d=xV|df1*e_CIUvqxy>Pj2-=@STA`{QG=G<4BoNysB9MrkgSJg=5!xPlgg59Lf z+OMn&ObmCV8_43!MdBBbhg5S!S*T)f1Qj!TohnMVYHsG=ZDFb+)s$~Fta1OIEmz}N zN19gtiBY{{6-1Et-okez*_Z2iWFTj4q5~?~Pu#ZBGIFsVIyz>ks$%!Yb?BeNS0c~z z8F_6Mib5ViQeJL}D`1<9 z)e;_;g*6{EMa{&mmO^=>Q=Kre^dTN}CypnGn9&UY)h*!&7f@@suK7q-bBqM{`US(k{&bRsunLy?43F6NZD~i4T&i8^Gb#s_ z$bGycJ^VG8%mV23%) zBSO@r6z5v-O{!lt-6clbu%pvT%jtC%PDHSgARFb^C(8QQ0sGqU+Nzi)DOqrnlC%>nTeG{_3J zIwW~V7N*J8uQlbS>0ZBd-ewD9+rpEOIZ%W=wPC2c+^NaHnHyMEi5NDzk_GPxYWV$X z=GeEqFTyR&dleG|WXlG~PgWGUwj+0Z-=Eu$qGN~Tvj3gd)~h(;8#}~^kfszpN(H+n z($5~;GHpzBej<>o%z1%q_E7r0=1ixIYqYbg+((Ib!97HxrZB_Rvxvq{z>|lk@lU%8 z-tTb^EST}%$NdlpHgUOX`O>+O&Tb=H2NZ2|4>lDQrfdegyZ(NTs*4uSO7$gMgobHX zgYB>fi3NX1=Gyc}JeN*H9wR?-#(OW-11>7L&d;NSn^x5Aa}U+5xaVWRrzwdL-XAQy zw~KhlC8wlLB{XOf_00k9$0dbNk+zavl&oG!CoM1Vk`Pef7XmWX{Wdoq@Wfvoda7!< zi`hxAZve3*MeQe&MtCbrVNl{sLx7D^nr!R!bI&}um8>pl0re_4C+;1C5fE-X%0N99 z7yIepB0DuEaV?Nr28U$%^;X>gu``V}2nj9%-0+CeXX(aL2oJ*+)2&P$UWc!rbPj@k zLf&|swP5M^@!H_g4y`siYEkD>-J;86TZeit^de9VeQRSO2QbD)TCn6d%rX(k{OJY1G&@o;@>t<4q~&;C6N2^nMFjN99K@&ZyNlWm05`wRDjI z&Z?v=7!e47JnPxTC&i;B>}1Kd8>9-jAxjqP<6Db;xf z;IAD%atnUK%PlI71vS2R%pB-;fUw~nPuffrk0g{wVv=RAiE$(GHXX%X1fg8?1$&Fi zbXXK7F4?z9LpXu4QjZsy$<7GmBflRhi#r0aW z%g}Vkv!ox9bTw^dVa;0Zi8`gP-UHHCpT7!w#$^-8a%Z}+p_sS{-Ac-)CkW2@VD9&G z{k!JGX=MbhX7diEJ~RIn-oDl_!$;;@DlL9AS8?d~d!i-Vbe^=ATX#0z-+wSk0V8lr z$&g#KY@!+0a^v-V2i4fPYcK+t(%$p7W%i10cQvs={cP6>)fg??80q$ga{$WL+}bCJ z9IDvS5cMZY?wny zSQTzim79PS@@^4 zDA|~GAt=l(m>vxgzq_ps%}lHcK1}NZ0`xt_--o3u%4aP zgzBh?MEZi-&jUpLza>hr^i6p@&I~$qRM=NikB(kdF*a>(Mqa31RIo{WssRAkEY

F2=0PTi(i>HsBc6^ENNvG3_bLpMf1um4p5MZKPN;<8l+OwF;HfV`dk*n688ZMr+ z!fB7Z7|p@ekv_HJ$XC^;;VMYerE|tY16%aK)%b2FuTx z(fcaWyY3rbv(LetXy-$H3{>{>$vtTF0n<<6S(ZG7nyTHE==HVHmmbwAupfMn<-=n~ zroTVs$*AoPIV{-R=B>Vw5Rp44w|i&DqPz;_w8F~YE%b(HYBz+GFpRL#VIFriLn-2) zL(THm{`YWQz0GKy`_6q=>ULX1y(^WG?`B^6=@HmSPWxG-6cbhaE^gD1hetdSIZ9dc z4Kk9c3(QRFX4Y}G8J8O&fmM~#(eI*xY3Uq}t)%c29_n$0dr6h3bOEt9O3*6&k$03R z)H#f9y_tj)=hAkW>MMm-EunqjGDW|Uyh1rKp)-7nzbb{ZB447z(L zzFQm8GkQN-np4BS6^+FJ{A?DZQwD5Bui6}EmRy-j1fK}#&A-=CP#I|oVF{eatEphZ zPV>hP?a8`7gzeZlh&u7@D@I|qb0ra?>KKW?w84%=Uh9g=Qo~Rt0$FgfXCCW29A`DJUKrHzS?rAMOA&(!RrN9`< zD5Po5dTy%MA5!ncWauo7%j(K^zlj%o^k&j=A*A}{QS6`d<<86-##~icM}6KRT1LbM zE~ivuj)PVl4I&x^id{RQ=f*Ivj7e$lNz)qH=cqn|_B&fnDy1DnIEs{_+AmmQH2!fZ zb7n>;G6i7q>Qw1r@CL|l&yc{A`e z@qRI;nzLoSSa)*fKtl|nsR*(rxt~u%AA2$j-W4Ig9s77R?ypZQJN2a~uSj8p@G@20m#!XrV)`6J% zQ0aiK33Gl!+eYF0E$$>u2YG*!^b9k% z<>vEuBdEzPASN_J^^(MwPJF``d{cyi;_gHfZ_GO~<0fMw=7u{VTqsGEpZrtqATF5E z7$?gs?{cyZpSOPQ+ohw{HG*M%-mYhuXt$>|;KUXFQ?cpOADTkwKX2{Ux`rd5wV#cPh+Xw$Wk5LFoh3AI?pTxSGV08hDrFuW0eyesE3iHlP+xIN@ zl%m{tgDu(l>#!YeU)km!-n&hHe!ocE2$|_&T4rFJwu0*&rGzEYYEG+r{yC@MhJiK# zJQxK%B`MVc0~>0+HWqLR6bB{fh@yX;yU$5&E?Wg^Z-si?5C?YGT=o2_b3Kzy$f&<3 z4;`zzA%YS2S1Y**NbdSII9=do=SYlL7kyyxZQ`CS9Fg+R*g!*DMYj88X-^By1VgY? z6j`cKQ&}bPBOO`ElH$ZD{3J53S^S$rsspZsAq7Z6j?%CC)yTq5qi^tt+y>%G6eIVm zg`FUCcqt4LaGryC9f(j!xpM)v6-r?k;zt#4)AfxhzxOu+`;q1<*qq=9)IUi!o?2kM zg{p=o-~YDwa>vuBO~eLFsc(BzG1kgUIx!QjpNGd|`CR*>RlBJ`6kl;iw3{j$gK!4( zwj1YGSXSd?T`M%C5KVmH{c>2hip%jvRH>w;o;ZE@@VAwLavNWvrbjp7EAHK$Rw2s( zhak*X*!aD3DjR`vz%T`U^^y>*$nM|EjgEdYi1&EJi2)Qt&mE}iQgj*7VjY~pF3|$e&PuVa? zwC9>a81EVLsHSj4jS#JR3~eBZY&_zz<&?x*p;J&je)M786TKIQxd*K@A`(q3?FT-p z=G?1anC?%hdB-P}0Um_9qI`7Q*$^KHbj05r$ej4FWeD-Y5Od&5$VJ=6h1&$Jhj|9a z@rdG1o(U<&I}D5m>yC-`UR9NCo%DeJM8r0`E{ zFxfvPe%;fK(eEC@n~N|pR|Xn2K{mt&{1qFWPKnv04>MB;m$@}!fL2m za7i>TOJxm=3(YaAP7p<>+P==^H$f2k{?Xx*G0K`lD5bP-T{z9H0F!j>9;Vw* z6O&&(|D|LVoU5rOKF5N&BExiw_*^7m3OnCD*Uh4Dj#C30>&iqDtI{tbl2C}RkHMok zP{ERCvSii)N4*WJusGGO`LaT#=5MuehOx=F6CTrxlME<7^~3X8O3;K|x&7$}y7f1& zYrIyKZ^M@CHzXjaKi!E!?^vnY&id*>vLcO+4*U`DxLcc8>XF3DaMZW^ZsBBdIdWmg zrv?#-;ys(Mh^<4ZW3DQCFn7 zHB2D*Tl6Kp&@7fk|i z{XHX}_&q@}3*XZaZ?-vvD*^M#DZFmn&3~Qap=6?EclSH*SFBvn5E&DePgs*r)8oxP zkU-@Q?jx6rdC1@78^P=ABeg)K7Vt#Y!$;>4=GX}x=C8h)uz+*cO_A0l^%*KyH@I0V z^)35yG8M^WuZO&u?Lv_#X?Tv65InQQNh5E5B8)L)7J)HMJ9tP-tl66#U|EFXvOzb! zYH3jy*(0m4+Q>Cm>{WLXD65%7bNLbXNM;y}t>{k06LI^EaGDCsf2&l?_BL4NtGq{J z61>CYS|0M*bp-h`%%A=32kM3GdJ6kOkQGOU6Gf(ThSU{{^6=9xq-7$+#3@hwdp@by z`lXB$?_!2e4%WNsq?oxaw472US45D{Ij7l(e2HiXF6xD2L_HnMILBxS8#$&*Y_R5d zP9gNLXBO*AuRQ1+#2B7@(PdJUn7hWKdPrcu+AkNu4hgIG?Q5i6q(yy~MOVnmcV5mp z+I*8O3tQcbX+UTwAc_an(raM|is^CGM&@>XR)t*soqz^Jk{lxAn8biv^>*iwOC+1* z&Wbl=(Q_%4W^E6D@wiUywKv0pe}C<1v=bpX&N)}je0G=hJTU;rqcox9>=(HrjV9SWi`O2- zS(x&0n?I>9?^P3kY^-DYAY@aeK?XjO2QdBEz`Ao98hyqYs09?h&^M47ZDkmRl@+ep z=a#>5LPr`}j|2DMW0R7+T%0H@6IXcS$JS3kS|_IMJie;@3Jpnhfb@3MOUb6I$qR0| zonU>!Cwui$HY{NM8PNRs*LDzrj=-P`N&KDj9V=5N*+ePsJW?nL?T0& zhb2LT@}n0VDu^1_M(i`ayVmN&{>`?x-?%(j{ll z#bP&iziGDQ)kkkvuZh;2oxuM|%WwH$$n`yc3La`fW$DcWRlW3~G8Dv1* zd}GnPBTs9qSkvXC3X%G=5DKy0;z_T+iTy6~O8dRrCpEg#$7j-vP=s?j`Gr75t@+{k zougw_p*_i-h|#dauWqu%KrZ_5b;f5P&FT5b@yx8;p}rd&504DB5?(AR!JHpORro){ z+U1zGE4oQbN;8Y*`!u;**&ffM6MrjdxcV0{2eBgD>z-LtC4NS_q(qoELr1NuWt4Fy zpb;0NRM}#UU+srXtRbhHImp5bl~*-7TNj$2eU;axiOvy2mKcPw?e8fvTu8Y!Hdjd{0$NCjti>;F`AY}*UJbGUV$Rsrx(INpEud- zO9C)-1x-yHtR3fC(17V|nPB>}d$SIj(xI2~*=^-ax*c>-3;*NjES4+3 zDGl!ETuKsY4+^+Mv+J994_)qM#mQ) zg;BhJv8%0uVWMupCMP~4TS*4<@VTc3il7$|D7z7m)Wks-N7Bw_Q=qMtSEhmcTZg{P zV6ScNJ3{EdMfZw$570T^HaZkQmuG=lA|b^YjQ#h-=8+Qu3S-|X(%EfR&xrGNk^N{{ zYdnLBSQ8hkUN5nxeDd#%#3F#y?12DNK&-!!c#@FGx~@#*iT+tg+l#cnHvv{21p=tt zI}+ZpZxraymwZ13OBR_v4R*^@NlFfy@M4$zF2`kccwPjdOco>ssKbD7a_StFolqBs z7PN;ZbJNt^upHUx9Tgq#=X}Uf6NlN@l5Mr#tiXwPl3<{7y5SsG+SDT8=4&E~#Su!} zRyv=VIA#dOqD9IkZXnh~5#L$Q<~Q3db0N4jE-34OS`m1roy!Bg!@ZgK#(WO%@Nx-* z5YwZf2!?fJdmbahwbBPgH(5KoE#T##W-F9(pCYJFK$s=6JH}@IHqe+m6 z{t~vg)2Le%aTK1REBWnB?1=lUPjodqWJ3xf(56#|G8SY9)A%bPal}9cxYf+I5@t1| zv{zHTWSrWs;m==ZPf9)&Vv*VeCZgyR+o9M9Ia{L3*wF0V-!m`v(}AJzs6pr%BC%*k&UZNLT#edJ$C)$6S(~ z57$Fp22H_bR9M&&=Y6O7@%j?^M>HE=CkQu7xbvM7r#{sVN#1zUg>QEq&aZ;EG!jQs zzE*lRjAkoMNO|LgCju1tyKz3w8kFB_Aon>jU}QF~Q8nO6dJ^yt96uN(CYaVs1t++I zs%(h-VqDq}g&d#EZB`ZSD>*845N1q#h}hb0LQ>P#3Lh@TBY74@W4=d(5Lt)D#n~?! z(X5Y!&fYVoIm5zT^<-HOcLmf%#70@!Nw@E_&k+)nLZ7&jM&T1wUzC~nBI`i)Q(w{0 zPIgJJ+?l2~%iY8B&h~tAF{gcUKT;#8B{RJUy?k(luU9+W%GNGSlR*bb@Fy2$v5>g; zLsi*o-N-c*BpZli=Q|NHqs;*=(5BSfr=M$`N@D05As|*bi%Fsa-C4V`x}C&IcC?T5BOe&> zwvj+n-dD|MtO+*BoWX&I7$X#mn(Cq3NU>0YJKZcfZvr0Hee0W=tFe;xv8vFGVnRW$ ziWbiGfA6V^PQFke6%LQjfXE9Uz ziW@qa6=q;ymx(%rznMn;eRhxsp|LRA!k@CNMkZ!OZ>3Vft{j?~eLRW7By+ z2Ue|hH0c-`Qzydkqr5^7*+?|(-_KGl^9*<_SL0><5y+t5LZIl|{kw;NWkvRP^VO8S z+2J{8ayeBdIQS7)*#d`5)x|G96eYySmsdR9xBQ%uY2-^;KMnM017XKeW|#bp$3^tS z2e&mU`RIgPS3LOj%z5_44U)WBx6FM9jKYDq(+YUXG9*y%+z|E%X#SaF`8$%0UPEG0 zNjrOzsbAfd!w$l+$6!$`w57%|b6^x_n0jT%o=+u2es%d_&1xUREV{6P@1IpJVZxN1 z$((f7CvF={MH-kc_6dM<@vvF&iGA7?y)inI-@{>kc<(dXBgy$LV+08}OWre*!5Q6m zbSRCFD(Qn{*Q7r zvZ{lwG;cj$tTY*67)vZXXgEUA+&#rZaa0>0`!LsJZlfOm?umu+79DL{9;b4!O52Bg z_n;$>2Wz~1C$)3EP&Hk7DKQ3HU-GzvK}T1JBxP*2KlBoRK#&SKRU%UJ*^@H95_#N_ zwWO$eX}O$nKq4u@(XiCXJ=<+TJT&57ycjoNxOJu#rt>>C29E>Cb!XWxZ}NGvpdU(c zjIjuU>{8))jP0!;!qcAL%=5XOh4*N_HP}#7PB;|>&t-8Pu4`hvNs2GD1I>?Hho5AJ zttK2VN?n^45+TxsAac>XgPD5hBu<-ur9(Kiekc}v({&rZB~^P$KYq!L9xhMKXQ zi~Xa^p!dZt&%E>Ii_XR*j5Fn-$36u~wqm#~8MU;53$U`=HTE#`NnbNL<9l+4Me&D| zffX}aOq7H%u9NfOf#A!*&a5BZm0FkG)wOn;o@P~8+56y6{zl{)17uMg(AzBAWy~1s zu^y@>tcm;pp0=JtRVo+bR7_j(cEygnG_h_s0?}r}4|%$sc{Yd5q^enE1dONVf@D`? zeKVjA^)yJhsBhA?`=G~Py=`DNqQVcEQ-?@OY{}3!VFGI!QCIbX@2GH{GMN+@&wxD? zS7ZmL$&mk?H&Slz`t;D0ld!^0MkJ-^+)IWFquPGhXCy%~rPagOiom+dtCjV|Dm;8i z2R(CQH}37@th2yqbY>3+rYlFg?mS#A7M~gexkERG_QHPlxr5dxIFc(`Tpa4d={Zdw zO~_43{1$GDhV%b&s)KYO_^>v{+WHD5PQ~h8_HJW2Rf#+bEj3Ah9xcYM?f5;OVM~+N6M#jI7p>$_nWEL)X78JcskymDI=`__{ePW*SD*La$ z!mLZk(ti*NOHcvy$>!%trocr7DP7v~krxa_jLb3o3iS2}q*FKt^t2f&4kPh_l|>MA z=&4*7yB_E;+v`|J9Hr4PWP)YQuG`WU>Ordx>bGkU8sS0;6B8jH zD0pBE^TL$+cSN*T87?0oWb%d^W)aU~$!OqPQkO=}VW=*d(orLFjlqpws=rG@kOc+Z zBdhah_V9KhYKlCqL~8j3)-Wz?>w_b81kB$*Jx%e{qDuh&>YGuEPZxAMOXOo|Z(;EMrmVCRI)81zd!$|LM z=%|PUah%AZ`8+*>iDrr|M@Jp>=Pj|X*3-Fw=RA}jd4AMxhqT1{ zorz@;Ezc+ur-PbGcUWrl2Ux4HB^2S;bS=r7etpNhiyu#NipHk;D6+Mq2%;tSeIWtS z!^Y-N(mZ^yfo9m zRf=_d&3Rwf%SpV; z^EJltWfc2@D`ecqJ zv^aL)@%`oV{-=HVQ=jS9C;4@rjynO$*(X>T+Odb^Lxzo%ZNYx9qZy5T7RSaanQzZO zDss1|hZA~piTW%>hKb8!Y0Sw8aTEkw?qvPly^=aZ2cOiGHZ*9iy?=owtV3K4N$pN_ zeffyaOu|%j1_7HxRO6kx^Qyfo)2O`k$3h5-hVF`wY)Ps^>oD)jjRrk|a!*}&HL0!!L2Ej z=A{5etJZb0bM`kjo2-U4j38~t4b_v7*8yX3cZxoEe2U0sMNp;@LAG$M5&!O|=I z_(*osdbqk;eYj^vwC>6mFSXIQ!R0NdV^5??Rf32TKSr#7t($ikgoLM@OsPR8TYr1@ zOXFfq3TKo`_HC@po!}BN6?*enN}j6>k;6zs#ox1I#m)@den*8p(3rky0gU^_sK+(` zuJm^*uewtkfYO2qe+Uh}7xKhld9ed6a* zH%%mO%hFEYe*6T!DMt}~wzI@rnfuepfTWpME(?#TX)Xkh# zDA?bmFNap3qpVvN8X9O_Rc;2!Jlpv+iOet8Uv|4UMM_K#d-{7t{~c0NN@nut#(dL@ zffkc4+|D#+R=)jS?7Hm=c~(^M>f-u0aSq8?wYYpTf0oWh zZtpNwyCN;nH?VY3h+s3x`TTrmN-hf9nvRQv1${O=s5*xGibqwDCv3cHl#OQ!0P4Yt zLZzG`P-mlZ{6?N1QE;4nY~S>^OqKGWx{{sAv(+>|Q9B*3HB30Z zabN&A-5RA>iv`^5mM4zy-z(9sv$o6_qz@noof;b}35!+#x6?*pH}*nOW)9}Gfg+q< zKP{7yw(aWQL?vxA9_z{UOTja+mheewiHFiwW=lV;nO}W?F>TyQF}$%4 zUB-|mi9k*-I&klyhJz&TAfaUS!#N7S?C~eNPJOD}84)Zty3{hhMkUGRt{3njV~>P| zaS^~~$QIfy%88|bH%ZZi3bAC5nV)|l@Y+gN6vC8zK~sBzXd;@}$*hw?M!*wX0Jj$o zvljpHT1L~&Hb*RklfHl{0gUxl@cNPd7QOMd#Q8blWC7jk$oxG9DjmlTyoc=Hagy$k zT3(T*v*XQuc9&nwVMC0)<`+C+z=39e)#$is-{2lYuE&a*k8f7ic?v!mU3cFprD%;$m%t84%eLXeAy}Kb3i8~RX@@Noh z$%7#*>`s-3VQU{O(}W;nsW@9J{l*u5yQt*f!OOg(%KS$ib%zK#lEpk}!1a+=&=o?lRu4#7^Ua&^d2&n~VTG@sF}+3E zt6TWj-JSKQo=3e=O}(!ZZ(+a8v;*G|PoruGXF)qrsgB!&C9o5$m#C#HWM65fl?X?0 zn8ze7sp$oA~DH8NnP62sq8G!XV+ zcx!L1wrIxW@Qt;oTKwba?W$2avtD>LbR7zGlMdwYcT^{Ldrie)56S(#ipxEL!d9ol za)8|$pT(%AYD8%5){p zinJ&(Ok)w7%TXAkGMHs2^6FAn5`v@}r9LNKZ9?_6S(>9Iry>^n-uwjMcjX4_9wp)< z8sUX_@rUszY<+&q924j>OWI6+nJA0kDdz?Vht^ym2$N2*Y1xn0xF|DOfBXsuQa;>df8* zsi*WdB!p^3=8UegW?=&tOzs9;)>=qb8c{+cqni%qN!WGg#5fOeG_lOOU!L(i_2ky3 z@DIHpS@H>z;OuKHr!~DdIyV59S2veeDqZRIGOjnZ-Q>x0({*SODixSEfA@7m@h8k^ z@N5Md;K{=!d|{x2oR!ASYA*Zidzj}p+rPQME{ExCHC0TVWa{yRyYi+TDuZT6MfL2o zJ9c*-O{Vb2iRSX4y-^0zV(Xd6vmtl+s!j$Z)98}Sczn+aYd#yljli$LlltRcSJwBr zcl)m4yj?+2F% zP~z$58)`e1n*ho-T{(Y{_&gmo(^oPKExFVek@%0NtWnQ|& z)S*>vRfSJQ7zQq4-`*m_?aki$wds`MF zizHPzzYhE+rpQz(3l_OOfq*T0Ny%q}19h-z%g>U>iNVr?yXB#4Z@ZIfS`{{`rP(wS zMONk*(P(rdR3AJ8VFW$3#o1npM!7vP5X&Tr0n@6`a=;6guy6(mAEmo@>zjMXE=i>P zLO(oaN3aIOpQi`zG@+(o%oQqh=u_|;IHAwT+2m&7tn+ zbAgj>(d$V#uR!%=ucks~^lGt$RjU`G|gxvN5#;t=t-<4v4qu%!mdN{Rn?j8BH&mzm z#P%@(qTEj+*z)zq9p9DPMVBkx>gAerF$wpgjh-uDIT;mlWJfZ))-s^H^gi^)4+J+b z2y$Pv%nS8UP$I&o()>=ZevFfrobCi~2$r&Nkw8|S;+{C+vpsRO!~yyjg=<)og-7So zq9kt7ubmIukwWoeB_Aeg;dJz?7Wy*eTWvK8(9)n{JAO{}9R9z3&F1P4A^>P}wdvqUa|HpAp&5NzKYaN~bm zYGrn>NjV)6_z#tS*Dxp6ituyHGzaZW@g|6#iwl}IN3otPyZl62I{dHy>rg~i*Sb4;?S2v_p0Dn~3&dHSCpp!wnt z0EsJ{P`?lwqf)}C^Z0$V5E@9=lU2jXW5iE}vkDY_H?9tlV@8-`mt2g%$nB|suoJ$JO zD*H|S#J6?_G@QN0(+--VyZScFENXl0?UtNNH{Wz;^Bcbz;W(V@mm7??X7RZxCf3Xg zZ%>QMujbAf9gdp?cP&%!VbWXf_HS?kk*o2~r7uW|mG2^Kg!WstGI3YuB?6_;xwbuw9T+A)Rfi-Q? zkT2EOt%Ttxn$BDdcj@73mM!~VKkNqq6ffVqn`OxHbH^FtyO=9|J;-Bdz#^VePkO8m zDd8#g%aTk#mP9r$$*tOD#b> z?xQ)@@ByC~iy8bSd;!jmzdmo^$z!z{L>*Y+Ym`R%3Ak?g>4?Mm7>#~FM=)UZNb+*fdRkqCtpog%7Ve-4#Xx96ZM ze1eLMi|~T;JAi#5LQamk*lzfx$7Q~;2qU35tb7sHgmn6I(Oe=FA@4T{BC*=9dM(AOzCme7yU$5|>ifNBNeO(ejCfN8_E-Oe7M{ zmYRgTbc|LbtH4lVXNQ!zdGX}zyx_78abUY{hux+euYyV%CnOJU3eGz)H#T3STS-sB zZ*z0RL9`>g3!&eHuc5s~(c9|85!G^78^SYsIz$z6g^M9|B8Pi?Zk@vXyW^b3DyXM; zyeCX$(KDm!n~lTb(^fnliRDYIcXNqUijIQ_w=#VD92Y8EatOXN?%z;B4sFV5k0x%rc(AYW4FHsg+lck6%BPAso0Pd@7V;#LHPSQttNyua50H|=RSy{?d-sM8w5KxyF-~#0mq>|o z82f|yjv7?Ou&5LmtdtpxXobq`@pbm8^y_?9L%!Bd(f$gjOs zau#Yt<_MtA0h-!c+;cn%;*r6ZgZ&B-|~{Mrhu8e&xNTdWGA=Fg-Ar8F|?U?d3V z3UE|fOscLUa!h}L^!x>*Z}RW%@*!$OLy3&c!iC4g?jOt%(>znEEB7&UG^mA@hq^=mK2Yv%;JU88m6DL$jDtJp0FEn{$2pM zgZq6I`|`)Cp7(-z6A$H(Gp=_+vISi`zIxSq_RLcqx%W*!Y>X_^+R>*wSW5y>O zPv}ln$e(08q91e3x&|qn-R2rJbSEwg7Ubt9Ysj`0x5jF|TfWxWY@IURcCC3EU_W)4 zT_%2D;9=ynS2}++tufUZM`mW~o2 zOCrxResr~HB&g#|{K!+793V^~ngf$rR zSfY4sfMFh#%Hnp)lphu5vcWtw=GQRH-yN4l8277=*){dK=U^SHClDCCxVKRrEddCc zKzD&D`@icAko(S7R#M6+fOHVkv*Jh#msrpPFsV+!?zIy7jvMz;?2q<&c-8SBC?Di5 zLSLT2Q!!s7u@xy0kS-xZM0|7p$lcM)jYS$Ex2o8~;|z&n`>-y%g`X*le|q{MKMb{r zWs7Wqk+80#&7?w|`$}>w@>bQPr}sxSyPc*6?84JMSbeuvZ!=2KL-sK>G&h<$OI8Wi zk!FN*nyBM!>R7VT0F!+nkH&CR@C*01$eZ7{HrVT6ent;~JlkP~WH(isZR)o;k^v%1 zr1k;{$GPbIAiQZ0IFUmCZg=WCyNphDqGie=hvbAyu^-8IfwgD`YjeV4jpxr#VnIE7 z70h)OrY4fxRbqh?aHGkZM+Gs<)42CHFKOCV7$V-qFDb&)cY?sY%|eCM+LDO){dtR= zE#Mm6pUE4)GtIoDhvZI!Zs*hR42kA)b^8cQ2xnXdR2|Uj%XQ35xkP|bUjbrN`ED)Z zOI2cuqZsT5%fMKH#s%kWX1c)uAG;=P2|{ioDR=*fc|}NMB#+%svK?ey7tN6qw2EC>JZd@DS{5|vXLA+z*^p|q zmrXWD*y<#MsQF68b3U@v@%4Igkkm+o)hH^<2mxhdsgT+3&E62AZVGI*VBLToGSK>= zI!JPeW_@N)NWg!4mn0h;^`QivE14r$_1I|Q$I07rcAVdo6rK|0ib7^y@7hn^PkuW* z%py`aqz$7h*TModz2u{Vh60;ABdv?j+A!-?Gr>Rdy?q0mT$x_&d54pBoQ-EA^gTd4 z-UkHLkiSRJhYw|Hs(yytIT>2NiVZ!X%<>UVU`4U2qoPi*CI;7KL)!>Bb?4UB&+_c& zZ*l!j`RzV^(L_DQzrt-<(F!Dq70S0rAS`pzUn^i%I>u6`F84gLHM5tdVhmmAd zop1Xo2ftL+bV2=o;6B?YJG$**X=_h6Jw#0kRlsj0mc`Fz2>3~o8o{0GWgM1+6JMGn zs-M_R%i;UfvLt6pndJqeLhDy0Xh=S33Vp;l!6G=s=UNOYD-_S_9UY$q1=P zxc+|=n=8-@`dhfDmlmofU-?7a?Ta*_!n?=09{@jC1gK@_iANO+m+f&>0E{ijxmgE` z3VFu1Jp#Q?Nd=+LdPh*6VYhfw+B2`&ICy%^l*69x2(O)65D0$Ib_f#>k69nujj&5J zwB8mkv~qCMFhB>LiTzUB%N-n~Nx<{Ykc+r+(zOb1o3<@0+!v8uq`Q&9MV^T0C1(0G z=2Yu*G{@5Jt7Dyzwl_A`ug0-*R;{k6?5k9R(ae*Q=^KT}^$+5DhnJbpE zg%RwAWUrdokv{gu*;U$pA{R420A>oZ*ZmymvYe=dA*zG*A9Ke=8mP$M%Vg;61)XMc z)ivS?@d3qrZrliIGrYU-%TUA&I&V{Mx-j+~RyP$9jn?yIS_4%PziJH`WFw#W6{U+S zY-GonL2o$R&a>Ty7cD)JX!-jvXCLG!hU{EMIIHk>e->>@5lcs8+3~m;h|6h|l7KQL#UBwQqS{Z-k%;=51gyCHMWY4Hw)BeDvbu{r1!cQcH%WWaBOiPXX zcCdAn$RD>?V@+PXf4f_IW+>(7eeqT%N7lMi3&8Bs)x+G(wk#XDgltab`hvz%PQ?}; zT*s_bE&+YTY3|BjG)P;@HzMKWw@UpE#;#~Z4jye_chKG;dG^b_62<2J=I(wDF~`oo zL*L!B$cX*HR^j2f3bKnqEXg5C9!3tDl3rS4PHiPY_}FT0-1&77->F*y55o7`)WqRYVr65c1-4q`?rfdck4Ub z08HP7Zj76+H*DTCob|Awx-TxzaSvQxGc$S$eR51O{u*jpuRDXQYySc@l01{SjWcZ@g7!Dm#D@52(|Mg zoGhJk%}00?bM|)c3}eHmmxZEJ4j5Q(G7958#m1@jv9e`c^tVS>e~%h1VPM)MIi74v zKVgY?2$-5pwa#K9pkjxqU_UA1PK**2RiqTrKc^R^L6a!O4?f=Opp=v$khdl{cclW4 zlI``P@O!=_BdWhHZ!F#$Hg1NOjKFlbsG;$@+WYTq_sY~Cf!cR}dt9%yM;kM2D6j^`Hr&V`fo+FNVwBOJQNSu)H$09KOuXjaET z`&mRCxlHQSOy3|s;v+*Ck%}QpYnb-f>bBj9lu%k|HtpjzSRK(l5m1Z4FHHUxXmg;# zwnugeFQc*9EQ?|9)2s{n4PTaK_jK8@t*XCegET40dEEAqJ!BnwCRPu+WPW<+_7jK_ zj?)(!7rG03HPN0qIzU@dB`DFq6{IT(gEd&=wVASq8H$fY5$D<^0dKE zvM5<4y-FGppxEE`IW%2tqK}B%a2L^WRs%Ngak9uLIFa{g5;hdxD95U+F0Qd^C43BV!oW} zP5^3_;vyW&36Z%T+Ri*pp+!kiruCL1LK^-GVML3a9{KJb z+8lTgO$stD-vOxlnGxuAqr(mzMV@Hc#maoBbqFpkq97DVSf zkmN+?^T{nDi0uPW(116#tUCt8LbRJ(&YQ2o9F=^9Y9jUiQe;GZqa6djjX{t{sFKyi zq6#Mxd;NNG7Do+Mp$ob@X#$nmD8XO{u5`OHTecNyRQ^&K(X~TIvw@$~aPMf|+D>YlcffkWH;^zZ*gm9^Y1qgAJ7OrRbsG*F%iiINOp!p=*i=u=O7qIFd0@^QD*g2;R$;~TD^6sqdF|zg&l2AaxojW zQ8P5$lFNraUj~1hkuP-m_Xp@OnKw{HX9Vk96u$MU9d+azaUT(@FW}s27k5@_SRB>f z_r0)6I8PUAtz$cTbd!9$tdJe+gPFE_L&`A%A_6hiucHB1`I1wTa!C>(|Erg8R2jOuE>uC1JyF8?l96&J3UjmU$&>Qvi%XME5jMqfNt(y!Y)tG# zF1|SBv1ZfFiM1y-i10@M;YQtHt(4CNnAlj+PX$PRZ=iFm?9l>?uRE zs_NP=RwiDE%J6J<$JhPzf~LwGO%tU*G|k|(9KsJReJdb3Yz5Z@d0relE%-;Ark<)# z>nc%qEXHR9IZ$wKU^ISppV`n??VPZ65ogMFPadzfL~fV6xBLi2Gu;pzw6&EM8MOw6 z6j;;J7Twmkz@f0LJih6%tSm*5TdQHgoCpMC4Z#!VHvd|BUOr($Sv~cz zQkd8LebB!xl_K$3;W}}8rwI|j`=O!f~d|xAvs`y_HmT(%b}dW`cod4<{PQdY5(F?aqF z=%-V-!>&_|P_`3^5M>vSX1ls`pM2?s`mh@N+@+3O)Zbu#CDv1Tl=#J1+s$E*RiBNy zyd=3LkUmB)_`FfNE;)Il&pb#+8C&X{35+^m8iGk5FoRIFs;eQaN^| ziPC*UiL)7|M1lhmudlU}W|YRc<%e~g3iBDg@goFtPPMszG#X~ng$&rD$@(c+Wm=r> z{#UzWzyDk@*cPa2R*vBoWJFp&VAA}lbRZmYY9qbLsC`gPoD|^K4xRFhg;Xo*NCCLy zz@nkzMT4tS>rza6EMi>$dn1pK;Gn-w!Xh(v`w485B^#~d7=J^Xy2^qmzenV7$lWq+ z9x3oicIHEA>U~nxn~Kczjbc_4q!o`$;$_fpW)b+@uyt+7dLi}=D;Z&ge%7sM6TegK z28AOAzqg1=tDQ8567V-}X27RR_x!-fngU5R>1` zdewyQ6ioYhu2<1wy@nZpT`LH+=pd$1R7RIlWxAtnAh2x)kzJ>w6UP^*;KmB*0I$dQ%D`8_K-_w3BnR8gRrP|#s2VNS>M06pjm44 zCJJ05Z)HWcDy8fEo&HMIXg@kQ16B9*pgL1f_KZHzUOJH_*qOPH^*IYrcE%kfDu}Hn zoOrxteynacP+rnv?NNBqHTrVoCQPFi+azcdYf;aQf50=@!+6UDT6+?m(ajp;cr1%3 z>=MbSQo!M+t}f}nKhk1r$PFl&!lbl zZ_XA+ukIu098k}QP__PxTS5OVHm3mCxrE`6b2`9pdT^WAr4v4i()K~&W0atwz|C4t#R)_Ch526HR?qj zCothV;f{Q)Ym`$qiAHl1Q83LlH*d_%PS;+*X=bX`3)cH8T}V(oJmR2%6^Ei`?}7eRrR1K7kAye?~@7GKBY5%x1;fX zu>qf`N<$>2fQiR?_pq=0-7fnJtxNQw_OjZqueEr1XcSMXYRM-2-+aSrI7oC_Cw}B4KO8;l z6hYKYrtJ46tTe4XD3v>=M1qu`qllwLGDti2oW4Ir*Lqn)))@pTOa7V-uPE3xHz3v$8zs#1TY@NP6~McymIESxE&Q0OiB{cMeu z=viA#A}hB6F}u|Jd#Ysavb2H(V=Ly~mBcr7Kytl0uSDw^PfoEoCUERiPu;=so{5@z zEZWVT!`sU{v8GJ(D3M1LWjGH3u;vca|F+ z{-;U2TEx}jw(tETu%Mp>Obg4v{<_0aAF3*xPxKFM!1yX3vI{SJv7F=aq76|~(Pbxd zN^To`VjThU6!Tf1BUWArB{}&XhDzB+cpBHtT&2pj;Li|(jYwNEPyV*WsHY$%-!JDV zW3E~uBRn@jCzdn7hcu@m*Hlgk96F`VaVCulKDf%6c891)QcYD8?Gai$y)CC$ayR_- zPtM&fwQF$=R-B+|_RCZCg)Z_Gjp$6EFYnV1iW2=I8;LN&A#q>4qe5FO>noS03~X4G zlW27QLSNni`K&8lS?kHJ{?6Cb6W}CdUb0s0ELYV;O?$A!gR}h6vk84u&)W2H{>5>( z2#!WtxYpD}!`&gZ&&CI=3!>R`I9w)f7!lpx``Gi{Co$>q6oFFL-dJF>qd& zpq_2>?ihys7;irG_)Wivh^S6$&>Ehwu*`fQ&whhDC|6;6070_4Sgw5p6RDX%*t`su zvDH699xH-ym5-8aJdGkW;T(V*(qzhQbTwS9i<>}%70Xd$`VdQ8wGFauVzUmc%kkFT zfVv8aL;CVb6KtZ?KH7E3tc2%!X>`Qcq8}pfeYqGtY%Ks$%%O_)XO)BYfRN9!-2rKN zZC*}tKj)Nk+%^4WC$-dug3%Q)VxAAgR&IkZD5w(NyS2(XKwrBeoSZW!dLX^1 zycOjK1>)qHN@=Y^lwDRyJ|?R>FH7&Sk$an5MK@}r){jXD?zP_Na&MLH9puomP#wca zR59XjmjblBdD*qo(9_;M4mE$p50xpzaCEOKKvFnkNy3+nv;<3-F; z`7v-)T40govSLqwbBfa1rAEX73LPhLIzqGncG6E=naS%sX4xrd`}-m9PVD?k#uFMr zH0`WmWE7%TyT+)P>fP}$XtU?^I zA85!Ed!NhTf!?-C`Sj}vT=Hs2l|C`3QM)Fd*c=5EyWnD5*5a3rdd;hcUWYGJhLrSX z`@Cn*O4jUzVT*z$OS*L+qxX+^?)(Qmq!}42jMvlTmu~K}qMNAr>0Yq1x)!CN;j13( zX%w}>c^vjri{uJ$*q3LBQQi(x#BYt1H-SU3FsM^-Q7y_rI!Z$GjE=sbW^wEZBP0zkBG7tjb5Hw_RL%Z_mL#veVrk{&|{2 zxS7@v^R1sDNXb_y%bCU7f2g-0HI^Gm65xCmU?u6E(9`<1DGYh}RcKcZ7?g8B0JU(t z-dV_67Yv&+*sS?5WA-So={-u{EUS1tW)n&;ud$YfdHfNK4m6z>AaC^N`IuxP@f=I< zDg9z%LjdbQ^EqTK3_@qzuj zlT%DI_V)O8Q{=~k4pjv;hivL|a4MEJpoklO?I>Fq$rlZAogsN~0zT-)?9{68iW(Q} ztsU9Wl};B3{3Zwiv;g}JC)-|X^Gd`v9nDSXBce9xscv>N?|dsM*$6L`2_jM_dzWds z8#KUwg))v04Pygu=343;@$OX$&aavX+KkW zjr$dmu9sJ^Z~U?tzyY|jCx>yn&kODR2?LflD9;63q9Dg840q{IdM-PAFs2)YO5COVlsh zwc75^u7ky5@#?mgV`Fr8mG+5ldmbPRC6q0yn7n>Jlu;e4e}4mXAe3ftfVwbpx=a0J z&INY7*`!KmPO+8&9a%ZIchQ5DD4bg!?7ub$2!wiO_>!9x0tOX($ z!nq|4l!(Y><4}unensutE2XP%f{C913#XWH5g74C?~gXu$JJe2TwC8{niljjc)-o@5o?vgdB$t1#r`C&fkh=s|&J{HZRe7^jrXb)o( zcT-E+R;lUSECE5q?&LrrefbvO7v0DaW5ylTrq;w~MsXmE4Qs1BB7*R{Q)0(=%0Jr+}UKkC=k`{_=cB&tLAG!~A=Ri?^wL67zm0~Mim-;p^=%>1VI?tXtsle0a_!Tki>q2jc;q<2Ef68?U*>f0D}w6MWE zQ8u-nfwesh-+~bDg{nsA>`PZ{Qw|7#G5C+2cpt50NmO84X$-Z+pS{LUQ4)V;y~Mbp zL9aW?18J6($Bx>$k? zo|WC{S5H=Px@A>%wf9YOcvU{QJ! za_3afz4PAYY%q{t+5EG!K{0igpf~ax-bW}lym9%Qg4V1(C2bT=>t*2 zIYYupe!V{(+n6>ZX0N$W^!wGnH>AqVd3r>;<|RBY{@DD11^CYvb?_q|f&BTfz!9ZU zP0dI6&N9x{n)|E&Gcdk6^oxD_yWFGL;9$FF{Y8x(#|4Y7`h7aX=hZbLuc270tidON zzg%>%5)TOF4b&|kwC$>j&V%8sIcVoa4Z$WXlj0a9 zTgQ0Yf7YH42P8&x-@_sE%__fEzoA|?UoBi|&HrG@f5^PKY3mUVX320|kzjwt&KXt0 zIPJXOCc9-h>noC+9geFb{Q$cm~yT$cBjoZXsiNURxSTG7nabNfg)eYST4G zd0x*)06}COY0ZX@frgW=31jEz>Tr&6h}fCWf6jgHS8gSregF@zoYn=PFrj^>?O6Tk9T7cv4gRMqH@*YC`__GPuVpM{Iwxv?$ z!*gZxTrdgd0CdS8MeMuj@d$rpF!ixH64|c zU(b`XK2a2*rU8<)Sq@Rt7JOJYKTKZ{e*AS5uL&pmZV{vpxI50-hKH|4h)2*tY0;(a zY4NBmu~SH0Xy#4G>i+oT2U}u~w4BS`m4b)AVE8yETxc?%adp>;jEHqgWO^Tkk(=bw zWQZRG`wj0wa*AxgSDCmcoZ|COO$GOOigOVLHk+GOCYn5Za>3#}oDe(zgKCj-SaX zA0nh(I@3EL+Nk^yck~awlQl^-YqORuk1Z5U@ee0D5r$9j|cGEYw&wp6m!Dd+M? z5&qsKX5yD|IoEl5s$SqQF34W_cC{C5UC|dKxrIy>)Cg~n>%P%;(R42q@R^wTS2+o@ zrczcDIIPJ4kQaj4i&K8!24xIu|Apr_A&W<@X@Zo$k~Cg@6&4p!*k zmdF}03x2^|-wc5=kW6*Bw2nIFJ>A9|2S=Y4oOiS<&N!n}OSKk3deMk5OBlB?#nWTe2i^&5_yxr^QtwU7=3j51>)};tU5Qt}j<0e>->fGoFL-3+38FK*QA~(X_DYnQvYAS(E#!)a5GmXLDv76x=P?vh zf$E6Je9S4Iokz!8>R2AXyajuY!a^o{xjDaRD23g-w>vB=kZFG(^XTd?I74$<4dyBD zgMI6G((%3_mB5z~o%&6;n^Bq;KkLW2*OiL5veBbOiyySxt8*dcCrx(qlc^kx>PJP& z%2rPP+}L8)t2RF~UU-x2mwpp^?R@`pr;xs;(sF7@aZ^vfcejU;LgO}zlx%u-WO`Bu z7V&!${Z{D7+*9+a_|b-SE%mh8ZB=}uGtHWt^uNOqX6|<&Y76It!Qa8aafBe%N}t^X zbwNrQmvd8?t*F_o9`m@rAn0L82u?=ROJPpE&QFWA{HUihDuxzr9v-*WjA|Q-9mYuI z0Jk@w3Txph>}T5&mHREsPFvjE?HfKE+#D(|90`wSAwYYTy-Rb1w&!)ea6_e$A!3~R z21Mkn&76<@L|JV%Z;ToXKN2Rl$YJdfosjKUNTGBWr}@>Cvc zR)Gl-+c#U)Wkj30496P5v|OB2bZsDsC1nUbfQ;JtK@ITh0%ty&h-d8M<=`7AtN$VI z)8V~oK{6k^`WBraU`{K+${bal+*;cZFVcoj>CKiBDII+q6_$DDdQNh8Eq;Du?N{JC z`g1lDQr`UEufq<(>7|H%os8LYKR%2GlNUm>cq z3=hx8G^!6Do!jc>eM!&2cw4IdO7m8{l>dr*FyDvdgqG+Kw>F0m%pW@~Cu_N*w~lQG z8OUUq>=kKEi4`m=_i%|;R*>K_r>U*ck7%L7#;$p|_EwI|m1?dQwh06D}9)3xzi*9Ry@3Jbf6 zIZIfB+8#rJ=9W`jj!1gWHWK$YfPdjG$qghURf@vlrfo!~ew1|EoQ2)3*kZ#j1(#d` zG+sPT?wYAoLgSEZjIH1+CJY_<`Gf7PhJkJfLM0gCMb+?KyUDYyCkfHYF$+}lsJ$k= zPoxR7DSu;<=3V6XXt(oiPxf4C=^j13NCYqmHH+4c+le)7CFB&Q$JGI_UA7%2Q|=rR zLGxzB`YkyO$eh{X6-a1{e(ncryv@MZw&GshlXbAD(Y}~2?U0b>N51ZR zFgwe}E@Xmq-P#p)YrlK+76N~>g_xD@A#cYlc@W811Q)Ztu+l_xrHD`G#?iqOuDE|s zHH!J0s6L(0KS$;FxrX9W3+SMu3@ZCZwU5W<&RIWy7n(IM-sK`zSRG^gr*j~)zfs)~ zojlbPJ`Tc>uI->uLSzI?8ogJ>`z&DWpMm3t)T^4<|oQpZSfg&j^Q7cU~L(ij;6ce+C%x~Mj zOgoz*UmtC_yJ6F+^r3gPj($6s+(G-K&?Wt7&WCxuj&~}g3e7o>k-N8K?RC3W{X_d% zlSn3MZ~pMyPolZ;;BKSQ-1ha1dI#>@2lRqR+$DHi6`HqcwvLqvkSV5i?3VVKD3 z*$)%7q*x-_2~xD`CgJLG9`U7@L)k-DAm;~;^Lh_2Vp-_K1Ejsto>#!JZ-1LR1V2*i zEDR_$TIUg&D%Rpkf!FP3M@RoM62Oi7V^xR!$`fO;Uz8p% zjV57IrYa(?*cRZdJR(Qmsj}(ss>%v4kzbETy=i}jI#OIs`Y0dMA?;-g1S2FH@MXH| zxKivQ*?-M#CEBuiqu+!(dja9!KnOJb#cp@u7FfBE9apb_B>ET5%-GrHxL*7am&+0x zI);!>Pk<6Oz|N#_(Gg&;V4U_M@=#4#Aa_jc+@V=Vf zp%~dXjhH>5Uq>plir@%;N0=$uYg(|*M4lqgKa~8sS;N*QE40nm2%f*L1l(Zno%D;sHJ7qsPt9fJ%WCJDSOC+|S^)LpYA49Jw@C&Isz;AT7x{qp3oKj` zV6Us=_^NPxfV#J1m~+taGkCQjUUTGrr@J*gU&Z7?2=)C4{3TUV>a%-H2vV~nv2M9% zGSqylU$?cbWLynV5&i^m_&M`bC^Vd8Em%Lz5)<7|2YB^Kh=q>PtwNnfJe2+9|D%s``+4Ujt-RYH z6=d*7f$gm-!~C(2INNcJ=MVf(VX{1G7KgUsPy86loOV8Dypg*7(tW9v=^x%q;)6J5r|)P1N%p_VJ$8C>$)pbgOHW!S*M zKKKT!LHxbbtJHC&*1Xl?KmpOM&Qm|`Y+2Sl6kf@Qt$3=u7A-HA0N?a)O^{Q5V@Mb{ zz$)B?8#3ghY#lB%%qmZ93nus*R791}XvBDgGuz&4>UE_sL^w!Zr(BNb5iib;TR=-w zsC4@L!tSxgWQ#HwkBVjUON?8h;2iQ^ea{%7Fzg^PO^LcqBO%t`CI~AJ>+*vZxLW8p zn-{;FjZ{Iua@hV*4ZdencJI%Wz zTkP+qn|@v#D~$2AES}x7ZkA{+pITm>`fxmw8eb4H4ffNHN44kzIyjcwGAin{h;Ivz!i0c%U_RX{JhhyGkYt!LJwl1F zegr-+bh5u7n8YQkB-u1nxi3ActB4z>X%x0GC;DUuQ63*8cPt8CCe{n;m%a~ByeS1&`hrQ8b&Y6 zL(CYp67HidrrFl}I`kA=VW(J+jw`K>UMc0SECKac8AcqD!@Ai0_e?(}a%fqFhMp+N zCJbVRJc7et?;u1+S;VC=QMc3(8WnjQy?cfABdgmlQT0n+Ne@1=zjE@kW#_@!W<&)x zg=@h%TFufU?{l{pJ?F%J*QblB*#p)JGbTec^Z1R=1S^7 zmR}NIHkXWxxn*}z!26i;G%pS95WPURGFD|W3bW_1Hi6e`G!Z|M+O$b)HrzUoOR>NM z5S*y6=mBdtQXRHBliXB+&B;|GsG4zg9z2Te&{=Sgtlz9_Qv^(be_)hJCYXs%3%;R_h& z)y?OCyEo1aQ?BNFrqrsMn$&0!n5WSn2P=mVor%Km#+hwRBJJ8mSNyK^W-RO^D1C}} z35N*ZV%aki-Zja*n8MHKEHUA?%KYmU4;L4Ywwmhf-+I?Bzg)`&f!Q3WRa4|jA@aK{ zsO(%?mO`hOq58AE464uzk6jNCau^;n&-_5528b?>l8u;QLS*!QO`oWxB;-(opJmc& z-EHyOV|(NVy%Z8)$_59$k*dxQ`n4Q+jT#w7TZNLb_JDU@crj0l*DC1Bd+rjCOwZ+W z`6(K6-Oxu%;$DRpaNKVQOAfI{mxvg0 z^Dz^hr@$did)|I~{XGfS{nSwKrH$00mP^43B)Ni?Ho&f4(hTckGTui!xf6OIJ*7V= z_WE^g>9^{e_-e48RF+k0no}s53$@43F8t=g?qa>AKZG&RHkwjt&+P&HIJoKsh!td6&aI;S`Z7z7RP(j9tx(-r}(IX zU@>HR;*fseY+C5Hn$oU@_NV#q&2n98mg#Hhn$yVg#B#FqUNfhnb+1Cy$r_5 znsrJ48j0RNf%nJ$uHY@FL+%^uWw0nta*fI(*D3|kf&QH9asyAyheR{=AC346!_$32 z$K18135-{DDr1q?q5gd%JhS2JG=~z;{am1%i`28KH>T+EhoKtRMa2D@ryR6sV17Oj$Tt4?_!P5+>V0_^Nr`9 zrLj)MDrDEUB-KEzj!7cgc=jfLZKeR&Fqf!7ryRrjp#wNXZ9FW&OaLnp>MbJE5$zHy zhqWMXI-pLJ@e_me1j2l@L@3ZTaN64C%EQ~nfdj|3h=;PI=Jg+P^;Cow$L`e3O97lU z7DP1JK|z%D`DS_ct0u0d%#-COlK*?%b{H7r)2<;~J{?pgb}_)=`S)V#!>dAfrsZPO zG|qtX2simGa$ZB3WO1=y{cjcExDE#Kho~j=^N~@T19B67wug;Ry!jCRT4F? zK(MKBTp3%c-q0~%9)>Qsn^p41;(irTvy{9Gm`Lnr#||Mjb}7aDV$7({JO=`8e_E~I zD2!vjOs_xkg=i+WKa};Uj>Fl>m4(oL9PzW?G>PF`eo|Yo;F;XR4c5?Pwj*tT62*F+ zF1IP=#h?O(e{NZ@?U2&7i$UY@w+|_W=99vnThQ!HXbsLSQ2uf>4tCrc6gtIA==cNS zeqJhCr8B03^^6W*^jW(U2UqKsAC!5c>VZ{d4dE_GGBuyyS1FY~g@n)ex4`lI zH~f3qAqCoxW6ABb(jVd3U4I&OAerE#W0?v>*O#Cqh;D~A0$r%zLXO(*bq8F+d%Et; zya$9pgSF?qHi4#`e+{g}K8S?yn{9RqIX~Xb@!6}ZOyanEpj1_kyym9wQ*UGp;i4f- zT~G6!ACKZS$lS@Zegjk zGS+0>_+1v`-Q_-sM}2slraYtVL8^?bLLM)a+797PH{{31=hJm?hr{W~EcCF;gNZM- zN&69%9;83#4)_vVjID~DTcr51KUpn~=7SNcpYJE%z|JwYi{|=D>puToiPg{BN|e6X z88fdYOevSPyg8q2Ac~4klo(mz-1Dx=zZhAsp67SU`eG05sV-rinK?)-fhQOKxE7iw zUoZ&@L_#T_$KCr|K@wC*PNsK$j%pdkhapjIhbJ0Xv(yI742tKqZF~C&lai$;0f&vqM+OEG(_|Ew3|_cM^jUe$ZW(*qztND*?*Vm zla1AEaFQD0r-zkdp2)pC468o(D^W{ev5&{?#}MtDQ%7;amFg%8UYNngt;w4`Zzbp2 zckag%C{Ltl`Lj3?wS6?8z$_i5T29;Pq*yH9$v2h_i@57L<1)|^60EW)CcCoI?erCD z)>N6|I7UUQPII+HY;MKhT)cf2EhXcgcI)8i-(;cp<&&u*KIP8Hon*j3s5u?9g z$9Vz4U&6K&pu!x8$((m|`t}>>QH0c!_TYCOOXi4OXa&Q1U>3^VCtOMFrOCr}er+$> zO-u*Pru6lN2wLi?-ny_?ls%afA1N&^WWWi2kVnesh)O0F>GD%t1OI4R&aKPFhTa*o zEZB*e-w?I4<3XBg9bLorbB|eHQH0Z*$%pWTT;$iXIi|HLftiXIU_v^uPS6FclfHOy zP`*XoX&Rpcm$jKD7QetcOlI<9=qu#d>QoO zI+v-JkwAP9m<~tBsbM41U-Str?QuG_7QaV-$>`T7nH8JBB`0P6s;i-NZ{A2_va!al zs#DKMHD=k@)vKTQEX*y<2l*vAES-wL@cic=dQw+P9WA3o(4n<&`)YhL`Zit_@soo= z+8P$h%u?=)1k}qpFBQ1(0Ur|;51(OIIMGrf^`_B12oOX_;Z;xSfv{45b}_HYSN&eT zs--in&c&Ql)E=QDM4+2wxYlK8K6M$M35T#A^hxwmO-(JZ($DYgS4$4VA`#hJh5rp= zT#>{2IVN5VJ;EwqBYDe=M;ZaUGKEq170GyBfm&x#*6VH+D=2fK&bSGhoqK-6-Yr8l z*t9yfBQ_;qrBAz5isYuo6&*ikCQ_UR5!Mg0{?7%*a)Y)a=-EfBZnz}z@)#+ zVrd~_Pvp^+0fw0C4{}!d+4xzhc&Gu*c+W?|u?P_)K)5ZA6s*>(V-vm=oy6#-ne{uS z*EM}w!CSXd;S-v}PMBvMtHuuKohZg{omXfbLr1N;Bq;{j!|@zSpUcg}M@@k?r=){9 zBigR4HPtH>XeK?mC9KLhMIcV!ShmPk1Zror0ITVJ(jWMhm=$JK;u&84ctReCDxJvk z1|SW=k~j6q12tMzf}>s<1W~^o(2-c?u;LBv^uDor&5rdPM0g@(d$61=E7JB+n#ezb zm-W4YR3fTy*FzH7WEnTcYD0)Ug%LZ(XYg)I#ZPOPS9ClqH_ZwNg<3T?> z6(mUR`PD5t7QG!#^5uzsBd*`o+1Hhrw9S~rH<`eGCkO4Or(61EPv2w|o^$(<4T=E0 zJ2@Xn@yuMaV%v?8eHhpRmmqPKIWS*f6nwXaXN?sGw1PY9ffGt%6_Cd^DNfdP6^HTl z+brdK#lx>hNFbzXzGA3?+i#qMjsbsws3Q6#*>ruy3Xs6AjUw)M4 z3LX}-qQV#F2@~N$s8Fs_%?>k@KMyKYSKK8v|dj}ox0MV`kZ^Wg{9~~q99V4KT0O{nzi$J z+Y>*;X}Si2IlEr{Qh5t0Dj+wTHt67F@Z5TteWtf3UInP5Fby zs}4Wr>0+R8nD0+^pR`4OywwM9NNod0B()~&*}|}gH}B1r9G;kA665Z1Fz>>kywuC4 z6%6aV=b60d6_<{tvvNuA%$x|OoaPLXDxu=a#`$ghl&dYuVO5V{9nx#=@b{d~J`b>P zfX$lpGVmm1|AT#c(K7se<@h(&S+W@%EuXT%ku*_Uf!4k)=Qq0xYijDY{qW5ddc1S@ zNb%k_OKMIA-Eermoz$NqIY8ArD`EAuBBWi=eLxU+InOGJ51Ylk&*WQXixvXR>qBI= zg81++f0py#EkE_?Ssh}9{$zHggLa`zhip4i$*3cF%m4kIP7UrdZ#hx+j_|hS9th#~ zA&p1}tZbG;f)TV%TzJYvry+7C@mmgV{$WNvyk>ld>BVjWOrBMklqc`oQzGCsYBNx8$ zP$|gW5~S?U3K_I5ua`}Je-UMPT*Pl~W#fpeKNPiK2bqx>Sfb;%H;DS)a0CYV3U2e$ z;wwT2sQEqZb!BJfwHAqYKKC#zp?W@|=PVYo-+U2=mV}E~v=bcaz z(t1CCC;?e_KhU4YNozX#Rn^Km76Sph>A2lbLEX!`SJjDpA3LpaVSo#Tx>nv+RK4;9 zEavM(Q`Qrz<;*2Q+}C;;Clj%ryIp@!0zOVN zKf6*(a}V)FW1I(P)nPWs4RR%ibo$W!kT($aD~27%S(nttyN;j4zCr4W3(*;SGtk73#HVPuE%oT|3Sxl)UfIW-_e=|@uCjg8BjZ2|%u`EA zx;f3}6ZlnFPVIIqs`x1iDy`@VLEVargu1NeP%du<&u+gBV6H@yiE<}ab#5#*3<0OW z&M~EtwLqow4V{#~;m?^-SCP?P&57>h8U%bd5)Y zHFaak7d^r$Ykh!0_m4N^SP}61@H_ps3%()md3k%;N9v5#3VYe+8n(Mwj!>|8F{xgX zN1`0&A(4T>*L?82wT@M`3_k500v3&H4y~wI$VU~8_cb#6g_E`6QhK8Z(#aqamCGq%39XDo*u1!UQ`zM{8 z`Po^a_~07g8!uGL3h9c_K+@A_O|5I+iprRB(M&mJ5Fd}Hr6cjDk%*2|a);v4KwB`z zV`?_JLUq9QUi+v^Um=?idRl6IyfZ0w6VdmrB&vfvN>#rh63U*PHPNz-Vj2_2nl9S& zdF+xAuwbamW@(Si@ToVCxS+mF6ICxkniQ7>5e-u`9I#a$Lb)1wQ*+0*j`xDrlgJYW zeF0(3!$-LXZXCvoa4n`dpWiqg^vO987Of6x!fMf7DUAo_Zx_Z+o_;;T@d_9Cxhl9M zw^cv?c!JoA>TH9eZ=@hELNP_~Mx6^pjIwE}ZwJl}%tQB%z->v2W&!6vde9AUa82=(QA%hZI^PYYDKdn%H_! zYrSAdjHH&2Etg)%OcZYAFw<6Rz z`ue4pT;@WmRS9}_geQtHqXPD`SWBUvLaTR`!P21v!xFJxf4YC~?}Fn_X&& ziea!QC&(!^tvcPpubJovDRf8afo^PmMBrVKvoS-_ygO`#2vNu7fh_4Ru>*I}V63^( zU$E(rDlrhs;vsj3td^{2h+-;C7iW(huBT`$CeGIS^|az`Zojyg?n6q*+v`@TAy+=KS z^v+!4){Ji?ZbX#85U_;_rZ)G}#S1^W_H*wI({Sy>lktD^3z;n)H#H8^1R3m}pG78q za?Rkg-k3*K>kcBw9-PSDEv8JneTA<(U#?gL@(|i`_1JVbL`JTBm~uuGGFBwUvgNd7 z{i)U+k-OVud|l2>ef*H?P)w4E^QN7MTustLXP|t`bOPk2-0f^ta0RA26j?`UA=4uH zjBH#N&gakecY2}pA&WH6d{3sQMS{i`sZ^3yH5Y6M5k?J#eTKXIE5Q6cUv(g*ryW)k z6wmL)#PwT&N)=Ut(n@hY#!|xRzZv?{77CV?Di&C>wdItXyl4fbTnk*>_PAmU)v1)~mK3PBz$DY<0L$vjV~VSFR{RY%4V&U?nL<}S&1LXi-#7<@q;ed^Mln>+=>G=% z>WEP{G+JR~>sVv{e&ed7(uQpxI^QM9zUxXJ+PJ-=_fX`oeWBZ#lRVy+(c{70g)cbh zEjPs$*)$B}ShqnOTx0}5@|~Y4#ir?s5eMnB_njcw)0iFY{-j{<4-k~ytpbNzH)*)G zUXnIpa-$XxRwpxGg!@IUw89rEe~rFEP)ZZ=r>16Zu#E|rSJL8IZWc%#fTow+V-Isz zHR$+`t-LtM8^SH0gAQjxN(3YA#RehF$|ceL!YDCU^ntI6`bp5WW9?6WRU}0)d7h}I z+Zy{}HB#xl$Y;0GJv1bs^jV3+=+~lTYiEmpSY621+DDz)3wux7)Sdd0a%C=8{jt9X8zs|1hSF%q0R3^I3m}w#ks}b0qtuS8eqTbjmNz&|2C8DV0Is) z`}a-5mhWwjDCe#PwziEgs9yc?CRxvFG#h|E{+&Iq`h$6T9w zE;U~s>N(H@_kNat3p8gAyW4TYRPsP|pDF!$8tVvccJy$k-lv4{%nU{YbAtaJ-*VVr ziwjnDdWkJIU|%nf)rb1v5SKq9!th~N4wrhC#=Fe;P|U*=gXL$&6@KrE5Ov2?hOa1) zCZL%8qr{eP+!#Fkeq!i6iKHH2+B2?T4chov0@d*x+Ai_kvsD&;C( z3S6WG2z9Jm=cDIRHm#>=?GGT)9utAo(Z@{Utn6dDn%)6bm%2tSQw3jEHW}Z_XVN)k zu}Qc*(Sbg>fF8IDK~*$+QX{wu|dE!c6yl0;BEg zWsE3ktZ$X0#?G0%N|m^TX_!8ZdLVSrxjH`v?<&!ta_4KMY#1oWxgzhxpopL25H(2o z5#8rtTU>Rf3bYY*j((i5e%D5jM?x%rjMjD)|ze=Coc) zh)NWII;9yXXG8Id2o{!p&bwDB+1%AJ$v1zkXuvi~@JlxD(P2(Y8lfsQR@5gk4Ib3G zw6xM^B5uF$8p*?UiH{3lIX`)xO8qqVqn+KzJXnsAeS>Xw$BmS2rG{}EDsbdEY>Tt? zm0LoYtXPZ>$z71MGG@LPadV9N(E{hAWe;0DuTee~O}?O3GV>GY+9UV3n#uSa6zyEC zuRUwj|@-&QcDf4n?5JffH$h*eOqc%@C$$k*Bgg_K#+``;j|vzJ)C%#3I~^rJJD8pB({bQ zmOMHuO~u`;{tRx|o*+Qx!s~WZLi5!#U}-S(Sg4n_iZ5$NIOhLU?9wtPG}!c8+6m$3 zYP?r$hI1zJo$`aP@+N}i;V)r=&=FW`x_kZeXQgDsInyU{Svyv=X#_B}(43GZsy1~v zsXXAumL7`(_XK}U$vCn`>(k$we23n_HQlJ`2vIgxny2UDUx|aIbjQ+SaTv&Md8&p@ zk%nOxsvMNFHlLyx_w1_`29Q0;ofL~-pNLs|5sDZUTnS>3+{OB}w*^`|R15L_-YyV6 zKd_4cmQ>CZN$`V>daq}T%o2}WV>&ce$AmfJZ^xyU$D3DIpVF&eS+r22qDj4tLIW(z zA~V=)U|$9O5%jbHLiXmn`7i?*CRG*#py+cgOG##{;5I`ju@5$%#8ZRhQfmD7&bFL3 zGUQNE*R?EdgmF?4Kp->t$i;Dp0>h(xP1oHGs6ScKK(XzcO8Fi!*1rUuk%jwV7{XAS zwl@u}7)l1?{FaOE1Wn9Q4gN(DaxsmDF4pMKo#X*&E$n4@2D=SwX6Jx}4{eR+dhpTJ zM0Sd0HgtbDyThaN+^n23CssMFC7aA1w*(p~YydZbCgw`Dqe)6Zb~Gcwzyj zi^(E3zuwh)vCiRP9?^cFj!w2{nC=o zunB~optM&%31$Dn{9_*LbhpbrtVO(vX3iCrRbY2@#tZrl%@X#Ucs^h1o-gHTqoGGPfN!AZR|y}PiMFn?E=u@WO1MFxeX|8#R)>L@*&Pc((G^Ih z0&_f{U&*+h{Sqo?XtA$a?!nuCSkY_n1Z z%QnfV^VDC`Fr`+uh{3=E$Ixq&1nSVZmF_RRY?Dvltz5b_Ntp{>@FJPo25)U^QK+98 zmiV5Z?h6Ka&A5l;g1fL!b-A$UxzD0O2A2v|Ifk3-WC5%YwvH;7!>CcfbVwI?QZFOk!2}T{Z*~ zt8)K#cGw*G$Cj9OB54F{VVwoK4+Pz z?z4hPtoR{GK1pY5rMpq+X1#a)3E#_`Jm@FY{b-=@%M9ZV`Y$nco??J=XA7Bd0t&Ng#lWvy13CTydU#)c3|xuk5XZ~Q>JI8dI0ly!i#Wsm z-}MV&!rL}~E}s6I1U^bbOs9hI4@|JuZP4F|HchoRNA`zfQ;N^f4?RcjjWGtUDV$J70l0pwh~ zN%dEp%x23o2TC0bz6EyG191{VZny-}P+wBcQgF#IMhz zyT{ixd`Dl#&Y?R;FYFYMo(U@RLw;?hWmCM3fRPD=$@Gv=A!Yg*sH!R`8n9-hIm?*) zlIP%@+$u!4^n!fMT#d)ySNuL>hQ{UGNak#j(I8l(O|vLO;IU-ScZmWOo>4(0{2t7T_ypZjb<;=rma1LZSq;QeOFe$zoyz1>FHvniRh6XM2K zu5l=?mw5%e#4)r0*C{PqiO%eZ&T7SQJs72m`j;@nx(ZBhU@e7qLgm#{kK$y~x=(Rh zD;|la6>sZZX8_>^t0^nzb#9Gqd6U%!;VTz81*1Jh`0Cf@`Y(6*kQ6WIxuCvE1yk}y zkXIQE>;C>2Bu_0#Z4n>4sfDe|Ir|tmhuE z3TCyFkW)2P<1(|~lUW5%`dR#3j%G&dnD=2U?B+GbVYgMToj|-sh?xY<71I&rAnF1$ zgS1TGOFuTC-ef*6ItzBi`ebWTRZ&b62@yxkmgl*~mHdqaaiU(0HUKey^)=dEKmQ1I ztfE!pMD{)(A+0#Jueb@zzw6?DY}s`3WHO#-TJ`V5Shus+nCGaUR-KNt5#!y!h0C)d zdRPLTZ3LYkq9~5Q#Vp6+WnN?J!%(F1p>o`G(H0pl>m)!m* zcO3~&;#-x}`#Xn3V*J??$A)yW;XpYR!n3cJV<&f^SqJ2qoE&|ZNt^nRBmESED%%Hmj1XEuok4jL^*X$JO@i6SNk_}(U*>43EE zFoo&R@14y#9R^Qg`#KIZ_IjG4d3Q~`E>grMCsFXuxA7-sQbOQ~z)x6nJM2U2)Zl0@ zWkzEzvn4>4-mSQcgPobozuZ1L5}Skc$k?zH}zuwD>|kV_!aLD;C@1D z(3ilNtYOSyD_p}hSS+%uQN?AXA_+|EMfir+gABDUU1w=%_C>`(o^~O#BfT$NAMlpG zJ*O$|HV~sCZ&B(W?AU(SN5%Sx$a*s{FMByp!v&r(*=FB&@RR(I-n>CD7{7XHj;FD@vo0yu!hM>|pOWT65v z9T=mBiEz>wsm|+Q5F6~CP7ZRm&NgJ4-`t<3Rk$@qjugDLE*8ao3!i+GAR86f zY&;}AIVHa_7JNWvYqMJ~r^UsRVR!@oKH8GtR%4OT1BD?2K{wTUIuo}=TNHaDuKm|J zqL0mj$TfrX1LLTb;ml$R;#qCnKccgG9L}NIcFueIp&2Tfy%s>s$#Ijls*;kG%BL$n zjtE+6;&DBth{C$GX~kv*&- z7&Eb+5_-y`uT-Xl3tID)2PMo?mx#RD`{1Rwj_gXF)4Yztc;Qj1`%0|bYodTfJlTH! zoRk1OK*PVZy8H3RbjY^6Idp@h8~v%*o>NH0LI#=;7Fo(Gsz=y3`tgV$b#x?62N4p5 zec(>HoXTnDt6#!Kd4f^;I3_*jHnwgu;&rGHD@>)G`?bH|-8*fgX9v1b#ZWF%MG*Px z*$l?LD*LM->OMAndfeHzSE9eXYP#dfFu=ErQIef#(GllcC_DW&R+xxQS3O59 z6oDEL=92rN`UJaHb{)N_!RNr7jwH4aJ!9|E{-;v2zuOBEXhW#%!kF3hlA>lNIS@Q; z^VgxCQWe{xOn*>N4_Q`>oy9~dVlxAiHv|1sdWyR?b>e~=tmGP*zs5Z6vJqY0PUq$0 z$h^V7;UHP=u35+m5#uacH?@Af#SheHuisKQs>5fs7Q07Zo#3|fL#avMj;n}c3DzFV zrbm3ynhc8`WF!0a=6mcqJ7;G)yn8;(JAz)L1ueWS8^L-+V_(VDQ`K4cll`B_#I_N! zcoXNLMNEr_&CRIJg($T@ty>B;W zhgPGoCfHl&m0PLotdun}H*@+htxmGEx&0%l!HDMa ztUa-IQRMeH0~J#m@lJq#NWi3w)R4Ubn~cGYXt`V8^gTaS*6kbF!?WTRk?$K8PcoH2 zUV#O4AUWkmC-;h$jEQ`$Ak?oCb^CIwOQc?Ow69tS(USzi8#hP9uF1%PS=PIWw16xo zc_n`6f^U{uvu)i8BCuEt)rIO8X?aKiSw*B*@aH8+QXE@Rhueu9)j9zHfw~=CFKfYh z^A}nbEZ#%*|An&`FGW0^vU#$?>Ns|M)tU<0UD3R<#NHeMH;D4rCiE&#L_F{%BQv@$ z#3|q-6Mm?+<`5AXRy%4;V&}AM2<5ts>4%>6(mNmOOY$3B(uBzmKv5NN>1}OcPi+o_?fcRp(=4_UiG5<3{Nz)` z>~Z#gW?ud3ag;N-NF$&8%IUF>%F6_s9(jtNjgi94Kj1?B@+cmU@M}RjKBGAGbl`_- z?7YuKV3910Kn$i2;`NZt=_|6!>AJ*;u<4?1uYV?`CeRx~-HJ*%_nPDxSjX56dCsM47w>2L37zvqW)&%rKp0y-+rfFfyFw zLRhZ&$$9K%iM?}v3>_WZRFYCx&?w-CUN5!CHs1X|o)r?c!sh%N^mN-sx2vq~z2tTSZ$Cv9Vb`v$JgGWu*q+y-3CI{U0K zByW;X{mX3+wY=e$y2~-cJR6+)yEZgWsH8LT5Mi05*iCw$ZAR6-7v+#wB%|G(H~`DVlU$Wj_PIzBtC)#E#%}pCRJ* zE*;w2kUf>w;VR3H6)CoVPoB#WE2m(qIH^C|BHdyMA-^ud@pl-$(l$%7l+E*)Gr|w6 zdutx+iUz#%xJOG1={ zIyMo%VqiD&{v=iGJkw=M2+nF`q>E6svQbS4SKgQ>Lm=9gM}V3J5I1C)Qjho(x*5g1 zVfFB;-KthC$|;q@Q#_R%TO&kwnxdu?(5Ok2 z9gkKDwla%78+uIL(ozPhH5afH4VejJRDlAHIaO+aaYg|T??xjNW@rZ>}-4V?%gmFgKq`;Vmu_bb+ot{HrJR3Ye`uY zqZh6obV=Crlt7LF-QDC#{l<(R^L!!6xngvj8`vSlA9>Oqb_gnig2b?-hHN_tEq?PaVHd z?*JKV zaT>RVY3}hIB}1gf;jH~j>zkiDUVP)R!C@r&W^#>)#ic-zdm-+4bUQ=KBn~V4gU~0d7SHS~v%u z#6mirOcMqJ*yA9%IyrJ>M{f!dIU+KmuAk2{Fk|j;pTW3LP*jnq@E9s{8?M~5X1aWhddXyR_Q*s3-U zEY}Yt-ob9zE$$u#!MRN_*nsxN83TpKLu(@wD#pIXjZe@x4v{t+*ObVYenbluw9!+) z3~Y;jUY-y!a6vzY`VSiEL?RjQ7g_;&$oaQ4CnGqu22Phq7PR%9m3v9tPrDfkQ9Y{d zC2<8I2D6IZ(+1UeSE+dM8xfW|Io7Rar-GDY)=z`D6A!f_xM5mRX~z{lqr~%d@LPV1 z7*l&xjV^*vRRU{-XNIq9pS~(v_lz_yBe67lugV|hPJw?Zk%pp+Q7N$!CF69kbQR?p zA}xxza^|gqpZmk?Hiot|o*Ayb3%OXLaV|M8!>ct|UM+1#?Z%3O&*ttYx7&rBC`UG6h_|;Hgnp0A68uqBARPoF^fq}qBbIW zoioZ56lXA&#J|RHS|{J>gBwq@G4YxY*p@K(q{n7M)ev%NQ6tMZm9{&nh}4_W?0gEQ zsod0lO+kOXqjizg1nZz07~eVFA||JrhIN+I?)%>Dsom?{MSPR4dv`W>-jHI4BN#=m z9Qar1YPxjR-Jb`hvUS)jyMD_Aa;O*(Neo@tB{AIJ>^SM5pX`V+8SHK8n5|bV5ge{c zCAKNV4;wq1&(cq~p`0(CA0=CNAywi%srw{oD3x(_UN*O>1=)eN_8T+kxWj>Jk@>|- z3Tz}~tk@r?DXB_X9)afLi1DoY3$`$IG`$r=X#hUN@sDao4<_ga)#hnU`@apazv2n% zQ{{$cmYOxlaa#Y(2tP4wL9MltiQSr_4dyUeJJ5zIe4R(EfLePI5Lr5p~;{ z3{4LuS;vQS|L;rnRy@L4?+{d2Tivsb5+`gc?0)eXauz5}rzQn0<3+;1!mdHb9IjlH zuCqfytBDqcK>6(dJbukVf)aU0?=p=M`(^W43Cr!nVd9&mi$()Udvxwka}H%rMqF2p zAQ;F05_N*Unyzsu#_Lw^z9LA1%^#TeNEjbtly|Z;pkiN!@VLggb=0Ux$DA*+ln`bZ zcgfu08x#(fB8C2Qh->=%^fad8)yTGt_xu-ivnL-}^J+I^1md#r7e|i!h7tCmCBdbI zX*)&~EE00_%_+D0u${z(IUlw_7Vd*SYP$L(sHRD`r2c<{x$P47_5F$4 zyYklrx`wX6yx{Cgey2cklXwz@3FIc3r5`9?lpW2SM=ZmnthjI-?K1kcl$4Coo-HH7 z!Tq%(S#i8?>UCpo@}~D@>sK4%3{}J^-#T?DjvQ>yV1XOF5A4R7MKYM`UoQ$v&d~Ws zro^XZi^*A0ol!1l2!E+m7znd-vDEGH=eWwu!C)Nn=h<4GZ}MP_uFYg6PeKO2Re$oy zDcB{8*-b<%5>U16g!#%@(|wlwUwByD$hVj?|?63=SQlGcLjjzFq;g$SUvQHJlwy(`EVg2A4^cNf+{dK__|HMot`%-n@ z{WdUuDu#H9CX?zy!SHPVh0$2Y_?`>q<)-dpCwfdo0JhlAh~EBvY^8hmSQ147?2I2A zLz=l)4GSaA1cW6PO8+Ro!wQ`30 z3A%NT^j6y5qkZw+J+4;(-8FLA@6g~BuP2;;sI78wY7vgyU>wXPa#pqGiU6cJvq+~{ zR#I1i&T?YQeix!Gg_(bt=Y+J=rLAP*4=zwz6D)pO=sRV(iQI&yUVtE#;b2v+yOGRu zrYZP}zF(C%F`f#r*vX&IlAfSjbT(_E&ljk*`<0A0}mi_Qh$ z7#m(Fhq<-wqJ4qT2@G6Ihr?oB7K>a}D!V2kk9BP+uk9qtc@=kQDH1T>#Ce+4%60#Q zj5+O*Y!O7-CqLHQ!lOaoxO2a0%9R@l(bE)xQuc!To^KmZkDI5L^0{Xxq(TN|@|9i7 z&DhJwR{R(rpJrk9S#+#=CUYH543EHW5hbZN+=kYK$HKY`>GNwb?_(Pe zc_X4zC6eUONi^70hHR^ZaXN{T-v>8hoXNCGyUy2H0;i;Lb3r^{w{H=}=#eh9sCD|> zC*>yU-CpC8n#wlsN{7Nj{966WIMwQO*?m_(ikLdjS7U@(H?+JYm@$I!waD$>ri(9> zyNqf@1f2?%4`%6=tqv{^3h;y`T9f>f)Yee$J*zXz*>Og)<%}C4AtiFAiPT)y@yv_` z!B4-q4tGojO&}njI)2xO;Ea1KX;YvzJP(?o#%RI^sPW8`t*$gaz>V~O3`D9=ZUl~M zlOJ+6{MWAUOI38llW@ihZmN1tDl4a=Z0PWFw~}@GYgsYO34-{@?P{a=7$@tDw-|ApWIK$$VI*fyOovQVwB{eboVuWpTLJiY;#}ju_49H)K z2mt&>QYaXi3Al)wWGjg3H{Tl!XQCEQQu=)w92X&-)b>)3OncfiV`Q36Er}MjSe>K@ zj8UvxX7x$Y^YN6IVXZF@X$FPMZL_XcPm@_|vWnu-jbTCXu(_v1_|l@B;q$<|xJ6;{ zDcsKMWCkCG+{2)N?i2>sKQ$RQ2&=1WsrHcc;fJh8Eyqt)vl6_Mr6it{q&;u9amF)< zb(IM;CEv=qecEf)M8}H}tajb6mhrl_6I~BmtFQ+w=X)EM8c=E4ru;~CC5c>w)p5x` zh$A1;UNx!0;2a2X!y4|AoGNlQU*OA;7A9-$L}xbU1q<*?bG?^&?74RYhLHNuA^}$& zQkO}`D*U*?gPsNZC{R0cxnvXScX7TLoG{3XOix2dyv z_(_bQF*ZT1RLedYk0*ze>pA`MNTW1^ymIUd>e$h?J@xp!@hOePBlJ(v(h2zuYb%`U z5L??ys1V8+SSWV$i{c+&@3u6Jr%jB{4}s15mluW{ezxpzp^7=k`f)a2!ubBtT#(-`;|D$#?eJ`E|d zhWQn(7j6)Ne{sl@@zD<70;jg8$@8|QrN+5bJ%-LUrHrPBB{%Hi%-`a#kB!oC{+fOs zKZAPKL#1mKI?Dd$wt^pbL)ArO0GX92MBUkF3ohie$i3Z`w_7Ve!LYrwTq>}nV1yAZ zeA&EHlMiXvg~bTZ3Q@?b(1gI)d*>V!8<-He806y``DdIumCR2{v%NY@NJ4=8s;PNO z=gpWfAXF>;&Ye{tWTu855Z<^AJ{&FJqHX zdBWTv0h8bE@Wa?>e!zx*;h~Of>UXy1(7uT8CZ`&dUgE>XYghkcnIqD^30uK09uQK9 z=rT)VvAD}x>21*7m&;*f0D1V4m8Xmhaw7bQ;J;{t?k7!&*=N=(%IO5gv;#|vBO@hi zdk6|oQ-!8h2G&xC!-xzs-h8U;lZN3l-jiIYB9G^0KV@xby6FTvO;y?_ujB04uBUI` zxC&?OLYb2H^d~>D7~5s;*{-TK#w)hOGW78Uf9nJ$D-`w7CyAMkk{_a3>u3Rzju~=s zc99d51lv`W!cIv_Ov3+jDmJyxRh!XAL8e#VX_?DOnk1))eX~q1OEyHHp->h|Ta$g= zoQC1_BJ4+ZaH~Ro={sN_gQQ!BE3$>6O&p;1qkbK~pWo>NZC0#!>B6pQYy(#p34y69 z)jjmVnO<^1t8MU8+xvgqlI1a&j69KPd1KLKyzwCDJp~Xuj&6odzsAHbhn4u+VYqk> zP7SI&6=&7yg(WepS(T^_%g*PQZDP9w6H0qO^mtBEPENu{%!g$fx`F|6h{+nXCuFof zYVo&B4~uS8=2=+y4!?_zSFg`)t2*J~m)!iRI0gl&bETIs=wo|XKV;71CgTMQdwL+c z)#{V~Z<;K*iZ|30qq1({bhhY6kHuffiNW)rJpN+uf`w?)V0Sv<1uB}|$;2ks93j_# z&geO2*t<~iI87J(pl-{vIMwPXzajrP%oJXOpx>VqSRb_Hd9p(W>fh(CL3g>(2e?j6 zK<%5%t_nGEOaf$;9|Hz!X#y!;I;DL2x#0Nvh5F(N%gHB@IK4AktWhZ(7o3N3u&y(i zVj&|H#ncpqD3r=C*QB3a5Ca>zlV@{Jv6@(J z$VgJ2Dw&bNP5Tz=W)!b=EIM%Nbe38{E5Q=BeC=5oqQgnLrCLuvqc~Em*V9mmg#F{z zO_i4Dw|t1>_UkL;QQ}h2v&*4$q-gmI^a$xMOx7n_AaAtgF;;6=;g!>SKq|Hg&WYio zlJQR%C6zRF4S43C%xHi zu%K&aG6e`j>!bEFLs?$01_x=(*S8_iro~-8 zQBUTM#dDbn&7?PG^i7NssQMFF$uui8%R3%V)E}Mp0T3@ojHOulm~EXiJfv1fR_{bs zm+EP*wMNW1ED6ZYB=4Qi295kqrkJZ7e434RBZ^o!UypGcP?Q8Ks0;<1 z_(l}$28LkW^<#E|&zX7`H<%24D~^NYHpy?vN5jMM7$ZAlc}R!Egg-8E`YINx=~_0) zguKp&N;;CREb9{y-lgdh1=zdWL{-$ie><;%%Jao0HXnWV*Ez8NBq zNO?E+Fu-qr8`T#LJLd8%e*kNK=VNIDc!i`QrdYt;HBE%BZ{de5c zsA0a3>#%VSu%{56Fv1LBBcG7WlTkpCB=3SvyOxX^;d-SrA&9YdWNepq&^l%9i0xIv zy;38=!q2SPFs| z4~n~MD-Zbj`DSs%J&q*DQRA^W;#q=GSW>oqTj!|2J~>+i9yKg?7QmAv0LsBFNKHRI3#= znSJNoxCLX+W@?Y=_+z~whfX^JSbV>cw{mO-o$vMSS#sn`p*^>Q^Yy zHAN1xSuRErGh8=e@a#t25yBByeJY122ZDf7d~_J=YPi{#clQ)c4GaKR_&#)njv({7 zQfr;#i8o!aBsrDWs@`amjPWkKxcss9RLm{n<5NCJnaz3X1Fc^1(pCgJgc!UJP{U|3 z$c@@-U#0PHfnrpbv5O(<$<~b?vHlQaR0Cyb;f+sn^M?cwlu2(S zq2Q`M0iltnh=-r`ZpE+M^|N+AgNZ6!g?{)N>@wM?XGBFmWA&Ge_MX7>u<(6o5J4$> z1so(SN@TQI@^ex0XAMM3MF5@*rccGyjAFr3`Gn`W$08`0EU-?TBo;Z8$pv|Wi@ zf1WjoO5?+>7hBDsM78y4MLyAWr$g5YtrwEhuZue zMHUJ8Jq&k5q3AQTapZE?<;4~k-C1Wv=)uQ-zf+#l95<>-VlkO%(Ee2gkMd^g7@p}) z)%s;iPotBUKuf}N_aTE?*7>!b%6TCY+|N=fqOgQ!W=__+*+u7Wc`S_8+1@q}bLxri zdEoCHc~PX=G4e2AzqeNiB>jlA(lBN60=KT5;g_&nC=~`*8BhMra1BBzUJ*?uEa}-_X`l9FHS679Vm}L z8{ThS85!4peM9SlWr|m(@7*tj-|FY`yyT|%#@G@we^KzM-MZ4~JS5+gx=Mj9tks-f zP`BOj`gFj&?R*yu&73;d!oIq$69L$*cM~Pt((d4Ve9M{IlDE3D#uNlDVcN}hX%VRK zsXlDs9v8v^oiyr`OgN3q6-#cZ#DHUlB`QK2#{_phcKaT8_1xww3!yUAzL;? zdGqt9qo?>x>il!LaMee)8!L}3n&%T0>Oy_K{C<2WsAV=4Fm`aSeGv1;>QGdG&^?1I z)6N@>=&Zi zwkdJ8`U6_wEo`coX6rm0IztrJV9b}XKbJZc4khsS$1~m!Qq+5MA49s@#H!b;eH4ja zu9{&K8Vk1~_nuZ4%%S4df%0UO+A?+nAG5HRxpT^f2K~H{$-9}90^s~uih!4%qC$9V zi9p>9iG?25)BJDa1fkP_ijaOlNX<{N$5{%ZrH5=S>BdNHR_pquJS=zga7>3I$G0++Dl1LW zpX-%yX=wfNI0cI=_4`Cu1S*36afE?}UM=XAV!+ zYk9vOv?>#ctmo~l8d2@|@zfV>P6eevGmEiE_b_$mi>_J<5z9Hh43)@yv^QOk%X;Y% z$%7I8XEJNK!c%jb5NqbawsP{$rH8;|>Tk7cPOo8&F)LwWj=Kua!-=6x-e0qaC+)D& z>R0Xjqldl5wC%eV0PL8H2%UK*>2ff5rPLxSl6B}FuRgC}w? z1n+X>k+q%;azdTTT04UX6>An9%Y0>7!c1e5FJpe|O6&`Q>A9cdUtVB@^0MsXQ{rUt zBh2FdOGnc(*Uz-&yk=pR^}G1TjPZ@?wV1-FN^%^+0QhxA=8$-GC? zWnqFf(sd8wPj$;ANR>){07+f}AG-rd3CD3@}?UTiSA%0SYWkCTvyI+kh z@f^1Jtx=V%#uSI|6zQyGjd`MP$&U*zz}u(#`+fik=1@_vg;*x#4JNdn2-Xgz>}g{; zj@tJ=H0~v{VVd=lzw6km@}A(f$MZY9Q~b^Tus_nff^j$OyI8VTxpBW~`F`eyM{fLj zX@>Fg7P$lZ$O}C)3fWb5$Xsib7Yk=##|BYqVHaLK^!8Tq%GviD zAjuoZ?k4_n2h&)<8pY*UyD`|&__NR4Dk#GZQYW=)GnoZ;X1T&#zAv>h_S-Yp69O#? zcv@Tb4W(agCpRbXC^wW!j#u@X)uVx(M@I1J&`|OriHN_l4~u!;q4(s`ktt?w2g3F@ zgU@A~v4{{Q5@T$-#j;y=0u3aCim!o{36 zF8O}G3PXW+BDaLDBm9+dVON;MA>He!cTpPr@!mE2!{4-lj2-EiAM7%KeW$zM{dMfk zb!<$^aeSCC6346l{{WkrCZ=jbMa6|_iibi)UK2s|Fo+(CGbrdNj!+y`NUB{9fu+T7 zblE;nx;Wzj+Knjoq@Y|qY~O#6HPdls7M468dwE}VKIQvb?E$<&jZM68ycYc;(s4mw ziRQmtVCW*F(!^lLg;qLH0aT&8+ZZ1t9GMiz_4{QtG+1@*-{3Ew8|2SzkKgwziD~kU zLvx-u$hxb-^R`m>PiAUv)pY1ujjeF+iQS>*c@{9>fmBl{*E416TezxWMYj4&q%m1zEhpF;q+-QjVC z6cgQWBIF!1T<_OL!U$TxC^m2jtBogQW^fh4RJAI?%w$wIlyP|Ej=egn^~07f^mJAW zS`noK^CuE>r6W-*HTo_orh^ykU8%r;@>^XzV^a*!x{8Upb(Gvxub$X~&`ziY<}K$j zFyIU==ZYkD-%yrM>(f`=c|=V*wT2sZqR%Q^EI2_yLdn#Zcfy}UQ=n! zlq8KNUWHYJ`$A4kZ+E1ho5l<>no(aaPgn4gq!BRxL^(?2r{|)USfb90>@d13pwCFH z`84D+z>{*>DGLOugnDc!&@UJFlJ^C)eg(rog;M?BY+4Z}HV9-bv&zFS>;7m-bW*jR zv?i+?XLiRyiKh`-?B(_R#CTN*=?M?J;x`l|FwZ#O0MM{>%czacEsIUN6Iau{X)RN0 zB&qp~*PA!CO$@tHX*_oX6Ihb=J?gl zuQPbK&{Z1Jt!WU#Mh(ADO|Y9M)hsTyjlu7Y&M~XsH??YLtjxp?#uRN(#ve=qYR%V+ zGbPf!@5yr9&U>&V)I$Q+#UXUpA_@Kcq=`1- z`Y0laz3#K24(mO_Ewe-bCA%!_Ydb$W>#b+jxMiyz7^MZydt$!^-e5}{MK{Mp{scq* z)TwTGNVrGXOqRQjYL|)!?rq&NoPX4CC`y$G z90ICr&EfC}q=QOQXtG7C6AP*Tu(=ra^+B{-+GFU{O$16Kh=-NvSCLz3P=47eJDp8> zA=p-l*E(t!v9L3ZZNI)43xTMR1zKs`@&C%u77qdPQBJ)HzHvux5#nk`j~EKGU#D_7QQC&rL;BFR&ph505dXq9p<#{d>gf!fIp zaakHeZQ$glVksm~DKrQrxvdc&zf^PWeEg(;JeN+&QOj!a5DYf_g5qNfATJ#`({^Ep zvJIcui_hUimscjtVq^2hZ24A^bFhB)oSj$xynN-Iw%u7Wyi3r?Zjyq{J5aQ(-=NNh zW1B_FOAxXCpR?F^E_}&nz}s*53qWZs|1Z)c@Tppi^@b1heRtjni>p`O zw7X}$O2nI^)-%k{j|@>2U+Lu>#qGdn33ed!?wl>M;aB!HC9Uj1*tL!KFnPO?i%rBj z3y`VSJo^`DX7s=@|04z*VIFR^RX3Eocewiw7E2K}c{_K)n-A!`+-{jlsh$Zb5|V33 z``>;r^0cz*XcYeMhK#3qI{%JL@kS_Pe|^?>0sTs@KP-yi0({~*=f@IokgP8Mi!dq`$mnzpS zbgcZ7+wAQ$@`x>V0UwAF8mf4aH=oU$e2FidcvjcMyz`fV^C9UG57aV#om#okt=qQW z&(7BO*foe9#g*6PD+#pvs8ux-U+Sl9<*rW%_4d7R`f6Kv`to*WDZJci1>tDfEZ_3K zRR!3CT0u2!M;Jn@}Ck*C2P^dn%oDpqTh^XzIqr%`!zie*SRdAAh>J zjncmYF&;lAb~S`dIn}aPVwZ*yju2-sazPJlD0z(&8a77G%;Q+2A({fWuYl@+PYAxp z++hxzq*o*%r*iGF8<}Zcs_Ohoylou<72n=+bi5?liS?w5hdfu+$72`?oKVyx z@#Cj&6n=psO@g5Q1Nm+JR>>Q{pmk4tsX_HSf~>1kAAWYM3nyKDlDz7yh(!(WP)quq zDSH)4TgEbhVv;Mgz|?8LoX`mE0AZ4L)ID~C=Ly&mAZ&+4l( z80$yR)Q?xL=xtD6qt(UaM#(b#Y;^X~rZR(#z@&Nj@1l9e;z{S+A zQTh6CB05iB(kt~yJ#Y*wp09)ZS0h^!KbAiN8a~ANqogv=r~UwSTW<7uS1l6=nEOA{^G>lEYoCU z%F;`qrX&7piU}DP5#6!sd+@U;GV6Z_8?2HMSsaFVEZlZotBDN#uO);*otw1mQ2d}; z>7^w;ZIcno|7T1E(2bs3&Okk_1Hf=*m5~V1HL>zm0;Qt=Hyq!WEZpP|Ot>rQ)Wu4$na&q>j$jMQhvTNT3eizjZ#Z zZBxNCCOyE zl7y2l$+LkvyE$v{*0)@6{OR+lY{S! zL1KUlVQ&j%&wRnNIY(vPRj;mRr0lL^iHd8knl#V*8jg&Y6vbD_HSQBE9RG0FKRsj- zru+V&|2=iwlS-lxMlazU=`?S9Q0`2fXu`<7q!IGic5bfCO;e}+|xox8kV%l z#;md3qocHo!FO2ApC=51Q>kyM+^rTD3i_il%!&#}Vk<27D@kLw)!jMEa#qOF+mB;v z-7>2{#o9(ZYE4NIU6-w)IX_;MxUsMwyi zH3psJ>c08)N1yuFU@g3tUV)fKA_>_-eYO)di5tx=j4mMkIB~ObLcI*V@FBc|nR0s-`4Bc<#KehK+PpL$%q+F-yTb!d zQV(F|z59R61M{<`cM&H~ca2)I(+*i)R_w}RbeF;IYjnrwYr?q=3imf|7x$Q~jZOF? zg{_mT_s8?=|9Ma~wG>w$`zB4|&z6n{Qus7PBu5g(!G)3se-?S_8^eel`pjvx5XDK! zCH~jBma{GI5~j(i5|(Z?_GNDE;OKva=8v>t`)xUy?Pm)VglToXm#0;#D_``Vof6Ly z%vqb!j)!#X?v@FHrh~3*FGOo%a?W1QgZW9L|98s^6fHl`Ip9HBiWei&V!U=KsN_`X z2oEP5bi9dtaevs%P1`lJ!R>^cGhqilFan`(j>)|S971Y{?SxQV8?#PUvCDz6s2*pM z+k2UY_pS%LFoV>`4Ph4Kmm!!m@p_2zB+jt+R-H$umq+jQ5?$?%iNfWPsc-VEIQW(0 zDp#s+C?-J&PS-6x&1DJMo{YN>+|OL}{r*|YeJ*2zZ9V*P?7&h>d;WeG3r{XrsKS+* zoWibAcflv|Vh=lre&EO3C(&ZuLEjolWiT$)Hg7P^ADx#XNZL)Xpro=#BfP%!aM}46 z5dbb>{HpYKiZ&|18?vX|#g>G#$3>$h!+VX~)dUgJ&}C6~qc8J~vl#?vOVYb1eo>S@ z{6k&b`x?U87}nSlt6n2#*PLy^*b3U?A@-ukmwpC6AhOv-vt3Uz*K6m%6V4ELnbQwQ zW^BS2=iFqP8zJcsqcegY$IRDT&sr9!H4Vkeo3mgFS`=n@y~h@blRWp!nKKLk#oK<= zsy_xfd}4L*hO;wF&hfWCOJF_9(&(6FbT?gmK8BFEB(yzn7Z-4A?aC@fBU+Lox3 zmA(9b+Tlj-X`rfGhp@|;$=Y;(KVrwxM(*`{8Bxfp`ZASDEqi**7X?~5lL~nQZ4A`z zs<603i2Y=3RXf3WND1C4Y*x>C$iYxOWnN_{M)7rldfuy4|Ji9r%`Fr~fI*t4JVg-y z)Dp-idMoRzv5I4bn)Oq+eckog_vArX=JRv#%o`iuhJQVhXq{ydD~yMMN8eKT5@j%~ z-2c0{4vBb3se7qC#$Ox`W{0cWuMB?jE~>KEQf;_7GOuPeT00QQZ$6|D0#_Iw>u3X0* znk`DCpy6ZL>f^2%Hz~*RZr!hw<#YKO=;H%;J4oz!eqxvEt$ zILhuVg_dV7`Gh@}m)cQ`+FN_*0Ge-{UXe{oPTjS!73~%Ol&)C zV4p?Z#^1a~BFSpzC?1WQ$}IG)QnS%}@A08l5r0IkbO=;Q&#zk0D3eHK$v|4w%i%&3 zS6xn&zjbr@U|86_+y}djOFv#R5ksTk*A9|CG9U_RTBD|P=ipubp)wnWGCN@6Y!d0E zO>Q6)SkpSdbmtr2A6rN}7}_Zj{*duJMq(zt%2G^QEmIPUk*Xjk-4N@6HHrZh)A7`FF4p$mew*@60rvOLgnLV?w zW{?jMV0U<#|J{^-I)%$bJ2a#oRbFFyLj1ErKfkEs!V%q)IrNoyWG>+64ZbXp_WG-W zMv{b);-?dcSxdRcnEZBgXqGsRK$lj8Z&F?iW6A&CUWTraq@J*coE>4^lDu%shBXuN zbruh!c@*#{8^2tHjYO%D;DrdaM0Q`jv`G#6HQDD{b3&@#lnzr#=VxaAa*!}8_*=(eu`MMCp&Va zPvw;C0iVOUz-jX}i9^b+@mQ?tvl6Gh%h(%n02b0YEM3t4)VW_(-+kw}^6hCT*w};v z9sx5{L0d4_+P*`*OslX_=f};ns((H?p-Zf#Z&bIfv4uyA7=+3}+)IT5A3a9HG%v6d z`Ut-;AEFVB*Cw3F)FSE#(Y_g+3S}^5j(>^2nazIbj`vx*g7~8+#9Kd%d?D0RDILHRt8=$QD0WSc!tZYqvVNT;5_rT_X((6E}$7Y*dJZrakFv-q*~* zjB?aS=fJ*fiz2e|$GA zlEuK?15fRh@(VH9=2`kuqFJNG`ypW-wb)ACIC1;{CAC37C!!09NJQ&IoZ_w}b5xlO zggn4W#VsOk=AFtXmF7AttX4i&cRHY|4rRicUt{fx1VKLjA#k?$TghLm0~)zWNdDj2 zGL1@sntGFr-^AX!*5_JUuXeWA*&O!Ixza4@D#4Li9xGfT6s4V^x_A8hzei~$w}E`n z8CFb!+9fI^UF%u=WY=6jz5=71`Mn>3-&KMTO z40}3-5$&+4Z(qg`yy zFy-P8!(D0dkm1B6!#hPtwr=H+-OZsvV+Xz%x>c7sJO`^o&r6BvaFv7O`Ukssdg)MB zgt;Nw$A0oe;Ii@3H!yxHZYD*;2}LEo^D3L9dGKYfIna>`HA@(sJ?mSfiB=Y|^FcOn z(cKx2&kGqU3fNDxiI0ArAYd@YK78Rg^td#BoKa$8%3a^jtS*02i?7B4gO^pUAf>q= z;iB#6K^2E(2)z#gJ`nUgr{PvGp#v|JB@08Pmtg1^eu#ZXnl20Q?~T_{)$`V)uB$Mz z59D^Y)lPlQ=~Z;^2mgV)6B-gv_{&)$jk~7$^Xb~cLWj+6t&qq6<0D9dPhUwbZ};wt z@H_#Qz3+Vf&?(0|tbB-|-jw569cjHHzR59WU|sYxrnIXi|Do_&81UFV$AK}K0OH1N zgS;zc+O5eA%)5DV*=j7M^3&B(Suy)sMF&EfF+oK6ix~aP&?(ri^;0gt4z~672pc1& zuzdx)bMsrO_LqkaA)ZDs(q0-XV4}W=?%HCmmZU{w7p{Wbl*3ck@=+aO=l{Ty&2J>=c_BN_PDs|2IOSG{@v zF5}XxRw+i>zgRO6l&F}p2VHW`DT3YLF%$mW^(wtuEBMA%eS96zNoXq(qlO>A#et_^!aWS?uCgom7@dsn}9 zY-sL+(%2>e4e}1rYSMk=1Ev@{|3)63(vXqjLahhx15Lub2@fH@2Bq8}`7D~%P@aAZ z{v}a_!MUp7v2jSTp=>tqcY~>wCy|{F$E6C~hGUm_#`%Iu`+vt_kxe!Hga#+5l%Ywo z@MopNu3MC*UTAeO-JJC8H%BrmcIHy>{tr_Na)ip9eEUmbr;`jUq5z7{*ih-4^K>7F zpYs#o=ya0PAZKf)wOQ@6p(m!Kh(n`1!X?T$Fv&6dy!@i6qfaJJ(N!$s{MuZG!iaBOI<~BT%l&iOt^ZzEtK*tVn}4mu_-~GUP{0KM?kp08!D<+s}~vBLNP}byt1x;k2FfL#5mWDZP{kh zDu;i+JHv&|(cIA9g*CrMmfX}!410>pznI;VXcSL7gojSN^X}HHnId!%>;{#HT=~SU z%ujN8w*nLkyfq;Or4)g=$0+f#(Cp;BBJzH80yHN1I{Mi|^c3uq!My2Q3ERj!q908e zsKzo;zk*D_3kJIagKxtJj>kXTqnq86jR3+Ek;95}5#<=d1EnL}t=M3ENNOd{R?|VT zb&qaGr9>k1{nB2GtpMZpt~T#uSI8XvCW_gY`!l->LAYY(t7S z3~h=vM#fO$!vj5`S|{(ipJ8DoT6^G^_v6DoGa|{?gth01FMh>B?{Via=RiCol|PzV z&YK}dD$-smUVt6=an5B}{2wM(?>Ju5LjRLs`}gU@v31k7#MX5EepO=XPRvkx7?n(i zf`iiKc%{Q8eR@iTHGySyLzaN(P;~l1CXzCW&<+)bbL;p(ZWx|}{1PXAz*m~pM8ljZ zrRCG=G%f(2x@gU9v5a#F5N4B)?O~f(VUvWw96GZB9Br3t6848(t|iM59Ugv=B(?jI zFJux?i2ojQlv=#-SD8QJ&QaXaz2A*_wpNGYzWVr~Q;&=$q@&h}+6!s#suoOyxj9kk zLK$yxjOu2Jxxk_3kW^SGV>gvA|ocm+{!bIDUl=tY~V?HY+ zU&;0cTkUvHT4T4zsx#a<19atd%t+{+xXC2-s;s|mDxqm^ZGh(gp)}zR%ShWQom>yE zVo)>A;|83vD^3_iqypTx!GSP0G)6Y>a_=kWb-#X_>eCMthjasmdfpSotLlPuL)elu z7N4C|9|RWg{+cLu>$_@vlws^9WxW>wGmw3Y7LGAOSBFWxP=3ryd`7Yr>RPQ{d<4|1 zL)WDxxKPZ}&}KO7sF$pgfN;oDxYmQCS<;|GS0dY11_(}6z|yH)*JUULjS)HC5-yg8 z+U<}1Vy_UH_CMaOIc9MgI!;F+B65;%9x7UEmI*7ur57{=Iv3HC34vM0R0EtGd z^A=aVyj_2}-ktHg0!pnda(aj%mX_LZ!NBo(EBaFs4&eJ%@f%AkA|}Q#13d3#GWUwg z_|i1*%P)|~9hCZKZtk9SeWicdf)5nR;`@6N`b0Xe5#eDNR-JWKLbm#JwRo((O9cC} z!>!CS48pCSV1d2_x-XP0YCeV@1K~{DFR;UxGgoR_-3?_XJ>%yF)gG@MO@Z#3op^6; zgXs)m2=X2h_U!5AwP>A^5 zEiuNJyL0ASWDwBYz983`7M^+$!Nw!UF3NKe&FG7!%V z=)f{T$OUJgL*D-N)9uvVv~YK)zrNxOwt6tuPbEY;nu|JCrzN|lc=cS)ysb<5ywl~X zZqg`TCd|)?QuWTR+*2VU;uPX&2w*@aA4)sytkbZsvE9p}Ixo+;vSV=M>2oa0ml z+-vg>Ji~I$?p=-_sTJ*kLc3({dJw=}hUDz)$tB7XDcV57w{mO|YrkOJ7cz@m3VuN6 zNnm=6194Qs^uoH9iRBYFih^_Z-m=4=>v}`Q2Vp}29XQq-SHf=EB1aC^hc>*H=Jmxk zl4G?Z)AH`t!b(krEf~4sw%@_pBkPBWO}ItQ`2sRUdIUw+4cB$@{3 zPlSaljC;LOyz(`KoS#ngX7i|Ql}1zMwgy|1I#cIH3y~$G!Bq?QV&B$Oj*l@P zKqJUWhn8J6T0G71aC0GB`kU-H>D2hvu{B}Wzc9ZbN#D=tm<2hzf^You^g-)oYO-kG z@y2m;E4!7-kUGnb<&Ho}XmIq@-;-$vhMnNUX05>~4e_=LtfV>3V^Euq#wxt$>qo+q zjjhcQYea)`pWiCqhbu5)F$OpTip8hRfAvTmKbAvMiEJH4PT+cd z@uqijTyP>Ckco5$eJ&@(qU1?E20Qx%HUU{@?m_i}!mc}cpWYChU$BX1ZInwH#l zjc5sa^RP(Eh9zVX3PrLSHUNvI$dkQ^!;;-M1n$Y9EZRvd?3e1!0{>)D&0cN|S@2t@ z99I{Q;P*2>J4i0h>0};vdfsmpn$W-aF39+7<=($X@gJsEJ^6dWavY85DIBlfw!`+x z(yym3xrCbC($Qkw=WiqCdv%yt>?>BqKvAVj*2O<+`him4(|W~UTi40+=lkcuBXMeY z%J*py;UMGnlOr@@Z_$9?n|djYLV3m+?NVQ!e72yXe-xAQoL(WT!6_UU2DiATH6E*_ zR0mupu9jSs!z=38W$LC>n`sKdX>i+;s*IoCl|0o;Xcat1#qG$U^H0$@^laa|jyFt> z9;gv;vJ-6foa?7(q^p-)YAi;-Eidf*mQA3-jKnf$PNQ#Y1v=la7l%~gq+J=4n#Sy)D zR}nLh5~u2y$75pN%XUOv^~MZLBtMv|Xf;0g8O7tR>eab@fJ0o6r*2l}h&A_f`1<@e z@)=k3$QVTM=)A1#PC=m42m@n^0#-Jd}VWccw*0Xcjxm^NGEVr z+fAK~1!F}fcjlRIrx5Gf`jH6F1Y0`5k|57#e4GBcy|NR!mKgU^KqjpEWEH2wt!9Ep z>+weblPU#`O+#eZW)irUQrSLX?fE&qRU;O{l;CgrnC{=C^D#hBDvoZ3S}nKbJ04qq36+Q$yuyEej|2VL(1hxf^kGn_@K{ zl&t-iPKWft*j3L(Pnnp?WH@%I%rEh95g9)sqW?GeNWRso%Sq8O5-m~zyzJ9|*izr{( zah_w*ScXgc47jpGVl!8Gb0m2lxhcWi=CxX8Q;B5Ei#9HxwCc8IK$Ued2;^cJQ(L;e z`a34S)bLU__m~~+*UxvXx22{#y-6%}+a(TO&7`{LI-_qqA zMI}>C3mu*nW@u2#I_>K-@25`QUnm;Hi2L%VNib4RTqiE>78#FO;c6(*kr;-lNm#!| z(L6k?$RHnwmhl^v)^0eLvJm#oy$~yZrkF|L9pntMECZ;%nGC8{IfI$O_4yd(X?fR7t!R zc#;d&`gx;IRT#?q;nR?V;)W-y=p;G(LpwO3axl5*ORIol1^u*7 zrG;pDBSUO(rx(Gix??sU4DhI6yH5QIaxAzy zG@tMHxPG1Gd@U4ceOT)NttTeR7G_gzNz%A)Vf5JZT|5<=>$)jKa~t2!aY)mYYseIB zXlCj#nXphFAtF7r-uG+;WDZ}3Sj>3I91qL57SwfcJ`-Dl@xx?Mvw#BZiv!|Tr4UIJ zGs)l?DJrLutuU(s0_?n%?-mlzC+E@Gib|^lZe=dX?gZ zv5E=5QHTPn_O+<49J+z9f!w&rEaF#&-l<3XXs`?&zt^J-eh=||dbF77dW2bf zP-EuzZg#Hx)-nMB$u4$_yz@S9qe9Ta`Lb{t$}}L~OmHxI02ifX!Pk2B9_`jlRAORG zeVIAB>F!DyOCgPw(p_B~s1C(S;a_ff$q@aF7LQHlB$N78;on}~-l@2@lUcAJb9{tC z60tD>7te#dHerQ%&#fQyj%=-Bo_}xMxK{>iDTR|56|E(rlwcy4@<85p@*Ibm;X^Bf zTes*=NwCDUfOK1q5)IkzEF+(N5)D6?f|M|?oPR58bFBo#ei zO>@SrtbRY?FXT*Z88JO%h-)2t%8HPR9U>3MR-w?3nlKphMZ{IDo8xU)%4nKXu+$e# z>^(9z+bSvu?A%4mXvCFaX_Es-uewE0L4gWM;JTrq%SGtD0Ax=#;CB?uBoT}OdD2T% zH9hqsFp87{+p*Y>T_%Qg4@;Ag36aD2C%{q%OVh_bSW3Um(aCfyp{2H}_&Yb@_v}`8HoHFK0$$M$(>8&oaPYRm;u=9sE6*U^P-MSLjCnn zB2DU5PFr8C(1fjq{#X%!$ii4))Mk6}b?JcVb0`^;;B8x2?P1K`3C&Re@-6)yFNnLp zXdI&nJMy#s69HlL`$|=vGFUfOtRHueoa1Q88lWofSv3%L3;FTdQ0WrM-}^)i`f~g0 zoc0S(X@h;Ei_A!Bwp5cni!(n;YBV2XcA&B=o+!xNsPY~_#Zj>G#I_)9hM_DH`NSdY zaM9-}nJl^b6O5S{@*hQEWI@|#V`@RK5(pihZ5n85(aCjrsB8)JNW|V&Q(3FA6Jf0B zF{~biW`iFo5x?01n*W=LOz!#+ldk?;-4aI0OJi|YtcDR%gsfuCp^U{}2u@(VUladi zAV+raHW5k$4XrSIN!&@2m{6A0_fX*H5}vLCIy9^mJm6DxOAUBSPDOqPe-*7{FK%2V4O#UC{CY=kc|b#@Zz8?{ds1EKG$exNKr!< zsVWgHwW@4QVzXCtBZ!$QEvma=U=d|64h;)!IH_4zPoDD!jl9}_hE&Vv?&vBdQ^mj; zQgvjZ&{3;P_ziBI+INHjNY&--wYLw%cV)}N;y4NgXvaX5&|8P@#1B7mcQQd?D)+(j z!q{LB(oa^ifS47~D@oKpG1$U+`02c9GEq5RD1s@S(~B5gvmI-rf`v)_G7iCDBX1&K zS8DD%bLtRHnXoQPecA=}hxwT@xtN(^2>fAlTM#Fh49GmbPBbYZDiI|NMf*{oyBX73 zvq+khJ_GieWcI~yH{v-0DHx-^oJ;YJY^Y#^Dh{(94ahix9{-bpDB^zIMTYBdi;=$H zn8GI?PW<%MVEP@pP;)-6Zia-&dIBnGS&t55mbLx{wg}TU*k4$C>aG+Qu$2 zVBlzqln6ZyV4lzAHCINeS#jx(&S-)vsUQ}oJzZAV_()6!rtZtV&Z$7#7-J`R$MH51 z7WjheW9k_pDX4y>AxV6!Qpojwm4PnxD<30;hapi*!m~edb+@$)jP2(`L|ONJS| z`0P|KzBgY5?}u^D{ypy!jh)zj`nj>7u$Y;7X zxF2C7H@&`Um`d@pcvAvy`7?BgP)B$8olR`8$Ie>^r4<9r>fpB?3gujpcG9Y~9qv(u z?j<5rfhHOZ1%6UC8l~Kz9YuJD{P>XDs+R)wZt82>kXJvxGv{bt*5*yndIB=ohyM;% z#i%*jp*cU0J92$pR37Z-e^uvB2N1^KPBP+L9qxAsj@=o4Y-UK)c<^tKf1XZ40LKxN zkhb$BZF@`UXGrCn9PVw43!(T;_uWZlNMv@0)0Mx@s{>GHDve7LdHn4r#%I*UR$>>DRkEIn;?h_ZON^cX}O;Iakt5EPc;Oo6m%_LtCH?-6%461e^M zcMH<#Nh}Ps1UBm;R>(5Kw2e;b&JbcI#V{4H(?B6N-yQiE{8`p`w$L9*ctWaf5ls{j zTWW-r`BQ=4Dw_`_XC=ikWGhE#@ZokW;+sy>9w1I!5F@W!KN=(M3)yDUB*_sF(%;p1 zAn_C84oi(nw;hh2yfB|k&sz92y%JWq}plOL<0z>$J4JgOx1 z?!k1#s=W#e6R>-f$>h|;2 zBSnbgwte3~ve@9&(d^eeqrYXr+&H%Bc=1%VE_U`U{Akz9V}AXed^%lFP{|_KlUGX5 z9LiZa?FU`rv)|;EIU2IGD>&uLX$p-CJs*6;J3jaY($&&PjR~vF8AM?Ap}0uh4{=ap zU~S0Eqg+-kz~tkREl~L|>bzw#dAqqDz)=^UJ8#8LdDGsiD@`H6RX)`&N2y0qh;^OEeem^p-c_MX+(Vx6xg z=#0{>7xiipga+5$?uQ>`L?qW@iBU~bjRdH0(a@(;Uw>`u<~s=YoI@G^n^UMgs}$w& z{1O&|(G!dd&i9D(x*OAk-F(S^I5H%ZNna2r%W_fjkq%a*#XqlE%r$*!v&+T-wS~~!0hKjH;F@jWVI!D)utNBLQ?_^! zuX>^r2BYPopwzMdidj~g_#|reB}oZv-S1v3$=Hxn;~?P3Ark3_h$`C-j;n<%lD)F! zR&;uqTz;Jc7mbEG+mEMiDT;-2PCpxJjV)2J$L$iL_*_iM6X|!GOz6ou%gFJuAFcG^ zFw)?Qzea=C}j!-FXfP5K%eH?celugz(t^*eWJ_kSNmy#x~S~2jVT9K<@O@t(IStkD%Gs9;$^&DwAWaO(lY;5WMo^&lom8F>IcIL-At|(B$ zIgDF}r}Bhw#HXvAT6H!DKPge}`0LTA6VTJbW9BV?BOrD+EhwN@D~0KWy_SwJaBEG@ ziR#Qj%8@Vqt{B5FB6?Soa!C|e+dIuDw#MHI*l&04%quOTw%OlJvY)=n)@VnlP4hI& zIy}!|L`rzhewOU?i<;2I8M81@Sl78@RAds#{ZVEV%6|};uaOg8@{~ibm|De#$D5!< zk0Tj~9Vkq70udrMHPuP|^t!F*}Y6SVq&xXNd<{fIFK|FNPa$bIdy| zF*=^?e4~+`%J_PTxCmJJO`n|&FpKqEj)@o)&i$KStx>H!?{>Jw!#uo#N57<+h9|J= z8?}v__!ir0<_cgt`8mJ~YM!$Y7m{R(R;o0J2LH$v&e)MzEoHc{y{+jal8)?1Ecr}` z(%6oVK$Nwdh%YTZrtHE=hnlK^ZyP2M7#1;ehio&mU2?O)n$$teT~)@CH~=k)<}T9k zr89xEy@5A@>PUoQs?(>;Z4@V0ocdYK2$Y{#z)Ut^W&U_VPHM^9{ie9&fT^}hK7aZU zYEzLr`gnY9gmxcP^agw&q*lr=yhcOe$bh+NzPim#{XW|SBy=clOCwg%gBTKOQihOqV(ofBvKRDi8?x1XF zbM7A!+htg@p=Yz3xm?wp7)&0-58SeD*!5yum9y@yToog=wi^<2M+o989SI4LLDz)P zKc02w-)0DG^^MPJ<@^sc1oMR_7^_3Yhxu{#Tl;?Mq4z6 z;;GOPyl#7A!zgnw+`H^4rWg6Spwp0-<7wzp~a!ZIc;PaogzsmyfU2Pu-d z?da3LtC!Rmk?_Ez>co%nf(g1z|MFQeUd<%2s#js|#K8PeDCWxXm1$0)t>7V(C_d1~ zz{hc|m7~ngQKK!tbbPJO;yjjuP$SZvkKk?VTYb|5K0tYr##fH-NLGh8967rciXpT0 z`Ve)ZFJBlWvG(K55t}0J`4(6hato4k=*aTTBV*KF+TPk?&3($nlqbw#XLcDliLM#dB&r6IdfAI>!3nO|tD!Pqx zOHg3`%0gA(?!cesA}e=;ihBI}s&UW5pIhesB(M(!$6On4vBra`gg(#hTNfSf`e&fI zp7jgLa3>J)3}L~ z`Lx7l5^<9~{C=@ogJ1bQO9F~DAHU-Ur8G(5(wWR#^wYQ!mvfexrN{3gZuKm`U^pNs znli=cXfWtf!o{JA+fO&C^5t9Vo+6^A)i!HRjLg3E24+yJWxCrca1W$?qkE2&!dhNC zhj~&bTCy8)`27K?hjlv`#JJD1m{7KK9%uPt3J;iAPasiLh0`n>2&#ta+Yo1q21Q6q zpHY>nN&Qs#NuiR1{qrjW4vz&Uu+}qfFDZM-L0HOBi@;)i%TqAUgN0AVj8MC-0kUcHW|iQ`>^vk|Ea%$IJDwJIqgoixN%*}%vk&A4RM@Zk|DWzz6HH_{@PZK$7u z!wIESxj@V%5@ymf-FZxl8sq+54>bSk4_v!_fCD`v?`8#)gU};nc?rES_)|f|D*%Ke zm2F~7M5>=BBm7jmXlNc=JYhV`lwm*YC7S=PbL3_1^bNdhDQ>EID@YGIwS%okEWXMT2bKxz^a- zjN{B1+@znQc`-IQ^gx95I>?J5CId>KfzlY}Mv%}FY}BlxNi0t#5xSTxKJ36Q;+8{n zaA&Q@xv}F%^;jvq8Sw7Ibv;9!)GX|5B9I)hYkHN-oel;995?qap(UE*`w{>6(q1(J z`!S+R3B?0(T9irBEMj&Bmx-g`5S#l*RSm8O%ZqwLo0~`_$iXX6ur+!xcwy~vNOiK( zIhJwTYOX|1jiY!)$kT$kj+xF9q^&)=i|Ws3<6%G6?CGb~m3M~tk$YIs+%0?Au?4+y z7=TPUExa*5Oz^bFjRoooK2j^&&8}|DI*is8ed!YkdqV*o3s6m$e}_bdC{$+a}{ellCznc)bi#{NtRtO3Rw zA~Bf&DXJgQfHd`>-}$+k08nd}uFbRKqHhsF^yPT%f#D6iA&Lk$*z-P+uf?P@M$|o@ z?bhnf`l;mAG5Vy`Hb|fTN=7R+mj2+}9JV+iX_51$iX5Zv2NK4m@MlJaJnmoeY0_M- z`PyXh+>ahuf1xdwmrAdqGPNsXS$xkQbGj-%x zAR?gV0-`oM$gdq$5cI61jlSHuw@mS~f2*4v!;g(IYI=!7q0;p4U3lylxw#c5GScFX z)3$|yeeof<5I@tUQay+CChnYHl_8^`!1jpe;sG}LA@Dn@=s$e#-ItSXtF{?fTO0P( zTmDyl?AC(!zJed)SK&)BiFH0!y$w~n(ox^#FHzFnb8qe~m7`PpHgR4TfDr1k3j`}g z*hP;<_|YUPT8kC_FifFW_5K}2B;zag6V<_iG>E%`z;{LX^4lxNS$V6M5#!a&*NiEw zzdGD233p?|!g>ua5lF^b{j#aoi9_>S)A51^!Ya@*T<>q?Rb$EAYJ!l;o90M*PeaJV!1~0yreREUduNQ_uq#MY4_%DFk6B9*6fQEM9}@Hg4ybEbZ7x>H;Dfr z(b!>Jl79p~V=j+UxDF3Sq`x+f)PsYbr612CSl+r8Ux8VdL!5ZgS2?s>_?pMiTW-;2 zkNUPlDf*U~#dLU*nGjY&!hq!VrJy(%DOPop{oa%i;DE+HCm0_67ih&Jb-p5JC^ z?zC)AK|~4&V|XoKv6kR+N|Nk>RK1FC&Br9RBYyP+`n$Av4F^*{EU}Y66&{Wy@GT36 z#W{$6v&ilbQ~C$vifof+_M|PFGJ0h#<3FIDFixXOgU0(jp8jgUt{zo)MK=cY(-hE> z=_bJC;r?#MW{XCEn_BY&2NhuB9)3LwhbJ<&5CC)m`lg`MQ6KMyNJi5sV4+T zg3JV=c0)!RuSYEh^EVA^Vv@X4Du!!5ov1Ckhk{N0)TBU`7$!b~R371e<^)&lz~(Hu z4v~;zH=={fuRXzF1QI#CL@=Lw_l($Cq_)j+@T;@kzKi!Il1Z|$je?%PRs(vT;!o%i znGAncPs&`UsT2^avohL|=BiA;PI>|-iRoxz zRV(+Dh&$U2z9pNE-X0Qdu^zq4>|kNmD?07V`33V3^9AS}Hjtt;=gtx$H=R3ic&Dh& z;5;<7_^nD@LLYvud6W%F77g2s@XyygqR`7*eedd>mPVF}2GsITt=xILpM_n%GoLit zFRS8>SFP{n+(+#`b@KF8k?C=vhyKa@JfKl7&Q!0cP_Becr^iiHL|IEN*5hPzK5|!7 zO2U(|->)97)zSTwsIq|Ny>W`1k%-yXTP=0_l##DO9c_vF+?vNa#Mg{)O-fqxOQAg= z#1UP<#$Bm6b7{V&3_#1aaQkJpA~Cnjg{H0^8RFr=_<-vCU+Z_KD^ipK*IXLO`?NvxE*qzrwqx} z!UVV&7Ecu^#+2VHCI_j{HwFy|c*5l!*{WR*z-G89P}Y&_oUYdvZwPYjisJ(6R^x}5 z1%rx|GJB1QSpW8)ua6JcUo1AtXj?J-lKN*NEpBJ^i{;42B679=>x#$c z1kP>ecI?rgiVf((WZVH1Sh>QKIe_g_ zB*bs4q$;NKfh4K*I*B~q=<#YdghvnYi)omLVoLd{bahf|8D``s{V;aRGHw%{^=K?| zJ|T2XVVIT&YdPEe_=Ono8ghQbIlT{ef77+O40dF$z9ZI6rSawODWx7fyw|#chQeY6 z6?TuOEM?n$Puog+ztmi6ru>J|{yai%K#8e|`zL6f6>OhOJs(rO}_-(i8 zyB{nILmGzDTd+(nT=J-&s8fOcxdY_sn1UDQ2S+KVqw~EZ9a`BNMh83oqw*C3`;}>} zOm3^vXBLL%mVCPddHJZrK7myGX3tZqV3#6?D-XxQM9k>NwRFmL%hvXln9Jl!I_hcw zL2uLxf(O_>50PCh(a&JvqkGONC0+(~DEBOk?rKRvg6)V9d9U-B7Vg2{sDugU&IVj( zea0MuRc1s^fy{1;8v4OyNM5wN$FD8a?4R)~1Un9Y(Rg*e zk6_@tCVlNFzA#MxnMLi9u@z_Df)V+n_xw+6cG|NLNA&cwrsXs#K3`x=eBS!4l*6X*BpB<;o9CteU#EJ^N5hj3@w>ZR?O9x z5!0RN74t`x)ebr0da)H5Bn_u)EB5vb zl`_e?in;mRn1WTuCPIn|rf6i;T?c-I43dB}R|p|dawV=yuJDs@Xyl!(JU1+EaY|)x zm6#V*s0umPi92?CS~Aw(-EF;_n5P`7n~{B9?bWmsd+b1@796QpUZ6KUI5uwNb|L$G zj7?@oAJ1)UqNhiq=2i)wxmK(!Vg^X+;(x#m+s#mSkRjE16I6C^&ofr#+Afc^W3fsv z=&N2^iMw4ANxF>}-k>%sbmg-r|gsSA`4Dfy}Ac+7!9}tU%5S zT3%@0)Vc7tRqmLsZzS@d59o-?iYEX%$ zR4F~$L8hF(xHyG)Wz#psv)mQZ0- zdLk_vK=usaA1mvwi!?rx+(pHMMV6Fy#kzs$zG3nUmslgqX#N{)EJ5?V>~$&xjc;n7 z7Yky1h>X&UMz7V7b=aj2r<94S45WS-q#YA|MMEUvwqkHQRc~X7CJ&AI^woJ$b@%~i zq=cyNuVuiu(hpZDrNwU=-$CfdO%sGaz_Hxj4khghoWL)|V!&tgv#Cm&HF?L4Xj%=t zU#eeE4#{#V`ytO>fz@f;P~yJdtRQzJEJ=yt+r0xPB95N*vf4CJw-{g+T`X52&VzAm zano$0YCk*iDuvONtXx7`l`BxUikYUzZT1>)fBonf>_r1(H1cdovYC}zI z5XO(D-*m1m(N=D8bv|9=#7(PwC0O;za6%jsmKXW`IeDbzS*de=n08>=N zK1-tA{OhstMrqRC0qRRR5!VnU$#dW8RSq~+H+y3)_%{gncQVjCEnRDgrJIU1yfekt z%f8?HAoVz0?+hmz%FQn;{Op^P!Q5JIjH1W2$QL6{2wi16cPltTF63jHZ ziUvG`B$1jI9ew)K?nLl-Nz9^q^a!f|@pX;nKPk0xnlmFK^N~VE%sD$y_@NXHsg2-c=7L*dFs|>JUX=po!^*3 z!D?8wN+dJdq-ur2y+^USM+%WEpD|;?HnH)eC!{8n;dKv;K9;%`638?8`#J;Ro``qi z(*%lK@yNlP-Vd!&2J}AP57!Ih6|N!2sEOaZ%-cz~{ydxpITN9^nnW@fft#BtV)iid zX_Y7j5 zE;QK_fRghyA6MZkX@YC`fr zTcb^HS+d{NHFWe%sYVgR598Pz&!o;deY+ zJ?SCnl+&>KA^h>+tho2ji9AG0(D*qKJrqTPN=vdE*RV>cV<_mq7g2~L`K>D~5ZtuQ zjhc&OV|m1OPMbu;INr?UjI#n7k_qR!5=(4BL!Qgqne2Em(B5 zAy+d78|#%&xJD1&^qJU_K;l+V5GrR+tm><0#0U6r0Ci0(7eQT)sM)I$3f6jwh*ZCX z%~K?OvJlM`w>zDkapvt!t{*TXV-iG)IDwy^7ITVUizXp`dLhX#hz9hV29 zs`_y{>qiW9i5IzqiN|EH?U`DC_uBhj1B(m2m$0j5l)HT68%?T83($7Az5S(C5Y`Ui zx{m;TN$Bi(56_M|!qtnln%-m}=8>E1v!WHE^c1XSLs>2nYyF#v>5p$y#)^x~swU}< z^S<5b9=&#i&!Dq(vTbdqE+{r$yol`3ct?dJE!EAj;3u(K83RjZ``FO1Eh~htW;FL4 z;!WNz=IL4u(0P}({c3MSd!_vxycpFCR?^Qktj4ej-<}i~A<2hW$OGRb2jWK;{nVo$ zlWV638^ zH@42)SA-wj{R!B)@A%LU$CMOOBms}miyog%&7%Z?lF1lk_G>DeNh;jJaiNXNi$BoR z5cn#kSA@P{bEQj}mms{-kU3E{88D_c0B@Y0; zCkeb%s_6QoZa<>JM@ll^ClJNvK}ionCmC+B3kjh!I@HAJ{d}Avbw@;KJgBByR9(PL zl9*r<@Ip9arj0OM_ht+}Z^)dS{JhCFoOZ#qA`-uxGZ$zz7ej!x(RIdBvZcJM-}oNd{8x1;|h8f4^o zSmipJrTl)7)w+J;s)DAA3dm@JB8_xuce&9i|4I;E7xC8QfE`W|sVi9~@ zji%vg&6vAyx-6!hte7_KC8^{N&rctfiPSfdT-m1}Sx)gCUR3d}{hA|?o^T0rKa#2y zXI-;w?mUy9F8BQMYM&C=8?YdfS~R^aBQw|)tX=h4SFkbK&Pa3DTOleTBAT_vxS0;OZ z-6@MhkQ>Aqw#H>^q7lczLyM}6Iw1F(5~nMeASSI&PxERg&UaTGhA;+M8ivdo{#BHl z>4l3x>;l(moA%3xxme}=8Sz!b&evkcq=<=nJn8H_znq5YP?o_gH)vB+#Kh1s(Z35$7ujV?)#KSTF6-Md@S6-X7 z6zQ0KmOr(@j)}3n$;HdxTHuazX(&mBPSc5v>M@;GeSMSU(DWd}be>HAONGvjg>5>q zIok$E`V@2{6UmdQ*PT(90QMWq68fE|3NlkC3GS)=ZZ9_=GgKtN3a4-_E>5?>}qDVIwbENL{*`U9ig- zNBEGT;K;;~uf{{c_D7MZ#+S$nI~8k;shHHGJ5N9j6rbJfpQ;_?l#Zd96F0T%^|q#n zFGe=PS~#-so;x5}gUwyB-!0$s`_!_O)@?P+r%^q6A6J+nT4Z^FO^VT)&T-i`F5w-i z^D+h_uLBgdiP+vstj!$OYAjX%QCU70h4=W-Mo3rMOxd>^eQ<(yjx9&G_GS zU)`HL!AEwF@)rEUStT>8)+x)2Epo$EAHHS8?Z?Qx9`D^upFi@iifPP-rUO=+8)(7_` zrTu$9h7ZlXovycIcaV2y^aWvi&63US9@$7QrEjzq+=sfGoJ*ba=13o}P=-(SQ>UIP z%>BHAs}ApFtXZ-k>C^;ri$zrZnA~4-P8c`U#hM2BEeFQ*#FhhCUFcM-XvR>wERUKfpG%ESD`F}59km2|zMO6W=%;Ao8=u&69rUnzs4 zQO+*?+(I^Kbj+xPSm`-E=W?Pf$^bch?wzKq&cz)H8*Z6dSXB~<9Y>1Af*>r_qY0I~ zZtTPON3>6}$i6I$u7OaNwBG~kdA@!uZOzz0>@-8`1l>rxTXzYX)Clc3z`u~-Ag0@Q zGhK9aocT&CFELk6UsC>V5+?O*D}79^NhTnW8(cuM&qA}+;XxnQu!xO^o{C_2n)wkv z78cK~@@_E~x6z9uRInTrXhlg*9MUZgMwvb@+bI5C!tA~Qs)cMW>kEw8I&HvhjO{(` zHJ{T!@$L#Fe6PTO-e{? zy%+b#OrHE*@~^IS=v%<4K%|$$#bsRud-p?;F-JHG^4h|ig=g5VHBwygOn$k7mp2b4(A(qKmK8s%*j?Q)rSVP5rJG;w%DYp{%=mj=Wp zsy@Apl7()X3KBL0XdHzzN-gj=$Pn}}hu_kF>&cU_>iq=vN2;AcsKeN{ZBQYTp!Z$M zfrc;~uUrkr^KUql)Yb07)cyW99bb~OU1yjpqRM_=0)u$>R68ybAzuPuog^cmLw2_T zX|L%O6|&IYjnTzcMiL*8BIY&Q;Lem({)63?aK z8S-$JNLb_!(lE@pjvWPecK+s)4h zp~oNn(Lt)JmCc0$k_j2}@*Xe5TVrDKfu?gVq^mN9*rJ;0RQ6t5-A7@vW1j27#2=6M;{$RkTk#CLF1t1WGeFG0 zsVmo!Nz>lVNt#RzV$qe_FjBrm6A6(-OfeK=xlU=~F5PzOun>f^l$#jBVzR#>W6jm@P})PqL5yWVv;~xa)ci-x17GM~ig2GLR;u;Ma!RuB5JM!gg~pj)O-KhaQaJ9; z*_LIsOZe36EhDoq*FxaA%Z0+f=6vv4qB$ZjeSiI#I-(UqO~hKWDYu*ycO$H1CkIeZ;aE)wJH4VrC*>F{uov zQW|;R%RAjyPf&S=Jvg%O(k+&CvtP#=zLDFn<0s)a?{HpQt*JY#HgPYk0jGU=B^yl= zDn#YK&s;;2q?HBKk)ze|H4~Wat7Y~Hc~o#^ZjQ$@+P?}450C|6cILRqXxMU1;I z*Nx^`3_^q%U@h97TB802TiGI2mXpkp^*b7%b-QnihB94hqFMj0*?Oz_TJ5~;T@ zk)rlIWG$|2ADY~_De0`fD)@Vp9nycUGQ8B9JdA-~nS`R9NUgltqN~|=d-AJOFfhQy zQG)QxgBYg@LMW3;sC9Y2{@!xWR7q1FA42OcjYE%LeVAF=N2YIJ0|P51mMY7Qh&tQA zm#o4|;M^?PtB6$^07pXCX>}o5sX6ZKm!%cP2gE_bTKwv)Cn&e060IR2&*)uN@_@;XOfN>rH>E<1QTmu><`5E6qk{ z)!jj6GdQQczh}ojgVD=geTIc@heUe&=iVP0wJ#S=nwiXb@D=V-xER-|piR+oIVt+r z5km-EpLxop`1@5RHAA6i7oRO`w}i8#eHlBRp@3y{jtYLm4sZ&i)26T0 z)hP%#9|JUr&MP;)+gqR<^`TtJD=>Iiwaw($DsVI*gwJ&u){`PXcJHM z88Z?-Eb9zonsKUK#q;r@S2ea($*JAfALL8jjCUGtr?IbO`l@pm1>*V!4@ntiaSz3( zh&dc?+|(uRMHOIeyiwDZmQ-uPj??JFm}&koQtGX!zQih zgm5}Ik`&&sewo1Xph-;R8uqlKoK}ZVKjLV?)TFFfrxYr;Rl5nIz+RQgpC)w2_G>#PL?mO+-<&OP(T<+Ro-Lg&9ZnjLp6m=&d>7pT*FFb8Z+Ad`w8tiSQAvD5v8eS54S9SozGGNjNF;1^)>gk+?*lvxVl+s*Rs^I{XD#C zv5LgKBfbR_izWps6(x=EA`l&_*(u=-N-meUbINcO$y~u!@1bAlzYjVTov2=cuH`F= zH;oYL+`2Pt6zPRqzJmHDYF(wFPc5{dn?Lv!5QS6v2j~(q8d~AD&QoTyYvitBdOeE5 zG`7s`Bo3lf$jpel0}8>T>iP-TT21^_nVumOdvf;y>Y^pHz4(PcK4@{tf8*czXpUdT zWmuqVIJ+1EolZg=VZii=&G7f*MgAuSpr#xH2$56xoV2yWa4oelJlc_TF?N&^W8!_= z)UuCiT+Fbh@!RDyP;EpL?-w4^ z#U=LJy%D;kM$LEU^4)t9#R%iTDQ-(90ebtAp4KTw{T;e7p8Ag0p`k$H6N%=>l$dr0 zcKEz_CL@7VUg8bfjg%w`ca|&9LryPuJb( z0Wzq?Qfu=`#nh99zcW3ub<7U0X!nueT5=QS+)3|Y+-Mo8@$5K{#pm?9h^%&iz|qHv zN!mkKET)t5_kI`M;} zMZZQv#N@{tl`Y|L%XD=Nn9UlmZCG$y+iVcz0vTlFASeVkl5=p0tyz^KXDrBJRss{( zA{NR80Y-_>&Np!3lWHb&h?RQ2(sTqkXAd53e-%5w5AI_Ij>}7O=rs|EPrKm!)^5G1 z9_160rVbUE6gkZQFBsyM^K}^N!|0+OyE$sA7WCJEn(*1lV}`jf%-$Hk-ZwOT=z8tG zoe-{rebxu^`&j5T1+N+4urE9|bz}q`Y~9q~_)sE8a_{6V5$=svqYS_HjT!bhd8j7v zKNmCE>GaP}m%+A4-_hTPc1BatY z`<}1cL<$t}Zk^^laiKwF-{CeQP`S%xDI|etEEM~i^w6X_sup&HG~AF&3@kzhEClyz zM80#o{+1Ag#03~_j6$#ko;h#>D`39C5&IBF?|87(-oV+1Zwf`z-dndDX2g`ds_?UG z*G4?f{1dcVfbws{7CQlqak?Q$_%6S7cQn5l_(gm6fbJ*^f*h4(I}@eWZD~ytlMru= z1v%=D*hpC_8OfYriaL0fC2z%2uAQC@4I7*}&f_Y9eoGd5wN4sgBq;(hA~^+Mp2LyBAt%JD>F#x@Z<)=-w}H9B4Y zqL!Jk&OE`0c2vmMXs}-@L7KuuE)R8zGvKv}Jv5S4gKeq=b(hbYSpkj~s<6vMGmt-g zbA07+MT(aZBiMgkwNY#wX>~I+l&T4NtwBy&e%5>!NJ*7Rr&>n!a;T+G`1!`DVs4IZ zUh-tiLTuNa*Mp%Gk@}}U`c>1gfohtc+aL;iG+t2pM&3efrG@iDv`lD^7GQmZ1mugB z0&(dW_8HsDuV=2HBUX^$Z3=f&0)D~Iyz!KbYi)dZRiZ^jhPmw$L!wxiV3)i@M4$Wp z{aFaqEAW|lgkuZI+bo)oIXOwUHQ`P{BvX-u)I}qesQW%nWLIzqKUk9QJonsrxc1R2 zh!j)67prkLi^axn!8JRJh@9C)`^7kDB`W8=C8xweCWb@QyihO2k39-s*CLOr7hnb~0MURfJy?_v`OT6pGcV52&g+N{6-IDj>nv zuY<|kw##*+_4h=H7%PePYCi`_|5PyKp9vsG!N`DU2|EIbaPf+aa-w9hFN z*ZKfsfkpes?X0?Tx$*awDOP7v{F_>5`g2@Uql1G^tPWC5&#avE8l*l&fARUV6;zYu z-QcAxbi$SO-^6^cj8jNKPW18E@iu%a~mUJL;sBV zC|RnOvfLUgL$dq~b#+vSD{#J$-mob$OwUl4nOY6mU+u1rhvkh<6&p&5RK|*L-kBzt zx+dc4)9tseTEQUV6ix7ZF*RmYXTuIeadSTvkD`kfQ|;kQIGr#I{17f`3($`@j5)=V zxpaNV!Hm<{vzDTS%fE^E?Ms&T57M~O#$eeP^_mswc0W%*#|CT#Q6<{I`t|l|2^-*? zgO?qbl>2IM<|0WosOoA@X@)(Jb8gn)XARq*v~dZSqFuUO!|HLW8;~|9_I|HvwP+N0 zc20PJ&{x!apf{VV@_XA7Nu`Eo&DuVPuZD(jRL+rE88-Z~YV6;5nChnWY5SmO*7g{` z!e+3Wuqq25j5osJ58ZF!LPDk4_Q}{$^M^K@kz-Xiv9^U;WUr5R^6lMQ33jmqzxiCq zuCY%XdJGCUb~Ucd$ovh5eHUJAIB9Tieae!FqPL1K9rwitoajc|eOu-{4TBx~uUI8x zjugN4U3$OM+wF@NzEw`8Dq^KXHp|TGL%kyoRH4SV6yvB#7@eG=bPx_!JzIIApd;*= z{Q*LUF%ar?ap-m$c&j<3YhHVU$DujMcM0!}##YYhLi(zml-6l}@?~@~yjD*6lrbt@ zFT=_sChFYmZ??}@<19E2rAU?`yQ(p_UZUcj`GIH2LrBUSQ%0Px+?9!76sffHQX-q_ zLu+z{@BNn)w|gS)ajRQ<))L<*#DxCPL=62@nhnyP1oTTYd+K~)ltnWZ<=eJU;>eeE znRs4w<8%3#p`GEswiL8OhZ@?Yn8{Y|F(l(URZ2-vpJO~@A7wDEykuDPuzenFD8jtr{r*Z;{{hu*`oZ&b<0jG zNgK(#8k&ihN0P@-{AKsuvQhC$n&v5q>aDt1P;?&}hD{i-b8JK!1(`jWv31!rjAG~g z8Qk?t)=522Pu1f5G8GmJN_2tdT#X9aEM`W4E&)Ru`h$yJC6S8B)GEIqMB>G9Dg1Uu z$4?|QRGlr_Z@|askz-;V!CR|hHPD%WJEj>GyiH&&(A9qMmW7DMt9mTG{XWZ`nt6*i zjaQ`p{D9wur_aeN(})ls)d3RNOiTPbS-!oEKfksZXfCV@i-$s>>QIK^mT2!q63aB{ z6`GFao0HOOwwN&If1}L2DSp@6DBEb6-$_@jdBpY5O87;IFfJB^pZ?)ni6K-r(K3IO+|IXOu6~tR`_a$L zOrHH>Bwfr~^eD-^5>5x@d3?*rQ%kF z1y%Um zcht_+w}bFLWk||Ps^y;}ZoD_(beI1Uv&apxC?#~6ezH;ZYgr>MQ$vX=LGc&h%>X^W z^q93C9Tf%<%*~vKgYnoN#Tk$X?xoHgS9ni=)KO{ZjUf}V#Xa!JZU*g3W&Av3Q>2ZL zn^0ojUy_eOnuYqi@B}E$owY=-Oxw(jQg1HRrKPIsjs9MkjWpPk9Ju{t4kz^+} zb90ExcWMg-;!u*Ewnp{5^tGD}d%%tO8M{edfz~woJ`KC`RN+qjLe}tu(iB$(8?C;3 zw2T0_9xWH*^|??7D6}WhuWckzQ#w?$WMQWlv8UB=2$|%VwAi@uOD`cz5idj{e&*7h z)^-;xK(*<1&@Z^G23Q1hHv}O+Wws-N2Z)Fyf9XzmeG;C+aYT}%{;;c8mv5?RY7=}V zPfyjoQy9z#Kj)CQtTo8|2bZw6G?NqC$dicciU*}aoS*5q0%r+{E$d`;sT7cClaMi# zX5SJvAyeTmWX$63F<(7FK%Vp|v2xZn*AOo05N{%G4Ac4I!L1f@m(A914&jDCa)-?) zF( zqlaY8W?b}$VGFp8f{4_sSOyP|r;XyGT0$<(>4`Xj{_F|Adb%@&)0ZFgq3O%L`Nkf+ z$EkCDbP6X-tq*F;I0TUR+ZU~QDZPQT)kny8v@MgMnR+B`H|ntML3JWPr}lt^UgY4X zFRfL*4m@&uy@Ah|2}J@M-`xF!IuCm)vg!`aKv#|&EP_vwsO@Q+dV&a4!r#4Y)h}4R zFpWOvJ>BgM6E8b-!GZ*=xkMu4tcLXXYHc#)vky1r7Hl8egZA2n5G~`9=)uec1cFm%4Yw*?m2-7)4Gkkg&Qs>Xu1$7!30DB z1yVVcI=o@HMG`G!xfWDOKk8E7|PK!T@(M@cNXzqJd}LF2tF(Hmo>ZwhK(LIP?Gx? zC`Wy(A!Q*s~lOuIxmBxchO2jl*#6kx1bWJF}}b@ap~lr|};dc8O# z-wXJSRjZkE!Omkti5WHZdU6>MPX@I4Xo%d0Y@KpJx&~NyzN};)1biuRta=BVak`(m z;*hhbgnLHtXry)_#&!E{;6T4-GY#$ zF-iIsabL$#Z@#cZd*rR&QU!XIT@MA3x$&BleGd~+o=k`$(gfqBNDlj@G2C~MI>y&& zfL=bs05r`Wfnh<_-ny9R;JMfS3fnIIbtOl4G~?mzJg6_aQ7soYXecPAO8MRI);7;Z zPrd~sq_g<)t)u7!fA{tW3zo~jwFh{LGjE%mU6ybktEbs70)YM%&jtxktWBG%!7Eht zuQNy1nzZGa(hsW)A7(R)Jaq77KQ&4$B}S+=&?J&7?v-F`e&13r4x%^~lYM5W$?eP` z5V(p}_p+aM9xiV@w)TFiv~@2rOzba@d2Y6_>9)FD?8H+OcPg zy@PKD?=C>`d&ZfZJ^ED9Zd2*9Rs?(kE>W7pq%pHYOcFO9$GGxiIBJc*7v}m0kueyb zpf=6t#3w#9JsY@J=2`_Ey)ojMojm5_RM z#N?^l6dt$)&Y}dwv?RXzgZeR1$pnN`Sf5JDThn_|o#nKanLY45-b<$@*KtX-PdP^V z$)%B`n;ur*P}7A{E#`zm_a+TdAI{%#VB>x2cG1s&Wsi|;JUI)`oX~0tdx3& zA77-dlK~}hHQ2QMzlw;xs86S7Io-5!I1n=S0~CP z>4bdy3mgu$D$c0TWlytwG_i4Vv3zFiy6=&Ea}G3c<`+uENro7$y!fQDS5%4>S}m{Y z{5+j-xGsmjbKA-p@Z~CT2jdFYbA=mB+RpoKJkhTkLvj=U?Kx?|wv=PcFHIR!${Hpy zkboxv6l6fnP)!Uh1s9Hrd(1GbjvHdC`pZ{Mf6nD>-(0I(CTZheC@o*vpiZTVA5s|+ zXSdIelIA(Zf!tkgnxcsu|7<-ZJtfZ6@mApm7CR^nsmF3r3hnejKj|n1@5Zn|^*(RZo24)qw?%q06 zgTZD{pGT4bx!pU*BPbNIX|2p806?cw_|cLPhbfHDeo9TMqe zN0j$M@N2I^wZe1Y<`=X;h{d4%PO|9LY<&n9pR6YaQl$Bc;eUeQF8G(>ll5^Xi9~T z+YEv^NfYI=9=W+6k~#$`5@pflMK})`v-LmpiRn|ZCY)!suxq{+n4-2bNM!R%PY5oq zo}B_swu@Z^VG->+n#A(59j9GQ!6!Qli7GroG3#gbvEKPc?9qQ$Te#z75qqJNj_Aw1 ziENl2t%iETfZ@>0Yp6K)s`RaewR4<|{vp||dHQJU5VdA6OEZCD?6Rg2>s!FGoYl^K zpeCouKR95RPf)rLUB${w_WO(l$u;!C2Tv6vWTs+ud>?tECl!{rB-L8K*i+cJwt|y4 zu8LT%g0R0Gzxi>*93Ba*?8z3yj6AybJ2%D1^ZZa-8ho$mYghL|?d{GS`F*ZfC$`>S zS+BFjj8O1I=Nj=t4#a;8i^3R2zeGJ}tnr(3+ajZh37_BO4=~wJGYhZuZ<4 zffO;^apK!}p&F<;ICfDFQKMBf87T$Re^p3SQ%o5p;&p|vi)~vlQ%}+;>iJx%Lx05& zJ#8Lt*u1ugU#rA>i>U9?D-IVri77>x5n+mU6~$24ZyQhyK52!4>W)vVs86_EO!KPx z;$^-f^73UX9K+P2Uv>cY>UAw+QEHXpG~_vhvxu$It{xs`j4c?)G;4jvo*z~F)ydJ^ zh`3A_iI1)oncmGeA56m5$s;(4E?D&eo!+N9y`VLso!&;Uy|+xP@+sM}nVD{s-7s!N zX#Y&*_kJrkZ--kfXyQp+I!swFq*%4B6VTCixC`U)geGZv`^|qwZsa85_9wZFJOT+t zV;qYXB|hulurBrlUSfe`&P0`ngEuoYkwqRKmvjw*wZ^P#jz5ULPdaV#VK-q3<=y7) znJ757;JHUadDO7sA|uY71ft1{Qfu+`SY6>|PC+>C!33BrSuT}(VDmLyi%it4~yssg3dZ{*WCQH2B8WLKqRRg)!aT@HRD;mp#0?KqYa>!{Jt zGt|z$G{Rrg0M8j>;&CvLJJ1whlYzw+#lg7v@2(_WM)%dKSf)(Oo+u+GB+jDvctf zgBIKhrY9yg!Gvv+T}9Mfy;`c8D!NxG@5@Iq=L{9*bN(>Zx#|u}xOYQi-sug2;O)iS zgYIZk5^v;LryU)68GmD_&fWQ%#UOlAHVT-*cnX=GHx6*rqvf@#3M19Xu82eGG)Ceu z`s$(rKPamlcd8`_VEy!~4^^z6yq)b%)jTW6GUA(gv`ObX2X01s?K0;j776wQ;Y%2HkBh3Zx3Urq zem@jM5+-R@*475$PP`qh7pf4*{z7Gg2ft;%ZDvJcOi(}FM-{Zh+H&}&H~l5Xe+el0WW3_!}D+p$N8%PIP#(Sho^>n*n~|QtKD({~Cgx z4CmS3m!K3M5sr_^t!aE8VHcs|^u{L|tU&QTqr|DkAT2*pc9mY@0)-|)E?VYzP<@rQ zU(3=_@K};nlr7@X2Y*6v0r1bAJ67wW3vb>{=aN8Jb0Qfxc`BU5zZKL$8@F}tb~|`g zxOh{-QBIZduyQQ9klR(?9#{xXcKxaG_5%caV&Zv3g!B?zTBXzPXC^BX8W(;mkbFeA z6qs~^Yu#4{)=!_SZ)SQHJ9O#s&=VzvU~1js?xWrg$q-o!k!u)7sNra{ZiqipVI~=N zqXv$lY`kerp|>ZIosi_2=t%x69z0b49xs&$J((y|tSZFD1xJ^_7G}Lgi?+TnB-KibQ)zk7cm1qrj)9M%TzFbQkJ5Q8}GB z7{rco|FXTXCci#vsJIxIX`&okIE#caR5ccYk!-$4#kKqL+CigaoKCp|q(f+M6tK%@ zBlaFKVwb}xZ`fJ+2KT!N(Y( zV)D`j+L#T$3&_-<4vb|m!%tOap%r2T0!~b~DX48w`Ga8FslM4CLJ3O@y%d73k~ASG zhi@aqSgC$kGAWe%ifOj1HI&f#53Y_BP0B&Y(q}XY;5eNlT82RoI8v>w^_0YMinKsL zPHFghwFY^433g7aW1ZmOY~#+=12~&U7JPIN zJItILx+I)egerLbk(?{G3RW+=Y9Tkfe~~dA&!XQ;Qe*=NJsIm|sAroZ6F6~I5y^)F zb%-|1I76M%CQ55!{q)XgNW?hNqsVT3kJ{f|QpM7`3a0)7)Ynm5dmSP*+vRXMaqbAL zDyPFjvt`$R_l+Qmo`v)<%kTJuSmCGtCe_Mpa-4?t~(wABB4Qe5D01c zRjH}|_|#c$KQmvS>`unS^KNK1wC>lMkYVDU4XniYGVt^3082L$P+QqIW@s6L>&(%! z;&8*9Lx-ybjcIQ53r0*+Be9fuBb2n1;a#5E#mCG^yzPtY4J_M6zhjc8nW`Dw4bv6&6Nv1Hn7Bk*)qP4w%!LW>`v7;dud_p`Rm|_`L10 zNwaCdwk6w1Fj!|iXi}i1!8_Yl8y$8+UwmK(UGj5&HoYC)hSee-3sjdcVtGc04oS`h z&0RpBvvyCfSIX5+8rT?S@zLc6{}R20Okdt#+*4V6@XWXp`E?M9zcu3*YPJ1OK;!2I zc6!-ljZ^%$ZZIO2p%h51!3wSNjj(QW9vQGPjdnCIFC`H3 zI1~YlT$@%N{b8TMy|3%aSim3UtQ>Ul(VEwg4(=}EPK!t0UCR@3Sc~YZHfKimdJ$nc z=|nuP==(6Lq{8jIxeN*V5lx${->PopMnzDJCIqMKmnSo3AbQG4Od7 z#!%jYu+n^*p?*=hX~235Ay3 z9I)hyd0D5Aj|M<-pu#WE1T(wCepYZca;&^&W26qLZ>1!+h+b)ctT_rkwQpo z6j?Sf|50=v32p*G6ulB6vn1!7dq>VWbNXSn_E>}Hs`~HAqC0!T9dT2shO%hNeTs~> z(D-f&C8O=YWy2aCsiN1rriHW3%-eu;XdU0Y9>kGaP5~kxbJ6hwQ(_jKp32Y}_ZF<< z2ae%bsypJdYSX10X8z`N5k6b;sV&D%qUuamHK^F{YA*b6Nzr+b9(7DGy-)MHtg`pW zSwvkq>6Sr|86J*tSBt$UHB-x2tQlmD$b7Wag|k7ORCPB%Qv@D&t$k70x9?tEVWNq>!v5cU2DDBJU)?BA?BijJ7T|+i^xk>ReK2OPdmY?h1 z6~T$)X87wve~uNR>mH_IH-;^*rQny`3zDuYnrel;NqQ-Fa1xbR{OrKdadBq`&tMOa4h@zGm1A! z7uGT-YUelXXKt-OYaL5xfph_;N9P36-ZyFUb&)Th&}t%6wCb>De?R+HDuJ242DQTC zt92y6FcbJ68q6g^MQRq+t*19k{7s5;RGs7L1r{^B-_v3X8!y79@Gq#nom!~gqfdI;(Z z8fJ@Z_qm2IXv3-hRM|0R@A>Xn>8YzQqrZcK20bbWm&JcO7IUwhv?0kfX7sxvMTTTB z+1R~#E`MmQAxPKTgI*n`U2_jbQzqOyDt7H*yf6Jn*#$;L1|Sdph@(Y zuICr-D$0T0>%4_b#3OIMyJ9ec7O%+nFL?K?ecWD2VqeK$!{HaILHZFN9JIB9FJ!Nl9XZAzrbdLQD^)}nz2e5Q}Cku2tyqLjf(z$R< zNgr5qXfiDLz`%KQIrYy@Wso%iS9_YF+W1&P`A`XOz zzY`Stl`z}s4ys8b7ojTUME7u2dXQ_IWMM)$&KbEFrf~C+J}YuIGBsG(ekB#Xtue@Q zUZle&%EhWKV8nM7&AR(0g-sk}@U51qwx8iAkZk?XzGwGOId#l<8BC)_nbwf73$Cru z)3{kq-QaY^fEWgT1Ez-@!bSo)io`wG6lcyEADvwR?>$JVEHouI%ziA#!N%ISrtMg{ zl!58t8c(JrYNJHz(3GU<;1DYHMD&^K>5)082Kl;}KjJTi;TAbk-7w9C?LfUY{J*VmnP>7Bzzi;?)0Rj-6~QlEyTQ+`|PG z7iH>aZ1inw!59&4k?jP-IFmd$n6jNCM-~%EMk7c@$c2T_`(Yx%@|Px;!N4^PA`SPQ zf#E=)MABIeD%Ic>sy)tsq&}uZqERZgv$Co@E7RW{3_Z9}&QJ*OZo(IPK4sYru+<@e z&y}e!Om_tl7gBp9%imEZiM9W9-~0~d^OQ5`g@q*29{(o9pH@ zJ5c-ZHIm$AmKTQCQK5G`+3EYyZ!bC?;5k`Gg+MV()Dvxh0TG3N*y6*tX9-LgOpW3pk zY>&oOnw{{`Lr6J#ybTNaSk1f5g58UCAOnW;3-{{+_J}&Va)v2POB1L~!fdl}#EZKI z54uW&g%D*43z4G2&_w_Cf(r7pEh?V#E@WamX^LMqzf5Q_x-D2{g{zL2O52bPhNigv zX_dvm-ek%hbM>m9HHt2G-Rzy#vjU50>2AGdJVARjOb0IN7`hkQw4oSGAltVd#S^%l zBuNouWBPm~c8b158+5kqQm7k+MENh|C%u!JR`Rk1wfyC|-^%`YjtOM$g!knM(G@Ai$Mp3f4Ue!&tNtPI{9tj`8TOx4 zD19@y!0qK(3g%A0)7TsE#i|TB4t`jGvrq9Q0R?00r3IQ)vXzO22pV3LBu=wR5!uvW zqdu#?W9aiszqb#XIHa0LxR|XuHKdu0mCtNhK`-H_9XFaRWnk+^Gn^|a{k&=O5M(gb zLu}Yl0XW=47>4`PH;TQ;EJo>L^p-+N))mlA=+xT&gf{Qwd}Z=z9EDam4hS|CqxrZX z+eWh=M^-dov>UgxD1V|ltCy;A2F`r9G-`bNUCGv} zG@1%5%eCE)mSk8Op?bTZd8(xaAVWvLd~|^e)P3A7wBC?df}s$z$22#6u0GI?%4kb* zx)XRPcO{F+KCcPx=v*%rU3Aq~H7fQ)eOu1%r9VA(dr;$1`1Wj*hl7!NYY^$zLb3|5 zsaNq5uD!WA1S3Nx*LRy(7J0-bdX#R>6J|k$Y>1aG(EonQuh7K!8m}L|uw9G=j6`76 z(+`u=h5CgN6uwY(b8{>Uqt*KM|>thOR#trNn`ON9jdY zAdi2FiW&=des(eDmBGy)8lQnRxy@iCJkmv{T2kAqFLHh86(g!u^9_-2$TOd%n1=6x zIgLIQOT7{=psw*Fz-J$HKc09|{q0|U@*%5JGm1U~I(gw!r;UB*@<7=nOz(tHM3#uq zI{K8{_^`M6Yc4N<4*j>w+CMT6B1W3uh_4zXIY{K5Q_RrCWwi-LZ}dpPrcO&-`5aFB z4Ird01+-9r0emF&4)Ke~d>cZ(ZnS~EOmv&1{a*XugbMU2ZMGPC{04~kkT zLQI@$pvO?iP3bPM?ciA7o9VBmJ;P)Tju$Wc=+%z5o6IHcs+!OAOU?TaAb=L=l+5Vc zNtY}&-@ZpN1rJTbaJ&{~sa+pCVxQ57W2_khmkULsjoHdctiJVk>MK;;a)WVR(7za5c4v~(|hOhhu z=Mxv~UB)UQi#_vQ=V$0c?@ORmVhPBQw*K(Q%jSMrrF;GTD$0TDnquI3R!b)xqJ!)lFBW92_j}i4WJUE1Ci;i2YrHj^P>w#AXa-& z%}dC95z=!lX=A^GH067=s1JG5cPG4iTTNOiuBpST94B`^TjOO(vhJ-CxGqdFJb68Q zKXj}{B!S5uRAHtDt6C39@#f3@1Sa#$Fq4M$4&jXQmQZenZKi*N(8I?oU!KEb ztu|hByLy>ub2lX8DriefyVC0&Sz0l&Z_S=7-S+~?DGTUp&wG6#SF&a4rzK!RbD6iz zL~EDu4#&kF%=emiYX`Np-Z#Zl+K3Z!wWvHhw!awirymsm!t$ucpYSURXMgz^O*`@p z_|^8vw)2T739!7I(B9|kr1EFnU4I-GUz7Q9I;jD=T;a>y{n+diFYpKQ>7n!M;%WZ+ zwmn+E^qxz`ZbwU2A>A@25iKZF^H@{S7W6ip;Rp7dZ$H%-dp@pc64!6RW@=A!v~m=} z^GZ`>jp1YsSsSEI77vL#zEDu>XDG+eeCZn?IrKQyi!8V3pdmuAvcl-QE9Vi}jLh$& z8FlbHPkmvuB$xPgM+qX-NZFcAIHEbIbo7obsuURb7Lsk%o-BiZ$AYe3J43F zKM}Hmr(+1tYOf(2^tVnuk7=)})BHB)yaxZJ`dbFaQD|{12^rEKUg^EmNLR{8q{+OH`KqPFYhE3RR@WT0^HYIQ>F=9= zREL9z7~Z?07PsCKnOh?r%ah7OCL&R(oe^cJe3(1w|KOkuMQCW<_;OX1SEOto(dM9? zd4jg3z3yBk)m;6X`c|b0L6<#r%pKVe&+PXnDOVXtqrA4VQXsMF+noX{l;q9JlGAM- zW$Tx&^_PpPQ#$&T6nEgy`BR(c1wBdDV44T8frQ?C51(954NgutNSBtW%9iR`n9-bG zSR04*C|0@TE79vmgJ&Nl0LTZQQ%e-5H!+9jj(Gd{s;5#cU@D~tIPwhl5j$^# z>^tXgMP}YL5&Nd8%6TY{0hUl)HRSM?(eMZGRnKFg$CuHrYS)li`3H6*Ch2$;(QAEa z*QZSX@>--4&$D|TsFIhia;ss-bsnwOifJBa5`QD{*mIZ$x~?vH4C1BcBiwV+ujS(q zNX5(=1@dzRx_!3SI&a&b=svqpHQe55{L|P!90n3$Xv;9wtegzpF=_11GyCeLF zl5tVGZy4%CppY6S%n!jcgUhxTfpSXVMN9AT6rPfGpy#H2xEg_2il>c9`s<34XQ zvwJ2dMD59Wu~s zB@=-$btAGSry=0v2%-6iVJuzSZCK1Y&E+~~RQC8HWo7EL(ee5lYBOw6(}bX8DL|4z zL37EE9V!cs_as-sb&_N=)l3f4u{iRSknps*38r(?HyV$$jp^ZT6UzCXJwq}BMTXnZF$y9g=f~Au?ri?^po!H*y zodgy6t=2mZxHO)){PgZvx>TWU_p@#CxqMBszT|#J^V`NI62`mr_&YRC)kVLF9e(O< zEeCtS+dk08!X5BP_t$LT+l8-E>@eW! zlfA)lkQ^QNjdfw%+195ZK>q+~)>Ag{qLF!4|AJTMEvcxg_h)5*#MSmQV9*(78%D`u77E7RjivfX z?X6E4?WG*o>$?&k24L8eYJGL^F}ZGbA*yqK54SyIu~Gs{hSG8RxeTd%=3%--+p9Y4i-!_LHZ{&*&!;gsX@Jg5`krwZ8ieS3fp-9npTa7++A_^cIenn`Cv&gWdOX>1(z?pyKb;KQWtOG5jZZ;2{u6%it zdxlr?@@{yN)$f&A#J6aiFNAGwlME>U*X(Z}T(=7ECl;VT4%G;uvM|CoD1iY!uZ|{y zn6jK?N@yOQ@r&_`IE93}De>DDXK%6xx(jZp*95mh17`*H^jNH;E3x9EMNl=?aq z`nC%8kD=}uTANRDNtXVGyuPiC^47bhCpza>caVQ*G$!W(Nq|$KQ|N}Gf1Cq;H$ce0QhOj?=RN0!o|e-)f6kQ|&_F6N_PqEIzK+cFp?A+ev*g{+aNN%`R%1)5 zN%`{Wyme9{J`|;6m?V!u1M5|Uj#}I-amSN<|GpJA;&5&IKE+j(P;8brRrz;R2b)4* z%6%?_G-=As5vR6Wnp=qFz)@JYQpu@%7<1-oIGfQ45KeX0sEffh3mDYk>lg5)@df& zNDp~^iaY#jHw(c@;R(idNWvz~0vak$q5|p2Oq5ta{7 zE#n%KLol(o78jFA%##aO17cDxeP|{o5~Q&Hx567n$5r-u+b!hC++yy*k3R3+N4c$CtN2^O@^F%*B@u!Ia%@GwsQ@PhldosFqIu) zywi`He$pG?%sMD6Biju@Xa9%zI;>0w|Xb%%a~?i$aM0Tw^+f`{bK@q zb(0m?t~64#XN=cV4w&yd0Rl3mI^D@IPC=R(M6m4G;t2|!_L8Bg3+j@v`_Md zqU7bRQKH7ET1M1Wkds z$9`&NiAWuPpjZqwlC_q#EVY4g{LC9%+) zPE$b6e!BJqd$|=8ieyMV>`sZP)A(ShNo_5J#3W1)k6Bj95CDN@A`-#$tP0(a57#3p zg-XasRxR7(gf(RoUnLvjm{Q-cL>CTxZR50&)BbkOZ|s3H_y9*2`kLXE%(08$W4JZ4 z5bM+O*?*pcc+v6Z{mQfhMJttThhNgX>ej~B~ z#C`18he>{HtHv}Ra_GE1;#MGKJHYtDMVnzXKl>JMtDqt^SAlz<%h1DyCu>XsLzd2d z-iXmpuL57PL>|Ey>h8vy>Dox}lQ?SYc*JYNEt{D=DrE4O+%iNuAyZg-qPe#@TIh`Esk~|E66}e4$5iV<(Y2mb@Dkn3#$uUo@&rlp`#W=o+Mc|)jNa){>k|wishj1P^##8cVI@wP4#Rt~8!?~nfUj^x__CxJ9f`db zrnbvtIMK3XY}%3U?Yj26%Iv`vbuqZ}>I3pv!FB?5?s)l?ki(KT9YBwGm8pgj4oy@BIaeZf-xz! zLQQ!$gsSUl0D$_JmwL^O0n^fLJ&ipMV))s;Vo=K|~ z)ZEMB9N`m4(lzWhcbEFAVO0kGapmggI5&o%c^jc@5h4pMs zCkKNl3@thFDDx39QQ*lnGP%PXJBKmpRr}2m;NT7QMHH>CF@}4+oGi~nbIV`EiYaM- zWk9dT7`(5}8XHYuSoSY$N6(5?1qNZ;juD@ktmfNc>n1v2WK=heB*0izZHYuwCcQZA zlK$o|tUX%aQ`F+PI1#f3(dsb^(_d*(tg*kdIB7X|0~1Mxm>qa2doSgl=E|!%7;gBu z3*Nauhg*JabJ1#=O<2T_Tpw_r>C+gbI^lwHA#ODZrS{qBU(w#@br74L3gzno^Zv5x zbJ#e#@tbvGMz@3P;LvS9(7j}?!`a29^B!sap!D#(w-Z`=C-EKXLc(sgH~k^R;Uhd}utD2*uPvPG2@Kg*_N9>D z{T>Exr97l*qiMCw!1yuy-hEQGQB16Z$YJeaK*f!J0&%C~=<&QnOr#;Wa0Z(w86nC< z1J2TgC3AD7itgmpmof86c((O1=WD;zm>zhmpZU&E1T2|ZKw?q1(WkFJ2MsN9L3<_F zaZ!H5i-h-&W>+NbgLGjCPS`+zIJj{r^(85bymEJsek-l0ARdT|>V$ zX?KzijN*VD0!ovO;!&II-(_~*+I%uVdKq6+j^O{It+<(DD4Uq&uJxN5%aQ4Q8Rdz1 zQBloadE4BObo;+USq4@$D)2EE+56vw@UpG+_e1&NXR>+di$hZC?I%bPnx(=I@(<=z z4Q+Mv1jC8%(nOSCJWmLw2Z5EATpsRqnOh>(BXAk9br0_t?jaQ@MT^r}ltwK5!sgs? z6@M1a68KD+n}YjD5|$MXCCbuo7^Ephs100f^*#d75SNW!y-YC5tO++2Qij%*-{2Xk zxN?Z$4Hn*;@X440spY-|^3wsHKdwaRqUfx$8P^k^HP}Gk>fYhh)}#&Mqy?{L7c(oy z)#A?0vF%N5b?O$tx~*dVjNY`>zi8eGmOo>deQkmPp_}1iSav}%Sm=W6Qk|wNf>ks( z=lP+QmBJbjIWK7zzu1C&td}Ip4nl_PFU!iNdPPin>4O7HlgwVaWsGjBLvy(qka%$atjNbSuMJsP3a*r)HB2t10*ipw~k=uEIc8A?m zs)wyz`}ufvYwY;$@+A>NCv4pq)Jnqs=g`A-!b|NJ9r426rX1s^;HpR8Nn_B?%1gzI{*A4L!mM@n%AZlJyT%}2 zJ7fX+;J)MEn|4n%(}wN;&dbeZ`bC2KiX^^;lJu>zWwt@Pwa~UNtG9YzHJSk#r)-Yy zuuvKV?j4POxQ5iixjZtg+G~;8DLgRc91QzUu6#Pw9vF?GI*y_H>T6x!Zx{zE!OVE5 z0Bu`;E$i&s#&6m-X(sjgHOXcN#yrB0FC-e41*x?p)2AZ1y6BbauN?$*bkXC}kzw{Wjg-;3H%+wFZ7Ltaz~+C>Cz&_*1HamZIv_ztIIB!-HDBKm22b6Ki1!nOOapz?lH~#EXryiw zg3atZ$I^+xcD94AMByg!3A;qX)KHT)bGIt&vpB1#&W;A{U!ZJ~SN3$3qh2X9jR2VGc+lIy!FVbd}p%cQRM&bAW^9CfIFJWI zvrnV;CUy;ZL>bat)TuC1y9ipBep%qvYqzO1CN(1BByOf8L$A2NS<%S&VPpQQZZ}UE zUJVG(!n9F2%M^Yqei(tT`)ZQkiu7v--=sQHLEzv4po66tTb?GOPmv+cbPq)89taMd zLM}2*gY@#v(RGP*Wq-^(0l7${!F}#P1(_S7-`ydMG|eX`Z+ul9)?n2Ku;#bwVw&UT zKz3o9Ro}&}So2_&*h? z>gBo4#m{v`?1Smd)+V^SvUDMoE|9`rXA_0}^Vj?6kH$`NrgV`1M)$5Bx=~!AO2V|( z^xfPmY`qD7AFE9B8CzCg>Z@w@8hF{`Z_d5^D0D9%?A)gFy*-oAH;6=wr%0TVBLlx+ zcWf2&G9j6nw6stk@W%9|-kT8HQ1Ud(#5-`M$dnRf{>&7=h%e#FNEHlr)z0ch1(a{1#bz z7A6ZPpP3|$?yurBHD|YG4+5^}LivcC4_1{x|2PP1q`W5YoAGmoBW?yx4`08{UV$)0 zFmPNGlK%MKJ`bls-mg5lxFvls{0KmP!@CHWp_v0Q0E*<4OzsD7CNtLq?jMe<=~ zLcCi0Dkt%gm{8zt)qaB44Fi=+Wg1zWl?Lfm*w6e^Fc>0u>ipqi)ox?HF~ADYu4xU& zU6-z@i=Gp5f-kGU7i*l+LVY(bP{ratqW-Kobd6LiRU&Svkgv8dXeP%+NlK?VZj$cF zrEJN=dT|e1yV`h;K_ciUy2$zt8-XYAE&mowc5IKWV29$mE~6M84h5EVLUp}iMG;h* zgkVX)ZKTjW(^iG^&O&<}^Q4`-q{4 zNA@P@VsGw^=1IaVElqPx@AbS11^WQZz0KV9I)oydzw?O#8bV%2Bs&Q}j{Nd>B4A>; z1Kz&~24pNsml6nO0_C4A{b`_g)7j&rM7Ns)V{Zg-o1w(<=| zB5C0~w5#t|i|hb)eZ7W?{$Ph?L12xDl$Jq@q;rgHmqb0Lp|Pag~o&J#I2{o zf2yf5{Pw%iIaX$}2ZzNi<{%fk+VG7s{+Jbn+~yWm^tHjtpd-P^T8wBqHpx)8(F=G4 z#oN}$CuwJmi8(kUX-~;HTK=DiytJ3jR_>9+Jg!;jCho{6%+`VS*RD3G*wMqG;THVf zxiKuH3YNrqAHga2Rn$e%Kdx=xU4e-|rirar9DPUoKV2wD>ibiq#R_u?FUyY+vkJj> z744dTE7A}jOPQ)tP{)&s);2PAUqgX}?>c6%4qT?CQ0N#Lnw<|3 zYt%oup-m8NWavOUmb|m9(G8h67G$UD`{nS5XWe4HiArOAtz{5)QCuL6R;U}?is!rZ zPFYtPo$s90cqvsX7@#qQ>v1a=As5XNe^UGUmQchY1=anuyvvQqMs@PTIX`B7Not;7 zWgdF_)PczD{^c|*Xm^Oa!%16IY9a=n<=`=0{Q!x~jC8UD$ZW1{znl7x6MB2;&#N?j z9wJ-f7-Gv04d?&i5me)&;Db*q6YZO@i=e8(AglzfTPG0vqYC&!g-@oaLq z)ageF#sj6zJoFEtJyeBht(}O%<4SXRGV36-jqeB(a!MF+&*wVe3Pa|~gjD@4ieAZ6 z>eL6n-q76Rpn6pM>2s!L@N@;)nrXsn)PRjzNl{IU$O@tfYmF;#^VrYENmFHX7;RX0 z5jww09Wx)HtJ95{+NiWr7Xi*EivA0ZOJJMX;5TcF$8HtZ58X2xE)-q4`rcT*rO$94$y| z+ilL4LVb>Pxj;(XpxO@CUki)S?aH{ct!Y;<3#Py@g>aXZQ0Hw)P4oW{^SJV5bvX4| z^enNpx_Uube@v;iPUBJ83j=3)I&&%^F_^%-{G-N*3TzKW==RM9jh_3);C4+%^-U*J zh3(2Wk>cPiFW08hLw6Tel&6qyrgdS{^lrprTduYHyn7JB1t%ifA3eFTE{0V>oNp1x zw~-;3S9)Twv&>y;oA`LNimF?}mlKBOY>KH7ZSk^IJ?^a1zt*tJb({t1K)b$v>q#L4 zTKjQ$hY_F<+GV`)cl#`kS7&exOi{)$tQJ&%jxj4otzhE=72_jRrr1Y? zp`X%4Ci&aj9ba7WQ)+UEeSrO#I9=rsE2YV~3o}3^h?pT&#wIy&@@{TT%B+ zT}6`utYh_sOT*dSix{YRQe?NHk6ct-p1Mthp899lRvqaXi_L;=?XIL@4X#QRI7aDZ z8GjZfGzp^KBsNT)9bx1O*+M=i9O9M>D!scq_FlQ<4qU$A#XBzOi(3AP-p}UKmMIt% zTr3B@!8H?xfjbu|*7tdG`if!*zPhPQG{aKi3fI)M_e{C(SW6l3j7-oVZXA3&`dXjW zY%_E}=P}`Kbr&mqXK8y1U(uJoaI0LF{p?=cskt$D8*Z92gb`jb=5PG#!%H9>(>{)! zJc5xd!f&l?&nJf>%dZi|l(7_F`8n5>bno;{_Xd3*yfw#h*4;^Z3Brv9!XLL0KjRLs zVy8nN=h@+qWET=e7dvVg^(vmoIU7>O`t=uvsvo$~v^??(p30lpBl(r^v*n~=i|k~l43tfd<-n)k*=RXbB1uC`Pr!+&I0{1cgU|%a zK#-JZmwPi_{)B}=0HcQoK@)o{uuE-HsbQE=ZC|K}nhqP;*sdft zHorFzyQPwDWZFvRb$*(hZb$#_9$DU*lLz@KQ$jnH zniac2TKSq03a5P|V)s~8J|Tf#KJjp1Dcw{4Q%FT=kjg(@wKL9Le(EyvFIL!gPq&o} zXO4aJzA%ukUp^YR(W~`!N8sbC8o~j6-fI=mzi=VMTK}|o{{H}XK-0Aa(r5a6NJjq zjxNC^>QB91w{okr4e9ECKaB%F)M4VdFCW*wL@j^X^gVk_nfI`>q%nk1R~6jYP+=RN z*%FQ>)Vb~}jz!Hnc`sx+&0tf^elwCj|L<;NCa6G)6l}cbxW~ki)T}P-*B-Zf1A6u@ zG+Kjm`{Du@CM*vTHGg{!AQOZT{$hQftDzgqOl;?`=^zIyqyQ2@Z*-k+Y7w5BLR9hw z{x&m{x||M3A`z!noA~4Q>g)Q6EZ34^i&Gk_R3Q4@u&tAjLcu0``$DjSJ!?*{##Fl5 z2)%*BF~&0?FsV!Ov>LvZT~gIL;7^zceTiIc?&o1|H9&-3oM{jLa??V^zVB$XRDwoo zjIO6rAZM;+OR+y6%SQb6%Mc9)af9TEiwH*+j0jH3F|ni_U%vQJ-XLQeqbMx-${~qN z!^UlLbe9V1Xp@OLkv*yaIf~H_C5a5k$RHq{uWF4Q$V9RtXs8cbXe0>Y;P@0X86+I8 zc|bUA(UYBBRtrTsaVPG7PX4T@bUgjaDG7$ysMw_^Ue{LtR2hY&hOaBN>P@fRv z-nvobt~C4RGQ-ZM8;}3wK%0d+j5F&^`p{-OZVI>PVQgd=`B|f-DD&Q?&EAc3&|*#A z3L-ka=W-hk!87UU6iEM zM&tiX-q_+qGG+^aYq&x`r+sg=An*mZ)SsR2Th2WEO;Hg^yJ#8+Ey9#;ebj(rIBa`;>5om48)=haC-Rk~(!$a+1Gwq%UXF1wn;!VhLf`QUDp~aSn_+w! zVACtN9t7jN8#J^ra7oV`8u-iyZFhq-+m@@6n(aUFS(Z#_rQ{DVgmHXNN$wQe&fl7sHbye53O z%ja6o08ZZK2}m?_Zkr_}S~QpF*ilVJn>|U?qtY>If$yO*6!U(C5g50dM&rUbcwP92 z^pV+6l4<{~7+vzAxI;Ti?WOrUDrC9(j8Tsjp7M$yacnEY?0fxq&W(vAhpx!LTEf5Y{i&nzQOI&k z8EOymV>@Qckg;gCn}I~3#zO6snBEg3(hPG{lGeHU!=Y=9QeTlAGi`hH*rr-?_s1!^ z+Mj=oPb33~hr6C9N`Id^~fuGl%4_b(X>TD~<#cX@>;k-k2v%Vwj6!&~$ube!!oXOxX zhrxEG;4gSD-$L@g4Az!`bWdtp@Z3?KB`xyy-O&C?u;auMe-VSmqRD-Ej*O+_fV~ZJ zxlac#w@f$JJkW}WO`E0J_J;SMlQ|(afo01Y1({$;N$thD|}|- zIJ}>imOkT$u^T_Lihf8q&|HAFt6ASj#g4oJ#(_P(KtA+757cs5HE*#MHBF(M!eS`i z8=Z3kZ74s2x1lkEFmXu5ov9c5xbUq&&=9Nei6&QuX6&E-BeUK2nR|MoFR{^-cv*$; zmgTgGc_6U=8^ga?;ahG7m#F^ExC#u9^&Yw7mF7)hb{Cl8Y=}0D7`{P zQYrN2Lx;=K8RZelvi%>{O`LvB?JRQZs|N~lY6yEbviih)zZ^rAK8IRGa&M}^JA`jF z^4{|38vCg%Zd`O^t~wN&&-S4&D(0J}Ko3d-j*wWxyV@D7}y%?xhcf6;!F`!UV85DQi^6zE*g27_LBadt`1QWGO zih%^6U(ip)-C2@Lx7pvYCU!BR+Pt`H-ncmgH(HZ{;3nOqP7{KO~$HGmWIjN^Qw9vi&{ zpf=RU^g@%cvD=^-ZPcbEQt>hF5a67LKRL2~D75biWnbEI>8hR6LZVMTsxUMxrw%V` zJnEZ|6}Oy+HHRc2U)I7_3LdZ%`n9=HZyivJ^hr8F)ruY<`-GQ(&m2{JIZt8aaDja+ zh*EG0(s)a23D?EHdH0m!soZ66@xEIHgS4y+gy@6XW`qhFbPdDS06Dv z_THUtjllkb9@CD_691Xj(8GIDHpYXNb&oYpU0U9`YbkZl0cuc=_Hn0iqYJMNffY4f9KJN7ELG9^UHL@6PKnfs zT#J0jPD0GXWiVH`#Op^<+S6Sb?R|4eZb0_h`tJEkKQr@HycrNpXrhrdBBWG$8h^|> zmnxB_^4l9<&jTYJt&cIJS4AES+(!CuDhK-Eh;HAtfGYKcJKyW(xl#%Xbm&#Rys}sq zYi~{i+m*E_St@FH4y9A_;<@nE zm-*MaOgPq~Ue%tE?aNWeMEOIa)E82g1DZON3U_kr9z#lv z7V!5%sN%DNN)|`%f8QMghgw;bnzIyCyS5-v>_#b=BTP?PvZLG`T*z@I{YMNHh*OQc z>Fv~s#YrnspC93noEBI2;oLL)L%9Ku>s#?>+{jF% zi$|A*@MVO($uDHa{9%QefT2B#~un zv@PF^ZMw|&$KKGTA{F|S4`D3iz@FiZV|`f-3mZ19rsPJjIl2rNG+D`d;CI3{vEqbk z#gh}fHqWy(5$_<>+*)AM{H7!s_w3Z1kXW}#Y#J|mDI{yp^iab0(-jvF*$Oic+=L)z zDo%H!WY>BFBU3qs;V_c+@HRYISx^|)ywd@}E?JilAr_x_eI94!DR!`e>NR99oYsyK z>8#>D2~PdjpXh+T@o|*XN(6^xxz*JZO675lk?FU?ICqgVSZG>!HmHSihS(OxMbd2D zo9KyjgTmm&?n9;2e8gQYQIL5l)@5n4xi2qiO#88`npEIy)g1n%P=xBoVj4W9@z&P; zW{Zg=Q>*Kb{?EB}p6x|;PLJxeZldY=eiOdf>$`=fE5a&`Dm936u*GLk273 z4%nHjdV!IVeoqjB*|b3ZnUnf&r8u^SHxkU-v9ikA8Kq)(!!#)5ex@PhSe)_r$;t}XU81SXUc*V^sy|eX`#S7TQGx4d z%xW!Ev5Wq14lJ^fn1QI{)1==ew0S0+_vj{H$Ocka@m-2XCskn%wwJ_V6y>3WZ>Zef zO>+zVVjdpyAxz!Is1~yLhfxcU7&mQ=fBooRmufs2MM5-cWJ$)2xeriO41RMuSRPCV zPfyc=uRQS+>vuzlt8t)?-r>Co4WOpuMP>R3-t--ahnF|2Ze`)*aNHW{nL9j7tw1_* z2mUZ2v4kq5&Hqqm=RrNNY_yUn{Gud|1(5KJ9u*ga%7yv(=EeAcO8B6=cslbI1D2mR z?WZF&swGPo9c1C-?nP2UYm)qx><3-+zjYs5MBl|J_5TLGKyoZb8^tnz4lbSHt+HsZ-T z_q_uHl$psZRkNmJM*k=;zgx61P(jlYHwKRLGOyp*SDeI@v@7wG;!xzog&719*Hf%b z5{{Yi{Ty&H##QfOOp76)N~SH41Y@d)!GNXM9(YONc`PYhrkVkYj(nfq zf*J;Ol`#+QwjjBDVk>VT16y22Q?gE{7t(rWl z!?)_FLC-EnL?7E#%IiT-uyVEfYB%co!F{z*j4U_XoA71 zDXJxOvQgcRu#k{T2i)85^lji9{W#N7NT??+-;t#K6aO*U&6vz=)NkA)A6u#1A={h3 zFf4ri(=f7c;dN|rO5r4u>CX$#6=$@LfQaEqu@0HR4vzm?XM_EV?&_SM!nPRJr;h38 zn|lrI3(w8{`j>!+4xz?IHU?SnOir?X$|pjCbU{PQ-fOrh+6K zM+XdkyRuz;HY3R|S@I>zL-KRY91&Bvz4!`FjfNto{9JzEmaGw`ma2z079J5KPbZm- zJ{B?2W228H=V;8%N;D1xQe)e6@>Q0hW^=mN()(E5|eyNMFdw!?MbQ?*`j3}l7%ofQHZg}SmZ@hB^p{E_dA~? z(z8!bo!(;$kB~B3yTbZa>#L=sbL-s*p>-h1S!fpZ#j0u=UxzPq^V0TaR~crLKG*kd zu?DGS)Rul)0bXgTYgr!z7QbZgHa0>l>gCRFcRM{Mgam;|@F1-bc8H0p;pX3TV=)GE z!%rjTIY}_%2V>~3nbcBYA7zbtOXNpgqt`Vd(WPiHn)%&gEJmk=bG`k6Ch;2vJje*O zDo>$nK#nXV36OgLEUZB5(H!C6={3^K2SwtUoQBUf!)x`?&&T2nN_W&!QYnC~ZV~F` zH)8x(gUUX{kR~mppIn-%>CmIuG!v~xYKYU8orOP*1&0+&y|ngPTCqH2>Dp&@zp(lJ z)5`8+fD5i;`LPyhDl+C)$0s#zE!a{<(UhAvR0kBfh!(5J%3B2v7z>ndrfzqy3m><8EYVqW!kW2kAZ06n6c(#U( zPL{UCVYQ%~PzS3I^WPvE0W%}ybmP>BS;6ER zsv}2g2+x~7&?$%$l(paojU<7~k5K82<7y5qfld4)ow6ruBC*g|aOJ&mTx%5*kBjWu z_Dfs5*v|*OA^`FEE3k1BM2cVUMYJzO?)JwNO7vyckG1#k;hMrck`U{1=zUg%*^9Q( z!*2LLG(I@9(Y|bsl$;k)vJ?5tC@~6K#YlhR*IDJ|^9ntJg^rpfGzuoT^cxy)kELzl zl2)5rpMgVYcSdtOA0k}R(RF6CdPP?+*W{tU==oqI_KAYmzPOA|UHTyL92Ey`7%W25 zaWhW|VY#pEbEn^_txa>Hbl2P)cm2J5CUJfXh4I)=UD!A|WEvp~Fw2m88<(e);d#(o zBM>$OtD)>Zo1E+xG?A6rXnweJ)>V>Bo#8FckIowH-jDaij*W)tkX~VO40J)DKn*fH z@AzN&`Ei=885xk?PsadXQ5N)tPU;j@a!TA*B{xyiR9ykJfCKNHoEeRx06$26bA&n7 z1K8z!Q2P6C{tf0PwN~Y>!6mUOIYMsjx-dYkS12f5C~aSJLSp*WIaG#ktt0y3%~{*} zjN`vLZGOVbR$VQ)wHXKdJ+(Afg8@RJH93VZTZjlnTNObhp!BjD8*nnwwVN!^TR#Ql zWPOpE4MWuo=XtlYFaK}RDJqPD+?b4E!omEa2EV%01wCZZI9 zzDj?JPc_v#eLiR-F*A}OkInkXNpC``gwL@LDma$dwmvwokO}8b!SVwg;rCNV;vCm# zGdB!B3D%Cuq)a7Ivbjl_i`z{P!DAlqS!Zm%roD#NuwL7DA#j%4oF zJFIjf^XaIUU%~;oCR2&hUk~_~md*2$Snbs<hzJY6>pH{DN~&PAS~WmIP|G?}ToJs`#9xpxrtfz7AC) z=#}-U)7BO~@o|3v)x*wPAqP^sSzGzFBivP-zl=$iE>?zA5Ssw8xoReyz9jiE0a_n4 zA}iinsZfHo{OBXWe+MibyU7%cD3@l3EMGSv;%}(|s^9Rb+L=~9V^+HG{E2VV8 z#}pE3dVVXKJJ0{uhT`ffh0pYKIi@iaO2J4$SYQ|R+&`N-TR3ms!&y^=EZII=Bw- ziAj(hOw4eYjSx}77CL9Gz!y&Fww0N-9@cqPRfCd2v8b0pWl{3j9A}?W{$O|F2mb#s zA%;y!OGcXFvlLmg(EuMnj5Pp;R9If!cRX#~d-=Rs_Hg#j(XMm4WRcgi4x3w&80%4Z z7Hh!O1zywMQw_+PpRDkS-?z!p8T6am5I z`{X4}V^GI;DZ`|+iv#n>)3Ar^!q4&BDra5|RD>zzI#&n^vRhniz;-h+2GPkjKT94h zSN!qzDh7hg;q{4)cRUU=S$=qV5`@O6VUot)ugsr@OTO`4Ydb^ zJl|G%XZF-S#h&iD)s;d$ZKZHfHiR$?BxY)b5q6b$5%}6iFh7 z7Vahehp~g9#-ER{X_HE+wLFBbrZTKMa1a+=0D~wemXy`7I_r6x{3X0KB zw%MJKUsr>P5+)JGB5FWZw#@o^9!Sn$#&p7seH4%-wQ)@NlOHCRu<@;)}9 zD>)n=>SKgNo$3N;P|jvIqhz`6WF+k~n2wTnVtand{OVKH({X9y0T5QE_de6BMB140 z=%Q3E)W2Hsqj;D<;$=yA!6mWi?T=)K?nPH~zrtsz8XWW8UMQoQTAJxz)E=C&yI;t+bX%Id~;Puke)3np(!;^s~vTbN*cpUIh=S) zJy3$cMg%PZl_+2k#vr63Kgc64+gpOAm}fCbdUDqTE$kq~n{RNIlq3IU*zRCKC3)yf zq*dfN6D=^`P|hq4XqQhYCbK-9;0O^(A5#q;CywkO<8llD^fI%8fT6CL^E>b}m>i6@ zVf$*9+n^4h53<`?=6F4UGs}X+%f{FJc*rKFXzX;6taEs2I<+MW1?T4zghk%RWw2)t z|39{kMG4o!+Ozi97XI{&n`cH0LskpM{uzx5+owf9kZ+{UX0U_4m$^<*PG0|g7fbBT zyN)9z@$VZ8HhxM27J1#X{w$65RRS9mPUQvXWuZQ4FX{hljs#3FO=goSW#KKQlb0rm z+G3Lm+bIhW?``ARxlaLlA1)#XWt+ibTgW9b1#Qylgbx zMaC@Z$(v2S0pW?)n;K@%e9_jDToLiE<|}*aYidk#Kf>90pJ{R?Lz?Al3c^pm=EaqC z(XT@Q-N%J^v{OnCj3I_(b0A+q<_u9&+Nh(`$Ajgyp}mC{MngK%+4VB6 zk-F({(~4^Jx+Kl(4s*3`X|%E+$pT6Obb>j+8xs2rsHGS0+!c$*we{<}E40;Xm5Z}Y zii98CS|12;^P0D;ip8fHy#!_rkNi^ixIvrY1 zYIAK>q9@c*M{a^NOdr^bWM;Y9IX$kG7ivCMt*M^oj>f+Ub$So-gLv}`OEW$&XRuT1 z@erS%E{z%$9iNd#uvrCf*C~Jbezs) zrT&rD{W_7;>L!PT{APH9x1(a7*=3_pCne1(c`MLQn}6o$2E>}-Y=Cw!x0^6-urL^+ z2z-NgVCOyv-$qfPn>dgK4s=>>NJg=aY%=5B*nJY7Bsj98$`Ues-wsijf%eT#UcL1n z7RV=PBfC9^)s*Fjx|u8zQB^f3>{b9+X6`{nv72aX4=?rVt#(iLEB`KgGVil2-e3MK z;mHRdRA=JUB=vjnN5l2_gTd&;T(xX#ig4tt&)Db_61jZCtzCIP?x;?YM>!{kLnX(v zAg>*MA8!6^afkVCtRsU+gB1FixONVlf%OW^tnD6ZBE+4D1006!Aqz6{rq(iK*oa=l zqBdK-^nO7{b$JIW-KEa5`~?{~L5W`(=iz+jyI}slsio*=a9WlKP;Ffx?8mHRR$Y?e z$Zg$K*=V_D$%m=W$6y4ppU;cd!hD|k8X2OmjvNJOj? zZu45ALjxhBb0p4J>-U6Za~Qve3X@>ZtQuCTltI)5MZeq_#z`nbIf6@X!GVfb^RkSN ztRJtrYJ1>W)H1)a!3A!?n9e}0RPv&e?lM>&3PBV+7*{2Yf+ZV@IOZ?~q`|WGyN~md z^+>*BOn5b|g;-$Gco2ykSk8U)H2=wvYJ!!f6Hh-dECAUu#=DxpJfI zu@Zq6(>M31u>O*w#pxKM?y@AW`xgR`SE?*+>@xYh1hbI7Ks(YHoy`(`cpZuqTkFkk z;}E2aa!8rGHP}YySC^q2g!K@f_0X-}P=f1Gr@acMlTglu#K99;G{t=AmeXfc7Pb7& z`fIeUCwk;X@zu!)tk#VKp_ve@)e^3R1G>msMVXwh3_;DHP*jT~#z^M-wf>zlDl2N< z^Dn2uZXS5O1<{nA%fy5LS3s!0LEoR<1`&~e zZ8lcpPAu4@LfQWRJe~h(&_1aTrFr$Y_RO=2!WBk!+vBb;9Sa6+@xA08{?bL(#mjv4 z;;gbW?g(U)`37#H1fnE_CY!F3xG=hx<@`#<7%PGNvzeYlG>6y9c9JWG85J)qq_kQ~ zSSuWemDr;8M~l~5D9i`jE&qm{^a}1ER1Nb6!kO#Pgaub4S|f{R6sWe} z8^x-z=syKQ#qJ&0Gio>hV67cqJ<*Oy4s}CVome1_{K1kTtM=g4&_om8%omk-Nd|&= zJ?Y+HAsuwlYsTsxU>Qqs}jE<~_VXc3nat2o9Uo#caJ3^(#bJ+v9w zP5CzkxuH+BoZWQ0fsM7Wwy42alg(y(H^qT!yfld4>-lNw%1a>S)5dv^zOmj=aKFJO zJAxLcaidg$o_hZe*$jvFy8maN@iuHIt<#RaCm~&xoWM_K55>sW5Bb-J`;}!YXC#y@SFcYm;wf(pb(Yrd_yf8-cfeaLW@J+UsK* z=5FVw?zA`ZIt~L|n8b&O)w#3#g!L(XQiegzlD-?f{E1JcP3WB1%xQnh7>5Ws(pGGWp1Va)ruoLUQQtYCfy!J!V@)gOb0}`_{i-xL0t9!@^W24 zTXNY9Ec$|QWF^(4rg5}WB6Da<1TS1g0UeRCqpn2B6fN_4ku#vY7tV9o2c$g@A=#ZX zw2aD7A#Q@YTL$lFMWu{ckF!vQH3EhBUeE$V@=~`;0OyaXsPTSP&V-YIrl-R0OkIwG#ICs%bDpF44@%2ET zP`3>tOTg~kGB0I1wM$a_B2;;&@H+ciD5N(=I0jVx^xlWCiI! zUAq_d5F^}=71aTOFPp2wV_v<+u;Y#R9a$7t17PFfOYI&cV3B#>W}%=^>ZN^GLcJLQ z%GC-Y%%yxMdO;GcuJq@$58}gf!BjxHyp0~Mu^x8YbdUhT0YGg%y|#Qf806fX39cX^K90E?NgN#gV2 zNx!Zcj#7`=ag@**iXmJbVTv~uifF#!Q7NO3*e%uelSj5yo`?^ulA3y5Luy{En-Zd- ztb3;`*xd^f9_-yL_y@SOsxt|}u2gDh#zPu(atcB<{TcHP%c3hz8{E{bc{~PsYX6T^ zJ=EL#Z&q_Ob@Ql(VI5mI;{H?c*A8yri^n<(2WP_{(6+H2A^a5t%S85>uA041rw;3TPlii8s=Z0qv{mmc=GGKgeiTZQK3IXP{HiHFJI>*+ zez0k3)?*mr1XoCXpnAD$jv^6e6zfphXKz%n_E{M}*e2c`?=LXi{&;Dol1Eu`aKxGW zR_uR+n+rgHRp=ZlN0x(-MS(*dMEML-;jjAuKj`Eg@W;&?%rNw{h+je@J*C+Wl^a#|K#XkJ(Q{C!V=;C+B-Nl8b)Ys&@ zB6pN3Vf*kJwaFev>FLf!b`rr2Dlf4k2EYD@IyQ{(o$ktdWVZNx;Cm*U4Jplg zYX@YMASKZ9|2=Ujjx^3L}ab*+KUxfdE_X>k8`qQ_LbfE-W`T?3j+ zg>?-21+q~+peD?+g+dZ4=IdmSEYrj<(z3uk5-i7#wf@6P#5+f%MwM-2O8rcXNcbm{P z2LJz;m0qoqYuCc zTQoH3I+E57C_2GJfmpc3Wk=9{SKEe%+k7TaR=uC=vvBZw&tTt}KI?b+w=F+*q- zP?6~=^!;cz#H=o%x~OC*vi1yVd31R-V`r!o)SQNwzokl{wm$_<^2xarwQ;|}Ohc1| zLK9!LOW7J;8|6iWf4w7Gr$NyCD|WZsf1UCEbaIKzUSHCa3n!&PZtAdjgV3UJXSzF` z&aniU%$I40imi%BwmbNI6z|ORbMk$f^PsDkyN@WA_O!T|P$MF*Jh&Ary1Jma%+@JH z3r6C&t$ix#eH?n@sO&TE#unfyG8 z2^`YlGOKOf4^n5BuA^pwSye}JnlvZj;1q7$V_i1yN!18y?`geQq@-kLubqwz?t