From 8ba0bed380c5a5a4cff014249dd14b980f976172 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Mon, 17 Oct 2022 12:26:40 +0800 Subject: [PATCH] Revert "[CH-123] Support short/byte/binary/decimal/array/map/struct (#128)" (#159) This reverts commit 729900c706c0e8838c5b84f8fd8f5c75769ddda5. --- CMakeLists.txt | 3 - .../Builder/SerializedPlanBuilder.cpp | 108 --- .../Builder/SerializedPlanBuilder.h | 3 - utils/local-engine/CMakeLists.txt | 1 - utils/local-engine/Common/DebugUtils.cpp | 114 ++- .../Parser/CHColumnToSparkRow.cpp | 891 ++++-------------- .../local-engine/Parser/CHColumnToSparkRow.h | 145 +-- .../Parser/SerializedPlanParser.cpp | 706 ++++++++------ .../Parser/SerializedPlanParser.h | 7 +- .../Parser/SparkRowToCHColumn.cpp | 421 ++------- .../local-engine/Parser/SparkRowToCHColumn.h | 357 +++---- utils/local-engine/jni/jni_common.cpp | 8 - utils/local-engine/jni/jni_common.h | 3 - utils/local-engine/local_engine_jni.cpp | 114 ++- utils/local-engine/tests/CMakeLists.txt | 2 +- .../tests/benchmark_local_engine.cpp | 6 +- .../tests/benchmark_spark_row.cpp | 121 --- utils/local-engine/tests/data/array.parquet | Bin 292626 -> 0 bytes utils/local-engine/tests/data/decimal.parquet | Bin 190487 -> 0 bytes utils/local-engine/tests/data/map.parquet | Bin 13869 -> 0 bytes utils/local-engine/tests/data/struct.parquet | Bin 11153 -> 0 bytes .../local-engine/tests/gtest_local_engine.cpp | 28 +- utils/local-engine/tests/gtest_spark_row.cpp | 443 --------- 23 files changed, 1077 insertions(+), 2404 deletions(-) delete mode 100644 utils/local-engine/tests/benchmark_spark_row.cpp delete mode 100644 utils/local-engine/tests/data/array.parquet delete mode 100644 utils/local-engine/tests/data/decimal.parquet delete mode 100644 utils/local-engine/tests/data/map.parquet delete mode 100644 utils/local-engine/tests/data/struct.parquet delete mode 100644 utils/local-engine/tests/gtest_spark_row.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index 3157d3a1ec49..04fa1c485801 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -187,9 +187,6 @@ if (COMPILER_CLANG) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") endif() endif() - - # set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-omit-frame-pointer -mno-omit-leaf-frame-pointer -fno-optimize-sibling-calls") - # set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-omit-frame-pointer -mno-omit-leaf-frame-pointer -fno-optimize-sibling-calls") endif () # If compiler has support for -Wreserved-identifier. It is difficult to detect by clang version, diff --git a/utils/local-engine/Builder/SerializedPlanBuilder.cpp b/utils/local-engine/Builder/SerializedPlanBuilder.cpp index b51a4a810555..7b2f8c721f55 100644 --- a/utils/local-engine/Builder/SerializedPlanBuilder.cpp +++ b/utils/local-engine/Builder/SerializedPlanBuilder.cpp @@ -1,24 +1,7 @@ #include "SerializedPlanBuilder.h" -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; -} -} namespace dbms { - -using namespace DB; SchemaPtr SerializedSchemaBuilder::build() { for (const auto & [name, type] : this->type_map) @@ -188,11 +171,9 @@ std::unique_ptr SerializedPlanBuilder::build() { return std::move(this->plan); } - SerializedPlanBuilder::SerializedPlanBuilder() : plan(std::make_unique()) { } - SerializedPlanBuilder & SerializedPlanBuilder::aggregate(std::vector /*keys*/, std::vector aggregates) { substrait::Rel * rel = new substrait::Rel(); @@ -207,7 +188,6 @@ SerializedPlanBuilder & SerializedPlanBuilder::aggregate(std::vector / this->prev_rel = rel; return *this; } - SerializedPlanBuilder & SerializedPlanBuilder::project(std::vector projections) { substrait::Rel * project = new substrait::Rel(); @@ -220,94 +200,6 @@ SerializedPlanBuilder & SerializedPlanBuilder::project(std::vector SerializedPlanBuilder::buildType(const DB::DataTypePtr & ch_type) -{ - const auto * ch_type_nullable = checkAndGetDataType(ch_type.get()); - const bool is_nullable = (ch_type_nullable != nullptr); - auto type_nullability - = is_nullable ? substrait::Type_Nullability_NULLABILITY_NULLABLE : substrait::Type_Nullability_NULLABILITY_REQUIRED; - - const auto ch_type_without_nullable = DB::removeNullable(ch_type); - const DB::WhichDataType which(ch_type_without_nullable); - - auto res = std::make_shared(); - if (which.isUInt8()) - res->mutable_bool_()->set_nullability(type_nullability); - else if (which.isInt8()) - res->mutable_i8()->set_nullability(type_nullability); - else if (which.isInt16()) - res->mutable_i16()->set_nullability(type_nullability); - else if (which.isInt32()) - res->mutable_i32()->set_nullability(type_nullability); - else if (which.isInt64()) - res->mutable_i64()->set_nullability(type_nullability); - else if (which.isString() || which.isAggregateFunction()) - res->mutable_binary()->set_nullability(type_nullability); /// Spark Binary type is more similiar to CH String type - else if (which.isFloat32()) - res->mutable_fp32()->set_nullability(type_nullability); - else if (which.isFloat64()) - res->mutable_fp64()->set_nullability(type_nullability); - else if (which.isFloat64()) - res->mutable_fp64()->set_nullability(type_nullability); - else if (which.isDateTime64()) - { - const auto * ch_type_datetime64 = checkAndGetDataType(ch_type_without_nullable.get()); - if (ch_type_datetime64->getScale() != 6) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support converting from {}", ch_type->getName()); - res->mutable_timestamp()->set_nullability(type_nullability); - } - else if (which.isDate32()) - res->mutable_date()->set_nullability(type_nullability); - else if (which.isDecimal()) - { - if (which.isDecimal256()) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support converting from {}", ch_type->getName()); - - const auto scale = getDecimalScale(*ch_type_without_nullable, 0); - const auto precision = getDecimalPrecision(*ch_type_without_nullable); - if (scale == 0 && precision == 0) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support converting from {}", ch_type->getName()); - res->mutable_decimal()->set_nullability(type_nullability); - res->mutable_decimal()->set_scale(scale); - res->mutable_decimal()->set_precision(precision); - } - else if (which.isTuple()) - { - const auto * ch_tuple_type = checkAndGetDataType(ch_type_without_nullable.get()); - const auto & ch_field_types = ch_tuple_type->getElements(); - res->mutable_struct_()->set_nullability(type_nullability); - for (const auto & ch_field_type: ch_field_types) - res->mutable_struct_()->mutable_types()->Add(std::move(*buildType(ch_field_type))); - } - else if (which.isArray()) - { - const auto * ch_array_type = checkAndGetDataType(ch_type_without_nullable.get()); - const auto & ch_nested_type = ch_array_type->getNestedType(); - res->mutable_list()->set_nullability(type_nullability); - *(res->mutable_list()->mutable_type()) = *buildType(ch_nested_type); - } - else if (which.isMap()) - { - const auto & ch_map_type = checkAndGetDataType(ch_type_without_nullable.get()); - const auto & ch_key_type = ch_map_type->getKeyType(); - const auto & ch_val_type = ch_map_type->getValueType(); - res->mutable_map()->set_nullability(type_nullability); - *(res->mutable_map()->mutable_key()) = *buildType(ch_key_type); - *(res->mutable_map()->mutable_value()) = *buildType(ch_val_type); - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support converting from {}", ch_type->getName()); - - return std::move(res); -} - -void SerializedPlanBuilder::buildType(const DB::DataTypePtr & ch_type, String & substrait_type) -{ - auto pb = buildType(ch_type); - substrait_type = pb->SerializeAsString(); -} - - substrait::Expression * selection(int32_t field_id) { substrait::Expression * rel = new substrait::Expression(); diff --git a/utils/local-engine/Builder/SerializedPlanBuilder.h b/utils/local-engine/Builder/SerializedPlanBuilder.h index 3b0638a3eeb4..66345c55260a 100644 --- a/utils/local-engine/Builder/SerializedPlanBuilder.h +++ b/utils/local-engine/Builder/SerializedPlanBuilder.h @@ -53,9 +53,6 @@ class SerializedPlanBuilder SchemaPtr schema); std::unique_ptr build(); - static std::shared_ptr buildType(const DB::DataTypePtr & ch_type); - static void buildType(const DB::DataTypePtr & ch_type, String & substrait_type); - private: void setInputToPrev(substrait::Rel * input); substrait::Rel * prev_rel = nullptr; diff --git a/utils/local-engine/CMakeLists.txt b/utils/local-engine/CMakeLists.txt index eade3b81bb74..766610a49261 100644 --- a/utils/local-engine/CMakeLists.txt +++ b/utils/local-engine/CMakeLists.txt @@ -111,7 +111,6 @@ target_compile_options(_icui18n PRIVATE -fPIC) target_compile_options(_cpuid PRIVATE -fPIC) target_compile_options(re2_st PRIVATE -fPIC) target_compile_options(_boost_program_options PRIVATE -fPIC) -target_compile_options(_boost_context PRIVATE -fPIC) target_compile_options(clickhouse_common_io PRIVATE -fPIC) target_compile_options(clickhouse_dictionaries_embedded PRIVATE -fPIC) target_compile_options(clickhouse_common_zookeeper PRIVATE -fPIC) diff --git a/utils/local-engine/Common/DebugUtils.cpp b/utils/local-engine/Common/DebugUtils.cpp index aecb1dc827a5..9606e20ee33d 100644 --- a/utils/local-engine/Common/DebugUtils.cpp +++ b/utils/local-engine/Common/DebugUtils.cpp @@ -17,9 +17,10 @@ void headBlock(const DB::Block & block, size_t count) std::cerr << block.dumpStructure() << std::endl; // print header for (const auto& name : block.getNames()) + { std::cerr << name << "\t"; + } std::cerr << std::endl; - // print rows for (size_t row = 0; row < std::min(count, block.rows()); ++row) { @@ -27,10 +28,70 @@ void headBlock(const DB::Block & block, size_t count) { const auto type = block.getByPosition(column).type; auto col = block.getByPosition(column).column; - - if (column > 0) - std::cerr << "\t"; - std::cerr << toString((*col)[row]); + auto nested_col = col; + DB::DataTypePtr nested_type = type; + if (const auto *nullable = DB::checkAndGetDataType(type.get())) + { + nested_type = nullable->getNestedType(); + const auto *nullable_column = DB::checkAndGetColumn(*col); + nested_col = nullable_column->getNestedColumnPtr(); + } + DB::WhichDataType which(nested_type); + if (col->isNullAt(row)) + { + std::cerr << "null" << "\t"; + } + else if (which.isUInt()) + { + auto value = nested_col->getUInt(row); + std::cerr << std::to_string(value) << "\t"; + } + else if (which.isString()) + { + auto value = DB::checkAndGetColumn(*nested_col)->getDataAt(row).toString(); + std::cerr << value << "\t"; + } + else if (which.isInt()) + { + auto value = nested_col->getInt(row); + std::cerr << std::to_string(value) << "\t"; + } + else if (which.isFloat32()) + { + auto value = nested_col->getFloat32(row); + std::cerr << std::to_string(value) << "\t"; + } + else if (which.isFloat64()) + { + auto value = nested_col->getFloat64(row); + std::cerr << std::to_string(value) << "\t"; + } + else if (which.isDate()) + { + const auto * date_type = DB::checkAndGetDataType(nested_type.get()); + String date_string; + DB::WriteBufferFromString wb(date_string); + date_type->getSerialization(DB::ISerialization::Kind::DEFAULT)->serializeText(*nested_col, row, wb, {}); + std::cerr << date_string.substr(0, 10) << "\t"; + } + else if (which.isDate32()) + { + const auto * date_type = DB::checkAndGetDataType(nested_type.get()); + String date_string; + DB::WriteBufferFromString wb(date_string); + date_type->getSerialization(DB::ISerialization::Kind::DEFAULT)->serializeText(*nested_col, row, wb, {}); + std::cerr << date_string.substr(0, 10) << "\t"; + } + else if (which.isDateTime64()) + { + const auto * datetime64_type = DB::checkAndGetDataType(nested_type.get()); + String datetime64_string; + DB::WriteBufferFromString wb(datetime64_string); + datetime64_type->getSerialization(DB::ISerialization::Kind::DEFAULT)->serializeText(*nested_col, row, wb, {}); + std::cerr << datetime64_string << "\t"; + } + else + std::cerr << "N/A" << "\t"; } std::cerr << std::endl; } @@ -39,14 +100,49 @@ void headBlock(const DB::Block & block, size_t count) void headColumn(const DB::ColumnPtr column, size_t count) { std::cerr << "============Column============" << std::endl; - // print header + std::cerr << column->getName() << "\t"; std::cerr << std::endl; - // print rows for (size_t row = 0; row < std::min(count, column->size()); ++row) - std::cerr << toString((*column)[row]) << std::endl; + { + auto type = column->getDataType(); + const auto& col = column; + DB::WhichDataType which(type); + if (col->isNullAt(row)) + { + std::cerr << "null" << "\t"; + } + else if (which.isUInt()) + { + auto value = col->getUInt(row); + std::cerr << std::to_string(value) << std::endl; + } + else if (which.isString()) + { + auto value = DB::checkAndGetColumn(*col)->getDataAt(row).toString(); + std::cerr << value << std::endl; + } + else if (which.isInt()) + { + auto value = col->getInt(row); + std::cerr << std::to_string(value) << std::endl; + } + else if (which.isFloat32()) + { + auto value = col->getFloat32(row); + std::cerr << std::to_string(value) << std::endl; + } + else if (which.isFloat64()) + { + auto value = col->getFloat64(row); + std::cerr << std::to_string(value) << std::endl; + } + else + { + std::cerr << "N/A" << std::endl; + } + } } - } diff --git a/utils/local-engine/Parser/CHColumnToSparkRow.cpp b/utils/local-engine/Parser/CHColumnToSparkRow.cpp index 8ed0d168b0d0..96d093a5d36a 100644 --- a/utils/local-engine/Parser/CHColumnToSparkRow.cpp +++ b/utils/local-engine/Parser/CHColumnToSparkRow.cpp @@ -1,16 +1,10 @@ #include "CHColumnToSparkRow.h" -#include -#include #include #include #include #include -#include -#include -#include -#include #include -#include +#include #include namespace DB @@ -21,810 +15,299 @@ namespace ErrorCodes } } +#define WRITE_VECTOR_COLUMN(TYPE, PRIME_TYPE, GETTER) \ + const auto * type_col = checkAndGetColumn>(*nested_col); \ + for (auto i = 0; i < num_rows; i++) \ + { \ + bool is_null = nullable_column && nullable_column->isNullAt(i); \ + if (is_null) \ + { \ + setNullAt(buffer_address, offsets[i], field_offset, col_index); \ + } \ + else \ + { \ + auto * pointer = reinterpret_cast(buffer_address + offsets[i] + field_offset); \ + pointer[0] = type_col->GETTER(i);\ + } \ + } namespace local_engine { using namespace DB; - int64_t calculateBitSetWidthInBytes(int32_t num_fields) { return ((num_fields + 63) / 64) * 8; } -static int64_t calculatedFixeSizePerRow(int64_t num_cols) +int64_t calculatedFixeSizePerRow(DB::Block & header, int64_t num_cols) { - return calculateBitSetWidthInBytes(num_cols) + num_cols * 8; + auto fields = header.getNamesAndTypesList(); + // Calculate the decimal col num when the precision >18 + int32_t count = 0; + for (auto i = 0; i < num_cols; i++) + { + auto type = removeNullable(fields.getTypes()[i]); + DB::WhichDataType which(type); + if (which.isDecimal128()) + { + const auto & dtype = typeid_cast *>(type.get()); + int32_t precision = dtype->getPrecision(); + if (precision > 18) + count++; + } + } + + int64_t fixed_size = calculateBitSetWidthInBytes(num_cols) + num_cols * 8; + int64_t decimal_cols_size = count * 16; + return fixed_size + decimal_cols_size; } -int64_t roundNumberOfBytesToNearestWord(int64_t num_bytes) +int64_t roundNumberOfBytesToNearestWord(int64_t numBytes) { - auto remainder = num_bytes & 0x07; // This is equivalent to `numBytes % 8` - return num_bytes + ((8 - remainder) & 0x7); + int64_t remainder = numBytes & 0x07; // This is equivalent to `numBytes % 8` + if (remainder == 0) + { + return numBytes; + } + else + { + return numBytes + (8 - remainder); + } } +int64_t getFieldOffset(int64_t nullBitsetWidthInBytes, int32_t index) +{ + return nullBitsetWidthInBytes + 8L * index; +} -void bitSet(char * bitmap, int32_t index) +void bitSet(uint8_t * buffer_address, int32_t index) { int64_t mask = 1L << (index & 0x3f); // mod 64 and shift int64_t word_offset = (index >> 6) * 8; int64_t word; - memcpy(&word, bitmap + word_offset, sizeof(int64_t)); + memcpy(&word, buffer_address + word_offset, sizeof(int64_t)); int64_t value = word | mask; - memcpy(bitmap + word_offset, &value, sizeof(int64_t)); + memcpy(buffer_address + word_offset, &value, sizeof(int64_t)); } -ALWAYS_INLINE bool isBitSet(const char * bitmap, int32_t index) +void setNullAt(uint8_t * buffer_address, int64_t row_offset, int64_t field_offset, int32_t col_index) { - assert(index >= 0); - int64_t mask = 1 << (index & 63); - int64_t word_offset = static_cast(index >> 6) * 8L; - int64_t word = *reinterpret_cast(bitmap + word_offset); - return word & mask; + bitSet(buffer_address + row_offset, col_index); + // set the value to 0 + memset(buffer_address + row_offset + field_offset, 0, sizeof(int64_t)); } -static void writeFixedLengthNonNullableValue( - char * buffer_address, +void writeValue( + uint8_t * buffer_address, int64_t field_offset, - const ColumnWithTypeAndName & col, - int64_t num_rows, - const std::vector & offsets) -{ - FixedLengthDataWriter writer(col.type); - for (size_t i = 0; i < static_cast(num_rows); i++) - writer.unsafeWrite(col.column->getDataAt(i), buffer_address + offsets[i] + field_offset); -} - -static void writeFixedLengthNullableValue( - char * buffer_address, - int64_t field_offset, - const ColumnWithTypeAndName & col, + ColumnWithTypeAndName & col, int32_t col_index, int64_t num_rows, - const std::vector & offsets) + std::vector & offsets, + std::vector & buffer_cursor) { + ColumnPtr nested_col = col.column; const auto * nullable_column = checkAndGetColumn(*col.column); - const auto & null_map = nullable_column->getNullMapData(); - const auto & nested_column = nullable_column->getNestedColumn(); - FixedLengthDataWriter writer(col.type); - for (size_t i = 0; i < static_cast(num_rows); i++) + if (nullable_column) { - if (null_map[i]) - bitSet(buffer_address + offsets[i], col_index); - else - writer.unsafeWrite(nested_column.getDataAt(i), buffer_address + offsets[i] + field_offset); + nested_col = nullable_column->getNestedColumnPtr(); } -} - -static void writeVariableLengthNonNullableValue( - char * buffer_address, - int64_t field_offset, - const ColumnWithTypeAndName & col, - int64_t num_rows, - const std::vector & offsets, - std::vector & buffer_cursor) -{ - const auto type_without_nullable{std::move(removeNullable(col.type))}; - const bool use_raw_data = BackingDataLengthCalculator::isDataTypeSupportRawData(type_without_nullable); - VariableLengthDataWriter writer(col.type, buffer_address, offsets, buffer_cursor); - if (use_raw_data) + nested_col = nested_col->convertToFullColumnIfConst(); + WhichDataType which(nested_col->getDataType()); + if (which.isUInt8()) { - for (size_t i = 0; i < static_cast(num_rows); i++) - { - StringRef str = col.column->getDataAt(i); - int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data, str.size, 0); - memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); - } + WRITE_VECTOR_COLUMN(UInt8, uint8_t, getInt) } - else + else if (which.isInt8()) { - Field field; - for (size_t i = 0; i < static_cast(num_rows); i++) - { - field = std::move((*col.column)[i]); - int64_t offset_and_size = writer.write(i, field, 0); - memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); - } + WRITE_VECTOR_COLUMN(Int8, int8_t, getInt) } -} - -static void writeVariableLengthNullableValue( - char * buffer_address, - int64_t field_offset, - const ColumnWithTypeAndName & col, - int32_t col_index, - int64_t num_rows, - const std::vector & offsets, - std::vector & buffer_cursor) -{ - const auto * nullable_column = checkAndGetColumn(*col.column); - const auto & null_map = nullable_column->getNullMapData(); - const auto & nested_column = nullable_column->getNestedColumn(); - const auto type_without_nullable{std::move(removeNullable(col.type))}; - const bool use_raw_data = BackingDataLengthCalculator::isDataTypeSupportRawData(type_without_nullable); - VariableLengthDataWriter writer(col.type, buffer_address, offsets, buffer_cursor); - if (use_raw_data) + else if (which.isInt16()) + { + WRITE_VECTOR_COLUMN(Int16, int16_t, getInt) + } + else if (which.isUInt16()) + { + WRITE_VECTOR_COLUMN(UInt16, uint16_t , get64) + } + else if (which.isInt32()) + { + WRITE_VECTOR_COLUMN(Int32, int32_t, getInt) + } + else if (which.isInt64()) + { + WRITE_VECTOR_COLUMN(Int64, int64_t, getInt) + } + else if (which.isUInt64()) { - for (size_t i = 0; i < static_cast(num_rows); i++) + WRITE_VECTOR_COLUMN(UInt64, int64_t, get64) + } + else if (which.isFloat32()) + { + WRITE_VECTOR_COLUMN(Float32, float_t, getFloat32) + } + else if (which.isFloat64()) + { + WRITE_VECTOR_COLUMN(Float64, double_t, getFloat64) + } + else if (which.isDate()) + { + WRITE_VECTOR_COLUMN(UInt16, uint16_t, get64) + } + else if (which.isDate32()) + { + WRITE_VECTOR_COLUMN(UInt32, uint32_t, get64) + } + else if (which.isDateTime64()) + { + using ColumnDateTime64 = ColumnDecimal; + const auto * datetime64_col = checkAndGetColumn(*nested_col); + for (auto i=0; iisNullAt(i); + if (is_null) + { + setNullAt(buffer_address, offsets[i], field_offset, col_index); + } else { - StringRef str = nested_column.getDataAt(i); - int64_t offset_and_size = writer.writeUnalignedBytes(i, str.data, str.size, 0); - memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); + auto * pointer = reinterpret_cast(buffer_address + offsets[i] + field_offset); + pointer[0] = datetime64_col->getInt(i); } } } - else + else if (which.isString()) { - Field field; - for (size_t i = 0; i < static_cast(num_rows); i++) + const auto * string_col = checkAndGetColumn(*nested_col); + for (auto i = 0; i < num_rows; i++) { - if (null_map[i]) - bitSet(buffer_address + offsets[i], col_index); + bool is_null = nullable_column && nullable_column->isNullAt(i); + if (is_null) + { + setNullAt(buffer_address, offsets[i], field_offset, col_index); + } else { - field = std::move(nested_column[i]); - int64_t offset_and_size = writer.write(i, field, 0); - memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, 8); + StringRef string_value = string_col->getDataAt(i); + // write the variable value + memcpy(buffer_address + offsets[i] + buffer_cursor[i], string_value.data, string_value.size); + // write the offset and size + int64_t offset_and_size = (buffer_cursor[i] << 32) | string_value.size; + memcpy(buffer_address + offsets[i] + field_offset, &offset_and_size, sizeof(int64_t)); + buffer_cursor[i] += string_value.size; } } } -} - - -static void writeValue( - char * buffer_address, - int64_t field_offset, - const ColumnWithTypeAndName & col, - int32_t col_index, - int64_t num_rows, - const std::vector & offsets, - std::vector & buffer_cursor) -{ - const auto type_without_nullable{std::move(removeNullable(col.type))}; - const auto is_nullable = isColumnNullable(*col.column); - if (BackingDataLengthCalculator::isFixedLengthDataType(type_without_nullable)) - { - if (is_nullable) - writeFixedLengthNullableValue(buffer_address, field_offset, col, col_index, num_rows, offsets); - else - writeFixedLengthNonNullableValue(buffer_address, field_offset, col, num_rows, offsets); - } - else if (BackingDataLengthCalculator::isVariableLengthDataType(type_without_nullable)) + else { - if (is_nullable) - writeVariableLengthNullableValue(buffer_address, field_offset, col, col_index, num_rows, offsets, buffer_cursor); - else - writeVariableLengthNonNullableValue(buffer_address, field_offset, col, num_rows, offsets, buffer_cursor); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "doesn't support type {} convert from ch to spark" ,magic_enum::enum_name(nested_col->getDataType())); } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Doesn't support type {} for writeValue", col.type->getName()); } -SparkRowInfo::SparkRowInfo(const Block & block) - : types(std::move(block.getDataTypes())) - , num_rows(block.rows()) - , num_cols(block.columns()) - , null_bitset_width_in_bytes(calculateBitSetWidthInBytes(num_cols)) - , total_bytes(0) - , offsets(num_rows, 0) - , lengths(num_rows, 0) - , buffer_cursor(num_rows, 0) - , buffer_address(nullptr) +SparkRowInfo::SparkRowInfo(DB::Block & block) { - int64_t fixed_size_per_row = calculatedFixeSizePerRow(num_cols); - - /// Initialize lengths and buffer_cursor - for (int64_t i = 0; i < num_rows; i++) + num_rows = block.rows(); + num_cols = block.columns(); + null_bitset_width_in_bytes = calculateBitSetWidthInBytes(num_cols); + int64_t fixed_size_per_row = calculatedFixeSizePerRow(block, num_cols); + // Initialize the offsets_ , lengths_, buffer_cursor_ + for (auto i = 0; i < num_rows; i++) { - lengths[i] = fixed_size_per_row; - buffer_cursor[i] = fixed_size_per_row; + lengths.push_back(fixed_size_per_row); + offsets.push_back(0); + buffer_cursor.push_back(null_bitset_width_in_bytes + 8 * num_cols); } - - for (int64_t col_idx = 0; col_idx < num_cols; ++col_idx) + // Calculated the lengths_ + for (auto i = 0; i < num_cols; i++) { - const auto & col = block.getByPosition(col_idx); - - /// No need to calculate backing data length for fixed length types - const auto type_without_nullable = removeNullable(col.type); - if (BackingDataLengthCalculator::isVariableLengthDataType(type_without_nullable)) + auto col = block.getByPosition(i); + if (isStringOrFixedString(removeNullable(col.type))) { - if (BackingDataLengthCalculator::isDataTypeSupportRawData(type_without_nullable)) + size_t length; + for (auto j = 0; j < num_rows; j++) { - const auto * nullable_column = checkAndGetColumn(*col.column); - if (nullable_column) - { - const auto & nested_column = nullable_column->getNestedColumn(); - const auto & null_map = nullable_column->getNullMapData(); - for (auto row_idx = 0; row_idx < num_rows; ++row_idx) - if (!null_map[row_idx]) - lengths[row_idx] += roundNumberOfBytesToNearestWord(nested_column.getDataAt(row_idx).size); - } - else - { - for (auto row_idx = 0; row_idx < num_rows; ++row_idx) - lengths[row_idx] += roundNumberOfBytesToNearestWord(col.column->getDataAt(row_idx).size); - } - } - else - { - BackingDataLengthCalculator calculator(col.type); - for (auto row_idx = 0; row_idx < num_rows; ++row_idx) - { - const auto field = (*col.column)[row_idx]; - lengths[row_idx] += calculator.calculate(field); - } + length = col.column->getDataAt(j).size; + lengths[j] += roundNumberOfBytesToNearestWord(length); } } } - - /// Initialize offsets - for (int64_t i = 1; i < num_rows; ++i) - offsets[i] = offsets[i - 1] + lengths[i - 1]; - - /// Initialize total_bytes - for (int64_t i = 0; i < num_rows; ++i) - total_bytes += lengths[i]; } -const DB::DataTypes & SparkRowInfo::getDataTypes() const -{ - return types; -} - -int64_t SparkRowInfo::getFieldOffset(int32_t col_idx) const -{ - return null_bitset_width_in_bytes + 8L * col_idx; -} - -int64_t SparkRowInfo::getNullBitsetWidthInBytes() const +int64_t local_engine::SparkRowInfo::getNullBitsetWidthInBytes() const { return null_bitset_width_in_bytes; } -void SparkRowInfo::setNullBitsetWidthInBytes(int64_t null_bitset_width_in_bytes_) +void local_engine::SparkRowInfo::setNullBitsetWidthInBytes(int64_t null_bitset_width_in_bytes_) { null_bitset_width_in_bytes = null_bitset_width_in_bytes_; } - -int64_t SparkRowInfo::getNumCols() const +int64_t local_engine::SparkRowInfo::getNumCols() const { return num_cols; } - -void SparkRowInfo::setNumCols(int64_t num_cols_) +void local_engine::SparkRowInfo::setNumCols(int64_t numCols) { - num_cols = num_cols_; + num_cols = numCols; } - -int64_t SparkRowInfo::getNumRows() const +int64_t local_engine::SparkRowInfo::getNumRows() const { return num_rows; } - -void SparkRowInfo::setNumRows(int64_t num_rows_) +void local_engine::SparkRowInfo::setNumRows(int64_t numRows) { - num_rows = num_rows_; + num_rows = numRows; } - -char * SparkRowInfo::getBufferAddress() const +unsigned char * local_engine::SparkRowInfo::getBufferAddress() const { return buffer_address; } - -void SparkRowInfo::setBufferAddress(char * buffer_address_) +void local_engine::SparkRowInfo::setBufferAddress(unsigned char * bufferAddress) { - buffer_address = buffer_address_; + buffer_address = bufferAddress; } - -const std::vector & SparkRowInfo::getOffsets() const +const std::vector & local_engine::SparkRowInfo::getOffsets() const { return offsets; } - -const std::vector & SparkRowInfo::getLengths() const +const std::vector & local_engine::SparkRowInfo::getLengths() const { return lengths; } - -std::vector & SparkRowInfo::getBufferCursor() -{ - return buffer_cursor; -} - int64_t SparkRowInfo::getTotalBytes() const { return total_bytes; } - -std::unique_ptr CHColumnToSparkRow::convertCHColumnToSparkRow(const Block & block) +std::unique_ptr local_engine::CHColumnToSparkRow::convertCHColumnToSparkRow(Block & block) { - if (!block.rows() || !block.columns()) - return {}; - std::unique_ptr spark_row_info = std::make_unique(block); - spark_row_info->setBufferAddress(reinterpret_cast(alloc(spark_row_info->getTotalBytes(), 64))); - // spark_row_info->setBufferAddress(alignedAlloc(spark_row_info->getTotalBytes(), 64)); - memset(spark_row_info->getBufferAddress(), 0, spark_row_info->getTotalBytes()); - for (auto col_idx = 0; col_idx < spark_row_info->getNumCols(); col_idx++) + // Calculated the offsets_ and total memory size based on lengths_ + int64_t total_memory_size = spark_row_info->lengths[0]; + for (auto i = 1; i < spark_row_info->num_rows; i++) { - const auto & col = block.getByPosition(col_idx); - int64_t field_offset = spark_row_info->getFieldOffset(col_idx); + spark_row_info->offsets[i] = spark_row_info->offsets[i - 1] + spark_row_info->lengths[i - 1]; + total_memory_size += spark_row_info->lengths[i]; + } + spark_row_info->total_bytes = total_memory_size; + spark_row_info->buffer_address = reinterpret_cast(alloc(total_memory_size)); + memset(spark_row_info->buffer_address, 0, sizeof(int8_t) * spark_row_info->total_bytes); + for (auto i = 0; i < spark_row_info->num_cols; i++) + { + auto array = block.getByPosition(i); + int64_t field_offset = getFieldOffset(spark_row_info->null_bitset_width_in_bytes, i); writeValue( - spark_row_info->getBufferAddress(), + spark_row_info->buffer_address, field_offset, - col, - col_idx, - spark_row_info->getNumRows(), - spark_row_info->getOffsets(), - spark_row_info->getBufferCursor()); + array, + i, + spark_row_info->num_rows, + spark_row_info->offsets, + spark_row_info->buffer_cursor); } return spark_row_info; } - -void CHColumnToSparkRow::freeMem(char * address, size_t size) +void CHColumnToSparkRow::freeMem(uint8_t * address, size_t size) { free(address, size); - // rollback(size); -} - -BackingDataLengthCalculator::BackingDataLengthCalculator(const DataTypePtr & type_) - : type_without_nullable(removeNullable(type_)), which(type_without_nullable) -{ - if (!isFixedLengthDataType(type_without_nullable) && !isVariableLengthDataType(type_without_nullable)) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Doesn't support type {} for BackingDataLengthCalculator", type_without_nullable->getName()); -} - -int64_t BackingDataLengthCalculator::calculate(const Field & field) const -{ - if (field.isNull()) - return 0; - - if (which.isNativeInt() || which.isNativeUInt() || which.isFloat() || which.isDateOrDate32() || which.isDateTime64() - || which.isDecimal32() || which.isDecimal64()) - return 0; - - if (which.isStringOrFixedString()) - { - const auto & str = field.get(); - return roundNumberOfBytesToNearestWord(str.size()); - } - - if (which.isDecimal128()) - return 16; - - if (which.isArray()) - { - /// 内存布局:numElements(8B) | null_bitmap(与numElements成正比) | values(每个值长度与类型有关) | backing buffer - const auto & array = field.get(); /// Array can not be wrapped with Nullable - const auto num_elems = array.size(); - int64_t res = 8 + calculateBitSetWidthInBytes(num_elems); - - const auto * array_type = typeid_cast(type_without_nullable.get()); - const auto & nested_type = array_type->getNestedType(); - res += roundNumberOfBytesToNearestWord(getArrayElementSize(nested_type) * num_elems); - - BackingDataLengthCalculator calculator(nested_type); - for (size_t i = 0; i < array.size(); ++i) - res += calculator.calculate(array[i]); - return res; - } - - if (which.isMap()) - { - /// 内存布局:Length of UnsafeArrayData of key(8B) | UnsafeArrayData of key | UnsafeArrayData of value - int64_t res = 8; - - /// Construct Array of keys and values from Map - const auto & map = field.get(); /// Map can not be wrapped with Nullable - const auto num_keys = map.size(); - auto array_key = Array(); - auto array_val = Array(); - array_key.reserve(num_keys); - array_val.reserve(num_keys); - for (size_t i = 0; i < num_keys; ++i) - { - const auto & pair = map[i].get(); - array_key.push_back(pair[0]); - array_val.push_back(pair[1]); - } - - const auto * map_type = typeid_cast(type_without_nullable.get()); - - const auto & key_type = map_type->getKeyType(); - const auto key_array_type = std::make_shared(key_type); - BackingDataLengthCalculator calculator_key(key_array_type); - res += calculator_key.calculate(array_key); - - const auto & val_type = map_type->getValueType(); - const auto type_array_val = std::make_shared(val_type); - BackingDataLengthCalculator calculator_val(type_array_val); - res += calculator_val.calculate(array_val); - return res; - } - - if (which.isTuple()) - { - /// 内存布局:null_bitmap(字节数与字段数成正比) | field1 value(8B) | field2 value(8B) | ... | fieldn value(8B) | backing buffer - const auto & tuple = field.get(); /// Tuple can not be wrapped with Nullable - const auto * type_tuple = typeid_cast(type_without_nullable.get()); - const auto & type_fields = type_tuple->getElements(); - const auto num_fields = type_fields.size(); - int64_t res = calculateBitSetWidthInBytes(num_fields) + 8 * num_fields; - for (size_t i = 0; i < num_fields; ++i) - { - BackingDataLengthCalculator calculator(type_fields[i]); - res += calculator.calculate(tuple[i]); - } - return res; - } - - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Doesn't support type {} for BackingBufferLengthCalculator", type_without_nullable->getName()); } - -int64_t BackingDataLengthCalculator::getArrayElementSize(const DataTypePtr & nested_type) -{ - const WhichDataType nested_which(removeNullable(nested_type)); - if (nested_which.isUInt8() || nested_which.isInt8()) - return 1; - else if (nested_which.isUInt16() || nested_which.isInt16() || nested_which.isDate()) - return 2; - else if ( - nested_which.isUInt32() || nested_which.isInt32() || nested_which.isFloat32() || nested_which.isDate32() - || nested_which.isDecimal32()) - return 4; - else if ( - nested_which.isUInt64() || nested_which.isInt64() || nested_which.isFloat64() || nested_which.isDateTime64() - || nested_which.isDecimal64()) - return 8; - else - return 8; -} - -bool BackingDataLengthCalculator::isFixedLengthDataType(const DataTypePtr & type_without_nullable) -{ - const WhichDataType which(type_without_nullable); - return which.isUInt8() || which.isInt8() || which.isUInt16() || which.isInt16() || which.isDate() || which.isUInt32() || which.isInt32() - || which.isFloat32() || which.isDate32() || which.isDecimal32() || which.isUInt64() || which.isInt64() || which.isFloat64() - || which.isDateTime64() || which.isDecimal64(); -} - -bool BackingDataLengthCalculator::isVariableLengthDataType(const DataTypePtr & type_without_nullable) -{ - const WhichDataType which(type_without_nullable); - return which.isStringOrFixedString() || which.isDecimal128() || which.isArray() || which.isMap() || which.isTuple(); -} - -bool BackingDataLengthCalculator::isDataTypeSupportRawData(const DB::DataTypePtr & type_without_nullable) -{ - const WhichDataType which(type_without_nullable); - return isFixedLengthDataType(type_without_nullable) || which.isStringOrFixedString() || which.isDecimal128(); -} - - -VariableLengthDataWriter::VariableLengthDataWriter( - const DataTypePtr & type_, char * buffer_address_, const std::vector & offsets_, std::vector & buffer_cursor_) - : type_without_nullable(removeNullable(type_)) - , which(type_without_nullable) - , buffer_address(buffer_address_) - , offsets(offsets_) - , buffer_cursor(buffer_cursor_) -{ - assert(buffer_address); - assert(!offsets.empty()); - assert(!buffer_cursor.empty()); - assert(offsets.size() == buffer_cursor.size()); - - if (!BackingDataLengthCalculator::isVariableLengthDataType(type_without_nullable)) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataWriter doesn't support type {}", type_without_nullable->getName()); -} - -int64_t VariableLengthDataWriter::writeArray(size_t row_idx, const DB::Array & array, int64_t parent_offset) -{ - /// 内存布局:numElements(8B) | null_bitmap(与numElements成正比) | values(每个值长度与类型有关) | backing data - const auto & offset = offsets[row_idx]; - auto & cursor = buffer_cursor[row_idx]; - const auto num_elems = array.size(); - const auto * array_type = typeid_cast(type_without_nullable.get()); - const auto & nested_type = array_type->getNestedType(); - - /// Write numElements(8B) - const auto start = cursor; - memcpy(buffer_address + offset + cursor, &num_elems, 8); - cursor += 8; - if (num_elems == 0) - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, 8); - - /// Skip null_bitmap(already reset to zero) - const auto len_null_bitmap = calculateBitSetWidthInBytes(num_elems); - cursor += len_null_bitmap; - - /// Skip values(already reset to zero) - const auto elem_size = BackingDataLengthCalculator::getArrayElementSize(nested_type); - const auto len_values = roundNumberOfBytesToNearestWord(elem_size * num_elems); - cursor += len_values; - - if (BackingDataLengthCalculator::isFixedLengthDataType(removeNullable(nested_type))) - { - /// If nested type is fixed-length data type, update null_bitmap and values in place - FixedLengthDataWriter writer(nested_type); - for (size_t i = 0; i < num_elems; ++i) - { - const auto & elem = array[i]; - if (elem.isNull()) - bitSet(buffer_address + offset + start + 8, i); - else - // writer.write(elem, buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); - writer.unsafeWrite(&elem.reinterpret(), buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size); - } - } - else - { - /// If nested type is not fixed-length data type, update null_bitmap in place - /// And append values in backing data recursively - VariableLengthDataWriter writer(nested_type, buffer_address, offsets, buffer_cursor); - for (size_t i = 0; i < num_elems; ++i) - { - const auto & elem = array[i]; - if (elem.isNull()) - bitSet(buffer_address + offset + start + 8, i); - else - { - const auto offset_and_size = writer.write(row_idx, elem, start); - memcpy(buffer_address + offset + start + 8 + len_null_bitmap + i * elem_size, &offset_and_size, 8); - } - } - } - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, cursor - start); -} - -int64_t VariableLengthDataWriter::writeMap(size_t row_idx, const DB::Map & map, int64_t parent_offset) -{ - /// 内存布局:Length of UnsafeArrayData of key(8B) | UnsafeArrayData of key | UnsafeArrayData of value - const auto & offset = offsets[row_idx]; - auto & cursor = buffer_cursor[row_idx]; - - /// Skip length of UnsafeArrayData of key(8B) - const auto start = cursor; - cursor += 8; - - /// If Map is empty, return in advance - const auto num_pairs = map.size(); - if (num_pairs == 0) - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, 8); - - /// Construct array of keys and array of values from map - auto key_array = Array(); - auto val_array = Array(); - key_array.reserve(num_pairs); - val_array.reserve(num_pairs); - for (size_t i = 0; i < num_pairs; ++i) - { - const auto & pair = map[i].get(); - key_array.push_back(pair[0]); - val_array.push_back(pair[1]); - } - - const auto * map_type = typeid_cast(type_without_nullable.get()); - - /// Append UnsafeArrayData of key - const auto & key_type = map_type->getKeyType(); - const auto key_array_type = std::make_shared(key_type); - VariableLengthDataWriter key_writer(key_array_type, buffer_address, offsets, buffer_cursor); - const auto key_array_size = BackingDataLengthCalculator::extractSize(key_writer.write(row_idx, key_array, start + 8)); - - /// Fill length of UnsafeArrayData of key - memcpy(buffer_address + offset + start, &key_array_size, 8); - - /// Append UnsafeArrayData of value - const auto & val_type = map_type->getValueType(); - const auto val_array_type = std::make_shared(val_type); - VariableLengthDataWriter val_writer(val_array_type, buffer_address, offsets, buffer_cursor); - val_writer.write(row_idx, val_array, start + 8 + key_array_size); - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, cursor - start); -} - -int64_t VariableLengthDataWriter::writeStruct(size_t row_idx, const DB::Tuple & tuple, int64_t parent_offset) -{ - /// 内存布局:null_bitmap(字节数与字段数成正比) | values(num_fields * 8B) | backing data - const auto & offset = offsets[row_idx]; - auto & cursor = buffer_cursor[row_idx]; - const auto start = cursor; - - /// Skip null_bitmap - const auto * tuple_type = typeid_cast(type_without_nullable.get()); - const auto & field_types = tuple_type->getElements(); - const auto num_fields = field_types.size(); - if (num_fields == 0) - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, 0); - const auto len_null_bitmap = calculateBitSetWidthInBytes(num_fields); - cursor += len_null_bitmap; - - /// Skip values - cursor += num_fields * 8; - - /// If field type is fixed-length, fill field value in values region - /// else append it to backing data region, and update offset_and_size in values region - for (size_t i = 0; i < num_fields; ++i) - { - const auto & field_value = tuple[i]; - const auto & field_type = field_types[i]; - if (field_value.isNull()) - { - bitSet(buffer_address + offset + start, i); - continue; - } - - if (BackingDataLengthCalculator::isFixedLengthDataType(removeNullable(field_type))) - { - FixedLengthDataWriter writer(field_type); - // writer.write(field_value, buffer_address + offset + start + len_null_bitmap + i * 8); - writer.unsafeWrite(&field_value.reinterpret(), buffer_address + offset + start + len_null_bitmap + i * 8); - } - else - { - VariableLengthDataWriter writer(field_type, buffer_address, offsets, buffer_cursor); - const auto offset_and_size = writer.write(row_idx, field_value, start); - memcpy(buffer_address + offset + start + len_null_bitmap + 8 * i, &offset_and_size, 8); - } - } - return BackingDataLengthCalculator::getOffsetAndSize(start - parent_offset, cursor - start); -} - -int64_t VariableLengthDataWriter::write(size_t row_idx, const DB::Field & field, int64_t parent_offset) -{ - assert(row_idx < offsets.size()); - - if (field.isNull()) - return 0; - - if (which.isStringOrFixedString()) - { - const auto & str = field.get(); - return writeUnalignedBytes(row_idx, str.data(), str.size(), parent_offset); - } - - if (which.isDecimal128()) - { - // const auto & decimal = field.get>(); - // const auto value = decimal.getValue(); - return writeUnalignedBytes(row_idx, &field.reinterpret(), sizeof(Decimal128), parent_offset); - } - - if (which.isArray()) - { - const auto & array = field.get(); - return writeArray(row_idx, array, parent_offset); - } - - if (which.isMap()) - { - const auto & map = field.get(); - return writeMap(row_idx, map, parent_offset); - } - - if (which.isTuple()) - { - const auto & tuple = field.get(); - return writeStruct(row_idx, tuple, parent_offset); - } - - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Doesn't support type {} for BackingDataWriter", type_without_nullable->getName()); -} - -int64_t BackingDataLengthCalculator::getOffsetAndSize(int64_t cursor, int64_t size) -{ - return (cursor << 32) | size; -} - -int64_t BackingDataLengthCalculator::extractOffset(int64_t offset_and_size) -{ - return offset_and_size >> 32; -} - -int64_t BackingDataLengthCalculator::extractSize(int64_t offset_and_size) -{ - return offset_and_size & 0xffffffff; -} - -int64_t VariableLengthDataWriter::writeUnalignedBytes(size_t row_idx, const char * src, size_t size, int64_t parent_offset) -{ - memcpy(buffer_address + offsets[row_idx] + buffer_cursor[row_idx], src, size); - auto res = BackingDataLengthCalculator::getOffsetAndSize(buffer_cursor[row_idx] - parent_offset, size); - buffer_cursor[row_idx] += roundNumberOfBytesToNearestWord(size); - return res; -} - - -FixedLengthDataWriter::FixedLengthDataWriter(const DB::DataTypePtr & type_) - : type_without_nullable(removeNullable(type_)), which(type_without_nullable) -{ - if (!BackingDataLengthCalculator::isFixedLengthDataType(type_without_nullable)) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "FixedLengthWriter doesn't support type {}", type_without_nullable->getName()); -} - -void FixedLengthDataWriter::write(const DB::Field & field, char * buffer) -{ - /// Skip null value - if (field.isNull()) - return; - - if (which.isUInt8()) - { - const auto value = UInt8(field.get()); - memcpy(buffer, &value, 1); - } - else if (which.isUInt16() || which.isDate()) - { - const auto value = UInt16(field.get()); - memcpy(buffer, &value, 2); - } - else if (which.isUInt32() || which.isDate32()) - { - const auto value = UInt32(field.get()); - memcpy(buffer, &value, 4); - } - else if (which.isUInt64()) - { - const auto & value = field.get(); - memcpy(buffer, &value, 8); - } - else if (which.isInt8()) - { - const auto value = Int8(field.get()); - memcpy(buffer, &value, 1); - } - else if (which.isInt16()) - { - const auto value = Int16(field.get()); - memcpy(buffer, &value, 2); - } - else if (which.isInt32()) - { - const auto value = Int32(field.get()); - memcpy(buffer, &value, 4); - } - else if (which.isInt64()) - { - const auto & value = field.get(); - memcpy(buffer, &value, 8); - } - else if (which.isFloat32()) - { - const auto value = Float32(field.get()); - memcpy(buffer, &value, 4); - } - else if (which.isFloat64()) - { - const auto & value = field.get(); - memcpy(buffer, &value, 8); - } - else if (which.isDecimal32()) - { - const auto & value = field.get(); - const auto decimal = value.getValue(); - memcpy(buffer, &decimal, 4); - } - else if (which.isDecimal64() || which.isDateTime64()) - { - const auto & value = field.get(); - auto decimal = value.getValue(); - memcpy(buffer, &decimal, 8); - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "FixedLengthDataWriter doesn't support type {}", type_without_nullable->getName()); -} - -void FixedLengthDataWriter::unsafeWrite(const StringRef & str, char * buffer) -{ - memcpy(buffer, str.data, str.size); -} - -void FixedLengthDataWriter::unsafeWrite(const char * __restrict src, char * __restrict buffer) -{ - memcpy(buffer, src, type_without_nullable->getSizeOfValueInMemory()); -} - } diff --git a/utils/local-engine/Parser/CHColumnToSparkRow.h b/utils/local-engine/Parser/CHColumnToSparkRow.h index 5b10aa3abe53..ec49cec6eebf 100644 --- a/utils/local-engine/Parser/CHColumnToSparkRow.h +++ b/utils/local-engine/Parser/CHColumnToSparkRow.h @@ -1,166 +1,51 @@ #pragma once #include #include -#include #include -#include - namespace local_engine { int64_t calculateBitSetWidthInBytes(int32_t num_fields); -int64_t roundNumberOfBytesToNearestWord(int64_t num_bytes); -void bitSet(char * bitmap, int32_t index); -bool isBitSet(const char * bitmap, int32_t index); class CHColumnToSparkRow; class SparkRowToCHColumn; -class SparkRowInfo : public boost::noncopyable +class SparkRowInfo { friend CHColumnToSparkRow; friend SparkRowToCHColumn; public: - explicit SparkRowInfo(const DB::Block & block); - - const DB::DataTypes & getDataTypes() const; - - int64_t getFieldOffset(int32_t col_idx) const; - + explicit SparkRowInfo(DB::Block & block); int64_t getNullBitsetWidthInBytes() const; - void setNullBitsetWidthInBytes(int64_t null_bitset_width_in_bytes_); - + void setNullBitsetWidthInBytes(int64_t nullBitsetWidthInBytes); int64_t getNumCols() const; - void setNumCols(int64_t num_cols_); - + void setNumCols(int64_t numCols); int64_t getNumRows() const; - void setNumRows(int64_t num_rows_); - - char * getBufferAddress() const; - void setBufferAddress(char * buffer_address); - + void setNumRows(int64_t numRows); + unsigned char * getBufferAddress() const; + void setBufferAddress(unsigned char * bufferAddress); const std::vector & getOffsets() const; const std::vector & getLengths() const; - std::vector & getBufferCursor(); int64_t getTotalBytes() const; private: - const DB::DataTypes types; - int64_t num_rows; - int64_t num_cols; - int64_t null_bitset_width_in_bytes; int64_t total_bytes; - + int64_t null_bitset_width_in_bytes; + int64_t num_cols; + int64_t num_rows; + std::vector buffer_cursor; + uint8_t * buffer_address; std::vector offsets; std::vector lengths; - std::vector buffer_cursor; - char * buffer_address; }; using SparkRowInfoPtr = std::unique_ptr; -class CHColumnToSparkRow : private Allocator -// class CHColumnToSparkRow : public DB::Arena -{ -public: - std::unique_ptr convertCHColumnToSparkRow(const DB::Block & block); - void freeMem(char * address, size_t size); -}; - -/// Return backing data length of values with variable-length type in bytes -class BackingDataLengthCalculator -{ -public: - static constexpr size_t DECIMAL_MAX_INT64_DIGITS = 18; - - explicit BackingDataLengthCalculator(const DB::DataTypePtr & type_); - virtual ~BackingDataLengthCalculator() = default; - - /// Return length is guranteed to round up to 8 - virtual int64_t calculate(const DB::Field & field) const; - - static int64_t getArrayElementSize(const DB::DataTypePtr & nested_type); - - /// Is CH DataType can be converted to fixed-length data type in Spark? - static bool isFixedLengthDataType(const DB::DataTypePtr & type_without_nullable); - - /// Is CH DataType can be converted to variable-length data type in Spark? - static bool isVariableLengthDataType(const DB::DataTypePtr & type_without_nullable); - - /// If Data Type can use raw data between CH Column and Spark Row if value is not null - static bool isDataTypeSupportRawData(const DB::DataTypePtr & type_without_nullable); - - static int64_t getOffsetAndSize(int64_t cursor, int64_t size); - static int64_t extractOffset(int64_t offset_and_size); - static int64_t extractSize(int64_t offset_and_size); - -private: - // const DB::DataTypePtr type; - const DB::DataTypePtr type_without_nullable; - const DB::WhichDataType which; -}; - -/// Writing variable-length typed values to backing data region of Spark Row -/// User who calls VariableLengthDataWriter is responsible to write offset_and_size -/// returned by VariableLengthDataWriter::write to field value in Spark Row -class VariableLengthDataWriter +class CHColumnToSparkRow : private Allocator { public: - VariableLengthDataWriter( - const DB::DataTypePtr & type_, - char * buffer_address_, - const std::vector & offsets_, - std::vector & buffer_cursor_); - - virtual ~VariableLengthDataWriter() = default; - - /// Write value of variable-length to backing data region of structure(row or array) and return offset and size in backing data region - /// It's caller's duty to make sure that row fields or array elements are written in order - /// parent_offset: the starting offset of current structure in which we are updating it's backing data region - virtual int64_t write(size_t row_idx, const DB::Field & field, int64_t parent_offset); - - /// Only support String/FixedString/Decimal32/Decimal64 - int64_t writeUnalignedBytes(size_t row_idx, const char * src, size_t size, int64_t parent_offset); -private: - int64_t writeArray(size_t row_idx, const DB::Array & array, int64_t parent_offset); - int64_t writeMap(size_t row_idx, const DB::Map & map, int64_t parent_offset); - int64_t writeStruct(size_t row_idx, const DB::Tuple & tuple, int64_t parent_offset); - - // const DB::DataTypePtr type; - const DB::DataTypePtr type_without_nullable; - const DB::WhichDataType which; - - /// Global buffer of spark rows - char * const buffer_address; - /// Offsets of each spark row - const std::vector & offsets; - /// Cursors of backing data in each spark row, relative to offsets - std::vector & buffer_cursor; -}; - -class FixedLengthDataWriter -{ -public: - explicit FixedLengthDataWriter(const DB::DataTypePtr & type_); - virtual ~FixedLengthDataWriter() = default; - - /// Write value of fixed-length to values region of structure(struct or array) - /// It's caller's duty to make sure that struct fields or array elements are written in order - virtual void write(const DB::Field & field, char * buffer); - - /// Copy memory chunk of Fixed length typed CH Column directory to buffer for performance. - /// It is unsafe unless you know what you are doing. - virtual void unsafeWrite(const StringRef & str, char * buffer); - - /// Copy memory chunk of in fixed length typed Field directory to buffer for performance. - /// It is unsafe unless you know what you are doing. - virtual void unsafeWrite(const char * __restrict src, char * __restrict buffer); - -private: - // const DB::DataTypePtr type; - const DB::DataTypePtr type_without_nullable; - const DB::WhichDataType which; + std::unique_ptr convertCHColumnToSparkRow(DB::Block & block); + void freeMem(uint8_t * address, size_t size); }; - } diff --git a/utils/local-engine/Parser/SerializedPlanParser.cpp b/utils/local-engine/Parser/SerializedPlanParser.cpp index d1f26006208b..075a89ed9b1d 100644 --- a/utils/local-engine/Parser/SerializedPlanParser.cpp +++ b/utils/local-engine/Parser/SerializedPlanParser.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -11,12 +10,8 @@ #include #include #include -#include #include #include -#include -#include -#include #include #include #include @@ -59,7 +54,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NO_SUCH_DATA_PART; extern const int UNKNOWN_FUNCTION; - extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; } } @@ -78,10 +72,71 @@ void join(ActionsDAG::NodeRawConstPtrs v, char c, std::string & s) } } -bool isTypeMatched(const substrait::Type & substrait_type, const DataTypePtr & ch_type) +std::string typeName(const substrait::Type & type) { - const auto parsed_ch_type = SerializedPlanParser::parseType(substrait_type); - return parsed_ch_type->equals(*ch_type); + if (type.has_string()) + { + return "String"; + } + else if (type.has_i8()) + { + return "I8"; + } + else if (type.has_i16()) + { + return "I16"; + } + else if (type.has_i32()) + { + return "I32"; + } + else if (type.has_i64()) + { + return "I64"; + } + else if (type.has_fp32()) + { + return "FP32"; + } + else if (type.has_fp64()) + { + return "FP64"; + } + else if (type.has_bool_()) + { + return "Boolean"; + } + else if (type.has_date()) + { + return "Date"; + } + else if (type.has_timestamp()) + { + return "Timestamp"; + } + + throw Exception(ErrorCodes::UNKNOWN_TYPE, "unknown type {}", magic_enum::enum_name(type.kind_case())); +} + +bool isTypeSame(const substrait::Type & type, DataTypePtr data_type) +{ + static const std::map type_mapping + = {{"I8", "Int8"}, + {"I16", "Int16"}, + {"I32", "Int32"}, + {"I64", "Int64"}, + {"FP32", "Float32"}, + {"FP64", "Float64"}, + {"Date", "Date32"}, + {"Timestamp", "DateTime64(6)"}, + {"String", "String"}, + {"Boolean", "UInt8"}}; + + std::string type_name = typeName(type); + if (!type_mapping.contains(type_name)) + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type {}", type_name); + + return type_mapping.at(type_name) == data_type->getName(); } std::string getCastFunction(const substrait::Type & type) @@ -131,8 +186,6 @@ std::string getCastFunction(const substrait::Type & type) else throw Exception(ErrorCodes::UNKNOWN_TYPE, "doesn't support cast type {}", type.DebugString()); - /// TODO(taiyang-li): implement cast functions of other types - return ch_function_name; } @@ -257,116 +310,77 @@ Block SerializedPlanParser::parseNameStruct(const substrait::NamedStruct & struc return Block(*std::move(internal_cols)); } -DataTypePtr wrapNullableType(substrait::Type_Nullability nullable, DataTypePtr nested_type) -{ - return wrapNullableType(nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE, nested_type); -} - -DataTypePtr wrapNullableType(bool nullable, DataTypePtr nested_type) +static DataTypePtr wrapNullableType(substrait::Type_Nullability nullable, DataTypePtr nested_type) { - if (nullable) + if (nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE) + { return std::make_shared(nested_type); + } else + { return nested_type; + } } -DataTypePtr SerializedPlanParser::parseType(const substrait::Type & substrait_type) +DataTypePtr SerializedPlanParser::parseType(const substrait::Type & type) { - DataTypePtr ch_type; - if (substrait_type.has_bool_()) - { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.bool_().nullability(), ch_type); - } - else if (substrait_type.has_i8()) - { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.i8().nullability(), ch_type); - } - else if (substrait_type.has_i16()) - { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.i16().nullability(), ch_type); - } - else if (substrait_type.has_i32()) + DataTypePtr internal_type = nullptr; + auto & factory = DataTypeFactory::instance(); + if (type.has_bool_()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.i32().nullability(), ch_type); + internal_type = factory.get("UInt8"); + internal_type = wrapNullableType(type.bool_().nullability(), internal_type); } - else if (substrait_type.has_i64()) + else if (type.has_i8()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.i64().nullability(), ch_type); + internal_type = factory.get("Int8"); + internal_type = wrapNullableType(type.i8().nullability(), internal_type); } - else if (substrait_type.has_string() || substrait_type.has_binary()) + else if (type.has_i16()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.string().nullability(), ch_type); + internal_type = factory.get("Int16"); + internal_type = wrapNullableType(type.i16().nullability(), internal_type); } - else if (substrait_type.has_fp32()) + else if (type.has_i32()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.fp32().nullability(), ch_type); + internal_type = factory.get("Int32"); + internal_type = wrapNullableType(type.i32().nullability(), internal_type); } - else if (substrait_type.has_fp64()) + else if (type.has_i64()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.fp64().nullability(), ch_type); + internal_type = factory.get("Int64"); + internal_type = wrapNullableType(type.i64().nullability(), internal_type); } - else if (substrait_type.has_timestamp()) + else if (type.has_string()) { - ch_type = std::make_shared(6); - ch_type = wrapNullableType(substrait_type.timestamp().nullability(), ch_type); + internal_type = factory.get("String"); + internal_type = wrapNullableType(type.string().nullability(), internal_type); } - else if (substrait_type.has_date()) + else if (type.has_fp32()) { - ch_type = std::make_shared(); - ch_type = wrapNullableType(substrait_type.date().nullability(), ch_type); + internal_type = factory.get("Float32"); + internal_type = wrapNullableType(type.fp32().nullability(), internal_type); } - else if (substrait_type.has_decimal()) + else if (type.has_fp64()) { - UInt32 precision = substrait_type.decimal().precision(); - UInt32 scale = substrait_type.decimal().scale(); - if (precision <= DataTypeDecimal32::maxPrecision()) - ch_type = std::make_shared(precision, scale); - else if (precision <= DataTypeDecimal64::maxPrecision()) - ch_type = std::make_shared(precision, scale); - else if (precision <= DataTypeDecimal128::maxPrecision()) - ch_type = std::make_shared(precision, scale); - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support decimal type with precision {}", precision); - - ch_type = wrapNullableType(substrait_type.decimal().nullability(), ch_type); + internal_type = factory.get("Float64"); + internal_type = wrapNullableType(type.fp64().nullability(), internal_type); } - else if (substrait_type.has_struct_()) + else if (type.has_date()) { - assert(substrait_type.struct_().nullability() == substrait::Type_Nullability_NULLABILITY_REQUIRED); - - DataTypes ch_field_types(substrait_type.struct_().types().size()); - for (size_t i = 0; i < ch_field_types.size(); ++i) - ch_field_types[i] = std::move(parseType(substrait_type.struct_().types()[i])); - ch_type = std::make_shared(ch_field_types); + internal_type = factory.get("Date32"); + internal_type = wrapNullableType(type.date().nullability(), internal_type); } - else if (substrait_type.has_list()) + else if (type.has_timestamp()) { - assert(substrait_type.struct_().nullability() == substrait::Type_Nullability_NULLABILITY_REQUIRED); - - auto ch_nested_type = parseType(substrait_type.list().type()); - ch_type = std::make_shared(ch_nested_type); + internal_type = factory.get("DateTime64(6)"); + internal_type = wrapNullableType(type.timestamp().nullability(), internal_type); } - else if (substrait_type.has_map()) + else { - assert(substrait_type.map().nullability() == substrait::Type_Nullability_NULLABILITY_REQUIRED); - - auto ch_key_type = parseType(substrait_type.map().key()); - auto ch_val_type = parseType(substrait_type.map().value()); - ch_type = std::make_shared(ch_key_type, ch_val_type); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "doesn't support type {}", type.DebugString()); } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support type {}", substrait_type.DebugString()); - - /// TODO(taiyang-li): consider Time/IntervalYear/IntervalDay/TimestampTZ/UUID/FixedChar/VarChar/FixedBinary/UserDefined - return std::move(ch_type); + return internal_type; } QueryPlanPtr SerializedPlanParser::parse(std::unique_ptr plan) { @@ -567,7 +581,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel) for (size_t i = 0; i < measure_positions.size(); i++) { - if (!isTypeMatched(measure_types[i], source[measure_positions[i]].type)) + if (!isTypeSame(measure_types[i], source[measure_positions[i]].type)) { auto target_type = parseType(measure_types[i]); target[measure_positions[i]].type = target_type; @@ -796,8 +810,8 @@ std::string SerializedPlanParser::getFunctionName(const std::string & function_s { const auto & output_type = function.output_type(); auto args = function.arguments(); - auto pos = function_signature.find(':'); - auto function_name = function_signature.substr(0, pos); + auto function_name_idx = function_signature.find(':'); + auto function_name = function_signature.substr(0, function_name_idx); if (!SCALAR_FUNCTIONS.contains(function_name)) throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unsupported function {}", function_name); @@ -889,20 +903,18 @@ const ActionsDAG::Node * SerializedPlanParser::parseFunctionWithDAG( { required_columns.emplace_back(args[0]->result_name); } - if (function_signature.find("extract:", 0) != function_signature.npos) { // delete the first arg args.erase(args.begin()); } - auto function_builder = FunctionFactory::instance().get(function_name, this->context); std::string args_name; join(args, ',', args_name); result_name = function_name + "(" + args_name + ")"; const auto * function_node = &actions_dag->addFunction(function_builder, args, result_name); result_node = function_node; - if (!isTypeMatched(rel.scalar_function().output_type(), function_node->result_type)) + if (!isTypeSame(rel.scalar_function().output_type(), function_node->result_type)) { auto cast_function = getCastFunction(rel.scalar_function().output_type()); DB::ActionsDAG::NodeRawConstPtrs cast_args({function_node}); @@ -946,151 +958,234 @@ SerializedPlanParser::toFunctionNode(ActionsDAGPtr action_dag, const String & fu return function_node; } -std::pair SerializedPlanParser::parseLiteral(const substrait::Expression_Literal & literal) -{ - DataTypePtr type; - Field field; - - switch (literal.literal_type_case()) - { - case substrait::Expression_Literal::kFp64: { - type = std::make_shared(); - field = literal.fp64(); - break; - } - case substrait::Expression_Literal::kFp32: { - type = std::make_shared(); - field = literal.fp32(); - break; - } - case substrait::Expression_Literal::kString: { - type = std::make_shared(); - field = literal.string(); - break; - } - case substrait::Expression_Literal::kBinary: { - type = std::make_shared(); - field = literal.binary(); - break; - } - case substrait::Expression_Literal::kI64: { - type = std::make_shared(); - field = literal.i64(); - break; - } - case substrait::Expression_Literal::kI32: { - type = std::make_shared(); - field = literal.i32(); - break; - } - case substrait::Expression_Literal::kBoolean: { - type = std::make_shared(); - field = literal.boolean() ? UInt8(1) : UInt8(0); - break; - } - case substrait::Expression_Literal::kI16: { - type = std::make_shared(); - field = literal.i16(); - break; - } - case substrait::Expression_Literal::kI8: { - type = std::make_shared(); - field = literal.i8(); - break; - } - case substrait::Expression_Literal::kDate: { - type = std::make_shared(); - field = literal.date(); - break; - } - case substrait::Expression_Literal::kTimestamp: { - type = std::make_shared(6); - field = DecimalField(literal.timestamp(), 6); - break; - } - case substrait::Expression_Literal::kDecimal: { - UInt32 precision = literal.decimal().precision(); - UInt32 scale = literal.decimal().scale(); - const auto & bytes = literal.decimal().value(); - - if (precision <= DataTypeDecimal32::maxPrecision()) - { - type = std::make_shared(precision, scale); - auto value = *reinterpret_cast(bytes.data()); - field = DecimalField(value, scale); - } - else if (precision <= DataTypeDecimal64::maxPrecision()) - { - type = std::make_shared(precision, scale); - auto value = *reinterpret_cast(bytes.data()); - field = DecimalField(value, scale); - } - else if (precision <= DataTypeDecimal128::maxPrecision()) - { - type = std::make_shared(precision, scale); - auto value = *reinterpret_cast(bytes.data()); - field = DecimalField(value, scale); - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Spark doesn't support decimal type with precision {}", precision); - break; - } - /// TODO(taiyang-li) Other type: Struct/Map/List - case substrait::Expression_Literal::kList: { - /// TODO(taiyang-li) Implement empty list - if (literal.has_empty_list()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty list not support!"); - - DataTypePtr first_type; - std::tie(first_type, std::ignore) = parseLiteral(literal.list().values(0)); - - size_t list_len = literal.list().values_size(); - Array array(list_len); - for (size_t i = 0; i < list_len; ++i) - { - auto type_and_field = std::move(parseLiteral(literal.list().values(i))); - if (!first_type->equals(*type_and_field.first)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Literal list type mismatch:{} and {}", - first_type->getName(), - type_and_field.first->getName()); - array[i] = std::move(type_and_field.second); - } - - type = std::make_shared(first_type); - field = std::move(array); - break; - } - case substrait::Expression_Literal::kNull: { - type = parseType(literal.null()); - field = std::move(Field{}); - break; - } - default: { - throw Exception( - ErrorCodes::UNKNOWN_TYPE, "Unsupported spark literal type {}", magic_enum::enum_name(literal.literal_type_case())); - } - } - return std::make_pair(std::move(type), std::move(field)); -} - const ActionsDAG::Node * SerializedPlanParser::parseArgument(ActionsDAGPtr action_dag, const substrait::Expression & rel) { - auto add_column = [&](const DataTypePtr & type, const Field & field) -> auto - { - return &action_dag->addColumn(ColumnWithTypeAndName(type->createColumnConst(1, field), type, getUniqueName(toString(field)))); - }; - switch (rel.rex_type_case()) { case substrait::Expression::RexTypeCase::kLiteral: { - DataTypePtr type; - Field field; - std::tie(type, field) = parseLiteral(rel.literal()); - return add_column(type, field); + const auto & literal = rel.literal(); + switch (literal.literal_type_case()) + { + case substrait::Expression_Literal::kFp64: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.fp64()), type, getUniqueName(std::to_string(literal.fp64())))); + } + case substrait::Expression_Literal::kFp32: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.fp32()), type, getUniqueName(std::to_string(literal.fp32())))); + } + case substrait::Expression_Literal::kString: { + auto type = std::make_shared(); + return &action_dag->addColumn( + ColumnWithTypeAndName(type->createColumnConst(1, literal.string()), type, getUniqueName(literal.string()))); + } + case substrait::Expression_Literal::kI64: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.i64()), type, getUniqueName(std::to_string(literal.i64())))); + } + case substrait::Expression_Literal::kI32: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.i32()), type, getUniqueName(std::to_string(literal.i32())))); + } + case substrait::Expression_Literal::kBoolean: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.boolean() ? 1 : 0), type, getUniqueName(std::to_string(literal.boolean())))); + } + case substrait::Expression_Literal::kI16: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.i16()), type, getUniqueName(std::to_string(literal.i16())))); + } + case substrait::Expression_Literal::kI8: { + auto type = std::make_shared(); + return &action_dag->addColumn( + ColumnWithTypeAndName(type->createColumnConst(1, literal.i8()), type, getUniqueName(std::to_string(literal.i8())))); + } + case substrait::Expression_Literal::kDate: { + auto type = std::make_shared(); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, literal.date()), type, getUniqueName(std::to_string(literal.date())))); + } + case substrait::Expression_Literal::kTimestamp: { + auto type = std::make_shared(6); + auto field = DecimalField(literal.timestamp(), 6); + return &action_dag->addColumn(ColumnWithTypeAndName( + type->createColumnConst(1, field), type, getUniqueName(std::to_string(literal.timestamp())))); + } + case substrait::Expression_Literal::kList: { + SizeLimits limit; + if (literal.has_empty_list()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "empty list not support!"); + } + MutableColumnPtr values; + DataTypePtr type; + auto first_value = literal.list().values(0); + if (first_value.has_boolean()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).boolean() ? 1 : 0); + } + } + else if (first_value.has_i8()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).i8()); + } + } + else if (first_value.has_i16()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).i16()); + } + } + else if (first_value.has_i32()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).i32()); + } + } + else if (first_value.has_i64()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).i64()); + } + } + else if (first_value.has_fp32()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).fp32()); + } + } + else if (first_value.has_fp64()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).fp64()); + } + } + else if (first_value.has_date()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).date()); + } + } + else if (first_value.has_timestamp()) + { + type = std::make_shared(6); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + auto field = DecimalField(literal.list().values(i).timestamp(), 6); + values->insert(literal.list().values(i).timestamp()); + } + } + else if (first_value.has_string()) + { + type = std::make_shared(); + values = type->createColumn(); + for (int i = 0; i < literal.list().values_size(); ++i) + { + values->insert(literal.list().values(i).string()); + } + } + else + { + throw Exception( + ErrorCodes::UNKNOWN_TYPE, + "unsupported literal list type. {}", + magic_enum::enum_name(first_value.literal_type_case())); + } + auto set = std::make_shared(limit, true, false); + Block values_block; + auto name = getUniqueName("__set"); + values_block.insert(ColumnWithTypeAndName(std::move(values), type, name)); + set->setHeader(values_block.getColumnsWithTypeAndName()); + set->insertFromBlock(values_block.getColumnsWithTypeAndName()); + set->finishInsert(); + + auto arg = ColumnSet::create(set->getTotalRowCount(), set); + return &action_dag->addColumn(ColumnWithTypeAndName(std::move(arg), std::make_shared(), name)); + } + case substrait::Expression_Literal::kNull: { + DataTypePtr nested_type; + if (literal.null().has_i8()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_i16()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_i32()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_i64()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_bool_()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_fp32()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_fp64()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_date()) + { + nested_type = std::make_shared(); + } + else if (literal.null().has_timestamp()) + { + nested_type = std::make_shared(6); + } + else if (literal.null().has_string()) + { + nested_type = std::make_shared(); + } + auto type = std::make_shared(nested_type); + return &action_dag->addColumn(ColumnWithTypeAndName(type->createColumnConst(1, Field()), type, getUniqueName("null"))); + } + default: { + throw Exception( + ErrorCodes::UNKNOWN_TYPE, "unsupported constant type {}", magic_enum::enum_name(literal.literal_type_case())); + } + } } - case substrait::Expression::RexTypeCase::kSelection: { if (!rel.selection().has_direct_reference() || !rel.selection().direct_reference().has_struct_field()) { @@ -1099,7 +1194,6 @@ const ActionsDAG::Node * SerializedPlanParser::parseArgument(ActionsDAGPtr actio const auto * field = action_dag->getInputs()[rel.selection().direct_reference().struct_field().field()]; return action_dag->tryFindInIndex(field->result_name); } - case substrait::Expression::RexTypeCase::kCast: { if (!rel.cast().has_type() || !rel.cast().has_input()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Doesn't have type or input in cast node."); @@ -1130,7 +1224,6 @@ const ActionsDAG::Node * SerializedPlanParser::parseArgument(ActionsDAGPtr actio action_dag->addOrReplaceInIndex(*function_node); return function_node; } - case substrait::Expression::RexTypeCase::kIfThen: { const auto & if_then = rel.if_then(); auto function_multi_if = DB::FunctionFactory::instance().get("multiIf", this->context); @@ -1165,85 +1258,118 @@ const ActionsDAG::Node * SerializedPlanParser::parseArgument(ActionsDAGPtr actio action_dag->addOrReplaceInIndex(*function_node); return function_node; } - case substrait::Expression::RexTypeCase::kScalarFunction: { std::string result; std::vector useless; return parseFunctionWithDAG(rel, result, useless, action_dag, false); } - case substrait::Expression::RexTypeCase::kSingularOrList: { DB::ActionsDAG::NodeRawConstPtrs args; args.emplace_back(parseArgument(action_dag, rel.singular_or_list().value())); - - /// options should be non-empty and literals const auto & options = rel.singular_or_list().options(); + + SizeLimits limit; if (options.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty SingularOrList not supported"); + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "empty list not support!"); + } + ColumnPtr values; + DataTypePtr type; if (!options[0].has_literal()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Options of SingularOrList must have literal type"); - - DataTypePtr elem_type; - std::tie(elem_type, std::ignore) = parseLiteral(options[0].literal()); - - size_t options_len = options.size(); - MutableColumnPtr elem_column = elem_type->createColumn(); - elem_column->reserve(options_len); - for (size_t i = 0; i < options_len; ++i) { - if (!options[i].has_literal()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "in expression values must be the literal!"); - - auto type_and_field = std::move(parseLiteral(options[i].literal())); - if (!elem_type->equals(*type_and_field.first)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "SingularOrList options type mismatch:{} and {}", - elem_type->getName(), - type_and_field.first->getName()); - - elem_column->insert(type_and_field.second); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "in expression values must be the literal!"); } - - MutableColumns elem_columns; - elem_columns.emplace_back(std::move(elem_column)); - + auto first_value = options[0].literal(); + using FieldGetter = std::function; + auto fill_values = [options](DataTypePtr type, FieldGetter getter) -> ColumnPtr { + auto values = type->createColumn(); + for (const auto & v : options) + { + values->insert(getter(v.literal())); + } + return values; + }; + if (first_value.has_boolean()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.boolean() ? 1 : 0;}); + } + else if (first_value.has_i8()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.i8();}); + } + else if (first_value.has_i16()) + { + type = std::make_shared(); + values = type->createColumn(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.i16();}); + } + else if (first_value.has_i32()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.i32();}); + } + else if (first_value.has_i64()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.i64();}); + } + else if (first_value.has_fp32()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.fp32();}); + } + else if (first_value.has_fp64()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.fp64();}); + } + else if (first_value.has_date()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.date();}); + } + else if (first_value.has_string()) + { + type = std::make_shared(); + values = fill_values(type, [](substrait::Expression_Literal expr) -> Field {return expr.string();}); + } + else + { + throw Exception( + ErrorCodes::UNKNOWN_TYPE, + "unsupported literal list type. {}", + magic_enum::enum_name(first_value.literal_type_case())); + } + auto set = std::make_shared(limit, true, false); + Block values_block; auto name = getUniqueName("__set"); - Block elem_block; - elem_block.insert(ColumnWithTypeAndName(nullptr, elem_type, name)); - elem_block.setColumns(std::move(elem_columns)); + values_block.insert(ColumnWithTypeAndName(values, type, name)); + set->setHeader(values_block.getColumnsWithTypeAndName()); + set->insertFromBlock(values_block.getColumnsWithTypeAndName()); + set->finishInsert(); - SizeLimits limit; - auto elem_set = std::make_shared(limit, true, false); - elem_set->setHeader(elem_block.getColumnsWithTypeAndName()); - elem_set->insertFromBlock(elem_block.getColumnsWithTypeAndName()); - elem_set->finishInsert(); - - auto arg = ColumnSet::create(elem_set->getTotalRowCount(), elem_set); + auto arg = ColumnSet::create(set->getTotalRowCount(), set); args.emplace_back(&action_dag->addColumn(ColumnWithTypeAndName(std::move(arg), std::make_shared(), name))); const auto * function_node = toFunctionNode(action_dag, "in", args); action_dag->addOrReplaceInIndex(*function_node); return function_node; } - - default: + default: { throw Exception( - ErrorCodes::UNKNOWN_TYPE, - "Unsupported spark expression type {} : {}", - magic_enum::enum_name(rel.rex_type_case()), - rel.DebugString()); + ErrorCodes::BAD_ARGUMENTS, "unsupported arg type {} : {}", magic_enum::enum_name(rel.rex_type_case()), rel.DebugString()); + } } } QueryPlanPtr SerializedPlanParser::parse(const std::string & plan) { auto plan_ptr = std::make_unique(); - auto ok = plan_ptr->ParseFromString(plan); - if (!ok) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from string failed"); - - return std::move(parse(std::move(plan_ptr))); + plan_ptr->ParseFromString(plan); + LOG_DEBUG(&Poco::Logger::get("SerializedPlanParser"), "parse plan \n{}", plan_ptr->DebugString()); + return parse(std::move(plan_ptr)); } void SerializedPlanParser::initFunctionEnv() { diff --git a/utils/local-engine/Parser/SerializedPlanParser.h b/utils/local-engine/Parser/SerializedPlanParser.h index 0da7363d4382..22a2f5428f39 100644 --- a/utils/local-engine/Parser/SerializedPlanParser.h +++ b/utils/local-engine/Parser/SerializedPlanParser.h @@ -78,9 +78,6 @@ struct QueryContext std::shared_ptr custom_storage_merge_tree; }; -DataTypePtr wrapNullableType(substrait::Type_Nullability nullable, DataTypePtr nested_type); -DataTypePtr wrapNullableType(bool nullable, DataTypePtr nested_type); - class SerializedPlanParser { public: @@ -135,8 +132,6 @@ class SerializedPlanParser void wrapNullable(std::vector columns, ActionsDAGPtr actionsDag); std::string getUniqueName(const std::string & name) { return name + "_" + std::to_string(name_no++); } - static std::pair parseLiteral(const substrait::Expression_Literal & literal); - static Aggregator::Params getAggregateParam(const Block & header, const ColumnNumbers & keys, const AggregateDescriptions & aggregates) { Settings settings; @@ -174,7 +169,7 @@ class SerializedPlanParser struct SparkBuffer { - char * address; + uint8_t * address; size_t size; }; diff --git a/utils/local-engine/Parser/SparkRowToCHColumn.cpp b/utils/local-engine/Parser/SparkRowToCHColumn.cpp index 7eb9a7bd4187..5b6c46812c78 100644 --- a/utils/local-engine/Parser/SparkRowToCHColumn.cpp +++ b/utils/local-engine/Parser/SparkRowToCHColumn.cpp @@ -2,11 +2,6 @@ #include #include #include -#include -#include -#include -#include -#include #include namespace DB @@ -14,7 +9,6 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TYPE; - extern const int LOGICAL_ERROR; } } @@ -27,356 +21,139 @@ jmethodID SparkRowToCHColumn::spark_row_interator_hasNext = nullptr; jmethodID SparkRowToCHColumn::spark_row_interator_next = nullptr; jmethodID SparkRowToCHColumn::spark_row_iterator_nextBatch = nullptr; -ALWAYS_INLINE static void writeRowToColumns(std::vector & columns, const SparkRowReader & spark_row_reader) +int64_t getStringColumnTotalSize(int ordinal, SparkRowInfo & spark_row_info) { - auto num_fields = columns.size(); - const auto & field_types = spark_row_reader.getFieldTypes(); - for (size_t i = 0; i < num_fields; i++) + SparkRowReader reader(spark_row_info.getNumCols()); + int64_t size = 0; + for (int64_t i = 0; i < spark_row_info.getNumRows(); i++) { - if (spark_row_reader.supportRawData(i)) - { - const StringRef str{std::move(spark_row_reader.getStringRef(i))}; - columns[i]->insertData(str != EMPTY_STRING_REF ? str.data : nullptr, str.size); - } - else - columns[i]->insert(spark_row_reader.getField(i)); + reader.pointTo( + reinterpret_cast(spark_row_info.getBufferAddress() + spark_row_info.getOffsets()[i]), spark_row_info.getLengths()[i]); + size += (reader.getStringSize(ordinal) + 1); } + return size; } -std::unique_ptr -SparkRowToCHColumn::convertSparkRowInfoToCHColumn(const SparkRowInfo & spark_row_info, const Block & header) +void writeRowToColumns(std::vector & columns,std::vector& types, SparkRowReader & spark_row_reader) { - auto block = std::make_unique(); - *block = std::move(header.cloneEmpty()); - MutableColumns mutable_columns{std::move(block->mutateColumns())}; - const auto num_rows = spark_row_info.getNumRows(); - for (size_t col_i = 0; col_i < header.columns(); ++col_i) - mutable_columns[col_i]->reserve(num_rows); - - DataTypes types{std::move(header.getDataTypes())}; - SparkRowReader row_reader(types); - for (int64_t i = 0; i < num_rows; i++) + int32_t num_fields = columns.size(); + [[maybe_unused]] bool is_nullable = false; + for (int32_t i = 0; i < num_fields; i++) { - row_reader.pointTo(spark_row_info.getBufferAddress() + spark_row_info.getOffsets()[i], spark_row_info.getLengths()[i]); - writeRowToColumns(mutable_columns, row_reader); - } - block->setColumns(std::move(mutable_columns)); - return std::move(block); -} - -void SparkRowToCHColumn::appendSparkRowToCHColumn(SparkRowToCHColumnHelper & helper, char * buffer, int32_t length) -{ - SparkRowReader row_reader(helper.data_types); - row_reader.pointTo(buffer, length); - writeRowToColumns(helper.mutable_columns, row_reader); -} - -Block * SparkRowToCHColumn::getBlock(SparkRowToCHColumnHelper & helper) -{ - auto * block = new Block(); - *block = std::move(helper.header.cloneEmpty()); - block->setColumns(std::move(helper.mutable_columns)); - return block; -} - -VariableLengthDataReader::VariableLengthDataReader(const DataTypePtr & type_) - : type(type_), type_without_nullable(removeNullable(type)), which(type_without_nullable) -{ - if (!BackingDataLengthCalculator::isVariableLengthDataType(type_without_nullable)) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", type->getName()); -} - -Field VariableLengthDataReader::read(const char *buffer, size_t length) const -{ - if (which.isStringOrFixedString()) - return std::move(readString(buffer, length)); - - if (which.isDecimal128()) - return std::move(readDecimal(buffer, length)); - - if (which.isArray()) - return std::move(readArray(buffer, length)); - - if (which.isMap()) - return std::move(readMap(buffer, length)); - - if (which.isTuple()) - return std::move(readStruct(buffer, length)); - - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", type->getName()); -} - -StringRef VariableLengthDataReader::readUnalignedBytes(const char * buffer, size_t length) const -{ - return {buffer, length}; -} - -Field VariableLengthDataReader::readDecimal(const char * buffer, size_t length) const -{ - assert(sizeof(Decimal128) == length); - - Decimal128 value; - memcpy(&value, buffer, length); - - const auto * decimal128_type = typeid_cast(type_without_nullable.get()); - return std::move(DecimalField(value, decimal128_type->getScale())); -} - -Field VariableLengthDataReader::readString(const char * buffer, size_t length) const -{ - String str(buffer, length); - return std::move(Field(std::move(str))); -} - -Field VariableLengthDataReader::readArray(const char * buffer, [[maybe_unused]] size_t length) const -{ - /// 内存布局:numElements(8B) | null_bitmap(与numElements成正比) | values(每个值长度与类型有关) | backing data - /// Read numElements - int64_t num_elems = 0; - memcpy(&num_elems, buffer, 8); - if (num_elems == 0) - return Array(); - - /// Skip null_bitmap - const auto len_null_bitmap = calculateBitSetWidthInBytes(num_elems); + WhichDataType which(columns[i]->getDataType()); + if (which.isNullable()) + { + const auto * nullable = checkAndGetDataType(types[i].get()); + which = WhichDataType(nullable->getNestedType()); + is_nullable = true; + } - /// Read values - const auto * array_type = typeid_cast(type.get()); - const auto & nested_type = array_type->getNestedType(); - const auto elem_size = BackingDataLengthCalculator::getArrayElementSize(nested_type); - const auto len_values = roundNumberOfBytesToNearestWord(elem_size * num_elems); - Array array; - array.reserve(num_elems); + if (spark_row_reader.isNullAt(i)) { + assert(is_nullable); + ColumnNullable & column = assert_cast(*columns[i]); + column.insertData(nullptr, 0); + continue; + } - if (BackingDataLengthCalculator::isFixedLengthDataType(removeNullable(nested_type))) - { - FixedLengthDataReader reader(nested_type); - for (int64_t i = 0; i < num_elems; ++i) + if (which.isUInt8()) { - if (isBitSet(buffer + 8, i)) - { - array.emplace_back(std::move(Null{})); - } - else - { - const auto elem = reader.read(buffer + 8 + len_null_bitmap + i * elem_size); - array.emplace_back(elem); - } + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(uint8_t)); } - } - else if (BackingDataLengthCalculator::isVariableLengthDataType(removeNullable(nested_type))) - { - VariableLengthDataReader reader(nested_type); - for (int64_t i = 0; i < num_elems; ++i) + else if (which.isInt8()) { - if (isBitSet(buffer + 8, i)) - { - array.emplace_back(std::move(Null{})); - } - else - { - int64_t offset_and_size = 0; - memcpy(&offset_and_size, buffer + 8 + len_null_bitmap + i * 8, 8); - const int64_t offset = BackingDataLengthCalculator::extractOffset(offset_and_size); - const int64_t size = BackingDataLengthCalculator::extractSize(offset_and_size); - - const auto elem = reader.read(buffer + offset, size); - array.emplace_back(elem); - } + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(int8_t)); } - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", nested_type->getName()); - - return std::move(array); -} - -Field VariableLengthDataReader::readMap(const char * buffer, size_t length) const -{ - /// 内存布局:Length of UnsafeArrayData of key(8B) | UnsafeArrayData of key | UnsafeArrayData of value - /// Read Length of UnsafeArrayData of key - int64_t key_array_size = 0; - memcpy(&key_array_size, buffer, 8); - if (key_array_size == 0) - return std::move(Map()); - - /// Read UnsafeArrayData of keys - const auto * map_type = typeid_cast(type.get()); - const auto & key_type = map_type->getKeyType(); - const auto key_array_type = std::make_shared(key_type); - VariableLengthDataReader key_reader(key_array_type); - auto key_field = key_reader.read(buffer + 8, key_array_size); - auto & key_array = key_field.safeGet(); - - /// Read UnsafeArrayData of values - const auto & val_type = map_type->getValueType(); - const auto val_array_type = std::make_shared(val_type); - VariableLengthDataReader val_reader(val_array_type); - auto val_field = val_reader.read(buffer + 8 + key_array_size, length - 8 - key_array_size); - auto & val_array = val_field.safeGet(); - - /// Construct map in CH way [(k1, v1), (k2, v2), ...] - if (key_array.size() != val_array.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key size {} not equal to value size {} in map", key_array.size(), val_array.size()); - Map map(key_array.size()); - for (size_t i = 0; i < key_array.size(); ++i) - { - Tuple tuple(2); - tuple[0] = std::move(key_array[i]); - tuple[1] = std::move(val_array[i]); - - map[i] = std::move(tuple); - } - return std::move(map); -} - -Field VariableLengthDataReader::readStruct(const char * buffer, size_t /*length*/) const -{ - /// 内存布局:null_bitmap(字节数与字段数成正比) | values(num_fields * 8B) | backing data - const auto * tuple_type = typeid_cast(type.get()); - const auto & field_types = tuple_type->getElements(); - const auto num_fields = field_types.size(); - if (num_fields == 0) - return std::move(Tuple()); - - const auto len_null_bitmap = calculateBitSetWidthInBytes(num_fields); - - Tuple tuple(num_fields); - for (size_t i=0; iinsertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(int16_t)); } - - if (BackingDataLengthCalculator::isFixedLengthDataType(removeNullable(field_type))) + else if (which.isInt32()) { - FixedLengthDataReader reader(field_type); - tuple[i] = std::move(reader.read(buffer + len_null_bitmap + i * 8)); + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(int32_t)); } - else if (BackingDataLengthCalculator::isVariableLengthDataType(removeNullable(field_type))) + else if (which.isInt64() || which.isDateTime64()) { - int64_t offset_and_size = 0; - memcpy(&offset_and_size, buffer + len_null_bitmap + i * 8, 8); - const int64_t offset = BackingDataLengthCalculator::extractOffset(offset_and_size); - const int64_t size = BackingDataLengthCalculator::extractSize(offset_and_size); - - VariableLengthDataReader reader(field_type); - tuple[i] = std::move(reader.read(buffer + offset, size)); + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(int64_t)); + } + else if (which.isFloat32()) + { + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(float_t)); + } + else if (which.isFloat64()) + { + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(double_t)); + } + else if (which.isDate() || which.isUInt16()) + { + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(uint16_t)); + } + else if (which.isDate32() || which.isUInt32()) + { + columns[i]->insertData(spark_row_reader.getRawDataForFixedNumber(i), sizeof(uint32_t)); + } + else if (which.isString()) + { + StringRef data = spark_row_reader.getString(i); + columns[i]->insertData(data.data, data.size); } else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", field_type->getName()); + { + throw Exception(ErrorCodes::UNKNOWN_TYPE, "doesn't support type {} convert from spark row to ch columnar" , + magic_enum::enum_name(columns[i]->getDataType())); + } } - return std::move(tuple); -} - -FixedLengthDataReader::FixedLengthDataReader(const DataTypePtr & type_) - : type(type_), type_without_nullable(removeNullable(type)), which(type_without_nullable) -{ - if (!BackingDataLengthCalculator::isFixedLengthDataType(type_without_nullable) || !type_without_nullable->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::UNKNOWN_TYPE, "VariableLengthDataReader doesn't support type {}", type->getName()); - - value_size = type_without_nullable->getSizeOfValueInMemory(); } -StringRef FixedLengthDataReader::unsafeRead(const char * buffer) const -{ - return {buffer, value_size}; -} - -Field FixedLengthDataReader::read(const char * buffer) const +std::unique_ptr +local_engine::SparkRowToCHColumn::convertSparkRowInfoToCHColumn(local_engine::SparkRowInfo & spark_row_info, DB::Block & header) { - if (which.isUInt8()) + auto columns_list = std::make_unique(); + columns_list->reserve(header.columns()); + std::vector mutable_columns; + std::vector types; + for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { - UInt8 value = 0; - memcpy(&value, buffer, 1); - return value; + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); + MutableColumnPtr read_column = header_column.type->createColumn(); + read_column->reserve(spark_row_info.getNumRows()); + mutable_columns.push_back(std::move(read_column)); + types.push_back(header_column.type); } - - if (which.isUInt16() || which.isDate()) + SparkRowReader row_reader(header.columns()); + for (int64_t i = 0; i < spark_row_info.getNumRows(); i++) { - UInt16 value = 0; - memcpy(&value, buffer, 2); - return value; + row_reader.pointTo( + reinterpret_cast(spark_row_info.getBufferAddress() + spark_row_info.getOffsets()[i]), spark_row_info.getLengths()[i]); + writeRowToColumns(mutable_columns, types, row_reader); } - - if (which.isUInt32()) + auto block = std::make_unique(*std::move(columns_list)); + for (size_t column_i = 0, columns = mutable_columns.size(); column_i < columns; ++column_i) { - UInt32 value = 0; - memcpy(&value, buffer, 4); - return value; + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); + ColumnWithTypeAndName column(std::move(mutable_columns[column_i]), header_column.type, header_column.name); + block->insert(column); } + mutable_columns.clear(); + return block; +} - if (which.isUInt64()) - { - UInt64 value = 0; - memcpy(&value, buffer, 8); - return value; - } - - if (which.isInt8()) - { - Int8 value = 0; - memcpy(&value, buffer, 1); - return value; - } - - if (which.isInt16()) - { - Int16 value = 0; - memcpy(&value, buffer, 2); - return value; - } - - if (which.isInt32() || which.isDate32()) - { - Int32 value = 0; - memcpy(&value, buffer, 4); - return value; - } - - if (which.isInt64()) - { - Int64 value = 0; - memcpy(&value, buffer, 8); - return value; - } - - if (which.isFloat32()) - { - Float32 value = 0.0; - memcpy(&value, buffer, 4); - return value; - } - - if (which.isFloat64()) - { - Float64 value = 0.0; - memcpy(&value, buffer, 8); - return value; - } - - if (which.isDecimal32()) - { - Decimal32 value = 0; - memcpy(&value, buffer, 4); - - const auto * decimal32_type = typeid_cast(type_without_nullable.get()); - return std::move(DecimalField{value, decimal32_type->getScale()}); - } +void local_engine::SparkRowToCHColumn::appendSparkRowToCHColumn(SparkRowToCHColumnHelper & helper, int64_t address, int32_t size) +{ + SparkRowReader row_reader(helper.header->columns()); + row_reader.pointTo(address, size); + writeRowToColumns(*helper.cols, *helper.typePtrs, row_reader); +} - if (which.isDecimal64() || which.isDateTime64()) +Block * local_engine::SparkRowToCHColumn::getWrittenBlock(SparkRowToCHColumnHelper & helper) +{ + auto * block = new Block(); + for (size_t column_i = 0, columns = helper.cols->size(); column_i < columns; ++column_i) { - Decimal64 value = 0; - memcpy(&value, buffer, 8); - - UInt32 scale = which.isDecimal64() ? typeid_cast(type_without_nullable.get())->getScale() - : typeid_cast(type_without_nullable.get())->getScale(); - return std::move(DecimalField{value, scale}); + const ColumnWithTypeAndName & header_column = helper.header->getByPosition(column_i); + ColumnWithTypeAndName column(std::move(helper.cols->operator[](column_i)), header_column.type, header_column.name); + block->insert(column); } - throw Exception(ErrorCodes::UNKNOWN_TYPE, "FixedLengthDataReader doesn't support type {}", type->getName()); + return block; } } diff --git a/utils/local-engine/Parser/SparkRowToCHColumn.h b/utils/local-engine/Parser/SparkRowToCHColumn.h index 1c84c69269ab..c303bdedf7a8 100644 --- a/utils/local-engine/Parser/SparkRowToCHColumn.h +++ b/utils/local-engine/Parser/SparkRowToCHColumn.h @@ -5,62 +5,123 @@ #include #include #include -#include -#include #include #include #include -#include #include -namespace DB -{ -namespace ErrorCodes -{ - extern const int UNKNOWN_TYPE; - extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; -} -} namespace local_engine { using namespace DB; using namespace std; + + struct SparkRowToCHColumnHelper { - DataTypes data_types; - Block header; - MutableColumns mutable_columns; - - SparkRowToCHColumnHelper(vector & names, vector & types) - : data_types(names.size()) + SparkRowToCHColumnHelper(vector& names, vector& types, vector& isNullables) { - assert(names.size() == types.size()); - - ColumnsWithTypeAndName columns(names.size()); + internal_cols = std::make_unique>(); + internal_cols->reserve(names.size()); + typePtrs = std::make_unique>(); + typePtrs->reserve(names.size()); for (size_t i = 0; i < names.size(); ++i) { - data_types[i] = parseType(types[i]); - columns[i] = std::move(ColumnWithTypeAndName(data_types[i], names[i])); + const auto & name = names[i]; + const auto & type = types[i]; + const bool is_nullable = isNullables[i]; + auto data_type = parseType(type, is_nullable); + internal_cols->push_back(ColumnWithTypeAndName(data_type, name)); + typePtrs->push_back(data_type); } - - header = std::move(Block(columns)); - resetMutableColumns(); + header = std::make_shared(*std::move(internal_cols)); + resetWrittenColumns(); } - ~SparkRowToCHColumnHelper() = default; + unique_ptr> internal_cols; //for headers + unique_ptr> cols; + unique_ptr> typePtrs; + shared_ptr header; - void resetMutableColumns() + void resetWrittenColumns() { - mutable_columns = std::move(header.mutateColumns()); + cols = make_unique>(); + for (size_t i = 0; i < internal_cols->size(); i++) + { + cols->push_back(internal_cols->at(i).type->createColumn()); + } } - static DataTypePtr parseType(const string & type) + static DataTypePtr inline wrapNullableType(bool isNullable, DataTypePtr nested_type) { - auto substrait_type = std::make_unique(); - auto ok = substrait_type->ParseFromString(type); - if (!ok) - throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Type from string failed"); - return std::move(SerializedPlanParser::parseType(*substrait_type)); + if (isNullable) + { + return std::make_shared(nested_type); + } + else + { + return nested_type; + } + } + + //parse Spark type name to CH DataType + DataTypePtr parseType(const string & type, const bool isNullable) + { + DataTypePtr internal_type = nullptr; + auto & factory = DataTypeFactory::instance(); + if ("boolean" == type) + { + internal_type = factory.get("UInt8"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("byte" == type) + { + internal_type = factory.get("Int8"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("short" == type) + { + internal_type = factory.get("Int16"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("integer" == type) + { + internal_type = factory.get("Int32"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("long" == type) + { + internal_type = factory.get("Int64"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("string" == type) + { + internal_type = factory.get("String"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("float" == type) + { + internal_type = factory.get("Float32"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("double" == type) + { + internal_type = factory.get("Float64"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("date" == type) + { + internal_type = factory.get("Date32"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else if ("timestamp" == type) + { + internal_type = factory.get("DateTime64(6)"); + internal_type = wrapNullableType(isNullable, internal_type); + } + else + throw Exception(0, "doesn't support spark type {}", type); + + return internal_type; } }; @@ -73,13 +134,12 @@ class SparkRowToCHColumn static jmethodID spark_row_iterator_nextBatch; // case 1: rows are batched (this is often directly converted from Block) - static std::unique_ptr convertSparkRowInfoToCHColumn(const SparkRowInfo & spark_row_info, const Block & header); + static std::unique_ptr convertSparkRowInfoToCHColumn(SparkRowInfo & spark_row_info, Block & header); // case 2: provided with a sequence of spark UnsafeRow, convert them to a Block - static Block * - convertSparkRowItrToCHColumn(jobject java_iter, vector & names, vector & types) + static Block* convertSparkRowItrToCHColumn(jobject java_iter, vector& names, vector& types, vector& isNullables) { - SparkRowToCHColumnHelper helper(names, types); + SparkRowToCHColumnHelper helper(names, types, isNullables); int attached; JNIEnv * env = JNIUtils::getENV(&attached); @@ -93,110 +153,33 @@ class SparkRowToCHColumn while (len > 0) { rows_buf_ptr += 4; - appendSparkRowToCHColumn(helper, rows_buf_ptr, len); + appendSparkRowToCHColumn(helper, reinterpret_cast(rows_buf_ptr), len); rows_buf_ptr += len; len = *(reinterpret_cast(rows_buf_ptr)); } // Try to release reference. env->DeleteLocalRef(rows_buf); } - return getBlock(helper); + return getWrittenBlock(helper); } - static void freeBlock(Block * block) - { - delete block; - block = nullptr; - } - -private: - static void appendSparkRowToCHColumn(SparkRowToCHColumnHelper & helper, char * buffer, int32_t length); - static Block * getBlock(SparkRowToCHColumnHelper & helper); -}; - -class VariableLengthDataReader -{ -public: - explicit VariableLengthDataReader(const DataTypePtr& type_); - virtual ~VariableLengthDataReader() = default; - - virtual Field read(const char * buffer, size_t length) const; - virtual StringRef readUnalignedBytes(const char * buffer, size_t length) const; + static void freeBlock(Block * block) { delete block; } private: - virtual Field readDecimal(const char * buffer, size_t length) const; - virtual Field readString(const char * buffer, size_t length) const; - virtual Field readArray(const char * buffer, size_t length) const; - virtual Field readMap(const char * buffer, size_t length) const; - virtual Field readStruct(const char * buffer, size_t length) const; - - const DataTypePtr type; - const DataTypePtr type_without_nullable; - const WhichDataType which; + static void appendSparkRowToCHColumn(SparkRowToCHColumnHelper & helper, int64_t address, int32_t size); + static Block* getWrittenBlock(SparkRowToCHColumnHelper & helper); }; -class FixedLengthDataReader -{ -public: - explicit FixedLengthDataReader(const DB::DataTypePtr & type_); - virtual ~FixedLengthDataReader() = default; - - virtual Field read(const char * buffer) const; - virtual StringRef unsafeRead(const char * buffer) const; - -private: - const DB::DataTypePtr type; - const DB::DataTypePtr type_without_nullable; - const DB::WhichDataType which; - size_t value_size; -}; class SparkRowReader { public: - explicit SparkRowReader(const DataTypes & field_types_) - : field_types(field_types_) - , num_fields(field_types.size()) - , bit_set_width_in_bytes(calculateBitSetWidthInBytes(num_fields)) - , field_offsets(num_fields) - , support_raw_datas(num_fields) - , fixed_length_data_readers(num_fields) - , variable_length_data_readers(num_fields) - { - for (auto ordinal = 0; ordinal < num_fields; ++ordinal) - { - const auto type_without_nullable = removeNullable(field_types[ordinal]); - field_offsets[ordinal] = bit_set_width_in_bytes + ordinal * 8L; - support_raw_datas[ordinal] = BackingDataLengthCalculator::isDataTypeSupportRawData(type_without_nullable); - if (BackingDataLengthCalculator::isFixedLengthDataType(type_without_nullable)) - fixed_length_data_readers[ordinal] = std::make_shared(field_types[ordinal]); - else if (BackingDataLengthCalculator::isVariableLengthDataType(type_without_nullable)) - variable_length_data_readers[ordinal] = std::make_shared(field_types[ordinal]); - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "SparkRowReader doesn't support type {}", field_types[ordinal]->getName()); - } - } - - const DataTypes & getFieldTypes() const - { - return field_types; - } - - bool supportRawData(int ordinal) const - { - assertIndexIsValid(ordinal); - return support_raw_datas[ordinal]; - } - - std::shared_ptr getFixedLengthDataReader(int ordinal) const + bool isSet(int index) const { - assertIndexIsValid(ordinal); - return fixed_length_data_readers[ordinal]; - } - - std::shared_ptr getVariableLengthDataReader(int ordinal) const - { - assertIndexIsValid(ordinal); - return variable_length_data_readers[ordinal]; + assert(index >= 0); + int64_t mask = 1 << (index & 63); + int64_t word_offset = base_offset + static_cast(index >> 6) * 8L; + int64_t word = *reinterpret_cast(word_offset); + return (word & mask) != 0; } void assertIndexIsValid([[maybe_unused]] int index) const @@ -208,153 +191,103 @@ class SparkRowReader bool isNullAt(int ordinal) const { assertIndexIsValid(ordinal); - return isBitSet(buffer, ordinal); + return isSet(ordinal); } - const char* getRawDataForFixedNumber(int ordinal) const + char* getRawDataForFixedNumber(int ordinal) { assertIndexIsValid(ordinal); - return reinterpret_cast(getFieldOffset(ordinal)); + return reinterpret_cast(getFieldOffset(ordinal)); } - int8_t getByte(int ordinal) const + int8_t getByte(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - uint8_t getUnsignedByte(int ordinal) const + uint8_t getUnsignedByte(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - int16_t getShort(int ordinal) const + + int16_t getShort(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - uint16_t getUnsignedShort(int ordinal) const + uint16_t getUnsignedShort(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - int32_t getInt(int ordinal) const + int32_t getInt(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - uint32_t getUnsignedInt(int ordinal) const + uint32_t getUnsignedInt(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - int64_t getLong(int ordinal) const + int64_t getLong(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - float_t getFloat(int ordinal) const + float_t getFloat(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - double_t getDouble(int ordinal) const + double_t getDouble(int ordinal) { assertIndexIsValid(ordinal); - return *reinterpret_cast(getFieldOffset(ordinal)); + return *reinterpret_cast(getFieldOffset(ordinal)); } - StringRef getString(int ordinal) const + StringRef getString(int ordinal) { assertIndexIsValid(ordinal); int64_t offset_and_size = getLong(ordinal); int32_t offset = static_cast(offset_and_size >> 32); int32_t size = static_cast(offset_and_size); - return StringRef(reinterpret_cast(this->buffer + offset), size); + return StringRef(reinterpret_cast(this->base_offset + offset), size); } - int32_t getStringSize(int ordinal) const + int32_t getStringSize(int ordinal) { assertIndexIsValid(ordinal); return static_cast(getLong(ordinal)); } - void pointTo(const char * buffer_, int32_t length_) + void pointTo(int64_t base_offset_, int32_t size_in_bytes_) { - buffer = buffer_; - length = length_; - } - - StringRef getStringRef(int ordinal) const - { - assertIndexIsValid(ordinal); - if (!support_raw_datas[ordinal]) - throw Exception( - ErrorCodes::UNKNOWN_TYPE, "SparkRowReader::getStringRef doesn't support type {}", field_types[ordinal]->getName()); - - if (isNullAt(ordinal)) - return EMPTY_STRING_REF; - - const auto & fixed_length_data_reader = fixed_length_data_readers[ordinal]; - const auto & variable_length_data_reader = variable_length_data_readers[ordinal]; - if (fixed_length_data_reader) - return std::move(fixed_length_data_reader->unsafeRead(getFieldOffset(ordinal))); - else if (variable_length_data_reader) - { - int64_t offset_and_size = 0; - memcpy(&offset_and_size, buffer + bit_set_width_in_bytes + ordinal * 8, 8); - const int64_t offset = BackingDataLengthCalculator::extractOffset(offset_and_size); - const int64_t size = BackingDataLengthCalculator::extractSize(offset_and_size); - return std::move(variable_length_data_reader->readUnalignedBytes(buffer + offset, size)); - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "SparkRowReader::getStringRef doesn't support type {}", field_types[ordinal]->getName()); + this->base_offset = base_offset_; + this->size_in_bytes = size_in_bytes_; } - Field getField(int ordinal) const + explicit SparkRowReader(int32_t numFields) : num_fields(numFields) { - assertIndexIsValid(ordinal); - - if (isNullAt(ordinal)) - return std::move(Null{}); - - const auto & fixed_length_data_reader = fixed_length_data_readers[ordinal]; - const auto & variable_length_data_reader = variable_length_data_readers[ordinal]; - - if (fixed_length_data_reader) - return std::move(fixed_length_data_reader->read(getFieldOffset(ordinal))); - else if (variable_length_data_reader) - { - int64_t offset_and_size = 0; - memcpy(&offset_and_size, buffer + bit_set_width_in_bytes + ordinal * 8, 8); - const int64_t offset = BackingDataLengthCalculator::extractOffset(offset_and_size); - const int64_t size = BackingDataLengthCalculator::extractSize(offset_and_size); - return std::move(variable_length_data_reader->read(buffer + offset, size)); - } - else - throw Exception(ErrorCodes::UNKNOWN_TYPE, "SparkRowReader::getField doesn't support type {}", field_types[ordinal]->getName()); + this->bit_set_width_in_bytes = local_engine::calculateBitSetWidthInBytes(numFields); } private: - const char * getFieldOffset(int ordinal) const { return buffer + field_offsets[ordinal]; } - - const DataTypes field_types; - const int32_t num_fields; - const int32_t bit_set_width_in_bytes; - std::vector field_offsets; - std::vector support_raw_datas; - std::vector> fixed_length_data_readers; - std::vector> variable_length_data_readers; - - const char * buffer; - int32_t length; + int64_t getFieldOffset(int ordinal) const { return base_offset + bit_set_width_in_bytes + ordinal * 8L; } + + int64_t base_offset; + [[maybe_unused]] int32_t num_fields; + int32_t size_in_bytes; + int32_t bit_set_width_in_bytes; }; } diff --git a/utils/local-engine/jni/jni_common.cpp b/utils/local-engine/jni/jni_common.cpp index 7df7d34c841b..a2423eb7e170 100644 --- a/utils/local-engine/jni/jni_common.cpp +++ b/utils/local-engine/jni/jni_common.cpp @@ -66,12 +66,4 @@ jstring charTojstring(JNIEnv* env, const char* pat) { env->DeleteLocalRef(encoding); return result; } - -jbyteArray stringTojbyteArray(JNIEnv* env, const std::string & str) { - const auto * ptr = reinterpret_cast(str.c_str()) ; - jbyteArray jarray = env->NewByteArray(str.size()); - env->SetByteArrayRegion(jarray, 0, str.size(), ptr); - return jarray; -} - } diff --git a/utils/local-engine/jni/jni_common.h b/utils/local-engine/jni/jni_common.h index 13afc8120f97..02d92cd154ca 100644 --- a/utils/local-engine/jni/jni_common.h +++ b/utils/local-engine/jni/jni_common.h @@ -2,7 +2,6 @@ #include #include #include -#include #include namespace DB @@ -25,8 +24,6 @@ jmethodID GetStaticMethodID(JNIEnv * env, jclass this_class, const char * name, jstring charTojstring(JNIEnv* env, const char* pat); -jbyteArray stringTojbyteArray(JNIEnv* env, const std::string & str); - #define LOCAL_ENGINE_JNI_JMETHOD_START #define LOCAL_ENGINE_JNI_JMETHOD_END(env) \ if ((env)->ExceptionCheck())\ diff --git a/utils/local-engine/local_engine_jni.cpp b/utils/local-engine/local_engine_jni.cpp index 185a95b41b4f..7ebd8dd1e256 100644 --- a/utils/local-engine/local_engine_jni.cpp +++ b/utils/local-engine/local_engine_jni.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -44,7 +43,7 @@ std::vector stringSplit(const std::string & str, char delim) } } -DB::ColumnWithTypeAndName inline getColumnFromColumnVector(JNIEnv * /*env*/, jobject /*obj*/, jlong block_address, jint column_position) +DB::ColumnWithTypeAndName inline getColumnFromColumnVector(JNIEnv * /*env*/, jobject obj, jlong block_address, jint column_position) { DB::Block * block = reinterpret_cast(block_address); return block->getByPosition(column_position); @@ -480,15 +479,80 @@ jint Java_io_glutenproject_vectorized_CHNativeBlock_nativeNumColumns(JNIEnv * en LOCAL_ENGINE_JNI_METHOD_END(env, -1) } -jbyteArray Java_io_glutenproject_vectorized_CHNativeBlock_nativeColumnType(JNIEnv * env, jobject /*obj*/, jlong block_address, jint position) +jstring Java_io_glutenproject_vectorized_CHNativeBlock_nativeColumnType(JNIEnv * env, jobject /*obj*/, jlong block_address, jint position) { LOCAL_ENGINE_JNI_METHOD_START auto * block = reinterpret_cast(block_address); - const auto & col = block->getByPosition(position); - std::string substrait_type; - dbms::SerializedPlanBuilder::buildType(col.type, substrait_type); - return local_engine::stringTojbyteArray(env, substrait_type); - LOCAL_ENGINE_JNI_METHOD_END(env, local_engine::stringTojbyteArray(env, "")) + DB::WhichDataType which(block->getByPosition(position).type); + std::string type; + if (which.isNullable()) + { + const auto * nullable = checkAndGetDataType(block->getByPosition(position).type.get()); + which = DB::WhichDataType(nullable->getNestedType()); + } + + if (which.isDate32()) + { + type = "Date"; + } + else if (which.isDateTime64()) + { + type = "Timestamp"; + } + else if (which.isFloat32()) + { + type = "Float"; + } + else if (which.isFloat64()) + { + type = "Double"; + } + else if (which.isInt32()) + { + type = "Integer"; + } + else if (which.isInt64()) + { + type = "Long"; + } + else if (which.isUInt64()) + { + type = "Long"; + } + else if (which.isInt8()) + { + type = "Byte"; + } + else if (which.isInt16()) + { + type = "Short"; + } + else if (which.isUInt16()) + { + type = "Integer"; + } + else if (which.isUInt8()) + { + type = "Boolean"; + } + else if (which.isString()) + { + type = "String"; + } + else if (which.isAggregateFunction()) + { + type = "Binary"; + } + else + { + auto type_name = std::string(block->getByPosition(position).type->getName()); + auto col_name = block->getByPosition(position).name; + LOG_ERROR(&Poco::Logger::get("jni"), "column {}, unsupported datatype {}", col_name, type_name); + throw std::runtime_error("unsupported datatype " + type_name); + } + + return local_engine::charTojstring(env, type.c_str()); + LOCAL_ENGINE_JNI_METHOD_END(env, local_engine::charTojstring(env, "")) } jlong Java_io_glutenproject_vectorized_CHNativeBlock_nativeTotalBytes(JNIEnv * env, jobject /*obj*/, jlong block_address) @@ -632,13 +696,13 @@ jobject Java_io_glutenproject_vectorized_CHShuffleSplitterJniWrapper_stop(JNIEnv local_engine::SplitterHolder * splitter = reinterpret_cast(splitterId); auto result = splitter->splitter->stop(); const auto & partition_lengths = result.partition_length; - auto *partition_length_arr = env->NewLongArray(partition_lengths.size()); - const auto *src = reinterpret_cast(partition_lengths.data()); + auto partition_length_arr = env->NewLongArray(partition_lengths.size()); + auto src = reinterpret_cast(partition_lengths.data()); env->SetLongArrayRegion(partition_length_arr, 0, partition_lengths.size(), src); const auto & raw_partition_lengths = result.raw_partition_length; - auto *raw_partition_length_arr = env->NewLongArray(raw_partition_lengths.size()); - const auto *raw_src = reinterpret_cast(raw_partition_lengths.data()); + auto raw_partition_length_arr = env->NewLongArray(raw_partition_lengths.size()); + auto raw_src = reinterpret_cast(raw_partition_lengths.data()); env->SetLongArrayRegion(raw_partition_length_arr, 0, raw_partition_lengths.size(), raw_src); jobject split_result = env->NewObject( @@ -694,37 +758,35 @@ void Java_io_glutenproject_vectorized_BlockNativeConverter_freeMemory(JNIEnv * e { LOCAL_ENGINE_JNI_METHOD_START local_engine::CHColumnToSparkRow converter; - converter.freeMem(reinterpret_cast(address), size); + converter.freeMem(reinterpret_cast(address), size); LOCAL_ENGINE_JNI_METHOD_END(env,) } jlong Java_io_glutenproject_vectorized_BlockNativeConverter_convertSparkRowsToCHColumn( - JNIEnv * env, jobject, jobject java_iter, jobjectArray names, jobjectArray types) + JNIEnv * env, jobject, jobject java_iter, jobjectArray names, jobjectArray types, jbooleanArray is_nullables) { LOCAL_ENGINE_JNI_METHOD_START using namespace std; + int column_size = env->GetArrayLength(names); - int num_columns = env->GetArrayLength(names); vector c_names; vector c_types; - c_names.reserve(num_columns); - for (int i = 0; i < num_columns; i++) + vector c_isnullables; + jboolean * p_booleans = env->GetBooleanArrayElements(is_nullables, nullptr); + for (int i = 0; i < column_size; i++) { auto * name = static_cast(env->GetObjectArrayElement(names, i)); - c_names.emplace_back(std::move(jstring2string(env, name))); - - auto * type = static_cast(env->GetObjectArrayElement(types, i)); - auto type_length = env->GetArrayLength(type); - jbyte * type_ptr = env->GetByteArrayElements(type, nullptr); - string str_type(reinterpret_cast(type_ptr), type_length); - c_types.emplace_back(std::move(str_type)); + auto * type = static_cast(env->GetObjectArrayElement(types, i)); + c_names.push_back(jstring2string(env, name)); + c_types.push_back(jstring2string(env, type)); + c_isnullables.push_back(p_booleans[i] == JNI_TRUE); - env->ReleaseByteArrayElements(type, type_ptr, JNI_ABORT); env->DeleteLocalRef(name); env->DeleteLocalRef(type); } + env->ReleaseBooleanArrayElements(is_nullables, p_booleans, JNI_ABORT); local_engine::SparkRowToCHColumn converter; - return reinterpret_cast(converter.convertSparkRowItrToCHColumn(java_iter, c_names, c_types)); + return reinterpret_cast(converter.convertSparkRowItrToCHColumn(java_iter, c_names, c_types, c_isnullables)); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } diff --git a/utils/local-engine/tests/CMakeLists.txt b/utils/local-engine/tests/CMakeLists.txt index 231c564ade74..1c6b1cb2c4d9 100644 --- a/utils/local-engine/tests/CMakeLists.txt +++ b/utils/local-engine/tests/CMakeLists.txt @@ -40,7 +40,7 @@ grep_gtest_sources("${ClickHouse_SOURCE_DIR}/utils/local_engine/tests" local_eng add_executable(unit_tests_local_engine ${local_engine_gtest_sources} ) -add_executable(benchmark_local_engine benchmark_local_engine.cpp benchmark_parquet_read.cpp benchmark_spark_row.cpp) +add_executable(benchmark_local_engine benchmark_local_engine.cpp benchmark_parquet_read.cpp) target_include_directories(unit_tests_local_engine PRIVATE ${GTEST_INCLUDE_DIRS}/include diff --git a/utils/local-engine/tests/benchmark_local_engine.cpp b/utils/local-engine/tests/benchmark_local_engine.cpp index 26769e00be78..3674faa60b52 100644 --- a/utils/local-engine/tests/benchmark_local_engine.cpp +++ b/utils/local-engine/tests/benchmark_local_engine.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -1252,6 +1251,7 @@ class FasterCompressedReadBuffer : public FasterCompressedReadBufferBase, public #include + [[maybe_unused]] static void BM_CHColumnToSparkRowNew(benchmark::State & state) { std::shared_ptr metadata = std::make_shared(); @@ -1504,9 +1504,7 @@ int main(int argc, char ** argv) SharedContextHolder shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); - - auto config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); - global_context->setConfig(config); + global_context->setConfig(local_engine::SerializedPlanParser::config); const std::string path = "/"; global_context->setPath(path); SerializedPlanParser::global_context = global_context; diff --git a/utils/local-engine/tests/benchmark_spark_row.cpp b/utils/local-engine/tests/benchmark_spark_row.cpp deleted file mode 100644 index 2d332f1632b6..000000000000 --- a/utils/local-engine/tests/benchmark_spark_row.cpp +++ /dev/null @@ -1,121 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -using namespace DB; -using namespace local_engine; - -struct NameType -{ - String name; - String type; -}; - -using NameTypes = std::vector; - -static Block getLineitemHeader(const NameTypes & name_types) -{ - auto & factory = DataTypeFactory::instance(); - ColumnsWithTypeAndName columns(name_types.size()); - for (size_t i=0; i(file); - FormatSettings format_settings; - auto format = std::make_shared(*in, header, format_settings); - auto pipeline = QueryPipeline(std::move(format)); - auto reader = std::make_unique(pipeline); - while (reader->pull(block)) - return; -} - -static void BM_CHColumnToSparkRow_Lineitem(benchmark::State& state) -{ - const NameTypes name_types = { - {"l_orderkey", "Nullable(Int64)"}, - {"l_partkey", "Nullable(Int64)"}, - {"l_suppkey", "Nullable(Int64)"}, - {"l_linenumber", "Nullable(Int64)"}, - {"l_quantity", "Nullable(Float64)"}, - {"l_extendedprice", "Nullable(Float64)"}, - {"l_discount", "Nullable(Float64)"}, - {"l_tax", "Nullable(Float64)"}, - {"l_returnflag", "Nullable(String)"}, - {"l_linestatus", "Nullable(String)"}, - {"l_shipdate", "Nullable(Date32)"}, - {"l_commitdate", "Nullable(Date32)"}, - {"l_receiptdate", "Nullable(Date32)"}, - {"l_shipinstruct", "Nullable(String)"}, - {"l_shipmode", "Nullable(String)"}, - {"l_comment", "Nullable(String)"}, - }; - - const Block header = std::move(getLineitemHeader(name_types)); - const String file = "/data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; - Block block; - readParquetFile(header, file, block); - // std::cerr << "read_rows:" << block.rows() << std::endl; - CHColumnToSparkRow converter; - for (auto _ : state) - { - auto spark_row_info = converter.convertCHColumnToSparkRow(block); - converter.freeMem(spark_row_info->getBufferAddress(), spark_row_info->getTotalBytes()); - } -} - - -static void BM_SparkRowToCHColumn_Lineitem(benchmark::State& state) -{ - const NameTypes name_types = { - {"l_orderkey", "Nullable(Int64)"}, - {"l_partkey", "Nullable(Int64)"}, - {"l_suppkey", "Nullable(Int64)"}, - {"l_linenumber", "Nullable(Int64)"}, - {"l_quantity", "Nullable(Float64)"}, - {"l_extendedprice", "Nullable(Float64)"}, - {"l_discount", "Nullable(Float64)"}, - {"l_tax", "Nullable(Float64)"}, - {"l_returnflag", "Nullable(String)"}, - {"l_linestatus", "Nullable(String)"}, - {"l_shipdate", "Nullable(Date32)"}, - {"l_commitdate", "Nullable(Date32)"}, - {"l_receiptdate", "Nullable(Date32)"}, - {"l_shipinstruct", "Nullable(String)"}, - {"l_shipmode", "Nullable(String)"}, - {"l_comment", "Nullable(String)"}, - }; - - const Block header = std::move(getLineitemHeader(name_types)); - const String file = "/data1/liyang/cppproject/gluten/jvm/src/test/resources/tpch-data/lineitem/" - "part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"; - Block in_block; - readParquetFile(header, file, in_block); - - CHColumnToSparkRow spark_row_converter; - auto spark_row_info = spark_row_converter.convertCHColumnToSparkRow(in_block); - for (auto _ : state) - [[maybe_unused]] auto out_block = SparkRowToCHColumn::convertSparkRowInfoToCHColumn(*spark_row_info, header); -} - -BENCHMARK(BM_CHColumnToSparkRow_Lineitem)->Unit(benchmark::kMillisecond)->Iterations(10); -BENCHMARK(BM_SparkRowToCHColumn_Lineitem)->Unit(benchmark::kMillisecond)->Iterations(10); diff --git a/utils/local-engine/tests/data/array.parquet b/utils/local-engine/tests/data/array.parquet deleted file mode 100644 index d989f3d7cbc18dc15f56402eb2be61bc812fda76..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 292626 zcmV(fK>EK>K~gal1Qp=m1QqPy1WXl-OcewY006MC0`xDKmu;!fMs1s)WqJYBd$|E| z7%wgZIS4Q`w2!B>c>tJm_GYi2)sE9hdDTTR+7o7}-{BT;DiJd!It=1yT8WRCcZ=5# zE-svRGXmsDso12P0!7kv4F;)Sd<=94^pR0Vc3uOF_C@;) zqKIdsv$_{q69GhJTJ9T1n{*6%Cvh#vUB<(_Al6mwZ1FWv;}8*kuNQJ-P6J9zYB*Q2 zR%0tiAB(&lhi@GYd2dh{whx$4X$YBsFO9)d!p0x=hOwpu&AD(XA@X+mXh+Kq)fnt} z^6pcUklt-cbVV+`(apu-wXxz2=wVuu7FZH7uDH!4tGS|dW8A6Z$+L`beJC?~=PvfU zb;8L5LK{3}ly&eAJG@s`gl3v0X@L+Ufx&A~ddR}?)yw^b2@erTnS3hpowjoL0TPu| z`K8$Dw{bcZH`2BS&Ly^!tRDsxPT9j&x{GIgXyx@WW{Lx+thGsI6apXXW(TTnIPUc$ z)lVHo0?CwMuThlGwc?#@_(kaU#hP48U%4D|++T2)Z-W*;SDC2*}_g z;57%&B8m8BM%fDOQ!CHP^0A(_C*;NpTDq=5Fai(teaiKIN|yH9F}^u>a?=JC4ss*&D8Mfps zFG(Ynd0_|GVLM)yWab=Tm1;Gvzd);)^fZ|aF^BSchECgknQ71xW~Y#BkV#n!Mfj4P zuX1;p+3Q%S`4lM^Yvh=EJR>?c_Mou{xZZP9)LGFMnSOrx3`C2B;BVlG^Vo><6iJyr zbV29c0u<$!Z;aGrtM@l;LYgU=<|u9$@p6Q^Q!iuUG?3<0%gcV5RH{-xOtOy-$5(2p z<(C?M*WD(o7SwkvpWe?}gph~Y=1?g>5q%|i$g0pnM_WVZPJN`{X}DWO)@^va*tAnn z&e>n62esgeZ|yej2{`7Q;LymF~fPHh~chlhm5Q6+ov;)%Y*Gpa+902`W=<%8EcHu&h+P;SnmU6V@G z7zYwS&@}d^jOSqm5(c>yOIJB1r&KIHGQ-5#NpRv@gbU6zxW+o%NfEJO*;gM zy-p9hsraxoB=xo~bO`143i+!NDqLr-5=Ph0Mp2r;VU${r0Z7Cvu#WxTN*7Q7{4^hl zlv|O0r5${WQ6X!Zc393R|AHbhff7s(egM;-L}SYh|5@|$oq;_DYv)?fG*^Fm?wTN9 z<7YEI33Y#s^{YN({bjMMD#9rqWYX0Z7S;m6C|L#E-QA4`whfX%nb`=+dtd3Tg}0}} zQIPUu?m_>7it&8F{R2T7q#D&Hr1s7C+^xr$(My3*W3c1z(OdH|ChPv51sqqU*II@r zbab8v&&*D(K02NV?|gz|f!XE=7@fMZgeTcCSDLl{E%>zuv%ij$V2)Q~q!b>% zx_!>+WgWy@OI?&KfKmGIkvN?NPuW9Y3k`o&*QZk!oA5B*Zz{XsV~oXnnm2Ry8gO|t zVlb>z!eUy01yPvG#0XU{PK3>Fsi{9XPxNe{ZfJ*u@ssmjcVfaJUr+uU=W~U9D zeFN_ipbo9Innw+Gma4(lpp#nf?tsFx$g?>kmu|>t4C3iNUKWiY;>1WuAs^I#cI&W; zc5u|~--~ekT|~C|4)Wv3P=C>HQ{+ppv{x%|k6RR{l^Zs<)&tT$7tGG9Xq(`y#zgtZ*5hQ4L_3z8X07h@l(?N1LACEtj zlg`6Qw&&2qCst-Tb;BEZno!jxUaZF=RS3*YIx$&GOmAM}2;YQl#0)jNt8tJ)>+SPK z4gW6itdo)z*B0vX{Z!^vh=dgu!Z^`7sZ1t)noG;VIV3ZAZUp5xC&oK`rG{O}OSwNX zV2g-j!*hT1v`EK6nX9-S=MY^~0J-r#=+lmCj+~0F;_sT1NCp?&V^UJ9?td{UWH&Ql zH(C4J;R+&e$N9o1rZ3Twf+g?9FSIxOz&->~5}Tm;ZcjSX|rE8=Eq7kHCuTv3>u05ATY0z?A-1|=(;F!&V`d!|ak!6eoZ~vxCjBtq)Zh^$nbbO(er_3vdgkGe zte7eyszQkPSD?s>v6{~H9q-L+D%hnI<1{*yeaIuSeQAA*gq{`n)uHsMh-4m88gcMQ z|F{@_12GIUkxoUz0!8=ut^NJ3q8k_}eehr(2m4ejCCAYEJQ}c~J=A*ux700htLt~& zGXRD;jmfjhHljNTY~eolodP5o)0)4Gd#6tF%Y7FU-$;l~q-vQGMQHnVPNqwb8u|~5 z(~mZ-2%_GF;qEigt&GxuCrh9{>P*hzOz~e~Uw65+j5fqV+w;lS^Oq2*nS(%Iaci`T zvCYxamoxwvbD5s%DRU*nYc*I8o(k3ZSL55thH3Akb%`3J;)(PyruEuF74WTb!gt;W zh$%~h^{Ae5#pq8oXH)|*#h+O~Z8yvyyXK{CU1NWp9}Tk0s53>V+{xo*mk{lO^*5~P z(54A;Mh8l^ZrqAx@A6r!{0jkLs+ZqH%PiJHY~5bK@Lm-lCdq_k2z zD@Epe1M_u1wZbj$^+EB!v?Sy|=??V)QVk}^#dSrvd$fX{It-6(GR4DxN~j9#0F8D@ z^=|O;6U7K+nk4Aq-xA;SXn`Ro_*w|sQ1Jy;whvqEPnanqkkGEloM>42x_te!zU+ZR z+bIL~^q<`yBGAO1W8k)9@HU6Y+^#XS&>qNHn$Vqyy>avwi&aYmgn_%ZfRvuPlfv7pKoya?lHw z-r$+7%s7gk<-eCcL6X1|pT?zOuF3?012INn$e_AhiA%WRE>xIhpnffFBX+_>55+hn zM=Qq@&is2$CCU#Gyp5QF)obY9N#eXQm_zm)>hGU6$L*F zCFAVK9ku5S-bd&|2Tswgc&ymg+GVkpkB~6bkP1=$o>j^YV7xfult(0sg#>Y25)CZ~>zwt^VIqZ(+7}rZBV((;obzlgVQ4mFpln4@;+; zz?Eo~XtTivL8eXXhKShPYH7~S|2{I1ZY;z08g3C)f(Wuc7>rfu)lS$wqSAT{wjb}5 zg&-MoEAEksI%V{EnjY#GEH^Q5SJc*=>A4LgRZY0g9oMe!dzjXyzGKIqITfr_Jo!tE zNApsMft2S_?!aspt3Vin*#z~>bjOAZTSXsuV;lXm#T5}zdPejLCgQGXiZ<3|+oiMx zgv-SszdB%4*+uK0X%QOV*4!_SF)WuQx4N~LT45dOd_(4Bwo?NEMM!Q3qmv$ z^C%jz*OOW9M6`X&Oli&MsZj@XbbB(Z!TEhcD%uzDc(RB}8|b+2X1SR;W-dTbifkH< zFadvdovw)B1$)awh}9Oh*jsQ%|v_mHZDkVu_H z?AJdB*Yqv}2=HKqBQR;>Yu_!C>bWaX9J=KY2xDVu%YTDywSbkZyv^?K3%g;ek=ui zsoB%LJwQRrc96+~(E$k^RX6D zh>?6J`*)LQ^!)W;@Hk346EBVh!*9Iv~pg z5%`!{bzz;XA5{459;{*}#&Pfr6bmF#tPKHDL$Pfat}Lo_ADlRCu7Lf?A3SH^G0*l0 zW8`EfZ5pT#wSc_ObSF#_rD1ub3^-u*fF_k|x(*U`u%W@VJ;q=!I8U(6v_2cX~Bg$Tc+hxzcGZ{M*!GdFh_! zV_g1Ihx57E^vz3>q?b%6^V7h~4XJmYXpzkw)SAvKGV*O%mvgSun$8>qSni-d^i{$j zfsV#L?s>0$8m|aLrh8{UGbo@FytTGyY49!yDj6Ydb5}V0thyke89$NYy zNikL>UdCU$=v!Fh3&*T+iUI_f^kD)@}xG@M7uCPIMJOFBHqP!tz=O~mxEP#+=0Z&eSgv%ZI5-a^t-kPA8(oM#y zm$o>M`P%-3L+tB-ZQ z;ZK@e5N0CSFV+ku%3j6}a|W>!{A!#wXUZVgk;R>Sx-ulUGo)`bO99|<`L61pb|B&p z$dH|s68_28C{QEQX1y?SqRrEMdW6%_b6KhSlh8({FG|c%{8$tuYIKGfb5&}_4cdsM z!`W;MJJNCPMtA2h4zd${cwuvF8`IdetY9R909GtlWa}?U9iK>kDG0|QK&hDdl9UBx z{jyWSV9Ygq$-0NcG+f%=L&29x*fa!q%y~XgdRILK)tzc(nh{K9fdw`K?TURg z{q<$VE=zmbf!60dv8ejz(IsQZrESsBiqUVTeTA7b&Pd?@v56KMVR|Id0QwRe~sRlN))fqI@| zh}B$81Q#6KqX;3Xh2N3B<+|ddDO@g#GJWFbO4tQ()(lfKQ5l9C6>`A)pj9-+o#TU=Cvk?fW25j={7%f4#@ zu{GjQ2EPF!Wz<`a+uwb3zk-MyWt@lGl*3{fuy)6_-Tr+*o+H=@B3wgf2nBP0Y~orG zXj=FXj$_W2w6+{iPqk*lWAxYA_HPhLHw`!?{x%D{*_u{@lbY*BD<{xzn|1zF%CvM5 z$r#2QJUdwKJqt-}{VaeI5!gI-zaA{moJuhipphfZ8f(b2j zs0OsYt>|vC^U2?`ahr$!@UF9K&skwHQk+Uz-4dsY{<_|N!NCO*=y@(|Bba?-n5ntx z4^eRx0s6_V2HR#EPTy512#Ry*bfs-+En2O1%t&iF9Xoeuf1c(>D>9^#@UHus5O)G> zP#l=X+fsYG`6$anb52WeV<$V$(X!UeINK|%3Bn9wHAy)>!}8a<@9LBm175UwNB;I! zYssHEMbXmT<+lWg6W*QHHozC-MLQ}$6bG&heACTeF5iyVaZ*<@5yYg9S;1=fhfm>7LcELnFn?-pBeei} zgu~I~6q5PHOYoi5M z)3@}^EaY!#zqdo8s4W3eF#46KB?xJ%dH5~SpPVV;<5HLb+9G~ofNqz5AjF9Jp!e@2 z_UFnHDbfgzt93#gIG34{;WueW$(~H>|GI);;HY*L8q)i5cft@H!g!4xY(naXR7W>t z?qqyR84st`XtC75{8grg7m)HdnoQRUiVlQ$Cv<&BtXYMzoZ2-s6$DFaBVp7Zy>lvn z@hLDy5mIgjsbiDX{)4evRyD+e?0qLSIOb1ASY+_fnJ(8CvZE^&Y-t45xKhw4gE`>N z#n_@rcQ7hxQpeqXY8-In1HPoC*_b2)@$!+t*HM19b0bQ4f<=-(I6c1DG&y{j@i11~k-7zV;r$nVj|58!foN(bT zvj>JVGmtyn>}Ms;Nc9BJOQ%3(6{S3ps2PX8SjG`xY_aVJHsH`3NIESO z6G0m(s30E4zM!bI_1Ip#yJ3brO|kgk4WMiIYMum3@wC<%Ek?185=qe=COXOf1U}D8 zxKIC)>#=RR`{f0LVn0XhXMQU=>hiIUkX9T95KTMvyC4vfB(tiN?~0U%y`z4vphh}0 zfqerrW&n`J$CDU`HW02ti6EDM0$uOqS*ezcz0c#a={q7roM<(_O^js}9E1x?qtZr! zRXJN9uqxt72MgQ6++5}u6{|8!ar!qIQtnI#F>!g^9};iUzl!Sb-NgX^&DY}tl0&)a zpQyUX0!wg_fq$UKP%kFXrcym`wZfR^p<>zpg3}G5I0pb)q$0AB$qY+qxshR^C%-i} zROQdIjR82?scoifjp5{Rg$qwkfoB%~Vz~FRxx2sx0M5DbE2y%I-A|0LA1N#)+GLGe zklt9qT8C|ypeAH{cd}9ea+YPoehE$P zBh{|QU7|SIISz3orJHns2y>-=G${Z%StWXk8q=@ga)jg2W49PlUFy`8fonf_*`uacS-&BgYuB`W0=v%ZPi$NY7yhP zig803TB~_650RHeQ*KfCpiY;#z?+X=4<&ew$X{_)V0oMcYUF}cK2;*85&6ttrkwW8O8`9Q@37Ox^e?~z)3f`Ub3#-T(C_h>?U z=#tV+z~dl2&3oNa&Z^_VoW{iEWniuIf6zVW;1?GoA9mn_v%45a~jKPRXc zauopapqwvKe68&@HHgbGLtF=lwq?(uV3bYK_pBFL%-$!9gZi}(U~M|F%)$4Hxbp}i zP7DvU(6#Ly!)oY%zQ}HSjL*hs^~8(v1h7nU)fN$cs9dS|ce+`V8NFe^}9?-J4{3tA9vml;GSc0^A)2#p3m$1-f(YvofYs9LK0$r}C#U{T~2x z2GZ9o4Eoh{*JzO_u}r7$gJ!gIH(+As?h_p0QE??-*f?R&9X}sBzeV0D0h;?LWr^r& zm;B>EVcWpl=5rnzpYh00rok*?J988%WK25mBO%alO+8>P+%$agK}xp?<5Q88POl;~ zzto(rqS*j-DNPv3W&zb8OJ8IfE++M>Ds7W%tp~7ZyvOoSyH@=eBUZz1N%u3AQ3t&) zlL?&ajlFD5up5zjD3xOc8*b=1T#Y^kr$6LiZoR;_ymX7c*=QFdFF5(nejp28x?*Pt z>e20Za-~a*(?7d%(#$W@TP^z7(GXe!)SP#CKXy5$6KFtudO(pr00!H$-3kKf_F{L? zyXiW%c^gawLgY}RbGwHj`rF)evO=vM=o#GPJn#_RK!0Pj4C*p$Z7*W~*&w7tcZUEM z^PculcWNHz`bM?LrJ_})w>mBr#qo%M95n9@e5kdZ!Ji~VNNn8X-G~ytXIZz@{W0x> zA5waR;z9OTkWc&&G%?N@m-AW~F8QCNd%X=U``bzQ8*Z&P$^xFKSR0}Bcz}=Oq(BEk zM@&n?8_Z`&H3R7d#PxU**|F?N_sw~r=?cN?POysOv3x&tE4`8&SRT5UXqCGs9m+!E zbaPCI5Nw?!=LxL7yjb_tMy)^kO9?EJ9)e-Blg>MMapC!Gly2x&8ib+{!cW%5Nlq<( zcv?d7a>(WA)i|zzZDg_VZQA!N3)6p&e;qK^1LS$LOhpwhUz8E zz64N`pI_hRpgW|F$wLugMYEF2aOSoQDtrDJa&ItxSxboL{;?02z9Y> zI8P}lw##6Tzlvt6reTsA@o}01)(0TvVjZb;ixo9%%C^m_uD4Vnjze`IO81pH3&F9y4$^`HBaV#v5VJeJ$%bh{hcrgJKgJ)3`XgcK9Z{c zqp8r5$gmQ(5SVoY!Sez48@C99MVrDVAKb2W0T=V`W62vMq*bXRBOI(@`KTS69B0iUGZuv{(L%rk zpsgC&ASFiu9&V~_9z_3U*lkkmN|PjnoiavJ~!;V1tt5+~v$=X{DKP9@d<6^v%rjb`~yAM#TZi3LC$?rt;(C=}7u!u7G6{ zMalmO$>KP!4rVGVE{VDM_Wy%T_4kH$y(J?2wYvQw=FUPXFt5r%s!at}Z~rQ1DDaG? zoy?iN!-)54ti*FbM(Q|4IY<|%+Ln7E);r|x>j8;F;HcapVwq8EbD{** zZ@7OUtxU9nUT?6?;Y}6c}CuWP-xd>B}>pCj`DlqGb;e_RSwD%%+jl?M-CF-#J5t%jo4LyiZ+L+i|% zdoLJK9bY$hZN9yD=XHWfu{MvHlYV`o43fSyQbgN>&*pl<_1CtNEJ9bffw1XtwZ^W# zP9of3t^_&pcHnXzYVXX5yCjctTpx|Bd@&LpgJopzJ0QYip`T*EB8GP5kTZ#E+_RrY0E9DwqJSgL9Gfg1o6pc?`Q;q zpER-5W!FL<4^EqjZEAOx=Giscm;YZ7AhbEnS+7NY169S;84P86S?kcErf{H7*Sn&o ze%@CDJ{rifrxLJ+DY}(>74{Oy>5r`uTk4-)WtvxN?wbS*G3TuFj8>aHZLQ?RgyHV_ zM#O@-+*&{&NbIlCy;N#GxJXQC$18U;9a30VE`!B`%GufQ_5`4VG{+aKZhksN3YPgd zDWiGNK4r#1K#D?aTlITBz~dsB^w;3+#H&!@Nce<*V6I#tSo*>p%Mff*`B1uSgD04^ zdVIqnefJK#Hzm(JgO29@Vf{1xDw$PywBl;xNr6p@9ytf|;~(_OjDJUVyi06PQE z$Fq-^a_QXku~Th#7R=1X>ng#&=;J7nqf97l%}?u#;GB^TGu3f*$lW_gv=h*^-kSvF zT<@l2*47^1rjTCx!a`rosTD9qV6WnDU3~&8KYVogB)+~I*s=8V)r`6TOREK6Tve#t z?QlZxv!c6_1AX21_P#F^t*y}x9Eq~hR%PXayV~PGAUNV4N0jIvCpzgyq{P+H7ciSc z2)TSszl9m8D;8N;w=}!h-?V``sb~67tJ}uO7j@Ax5gAn#rs4_!jz$XDfzA$+lgQqD z0ZZvcszzUgK3nnCRBV~X`u@V^dWx_)RQDTh0!B!SLKf-Vo89ZsNni0hv&^s4jPEL4 zj2~O_1-VA(?dI7|Jc%3P&n16YpIli7bS$l02=gxhn#B#g?3>YWzguR^Sb6#jV2EV% zGMEjUI#8D2lL1a_&u(58xW9Pw8V|#yE^RpehG7M`?f5#%IgUi zZ{!lMynPJa@7a;87kM$FvdwX6Y^cnAj6_xmP}Ju|Q`OwmG>BjXpqu^nVE_W9iPm}TLodlnmB2``AXwd&XK**RAKd}@1=~I8VjR2!c zFb;y$nZkefhEUP@3qI(|)S`YTIyg-ux}xHO{-dlH4B)xZIW6;mx-==*@hIpTw}QJ2 zW2r=|1$2baIjAZ!QK#IusLd;aU53x>AEte9(CwWUdp8gbb}>7wNPJ8`Xv(+}SGQJ- zucMiBTh}iM+h7)(!`7E4!hh1J1JpbAjE;e78#dk!x3@c@@t%Fu#hmzaA^dC`&RU${ z_4=+J^&IkNFjYc%P^*qx#Bz^--IY*MdHtw|wF)1UbDvO)C@YIri<|AGBNqokSF{p0H1(*^8-+? z7hwg0xKPc=PNeYd-=e)poywpHxYm{!&t^-fdOP}?n(f?r6C4N_qv-oP2I*RkTI&5p zf-`u|#_jytTspWUI04yOma&`IhG3&b2*_u5C{;*v30qSsCWDz{=G>M|`E0Qo zIsqr0kNWMqACYs>)%X<;SJ`R~vMS^z`F(bbj%Oka1NLP;L5?mmQV9UI!!ibU+18Lu zm{QUZ@Z67dn0pe^?_Q1}|oQ7ktRQFKg4Q@YzGGv%2>Nr)3g# zR3ZSA8Q!dumfUi%(K@lV`hUydzpg9#R-q2918W}8$6!Mq*lWM?G1{$E$L)QtWnln0 z?AKz4q#*qqn2)5o_kZq%=vMa*@BM+WJkuGkM@@tFyoeEz1mRO>$+b2GrA`WmDcd(VD#54{%Z-Af2i@Dwp8(7{|_&S`O#t=xgz#LU=4 zP3$HoIp$j$$j3hw>B2=&$25U2BaVY1j4;{z;&c0ZL8`DeJVYU>*%CT$VJyX2?%@h^ zTRsAnDMXyc1>;(7#Zx*M%7YPSK;seb2AecmYOP6_JjdzAgfYKHaW7KD0GgW%?$y+H zxz8=^tp?O@Xd{!LnUrv^=%<|XS3KX=dI5td81dxuX-X0D%fg*%;_bf*2u7{Mf7*DA zkz2?Lws^}yN3{7mv)|}(Fa}!9xz2T&=YU=frc}hdVWwBx+RK&`Os}RlH~~Fp^N&$* zz9;U<`#TzxRvcusq6ac}*iJSiQpcX$K@B_fAV zTL3sg@JNkFp4=0W*8c5^x;~=Cx;9KBHr6k$l4m416JH`3AC}$PySTh1g$$-(33L*J zDS@;eHE?y}5BO$x(Ylw(x*+1-FZZ5BK2UI312-P@*}Bb?D)kFue1$R;LY|>O7_kO(}JpPWXlkqqnl~Vb`;ZHxju&{BoU4gHG zBKdPp0X{gY6tb9mhKTnAK+xuI4g*Xh)){^$iRo}`hXPygoA8&DewOD%%<-|lgz3x= zAO<8sa(Di8`}vI>kwR%P$uexwVWsNA4^LM)u`eDd+rv6(m(B>wyAut|8h+M16ckA& zVfeW$uT<&09SvtN@=GBaO;iM|^nB$s(datYz|#nMw2Nm z`f4Wpa1HSi@lJ#~AgI~lc+eB_S|gY4iRY4~Jvi*PqTzveHG=1W3;gg2bnCYrC|do^ zlm9d9xQA80C;sc~nL32zz6TlV9{CNiBcnC=3vY{S!=$u`LvE1Z?(zQ~q89yaRYS~@ z+j5r$|BFJSx~%Gwf|tKVv^B2J6mk|;-)e@7zbO|8kM_dKfXDnPm<5IXeuR{Hh+x?! zIHwy~`{DmyqKFwoDXj6+63OIQU5Xqx-kXFaF{6DByfy$ z!Ut9Q27WLI9muj0`t!5;Smb}AP`Lk=Yx$b?*^2YAmRc4kf^Dp{iOwX3!l~sHK=ID) zJ0-eZ*wS%9oQ?8yk`w-&j38}t`vV}f$oe35XV-N8ip(JCxaH^MCF0XLwr6S>jxt3n z>(}yskYXnkzc_vrx_pkIv}jUUhV~pj?vIq{WpVOM+az~(#|L2IPyX-yWBgs_8HSh> zJACjuSxpu0BSv0!xB=>~6XM`-4&wTr&9Ii32~g(1t-Ho2fd#)M=Fdtz1{pLVh8p`jJ91d{|x?jh6UVkWRJaLd;`jZ0&`O>N;Af?nEq z=2#55XS&TO4Kz)g0-G zfU+KtR*|~M45tN)SN_V~BBO7DlvqYGq+@hyc;@mLM@cW&Tl`BnUPv(wBj87h;8@X` z*U}0@(`89tX9a6iyI0NF_ZM`Pc&YZ^D!7o3*T< zR)k+o#6%r6!M3a**?8bb+a&rCvX%_NjGUm-T@b9)QX=l7t+=Qn1guDY&Wjl+MBH}Z z|1ZndHZnM8`zMj|7tPnuWIC-yVIiUnMX^?h${(ukGD>~vL9`RGe=9JT!hQ46he!uf zIdtn^S06_A2&6C6mzh#(?^`i&`|oDQ(quhudiz+o8i6{PHfQ3i;a$VhhLJL7{`}bS zuF3Z%1B7Y=I?Wmq0)y?qBo@6~1c{vG9TaRjp*@TUtB4)Xu-bEO$EOknmKfI#ER!$F zAF><$6HAoqJ!&ZRfpGGQzpSWfKAG+}g&F4o&2DjS#5)wW)!hv~@r3Nev%$q@n0Tg> z;h-n-O>}wjrDHLtYX;ado7%~%rNL@)40qQ`etoj1ZV&0K?)^Eo*$T@->^4lV8rabe zCZY`;68UlF3lIt)EPyssg`2y94`kLHh{>>^V8@~N|8j37fWWwMmyX@4iU^c+&eaFA zW79Fv)=};6aV$w+fO10sbAN*=ND7MLaNs&y18m(+?Xi4gvvyJx)PrZj8^e-E=a9c>&!#QxE|Jp3y^>9@` zL|*}09YO>rbysqH=<3Fbz^&Oz?_6`Ou&_#5-egSLbe$(;$7)fpQG2`y)!6IQRa$Jk z%9_eUk0J6$W3sqtbvU$&<;lp)I|I6|3~4q1B~k3etqg84zKSl+?>|R2MMx@=DIgq3 z=G-jJH7XZzI2tWU$T6bOxV_mXAe(mgZf{By%1qCH6DUqtiJd3boX{3&kS`dXgnL^} zqud5FF?b;sO^=fD>%5-F``Oc*&-Y$pF|>P69@fgS7eS7pNAII3LTvrSC|#PxvRwv~ z$?6#eV3R)=IvON^b&JEg(VYJl#|bH>^Ws+QQVxRyzwcpy&lH^WU@zXuAUO*+DM{=u zYk#(37fZRAIr`Cwzn~z>I^CbycDN|>{2>Xq?dVxjbOu@;>RE-J?E~G9A(BS6EI3U> zY~P!R8-Iy?6Z*T`wh~i$7Ydq=uXBG$*_& zenZ4O*-Qp&3Zm>3_pHR)L=8MkD`H^76nZ7H9a@Ssb%t%SAlv#%ZuiqxDpE(f@lU$jcW-d{}RU`tOP{Ds$QQx zo)xA7IyW5GEfmN2>!53ILzIQmPypl4qLnAV+g-Iqnh=6`lPH0y-MAzsAz5A18Q54t z0mEkk_JupS!JtP`LUyD(NU z01=6j`>-N8K(^S%qr=`0Z$y_gnWg!C`%_*|co7JEgXRTuySA9NJ2yu5I24tJKh-){}^zItYU2u=uHTjP7q0vPD)sVFd3 z-!&)`Rr0rv_1%Ygoyc_!rP+u5pS>CaOEw^tI!fA=0U}c#3|C@&2H>UQFZu9j2V=o4 zS{>F-DZVXM>Xw&c(T@)nz-1jmpVRfp8mSg8TsVcgp`hhL@MXnNGi|g-^UZk-ca$Gc zW_R3T^#80oA#Mh4&c(Zbpmrk`+HD~0eWHZJP_vMYIx~hoKgOUI0PphQb3pmckCXXS z19Gi|;tN(gC7;(KkG}7NkvyIChGC?vSX~?{%}GTnj(ykLV;JlXBSL>M$}R-p`$*6q zC`2nVC4gs;V*>F{#hDK%GJueLpwQTmwm~K79e?SBUynnON~LsRgSp7E!JXUEp-8B3 zrWo$V)X8u;p#`KJ+`B?JAH`eKuuIOfkM@|KX&Ng&g-PG6Ta;Vr7xy1}`uzjxrFfJh7!?H2t}_ zoav^qYu?9VKV|y-8ODbB08d@lcL&1i#z8da`fK6yVOEy9krre7FBVv+r5qm(oZ7cG z%5hp~cASK90p7LemQzOm-p!6bTCrTA7J$#u^uy=v9XnCbcW(7|&+TXJhw8*}LG%W0 zGKOKO{v?#s>ud~tQFj>@=b6pU2QDKLJ-B6PZ&yu$mx!HRbU*Z*7oQ?xFoqj zmi`)|GJ@i9hvKq~J86AES{=62GKz#t568S>OWF^DTao}+#Ri$ApkbecP^BPQb;s8D zc*F%yZ=8191(7XI?=#_`{&p>Iyv9pyLtz0T^%Ts;#@8k;w2d3h=lKe7yVN}@PAo@2 zIl8Qe5yVIcdOHl;J9-C2Wp%nSMP7oodGTajgiCuf+kYs;;HL-HT9#yNL3OIW+Xp|l z#)0;KXIt}n_?Hhf8-)5rA-@rDd(esEqB5f!<`RFnxHbnKsw9d(0h_%~_LZx#e_5Ey z20ibV4Pxd{Art7xNto|jcG>aPuAr>Nb}8H1ao9v3$-5te&Mqu<$w>J#FT41A<+Y_e z^?K8cUwxSDqJi+U#ejZ_RkiRdu$;yaAkc;f8jjWTkN1I!bwg!!zWZ*-3=};Ua-Of0 zuWdf%J?lDbH{6Hy+4=+hgaX2s^b{PJw?VB5Ln8W343Aq8jeH#zx_nvDVgJq0vsqW{FL`=xBiOv-5+xxxb&Ne? zla`KFQwQ6Ab=>U4aDH3RjO`2zyg+P>IEIa+{&2{Q58HOwvW(nPGUDK8P>Pw zMaIBprFGbEj4LpTjDearZysTB;hh`Ex^$@G5Tcolw;gd9Azgzhdo)5CC)N?#Y>;Tp z4a&kFDy&&cd~g;z=?9BgEZ&{0j#D(am(P%L7hEcgZAO`G!iItxEnKkcl0?+Z9NWwL zjs(5e&BA=!kNa|>Rg*~p-0vYhkR&VQ)$p3>JWKANZeXbYn)f)QNqV(tpyUf#s5HCa zjT_t1+b3575~P}#C>)AEBiZgS;fN59dJ&YWb8Uk!sb**u%(U}^dWwN%+t+x?NPPU) z?AN?2Z^ANv1Icw5wWsx7b5vkGL43UU&v<%Dzy0#s*_riNVm3T6Ns2xjMhCHYQ7ae; zj$LTUnl3!r-$r=1U3?U?3;1i@Jh+SvG?T@_6oxg`cOHyaR9m3B!HnsN;t8FU9U=G7 zxfn}Zs!GOf3c^blf~;HS8er_FAbliT%#!1imoFGqI~eopF?ij~S%6bAGE;ib7QNs& z*7J9Gv9|SzKd)vs^Ky*EOB(fA=5T4#unCk|vTnkex;6KY-`>|?(`{7$@{LNpmsMWY z6EZk=PQb=uO z?!Bs;4hixYFWl*oR;QaOev8LbwdY4GrpsI+!pVK;qQ)J8(wKVzgB=uqf5o_5BFj}v z%I*6Ui9wxdIUsfUH_rn?p{~(hU9t_WsyHy&;wuqCa&u7@hdF3j@=FdpsJVlqhLb ze;`Hh&a-4W1`tn;v|%JeM7K&Fijj#hZcKA?T3aZCFJ>Sf=kmAGgwVcYehJd%lKmsK zpL*E`44H;k3lAw%z}Y{tbz%4N{SoR0i|k0pa+x?2qYQ9R`ahVWqbxbcDUoqlZZSnu`@}u%Y6Tiy2^`P#xh%>b+@~=y~wB_7I zxd=m4*Le_YzX?Q|yzYXy$QJdtW6Nw2^ym%DrjLkWgx9W@K8!{tbOtE`>!QU$rN&Vb zL}k~U(gh};lgiOon(9T&VJN=og%uCBX`!e$w!+!D490V0Kd5G{)PPhFehJC1T```X zppz&Iayuc0urY1+GghD%^-2-`QvhGa0cF7!Oofx*H%b2F?a4^`1^{o5L4wd4rpdl8UL&niA*JR%nmZ> zEFI|vbT#m8dK@xXb};STgNvtkz|>zbW*fwPzF)m|sOq#i`$Gj{TIX~Wgon~c*dJMy z@cd2c4#)IB&F7X>Rhv~q^L9)cY-nbSmS-$0Wq~t**JhGQ{y!%h=-Wl7v`1ada%wxz zJULMvM!DM1C+JU$=N#03FSJ#YH{q&+5b^Q^?*GHxNiv#%rN0`~zEjqD6#A|CF*mD( z6}^{>CUd*m+&rb+!?sQg^?h2TS&cTgUi!F|Ug$4t*`!R|%s1@%lAYZ7g~P>A&+X%| zal-YI$Dg9MU+J)8mj=Yr2GrV-SWhpJw`ALq5Oe>rgLfi)tVdhq+GqH`a%jkXcyx`% z85kmo6r;qn+&u2x{lE8W`}cu>428;rdtC>ZW+dm}622rypHcA{0m|;B6*@V9Ey<@i zRY%m|ozPn3Ov&oW`_dIK3Oo%souGFuxQ-Qv8LQIq18kJy&--;~Uji@+d|F1M)y9Y; zw0{2GR9vhc4Rhn`@kNmBGw_Idk8ZaM>#l>3F`J150HQXimFl8Z!BhAab5^9)^hAFs zndo7d|L_1Xb3r34!we}l34tGwBwz8h${!fHyVh->)^XF`Oi!|iUY%+s6~yd>jTI4!v`Bz=#l)wmvMhG6!_h10)Pv6-I2xhHv;vi-6%VS9yHmJiT!m~*~ zWynz=jIVn8t`P z>thS&+%7y79tOmm`$GTk(!~>{n0Xai6Xvz;?3Ty8PPRsn8-9=h|eJ>JoqRMk#m__*6tsf~P zbVN-H_6Lpr)LGvZ;B9frTm|anma7#?{7^NB>v4&xq;+I$4g!h}YXG3+WHdm(9_1m$ znnWz(hH}8YR$wfO?CMjU053q$zo*Yf{C^%$wa!u_q1h>;>cx`sRMb=D=5$~-QX6`! zscy;}dl*fG_!vRWBW8H6z)9XeGRS)Q+~>|AUlD`$Ys^WA%gCsXUcH7{32CQMh)*we zi^78vHGb#BvA!_hKC6ZVtoUtbaHu*jn<_WG-XaUN?KrcpDr5dQ#4wCg*rqS^=+bVY z$)5S5T~Kd_xpTPc9G)kA)Wx90=R_V8I=IhNg*?&ck~zY9bBSX77qA!5m@WV)Apll> z`AHsFQKW8r1#uovIe97Cuz_U`7*&mKf_-G*k3@{oui{|!3*(*>k$VzLDfI_BZE^H^ z&ZOa)B!sZE^=lzxdk?W;m8jeIB|O!!B_j&<164cjSB{^N1`J)P$a*qsIv#_AQF zEhvQLgZFnQS4h>*2BQR>@VL$4tvi^i=c^4xGJDxJ7A5GHWDGsW#BiuZn9HeZ{g68B!bQjR9VSmcFtVhB zFOs6)ipS;`*ioDYR;u-SdrHO`mgSHGW&V7bAZe~4yjkod$wCMhKZ6TD!38-7TbGwD1=lJ$C$)Zif0ocVF_ z3k+VPFFQiQt4I;IK|bRMp|=hX^&gVe&cd(Gy2^sH-X{35j!$-Ksv^?6otab*A19S} zFy*Yb?TZ`*=u+tg1M|>R+xRdo4$EkwM7D&}U2MOp16uA2G#xWY{wrzU`R>uWmcIBn zSgiCjWbJ~6=uwM!K7917XPw_E3HGRdqo*m1hIAI?J;QkL=nFVbFr_V0dT<$hIse-* zk9*};PeSH=E*rZbkqL(!lc{VrKY*a!Z7&Oynl6%K8Q2ry6-nxS4R``A6K7l322TFB zr>g8XPZa<1N_3+tw1p33kxID`d3sS|);Ht}ow zfLScb*TPdc&Lz*W7BCj=k;&Z2||AgpfM#~2t=}EF_9*1vTflKy4m%<;((&VD62}uQIf?$t`=^8cp0PQt4gXdEN7Xn z=tgQGlK2iyQ;jO%8eM&i1(nB6u^%Ge3?i;OG$(5`fuIPM&9eCs4@G6VNmoNsDE9RA zihYgj5ib2q%&;O7-)(z7pJ6mO6`q#{5od7du9H{9!(9PlE2K=cJ-+U@oB%FRK`lc1 z=DYD;Vc0r33Fhmt*gm-a#^+;baM3RLaThr-7q}e^R3KK*TL}*=Mu~hnCS>`d9a^n< zR81@ByjA`&Z&}jV=%ySiprwCRUHwshEPe`4RU6o>cvJOK{pJ8B9{w@K9#Cc6h5-#6 z9V6EXYul1FOY$E;@7wSP*nhWxZ@`N{tO-rnou(S|ZqaSP-UY`Giuw;A|8JLCq;P|v zq01C(HJpjjkR-G@guhf@maq6~mi;1sHOF~TpQvkip1sT5cO*cw@vREr8zLXZnbr-` zlRs=`+yR1klu7QPTIu$Nhec4{mgZ(Y9Mdm!}`YbuoKgk zD@Wm}=EOLWnoibUm`BMQ=y5CF-De1e9yfI)X>yfTGqq}D;ENLVGJ9Ln2@#7Ywd(|8 z$p_q(;pQJ_9^v1(O2t!r)XT6*YSrk3L>jIBo=bg~$v(PxF)g<|Be6I)csTs9=?{f` z%qJ~K%SHIq`zicJkSBaC_6%CBnr$XQ93$G3#Ge8fsQCX2Cl$d4G@~i%v`SFD@L2*% zVs{Xt4}Ow&|4I})^fgw z<|M?C|7Kt+Nf(pxmBK&*wuzuwEz7)lAw`|H-zSbEU0@xSS>ogC7sDwW6ezx)wFl0j zhD6?;9;j1)r#GlveMng_{B$^ug}aGFx)1(oA{Zn@m>KCuL}=~uv8ctm(loMj9zh8!2>tC;PeznV~?5-yw)zB5oi>3c5eN=e*xYCs~<=kV{NViCBNO8GFkv^rW9qIhCMsX`U%>uA#z#*HKj#Rx+- zIb{*5G|iQZe_!-A_TRbZk9cXN2;F@9X@!uN)&u;8ShfMuU=UR*R5e^RSz_}FSe2bT zZ^eK(@Do;P025xKk|=dbbEO<8O>L?v5xIWo8XN&S=y+6gDO}mQ;(UCl9CwEOH;e%x z`5m!XbQ_tzN8~;rdP-fsH#}Bfy9KvWSsnbqBG8Qj0B;m9XF*d9jqwZGL`j-^g@r* z<9b7mVdKIB65Hbga%WBH7`?a=a=a*T7t$1s1qbLHOnYwLWh2Mnd|!ZYK^J8IzCdb( z_zu5U9Tui0{F=TGePXO_2&J3$chSsD1Z;0mTuAu?KiBHOzDZCN3Z?uFkd(fTr=u`! zbuJHHk3cB7^=m-7(q#wm#%-tjX{zK=TcwV0rq#pY2T}VxePX@51%}cPTe3 zTQoZjTRV8IrJTd>h3K?KCQBi`YpW?Em*Wki)wuJ-p`PjWfj-(mH}!*qk}qH1BVH}* zR0P%`y(1KO4}u42-Fi4AWyc(&DB?>vUYPo%Y?JYMXKsW&KOK_w$gg6$C?TXJP1-Wk zUmR$^#+=|2G)<-h(a|wpxRG=uVF=A0clSH#VYTI_t|xP(P3{<4S37o71l|a$P1Tnf~12M+|Mz)uesH2!Gg$!B$Ru3PVQ=0|)^pOdkL%=-5 z5TIkwc~qdm$<@0QmbjgE$SyeKkoj3t06)dmVD;eQz2o?7)) z;>v{4?(xaZE-Vq)DV|nveo+Ku}eDzK$LDQ-3-=#v8uwu7oD zVRX!$4ll)~VI=T#$AU=U5+a=j;Ye1(yK%3o6DJPfj>oi3YaKup!yGm4l}xokFwcgu z*i%#|@xlQt55PdEHnRRivGWwHgwMAogTh}G(Z2!qZHnYn$JM0QC~Cj`9!0^gC;(e| zPx>%S8?gl3e>)i7ea=-yJ+zYHRXCO<_wZx1V@PnRMsEY~)4&84l|2_J7pgf*z({K3 zO60!&P!AC;*Gtgq4XLf{uK7@mmtsIcndiNW&MIA}ymxLlI;ud2ysQ=dC2vXwY>4q2*ZaS%D4z9-d0-2 z&mib}sj$C%S9Y`Clud$+kaiWPbSCT`uMi#z-Zr71Vq5NIoIV$wQTd|IAHXsM_ZtLu ztG;Aq`tKNU)mG%2u(s1zI=I2DbJbK}2aOTz4h4(NPM0|z&x^|EfOGVQikW9(uZ&!$ zsO7pN=9XnlX$D#Z>+`ow_phO~@Ryy}Ckau-yveoy0^;PVNL3STABVo`69*mxsBR(I zn5fdwB7&Pz#Tsw9-hcYd`y);&Jb~iizpRAs?`X?O@})wzr7p!dq$W>uP)`4CF}cSZ z{<{`#3wG$T=8}pmchU14=W$uUVYSp+F^bwk`(uZIP{($E0UYj4d~s|QRh!Z=gUb{Y z*oE2%BjPE&T3d#dSLpNR*De_e%jmsd!8W>E>)?TE5FkE=L+G*B8s)fWeLtyIMosbm^vru7-24ElnU1;SxUfl)h-O0U)JX zJn}zQ^Fo+m!4r1RpQ?`UK(-CxLg%iPs;?JH+%c4c_#gQN4m@U98y~h*+_Ii`IH%p{GLTPw^-XS)ig_Xp|;G?k2 zW&mLaI3Th1D}hIu z(fe+Ai~_STzV8utSb%{OhU-iq;l=F>Ji2s=zshfvP6RAT(TFL5Pk8N6;a%G|_(BWt z@A)`eckiHvDHdUc5(q>bNAFu+Y+sYgpOD{2)c&fA>F@mf6mI^QLdE;e%d{IjYfdHd z!>amUyTXj5!G1UU|MDYlULDkR_=vNviZPW4K{lP%nrLP{sc8J#=Bls@9X^N$ONj++H+H4p<{7y}rZ}tGa$+$wY1-H(d~6q^ zMgT(moAg_-aB~&uMukOvlYmh84Q#=PG;UFQsnh!5;a+Eq;zhgD;g^q*v-WjI+-L7_ zMCr@$JVTbHfsIPF>BGP{ycKNEu|&6;h_O+ocf7=A#8DJ+aSOaQ8R_l46|viC^zka@ zs&qml!@@@it2!I5U?Ue`wrnAFlNmcW)WfMW7(d0?aGxk@R(;M=XWNMF8|qwDW}cNn zO8PZ%_KBcsAVeKB4Y%u*Z&W>`(Ia`5?{Z%McDnV=dlhfcdBi!MQb&aDvIHS=73VKm z?wMwUeU>AK#4{M-(WOINMJgKR0L(Al4XLkea0bnGp8W}-FrEO}%kE&Ca~}W?n39eN z7nznP@Bmsgsd{tl?4CP_x+m)CH1@OlH`|+g7`DicvHy0RvdIR18CMHw#4jc+K`j?y zI!}LBAGIdCs>yP)sYFW3v-E@WA~pFfx|8JG*lA6IyI6*W_V|Gw6>E|vdtz~Vf;c!W zg=yQ(G^j%dK|_yJNp$sG}BBIyS}qMG^kZ5|GKA@KK6+joeVli9YKR~J?NJ$7$e z!g(YTc3r?mLP;%*2TZJCx;e>^`kj|>&79NTxsNU`iOoFKxYBXn_WO^dr0X_gp*Rt9 zDeZz13*o{l8NW3D|koJvzdwUIi5NHF)*s3q{AaC>N>pmv`PBMU5>NuKFdx91L3{oRKduZ5!pQTlNi@GB_7Eet&Zov-FJGW zso0+lZzJMolXqJlHE0*Hs)p%UEtkVLq}C&n+*qnM$2Gv@o&hNoPgj&0MSaj2t$`)| zh0hbIZ_p?qBqdRWz(?)=9Y@X~NLUEI-khsoMoN?m_xxoxEk5#rVCT{&7F=72TO{kA z8`i%VZ0JOtm8I`W+f&j_qA&IGV&SM2Oj)P|LmQ>P(1yqqNs~jyliJHo^LiI$fS*KL zh!tr<^`j41x!_g|$OFDFAsJd39>C1&$M;^gdYaR~unUm9=C%Tb{|m_}b&b6g{^@Cn zsrY7T2#^}pK>k#q@Pj?VDO@&TW|Vtf3|86K`L1NP=<^75m<&n@pSt)!_&4*jeda#B z(BQdp_FUoH{^>f#<8M1m{Ys^s9SP6k)J^o{ur%UBWb^JCL6~VTUTQp2+Vi7rLJ3!1 z4DToHF8u2qk5whRX6I7mf@;ZS=&Dj`5QMnHeFM;%TRNrrm@j>5Kp^~}W1A<^rb~B7 zsL^KAVz}$7QaEGwS4}AZdzm|hGU|Tz+XjDjP4uvqKej<6=9jkbO%iM~KMzk!Ul%#R zq9JY!dq@eA5Tjlgx9mqD@vH{`3N#m!+@dM^A2;au(hftB_T>qNW#>Z@XGBp$3E{UP zp`qRz(r3waMt0(!>Z@aRpcUk!3>ddXAhV`mqWR0 z*=rA|ZHjc>IfnG5P3>SoT8Jnr!Jom(i1l_m2ZPzzLz^!9B&tL@nORK5{L5}uf=|_i zcrT`W&2<~k*qd-Q9I?6UlipU1AjU4qsq7_Q&K33rkpK zKMBpkFDXfV_Zv;W8nf6(GNaCFC@47i?(2%2EIT$LqNow8hlz=>azN~se<__+uh&z& z^X<_hf+!>6weGK6Y+0Uf+zYL9{b{?IQ$$oai)Ag!qbJE4Fbv6y0&>~!NGF5&ucEk| zQ>teW{;|ogEG*OTBKua15O49SEz5ydPke1!1uiugf_uy$C2?#Aq!3x04cw4oAqM-Z z7RG%MN)CAqUBe@nkc8mr;o(ra1K>Oo5- zyl6tWD>%+FMfeJ7-|YYt+yqfI}qN(x7M@LNXwFT{8|) zpNwcwmviOm+ire-@De>Pp;vDT1%z5ur%^5{NWu|#BOeaNAE9%O}?J% zIJ{WY-}`3Vik8KFpC}%O=F$Fa_@H9N){`}8Ad9CCoUz?x>fxU2==X;l0tou$bT?`I zANuUcsti-fZd*lSNjFMART91Qd`qQtir}q8m$L?uXQN#bJQHhMZzvpFd-_(qv~-|n zIB>8htDF}j@4hTw(v7ABCbYeGh&0#QzDnBz7LG`eA3UJ{wKZ}GKtSee+*#71BJ?Jm zDOQg6VFp|eJ0$dR;(TI+-wZKu2VyrlT5^P=-%}LUDD5RIF3^ZCwLj7j>240Xoaop& zT8(9m`XPvQS|7}Iu9a9YkSOxBbqA1qUStbk+1_@Klm|!O*rr~B37j+ zr7N}<{cB=!HN^&F<<~E5t9li2vS~f0RsUNpFoOUMC^Qvm{~vBV-my+`TpnsaZx%G< zo_#K>&T0rRD~s~2)(pe-QpZ1ULBVV=ZOi(!-TeZ)e&ebM`HKO$dv?>U;YoSS%lc7B z$?Z0fIg{1B-d)yV5umRDt~)r(Ua+W7$DPdn=9ExF#Z%!k!~c7&9vq{ePUE@@qCahT z|AXnhB6B}|^-J7)h~27rG*U8yWk(58{mYwZ9y8$Es6!cDJWN z0LEp50E6(x*f%Wkk)4|eV6zu1c5WC1T}dA+dBD$CeDP$0>{j=W#-PjUy{6E)VL+W@ z@JzUVfMh=s*-UG^Lfwc_cY6;)y5#Y!JWzs%KI~B7%Vlh*u80X(xJnkIC*-(!pOrtr z%=QweeexX|2KEB1=yZ1{?=M|IMDpDg4eE=bL|mvki3o~)XNJ(0PrA1N? zu?M~m6DZjvrILn&p^Sm@;hUgUBMu7&SIZV-lBzV{_;vD9hFTg!^z@byMS-kvihDyI z4P6U3pK8^DX1&$>&lu>@sU_(c#{c@FnBKC$B7h!vayVB##Qv|pVH4dwCA6Dsh&nosCUJ4A^-!^evamLi{ga;>g_`N!54H{orQ|SY1CvTztYP4k zbLXV192gJ`4-$`ypIMN|&HwzYN{k^4rEF6g3!=zRSdNoKMSX2*iZo|%w!q3xk6I`fcf{teOa8x(IBWQJ52BwL*w{#SBT4_ zz-NO?nI^`{m6Uwc%(yvPD3W+8Px>Iqn;@LZTlQ4b;=p@R3r_|JS(%XF|N04qR*l{3 z%}`X{%5e#(MF?zCmZaeqEr7dz)&q zuJGXQK=NAIQU+Kq*_BdZn+Lmyek=21+uj-I93SI>7nf5L{21Z>cpsQ-cQ9D&LY!eG z*};$Gq1iHADZJ%AM{B?Yh>v)0bNln{W}k#0MW`&7DA-7Z!a5`6Vh1L4UKUeRY0qZ` z)AE%@sAjEM7WZ=;e6}CuHVVymol9F*9#kpjd}kVBqsFY`m~%1zT0KP(HDKYSgjk32 z3_uwsK)%s!h4aC=HO_GCi#@4JU3zv+9~U%d0jb^ExQ6uybe88|daR;Ue*Z^1S@)@Y z>4!Z94O1$Bi0|eTteT;U;AFMPX29pK#&pPr|1T(B<# zV}&u&Ubg2bD;lGsEnF6LnKyqkTnWoU(e5iIb+s530skWQGzOD7qUnyW&cfcLrGtdX zGXU?x-Z_J5nO-BeY)M@jw?C@I*oLdOkT)Jfk#2^ZS@72zL4tZ5Sia6ElynB$M&=7f zlan*{Rok@~NeX!}k5W_#G&k`8s$F+FNKU?DI!)bxR%;~TyKYdA`4QSbd!)+5FQ;~i z&~iI;6Y(*sk4{xIza)^N9PG1SE|$q_hm|gjJr^hn(^T{1G>7L>D~^DqoQBcSUOvP& ziVm`JY&f4gHyr`qDZ4xm2}OTqdgW{UYJsXXKHS%we&YV-3g8=;DX{G1n{eX0@fF^W zETk%2Mo~3Q29lw5>bB$gt2HTHqf91j@qFPRp^KJf!T?7yrror`Wb>-K*2rzbKz}WU zp)GdC0lOWxd+yrZTrhO3S3#ykf?@1GZ+c2!!iq5aL^7%Np-aG(S!kw}UZHW>XhmSx zy5K@noV83cBxtl9k#`~8De!+_ri_P384j8{hl^wwL)=*XswF2B4ba1W%&bC)V4{%L z>pWo9w~V|?yxnhYnRme{P)zOAC?zZTBbvL+&$xo-YWA2c50*7L98WK7uB1nfv9 zl;vfPSk>j28DmCoB5c($;uAQyM*>*Yk#lHV>&;5mXM`qEbeqq!fEc3tO zfvYR>V3r5WeaZ<V2o;_aJ|{{2O4Xa40!8I$}gJ+*1LM zi06_R*rv(wyLe)D@P3t7fQps^db53>PZAC5zkjZPLJS-QLn#hDCKY<%R6!8T8KD4~Q@?V)9U%nW7hZMM}or)qQlp9VyixVq)xlk|Ahwi`djc0PzYP%#RjN z`z~~SXvV6X<`m^D@!g@)i&9W~$Bke9ct~1pI3kXSw5BjdU2)_)4g7M$5&2#s(7Mfh> zrKfOjy|fT!o#P=&UsS-NVx}IpY&;{TS3%j;0obHTq^uzm%UWtv50V!6aDi9ixCjg( zd3AA*V!6);(>EaF;0y~=Q8z*dU!#379}!7@HSYw-&<{o_Xc(7R%e*_);;q-uqQp-Z z6^|f}(Z{}lp;{>$Ju4dW&i&W3+2KNjaw#;Y?d@LZx6a%>pV6OmUCIf-?UcqdKAj1AJLkG+6`@lFea zh?uN;R()yAO3Jz|vem>7d2bs}oRw|{pw*&Yk|*@+;bOTm3eiUpG7xoG5d)CXf}^xX zY#js{!s*CLJDgwBbh%916(meIAg$8&+HN0R$uU;1#nEb+RCLW+OvJaxslivlUmujk z#jn4@{8m&_n2anAje9M1Pv3=&0$}5Zp-i&5O#Y@?PG#iF+YPj*rT=~6XC%avNUZtt zPS->L^E+OKo^!kme9?m=#T*L}Z`vQxFTJQ>p+CQNuy)_5f);XV^0N9G=vNhDZ)Rby zjS>O6z4BXI`kO4Zk{7YGUh1Mi)@JDx!e+l(;LbVzCZg|b8N#ZXpSj_7JAz4cg^RLo zvv??`&b0{jsYgl<>n-3HB8f519hlN>oMTodJ2s)S3@61H!XsBv2gEVZ|Gn8piuGif zYYg_sniRRM@?q1kU?BL|m5@}m&jCg!RsYHY{~Ie&77C>}9}A9bXLmm;KD4(SfrDWLzt zA|vNJ4OO(X%rl-SCk+XaaeVD(fq)Jj;M3QDi>58-CBl}sG`jBKpBeiU_1>;%N7u_jY^vCM>oqC2Dht(1t*JbhiATjc+* zN={LTGzKl4Ofv|-7h>S2RI=^QM?Ib-%+61vFVdT7YrPU?SA4NRnMSGe!?)EJEnXgK zhr{#9vz?uQ)Vrq4lHRG{<+W(}H5eZE%f|ieImw~a{9O(A_%Uisr4F9@{mm)bNggquXAq8@Rt#A|2y!P0wtuEI$= zYEIKzL4eEw%r(|JW2MosupC&mUaS7A!YcDGN;FFD+L267pxdXVftQF*I$yqBZAXBY zL|TBPiXm@b%ME;ches%FGQd8{sS^c@XH6sjmpDINv@P~t)`H_BTh{QKD!9n5zVQU4 zH4NSKjO#=OWCFj0#IicA*ei}aqt7*ziNChQ2ZKff1t>=EeCbJa0zfHo)hfL497jx>{j-Q^( zuJ^Q5nB4EuW?``-Bc4Vns=y(!kH31Kt?EB8qN1U?YRhVcNT@S)ymF!< zh~FtsyStP?S(_=mD(>vmMO4)7&!-2T;NNAWOCqKvE-AYELwy}YPQn1T(4@%Xhy5CT zBamaSbX_*1#{u0gW&cZunS+4x;!%?SFO%IFw7!ctQ{5)!I}jk{^aqv9iBfyO7<)sX zP-6OjA%UWn@M^^dZffQsDHV)la(#y`%cF8r-FRr z)wjg0{=c8uA}Zkt2ype?&%wbu++@*7Kv;UZe9Gk|-BS-xiM!pL1pypMdb?y{o6~T^ z`W7dRL~xIUvPI7Aqtx6m9!JQgeEK60y+TCO4o5M~`Sz`FWTa7o-b=T4Q`9u>ygR2r z!oxr7n8s@bJfPyWG}Ps@d?*tS)QO_`q&j+!P!AGT9<&j{nZd93fn8iiUP!Ia^zf;pG2y2uG|j&}9q^{kamhc(qUMkp@7fj34M3j*^wJN9)T^Mq5& zk!d2lC7wdqpe8fVykpGLH*$+uY8kD-_MiU7iPP`qjIkTi1Cb3+xVv(BpreCnTA*96K+{2_)8h|2A?Rk{3*nJFw#W;wS=g8{;# zxG>Zgu5vWIf_c|X{rD3F>C^)&&)77v}?f9S^1L?tvF$?UnZ+$tYx@?|&OF;6REqbGcse5IQb zfDzimbwHkflu=~tvRdI;0PT#^sx@t!ZNg^9+0tx3PO?%?>_@Z&3h`zxPN&YwlXgMD zZEER~ozoKsoF)}N$m#O2;Ywo;)(AA0ZnPMAa>!iNA?bEIKd3l5C57;-3HX&6VSJXy zS0FE0A+*>@4*OB!R^)hNMItQ~TPga5s6SzZSISLR(8gCIa0zS1!czQM)evtbS=8X? zkO{fpy3Mn@4YPyz@=N@y5>#xw`%YaeprR@!p*^`1Q}@#5vUPl_%9I0>38R< zdwwXS+nf&fPsruSrQm2luk3Jtj+mS=46eMb7u6^gGf}~HVit83r$Q>y<<&KY(;i0A zhhA$P9{F(UTS7Bi77TOR=H>C0Kpvj{*GgWjb*g3R`V_39H9DGLCC*%Geq=8>)kZ=I zuB(*J_Z&79f)7f0;U2TmlYP6_IJMrZ>@3sb0TPUH5cNM#K3(x<=T^+!HV|fpch)m= zyLx7&y7uJ3<40DQJQ5laG2nxayiHSo-5D? z$~!q+7vhfIok}xeXrGITX8X!#`QufY0RdN@*Z+R_XS$qlVUt#&U+SB7QvU-G(a?d) z&q{B)ZZ=Vl9rpA754Fny7S#qFwGMF%8IMzNlPxik4x~e2euV-SpAH=%(_RQ#w#cTt zx?GD*wv!E#sENu6I|a9@jgLQ^gN)tZQ!FumqJ79U1?5ITrv~D{m0hhLu#kr*$G|-3 zYJ$)|M^-c%|-a}UAo=FE@u&Fq!>kQnS0-z@5eNQh(@Mu!f&ler%5FyYFYX7=s+ z3(%jzB*y<@hLmmc@(?`Khm>xP#Za^)tM^(H@V~njF(O=&2Sc`GSs~9lmi_mf$Ds{41Jmuv)F)aFZ^MgR!xZ{8hVKQfEgN(5S9;pPzFyxEW+Ze*BEGVx-6;X z9_GCVAQSN?F)-FpTd0)CB&>>V?jR?-+w>`*Ol06_-xH4AULCp`OSJIJXu&~4b0EeKMA zleBWA#}f^+6PNyES9R-=r`X*M7Tkqrfru6^H#*7T7hT@>`~rx>oIH^VuLKj!;M}w@ zfdQZdsU4hrjk`#UG<^l)hWV>-EEN z)V?0Q|Fmg|D9#IlQHoG1`f%x6Z6iss7B-(e;#F7X-#^=ka9=s)LLaog1@iKBE8Qbx zsn(xM=oN7xsQd@)aG0^OMtcAi5R&7{E9?Vw|DKO zJ}2v$=Ui~vTMU2s$m+V&m_H-^6ov-|S~L{Bj#srKd2XjN!iZJ;pG>ag{L+YN>lwd9 zT($Aysyi!%+I=G57*@Y?C8_FZ^72D?0s-urEMeUyiDlMpKtQ=4CHxN7^{H!JoFvXg z@S+p;9=`>0N53mu;L`JUE5!rG_+DVUSKpOw8Flh7@UK$Y@5O())fOAAOA_RQvNM4e zK7~MfGb+At7icJO-{WAAKB}4xHzj%jmu2WN95?!O1JDGrlr7B=bFfDLWO#U*DP&RN zjV)8Dy&>6ne%li~fU;jfxd>l-GI5W|z2rh?absRe6-=}^Men~P2~Z5vIP*g~qF;vU zlZ(U7xb)iW!Z+aTT;XrnZQqeI7%2{{)}3@rZyu+OI_1hKC;!H~D$ML|uA8LkMervJ z+9~Gm3A?HWk>ONYt}o(c!V6W;|H7q}hf*9ImwE2moVjFNiogs#))-EqoOL4MJXK^(S+UzU%&* z=94|c-^`k{1X9Ru<6L)@3G|$_TJm=-w2IUI$Z@Ax-~>apD@3hvW1AH*2K<$x0HDcS z%;*Ay^GHdhmf)KJljqUI%JGYlJw8fo9k_!f54FbR|G5Te6bQlLRH@-tz1Js?dr6E``b6v2867-$Z7+N)w^sRxhu(uuCPoGAqE7aL#CGpE`3Oyy4%nD zjWO&S1<2oQVLt=E-d1mSC@*?T~6A&(NXX)sH>UQ%GWg!AU!3i8+krR=)@e3II$QrXy~T< z!6>Xm#*}F#u?h}KU4PdEw%HEHHOx;sNY?S=s&nDXdKyaodI0O!Z4}mmpTT-*_sBIo$je|O{-RE~tl*SBOCDYwgY807vAd z7Sp8DeX2eaC?O^^+mqW^kv$QvRZ7erKI#w_gg-scJO-UNX9%Ea9TfF=RpGBAlwH;Y?dkvngU!}S@4N4;0s!c3L2Ku zxnuU+(zA5o$40Vb{iIa6A`!QWl=O(vg3ej8{s(sTG~RAsXmD@1^Vk1c(J|~th$FZN z%9@W}Qz2wj8#`vyUGjv_cde@$A9B(uB`QZqX}zG`h~y28)_m|p8`davaeMQ5e(pW) zii(u7QBFzhQ(5!LT|zXA_cTFO>);4&&K)r;Fd8A8U&6hJ+9)Ql>=7N=z#d6T8vRja z@Noj@Ccf{_y=972^?bZ1b%8Cp#IIUC)Y=xw?3JyRbM5~v{Nyu9npQh}!QyDa@i0TY zRlnau`thkWf?1kKj(&6?7sKoSizyIT$fUVBob%D?x2rXGwiWJgVb9lq1(z~&C3pEl z^p?pgFX*{SL@hXGS;t`HQAYM^k#|hjz=5{pAL_`%o0m~8n#B8CxBgm0SSbZ)JH?|P zYTEXE$g~aNAX8EYIJ_vMcxT|Nl(X)1jI0lemkQPAYc7)O>DCiqwg!}@;p?^WZd_yU zZ7H{KR*Sdy@+y-4dE&1OQTdjk15&)DJq37ZeZ9Os+uiy1Jj)hfwstqS%O8oW zQ0!LG5;+l`%efC+0$fvfD@@y2yzPC(zyJ89IdeK3qw3>VE+B@fkr_rt7Nv4)u_33x zx=UZAIK8vY4f^@SEldtXN?u4fE+8+ZMlPO$ISt)2T8#2D17OX-VXbwh?c42lOW2_P zaQSZgDGi!5EPQ<-mz9~QC8wBSlH9i|=IHt12rCt&b0arfwD1P~;)Wke@-$d7FgD&c znDMTw5w=ME20%8}h}?-a+&VhCO=mKU`oGNtkGIxFYOV$5#?JL2&96Y|_;3&7Qgl{O z)eG`^0G#>}Jg3TFMHHsv+W4ynq_OdM@f?7_@OMb_ciUHo$%jX2ak+fVL0jn2+;iz! z4P2K+CjRGY`fllsP0s)L_BUr~Nw5@8CngyFNC5_r@%saFVmkvz+X5I$uDVl?uZAfY zM%Ze-Jz^BkWM!}*h_Gz?zq+VDCXQOKxchRS8H74jzSzd!^vBx=Z2&ani2R5x{35M0 z%v0o5gq3rKdY@u}r-A^^d8$U(d^Uk-$MO4nJNim+NKo!Jkmk796sAz%f|gdUbyMKc z=+{t|0^GJl-(U~?5z|Mz;h0Tb1}VR-!bl0k*?iinMb#MojbdZLC^(1@sJfm{hc)1@ zhSkz#EVcFB8+;B@Y##NaZhp9N_Yz`FTlLN0|1GnC%m%8yA(f7X}r%MWd8{v4a?1|gz@*qjvy%b zP$u@`N-_!|w&nZ+diLt>vIC1ag1DS!#5 zl?)&4Wq8I&hj=ATuxhj`@j+nfKixu?Ldk-15 zc(!#xEPQ?j^i+Smf5_;IAhf4jc3{M1(=8%7W7J#=vyPIh8fw?g$`4Yfc*nA3l~?L3 z*$#8j5!qRf=&5Q$goJh`!8-{{93?;kE?u_i{q5vER?F=ICK#%l(pcH{EDNXrbEyg} z!YoCE?4K}XTSiv-OoLCu{?VAc2x*_)Se(zRtUAiFUqUz_*~!nCIEk_fh%J1wS=jU@ zoIF&}tO=J~q850~ zW_ocj!(D1{z)u;NIIdfg*!yNu3U^rybnhC-U7rp#+J(WNBaffEEpiy$MCGNSkpX~` zR2F=ywZk30L{Gb|T^id6p;Uh?U5h7D{K^VA@=S@yB9(yWT`y#Co--%ZzfP%X**CtNR5Eg6$jggZdV{^EU zNn(Fzw12HU8L?preXv^xLbTDc%^@>aTuK8L%)N@(+;|@SoGrBcKFC+1{x_-EOI@HI zIIJ*eOJjbES<_kL=(?MQwayUK-)6uNj(|AGX`Jrw(NjjoqWpefH8wg0b5HX@EP%$x z!0T|2XuiL7t)vjsI+}4V*I!R4<5lEI?wv{@lc{Ja8e{7BWuIb$HK6^T9|c0-@VwDu z^Ys6*!onAIdzR2RX+!+&r|*Jd*}VxjD-Ob*)300D66XB!HLVNPqYaC65CJz4w>!T8 zWr*DBy|v-P@mYNHK_5IcIU-C z_uSdf9Nxe^XHzL^60}iM5Mo-blg@(*m|Kj0k|F#h)&+2W4-UTh{9>VM+4*AcJ;lqf zD969OBaM}B2F)+GXs~qWu=4~fS(n@Y<+)#siAIw4#DbH(%2RJrZAVsC`IDfJf83^`zq1#)gt(}D z{bH1D|HXX6*Qm5M=dxtYRIIs15dQh4H zTBC@^^yT8V_*ENWnVg^=j;5u66& zb}M?+`7XX0X4|EovlZ`fwOD*-bzUL-Pwm{1beFq>b$hye{AxYlAIGARueleD9C0j8 zuec%$Zq2V079^D3xvt(Bb{r`9ZJ4>%Lx91jkPa^!&s$@MQSgCoaJ(m!& zRb`)HZeGm!V?e+>q~i`EsfHq?Ony<4`s{mJ&)0|tmhv8boaOu29lp+*Qd)nFjM%!{ zqHP9A(CnpV{ccbS1G#zYr}vfU$huuwfrC7a3XJG?7UK46m%^s}p>OHfnx_`-E%L{+ zA3o@b+*VEJmg((seS1UfZx2g{n@NY(hWU^{d-y%eA9j^><{>O@-==>iVKfx3PF@2C z-DF2MT?ZbtS(3~XCzK1cC2Yj#KesE@04x38wVla9=M<4qVa5tCS)8bQC_m=Nl5sbx9FCvv~m_A=Mb9C2bf`f9@Rj?O^+UYU|y! z&lxe7SGk{1m+m&yoF;WG{VrAmeEZ_ma6R>v^5?CQcc?es%hErJJCV4&C)rg8_x!$5 zKu1yqwVKXfAK%T=u=|qVW9Z z&~{6obQq`I7@M@OW~1_zTOU(k#c9feTNXHdeE1Zh_wa8_4)aI#{nN)5sDOXe5b}HE z6H7(jd%0UFvYPMRnr0Tr8-8`}mo&*+%;bBSTFiJmL2vh<6bt6|iQ5xjh+Le7Gx`Y|b;L(Eevek5U}?l$=!Hb62qThLLI$*I6wk zqp0-rZtEVe`mEB@PW=_NpRXC11nff!@5!h@B_xzHOT5gtJOD)ye39>(jl^1ATP z%u;aa_B8+e{L9ZNhc9;~(55z*h26#64Gv80{H=3^Qd%;84Aa~?ZiHz3B?!bSd8R$w zadqp{=zE;g|8F|>8o$4{EZ)467~Ik;m#gZ~>;UxI+VD}Eb5Hsel=3ZlVb9ng@-9YY zK%lYM>05K@njPf=k$B_$sR6C6h|SSxan)9@2cvF0*@92hwI2Ab3mrvz-Z(AzccMgQ zt0MKQsL*{jnE*!Vrm8?R&yGWrsMp23$J2w~%EiBSY1&?$G#T)U$TsxYV6)E|{?-|BlO z(8qC5u0PmerlLu9p(dvHZGlGA(mXJR1Ivl3?BvTNu9Djdvtq7AUGm@KOSz?4`bY53 zC%?u)^~uPBEU-?J=5nxoWh0uOd6L2(^@lviCD%%^dXy-fc*N zv{Wu~=dYin_B|{g|0F>SLX z5}B^S!L@HLmNhwXJo?iea@&2k^VZ09XS2bIA zK|L)+>3 zKh@%+7HdJ(B2W4zV4t;W99LD+wiK5tZrtYx3Z_)uL!7p$+s+9BmkIqw~zwe)F!Y!$?ht@y=qX^_!+wL{Lr^ z(~bL=-K;@Hr0bc3^BO%ZQJeSkFi%tXhtKgI4Nyd8436${+Uw?Ba^AwaqKo*56{`=B%VF5(evsCGo;^as>TQ^P~%(mwg(j8L= zqn#bQ^G=oqt|^Gs>RL>m-L*2;l-$nu#?N0ictLMn?5!!aL6}{x8kbdTTiosoy^9_B z`L6HE#VU0}bHddu=6Tgy>J8O1m*m}@sxv{mx2&Fw$$nP)H*EgybAa>ShwfhZnw?O^Z~gU9nW87vJEQ>IjQZ{9KoySMuTpAtkk-LzTPP+!e94V-Jo{QQ}~gN zMy`H=YnRWB$Lpw`ub+BcP6=~IqxwF5w=Q^N;VxLIL*nEu5c>1jy7jZ=jdwbf zHnXQ+g87vU zgWPmg6;BC7qHfgH5!N%OqCSt}QWY6@3d$6|4=2Oh5E00g;2IV#6YGkabQ|;LxL_by zao#tDD{&L-hBHdAi*Nq(I{*3CYj*0)Zqy#uGc&hi$U-t&B}%ylc4a*9Plbtjj95Z-2<%1?E$h}{1b!!r7(A^>tU#(vtULTL7Ung_^Y z`ldi?8~c9am(V%6@8Wy+C%afm);h&clQSmhm&N|Qw*GvO@UzG0$G2q3YUVsA{YVdb zjWDW9=0z6pg^>NZMF@*IUi9RjYxNHO+@oI@ywWtOLB)eL`jp2|M(%HmI+d9O9*rjE zcV7guM4Ser`8U#(o@KU<+{=(J2Fz`&J8CBLU#mBFG~x0-eIHC;1fVhld%B^&vXP=i z11q?=No=0~ZuUjdaIT&E-!BL19s2wYo)ma_Ik|Fgx{L|Ao~8S=byQ2}*5D96!rDj3 z=AI<%K8xb|WejOR>_KnM9++^C8e5%7xTep0 z+oS%W|2#|&aE9S{_499;v{$lF!vK-9;Oj@Ltp8rzG@uBYOVn?7rMs@?M;EbRjRfC8 zGG+PDTCd{4GBMWYBj#3NKc{$JG#aiLqEUY@v+>Vd+iW?tu)<=(dCW9%`r>in0Gjpk z-lv~ocOH3;Y(A{gZ&~&FSojyPuPFI=Ze#AF;QifyEci_I{%o+G$aNcj1=_|&fa!W{ z_~6-w2m6oIwniUC4vA^K0d6_bMlj|~Z|x~ z#>2(-0K`|m_ue7jLzXoaI0tnser!~x%U-W|CzZy>+2tZkc_&f2B>QIbA+4wUz5e+GKm`qlz8VVs zoSFa2)?N^t829RdnuJKettAE1N!4iveO+Z%P!aHFrnPTM6n&?Riq4_)L0Q zTJ_g-N>yZ@{-AquSAWeeX~byfK^ox30*&GKv+P>j@Jh^>pEBCcj~ff#xrzD& zOSdeks{gbkgVVXjlO23s&FbuANG{&OHMT#YM}1IHeoCCpyX_gWVmR~ba^uXYI#5iM z-|LT%#bUPA^@9h9{WBYsN}cI=oc31aOR>Kt0ad@-J-*+Qr^tV zgJH#xho^sRN0}z9e+&e9hDolUaVWb2`zEy@amP@P=mu>`cM^X#637~I?3MDG?Ho=k zp~30;h7RFRmfy(5=qN%9qfoQ+b9{ZT?Pso@<5^Ybv5Kd!ZhvLWI{CF4hvT!f;2fa# z;ZXnYDufFI({JrS79eIgEWOH))iEZ2-emdAr@MK4Rxc#EG8<*n%oR0T-e@8mZF`<> z)HKTLvXnOUnlR#QA|)UHoHoYV-dDNhVY|R4^77os@h<=Tqm* zzv&ErzrAc}YdvlD_H|H1xXzK81G5b()}{ZdLQ7eLxrDf-x)baTZMhQQV35R)thA4b zkhco3W)E(>n>Kx}ndNtIcU-+V+u#d5LumdRz*|y6#qP|O7T;74snZ^p@0iOda(d6< zBYW}F+nuN7$7Z*Ei(9VPXC_47A)Z%CgiyRbw9Ecb@NRn>r^zeTq#OJZ5VCo=zi8NqQcyv9C!mBH! z>DHt3=sS=ZUyxgP*(X|BNdtArCK>B}VP0EpTIm`~8z72R$H88=>mcv% zWBMWq3zv_9UK|f@d}PIz6st@Z@}c)m-AXihahV`(aB#tw9mz za8~au1mie!0lP1&!3nj5^#JTj*td&}D88T8U?FL{=%>4v zj$T8m8;sgyjsD@vH6gRlts-t7M1K>VMsLJ>EaI)3_#$)|7}pm0Y;0Au_6!z}-Z>#q0jjV*1Llenpk&N#5IrL$3yY6W*F< zA^W{Io`vbuf6#W+&KGOe7UimGi}|i8FjK0BKY$GNQN0>Y*9Wv*E;19<-fYE#wP)-< z-FP~^!s^Rz*!lU!Twh+Q0o%py;_{pQ@6H!zy}O4ExpQMxBl3f8%p)cZjPzJUtTZvHh(#O&Sqcug`xN=17LmkBKa1a&&mtXgnPuxQmmpM{6j45aHjd!^A!$j zS)Po+MZFJJ6<60kzbpyg3)Ga{jkO{H_$l{zdps2#XsrI${z$#RyA%Z3L6S3FK`_C zz09a$`-G6mu6-}>Yg_u%P9w*gV9j%rw|3*a(%{z?He=-hc86yQ6GENaqDz&$l#E8! z;F=}Y)U6I*%{;>6+jU*nChJb=iZQEpyPY|CWlHExMK#h&1301K+DeZQ?D@T0y2M_+ z^zFC(XYX6-H2mebtitJg;P;B_T}SHK>F_|osaOS=2;zc$NU7rov1py!EIow>(|yF z3VfMoC(#3)Nf$eKp81ch6+VkM1$?Ekv>VZinZwP;hx9>a; z%QJ~H_9 z(%U`vAKd)dCoi*Uc@Tem&r~cd^7aDM`1>K;?l9-)&tTR_R?R%~2VgeTJ=235;Q<&4 zvhisTgAd+W*ZT~En-Zq2m+&87RLf}7X zFO2UpPT;0o){}E3Rbr3~4poZA9e!sR2lE-)3YaNfWOJF*Iqt!uZ=X2Y4(y&~{~5=l z9S(1lom)0JpKT2Uu2G5zT0pcMLnD=^;Pxc64Y= zU*FN*!x-Gr9%FB%GA63;LW=w3VNWyTVDk9G9wRCui1F}CxZkpLih=hou$qodx=ARI4HnYuW+*s-;v+gHiZH;A5o zzu338IQjoi;_O}QFzDh1`qG287W)>Cdb)!Ct~8viY!kJz)%>xm*F7fscysanVaFb$ zH@a^zsPg#7;vVDI@p1Il$%Yj_N$yPcG4{Tm?2i!@IXZd7nCUxNTJiY#))8ZJv2U{P z#0gL5j}CTvgCcl=F+Ugg)))aXKl_6I?w_iGU{+op1@HtpCw(|D%roG4z<2V@aa_wPu-^SfL~WY4m>+lm9p5 z=6`~M|26R^w2I{L9|Q3p`JcYv|5}&-BhLSky#I~-pV;)je%t^0&wxynrZ6NFEc;^39u<`=nz_S26hYi3P$OR(Rth4q3{D9n` z8d5Nj9Ylhk1!@62fUGQOCu#mz=9qxS08u_p5P_r)O2GeAvjUJO1@%9f1Q7h^ zK!TTvI|cv&fMS7$0B|f&8vud;p|Py}0MS_HFaR%v2^Gs>2!zM7k^#aHW}jH@ejqZI zeIEdUuwY~P44EurIkbUL2rD60u%F2}mNN{9fUr?xMGTq!V!6pcZ3z2ntav{&CYEO( zXb1s;;-m~&@UeW_OehEkG)}Icr6g7$jL8-PM#aGlSxK>iWF|)lr%#-6KPx3xXrIXg z!i9}fGh`c&713t)fp8PzH2c}+V#UIkgCVD=aXN#^cw<}e7)YMg#QJ3Ur%pE(x7 z3yL=~1cBqEv{|qaK4`pIKS(rACX6Ku!jFna8*;$oV5CNZf>wXSooWed!734HF z-p&wg83)s5B|roT@%H^-=QzbM))vSaYP^#nr(c{hnUxF?T8($@=fuRR?z8qogh2_O zhFthKHElL3Lv(E#NHUwdTMi6jfoHj2= z+6ane9e{`?CWP_wN*klF97a;`#3VAWu(XK}mU}=7nV7QA3z0U(V)=}uEfaCtd{AjK z0#bj1M7gPQ{8C$@nFvlliozEmpDO12UMz%zZvXX%q-2Wh9GF%+}^dNu!}S zxdGXd#N04`TWL!a4rU}rO3WwoJ4##m;FJgCD2avp{2tO5u{bp&`SHYJZ2=!?YXVMl zKz=Tg8 ziYC>Doz9ZJjKZUhVDO}R^64V!D?WJZ0T?o=VgGcM^i?e0&IoRqMAQ}}NZS+e_5*O| zq{cA87HJ16-pNSOFR6(vNS3~~igz7Q#3VKE3-(Jpf>J$=l<-Nd+GnWJPS8~E0i}|p zr(tKNq@7WzzDCNVq;~R|S!oxa)W88{N>a!EnI-A#*whdsmGPubZJ||ZS3+v&fXZA_ zSD4VYv>P=w+(>mjsfR4IFYUgX8a1FwPwLwj0?2rP(&CH|;N$^qVUUa`G%aZWA(}iC zCd?~y1C@p|QiCUtkcEY1ynNEq2h@AFETS!Qdo`_m;2b7-a$m$y#s`#MZKQ!ueyuHvlJSM6*AHlvBu|Hl z+RFH$(uqczq~sa0sH2R(PkPgUCMEg(zNm*x05-kV==^x{2W>GQnLt8%`@s3RGN3J*@-^&i zmdssL2F*wZp0Z3nTO@POC*$RS4l-qB|7?}aeQd^SBcx@@nzlGWCX|pdGk|nX*$5MF zk$FJP_+X^#m$FF~C(Ars%~%-F#iVTQi}%ZffigcE>ETmOav@V?!l9YV19~MXJ7E%2 zG8k0mnvp&!WtS{5D-+?9xjCRuN!i<%SdxjvX8tm|FrIRtEx9TaMabM8xG}Rrpe&%VAs7qPfq-P=pjoVghN4*Ja0st# zJSvOB*a(hg?Su%+CirA=4;mq{><18tY$7&`&)C=!%b_C$l}#dK2@V=NV>!d65VFbC zED>W9KP-2rl(uZjYL@t*2?on^AY~|v1!YSao8qy2I?^av95h>Q(6j_A5H4*ii$`U{ zjLk?`!A@yM*;JowXREdXidRs9mp@qmSJ;4j4zGjbaWI}Wy=Y(u^^IMG_#_>)%iav5pg#7kFhdI1UxMHx}6KZ~^@wIilYo}tET=Qyv z&)_vW-u*x^R;~q9FktKmPW9AL!pgNm3q}SVMN_@Pm9pg8Pz5w&CwQuNr&5vJQ=fvD zgHFiQ+XqTja?h{@uZ^88Q+;)m33Ba(f|)^Q=T!f2HAF{+D%S}uTpqk$l6o&(WlD~MDqJ&mC8dUTs?5rD`4nyr zx>8af9;htIbz=*E8M}?AhU=)ttjhHe3U>$H=29cVRk!7Osf7o|?(3;hovQnCeXE5> zgYNXym;+USd_SlNXyO4*i_<}X(5r>HXcPuc*a3Cb!qq$~(`6$xp5p>kr+g zr&S)Di;Uq4*6r! zn-4Vmqo+bhK^j4koRQV6kQtzRFlJuwH=cnZ7P^G>mfu!{I&hxYK^FF14LxGg^ zj)U_{@*lCKAtram(>ryvR^=B6rJ+N2=F+>uwYKFyQA@*3g4WY}I<@xY7gtN8hJxtn zeFs_qg(Xl~oJlY^V?ak6r0^M9mNXPBnlTiv&8zSQRfaPOfoF_#Y6~lT^(jjq3PEO! z9cV)ozG2I$R?EtV?qM<} z4|EI_Rzc;}Cin3fuXT_pg*9k-{m}iAjOlQst-?C0oM;kC%9!ayIx1}Vls659QZn8j zAUza*V9Q%g9*k#v(9!i#*d&y<4?UR6m=D(tR`^LR?=*S1p0UuW8>X`{M2}}4=oqXj91$vahoa{)8Q})o3dhvS1CyBb%%e_&eT9Fkl}AG{^vr(; z1^_64qY7vm3(f)}4M9*KtcrCwRy2zlW5^3-LRWE^#=)~#DTcyOX5T9A;W%U#JKYch zWx-YPnZ{dYaUhMLP*!4<;BdTi7AM9C0cE3AiI^t%WpPuCw4v;4RpP@5m@FQ;ks%bs zQ7vVfh|l6f8l#{buxh#C#F8uljIk{gjIM^6CXuoPDaMXaPTy+f;Ur3y5Z%}V%7v>| zGff`P5pe3i#Go595&83Ut#d=xJPyohjZj z8-_F^Kn004_QQDRY(q6*FI$;nMurNl)wm9)VzO1~X8llMj#^LCG<>!i z(wqtvfz^5sr zR-xj=+R)*QxolmG#WqxeRvT`bxt^_0vDk-7uGK~jXVS9`=oSDNgrhFbGz*+#ghYX0 zQn0$D;VjV{6AX$MCXKGcnP$Ut%qS>fn2c{-`fxTf$AXT6z+`cC*`_&`IcOvr3X>z& zT|%_rs9Q!E`}ioW$t!}*k)YjjHwm=dnO)wEzd#|dfW15+l}w+|Q0 z<+xz1f?+DO`cBir^&D4rJ^>1Uu*Z=;d%RGWRz9ViimS_vp20iDj-Y(wYF%BtDuME^*HF z$5^+(&eI-!FfH}V4Ww9;VOndC7KTePxj}U6ewa2#!)Mbnd~OKx5*4NcYgisGE6Kfw zxikesq8rvs%SpMRluI$QFkRn<&EaxN?nC;eC72$r;g@N}cy2h-W)-GSY}g&Hn9Gg8 z*lfct&>9X*E7x7y&8on8aY$Pb+z3Ws9jOw{OTgIj z!i~`c4zp@_UJ}Jt7;fTA;2x<)=B3bWA#hV1fzPbQG7pEegTl>-1i_IS=e$&m9RhAn zBZ!#Q`sJll?6lz)YXtF;lL(zmx}6~$#X*!ZtHbAIBQK-iXc$p$q^=|{7jxMbZiyzs z%<4&b`IO6!a4TP;@<=@;uaJJ(1AY-lR5N=to>z>#;sduP5;aF2&E=J1t^~s`(TF-` z4eNR3lq)9yg=<9pkp_BRCH+b)+?Io6WJUnzS0k@t;dU^R*$6>2zZP>f3w{|*LYooc z`Sp~mMer-WB_Xw_5`>+kz_wYa?WqW*tftPXe1}IM!)O>4Zq~e>-$S{! z4|iW{j2daC=l9XC0Tewr9>ruva*B?Z$MXIn)-bQ966om4PGaduSn z_ibt#X{Qvtr#pKn2H=`n&7O}Jd_cPRCE@2NwZ#JwQrvuol+Qe$gW42oGMRcytSg`2f6#waY_TDZ#ge zdvpL<#Ln=5C?(=r_{;|_i#T*Wp-M@_7QxX$=OWGsPlQr3twqFq$ghaI%Trq^WvxYg zbO=+#!|*gz!g91qnGfTO_;hcelyIKw^ey` zgi<8LxZ$CchHF(bA000e(e?6CN+-5zj*iY1iA8t?D`n7Hb_p@NQ8mptU)fPxuupcX^YQ3fJ0PM<+1F zstoUbr6P`}p5`y`#cH~@s7l4Kr{1G4N{Y`#+?rA5x2LMDrir`%_rB3^}BBGD^;#NjT)V# z7aK5c1C*;cp2e9@flG{ZeL%|9uxCl5Q=%m%5k9=iHRxwJ^Vjeavo0TD7%ca zB^C@Hh;kk7S+@Bb%M!G%FI2gn_$+_)jdO`rgfBw*5$##A`Lth&b(gQUa>LrQ^3iEb zi4DWoP?^BdUTyvsUt*{0hf*fO+UrN(mXutH@UvAWq1%b(Go%vxEyFqwnb@ z?hOA}+~1H9C`E1*dEsqgcbqaTo^ zw;2Id%Fl4mUz^Wamip=j5|rDC&u2#GoJ;*90$Y@y)1H4YpZ6;b>7qq>6jmHXB@jz$;h zr7?^kfJ#3H8ECNtE{oF*2B{3d$gE>aqGbsY!MrMiXflVzXLwmsSFo_kkT02g>@%_~ zg%J!<8OD+MEWTKl;dDcwDkDU);Mf=EvebwWgvuz5EMoE1uPnVQL|bKSjVwO)6;qbU z2r*Qla&$^re8ZPz>)u7F&|sZ%W8X^3awG2As*IyMVHV4zviz>Qjw%zroyud&l(Is` zT@RHPxK1^T@8e~~y7zokUJ^Ss$G*>%l}6kPR(VD1)UjAuFDvi57p5|~)~P?XLNBXi z+>2G2;-DB=tb)s{b?;+UUc)G6W2>U&wGsETRNkN|Xp1#?d41RYB9&=hiuKqUvb=$D zze?pTj$&uAZdp#$4JD||5GnR!>(1qk5uq(A?`RY!iw(c>rmj%3%KJ5n>(~aSyqOW& zuQJQgV7+ckKbPv&DmsBy+g>h8f@a<_D+oe>zXJ9;3 zf8csrE&h&Ie9(>XQQai=w2%FrtC)|72v+?`>*=)ETd!E?iU?EPTI=Z<+oM-3G9qGC zw>f$TEcU^bpLHX#s=r{pBV+rbm0u$wvs8bhdubL2@XF<`$RgDp-`gV|_HB+GQ7ZQs(Mzg_ zxV~Q&$K#a;x-qM&N5sC}vE#W)MnufE>M^bFz~bL}G)Ka7PSSn&OP zC?=~a4!t-if|b-SNM&-V;*5+#AlSzHMNrKCRovZi+6eabesL;uL>13roFM`P9*{z@ zq*n3i#iI}$@Buk0OKFurWV|f`Y&if!u{Kr-cE>v+IQ<5csjOX9LWl7l2rm498j5YA zN<=Tg2fSBXU?1S3w359pxSH>$+D6T%QY>jU~!c1D%tVL~i|7d&W$0&!ML z=_O(jeDFatDoCtaCNeP#!EZT;MsX-s%XKFfAq4yet*IQk)e47+RfyC0K|2)Ksv4%3 zL_i3V2JNX}mukhxq!z@P@j)jPr+>9_cM=&Pv_9xc<&3CSJxuCH2!n?_QCz9jYI@03 zga~}do61#MeJ(P23L$Da2xpkC0p+ilXu`stpcP0BR8Ma2$%4v&Kjd3sRGU z4<}K1#cE6N8zQT$dlXgwTMO^!62 zPvv*1v5LeY)a1v9i%|mpHP+oYZ8e4U;c}`#M2*cM&QJ{s9;rs1POY)i!=uz-@R54z z>C&1jk$77*xa9~DCD>SF-;H-vQ}i2Yq6&7^Tsy>js43w`T2W^vYMk^^ebkgmBkj~P z^EED!sljS0<0G9Yp^X~X?$j_f)%B4cst}{b{V+9F4FMh;NUXQ5)5rxvyrhK6XTvWYoqSW&+d=!Bij`!dVxmmjzNcf>T*(5V5+1 z$ShuUV@oOrT1v4lsXI$p-NcW|O_S2COF7JfsGH)cd}wK_I-FiMRNah36{JbK)TKsd zBh<~usUm0@|GM<R?^XgE!dCR7B?n+aPsBb>Z?^kyOPk5r0QtMmw3aIK%@Ck34Qfd9u$bu<#XUhp+ zv~pv8dw0RCx{KdLAWgZezT>c9N&Py0A_T26QQxUoxT@|-nh2$-%-45C7H+G%jZcK5 zRX6H;x(oN!-Pb3gXsV3*zQaPmIS=rQI5dJ&{v;Sip7N9RtRRbsCJTT~L>m~!65xC9 zK8x>t??6GOGvn50vbI>ywEbk2FZs+O5Bh%!v=LD{iOyjWWCEOo?EF^&{;QTcgar3H zgf0pm{$&zm;RgV==Q;oU`)B#k>mBc@w*hTi6rH%OvNY)J0EYpma@KK-Zt4U9P<QO7{G9gq#vZujLZ;W5lA}xP zAvsL61`xXx>#bW{dKA@Ci&DTtof{&`@^ShN_B+My@QRj7Hj>MNMk@>l7vw?zaMSX3}gHmIs^roLojk1D% z|Ni9uT(E9L*#_IW%<|sBAWz4hs z&FKxvlbmtiFS$83{r(k(ZCC!X;9|4ejG0X1nFm~7m8&dnAZxRAY}LliEQ4hmYt=+Z zGFM~J>sbNABqcxO7_Udj#zv$j&g-TzSqi}VtQ)4vX7uDwDDcJ)fuW4(YhX6d4}>?0 zXvW@oLemeVhu6avC$%+VDUlpYM1SZ6|35r_hmzS(90cmA2>M88i#nQ>8sNVMvqVRUMSw1TyU=W6J)^@=AlY7 z@5%ooB$vX^;7&ZnKHMg`L1AQZHvktjL{@ozOz;QI?zSq+ByD9 zcfF}M&R8_O2*-FgeB?U)Z?J&j3%L`1l?+K#E`J6NIb>Ckb?=|s%;U0M(U_JJrdV)v ze)l#s@jL&toPMBU!p%MXjNNj>z^!e#J{S1KEc3wR=gZIOovVXqmM_@Ip9y?pVV6g5 za=orPv!GEt=w;56%hh&P#1aYc1g zC8Ip%mtQ$^>)5%FA4NH+atGfpe5A@P#N-}uS^66R4sf5QEZ6#87#F`3_ne*m0%ski zOM~K}y)CNcm_yJn;&iT`ZVb3H!|@_9xs45N{O2dbN2H>Zk_We=r_; z-`Xrls3X!*M|40V_P56Id%NV<;&IG0H5xg;h~skp-;BI?K|&4-FbZ+&i2TOdOnFt zt78u|dc5pl=z(zf^9BuGxtL;uO{#g#39#nL-q^_M=VDG&didj;jW|H=@<2er0Nluh z#nN3e8#`DPP@nTF@6?>(lr^AB|zH0977hke(z;(lr0-h{=TIn2--)lZS4^#|9Q-F|2+~g zOJl(rq-x0q7`!GRN)G?bKp^OqsCCXGScVn~sW1PR-FD4=gd!kNld=2fVQ8FYn_4K= z?}-oV+GYXXgtJNjy)NYg)4YZN$Y{9cC<ylB;$n4mlP=*oqRw3NTJRQH1A5-v`)Xad;NeiXUBTB(l}0-E z3C@Em>W$UQV8aP=nV_QmRmd*J-2S+_M??6Z2v~UhGOM1c3{wWJ76m6>!b6Up ziueIluK}G+g&@urh=TtVnWv8Mn=^a+A-Y~kwH*;W6&KXZ&YII7KnV6Wi*2wn{_1I? zS&KqN%RhkUQI$VQ8@*E*>5tUBgFg7mRDqn1ImT{D3%m) zv8H)cu|lf;E@qf1n|JpCagFDdMUZDmgCyf{4PPbWke8@Xl>$#Pm-8h-I*}qa?#Z?U zC?ZAdE3vQ5J*nUVnxnr_xOy}3lz1lM0&VeJAZn-}&anLlaJC3C%hJ~34J2R7&ShZT zefw>o(4mhHK(-(WtnQeP!#i3Yi`{3XJyf-=P@XQd{O5xv=4Mo_y0#%2r&NDUSKLA$_eTeOFm;H;yI-8;(1 z%F---Jm;jpsL>7Gr{Fa=RJJU&%6Q|6ljK2)w-Rk|w1ArBTS!UY<|s@o6i`O6hC3eV z(?|(8qXiH;Cf?O#Dpyjs`)L~Y*qolQNMds|PWgcB7D4>#R~?k3LIm^*NtV1LNlSeg zEqx`CUc>8Nt4tRik#o#7Zo_Vclp&`!ij_ru(j*mHzbY$2l1An%93U+1tCc?4XqElw zJW;)(ba*U|vt%_}YpiQ{wb&f?Qq-#D@ngum$iO^8}YMOc2 zbKN%Qza7keaz}0EcOZ0I@%}|YrYY}pJ{(kf>MmdvoLG#J5rUHQDt<|Z?jgGK)58hb zX@*8e&(R$N)GK#;rt=agKJx@Wvt3&OaWC2M$r9)Ax?PPBt&kgO6)m;Z57lf>ZXG{a z#u2m%dx9c9Lavrzi}%fmjY;leJAXu*#TPT=Rd7h ziAi2C$xwEfFBejnY{whcZ}M{)BqoCtmgDO*lSvW!3`UyU;g)sT6QMO5!+2z|(Usy> z$uqLcWgC+ZwpT_DT;p0}ZFmL2@9a+evp_;uh@R%K3>N9)rn^_0ZYfq4Z~+M1NNME^Z-_wP)FAAv3^wOkq{D;`p;oYMogquOa@!DLmN7@u z9c%qd>JwKuYz&9iEX7D_M++_b?14c_CZleJ}1M^q`u>tV&OBShmCSkIVR%7f0FGh_CSEW2C8}lXyO>+=e*EVWg5j zlD%lz87%!vZiF|Sf|Eeup|a_m&7>aQjd;Ca`QxP)Jt`GYebE2(<_L8d%(2}^MO>by zYo}+gs9+eU{)#R)OYw*A7CA|eg-x-5@edy%prxvmB)Kuh;Rc$SR-!kwsrZKXFUUT9 zMAnz@F3jRv?}(Us-WIc93@n;-4H=SmJYyL0XH2ljeTsf}!Cq0?>P($$F;U#rP{c_jB@8XVaR!@UGc&48dm zVq^1nZ$qvbOG|+Kgq11y^YS|Za!^k114zgAp~jFbFL)TIw5 zXT*&1#V|0^((ADjulib|W$LfSYr1IEus>cg+)mZ~v&=e@e9n_e*kR40a+RKV(ln>P z2F_`~+bBcJvCqumA_F~opu%9*iQIa|0Q)EDYjYeDSHOi7E?UO(G&OS~8Men4_fITr zZ{+2LPm9?rn23_ecK57JsDZ@vvXn}=o;pHhL znOZ;M1nq8)2ub98&v1ljaioQO_PA>dvvR7-{tY#+N!mm-qf`_soZH;l%SmWU{^C}{ zREiQ?x`5JxN6K3GA^MSa-P?wi_qd-~vwi?Zq<2veE7PH4M`XL0SI z^`PjcAi*GFzCFr*tW>d;88J4hHWEZ2{UWh zGM@%H_;}jo@OPe0z~~h4gNQ*fiiHA5l4!<+pPdbb@8<@MVEKnDF=5ch&Bzv+kh4TC zgd0pm*&dl0fr0Kc-kb6a{_r(<-p#~w8p#Z7DEG1|k1`s(5}}zQu_-RQfHwTPv|)iS zhf5wz4(WFnvl-=Lj;by{Tp8dd>Hetm7}0-?9JksH8R3yGMaQbH40Fvc9{<$VvEf95 zr+#B(u*ZF3uCp*_peq5hX&{Q4?O)Vo`C~Yz3`B#y?)d}s;q`h(nFAeh zZ98^g5ayOz%C*GysVuLqsX;K7JKm2nNqGzuSK?7vznDyhqok|3TXk zwVjr%p5H%WBf1`axs!1;4N@F)TMFS4w{lkjKqTtyiaIFH2U z8*WOIjQ8V54JdR=n36AXuaaXu-3f4k=1MLMBSYEP7mD6sc_(L&}%r!~EO!f*nu)$H$9eJEt_ftVpr z(|r>=?7ue+X{y_t8xg1?3w6I^tYe%w(u75xT)qS6=vGI>}DplUYM!W#<~dVa*TZFfQh%U6v7^Y6p3j< zVxorss^+552kDx2k<04?%+{bsL=gs;T=(Y_3puhKyec{&!lX6k2yi!ajx1d;dZQZN zTTIfN26C3E^dx90G7IUPQ11lOXVAV!QTx_!s&|lrdFTzHYy)c_H#0*NNlpQJ7E{L{ z&`P3$C#Y)O(GxN9eM!r^g9bZ=YPkD1O}2k<+`||rwNC>xJ%fW3fXRTko%2xJyZ(KH zQcx_0lsHc(R*Y{#9y=414%Ruet!VHEMWc;lqu)ZK=!EgL(En8v&7qo?4wC$wN!J|R*$|hk#dsF0vBaMoc zKS*Ah^{nOcgn2VP}}!gVci3v#i?gEZ`_i2Emba6NW}DB|zRVXLSV1BqG!l zFQNOqui8o^*S1&Y_Bd(SiR((O?Vw!GA*vm|$ly57QFKQpuHp~$d26j5sBVz}Frn$u>i1&u3(JYlLlA}eK@pn*5= zEhQ|_H>S}s@jU~9Inq8gZncF_qnF2u)!_Q)wR2SIa5Q)@K~Ds*YdJmX*e!-=bLS2I zO<*ZR(tz;vSLBlkSU{2}Zg@7~cNriYY>>hEsn^aL zoX!cl1+o^|_&{Rj6SK3{-m5~zA`i6?dO5%{py#S);4$=u%Jx~M(lXTqWB72%?nyoM>cSI2G{B%PiL8iV+TzT1l4g6e}bu;LX$B`%x|UM05=DzLTvb=-P{)=XTqT zEL76wmx*Tdiv1!pAGn3Mx!kx}SM<k`J? ziY2@VP+QFc5*8n4-qw!AWBsz4KH(nDM1IrR=)Z?|v6Kpi!dF7Z^T%W!5wEOcA^H7M zz#5iMy@@Tqpq*h70G;!d<~)OtKI9BwR7{b<^?j-;uA;Jh;2}$;?Gs0xOejgjU$aWI z!p+B-w@vargJ?Uk0?N~TY=sr*h~4+-x5b&=0*Rz7$e5gWdh;QnG{*H}8)6_}S$CA2V3|08p4m9qFnH9&zD7g&XRCEAUktGK zVFl|;tJc4}`wZT0ZkLcG@Wc_=y;?doQWR#LWOsj?&?_N+pN3aSW!BN8_NN9UvUggt zwsW+6wz5K&wr=-FG9Rzp=s%ceMpQ)mvqT)^xMkqie;I#`3dBV4z>41U9-&YM4>}{z z&kX}L{t;TeBbrH}Z+c*Cg77h0L7(|}LUe2)3bqItmQ=7@(vT-3JVDCAEWsy&@&E)1 zf#em?E{uG{rF;0T%gK8FRAg)!uNP9FS9J!T$?4Fx<8^s9k`5IVBc- zAlbcRJ2#cc0&>R4x?PL#0F=Yf34 zb!&)fcCyp?P<1&zu35>5`FA~*pJm4Fvc8)|9v3VfyWn{KIDiaY^{ zAGiBCDx939G;kOUd7@hMr$70C+6mcf{ibYX5xR|S*JO?%e&s|INV+hi;Qpr@MUs=} zuTw5*IM!_4!*a(nC6-;E1h8a`H-XA|RZaSX6V2u5_B`RV#BV@84=okF;rbLzB~@b9 z;-ma7D%%E?$@cQ)UsZGtJv!RIaBF?lwC_qF_mfL?pU%mH@PV^IoLk3r!b$?83uQ~h zb6~*l2_AUkLZJ498z2DT_!Vxp$j^0UubzWUYf43UiT?H0zlgq_pgE{ZGWv+*{+pXLw_ zhq5)-sjHFa>%WK`ojAg_kr)zw)bwVW>i8;ipF=^u_Reiu8+MMTjUF8kd74ZqMuFXQ zz36v4Kk~T!VbTw;YEUL?joZiIl%Cu2_GTQNQyYWKjo2oQHCiUNcUAm1G4KY@YJ3p2 z(Y`(MnP#EaFR%*6>rzddy8`CG>p(6y7%Oa<3HTAFE;xK&R2@EIB3bjg&>94wu_opD z^`lyQUs#1C6420_xXpI;X7D5)qb?2M0};dIV)Hq!T6K*iaK}oByD`x5S$CAo;sO}4 zO-KSig>R3*o(r31kqP}_!x(+Mr@`K}WBh!0m{Rj&o zwGN5lXc6mDF;bB0L$#gGU>FJwW8@Eu-f=2l@`g_bf0N@U1y=K8R$aFYxR~_qQ_o;S18`19A}Bpx_Fm4$wJ)Y4i?T?QMB%B%kEz2rEt1I+Drx;d zq-)zNMb+#xCqi`#c5O*rrEn{%P|D{--lG+EwjF8UE3W-2lsWKDSA-M)d`o2$06MAP z6SYiCyeEy#=cb-kCOFt>Y)YB|T>v3aW^H|lnXqod*0HN=giuN|h3!7m?_mh88!SdN}(&tz$2@n z6$8+ij|C0E?w7*?bXHr}BtAZBV9E+b~N5gCt}?IWP^(!Vj;-tZ{w>`@b{>Ol;pUkaA2V@=tiGv!|jvJxmU+T&8)}^`KModxUOEtwo8P9V05MzPeF4 zeCx10dYPxxCcC&MVnCmltUP7uELdnSlj#?5rI=~KWiw*Gj92*X%wM`SS4G^NOU@V+ z#VnF@3K*gNI#fz>D}rjnE*zD$uJR-ZR-ZS@Rcqo_L&_CzSq?r?Nm658l`+{C4VA3?*V) z1v!+0=z8Z;{jU!OZxzL{H#$2jq^)^v(wR*=XA}1_s&9*ecmgDai%lTO_wo`ra=ct@ z86oUW&03--bXI-cRa~*ptVSwY$PeFw56|G>-1|J&#FInppZH|Yz556&uB2?9v5OD{ zx{W-b)mvgwwW%&8-@Bc=a_GJocX^;b+&ygyrny{cMWd+WV;8JmOy`svEX;vS(;qUM>REDe_(Q!N~3c*U7 zzenYjkt$OQ4mL%W&f4gpF>xfeeBF!bn->@q7$AMB-7o-IZfJSx8%mKh7u2v z`L*_Jb+)4h^w+6&BhRE(yBe8qK=wWFTCL^`LXYxH&0J(FwD-8;Y_q{Z5muS>`-)*f z`t7;~He|^6C*_#2zM+$KnYfTMHX{)PtZr+MaG9{ATq3P#w#+k)X4T*z2Ysav%O!UntU>7o0^h% zE8g%9bW9SIFst<2>86E!ydg`>_Y^tz$Cns-0>qtr$v;O@glq^5z!5r*nB%nj17acQeOL0+&^OFg&Hy||H zxfAiO04X_hZTg-aGKo@VA;j30#)4lG(4|YCy8TRX2+f_hxn?DA3bgBqKfVl!sxvVg zT@%Z3s63zzWAkxp=DEe#U-O+c#gzUrLS^1+eFZrytqnzebx*BQOFEC0X*T&l8FQ3l zFFh1V-b~*Z?u_%+ldf*PnmQx@uQ9}Pk#nt`wC&f#6Igi-%O5pKntwtEE9q7e6UF(? zW<_MIs#n`&o$k~iVp#BR)@AnW+!5QGLgqdogk6i=2S*SVk{SU|_EVtSHzFx#>hYii z-Y7)u;HFpto{3GEbN@!AwCiSFCcj?u$Q*+nnLKku*c5>QY+W43J7@i{ewPs}wV4x6 zT(jsJE!|N#8~;F|o;d1(*E%MC#K^mff9^E32lN1A=Ocl|`JpjSNnhM=&LgC11@cAT zbM!L9qe+)bGRKk`szB&~{KzI9FUQO*1S_&t0Qb3*hFJDQ2A^HgV)@H%R6f#YBF1&H zjp&OP0xH5C$|IO<=*FWUh#S6CsP%a9=H3tV6bvx|&LwgliCD?qIDDVJy*63qq4zxR z00_}6Bf5242cO3ydbbCvR)z}CT)2C{^@LBP-$`y;IC-S8r=jtRE|_xs0uSMdaNmu4 z^He)uaqK*pmE_X)C%TCKF&8YrQR@lzqu?CK3RH}gm=V%!mlujw7SI$%8RnHjt{8w( zAqdLf$NvSkU==upzX7l~dq-d>8s`VbPu`xTh)Ab& zCh8JgwZLjtNKpMDBS&N^9Gtn)hnHo>s=D9{f&FznP8K_CRl|Emt)T;!FVpJbu?b3Q zQs(mkTP3yN5`F@Pk1}YceL@7kE)PylDC#(ezjEXXLvp|bF1g%;0Dgm1??Y2-Gg9js zT-A=N=^j4aKRde%NPTKA&I7S}xm1-WdBE6?OT*7*q;}!5Vte7xq7$y>kM##Dqc-GR zFB=OUdYo&((hw8L0FO~x-ss`2ZPr*K`4v|HO3ENfx%z-$Yt!qM&ge&}_{2w&Wv9)U z82NPKKrV5qjq*qN^ofmQaKXJ+^0;!8SZ|uK9hL%%NqmQHaxgZm&?9ky3YmKtvmVs= zPJsgK?L_qD>`Bb^sS2&6e2gE;hHYd>-K#HQGz7j~-$O-PjWg49Q{s@G;eV25lB*Dn za}@1h*!WFmQU`3{DAN$dU)6=Ymz%}`Z@}Ry*y$)L#ni%_zBUD@fH}^=;-Z#={Up!! z{)-cX;vVQ8=6cApVzO(nO5=$;P#a#R{pc}kdhSK&f?!tPt{;=d1E(K(gQ~_L%8_Jg-z=KeKJCP_*qfNo?;bw)J@l;78m&qb?CFXA)kr4llg8;vK#<@*Xa z-V@M8e<{s+$o!rfwz)ZDeAI{5(% zWppZDK@4aWC0+rOKH=u5K$5~#EU0gN{ey0VjTZG8Q#VsKbOODLYSF+Cl-BFi-OeKt zhvY)AX>z0!_eZ;%a->sqwddIaBNeONg zElr!6VFm4}v_t;P2q&J(zsw&8fV;M}5I@09H*<122Jk?qOrQ&kv(X$6s1iiCms)Pl z;{T>GiwDg=7b6IZju<}9I7|#D-czc}#gm+bCM(R&25X0=Ynrp{%REe_V&up8i13s` zYBP3YQC7Zq|c(J1tjFX@%Z>z+dy@X`S zc{=bsX{e-i9D~|lZRM_V>(#jjn_c75{q%tX6E-miso(Mx64^$J-JV(bd+9Z0edL{u zy+S{%*i*vl0Ke<*Ol{U1=o%5ajvX0&h%jNg7a3Q9(&(#|mFZlKp2lns8n8Z7d+;^v zjqgLa82u3XZnR)(Y(hOW7rT+8I>@+AGokS0`IM5kO1ucP;pz6SAhX$3E;h=nzd7hcb2dECJLS_5Qvb;%nOh3Jtl8{%= z4so6-Tw@jSyl#*a*Hvc{eo=ZM0I?bSA`RAX!5}dK+Re zXm-laeJ8dMa{_B4-cyH+AYm}1s5tadr0GmgIgV+4OKL<-Oe6N_NDNTuw@S?bSJfN> z;Cgd6@>0l^tumF)1~IuDNplm??YT1)8k>j?NKHkgOZ7_7-pT|fhL9AI`sohkl~95d zn;@qf+fUWYak;aw#bbCXPU%90(KV9@124{Ga7_J}t_18>QrO2oy2KMH({e5c;^sBP z#`(ku>&|V(9R7~S3*U@Ji0dYPk!yU&Yt`g~`TMZuF@a7#JxtyG?eM_197sh-_u zSPd5F?3RRD2<&C76v->x(scS^8mdZsIR&iNdl@8C1ml(E73V>K!%(Wh-Pp{Bg=D%L zR{;cdQTq)F8i>@bc(qOd+B0UY`*re>p$`o@>ELb~Kh;tTvL)O!%m8X2BPTcPZVU}{ z={2dgaHe4{=&|Kisglzs%~MS+f2zVCQYxQEs!n=)8J`-bB#`Tmlfzcv z!QXE#`>VT&ik{Zo8(uU%(@MDECbCNFmF-R}S-yUB{xYEJviN19lQBk_aXE?00N+a} zn3&&)^?C*K1m-d{vdh9wR&_qKC1z}$ld-m`Y=Vb30349g<}s9v!@od%@|c4`2FJ7L z$Yp?Om#-bRTW#nv+gDTsMQW`Dkes)*LhET0;X3ZnuV^iY^@zTs>Dcqe7SePSvcXc- z(AfkYv%%OQD3puqF8-+2&|kt@vKihWbFeem1j8Ef_F=t-D87lzbk$htl@AoSSZbEN zSMh+aJKm>Soz%tgW8nJwnN*4Z$4cO-&>>tdK0?%c%27*Z80h@R3mpi3-o!Sp_{Sip17|#Dnq3fh$DcVh{y5VaJ20cnMB;t-UZ~X2u!uD5?J) zAD)OPXh8lFmkK!{sHEEqwCYWys#*#O@a1M??6|JLpX~ZOK=UCYrkE=t$Tw4J@J1Rz z#R{9q+o~g>DThs%7r4KX_E5Y>(}o@|9JHc4#=ZfJppDllH6W+$Hge`{XfQBb0gpNT zqO6&v0+<>JDU!PTs~L7s)8>q#5V#70ej7Hwv5#>ksb6gt=N6Ko$0(1X6|qHo_)}$0 zjT-8NBd|c}o9csJ3LRqwFfQ)yx^|6DQYZs8b6?IqNDx?&}Ek zUohO@vjVTup6OP?6X?rD8$An?gn{qe9yn9JQ%-Mj;M&i;54$EefKS$v?asYS1YegG z=U35ckf*nI*a9A={)!bD@gNjF=DSE#4VRi|Xa%x7VGtT>*;$%IT3JXpWSqxlU8s!p65b)cFzOrSN6P6rF(%~c+x5t6EFHYZZMTX@UA(V8z4WHx;`K=c0-RMGUSU9s@S1+mXhErtSJsS7)}7}!9fY*g^cVo;*>`|FVdvUm*>jD9Bkv+V{-aSBt)Ok=#?+wF@W+`WI z=G&*IteT1Nm1MgmShiH|A-qY<+ktEBjI0WlyBR`iCQI)TQY=16@H3g)KwOW|y*?7H zRHdG<7oIPdCeHhnR<3prT5pzensh6ncV#%fD_aB>3~RE)x*WK5fou_*4H=~wGVVO$ z5~A{sNMD0Ip4}w0KCmj(B^TTJ;xMTA!r`uA4y_TvjZHgm7gO>mqfIh4&$km3tVc zkhV0a0AoCr-oFEnx0a_j2XJSWpV)HP4Eb!ld(~j24>FpC`h}k8z|}kRYZLGGDl-JB zjPcsc?rvxBJ>lv(xaW+CFU;7r#!n#DGQHW+w9!zlKEiA8WW8NP?eLvtUGIH{>UNuq zEea$CrupX14Oz=K=P>Haeq7HgGQBKY#9j;Lf#Ip%=s(qk_^?>lk3Y^+;R>J+0Hj_R zmNewZHH@-GlqH8w1u;Qtkw(cD3@Ty<8mJf+j_eHb7 zgD$VVn{&9yT7$E)(8g14Kvqp;j5DxJ3Ij;jTyRrFh^RD*DPN;n1wg@I?;u3LAJ@6t zfB-QMvVCA-6#!W~7RAOTFg${U7dJ*_fGxR9i`D$xbCG5O3N>d%oD)&^&Hw zIl-T4w@|fc*HI^Qp4UEt5VWLn|Rg?x2UAZ_PE5ux6OQ$@x`pb|1HgC$^}ic3p*I2u{%L6j6O z+lNDz7JEaD^|0y$_@=!74L_54pW7 z<UzL=UG%IRRB|7nPmf!4k9tZ|JPq2=rX&HP&8)S&QC&(fXr>=RiCWAi*6Vp zd&3I@d~CU^?GcEaH&Yk&_=_mHc53CrVd7`q^}S<%m3?*P30jZ=Ildcxdq0X}5f28^ zlD(~*)<(kGm{K6xVsN& z&ht^IIh(Go)3fMPzqa9~b?}Jm3k$mgXab~*2o6(YL-#Q2Msz$u;|~~Waw48i?zNH` zo$&AC19>0rwbG&xKvm|nWO>dutrvm6aF$=^@HNO@AxN8-Ck@xk(V|Q;A_pxgX{Vgf zVtp-E2iC@D&J-LV%|m?;PtfjA_ihG8lzfJePBewR2aX#53r$r}_M$rliPoMNtd3wE;3cumuW()z3iWbCa?E%!PDDt^x zBo{46sXjj`t7v1^#uAD z^X#Ei;m0~js1Q~%&x+s=Y32%qgc7;#0+DeE@e=R+KKJwHfFU1HlBAh{nh5*%5a})q zT18_(87Rbe=+4SvPll)$GR-SV1X|X5|F30RGQ74Uavlrt5}#zIzXUpGCA-6Q+#&?M-{HgadO@*UXzJb!zhCrcTtUT^{1&8bx>8mwUS>G%`U=kBvET6NV19V z>%X2lKVU7ZQ2LLja9t-}`;fQ{RIq&IN9}*c)&4#=LQ`A=h-`2M8^&XdJ+KK&^HFcu zxg>bq$>=fK?=J}o!hLKGb?e{@Yjxc|S$s6ZE)*-8U=a5!1z6*qf7u~eGht13%U2_@ z)L&tX5id*C5(BKnVGJznb#qp5Ax`mV{=*I`pgmVQ*b@CLyODVB zJbBi~M{#D=Dx+Zcpr_*`_8wU4!BXlLcjhZuRQM560d?E~)37lP`GV)hSQa_V1E`~Q zUysc^sidl0@e)eSqz=7cRzavoB=*D~qCgRFLyuqj?$TD8$^#;Z%e&SQqB5%CS=QUS z=Weg#bZjjv`4sc|)-+*cy7QZQV-k*j=mjt%u||AaVAz8ARch$XlS{97(CB89Lcptz zS3t2974>;;u7!a|-pDvOHhxtHB^P`_zUIB`)r|!lc6%Bx0TYEk^}ws>NX5&_>Ff$T zaezV6*7vV7`?*!z@+jR{+nfvhHUUq!A_Bo&ILMYRTfhI!kGJ>rX;$RhP5XQWU!+aq zP@RA~(XStLY|Q*g2V39ZLr-C#Vs6-2ZTN@&e=X5^<~|zm$)Cwy5G6oF@>5g@b6VTl zbJRfdRb$+-GYWd^!91=j(_GeNdI8NatzHB%onk|)e|+kzg4+s@`)XV${@6B@#QYjU zZD3=LeQ3yeed?&Yj90Td6x?&6Lf`p;$kw|y%(kkg|Gv1kzQ{yGwP zicF1fB8oxi&NpTMm1!G4G;ShGaPsdfgag14?4P2)bs}Di4!XXt{j!T;UCgYJGOVcM z5WOzIy?y*Hq+%tzgN>_VfF`!d)d6-VD|A`*{yM{C^U2H#dtB$9+(`mJz<>?EUhY~7aPGb&8&$tOm5U;u} zG%^s7fjs|D@&&5qIA=1>*kt81?L^ND-_)^r=Uy)UR3*+Dj*Fc)1zKgi9-8G7D-jZp ztGy%^bhb~q@SNIiXlEPbp?S~zKe9oL%v9jFYmO0VQWa3XUa{55Naw{>3u6E+^-?2& z71PVyAI_kn}XFZ+C&SG5W323e=3(|#I2==hw zEDOe)pKu;#MKqB2%Dxn1OVBv6%Tww|qpF3`lpBs?L6$38e3H|f1ib2HT0u$d*yGu= z_;ULoXL)TEYmhr;vXK$Pdq{!NK!kQ z=6l`mE|=#OL)#6)b_z1z3d#Z3Gvo}^5HApIi#dQ^CvP#ioy=DL(Yu`<5=!QK4iM~d z{C4a1tt#m7c|@@j1zdbMYnv9UHT7GXuSB)H>trQtnRAwi*B%H-FUHToHUP$*-L(Z+H&fO5^9i_9PRyQGSrh)H<(NNkWEh#}{6iEya(; zyb0l?n%OVQA#JViJ1x5eGm9!0znpG5SAo?~M_1EAb3sF<|1SNj-Ok*zZ}CQq8@NJ% zi|AG;JR?N^YkHrFH_rZ+w>Cmn(qDyTx zLNzIZwf?`WgzbGevi$gfOWb5vDnVl1Sk~i6hI{_7=29v%s;AIJCr;oUC27}%bB~gY zdHLX?kmO0Q-PF>kazHaH;^ltWx=^9k^jS!HX<+hg7{t!IiTa56HTp) zdZ`{*I#$6&+&Hz)9$%E__@$Vr18_^dWMVYj3T5&EE|KV_O6%WM)Og!OO#+7Kr|o9t zLQ@~5d2F}r@wHM}8Uz=#iT6vpN&th= z%-5Kt`atpfuSFD}IW_~B?i)H00Isd*);-*_X`Qbk?z?l>@sH>M<;LgKr{>*}ZEt-8 zf>+R))Q(K%#P|^;LZjd|-|!fN_`sjI-d*0FZ&It9t`~y1O2CEC(J$JQmAH0;Qy=%lI4bi z(8z5Vk!hRuR^WlZdhmAqL=%=mSttH-oh8l)T|EHiN)WD7UhvB|o0m9E_6&wB^b+jP z9WMYPOOZ}Dm^5@cTxt6H~|H`|_nLh8rphkLd ziQ%GYs1XUtlBCNl(K|HWOe+)!4bb!8p8mUh;$1lOa)%@5MmFoR&B{}sJ=E6^mMyw$ zdiZ9rJ)?fmn@9HLj^^-=AgH8&S^zoNTDHC^mPy(P2BFQF+G+A`W;kB09+{r z$)p>6!(6Fklh;yUT)0W}YPFM&s7c>YUCVq$Adm@C84;+HrZcT0!r~QFpM=gcsj0!h zpw)75pA`A7MEIarE-16XfozK$J}?Q2`Ygu;Rl;FaCy(RitsB>=In$9%Uw1V34ID~| zERX(@jPZe+>eFj`?C44qm8TS^N)9@Z^u5$O6u$>riyH)Ko!4%N8*nF-;xF0#_5~0P z#JE4Yi&^|Qy}u;G zm@=WmM_*Z8vgZ}D1 zhY9_cRb?+&ti>8$?vYJhbe270s@DELB2irp3c6hEH1Rm@VN*?^au;;T)c5wBZE*1Y zPW25s2%;fP@+6rziAt6g0F*npb1Pi)My0qyT-pB{YR?lD1YKFJLQX0XEnRZO*#)=v z_8Xnia}oGP#5rM^*%MT#CeXe-OsNFmycW>W6UgO;uxCz<8x{0&dO$0VB{L`Lm1#a!a;Ren{J%g z2^#EI5QAogHHM@Y#Lo?d=6;C4%zy)bIt|r>rr)~u)F-fQK6h31jjDbRPycD2ohf9- z;ci3Oxt`UV48?%c6J0x?QtqsDMGBj3`kmAt5o-LJD5@Rv`JFKgo6kWbaPqFuu5O@>m3}WU7ay*7?9zk+-{2Jy{R!ZYN~xE?|hX zZ!Vl;`9&dHhDpePr(<2$l-RMa*b%CMK0LmOOa6TMoOY)T(6xuomt&r3afNF0o~YIL zx$ezd+TIbm)@7bqdB@#H!*u4WrD1O@;tkWyTBHyUDcYa;wF9@YZjc|C02AywUr3<$^^9~xlu5stK~H}rvOomR0KHZ_nYN>*jDKhjbJn%YzHG*Mt4ORzK_iY< zO#-rpWk2WmUw0zsv(3ZKFQF)pxcYV|5i};TjZ+wEaku=gD>;#pT8x++AI3!AeaX8@~Lln z$6l@^8vdUr&h)(qemA9NcR3wz+q-2!?JIIJ#raRcZ*(?db$wp6x1AAv0DOOx(c>oV)exnwVXBYU&A)up?UN8)rMAd`YmqAkg2M5~DI0 zi$`J#PLInZbpSI!%)e59u6FJup>t^Pkya~G#i=M#$_RZSdD?Mu>z@0zE+6b=NG6rE zR;H~9G9uso*qfH=9y;GvGQyqR1@rFqR|!p;jHYoay0(I=$dHZEoIih55IOBIyJ&3{ zLQ7`51MpmOY33egHigs{aRg_xy3Z?r&H5)jI<*KGOtO%u)+(oUU>dHN`uPwx&QvOU zQDeF4r^X}&M2@wFu7RT$7o$9+Xh3KKy=R*ev>gX0)B7ILaF!!R7OUhH+zyoD>em^~mmqtAcdECsHCzQ6~YlL9$Wgai@NH_Xa#$VJJgM-OkcYWvkjoxe)tI%4qolwWVSVpTr_a3Wo(GA1(Liv7Cebt-Y{}WTp$c5J@Akx&P; z*JO8Z`(9XA+Bwx{X^z&-ikM=>$C4fdYobBrr#!@TdW_LNe!lxUw7;AT)>H{)uoQ39 z7I{WXmw9kK-5q`J|GnT!jri6t-I=#jH(6F}kkP$*`J`;y-_=^g5x@mk2Fl&vxr9b9 z?z07fNro5KpVBv4W2otj2$%*e#J}FePD%pm;WW787+(E0DLEJ>asx{!PfTq`<7GP~ z>2DC6bd^nLcrgD>m0M7ZXCL)5h?U8_PU;kpy1M~ z^{wE!$=@4dt2d%mK=v3PfL?(FwX9!P`%Xd?E622CQ1L+ODEAN#tIY?o5 z0`DfBIH>Q6ze9p*?&r(rJZPUCmE)WouoW{UV(WZ>aVZ0E%@WP%3$s~CL0-zF-;{D0 z44w34Gaw4{SIny4{0b00K=|&3_Yp5*e8jYQmTdNn5`d*WbKvX}%Eii~<6YXkVzpvO z8DBu-Sv|H?pksVh&hJlL7kr3m4TH!)6aiOEfMi#}=;a_E<5vq^0~a%V&e5JJFLy{& z!SqtP!>;0G$#H*Q66Vr8qsPt02fa#dRXe6OsW9eo+}=btgghxE)-53n(qJ^sH1znT zT1Bz~v=wd(WS&0DiZj%@Sy|fB|biH0WTsuC!QGiMC>e!LJB{E>@Va?MPV0uL@^jaxLX|PWe-p_;%xa2e4hY zbl>q%=sXVFz*GhEtJA9uA(h4xekKXYCM_?uI8yhSMiZOPnWiMMlC=zQY82CO>Sumgpz5{!C6--fMKa)n9(6wPQ!+92uw)wQcMrl|IrBEo&l?#%LmtoT8n45 zCIIJ^BPw2zU?2i77oeTbZguiTWEf9CF6WqeE5fi~2&DQ}`g`Y7Vt_bx`yZJDm0pNDC>g=r89D^PG_AK$=Xk39U?(ZD2bqGyX z&{aNaInInu??f-@t{xUdr<+%AKInE3_<$rxNh)7WT|bfRn(t7~f2r5OaiYIOdd^C0 z;s*9zKI_ z(~^7C60R$NDgyb@w?$?b<+*zZV$VHt-pp%XqH*B&FvO1>h5Pc;&0$C%L=)i!3}uaF zHZ#Zm+DQ5qef5_O@ipxv33dS0DlH3XRC|Sf$*FQ!o|0ZW^1G(nLqFHkM?wu7$HT99wpNUQXXZL zL(T}*rWb;%5%N@rQP+4Wd2wrAp*VPhWc4&6kE;7_viP?zzdyL3^25*wEq?=c$!}Rc8S~QV%0V>t%mu102BI4duF*Lf>UNfFS~3QlY{~ z^n`}8aiR13hlm(3^zXgL6xgdEEBmlKE}S0jyddeUOGUf?f@o;v_i2>EN4mJQkg)Gs zi`pBgBbZ=_(ye%QG6TN69fNOY?zji8%7GAjZvrKagSx0l$XT%G<7cYpi#U3Bqb2XS z2_Dq$7@^GqQ)d3RRWkG`g@o7cU%z^AjDH$_F|oH?GLe8;i0dP8^3djY@yHC#0*dDUNKn7KcS=;_F~??)pykMOkTsO~sa$6cf(Ks^=L)b8SKUSWKB>*jg3#<% zAJJ5JfaMy)$eloLV`i+W1nlKa)u{!w{QMHscF>|XNn3JIIvFe=l(_>6pDIJr6-+ta zwYblKhiZ@imc!U~0vtq0)1%j=J_v;re@6Aph0iApn;szWJ_)zFM#1Yw7 z*`(zZ1Ihqg~v@;ST+hHW?N-QI8NGo>lKKQqo8I5)YDu zYzdFBO*?5K4}F9B%*5s`Cd7ujLa7C4TypHe$>ob~q2|DK_;0m31=;%2tN3`6&;yQo zX**tl0C#3RO^lrRe|MN%Ob*G2_y{`QvJ=s}xsCLr8JQ?49fVj|Tc0IMQ()E@zqa)k z93E09Pm-_a?m@FbK+;vWeeX_Ve_R0OX8|vK-AJTj3CLtsITL-jHbPt z-^cFYlriN_7-5P>`*E_+)?*3ID2G+sh*#w+iAX^eX@j;w9H{lWRYoM3$d&A*xksW( zgLVnt9mccybp5N!@QruKCXxm*A^f$HXLt?Cdb-L0BBhA7+$cRbX3q zt|ma*4u@!s0v%XCKG$>qs@HWvQ_Rx7B>iueT0O!rT{Qluo(R%OjZL|qt-`_7IsXwDq-Il|I z*9{&tYi01mtImM%BbL&UEz2X*cgp2Nbwgt87Wa!v|8hzyn{!_M zLuHqoXuqcw_JP#x;-g=jN485GD5G)#34m;M8-S!Ym{_XP_tKno-Pkxo0&5)oh#yH! z^|@kIOijQey0i6WZl}td`Ydq8Y}2?GzQ7P%^9-qzXFHzK9;3_lfsD3R%tCUu8;X+q zV<_iuz7mtX{?-#Dr^LblrU{<*c@mAZw+g`eHJvM$%{A z7y2Yt0i7Ni;_C~5g5b!PZ>6e8DH z5G37v5p8xPZn?i8O^oS_UzUK`yIYYh*5OR{y9SXlTmS_^cas0Vcz=XWOIdS{NDN); z_>Cpeyx3>>uqF9YfBo9n_+eB1oBw^VtSH%jQxU3B-NLenAbrcp^(+XpiIUi6CfpeZ zGtI#9sY6}l2@xyvy9WaX*ry6b?viBPTj{kKia4O>3+u7a+t?T@n>AEPXC-+rm32vc zRt+z558>qm15bYddF63eQ1I@{#h5WNF3HvVz`(%WYvOaq#Xn%55RKrs+W&aJ!vDbw zuceVmQyMms0Kwy=4Z}3q`Q@$qYlpJgP@Fbnlzy-yd@2;VUkgnBN`k%Hv}3YWAu{#e znh4?jx{&%d!~epiSs(I9g+!qTb|qS_NH&{jl!HR_7P`dPg@lpUn1iwl7JXo0Y+vS& zv^OwYyvF35ccVRyEn&lSettboq)4Fy$_A^RrK3rn2mPD2jV8p3Cn&3SndK-6W*f<7ju$x2OH5=> ztq8bFpl8{Q(ez3F-s+(w36klj>T4)yFdho3+xk#M_UpO|%p=-}p4V&td>H`N3x0GGKDN2uAA*_z9&>b(8D!ijZKxj#wFYg`dX;DBj;+gxjxH8g1CHo}n1n>S2 z(-&wLb?ACukgSMO{u=l|N`^%%Od(i|v60XjmvQKu(EXHMJzpy!NAL6jw8eWk(|_3o zoGE46JH7LAsNfu_Ovin;YM=c!TVMthy%>&&3#}YN6Q7pzJ?!&lv1y6?s*{fN0Au98a}F`bJhQeIuo z{~wR3B3e?0`#9OepPj`b7kEi}^wJ%|ZXt+dcu99{N$qkNFjBs0?aZag$ukdjvMgzU zCZ(HCX8eYf!$C?C&feJ{3;hG=KB&*JhwrrFnVjwf*dF`iS1rHk-e~Cf1hU0Lz0;pu zzhZG+XM6%(*tg$Sw&EmlAhw7)|r$4YvB81}!8lEV1b`>a%9eua;$ zsd5pSwoFLOxd(#5>WJiW#@?ckfkqaA%J0uZeZGidc>mWT} z)x#P6C;C3SA5!i*LwlD$s@_%{4tehJGa{OEpYHOl%!?dYm(@}^`2bm1pywAgh=H|Y zHc209!15BRXTN=kqNGG|>yubp^BYGW>i0;YHW@qz{>bZR;VX{okkgPPv&qwE%LwAS ziNXIn>Cn-uvtn!CV#b_{E^vu|e099%4W>teewGnLBzqL4+X)hAagYL(o(J?%X#b$z z59m+y`zNQYE?sQ`Y8VQY@E)TtNmX`jLlh%5Y;SLv9GpP&)uLBfBO&2zcEcy|D)WaU zp3kQ38LAi5K*~x8=-@@rFig_Pw=fg-+@X3Fehb+0gdR89CS#oaz5mcm$i>QsEXK;a9x2w(2~)b!GeDkUaZQY-Z|^W8Ay#CJfEDb%XkX?FKK4)({EACsHE>g?B*^S zndUnEfS2bVJZeW$U+?#U$tH__-#<<4Q=OwCN9}}JS|aL8oN)|>%wMWG+ICI_pr_yU zKPz7(^s|nKwpuC)+Q`@CARA zv(8&3tIQqhegPH3pMy3$7eISc#t>oK|KSkl8p-n;To64eN&F&nC2Qug75wJF?#Q!wQdni!y;(Pv9tXEDFOF&HtD;BX=NZ8LVkI;crM2%2rz)5x&Yv5);8BdGVaa$V zEx2HA;iGJ}dRXfsWKYao7)1;`e)~%{_v-~4tT2hz)@BbnVh*_-QAe+Xu|WyjqS4gE zpt9oh#m-(>l@8U$sl{<}I!HP&FzU9-4KLx8wEVr;{M=f%0Ip=zFD(2NlLAJ~m!{pv zJF9B*hSpgB<9zdXyvXx+s$*&7E4WkZKn~Jwh^+8!l$RjfkM>$SVD0oQ85=%Dft(#t zedzA!zprxMnSqjbTz+Z~eC%Oq%aukl@3Jb{QWd8=y}{Nd##Uk{Qpb%g^kbq@0OMb^sbmPx-g%q1%w7i58Q< z+WEii0M|th5#lq`m#}K!B1-0Zt{b%O7>*K%ckAP9mIft@gE#E z>}JAQ$8is{t!k(7i8;C)8&Mi^HN9noewDW(L1n=;KY}@EsPm4T!wA`(Esy#r+|Lun&07k;L)e&T25)V6{fod z+|B4_JfGRfHJ8^GsYBDnq>UA}mlb<^gISuM@L`zw7ZCxy>NPJ~yh6cE!Wzy{Ub1?T zx0EVH#)B3@rtUs}!Xn6#{{nY7m2n1E@KVVK3~)O}RWuZcOjm0C3P-4MUYG}m(E{N# z_Yn-4&kEZ3c__bO>c+bC8sOlwcRFSv_YDmH39>X14!o28_@c%a-M|}N2Dv~*wy}q0 zVQ+nGe@%Dv@b3ntmvCOayC>nZ*a9Y*aU`&sEsrdcE_z~eMx0;A1aP~Fl0K~M#Omz6 zNm!JdwM@Yog3e6KQ6PpJ1rvv!WNjP)k^7@yRsN+ILe&EvMBg%j2zAr~WZfIjc_KD0 zjwT|?^f$5Vf3C%DD8@QbXO~;vS`u-0-`_kkdi}#+@Mr4K#GZ) ze?aZuE0n}<1@bi>1CPF&N{SJ(<{oa|wPDO>4Cwro#pBI8m6BzU?JtHFm3yHb7k3MB z5Y7fUqrjq-UBgo{*d5?qy@NU&>nu)BXu1+!`_>n$ZFc8LUzn#c0w?caCs`WNHDJRE zk&3})3|w#7ejnf_bLan7HQ6ndO}1Ivs@4t5cYZ1E*8Bfq2MT<5mA(#FwSjyqXMN5oyrxwC!~3Gjxf6M+Dqb9VBwQ3 z2bjIXO+<&0d%vw%>ur_5cRSrAzkZ4O?;8jE8`3Xk%se7>@X{u_QgVv7%M}vdlpMwl z(EY`;AMAvEPC*;h4?3^S>T^VtiCvI>!pQrKtQsy z!?J)ZLfcv(@1@e7&s<#e$l0cJ#t@{A9@NOD(B)Z$x#}w6i*&RQ94Byh`vFAaJhK4a zl&_V zjElv0$b3~VXac>H|1fNH0HYkVKzdQcqQ8gAZ}|MM^=1=6h>?^2P}c5p-owQ58fC70 z6uvP4jdF?Be^1Tx_Cbr5Kiqy+ude0pAl)J~&Q`b%F z-CA8oA3V-^QC)u4by^_uM9bG~kg1#4)F3LA8MWYopAuo199Wop{k$Lp`d8+98$_x5 zB>}msoTMmvWaAA@8?$cgOo_P*s{U(_7*xpIy{W0x3^1sHv@pr32dNH8kTCRf(#Wq8 z0!HH&dSFbGXY@5>)?{{$O_sm(k39`g_C&`PVhe(A6*xDf|Bc8KixcqUcl66STsgEP zx~~qnvJH1@|j63yj_U6hkM+?9|nm=8mh9y zabsNwpoZWUK*_3<%5RG(z3?V9=VpCTt=i4GyTu)S{`5~lgpf@)$dgeC$D}E{4(%m$ z+_1R+^V`{?gr38llK|VRLmyhd&mSu`<1yf_5ZZa&`mmpNKbKMSSZ7zV)JH^ZaUM)Y z3(=ilN&a((Ef%QQGrr1G?WyayH){oSV24<(ZRhm#)@mvggpntPbZhVQeu zT@W3&ZajE7?8rvIn>jp-#`#Wm+XA-XqSm{NSuva8DQ(s zTIScpa0Lru-J>Ok^9PsDaB*eEAqQbq^pbQ>*NGjXxT}WCQpH7(WfWhlVb{@v%6Fg-B?O&k=>muikxYXCa5PzC2$2QOS zi8gGp`07zmTL>=}6ps)Vt^~I6rJXBIB!M}IpN1Aw$z~M(cu417x8SIU5`rO`r$DEk zt-4L9(>Iwk^pZR@xn_wzzu{kjbvNG;7iKclw$uCuL#)0qeTiew!{f+#)?T)GjK>2= z7~5r2L&4ZEt?5#8$-@s^tl6Kv~Nv|yY>U_ULV@$3L?WY01n#qhd%82+s- z(H~Jwu9(mZ2~vvz_=|oO^Qg4*LgbS>oYU9~e3+}6w71TT7TF3Gf!bA(YWCk-1 z47x>UewXYF9|$d7^jloiaXQN zv=4rUWjh*oid6e|`TflrsWf_N=8uwf_V4f8sW=ep564ayyP&FnN_%x55n zR{Ajoguzm_#2Tb1Ic-(Vu;}{P1zHezAKW&8K?n1<8CtNh6s#-GS~|=W2L%{t#LXNc)-nv;!@ECY zZpAM>);k;E)ArN|8j>w)HfPgb^&-%#NL{$)s7Qd(jh;2<=<6wYupRx&YMKjeEFaV) zR13(9I%~vWURP-XkF{qxTaf{g6Nl4u<#?|3X{)ixHId_{ZlHVV>QX^8`UKZvft^6j zoIdK8oTr|~`yG0@bY3u^SlUX52Mp7<=d5YW2Gdwo!nnvz{9J4XqkGH$G~qdQ$87rk z5zeCfP-Som40D&aHDI#1=JbK}@Vyo`DJeypfA^RAHgQv$d2fAj#qy@gAcJXGBNTx- zhq$x6>T^fY3@pd~)7$81TKue-Qa46W*&>sDzLP_&1iR4MRxrJ?Ew9*CvVzA-xyc7D zjuFH^#Poetzi_DNCqj0#B6VdN+^;|dTlvx0WJ7_@a0QlL(4{z&MBh^}o+x{cU5A0ReW8j)gg}-=C7sutM#XLen2840CH#`}?YMKWm%i{M3XwMLJB-wl?Ukds7%Pci3 zvF<>8$B7}N0nRwE7@iTNL=GsDI)5r4h;d`c-FCZp@?Wr-^XDLvXn!}))+Q5u=D zfz_^su%MBdQWVXt)zhQl)FPT{_CfxDTI{N~G7(}=HhFDIiy z8c_PH0k6>6@TCtUZ$xWYq0~8wHzX_tGw_9S3-5U>2cF7ea*e01kge;v0t?UTSk6Gg z2a`ijdM9@2=*P_THfP*7(hXxd#8(Egl|Ieg0YH5Sy5gcycxvRv;&1_)(D#Fbm0-7P z3f%H;WO#hAJWV#Bc@r=kPiKlmJ%`1~sFGokm5wSg+%WQRUlqZbKrSgIjr<3&=%Hwq1&L@_XdV4l z^?azL!9nc8EEz-{B;El1LAByLajASKPGsx&(*2YMPRSAfLJ-ZzRZJgv2CMl zY*>h0TQ5|WIc_5eU45C>v%YS1#t=ThCH#?KbjtjtH3-y}uhp(PvlVV5)BrlBZFY_| zjtBG>$a9m6$-vB#;T@Y^Z9KGMkI>f!_}5NJ$ZWClNKJ!lv8$_fvx3`cr`?F>VQ!I( z$e#)cw7SY#?RP+c;huG|GZj-$c8}jc{!E9u9JGt1$dD97&8}(W0tr1R$ls-lD`v5u z1OC-Dd zXJ&`1m#}l`RWc~l$R&qfJl7Na9b{GqFG{%bC)D_UqgDr7!||G;q|q?N=;VFBRR2x@ zPhiHi=QzU{H3n3U?Q@UPzUe~t2ajtW_Y4zH%h@TZ<=YR$q1P;~>@N` zopS>)5yc?$@cZgqVXqW4_54R&YRh-}aMftuY2@Pq)YTr3z*_~~=J9bWT1P}MZjsp$ zik~Uz(%pDSyOz)MhMk_Bm3;S{a;~w<@VJe?p>I8(;2?Sy4AXO5=iFcs#zv$S@LyA@ zos}7%f*E4l$G_@5o-L@27_THdl>HBi#;*s)`$QpkXh7p5m1t+$R%vc#&q2pkRw#;O z4>_0^`@Uz3hyFU9MyO^&j}YQv|j^{?8*3y9>p|x?|Y>`7TQvxd~Xl+ z?s;O{;S0o&Jsavv&rJH9#}ztX6GT}}Kv+7sHAw%+cAq|55cO6ho?^xuL~W%YaLxj0 za_n=#F0}8-!;cDNTE$+R1F9gz5lP;TgEvAvkHI*pMF-sueX1PYvdoJz($e)cc&*as zWC~-Z?j0!FmzDOAq6nt=jmKi3Zs10MhwS#B$xBjyFUM^*raf&h>6PPqVN&1MeyZrg zI;+4X<^fiRa&HpN%VbZ;G98`CAxFp$qd5x_al?{}501XvZIVS1MSO0Pbt`Xu&Z{?U z0W6#|HA+=@?$lVM@}|AD+VLL^cFs%ez+@KwU)e=FL-BGDQ1aSuxzd`Zfl#N>Sz3wE zI5}Jo%2I~#&Lea7(-N<7!r|%Fza+%n;#-6StA-SnT5bKzRglqnyve{Ea+CJTUwr54 zEohY`ByKynLUVc_&bXRU|6=KVBc2YR)si>SyoBH1t9-Dt*`cyyogu=BTyCCZ)LaVA zdIFihq(wRgVDD-x_@5`*qJeS&lqhB4`f@Hh)>RM{M*9py-$Mg~%(MwEAF+9c0iQ3W z36-@)X2}eiqZf>#*nreI#wj7EpGhWt_Xx}<|GH=cZwwz}>W`b9-F^w8cHJ&!2{zvp zRqsoQM0W3Gv!ztZRIyWPm+mMj+nv)+6;o;{5PTm;Ja&wFO&9t&=ygyVnObOAPlGwe zFH{}>@Z%t)q0-sFl6o^K6)@vnXK4R!#>BK_Hbs1cC5N&dXggZpc(pf6S@w+*eVoCf zgEYxgO*cmCzI-u}P7|;uj8kA7$vbyc$)<2appnJtG|=pr8g#6Yf;uxU(@welG+2$k z7NvCHUtnSjBP2|1fq`%oy5-__d2rvEfhA(&PuKfpFEnwRNwT%OwtMVPce|RD)q6M} z>G9o`z`51X3hpu8ApA6A;a1Qb2N~mbl2y)(A~rp@(r2a4QC}8Xsrx6#dbb*Cj^|}s z`!N0j-v0?BD`zW=&?VBREpT1JL?x>j@*Z61=tq4;p-EEw3COwR!_r97`1P&4uPd3p z0}*=~V;n~RBODUwXcamsv=%|xtFkAw>+I0n5TQx+1Z5YlV&5NMXGcXNwe%gdycKq` z;=n2UZ^4P%q0+*q3Oi!cXtbS-oKcNGFXO`RS<|KVhKXx%msgZRcP`FyP1R$yksrD)s|1nLM23L23}e@`IUr@6+n0%Tpa9)4I5MHXpsajY;3( zUkv7v@o?+Q1h!5DxX4K*ceN=AJx=j2F(we>z}uAq3Ht#Z?UJmR_}sS&fgzp8BX?m) z63x?Nz?@tS`CulOnlR6Aa*(W5TmChMov8jdG&Xj~MOa7WV{!?Pg?7s?l$x|p{T<7J zQbp=DL`%QXgUCm(^@$MA(E6V8k@O)WD|j_)Q-qgmu-Rx6rq23=i6T9>!?(EmMqoE6 zSW5u0JW)c|Xb4+kIv!=n%>)weEW5@}i4q$EKP4S-O?7n#YYR5AND%f5U~O^iZhylF z60px*uj`ck`NVgXi9>!|)*_^(f}m&x_UJe2S_VP8ubPup@Cc4ekk-ws!b#wyvX}P8 zH(UKc#l`jpaP7PQEM)NJe;+|Hxb64x;i0Sd`|t9oa7_@AYz{vO4O`!`g&m}2S>{z< z-tn8XoAe|vOAUXP0(@iNjwN=i6dB!#^KC-`g2FB% z7{JvPAUCW4JQortP@)Rv|AUo&WQO=4POCx z2}eio7)OkS7>%-4=vlRSYot?Mr9?biqA26OEKEq?k|1+T2g@Au3SJYe{aj#cK%Q#z zL(69k<5rRs5nf8FPPK8hX=Xv}bh*&$qk(4Yl0|ErfQT>R-`yoW=VfE6EvB*Pk1K=w z?WT8S`T^&x=lH#^=;Dq#xW+ASeRKpG)0EIay>^UhIL1|D>mY7~VzHCFHaanXc%775b3;UDko6cgno}k>7bJ2y^pHg*+X3lq@U{q%*n#Mbm!x$c2K(_$@45Hkf zIed_-C&ajDUXrk~9}JYh80wBj6)5+pkP`(37z4Y@{DN?3wo}j}n`+%mDKxYcbo*&V z=E77s1N0*uDHJCydC_KUTP%xiWaS6RR<42 zpwz)E@RV2Cd~xJvHdwswkN6< z+;a!yCO&nFT>;KjxWkjmy%>twf0%jLow0WU06oY?@RtZ@=lvW`w6sD%xX|Mt@ElY_ zlx{kNO3u>-StB@LtsOdN>mw;$irS`f#?La4QqA!tu9JApJ8dsZ`KT75s{c++g?$y^jDwW)dnAm*+e< zsaETs&D^AkrD(LOuNu6TMeSHSkCS$cPk*AZ9}_?G2*VXbVM#JJ<<*T%5P3GD@R)E9cK1ey^5Ux%;#l6qcD=c+Ot}Mr-BAQC#8c*ctva zT}G1MsPcC3^)EeSO?>B_gj|HIfVfzIIn%Sq>k0Yn=URpiQ$ zkSzXN=SY~}RhIah3{baDH&AFrU_I1?@m~F8=u;gSye=9EJq!hP+-rl+5hASwcqOGh z6x~D()OpN>b{`m+MsBYzI~1r}dBrg)nx8V%vWuv=dlgQU4@8rm&7XTTVW9&Mu~DTf z`~&lIYwP^zBhJ@btwmNs)Q9B?LLoV6Tc+W*>#7O%3r5c3I3*W(qvE0LUMC}pch`c1 z{=m_6hPe{8VqN|`S4+l z@fI47GdPy3FGYBo%nm^f+o(i(>^n8}MInMYOHi6*4S!d+qdWIIEVNO+lRlpKv$&+O z5Q!R~5Z~8Ctl6X@6r!%e-l%rLnQv}Qd>ldo2~Xvia@aqb24a7^Z}}vW3iai@7Z>$S zsr=>pi(32{jHSJ21}>L2JTpLxso(ytcG*nvMw>sl#M=LMLFTtikgh@zYcBbsRoM2x zheY^J26X$!gV9#QsKJrF=!8aq5zV8gRhFip5;1N7`1J98#>H|(bQnwxLr)+G@cy1a zDS6z~Sf>u&SK82Ipl0cpDRkZ>f;4$e=1dveN;b5su|S|nVt*fPZoG+dj94$>%<00t~Axl&MQ&YkN?b-9LnQ z>vF@)ADLgQ> z+fPkZO|YVwyIG- z4l5Ga9yR{005Rwhyy@)xpUwej)m+&V3|;d#uA&)#8eWC!7q?!{BHt~LGV)pkZQ2s; z9b~7c$<#B)Fc;?}@6Iz$=?jdf~FfemHm{0&=6tPlsE)jra9VZR#vOmvq<9ILSTimur)sQ%qA( zN)4$S>EHw(5r4Y(iJ${aC*W(5vZ9@L_yE3A>l8D?A)DH=xh8N=BLbkpk=}htIX?ct zBJGo_cIi^{s`gng_zYvKg#Py*QirR|hi1e;h@sau|MzUKZRSdLWd|;KJz&%6i(oKH z#c2EkxCQ{>ov;xML|Cp5)Nw*$(9)H1 zFi0ZLi=#oI;?1$eNEz2XdVWbPA?1aesmtDUsqH+XaH3mIvFGSRrC|B{;!J|5<_d6; zAXI`SA4>Z8w|0oCH%og1!na*>TyfTQGKkj6;qs|SqnkiAsYkaiVMidC&@&IuWskna zVP7tVum|C1X{I%+UC5gwA&dwKbh*nyY%CRq;~o*nES^^*8O{s7C5MvW#4hiz;KlC# z;>84zV$)wQXKALgp@I6c$55U*67*{t%N?H9DnZ)x;g*yvl~DT;*&&HdO@-FGVoc`1 zzJfue>qafL^a|>9yVgym;P7&kL_6GuF|AAL)ZvROb@GcTdv#nP8}m9AZFI~Zb!wxn zg~&&a3+17Qv)C?^Ez5u{5Yd)@zcc_xH~u2 zjStPtODB9(Pts2NQLwwEE^Oip7zFj>fNZfIq@jvu?Gzbwg#k8p%GhhJK+U;J;3`+1 za-=d$oZ?7}faNb-%a>8)G7AngKyTX`ekQjBOjGkJx@fw{NiI&w{s%LIpc(c!jx%?8 z8Aub&jL6E^4U!^0-#u)Ek*sH%6_5URV}e+>_|j&uUDtR-Tb^FRCYC80xVyxOw)-_q zbEf2<`>@3b1*dk7VZaO@_$(8-Hwsaz#b85y2`~)KnH-6(?3TF6!D452q7hb3|1Kt7 zXo_EGBOE1JsB`54x8`4;pv4H0h;lj~ks2Fhj4&S``~Vh)o*d`Mg-F`3u~=O@40=ze zPgvsFz7X>`CVh2rI?#wlWK`iUUqv2+cj*tt(h#C;y1Y~2AJj4;!WM-Ej`DUCCu)&E`1#a~L zP$DKzGKTcn<)Bh!!N5n3?@VTV6k1*G;P+lG)F2=~n zU~4&rbw|o z+Y$Dsy{SFGo1BO#G0B2qG>oQXlj7$&3|QK>D^#KvgL$akwv!;x_|$?W$CX$=q8pEn zvoT1<%{+pYqEphgITdaX-L+s&X<{`YPb7cL3pp4Hv9<^j_o0S(ff&LDiY{*e1n;7P z>tyI>X=Xyd&ODOJhC!1iy-i6%)gOpXKy%eGiqrt8&Q(-vpCFm(R$x~{zQV7|Q>`sO z_2JwY;_oeg3v-Se<55HR8D1a!U!$6zy$elM;COBO(O>b@Bx7-^`6gHT(D?w4_bIUV z)6fKMTK4-YS>-B-Z4W47glZB+0Z^woPy%gfVhEgo0{wx@v@~^)SbpX9mLKK5W@ET0RR;&ds-ix z6Z zztW6WD30bbz3MR;*~~#rq_W32v~vDvh&LHf0_O!6s=yy3YUP5D;F}*|!664M7BBWE znHxu1%Tiin{qMSTwAYWtY8Mowg~Hbp9P({M>!$!%M8R`mHP%BRj_J4&oN1#0u8Wg{ z0Whek3UX~ZQ$)gsLl=~)QvKL&6z;RsrepuPUXca+v}o}*8Bxuxep4RVb_qCQQu`s$ zAz=gfuTz%?Q#4 zAy?eBhoBw zhSCgdM*#m)yIjRuNTfK5n#Be7e|x1LoVqGYdle}UXToL+$VNVa(lTg?KtP3gS? z#iDP#a36KZIqElCG1*B|-WLJ|o^CF?-tG&l^6-9aAWP`fe{-gvIOIbAB~pe9N|DX? zt`M<9-M93Bl5A>zTN#Ck;{ZoMxW5a??y2bEY`<&MDDjIOfO=HVI%&5%)aA~CClvx$ zNYg+KV7-Mg`i*sJDm+KL|cnzh}-QpB|lnOu6;dq%~J>g(&KlRA;otk*ww0V>Gn88e%$tkClz( z5ILv`=Whc5x%xoHBL|&YJ{fPD36P&!qr@%cO@*$Dd|SfaG@wn2F%Tw@BrnsD1ts-A znR(?R;VLBc*{$eIai!>qmr=t!BMIuLMTE&u0-$V$@WX%PXFz~j>^dHtp!c%*aZKCW zy38~xYx?@%X2CfD|_M~Prv3jPf`3BMZZ&rqOSW=E*n#3@Ih;5 zdtg=XqND110~ae%&UL?KU|gZyRyF+juKLK*c84syxJM+Lfl8rwL-|4^5bRVv5nAGR zDU=;@2{lJ|WuDNn=F0hPjg|^TfuT>+yy@NdZ}rwEx3Ol4h2Y3`uaGFC;jPo$)sK=~ zK(J17;ZDHdnluW=0WVtf%Zf9Fecn@6IlgN==bq4Ce)h%AzMAtxFa<%q>MbaWBR3zj z86eoaLH2M-70*@~kzOme; z<|?aGu&E`=S?S?)1W5|`u!NYAP30x{*_7oN4D0;C&xm$`!C9OwKL(?QLdek{J;drUv33%|9QQUmhnxs4-R*+K*;;nFPo9xVXdt7ot9 zM;q@C1a+TeRiwOt4>@Qq!@h%l+vOqT7s*!GC=MG85-KotFhjf6iCimIn5_k7ZXiqD zae?@m9k8YZsfO@#X!S^<4k+5^rn}(3mXg+PZ!sScUhXz8B>YGoRpTgQ3WZ>MAupPd z4jc)Ac@SV7G3ylbqz6tAQS&qj3$V%e&4$15Rr{Smo8UN(_a<9;MAlt|DOUWRe42a5 z*mZj}VYQc%_v4f0*Wo?50h-xZXi2yaz&c~bNiCgBcD;_V4~x!S-kyodDsR-iwRhzk zU=Ne-K_+DI8iKsIO{`UKKvxf!Y;=Bp2Df-fo zaasrg&3~D^K#QVE{kb<)WY`F|;>v1iF!0&W9sot*BnD-d-Jdg5--WF6e1Hy5-OW{u zG*0b1=U~$}13JN@dB?44#jj!ZPq-f?=YF(>$aW3yk_K4-B|*j2k{oWHU3=}ln9iHG~pYOFA71_z7mCd#Ldg9O2SdVm##OTu($-_63 zg@~fQtXY)4A^IM_-?spgPq&G*DHeo&=`gE(?fVvLUH5#!`-qpB}J1 zg@`pLJmCeQ&Yo-N6q)_iP`z+z#UlbaHZUPrK2c(fsot^G86y>Y@h;HQa2}|maEs<- zeLq6k+1F&xe^nX&uQPL<+rl80h4xj~RWKqmKpL9IDSD4cm$w!M;T-TL3yiVSvoKe% z*$$oKDNXxb1uz&Jl3T{qbqu;`7#8H1pr`@o(RL$3;CB11}Yy9p*BIe3Nd1dj)5UMLg>zD5tmHf)!MNPDCv=Tnaj5 z2ek{f_f7IHwyWm_$%*CvFqO(GOO~+`V*C!FAhOUtiguqSoRYnZpe+uEnVOE@SOot! zOqoNUKLVR&b2a5R%gMDmxPDwOx&D&->b%@~lG)lYIfA>zr!hKmowd2GC?mvz%mvY^xB{^CWBzzl-Pb;hBW-Z}GyFtreoIV7 zf({F9_T~Y=6cxwiz~gc@FhNL!>C`(cc7D=SmdD`*nx@C}2W8m>(tKzK`F*8ijUHr` zwaJo?Yq5KuS{dcF_pgFKHy7sKE@;eqH0uR5Od@^<=H|G~V=>t0pqV3CY!& zG2(8BqV>Ku%WRE}%-vAsNQCxqysXcYoXxp)cis%e6|?PBEldvqLp;yG zxmUNH`2QS0lTmVP3!S$ne@uBS5>3w@CUyO9{V2+;CASkQd?~%px<~~?2zS26oO+MN zxb3f}kpag)I; zm;?Yn;jWahm7=YfV@Ottx8Ym0x>Z0 z6oon$C!ALejBuAty`CbQW zkq7vV?%Zq;QRMW(;O1_kPk?g6&v5LS-#!x_wykDSWWQBLcW5_1xWgzVC&S;x&f=R@ zf+(;pAwtU3LLpYApucbJGEcp0d(t)^6R=j!Ezt+~tVLNxj=r95piwQ{7Be4%_&!cq z4H+i+0JnY+f8*w5+L6*2Kg3ihlj$QoR0V7#B^J+V` zu<1QQIl?MW)cNmzYcuv-B6aB~QCk|-ZbUNKtO&uaCbl%c&C!+x6Fp9ag%oSZiBD)R zkWQInLhx1G56uLlq{U?JV!u<(Hn;ZsuyOX(p;UGDzH&tmh&1|Pr(4{#8DS{9@JfL- zwuH&VHPR}J_EG-(SLZw#wE!0G9;eMJzICBrJtw`Z6lua|y#x>IfT`J#qkbAhNWSVl+@r~%U?Z>UTc=Cy()1{{ouPIqmMI31m>aksAtW|>3j%YkS78m2EOL9IzEQphMGU?4L-z1?r!!coD zE*8HL0KZpjG^CIeW<*kITqHOAo;`24lw3`s@2DQwW8{UvohKvhD0h6=b-98z^G{|u zIz46LZHvhWfpGvBX-U+miN(UYsOYqvNeELjy-X4}p0LF#8mvx3 z4A$JsXq!pHd29+*S%wMmU0^SYsd%J4?ktsEfXHE)t0{sC`kgEqG}+D;kT4ts;gdch z$CMo*_AqNozEg}i{`oE<@uEy=_h%svK{8iCRn+|=kI1ASk*ZPn&mNvO8=x$LwkkzrcAqD?JarISv_DrVw5#j?un3^ z*vdE1ZpPM*B?mZEf9A!$f=09NGeT%K#Tb6nO0<-$k)@FfEeSb40DxQ&0CmyCHX%mt zV2zMHn0=buq)96g(vFP(;m}kl2CXsP{-?PR_pyq13McdG^ac|~ugeLs__9i7M`}sa zc$G^hGAZ0hgx$2O-(FoGM*wS*wx4Ko1+6FG0*eH{oyYD9`FiD<5oVruIeMuBB1p*Z zMmZB&z&B(`82~jX?EtboV4bD^>}}2(jRK(Tcuh) z3n%=N3V*({=}D!m=kwqomNQycp)&2LQ73_+1RUMArI>t!v>8@M_vI~GOSIg#?sT)+ z78k7KfKk$+%eF~_{X*Hr9%9RjYS1!g={+Ca2vRT}ZYMDOk3i5rJS${P^n60R8fF*d11 ze7a6YB_I;u5h<0>!uU9`r_$Jevrs>;k%2&2A|cAIL$2AZWn7jpUmc5r@4LW;lLsS2 z(p3S{nTXit??88Uo^6wzS>kH*b>~q8L0ayHXM7%vXbc-4Ib|i3U$IqC>y&=qiRaDY@@eyo>dW#u};JEb#W*pQ4;26o@dK3r_Jl6vqP>F{?9e7nY`UfJ znn&%!A2z=3IMxyacTtlMyXh#Urz76)h>L@a7Hi%b^wpM;TaUiTW;QvsvjYKSmI#fw zD6}HWUZI$2J}_9;j$3{5bX*B@s74;+U%7#onKo�lxno8;3jti{s8%-)ie#Z)*lJ zV-)|)1Ry-M%1*9Zhsd^^5NUQ*D0_9k1TJXl51yIEt^*>Vv{ev;N}c!?X4&xtPkJwp z4pvVOktC$e+02h2;7|M;d7@0xq5y``k1~4DABcr?TaT6gvgROm0zX3jGe-`DCM>&Y zDB9fyE^WGa&iupP`|TqknTo=07L)@8V!30g*uX)=V_|9`;4|);HWk}IHoX9aE%}Bf zT09Sg46cHSk>gO~6&g|K@p6|Er?=)gjP{$C8 z9I_|1G&8#7S0CSIwr94Ki^v%}9Jil`g*_Q2x>BuqB^_J7!_jX`G~vF=a^0X!+ob;* z9B%6Ym$gaT)+9GRXSHc?YifqGfaCd* zTaj~yC}}gkP|f?_BHo>$>--=fvs}0sq%nhZ{hO>cjxHz$qUK$K&9a`^E+8z!nrjU6 zFGd075Tt-Kd*OUZZh#tAd7~qWG{~PS)wnLdZhuq{UC%fOqt&pY4DgG#buc+>&KG z9OaJ0Co!&QggX#ocjm@iM|qj65(K;X*|`Ljl=9{1b{o{XR1rH-McFfcbD zZxlb(`wqaW$4P?qBD6FwcaJhWmnFA|Pi#2x)y7+yYFWnE1)3yVdYC6Bm@xzVFJAiR z{tP-k_Iwrnt|MSo^?=yCvcosu{mS*Yj)C%u}6otLG!t zCr;Kp8%xnVBV&dJ%5l?6sFwQGX?0_s-M_}C#%{U5U_^*Z{cmQnC|x;$#udUo$vHj1-!&j$;$-PV9s#rozGP_PB@&EdDMI#zJHHH6)d1OR`Qh zn*g_?b#}eg2(eEv36wJG?kdSZ7=iM6iR+^R+WPbif3$8iP@70T=t3M5C2hxQw8;B& z0Q|2y?)RgT8RrDQ#V^dUv}O`gx_ADi{n5@nG2@E9Bwod zd)YyM=IY^0Y-g{CQ_Pypyh2%=Ss!aP5zJh}9Q-e>*rbH;LKO{aKjVu~J^uOnEOcCk z&5`C1``csMB4j`3EXiHuDk&tv@JqW>@Q}FSX!DW$`DSKb6_?7}b!%Dwz zAeD9Q3CKb3s=8)Km*F1H;8ZHnxr_&USwB!-rZ%d?sf0q~AUPgFeT5Sd1*l@7nk0*e zolBamzR7S6!zq)#nNJ21{ebcMtsZD^cl`1!Vi(8V0{>%Lq(Hq9VfKE&hP|$zY2l zyEL#(p}+SWdE=Dgffdi22$30({e%XAW4ljQzOdeu<2REPOiG$>yQ~hG~H6$ga1bC7i!^p{fPr(0W*vZPUf)JZr(Q|raS=QX!z59o{1PZAzAR3=zn zBZLs6O#AS!*giDC*^aM!XoY9&9y-*al%3WLP$cT4`aGB|$)&T?N7h;ueN3f{j67$? zk^yVvn)Da`dhhD7K8Zd^6Oj%`wqzHdPVVtML%s>KI;P{FU)Ra|1*w>RNb#d7&425F zGqXrTYr_FZZmAx5=l>BF43P~04i0MNV_f;nuTc>Z5G6&@w+Ltwm3hE~(dHvg+=E8g z<vBJg-3o@^-o^CKf<*Z%_6 zrx{l$vQPO9Vtk6V0gC=$->KAAGOx5oNXWZ3&}%l|Cot|~EnOMrkfw?iBnY0C9s0I2 zS-w21LjC{_8A3SDOJSHz)}%u{y>sD_mU6n&ZISFlt748qoJVYg`=+86LtFDD z$69F*W%5m)r~_p@kgIhn7C1h1>2Xbl>B z7|L>6fRj^rwO5_TGP|dR7_g$MrCxth6Br82%qHvzh)Tz|w8UDOncuds3O1>??zh6b zi$rt%?to?;@0)gn_m;E;Ku_6!y%oeJiv>TAa_LeA|3DVB%wzPl@gAE&GZGD@uCT=E zVtVt_;ebN91^f23-D}q3XN}=`JUSH>zyGiNEH6lL6UQsd-jY@0#6+RrC1)k09pd&J zyB*`xp{S>x9-|f0-`h>ku75y^3XlGHIh?OmNW0`36AUJj%~GyB4O7CaaU4$AVbJ=l0nOLjeqb=$R2mHej?A1txrXg)yNp20X$ z1al)L(JgJvc=_1r3GEQYw_~R{EO)$HK%?pLPMR(hX1#dhrLK@uxJQcL&*bm&+|`R5 z%0I(~)ukkO_N0TBxFJ4UayAg;MWL&dClT)RBsP$dUvns5I!hm^s-N;Mfbe7G09vsz z@(>c7{{A0tqn5w(6coQ{#lILiKTDr9cdBT5TpO&m*oA-~b$5KsF!y&fa6~t}grcL- zcD`9NZTx8Mx`r5GuW_tO4_WW*rJ)TF>rp$yXH@UG>0It`)iVloeUDTC_4@Dc_GX9^ zHIyrjwes7^$Y8pc_o>s$>cFBu1P(w%&C}I80+ZN(9#clwN^5Ps!1)FaZHki;(rCBbyJL$DRI71|jgYe;m*Lx>* zhoZ1RGmKWX-;0sE@D_9BDNRFZO}|}E3#-B1Q3`B)Uzp~&Ds*9!lQW@lD}8wS<8M-v zK4S{+w-VknBEg^^i(htLgKoKWJ01=zjWo7`D?qU72Q_@$0X>G96a)K9JYRfaf>!uy zdrm*xadKh2a>ApPgwB9-Ao$8VgMF-pYY)rZUYazlRDqEj$}7~ggy?N8@5H-{f)#>Y z36yK#Pm z$6RtQ_isI8qB_YYI82hTd`bYm0#tU^`o%68ZjCeZ`e$8X0@RYiXG1M9K$2hH?#T2& z7L}|ryE=2~fLa6w#ZB|O4FK=;YOQvI4g!hLfruw6sz912xl5UUc*{;S?nTVeo$DUK z$PxxAKnqc7LP%E=!$(_^RlT^}b@{7dEm9|FX?aQJZ(t7EyG>@YU_O=J{a|J#Z(ijn z>S(UnJ*5&KL20d4c~-Izk4&GP;r;?a%s|4vJvoBe{9xR`B_saYt{T65-?O6A102Ob z8UZ;ngsPN_ykH)kr#N3Tphjr^^KbW5Pppff8Nbyy4h%O9aPNfIzBEXvIaWBlTo%Y| zl#!F>iPtaG!IE`QiPE}4Z7}`Qt;;&T6u%F=OsRn281UXPx7bvrb1*YXJh2 z-~Mu!E565%BD z^YYl~%z*><3t;O2EiPj9z?H@Ue@&?v)Hit(=Z<(+^!S+aKwtjx)YxbBWh@#?wU4zF zY3mzRQbwWWCow?pu~(uuV@Xg<#%|^Uf}hj_#N*Xi(xQLUB@NQLgVibg5`!ArC*A8S zh`ADSwT|9k5^E~M?}WoBx)@3uYf~Y7$m{3akGDJvMd)4sMX&oh41=c1Y%(ti)X1R{ zZw?ZYKf!wV!@T!ZES`(&etRP(d7Haw;e++ZS^wAu{Zb|zSEc(mmjQ6YJEd|2%^klY zIbw9Pp;EDsh#q<2yZ!hA+c?ldDba}xHur&{KLyY~Ig8;Tf(|&U$B*0l_gBm7k~0ae z>9w~^vYW$GZFBK7+NL^;(-~Eq^aK)@Pc|OZO_4~e5BulK8DRU5ZND)9~~a( z^)I~5jR^U3J>ps>M_3W=5KL~pwoiq?eh#Vnz~qBxlwHv5?{43ETb4%5wt&jc1a zO3P4`Js7|{jATQ&7=`)LaTw_=2MtF6z1|>-OaoXiFw<-w+wQ0;_eF~SuHC}&|C@6_ z5*p#p7h9Y`EZ<0a^Cz7`jR+~EkN$7jZ>*FV#+QqyP&eF?5MSxUBLMjRM6Rk*oS%w+ z6MHKb@42T4Bde@nV(YAJaCw z$6|f^lWWO$GSgb=Xxs9oMqqIT=1rZvAgrFa3lB`kftt1fQ%>Tb4knq!OeuGSyb|i zlySQ7rH|D3CmQ6E&`nPQ7&85sN-csBd z6A}eIV)0bk#<|Gp(YKuLN6$766*|6(NNK7dv0=~&k<#w5kKt*c5qM&E3@@etZD0IF zNEiM3^I&d~56LEb-B}*5QGiW2GmZXmYleE0)1G;L{WDX{Jbwq^di&TQ=#$F ztXj2|!j#+Z&O!SpWrLhg9&hN`tOBn%=)Xu&qnr>kGd#ilm(;MupkV8+^jM&}2V$_u z1kq|gjE(zT-WReryi@tT8LV}$r#f;wVA&CxPJX|Bqv^chx!QAChkqtCOvL0>Q~R_v zXUyD@fQlql^NveA2O3%7*+IY^FJl>d#fWxDjzsBN5DG!-ETvJqS68Tqjin6?QQXHA zJ5N8)bzTThESrjP`cg^82!GUl2{Qb&?IeCWD7s*+2(IdRr2NgMDsl!eFld+QDL^N; z%Vxb))ALb!RBxq^gpdHVBc>07tY0DUGm4#WcCw6pgY&{)+Uf^&51M%t>X$-i#LvHc zIJxVlEnZ?KYvtJj=%Z_se$s(~{jsT?42>cbb19y_iog%u9u$(d;@7TW|Nfu|F2s~H zUDTYLsc$RnMpZ3clEmjbc-QH)M}0`n#ZR15aM@j|A964}J#)ZR6r;=Eolj#{7=8u- z%Iw%*wSQD_iHprAC{XJCCtd1+ZUD@c*#Uo+&}IYDKx*T1>Mk&F`98MpCHY-;ecu*h z&-y1=UxY81r`GEAIULMo496BW3^1(b-a~9(>Dl70{z%i*Bg&s3{OndW3f?p3Xn_?_ zPOO({$2wINbwejS63oXU^Lh+kQLRqJs|3fYDDwZR!c=a#zlEOJ;i(fX zz`KzJuVlB+m6`_YidI^gMb)L@m&2G7q~iJnZLxb}m=1yA88v#W6+EzzGG+(~Ko+$Z z=TT*JnESFkS9&_?+En6~2=u~k$*V-RYv28qNWWE6bJp3ULIxE(iQM@F8lA!dXsvwhL^Dt7($wGYegOt?OLh zHcm$0=sUo|1i^Yyd>YQYT`_2~JtI2SD4|HBCp zdv|V-xfO@G_p5mJI&j3W^<(a{6ay@Zvvz0?CU2*(LS`rV_)BfX=#x)ziBq#Z;&NT< zbp@Et_F(dV3j3y05gcRdZ3%5pBq7|KI&Kkqr@A%N8kF7|ro$XG%JUF18MI19p!d5l z_8sBN?)PD9?+UTZz2zpVmH%_|ckU?QVFy-e0VO(S|75rb2$pxGY==^7iDDhA?C7Vx ztINGcz{yr(b-H=t&a%U(yAYxma`T2UcYxJ9cG2ZEs1g(Tc@H7Z1Z?63h|t5>#EI+% z0_{GMgce~dK7(r}*IUE|!hU%TQ*hZpDOvQ$R~u@Tx}E(8BdqeE&r>AUx(Zj5c_aope6xx~@a zgv(3d%@lxd1kZnOfTeA+fk}Q)MjmF|va(ZJ2qR3#`l#R^qP^m<7sA?b@H%49ttuNmH7q(rG#R%53z&Tvzf`eCt2N?c3do7;?-r8h_!_fX77uK`~BbmN#v zuOT$rXi$OcVVeNoXMoQdl#7ice$bmnfZOAb+hV0gGhJq#6?YFDVp`QU`iocEQTCyIpXnL4o>Y2ek`pO z9C(iDud-Ac+e+Y#)ekRA?VlqDC%tBET{wiUm88h2LU`ms1y!@4JGRr$J9a1%H_xoy zlIg`)ue!!20(J_+DZ$uMPQL=5P>6A@;&A$IVx^16*zbdYu@WG&j%B3>K%P;x=&(zZ z8Qe;m)6E+^2zg`F=2Ch*+TYBsVugwnVWC)%CN}=Q6SH<7qw9=~&RJHktXBVF(Gk}@ zVutOA50292xg2-WSwr3S=pA{888>-;^hTxuB6>F8)x^$qW7t0YA951Zvqo>ESuwa-t zFt<8vCBkApxXYf@(ymq3CDF2(<0(gWb<^$UQ?tQO!(pKeQe`Dt8@X$KhM!=9Z`R{K zf94fir>I_Gb=g&!eO3DZ46Zc*0}*OvMP-ozJwl_`olBlza-N;~akDvsYC;wXxMz-K zT_&WW8`xYHJf^6bpLJZn#xWriG*8vc0a6Q|^wjpql?)b{*fWFs?oa zk!ZfP>miw)|9YNd<8zwR_X}rk18s8L?u$J_ZN5`P?=KyC8Ss$L{8pa&j8r&r3EO#0&KE7fNR3|QZ69El)svK@i1O@vJPvYAWH`%a%Q{4OFQTkknHSo>Q?UR3`z z$V^N$9f{4J0I9E%f8tsA4Kj}n1ts5tM9l${&tL$)GISpQZT5aJw=bnjjXYri>)`IP zNb4ZKDER9TuROj$x!1GO9ZM-|T@}VQ{nDJwNP_s*?apRrT<3`JeS%eG^N^ zGi6r-RvW16W{VgfVn5KK%F5kI%yb4H?=8@`e5qjq!}%zycXtiEw61d2nz1^Atk6K&b>i9y zW&}^PLtPovFyX!RQx|W;jJOc-%byF1#L4qvNC`m6-hc=bbso>zBvbO7Pn7|xB?Y5c z=-vdX%>-Xlk0844tlA7-HsM~vAOLb}>DxMfPCF`Y-2NQnz;WW*D|uzcC}p4Xl|9pj5UqsUpTdF)xoDd?^T!*&ZXv%Y zCEK`y6j&a!w(1aiuTmid(_yelZn$9DS(#aHo)BET9sLQnUI?r(CtV>IWa_;V!9lUc zE^{@`emFVV{U*&|2q=26okn2TlPuNZ!_tsv&bg#j20{ZSN~2%!z)Hf%hIWNxKUZwv zSve73k^ZxCal5(PA_Y(QUo<G=wE1I#Bd!3C_-J#0>l$FN3^cD zTa*@=vs-Iv0ZbE=S$- zr3L`C5Y~X?Su1jiOT7NIY{4U#LLn(Ferl+MY+m$eTV|ft0al4V<@_;b#7iqI7Cgut z&*Md8M|~TpN#`wKVl`=lz=8y@5v5K0)+d}V1Opp) zqiRx23<+t|PwTTz`P0ELfwIt!Sr=gyOwK4n!FQER_wvbq&H(JLj_@S`XkzVe&*A?H z$0yl(0Xrx+IrIbBqt$aItWVnHOU@U3BWNWl&he9yx2WxN&@EN|g7yUo4wI@jU=MZA z!UCF-T3JW*K=Y12zFMyv-&7>PB!kpl6o9&JvI(Z!9wT@o^aO-RTF)f2z|#m<$&M?6 zKy9kc8(~T6EAa`_s4qG9)ZGrvyeG+HE`V5(v=5k1fcABf;B5OeH=5s!6E(c9^f0Io zoqKSXqIhh35r2%do7q~g(>1jI6Mg+!Je(I|oYHayi5{>0LNe@F*;IU|_6Y%4F8o1n z@91#xpB)?pVCB#_`eZ!U5sb6#CLV>Pf9mb&4~2!ubRBE&32ji@4GP`L!O;LIqj8A} zo--lsAobN1s{$OCJdVgqgQAE^DWdJB^=6Tt!u6U;(^IN&QMQ<0T@V9HuCps{@T)Q`EUszpyqMk%G2APhh0n;^D}n~BZ}=0f78U(xn`l+hOz95e zlHf;TD%t5F5TOKlyFw_YktB#`OG8wj5SxT--mcEc0iIk;*?kLQlRKR z>2=nv_*6W0#ynL0;CMok2~Xq9n!pbo`HPqtljpMZhbOUI4mGx3WYTy20819^qB0pH zPf@aaIs1FObc%DZ+wK0IEnFs+5quI(oL2Yu3L;f=8b1&Tp6A*|xEJ5{G&SAf_2=LAo z2LA}R(un>v&jQ5>NJPPQh%lHRnLIWszuPb@4oYDRnGzYoW@zoR-T!bPmm3*u;toez zQZ)DUigjsxO~9X4G1f{u$32Mx6u&D7UB~In7FiSYZAy8qHo58@Qk-ff(TtP zqlC}37n<|TLO)$Z-OBUJiv@ALh(XndH=|czep4enJgy^cvDwq21?jz+2o5Vg{h9wq zvuq^EE8x%Lr@eJBB1|{O>nr?8rzQPHuk=);m02QgPU0*(T1=DP&MDTvj-NKbQqe#j ze`2U0S9L4bHnbt9d|im=&$mK)m?DRqth1i4l)Ij3-dJJrI@dSeMs>hQxSyo0G-XR! zt$~IJKP3-6X_jjxmsY}h?m1OxXI~2ArJ(;DX9;1&>A+jy!)eQ!jDWVoif z9KUxm>MnaH+%3m66?(4YoaN=SSP+(l9uMT)Vwq0}#F)ibe9qH%DcBWF@DPu=ozOTA z+a!i2(5CTNu5K`wWNG-L=S$DYA&P*PZeVDc!_kH(%=<3Ar(^(?lEkgpsSg0J)?lYCABp=o`o=1)-F+%#W?QY$f+kT-F4#Qkm>qkv>=IJd_ z7%)l%NA&|vf!CaV~tO-skDKSw*yzf*YOSsAgIDD zhk>61*$Gv_v;AAE%I?g88Xy z_JAxFem6LbsbDQQ8$R4R^OupR>D?2CcP@`slipKu;O_iZ0&T-DghWuLPs@y6zIwi= z1LWqFrd$`H9pScm0$AI55Y!)i3s^Sn;YGCFErr(}4aCu+2?&lsI_caNdN=%0rDVN^ zlW_@OUWlZ=@LwDiSNo=@at214-ZNUoldUehe4s8%t{t(ru=I-6(#49^rC&62V3Kt1 zpTkF0;YkdTpMgAH&?dBexaWoieWll(Cj=fPLzs=g5G>l%t?q>l(#xPa@Ci@YF@5gd z%PR~&8hlgJaY2MjaKD=qtP@w+ZHf5+j;2y#R|!A{q|e*9^hS%)?_Y>u^(=luFo6pr zE7!N6!lx)`d|yf-s2AD7J*bLU3TP&?uuC*NU6_0iib4TUfP0y?*87LO6lThWhTOlM>&}a!_eZH8~;{iV9rO$$-l*SA?4^x zq;HhhDW$()iXoKOusxX3E1@6Q78nEWg(>&mo#iaD${0kPG9AX~ZjXj<_ zyxCqYBT~m1ktD0unw*HzCgJ^t7s93XL*3wYqy*(ymsGFR@v3}*Xs8nEogg_*U;M)Dw6y&~ioVV|tMFzobq?9#v=<_NKvievzDPOTx{BKI6W z5RjPX8wlV`I6ZS_rxmbM6f#{KhC1oLulOCIRXYf~!-ehc2Zt7JR$v5EBuG0$aU%*4 z&#R-aP3WQiA}k6@G;F8@vk&c|D{@Y3D1@1r0`HCL%Z4wFgHEPGAi$_wgtHTOEufSU z`0g1}Ywlng%+CI@W_VGI%3QLOFn)?YgblD}snqnw$=Ict-V=fra0YX+1Xk<(t{XXC zy$H}$-kUUQN#nU@-@iuu&EQq8hsaoaKHPg?B4>vS&A#G55#qop? zUW5#|N(?z;$y)M3k93iviib3!!;i$YVHkKtORA|Rx7|SOefYiW{m)_`8H64+j0gPB zV)B6AiAucnH{r|X;JvyJIc*b{BK$xyArdmX3Z7O61H)gQ@%M$h_!ZTnOo}IJ+t<3O zpNz|`N>Q^uJNA_f@f}dIDzlNegrE{*%6Ydge)WslGqcv!Vr}M9UB3nckH7n(iLU0ZG93^;8AIUJQ$-Jz@t+Pl>Fq-&Yc4wt8gAxo0@;x)s z;tmJW`_N&ic~$32^zb`wNc6zxP2rUuy0)gO4TPl@#$W!p(qd?_&gbpHirDu2D20`! z$$e3`M@O~&+0@-NvLG5y#ad^B31ZU`Y?k#x_R^5Alh=d6+PCb2`4C2Ls zEo%sR^b-dUIi~P#4i*3UmKhRfq(Q0QhER`r2{zY zFIH|@aNu~8?7QW+4{Wsl%e&DB@81Jsw^&now5%#I?z;eunHeDBFosB!7dh=UMm|wEp4GXH897Ix^<Et3B^!Fi@NW_({ujEu z?$7b}?3b2u@U4H%Q9UT&s&49a#P`%7x?kpq_tY?$-g$N`{Pyr29~DxVm|?Gbe!7j8 za2J#kx3J64q2Fx7MpdVIz>$Bl$?Lu7CjN}`1pd_$Rp$Yv&|$d7{fWClADX;aYqbUey&i&HZ)m zM68P%VU!RXYOLq?Pt|E6kMUecj?>wHj+u$Ivlk!vucyIJ(F0Fk4pvE?O5q5VHUM$* z4DY0CWf}tuw2R|}2Xl-~2FQ)7hD}TXZMxY(*6y0|Y1YkLWnmv>M)2u#bv0YO0xRL2 z`bk)!-<>tP^(I<4XtD~zRyIF2{*w?La3N*c6eBvK$#LNideYpKnH=FaD6Fq3P|-H& z+W(MpQR#7|I*D3{TQ{$$H0l34G6m6AJ!I)+VwA&EQ$cd%#N*+A_0Wo?}S7 zFGjzKCObaq)(L0h4O&4)YdDc+>*Bws@Q&JJ>*XAQa>|FH2R#o_KKof;8Ts;@7M&_d zm+sTmVpmY+=vd#qMen8I^XsuL$|6?KblWM;wN_dG)){L5AH#oMtvif%3m2Ll?NB3-%+d) zdB#`Pwb_f{a3prF;a-8hYD$`9m=6syk1E@k`K5BaLZoW;+os0v^+Zy?f&fE6yuZ^) z83~6O+b@?)%l9%7{0p$h3iygIudPxSoiKA1mhZ-SZl!|F9XP_Sgd(=6_KsF~0>GoYb5`7a=-(9XYYt51`Iz0`}Q->Mz5Pc1Qm*hD1ys?F)7U&4h_Uwn+WCLMrqs1S#VjMJd;R&R3Wsecb2Dnd3I2zlIM}BU? z08|qTEWY(0*n|7OPe5Q+5mnNIIj%N*5Zq(Jh<%&eGeq(kd>q^<*-Ln8N=&KjTHt)C z;Fl{A{n@*=lIGMQPpJ~fQ-Ag=6TUoVGSa02ob?+KKPa#^qfTlBsC!v&G0gHmR`L z-qo9~FEuegaJC=o5~Q7!VAV%}99I~M^JT3Len4i=m-!!l`em{(vvu%9&eV%iAwHD1 zXk!uptFKkyr%C0pUre5*-t_oc;k<5qD!)PIa2I*75ZN1UW4!+2he;qKw4Oy6LvO%& z=rXECHBriDTA~p#pIhkO>@YRx1Tth;>x>>=cQ9J72RoAYJto)kE1rKffOD9f!DtrtrAsq5oZAvzEJ z1r@PQHo_%Iwxi5*iZhw%5Se!%%S3)*beh6hC6g0{@YtZ>gepoMb>gx%+{`J$13JwM z)=I6iw`L~_pkTK!5`_0k2`|Sv=47-p2;wEMveNPXV4(w}!9^YQSc^%sup=ib_dg~O zG{+kOp!!~5c@~dhL)d6)C?p2otCxcb!%k>ji3Bz^tt`7$zOIimFnN3 z-E}3X3IvKT*1_))!Q;?t3k|hVoBa}~_Rf3C5~0^nog^?h-&qeU0s`7BF0^`y z(@v2%fZ>yeJlOTAD>?)d2z_1qSH@B$2Q5l>l%BZs{Mt*?$Zk~8S8g813{?aa+N;@V zJt8l{Gnc1h0FlnAmbQLjQ0wSkqJ38 zV_3TaNJ%kl7Dhmpy9ZLJ&o#TZaS*@cI;em?Lv~B-_ZJ#%#|#vKh&Py>LBi2&fiwG7 z1KFXVa$9iwRj51xoG^__bywVkt3*dvuULs+j`EtzRa0>EYu(?CgiP~?YDtT$<)YK= z6|~Tlz^6KtC@SYBG?wp=8$3)&)OVCdbThS&dQ@5m=P<&x6S)R>yOMTb7NY&jM`})0VPp> zKN)YLVFUjXG}Z-t@cN}A6v%IsJ0bcZ&vs`TCXY#JmVwL66Tr09;+9225cmZgySE#v z>Kp=spAoaL`Y2eL<`1LQ7YU8MB}JN%HiFs4*f4aE4lddHJRB8XE=K7b#t!<)>xHE71VZoX}@CyZWfv@ms!t` z8O#K~xb*solu%mHGn!M3Tnz=;njVI(qk$tP8zdVv~jZ6H0r^xNQjVGk1r6GGdW3 z0-wM9p)>)k&;=yW)Tul9jR}r2r|obcuHptwDRl>ul!KqUyI`cQB8CBI=w~Y zJfx}2@EFMnim~V&nzCn@a8NruqdCPr9|77w#jS`3-_|{~%KwoqPh&`C3~Rt%q4Cf4 zb?D_=8=X0@r8&6ZgxRy6@ye;LyChM73?BzfzJ2X$qV$(U>aggWF74flG_kYrGf_~ST z4KH_qeOhxn?o*^65$Wf9&+x0ij3??eR@Ix?wM=zq@*Ym8wI+$Wl{wZvbF@Lxydmel z*S@ZCIq$`ZaxeXub3tf`T*&`;TOIuDf)SGw3T9p3_AHP0uEi7`Z{(ptJ=i2D(7D~fu3bKxt0Gi zmpeyVcP0WXVWNilAwfwat0IVugX4CQ7(#(8m3)_m%!1_Gcjf5Kd`tIC3Y4q-2Iiu3 zn$a1rY2&2ZR=cs_7)np7^9?NtizAN`>iY=fBCS#>!4XsvsJh38#Pkkb6Apb1fNNrQ zyTr-bDanGH#jA!5Vp_>1O-nte6()Dxg?s907KQpx)_ap74+hGjHCUo z3=-dybQGawD<!y95-i31e5WS3N(B=G+d)hXfp7bD zb0&l}R2<%n=1vb%V3@-;-?}sccP4Ai(UECz|G~w5M?*$;HN?jjj~<+8PgU>HBo$FD zuE-O;c~&OX#IXB=v_s3vx#dVjY$H_497W-3Y#_UjhcP`UVhK^KZk%dMTU$zji2ud% zov2Mx=&(}FJzde61)qo-DfrlbqI%dRQVkqC`cIk%{LTM%QP9iFB_x9F?^(snFb(-i;2 zysoVS2^5hEEs)wG4F*iOTX++_P{>$oulGcFr-DM5S=Qn;(2^!(E& zkiqC5a%Y5af1zJgf1#tk&d%RnXybpBsSA_Gk@EKxguKrgr-?|g5PGBCS3XP+T(PYK zt?TQuygUjuhAQp>gkv;rq%Fa;%v}@;3rFGXZ!2NTrv;V8qb-CKFB`I9+8n4ybB2s?fKFOa8gwPQp}z^+zaN0O`L

D`r!Z$1JQ zzyJ;{WS8DpCFWMEjYO2P1JmyOy{7@Mz-yIJ(nVV8SDZysJ%e~hz?1o>w_KEUn(@EX&9*^fL#A?iaz z+zU}vali& zBTAMQRn29JAd^DuPh=LU#Qx1Ttw%Kk50` zzdz4APhdpFTC3>#Wl7T%Y|%0jE~2n4VD4juY(CcX1w9Vg-XUlO?J!g(B!DgFp3!c& z>^1oK2fg+QeLd4-zjz|ZZPhw|R}*#%Z@!GvWpbUnVjKq6drz^;&0cC(+=Xo4i(Mo) zZ8m<|&2DZmfIcuyE*Nl&^LQfc8qQoIGU=Gl<5IQ2$Zr13Gaa zPS3yIkJ2ZSe|!$f!w}IysE&8w6BLZly!7vJGs1Nr;?KHeml7c^?{LaJ4&ww`Z*B8_ z6QXa{wb*W{0(fA>k#Rk9@sv1;pYw+vU;X4b@ReE;9x8`aWsEn@fnkYT|JL=n(arSK zM4pqblR=(=0>jPBo5#fqoFJG|Me^?RdvZ|JbJf$GCzD*7kjMtrlC?>5ES> zZDD4ngX{&N0)-Z%{+g|-<~+j)@&fM2Hmu%C@W4Jd8a!@7$~ z5M@;qIjqQ=*iA843wAF|X=bA3=Q$V6p3U$ij;r+%MtvNixOile%m6E=$`yRuB9gWg z(Scg}AMx0KGZ8afVMo;R%Tv#v+d!*}4g(TZAoGxQ-Cup7_0Q26Px@~)gz4#lr5kCD zkw}|>$2M2N(UKD{wry0i^#Qlf2Pq*!gv8iJAT-u=lzpL2kAC@l9glv6psC$J^T)z- zN7kc5m&9}V?T;qY)6M}P;?83!10Hff=pRqDLKmPa*_R!QYBa!}-XQd4XZ+{JNucXs zAA&U1q?An4DYq_l`dc5;a&2a)5;tUlwX5tR zwhJ?F0XiE_MCS=UrB~YEP$v6QLMGWvtqmRG*Tn#B2wl1;R;Y$#_?Lb0?LmVNtur(rU|7RPW#*ib0I$w%EUy3#Ir#}kNh)y2$XcAcKD|m zmi3d@&zAC_k>EFUF#2d5<kGCMlJen|#fc8&fUH*TDgw?Z}@}y7+vc)yni~;D@wt_!Y@?f9MmR zSs4!LZCcyqEv**j){d@_l^2Z#)xbZ$X-yrLBokA5&a9Dqa!)E30MN z|7=zVx#|2B)9@G15WT+x5=Te@!F&jW8%0IHLSVTA_MWI=~NUMo=<7SwY>_1qzvb!{M>S~rBB4i~z! z$kkmis9|xpO<@9~W{W_ABpwAWjLLihBm3u(E5#RVOk5wRpDClNX(6(qC&&46-jqV^ zqH;dL;DUY&!Q=%7YBSIgs%uAGmjPN!3D$vQM_`x?N?CRv(&~;YaHmDPftkIJ>|m$Y z(X++`$?*8)*T2v2Sa74! zuYY>vVN126%ZidlWFzU!d!2sRv;ia&34^6XjmyE$BGCMn4$9BemX4^r_l3YoZTf8d zpq5#N$NuiFJ1s_rr98I*x6#KMhP z#J!~EtGONe@wR6J;MSm2gPkF{CDMhw+-p=${)Phv8vsc>?l$7e6P0{f-zN#hUKLi~ zO4H3lzrF%#aQ3{->J(_^NsfliY-;vDyN_O&;U$D-4>Xuh9EM&>PqntbNEPPi-llgT zT7rlkH)aiT#%yQz9;SwUxWiTwR0Sk-BUYa1x^Xl!?h1DYe}|PuxSGE39~#1VoBe)M zT=R$ZDKvIzb0Y+L;In1)rX1V9PRs2j>^6E@~$6AW;8XINFFhif=)mT{1`CNeU zn*jtFL}SS8;N5Fm9`Ib_1XjITBzp(=e1^&B;J;Gu)#`wKS_f%VWn*KBdvJYPz)Vr^ z+oZ)`34VN_El1GhNkPek4JM722$A4`3GU2@W|{}+3scsiejQ9oq3Tvv%3RCH)eYU7 zuyB^~`PRJ}PM-c!{{>&KfcnBL6tNEC?~P;l_ksQtSe|uPEqQ9E5^RU<1{Tgw{#9$7M0IJcARk_u#AVKAP}UdDAJ53yxaNgDrApwoo8m|Y*x*%1N{;965}!0e4d zr?;DHqP_bm`t-x=-S6J{;a4k=K7k3)609_HbGg0#Q;1umwiqrdOJ_^#H^#_(mSaKF z0SiCUarFOlqSk_+ODr$!aBKpj=S0r+Aw6>#*R(+#ld@UY`XaA(JJn^`WRj5j)0a4c zrFdZW6uB0{>s$>FCx}MkGsiF>@7Dfs&eq_?Nj;py-f}B~fk>4ghgcqH&~!e3?mT#M zbWKGbF=jvD6C1spR>Vd1ho}m)#>_pD8+|_XgPG#MH-}b~L>u?vYM(UhX`F)kB91Kz z@|}wf%D)aX!{X@s?v}qtj2S`oFk9)UJ{feIDG-F+qnz`b!&63V_FDgu3$K+!=rNwt zAG(xycmkZ$r6>CZkbW5sg#zB};O7b!ca%Bm7iNegh_5qzd)5VPm^b)1h?#@FGPgmx zirghT56{A#GN8&Me_K4JN2~9vZCHI*oK!&mEzT$=>BA#JRp@vl2F4S3Gw z(W^AOFg@1XA}`^O*FZ7yKe-11ROy0K#fXY3)WB!IhV2w90p_Y|kO*WB)07Hvo|;7S$w=A2+yw>+(_hpJ%L zTo`Abq2QU#QT54nzd&T0R1rAQ13mCM$;PjIsCk> z7^K&@{FM2ljFx0IxFkC{c)f1+kMW(4ph|=^7wt_6gHV4Ql{I`p#)c)K;M7Eq)e2hv%|F!M? zyeZ0iM4JgDimlHszF=K4cUg+2a_YT-R55&}7jy(?(LkmPn&~0r?O3v21-wiHWzzTI`B)*$^Qjc} zj)G}Q1$HUue`RWXZy7;-)fryQ)RT z8YA341G{?=19ghV-AXL}e~AjFp}tk>)7#s`@Z;L&7=x}RpLLW2XSY%-h6+Ui8eW_i z+p&{xTc*BDF{fnxC7#rjTEY_ex%IgGqE<(o-5`WnMOA?PzyOh~kK(CN=nQn&+*8qM^RbiftB#&+9O=8Q4T5`7D(}Fe%goqPOuet6>afw8DKvKEgv`WK|NRkq%|3o`>hb8%1C^@9-D0SxMJhB*f09~F(4znZk8$B4bJ zW4E*UPefNv>tJxSOz$7(Qp9qm#OcjFeX{=q7F3<}oL_>Y)-h-j)X3m6vn$DlH0)g_ zdEE?5vyUQzH;qMhE+H88n2&Ou6O&PkA(6nwY*8Y&)>)6P`Y#ZeqFRU8UrG|SPv7J{ zD_d9ep8y`k)NXqdC*O$_x9XwOkmW5w;nXnxb_<>n&!8MiPF$0$9#Am>X2zW8v2Eql zD+vx@!|}PDOBRG15)boWuOX6X2w>3QNyu3HQGB77GT2}9Zm#kYhLK=pVk#)cEkyqp zg+CkR5HY}n%eAJlcR8ePbrOLRKfF`n5km4VnSQlpmm@;M^cpj4CEEv>3m?ijMac znTc2nO0Yzx;Y*B$RYezwexdn)HKghz7>!8p!Mt~8*Q7{Q_C?<8fb-!N%VP|(WrhRv ztfguW2;IOFaRF9tfKOr)abk-q$v{{aA^sb)lfqP8dL;<+EvahS$Vp-A!d8F~SqzoV zZxEmF-%)o{2k4`MEq{zZc!IIHX7zR%#UfkC8nnLOXg4ZG9aM7xiW;l+9+|z?Dg}7X z4L-h$+vq^l9Sutejn{gC?72J&l~c+eureG?hn|mb1YaCXatPPnb~DPyt20XziIyx} z+J@EPjKL>q8gucjp^{qwV$iPdFLd@6^0$+pI^kUxLY=zThiGxZVT1L}3QF(D61w4I zF`~#@7n=XHkbMZTbQI(Ith9Y#5|KUF@uQr)GA0kroDoG>QK26D7%dEPyD_wTwh;#< zA+T^-DxqVhw1RH-#)s$KJ|dnxXBADc^D(bj**asxogm`mm-3z;u$7M!QL4M&G_e~N zTyxXM6e$AZ))^puY+b5$L5?A;0s=DvAg}4Md?RB$MpBiZ)%ea?lyu520v#A)fWOfAWHG?T1 zq+>wc$z5ts{I=JF0>%h0B%)_?(g(QakVQJ@V?wSvs!$QYSlt8+cT6v0&yYrXypJ@_ z*8=mSD>fwN<4PRhzV&iq`)T!Ko%f}Q_(EnFC*XA#vki3$rWmZ;Jan6kE0RopEpJkIa$5Xt#0HvGDZK%NeYngDMSiW9u!hj_PdsqGY&l@o zsOE3?5y^of2`pVL01+fC3iNGOD*O^oP)kDIgZ?&a+l!I>Jp@&S|0@+FvLDM3`kz%X zH6=~-hU1*2Ythz@52}6w-!UZEUp;$NHOSlPkkHaJoUz$Jz+$+fQ~X@I`@<7%S_qLuCCE8cVKh&)UFe?2x`E#TS2kERg1>(4Of zl*O3uw^!j1>PRluP63aBX%AmjrF$b5Hb5E%{uK8|V3;c4HecmoN|PI3Fq)qT(FeRE z;ba`&M8|sR;u39*5N){c6YNZ_Ybt>jCRT||wP#a4#z7vYwKSAZzk5qN-W?LgDjjQX zr+=@%x@|L-Sb$>O)tpD`w*zKMY;K@jR2-lvMzavI7WTkb@s&o89c&!dp;!Y|x>3;M zNI$C9Ok?k{H&POyr?>6U{4a#3x@AuWEu_A1%0guXqecyXl|}P%&Hz^$CO`_N#VSAH zbMwMiUz@e1da64&@AfrS!U9Xc?J(JP-k(oiQd9VY^ua^z|~vu;98s%Sh;OP3Kl zX*gg%_1@#;f>4NXD6K4^B<^XPr9L)*>%>9VX80-B5}|#^9~i+5H$2M>dY>MV`Xk-e&apElYA466j~<`@F)>$OOag{(PRb ztPS9GN*Fi6LifLHV+a3jP+)x`6&GcI2A;rM04I4g0N%Kv)0oZJds>5FlQJvAvz+rX z6a{jg>hdOGHa63Nr1D8$?7l(|0xp{1daM7u)Gj1Vfd(28l=VJ9Q}Ml+R$784=E>iQhKM_S2}?G4+ma7&I4E$lQc&!7JL$ z4iMPdD!ZVh%nnRj#*IV*!Y|Jbwo*)obA=BRwEz-C4uGenVb80dS)3HyYfR?!Y8~6} zQU@Skem(hdgCaLUL+8^tfE`;PkDPVBR8)~kLmBO>^8&IvB?NIQ9m@MHZ2=HB8SE@f z83i$M*ROua?ok(n1G0_DX1|Z;$3n?J&Ou_e_?82oq;3e}K77|a{ih#|d#I-BnOk`* z;@48MsRZhfL{Xu^p1eZXHa4I|Nv2(>dyH=cDrAlWM3_ww$&~yG<9bOTV(2`fZMZ+x zk8IBmW4?4@D~yWZC;T676z*V#Kc?1ZT>p}cNhB(Kw3G+g-ZF(m8wsgXooQpiYRW~Ef~ecsA_)eJivmlvu*5}*w7>WKl$m22{(^2}-Ct#wY5#x7aVoZQE7~ll z?{E|U=>D$;2wWomU(NmBN}3>l^8eQen^a!L~lqRY8_4$w_6O^F~5MK&05d=6Z@o&41FQ1odCxZ!+QRP8-9hb zT-Lz48UUj6n0)+n zZhn4ZhwBSt7E-gaa!K)3A!a?GXsc4u8$dy(eq#J(*l@zTeis#Y-7@ zT|DBQZ!B5iR6amwLs;HjWE2m46ivK!eHwvM0XFdc*3A?{bG(!1&+3iLEQw(5lqg)H zx3}->x@8)Jmj28^c7M%%PA<3S*G)3_{*IS5D`25}<#*f?7-ssE2D)MG;iwv&lS6)p zx3pevJQ;A0_|!qC9~7(X2v~O8mW{R;{S@i#XHF-%6g{qKErXeP_w(uMnM|YXTB~Eh zT}l$~M#fwLQ+)Ng=5Zk@gt)`$3GL}GLaGscRfmWG8Sw42!CJ^qR~OA-qJbPLC#9p2 ze>MG%4%EpZ)JaO>OFTrl$POAD`>dl5yS-paQ4s@ki(pIf`3kOAp;mm4*8m%aNsN&Q z&Mi5pq;2WO253fS&dvNEtSL8oyV@eeT71Darb1*5`#!}hh|-IzI9}sPd}G7#{2uNj zW5{~z10$C1a0w0==9KrWD5ON&(MU-_)RmQr$_>)qHE|GyO7Pxc6E&}5ClaOWUB@pub(v6s=z2mQwF%up8B3P5+p<9+2-5qU!CNfEq5#J0u?!`=gBxW5e!2r`>*ZS8N6;Dg_cuw2xIQ|1l#)-!)3ri+2G<%S(c zQ5>^>QX*5abVl`!+!)cK+^&Dy9GDaq93*h}NDaA&R3X`2cskGkz#1A7qzC>gc|9{J zB(t8m^R!1z_A=EYz+!RCvb4^+{H13H-#?RqYfmJ5u4XRlF>_PvRnMlBkGvA9m?W)N zZ!$Z)ZPYzIrpBp%8NDzR)fhZkA*7}j#%a6f<@l<1W?S&_L-Arcqz{iJK6zevgWo;- zuUVbzpPa;dMIk~()r8;n1A8CjDAmWu0X{^;+zBSUv*b6&aczRHpgXD_KVZK)4A4Lx za>>0PX{sEPcpCy2lZ$9h2#xsssz{%nv77vT>eYQzNFZCo3=fyLO8xB!?&#GE7( z204>Hly6adc%(xliubm|8hz{~@qi(*d#_&Oy|*CQ!jydd61+=;tqg>I(K4(L*x#4+Wl8wXgVAg}>i zp3Rw~1mm@r|7uz}(jo|z6aJ&qDLhy|Bs*TT(x1t~)7 zf%-vTH(K8}!hA;7p;JGl{0eeses7d$i0W z)c!!ksD9X!nN;%9t*m)QP>-(-)^>j7qX5B#3Xn1<8w}-P!cRVDBcLl?F=Sq-yQMGEU z@4qbd$14@fjP};D8%Ci#cA8mt_TG@cV~B6Ha->3)Q8p+1WVr)&RYE@{dTLNR+8v1x z(rW)Yi+DHMt<#Fwe9)!1~kx7`*(LL}fsbV_P zC#aY}G8VH3(*KQS*27-!VfD7__QPjVsB0RV?~{#)Wbne)`0uc7s2~1!X1Vw6u18U* z0fxgdBu~x4$4rz;H6z_SitL92aFBu~u~*8RqwH*qo(g;ND|MRwB~C<|DlADYiRZ3d zOe%7Nwmn;2ngJ($wwiSai3Qe|gu-?G-RWs=2q|kKH)!-1%Cl{V_u-E(tDb?py`=PJ zKUm={%+ltUmTiCJ*6K-Th z`T4$BsCgs36E)DFqy1YyJ?p&YIV?o1^?F2#^g&fH^q}W`OnUj|sv@X@6XzGPjj5yY zSIF?{+b6jf(hFXecZ<_{e}j=qHXJ1?CJT29r+Raq)imZ&{o|{~kH-tMsOLo!h}qdG zSSnGsg|}xwqsgQ55#GVsH2taNuw>Sd@GEw^rBv4+HVYi|h4xCC^s!Ld+Ox22&6;ZY z=@+@MeY?$}B=hW^2vv~=`<}xZXeF}S2@cSl;KV-?g@w~02H_qj zcG9&tdWP6<y}W_07r1UChbZHsNvMN| zW_{hL#U3`OPMCu6L0HXv)9H%gz)&>bY|g!@nEQ|N|C3VD55gWd$pi{C7Dt|9uX{L? z>(Xbza`skjY_uNH{V{oBm+l;DG+(5Zj*V|yloo-9^IB~GG@z_KmM%maE&A*XL11{T z(02WWl=W++%ar&d9()NN?1B3Mv6d@WK~f&nq%1&V*>f~6Sp*H&%jjLKo&V-X&x zNnd6(B~9z8h^xo`dqElsbUNPJ1b!3@3VM3os;cJ|fvGxNh7%_C3z7!RtI;%?Ylbtpl!%?lJRs@M0ygL@Zs)jKXomDm zzZ+q~5I(*VC!x|zWwZ!Rc-p>?L|6j{nK?tCzs~Gdz!`qe@fi2e0n-?Ilgi4q*9z<8 z0XV;(^s^<5`9>3D>8>|d2sqfX5FP_iQ9n`N1W>y`t z<#%m}A*t<=zB=04@DbUI$*BVYMF{yEnUU2z)ra=k2d`ktl+6}H(?6U(mQkpkiR@WiwvW?UbMX1J()xzju4)Q z`vmN4a)pYrW$NaBYZRvW>=Qv`y`Pqu@y0-cm-mw?X5v9B7BDKicaWE3$Vt#+l3OzE zy9HW30|x9zRr|QXsdsZ>u<4Jb0fzG+LdwUFbstgNJG*H|x>nNYd_Z3aU(nzOFHVWW zk4M}|Z+vo{JaE0%Jv2H8B4LmQR{_7xh+`(kRrmK^tNEJ=Quux&SVnA(I%8HX9Mg+Y zIrpO%#w9%@3n9uB)pQF-qcx@qX<%vY_wF`(Z%d+1bP#FFC7k8>kqRM7lJeNJr}`%N z^SDG7p6APGM41LZ531SsxCArkN(m6jB)4oW@~7c*3_qbmp#394h>wmNtf!PPApr#@ zl4_4nmQ3h5ihX+|lz_t1^ajm3UDVgz+~_G;3U?0O!uB{%w^qw=dI9GDyjnfu)_WA~ zo_wTwMs@DZ##f!UUT?~1go%yBm@td{$0;Y0XCP!e7`NZkRKsJsk`At{7a)MGsyE}^2P9^M&?Aec=D55S=Oh+4&Dihkr@^Qq-2V#&>oICtvW%5 zd52a|ZQC%s0{i*XzvMYkg5&h8PPaD|HWiW({=1_M_-0`CWWe!hO=j;B%b~CtnmF3q zRi6AK)aH$R{q}jrX(a>-G&pC(fkl@SvK>c#6@^;nYgbF0$Xv0jlw0`lTN&b{TGdB` ztgx<0ppg{s?N*u}oV*bj)34vRBt<-xt=Z4HkNB{Y1?}Thjl}`0zr&Hq|drujEEwi2#VXqv7$! z6n0KO3-FAHqO0i1IQllmwd&cUIT(qox9lIr`&U5xrlfL(Hi_tD4b1NH>vAQGrC8(* z0%f_`>^SZ{J0b7tt=t!mThhqiL|p)|0T!Y|s>64!k#lI%q@%(ZThJVVPKgdyyM305n@%c9Yhd$6KJ+x0(kb*5%E*d#YyFY4Z6@ zab<8}N8&2@l|cwjx_K^5Oy`Q4YdfotoUwBX6crN_&dK1kY4a6b|QB zvFh+xo0P}r<{Aj8aW6Dj&aP7c&_m#7R7O=#alg2eR8RBd3OE65XvRdHCmrPoZsBx zSld8Hw7~}vJcU|N+{P4Q`E(ozqYUBw?q?RrY}}b6PYFYL zN2My9>L+Y(tBFq6E$#<18ghZPA{ua+iO=g@F9{R-F%}ix_@gP1-!FArg!%P;lONVA zf|fD=Zqu|tn?NN_S-G8>&!)zumoptbY@5;&2#tOPaZSV;@PCse z*XMfYm`<;V4#h4)Z@NaKd?sCili-hvtIA*N-^j{jgK>YkbCn zgs?+$BN+&VVjT9^zQn?qi->cFmmnQ z=KNaiFy*7A7?;wHi2UEwt#Se3AmOr6nb!(u(Jrf+X`~{R6Eqc8j|{4ICrhb3wD>BO zRjCp94ZU86=w$sd&8NJefcw}GDCDACUrm*sbM`3#W-nFT49LfQQ8bjZw@HQ{ z$#+cAJ+c$chcld3SR6h!Y`T?oaEW>zBgS+Q%?N0sxGDT`l4~_c>tGOztq$0USvt5I zSt_{ru>jKKWM&ya62nCd%u4!vP4(q%=G>`0_?_MHjMj&_8MzFgpOe3AHr$d`nZC^3 zvgq$H0#EY6|Lu(y_5T34zl0UBmcD~LTUia9SON;7eryvWz%9aTondhbX}Y*?>{apL z)r_0pE0}B3dxZOK7dDpnsGcSqM5$pIQKLnLb9%_cRlz zGW--yfMF$kddaf9e)R(S=|W&&dk7Vr6w(V>s`*|-aB5ylJ_Jev$U;bXKV&nku^5p?w zpl>9IR^Zfvi>*UET`lM)>RfI#6-{9ry<@QhHjHTB=o`&AuK8_WKD_+BjuzfI8P+v*~m)4Zto=-%SCq zc=w#(4_)0UCnlU~;ef7XA0I@paSREW9%^nt=9d!~(XCyd*?$z`=I$AFK5%6bf*&VH z_X-(PLyGVo_+``!Yqi#QSld^C0tXIcjI#eKJmYO(#sU_|rp_UlaugjQJ~-4NIPN0_ zD(e&^<(y5Rij&y8i|?r(QMZ`Hgedx_v_-;E-JZWSQ;)Ot8o!J<>R~P}+}Ts};!9G% zEC1KMAMPh!yd%c+r)8zsVi@1iO74LD&J`es19SL`+@&3EJHGgd|GIL2-f!qF1aIXw zz#*;7k`)Wgs{F(cmZ?&q2P#g&&-n-6Ynyn6tyDEaTA{F$Qw-{u%LKQEoY=d6w|~fu zF{)Hj<1w!gpYRs)@qwDg^iS7;;V`V&-{sny>J!nhOEh3M?el+MCVDgakf0WY{j8Bn z#3?>QTiBQiwkwpqO+fdy|$=nHI+PIK)?#1}i8Hh=_}-kq?} zHQYu(h+=0@yc-8XF>D3f_y9FP%D*HbVT<3p|bv7D^U2EyPSNwyTKr46# zqxBDl1Io$N*G+6{&Lwi_E9CC>SYxY=Ww~FL(z&$Rr3Th+Y z_O?;QyHQS;L$)58`DZ61Ev&ByG)WrzTV34mX7zNb%GT?V0uN%JAa)(%GMdy_O z(FxbX3y5yq#}K+V7x+n4A6OI}|`uRY(HSb zhGSo{;rwyc&i%f>x97wK70>`M3JPft7c1&t_f8}H)nMb zalp47$z-^X)F-dFo{JfvIVcF5zVHM?y4kbX@CN2F5R7^TSJezEojW&5J#=%$qI_kh zT%6f&4<4fWOm;oUT#CpLnrI`q538Qy;*u)@J8^&yO_4YT^@q)hUHOH^0p)5MX8qh@ z8*Zm(VOhd#Z=sVuSXg0*Y#Po|5@$kI-v-(O!*;ef<9bZ1$7SX9%6gcp$Mj|-gHFTT zmPqOS%$OZ(gcE%$06<%Z=A$8kTIpT9=@)n`7a7KKhhB2DbMZHmC?A7Rh*s;**@$D^ zDYb3SrhD0#%Wr+p!c+l7ON9_>j{Q+Eh#-xSK7cB3d9&Hi6a~)SJ=#*9*t5WEcKa)Bc;>b&^|QAQ=gi9 z)sF9Ozq&e4)f5SaH-u0Y1ElnfP&9@R{CB`O5D5ert7abUv>RDWY~u>gG^dRE?C&W| zmfUyB+4PM;gP|eL!$9gMatND@E<{Bw7nFWVCdQI5eAv*6D-)sc2JHi(g1A+_4%VGN z7&T^HGUnmv{dbAMa`07$cAzb0gV+sfuZ$Tfu1xm={BNe6-}4x{+}$UH%a0rY?1*MJ z_50VtP$ZYY=LXg8;(9A?JeP?%Cqlc;HqxkkTR1R164mk`;H(dtiRQ$vpK@z0qk{5R z>7VQzHso7Imh4!Q?gA=#*xrTL^I197 z0jv#5=oiU`YX{V6CvCz@zTP>@gdOSdeC=wYJk3i$>*Bar?pw^XCbhfGB9Kd@GSAl z;afhl#tbB=drD!Sh~67YBjxrWwD2HfDPxd0jWue<=+bkTdLraoKgg6#czn*cDAD;+tg!+w-z{9HJyU^ zh+PYXiitU`5*$xnR-M5iK&uo=xts4>+$)d@LZ4XfRpA9cUpU%t{ss!on_8X`}3!QBkJpk@ueWnMqXa{DcUCu zdB<>CYjzpog5JJ=$EGHufq5P5jKT+1hDL;avjg5eAu_P@bI1=(hh>}gCvw*OUmg;P z#gkSYylN7Q~Cc$+e#4R8Gy9*fyr4 z>|5Mb->bPpe36X3$6=Y!eK6~*2hn%lEcGREW*@}`Kb zPxQxLfAh2;@#=kg;cARW5)TAyn>l$SEp8uJxK;i?xn!3fo%4S)Wnmb^XZX{#xLNY| zzJ^0OfPKG}L1`7eY= zj;XH0H37xKVr2tUx%OfdRtJ%?X=J%|DG(-w$(VJ_>$FCSiiX3{{eb_4sYK7?&}~x= zM!UAW6;gB7!v<)7~8c7JltOhqWI9f z;`-Mv?k=V^<6bZ2Z+FivkRiA%ItggkW7J9+-(;zEYy8bwL;bWS9xM%PLVJiv+21?j zW5_J+WF2D)!ZUCsb>_b-dBZ&ZdiRGbfLq)|2Et(A(7AfSQdcXw^+Zrg8yOs z^R|X((O^%jT3HVnKxtTP;R-IadiYc9Ik%B+)3s&#M(H2{SJ@Jh;IjA#Q{32SBqgsq zPDB!TKq}%6?XS0W*zoJab}URqEbr;Z(swoaGX+FK@Y~?$B}M3j5Lvfrw{7T`uVj~< zgdL!QaXf?N3XB`#h*VGQDxR|{Q-dA*o`p~YKW7@jOX`>q>id+nK2?n`x)M9U(#EE_Qjf>zRK9O1Hvw_hc};<4}{tpQ&!B2U1Cjy8g#z9 z8#{|g*G_~(&O=>Pn^F1bbWbc6zQ*bq3>t+%5bsJO<)&YiQsAV^^8x|W==_oC!4&Y2 zAz@_{CE*jNl8Ii;U<@0U=c%L-c+=U3J@)-7}1fD9d zl465=&xdo6Dve(a5}hS5Z+7Pbg#xiZM6P5Mkuf+-HXg5>g{->lwf48|O3ePyH#1tF z0Fn`j!dh_d1Ajz~p@#LDgq=Ed2E_f@MSI6fRMZ8F8a7E+%a&~Y61m3P#T)cCkEG~;0PH3Nu?Sw_B8}8 z)JmoSyY?D?pvw}vc~^Kz(+F?4#HNoN1~V%9^X!PRJQ2JQ)g_iNIE2l8dQu>QwPDPt z%RU(Z!(Jh*f?2oAWp%7_L2BAeKx#?Lh>_F3r@P{^&<_q9@q_$^d#!U?Z?F6X9L;x6r!rjHK2K0whx|$NxRPI3sfUm!4gV{6DW)Ybuc$ zSW~MY=5jzl#|bBMPRXW)&$(J1ryUR`8|_2f(G5Qfe0{GcjoaD8`!_phW$LbrB;E#{gmw;tq|*7VtO5!kYId9 zsqFY=w|&ooUAOa`(WVpQ$wyAg%61y0!3VY`F6?;Mq0W;MUKIjRDa9EY6msEyK&8cY zbB#5<#JXRieq08S~r!I*Zjy|dsnx8eKxWgS&2p=mi}v}%eVLP>Og zMPDfhMhIfMWtCHVo7;oAdG(%zh@=x`#$MBy{4(@0F$upm89_)1l=NFE1`ydO5h%;o zDts(sM%7g5D5Jh@Szu&9TYmyPJLe(GvDg9X`x^|zvg!_9Al4PEna3GvnstUnS3Vh- zWOhVwWP8w!R5@UJs>r}^pDNXZFE8`aQs)Pq0)L9faA>;^tp=`nsz7U>CpW9@lpE^o z1?oHFLB#({3@S4cSEBCAFyLX9tn;Q)TxX5-q|#%9V#y!xV;mAT5`r$esXnC@BKx%5 z(bg*}^+C(Tu{K$t^-PcM9aIQ~ynuxL9HPC7>d~MDfB4S?5Gt{GwJFC*SO_GJtax5Y z;dO#ofG;dlTps3L#1eHWNO(uurEG@GfT9O-wxm4$9Y8vgDY`%RPhwtsEE~b+^p(ci zY_|iqS+%B8VTAIxoo^sv^{44qfl+;PD#rd(-<9O6rDf9et3;K{9YHGuLbykC<=5_2 zsqxqX(F>@R;G`kO^g?+@jgUkp zR*g`(q()+vmsSyR9XhY?sfvV2XCbr@x)s7@5UUoPlG2zjTqVNrbGNQs;|KH5izD+w zCtF*bP8ejvA|K*T zeOXB*+Wt8S;mm_#)ab?bA&+`8XA|Vz{VhX;gwy0G(YiLX?z$Z6Hxa?LOo2$wcb_-zJ6DaJuM5V5# z&=VZ-=a68leZ*I$*ySZvU8f5KfiRJ+!@;#pH3`n3J#n;lGv1H(Ec+N)ICI&zOW$RX zMKr~a-Ws?OJkhAhbtIOGGVML$n}M}{sr}$`>T#BL{!3mgua)Vkd|vqt5nT_3%zdRv zeh`-R-s%Z86*~3D)n3!YF(rcGmOdBULNVzkH-%JkX~XY%NCPQ3gwzn)lzdgr6p|yB zlf2^{(igC_2|s7eaLIvjN{Tb?t|xj~$P>h&*Ddj&djb2m6%#qgOl~H@p%4!T z#TT_B5Yj-W9*9{`OwElFLx9IYBpd|O61F@+CkpBR^&EM$-_A( zDFO60Kgi?FCpJlZnr)E0b`hYqBk#hbhHNe(shdnKil;aU}WiBd}zL$rAWDiFld4;kusn>Y!^#SYmU@>>{_O1xZe* zp0{&p^iv&5?(C0%GRL~lf+a^P;4wZ1g2P~4$R%xj@aCJhHqc85wZv<`y1|*-wJ9gG z3CcbS%Vpl;gE#=gyQm|JalNUH$M9&m>p~tTT!+}3Mp!|lhE*0aI|aKO&$uH8z#PNy zAV{46Db$Z-DRW5B?2g2wh)hk)bay1yz(wZQxKZtCz@CBPPNd@Q;kqO@aT*KNwfh_Q z5_Z)M*SHgy+JI%PbH=sx%Vha(L_cC)FBHk^ryyg3lpu|b7CBrr=h1!@1~W;GZ63~d z?lKX*H0LZWhf+CmL-*0)cTa$>!^8*ge2hq%alJyU;Rg_tq7Fg_t`xRH@gD~q9@G1* zM%#X@Xu>5p<*5nILmBnr$4}75ecM(%hBbp#uaM9bPGz9SruZ>Gtv5&jpOg4^gePad zVQHXyIZPW}j3fQ2;midHuN4*sd~Ml%8pbusbbLG105y8=HQiQQ?11v9?k{aIW-9=` zD2JGLYs}nvEjSR^7>1#qQppTb4gxNi8P1A-IW$r@4NEsihw3|d*0MsPcKSBvULoRu z@g9{UO77O~z?ZL1yxM4^UC%kyj{?QjNTFJj+6N$Jb11d2%dmdf*`IoAoZgA3o5Q7@wu7r_+D%YJ0E!O_ABR8WJ zJ)v39jhcY57bF(_UYJXs0Vn&@kCPL<3Bkv&jGc4< zy`To7c>#NC@(~&Z3Qfm*=%B~DWvPMWl>m*+W$okKQ9M(vJYJkKcMq&!x84m}JE6-C}N~{>OlS!?EY@a1WZNoR)Sm3<8yn!)8I@(U<7*lP;rh z;?q519sGxe%1he9(k00uSwBM+ZBtl9_qq*19i>Pa>%dH{Q@Qc8@4d}vtkbxVmg3qy z4D=wUs;Io^!h72k2S^Q-Lt&9_Tl2o8LWy&r26#9Z__T9Ii45(M!@Nb|;D%><0DP%E zd?2vl>Bn3704nC-go{4# zI};W7?yBR`%cx)TOJN+`ylY8I#6XexeqtU3qb;6JeKd}ypb;LjkG?p)nE%ZZV2fqh zPgh#(Z4^TRkRC?*RrB;{p{%1BZqY>qyIaq*6Q!DcC%C)Ug?yQO@TBs`Da2O|;5A&^ zGC{G#OcVcY+=p2vDO8|T$LkaQv}fi@A^N{1hH>dD^=U@yZ8%u{h^Y#XwC&*7AuV?j z-gdJUU!b~~T+!j!N*8O>$vw~7!;RC8|8MkjNODoy4_;-VFWR*cZ%)E3HY8czM3!#- zbCmE$vM&>^62|8fKCLoIiJQ=zJ?3<^KFix_o2-{laC^Id+UwC-KU3z~f#yKT@I09s z6#?0PB-!!}jp;{^(1~!?=mN3c)4WNqj$p0L!qqAng!_cuN6Q|HO`y&TJq2kt+G+~I z5VhAPGIvcQ_L&g&1n@59MrZU92=YToUch;DfEUU5=~IVOLJaiiM`?sS5f~`lGoS<(b1{MMGVmfEo z;_~ih&5g+yx-D|f;cKP*YD$YaJq;sJJ(B`&5oE_{tSE{sXu^dO*p3^|`!}_+)}kUC z_&S#<%yoBy#!Qv_~(JkZG;r;N5-A~(SLAko|b7wW_{s+y3Zp8CA^W8=b_lsuBt z9Cok`YUK;tKB(CErts3iEZsJjMy^_un<4Low`SH2{~(+CLdP#V#^8!Vr*iC(gaFyY z>V1bFbkgh+*3P@O_D1+j;AACOl$jM1NDIh|y*i{??j6WU$f*h3audX0YEs?%eA ztG#}Z9S&qH@lY^r)}>q!UK$!k>QgctOnMvxBZ0Kvij2+QQ5Fk6{INnGQ%*q1doBrV z)0-m`3=C1waa{%Sz%VEF+(pyn$c>S?dzb87{z!=ZzP?Rh*yJ0Gaj@U3F(anW)nyeo z>?+=KwJB!VA0_!)fG5grL{TjWkoE}lAK zT}+Nr{6uxhAGg131f2I2?wf@MGKy)Cu7y-2?6554|MsMscQKoIYK{?tb(@nUo5PJc z1q!mIgYQ4te%sM*AR<|COT+n6HDpw#nfp+pBmzcgNIyXRQXbgtps<4}Po*@=X1->U zA0QY%+4y_g6EX37v+26x%W4~71+PN#i-vEzaW3kOD<{m3a_*>k&dBOqFBt^kUWszA z?W9*#T|+66uNE0!sRDDAPMdstrwsM=qJb8gj2ee;EWz2=HY# z_~7K>>P8aG_5v0|n(!)}3?ueV9?f9fsD7)YU8-Ewk6&0<)TBNy0yZ#tbS2|><0G`n z*BK5bb|ZUzUpYm;;-!cW;4179$L&YaSp|DG-795nC~nZU;b6@J_mMp|OWR_o!q4$R zV33(_t!wyH#5u5vRea@BYV=-4)86>pIM0Mx73pb)hKx_c6t!KO6@s4MC&(L^ ztR&DBm1j4YcYS; zUZsk}M*|!5G!b>AzUkHixJ%#IDl)_~8VUndI|U8CG^WLa!>$;2_Ylhk$Rbe~yD6ea zLZP8Va%TbVxczY?>huzv2tmHhw*4r(&+TCox!&`>+S@PzLTo4t-Aol|SK3U#rYA9& z5q*R34TsmYa?`6oJc9Fal-bQ5ixPK{zp|c6r|?}umcDL713?3akLAU1KGk=qS`F6T z)gvLgXaSH1m)f$}z+ail9{i?+;>A+)Ugo>2*#Ndz29Dr@atpSD2+d7Afe>{TAxLR@ zdM^M4J{WtiIQR>We1h&=8VF`o$)rs$`{c|3>ZjfEUfAV{PlKqL)P^f$PRG^| zE{`0RgL|V_^ORRrfkX}|QsI3t_v)>jr3TCMd_4UTf-osrR_wyXei_e=9J>0Dr8gdO zof)Vychzoi2FW@|;c zmEqmYe_YxcftIl@B9M)^HiXY!LmZv%E<~hYOMWa#=$J5df~oU4oDX3u0@Nrp-;U6Z zl^7<7f0*vDF1<|S+2@(B%eAlLW(lVRCYNTVvZ6ILWj!M^;XY^U8q)rzKSQ5x)-GF~ z+O7N}@!v_gMD_Nya|7QLG3^R3DqXN9wAfw0N%2j1E~o`d`1YV&J;BiwS1aa000IO8 z@;eNS2}l%ddz;a#dale<_(x!S;iR5q5NM}!`VVtD>_By%>B_>_mwNKA#!btB;q7rv z64()XI~l$BqKCf7UQKSv=M(I0d~|mhRyO6-E&x5j^5DR|1wJ#)@zw-I=C7b6%vcAm z`|}8S{ZWHqe$GGKo}npxC6SG29m3jVoTj|cG^x@IH{502IzA@A64qvuF+{gdqE@%VSfO&N?j6}ICjj2wk2 zcxH|kbPm)}o(W7WAK{?46-2a7DskRB08IzJ7%RNSwp@7-zlfHn+G!gBVmcJP3=o!+ zePchzNt~m9SMhEDmGSPWGQ{~r7N4mrie!Pz0Z9eAsSnxGyT9agA}dSO%WxX;#ktbK zWrTYpg5bkgPT+r2u(Y|F;K|A`g!6g%B+{xA%Ur~Z9s$Jj7S+m{AM!1MX zNG+XkR<_3(tZu;$R16{yX9Q=?(3)j)?iaglKo{X1duzGg(;1&v|Bh@7F-#(*BW~Hd z-87$Bd&59L{l*B4fD&;W{;@XuzDYvJb?C>F>ZJS~W1m|>*i2A2L;=IVGxMbUrIh|$ zA#l#WL%cfXf8ziCeH2Wm6pdWr3K;7oO=)p^&*2Z0l$`S7b|fD7rYJ#pWqM*lO}f4T zb%?cgiXKZuQVkevaGaVzGkU`p&%x-COD8nGUiLr1_*$0ECrUDVJ!OmL(Aly987R)r zSsXCJOml&NoQt6$?xzQ7#TxtR_<5;Oj*m=ll!&bMxDpxN+A_4>`rb!o&|VxLuFg!# z+y@D_@GJw)4VTbR#xQc0@y_oG{*ISNPkD*PKBD`CMCJ#;$+Rw%r$(Z})c>3^wL@6+T1SYR8=Ju?9(Lx`0jsPW`M;ioJyqUo#!tI5YXl}#HDX6Ay8*xjg6uGYnO|_j2uf{mmPDztI!Z=s;-K5~xx7zXNP8SG0P?+v6s!aPxQ%zR&4w@ZcmD*uq@8Z9$}j zh5@;SYOPy0K|wr3&`z)yx@N?9$iyjeFz4f!VaX^W2oxxnC&s+!*|RKJvl5U`9*ICx zkKHyh=kYEJ*x3_fq@**4*vDHc)R|pf{(d{~Ql5IMxI{`Da?Sl#oh+~hbzd1Bs@~p1 z!rT+mUc{0Y`#Mxr!Nu2;V$}khyORYJ`0;#RgFaw}ay+nJS!o+q7Aekmz{ipWqy!JW z_|;5{@>V42p1GhR^{nmq6|daN^y=mOzAHcJ>(0xtu{dZLxd+TF zf2z~B*lL1V)eW=VBN9CuxcmB1PMOul0(5Z}aBAGXz*8sFQ@qT-toy8E+NbG$E6EM= ziI=S|9&AF#rqlEqPyh)tT;>S?LS-4Ai{l5^DqMJnbc8l3Vo@LI)CeY@njt9gK31Yt zOu3CmwQ1W9zq6~q#{!Po`Zkq&iQ#x=TD(W($sc$dUq%uoI*`~Y{?0n{qRfwQE6eVj zCU@8n)IF%5=HMWKaYOVPMh7ti$<%=cJgRQ|D)9x{hCT36i<-8Q)GxCVG%sHxd@1OT zPL#l>o(PC3s%4;G3n?X)20x;>Z<8*q%#jJXEQELq66{S~JWp=T;Vxv+)NJ;M#{>QFq9sbJHO*QGTIsq@L=LXz++Am3 zw<)id=dBE!8A^nMcdqkamj=WRnE226*wp;XfI{2X{> zI0(ah^;O+NPu)1ih!i@rF8d~i%xV~Ms#i-McA%-YR8e~~(?j0OMaC@sG>;vJeO-GA zMO5g-cjvI8xCaEL7UcA0^9|xmNXYA`h&@9b(&QC90LV7=6?kOpb<7`ZkWT# zuPT3x>wHnSKg)1qq5v0?AS0Lub2QLY+S5*_zTisu)Q_5A)E!q0%R3|2(hdbGY(}gXJ*O-y$_vu5#?DGrw@{V{1%1SL<|0xC7wNXY0mj^ z{6L<_j(K7XfU!dhzHrizookI4Hkj@i#@skPg!^Nx=ji@KEg?>s81bka4lt(GzJ4N= zyL!Ky*CNe!esCNjGzJ4j!mz4#i|0aEzqEn07QIM+Z0R$eq}Cx#kmeZmH%6aGP#!2D@*-CU-ITTRpNZR56amCt3TQlX+D3DggLT zwm7AkjBO@=A9*{I5PUhg<+ILqC#b0$Z(0KJI>~r+j7~!xF4#WXk6`gn&gb7~4Uybe zL4lm^sv#5s>#!Hde<7JTyn;We@(=SkdT!1{HFg5R;iVoLO~ht5@#Nm_qJ? z^aY}mpt+-^sTpSieyx3I@Qto=nAF}Djo(>B>VuD$#+QM*_r2wC`#d+Q*mGH!10TsD zYN2ClMcRAInk49jTSdr$_@OQZ22Tk%a9*&0jO@R`ni-1X+YDDEocA>{%VKaVjS^Yd zkZpOE;4ua1ozcf`iC~O72Qaskyzd1Quq!p&$z2O5ha`}}9sfI$e^8X1FLmL;V#&=Y z%*|Z!2_#!JqYCNkWg)44ackcpK*%|u?&NJ}kDYIt(z!LgZJHLMADyxFc60CDa8fhB zq?4gYgkdUVsay%6&8#QI#+X&|%xiOXo0+l5+x%}o1w#S0+-`*7W}vbzs%GIX&6=*H z&Dj7{YEXGCS-D^=7h{W9rs0-L#F$T^RKGJ+cy&xBo3>}3n#KM7lIW_r(Wr&W0za6F zAW=ZG4EUo7q|Rm1Z~w9ERo1>RYsr`Tm#R*1)YEP)q+uW3gCa1ZIc44lqb@1q9laj4 zM)wx3JY_=!}+x^;D88LZPNF89AjCI934Z#bCw8X7zYp+)4R4&? z`1D=BE82$U#zysz6dC{3+AP0^D0v#=4j-9HiKuL^Q488fj^#d9Oolk`mtJLk*d~*m zXVSK_aV(8jw&0Zo{X?U?DQ7Llb;P8aa0YGhvHcfXDMg^nABN@8$BE#)|FhvtBJE}p zws14S8LJ!uXxNcBogkEiT{_Ur(#J2MCz9f-oveBzo|^h5v$qRAPe2_)#OqtbxUGe? z>b#iZ4qLFSpVwkaRgf##Z}QN+a-ETL8QujGL2yWO*iYh4Yx6FeV(a+9rqH_{N~>Jb z8KD*736FfGDP$xYPknij9orW3n;)kJ<6|AZPY^-kzEV{2dyB2{Vg8xjb=Q0RYEu{e z0Blpdn0#9c1cM^xO415R%px>3|pB^e7TGNhsA^ZgMHR#nc9AIfGe&D-@r$Udy_5I(@(vw zN_4Y>B_{I9d)k_B0eyYL-Ok8JuACPyabf)WeA=$GG!gyUkcV~u+|Blx!>@Yg}1V+1o4UtlO z1RU;^4?pGfvC?qF3}xN)lu#vD(|ezXT}=Ydi-p0#8eTzBZFjzcSjQN7|kzq zW?onxDJGs-Rb?VcQO08~g7%MuF|8x<1`o0n(HhO|NB3s~ z{Ek}6x||Y2N`G43v7c=qzN0-yOl1q*HnYu%i>OF`lcC=sS)Ob^85mA2;qzrsuNzr< zV;|V&+<8XTMCf@P0ODxP z6~!3@vC;+z4Ps9Z$1L{bfZSy;b6}v|*$gZq*R#aPQO6WK(7Pl5{t3>6VsoUD+FbNG z8R-h|i+@)utqf#AEjx`49$N4h#gLr%y1GKgoJ_g|?A@vQCIeB{uG9|qGZ6V6&sy4A z5IJ1gtfw+ZUL9i^KS4a2nPl9!>`0=nsB+Yxfu=&_ThTxeWi>NN^Pa>O>hj1xoN~C0 zU{kaU${y0kY9$kAmPIbi^tM$gE3KC+Q%$kc6LPUv;gK%pi2m2+wyv=zhl%zE%$TW= zM8s{mtgIN4JSEQKd--z*C%7Xy`t#~BLkED8xDuHZr>uf%GIRH|x*0n-42b)ihN?vj zKj5%P#G5OWdUw^uHHro!iqvDLW<{%W9ZYj_^(}jOEo^c+5{ax?rEmW-s^K&Lqdh;* zpDxoLHD!>(0+-Qq8Ppn{3dGtlW*=wv?_z!N(Crq07M@bzwx6v0V57sP32Z;VtswK5 zt-7ZQUoW1CTlVf*!R)ArgDeV2eAUNj=D#V)k=+0zKS|`u5c=<58O9lV+4MO<=(kr{ zf$3}`7G{?aKW@#*O#n0LuWP4~bM6z3;z|Ded_v9-iFmCrnrFwz4w9bb%0vRTVLLta zg~^~r=n;ZC@QozQ9d4Xger?$eWcgvkD0?7TURI6%E^sCK-enhY@)&4zYflpIMHuiZ z#1mf4$r)x*?^+RT-~j|Kw3mEdLY<}scyvBiX+Q!H_9qz0h_(fzgg#*S`rq1fnFQ_Y z+C`<@Q&hl1jG?Hmphk)!gVGVRc6^6J#u&x3Z#weG_D3PxZS-X*q?=j5#7bVsxAtlf z!qlK$?BZn<04j-{6t!q)pdWQpI-IT$QC{|dC~&#V&au%-Mcs9gmF?t3@GHYA*~t1c zqa9au^-MkxmZ?&sPm+9U?L9TYLC?8vb5~z7kpV&+l$!{sH-v6Qy^JO@{0n=fxcU{a7UmG_)n+xBRN-(Ix3BdD}as(lM0G&|ke&cXsqC zfE(elFM7CRQCz*y4sMd*P!;^$1N&;M^I&$(sppmF0(A^gp$~oTrx!Da0~fN!Gho!- zI($(B07(>c)?{1Ax$&wRXwU$#U~P^-Q%c4Q?Z-Lt!ZI#wfRmFgkE7xaK!+TKYbN6x zep+B@+b^iz{zFN=?h5&O@f&psEVYgdQUpWXO#b%4(oX!hx_-e%w9DviZwZQ^{W=wl zD~W~%eLAlmL<9`SV$mQ`-w33l{u_V2a-f;ha5vJGYeTPQrr-Q zihNXx_yn}- zMYf|c!`GE*bs9m2!*qG_w1|`dJUV#kmG&8;V-$SNg>YFE#nm6+zV2wE{X35r+$wH_ z!<8hM?UO^{daImLh^jP`wK5j4cSFK7KXZTd=j-SOs#L-XvQ8$zdPm@BaifnT;xN_L z>qE@MZYFD@#meLFo|KWnc(@Q56P8>vwD8#JH{v$e6r+t8M`l_-?WVMrP1~BX@`_h!|GM>3hf$b)Uj#t11L}Y5+3!dc5)^z zx(sCAkJ)EAgek8mZLwty>TBd>eP<)|y5jXuB`4CI;y^_0UYJOuJ(;xl#qA5~+%$#{ zM1Y6SMm9Y9OX4~!Yju8vPSv`oI@*~eM6|LNjqhEH;zK!GM&5j~p&g`+uBFk1_c^jT z)5ckM46#r9g&*rU@+PK>oEd}0BiblM728WEt9|7{rW96UxqnyC7mXpC?9SQ0PGmwY zJp`jfC+B>Lm(W zzI@~Tr@zB`r95px8KL>jENf#HYra_8xgn}9aASNAL0 zl|wjJO^OvHcGd_E1_QE62E=KzAfq=5ivXahI3*nER_wbg56`^&UhLGJZS3;5`<>`i z?(DiTjgEcGWatCUivcQbxC`ybjs(Biug@#(xQXPP*Rn?j+AZYwB(Uw|U*jXXQmb%j zNq+kFjB_vce*}qiy6h2LVf_zgDmvIHa)I--Jt5EI_1|<4cp5ycy}reEE1P zeBqn8beO_KlbZc2~vtyUwy%1 zY7NedcgT=r@5&?oi6entXd-y0Y`OnOXsg#k+RH3j0J`h>_HQ)XM;&yw-lh9=M^=d<;b9Tj%q|kd_P$rhqgY{%k!gB4g*L$2aD~9f|z4xa8+w zK<~DX8rr6J*S3S=SKj@iP-H^Uq#A4ULsSoyz)Rv<5H^u_Dl~2(*;neAqd-e&H*wL& z#LExyUAR~7C{&y51Rq^$gy*-~=Z{{;w7|Roq4JJE00IyH16JSP1r+Na?WE;&e{$ms z&k9DMjUOWNS!uj>xsPK9=5JmWs{~3=m#DL`U6epc);gBa#`4#0M{5I632hjYNFjg3 zKho6&8vJyr-6*U2?qc^F-LOU;Nh7p6XxN>UGlE$T;jhJ*ly)})XX*HqL8nQ)wy$1I z=Y>~QuNS}+F;h#!vs(^~oeXRw{=397`%iHOHh@{0gP}{7EGN>tVFCtJ=2t6#6-Con zgxUV=7-j)10-=saq4t5AIE;?)x{pXh=*c}iucHTG*IZrxKm}V0PYaO)-5xY^F_9HD zgR(xol=bui)-lOjJDWyndrgCMc#~E?AZzf@q!FTa3h1;#Z)ZTsn+6*&%Z!o&Gdocp z=^Y*Y=SCo)sT5UFU7)jlsKJhnpsb%{fLvil2+UZ8_07MvRq)h?$ILOUOY;8w=fcr| z$PF?|@w!%BXW(&Lxqva&%4?El8QvYxY>#|PpWS_&Pg%fO;dJ;ZEc46gcSL0F)1g=l z$GK{Zhn{6a56Re1D4f*m)o%XplYubbu>(qko>0yRLp+hb(?izVs;y|{h9D~h?H_D< z$!1Z;dJtP+V@|Q&#BK9fsNG!P>lfqE5R_DORH$%ql=)kP1St+%u?s=R1aDZS4^NA{ z@F)(`Q$f-Mc0XM^g^bwc+GhF-R7D7(@$A`?bUxdpuM6B~+#a}jqXwtNL&g{CELHjhXUYF2k$0U9S(LMJ z$*y;)?2E{JXtGFge}PmN86zb4o=v*f@~4#hU6>gF#U#;Fpx5~=o8W`5L>CmEX$HGv z=e%qyR~U+>yn#9ZI>^3M9dKTP>2i5SpbVDeRbd+%WtSP^Jmgw1`(yW+I8o`y3EtR) zj7m0i4qVP5As390XN3O4S&KF1+S6WPDE5$*T)LSiFzJ-C zV38l-cS=9j#A3jhPi(;T(k4`_oI0aHlWz;v=e9KlntH~>o+`9Ft!T3Zz7TjGzTx+~ zMlg(GW3zp#pu|Vv|H7XBsmkZ^5H4F4A+;}l*l~3Een3rI?0t_N*q?jNQp#;~kF0v2 zyHlM4Kawx#Y7<5&)bbP#zM5n~|lCjK1%9t{jL*J?(Ghaz8I;7_Jnv>%qzJQ$%qO z-Ag)5OtG`CY*sn@efp^{eWhC_i!k(2FTP9~P4-d{m8T5Q0-PE6&v2IC*LWp!y|oWq zdp6MTQ=AxlW}L~J?myBoEyp%|2clA$fh8`Pgy{$tt3TRO`Z}u9{{)lOL!-Ub9L$Hg zf$SNM%y$2{G+O2)Gb+Y#7-UhS(LZn$j{g)U`QK?47oXRCAV-*hAwn5r;vsU<5Xc$7 znAETpFaExCuM!y`(CHKKqixPKO1cauBpA}=A0ymWdsB(?67zcnBw{xGI^PRVclsUf ziPMme5IW5?HbSuU(97Xc3vY``Q406$F^00#tQJib)VDFqcJ}A(zpB`;esPFjRNq3gtmx|v!TDX~w*rA#k z))}i(fdRIN6GmR)uu#Phyj$cYFO>MN6m`2bv>hn0L z;;(M_aiCU3%?ijaGv$WLCO);Fr)D`M#4?9?NIFwKs>6HS^?!}Pxm`vg=9xezW^C{x ztKxNk9#;A(asyeG9%LS3gedYZ%=lxD>uHBOx=MC>S~V29=6_LNKLc%b?T2%&YE^yB z@zas2Te;(Q8tu_Ra}55!pcb(p$I{HJeLl^AMj}X{I|n$zagek)G_zT-aegR+MUXN% zV6V=&VT8x_t=I|D)?`&=T3bFaZGanYRWW2e$)?u8aOb_>D{HkQYBN!W4dw9sCbw3- zzFKTY{__Lb;!wKd?#FsG33tA}R)L!kFwX?RY8>E@Qw}x4o9*f}B+Dk;9D6hOCYE=d zk$Iz(e@}U>W~3?6T|5F)9Vn+6TA)tDLIxL5NP?_WY4Z!Js(vQt@5ZJiJo{op11ie` zhD$A^NyFu>ir7Rs42bY)jd;Ht<2EwbTJZDD|KTlGJv2#bD&RW?2Mb#iMNXZjXo6K1PQ&fc{pxI+)n2`~; zVrYGOtDU_BD_GcMMn-{C$DVvx{~kY{6$SeHAW0)7l9pWVagV>m&`y%$}I(6(a1= zdZm+cdkS-y4;vc_MS?@gYN=ur<+Y7G>BXtmn$sFtW=H`bti#-6v`WNHIdONT>U55% z4R)(AFLO0QP7;MTV042>7C>eQ;TxM{NiKQ-M?kp07Lk%!XdC=^8EhH#I&PfspC= zu#WNj%Jykl@Q6gT?cmMr$s9S-e~0H;Y81wEi)&%eI;?n{r*MYthXgD%Kau_tKi?!8 zM445x1@$uhKZU}Y{JmUtEb&fE&8Ho{kWRYuz>c-$?oNQQMJH1D$`GjF0xVZu(I4Dv zpayKv45Tm@mqRmx4a&|n#os)rhHx$NA-@DHaV!QEJQ$Y5MJ4UVvt`$lw-3)wlc0d6 z>>XT(N(PW$+43E+lbu#_!84#H$8sOidj+F`d;2#OPnV`TM!0wYvL%~c2uE#1{7U@r zl8y$ca&}p|tpQRG!H*sGR$JX8JZWi4au3B%hZ-;)lLsqfo7exvYEb=eqv&KE^Dw#% zDQ2Xl7zakH+kDd$`omr;!}~gQR_ewhe2v28@m9C!nXyERna1EQPW|DAr4TQEN`8J; zi&I!Me+jc%b(QU022@TyB-7sFDT^lA+{LF2Qw#f$aU9Y}J|HV*OkgOe6wNY~F(F4E z^qf#Q{@BU|tQq>Tb(fhO{CE5LT{PuYAM=v~qUDQ z8P>$V>nT>E1vVk3E}@)07t(3HQzK|D9;7*@fmR3{0n(ej$y!0b8h1rT*d5b;8a z9VY%qO(RYo5Wy&QEO!R(IEqf?pg)_acO^@AJ$ObZW6K1F<#F)3QRQB^%ngup0J@w! zNlD`8HWc&LV33{ZYgV)wyl5;3P`8~p7y!xo#}5pr#WD=K!>ZOdiYiW?FO0T=d~igh zqMQygy=9>Qi4eIRuV^|{33pf+n;L4kqboe)twl)!>Mp%GN$(0&+qIr8h!C`z*o0SV zr3dSy#(g0r4BQ3T)^lo{hgBh9&4ho3Q-}o`+vyBg8w3#epaDQ6ro($Ml+SU@urmIX zEj2srQQIHDZ|&+>yA>92ied6UlKb9XGG}*6cTX?clG?q`M1KJ@Z}9rEK6L-58v)%- z&zADx+o+ieY!M@f12Yte-51;Q@Epq(Cb-=so58W}GQ#J`us&n_I0j?y>V zZi4aTBzr|6KLfVdD(RMU=6fwYUlC`NoiZuBAc4j2KUcJ>hA2#e)!>Tgkr*P7V(_(K%A;Kjz2r2VD+yOxoSe5n-iT1}mDq2u6ZQtkBGy!XV=+DhW$mP|od(R(hTlmlT z^Z2!Ol!FL~l1Cn^z_^2Hw=X!9mecsaa9layX)@_XGh|yRxO<*iXig8a#TeTSZBHyL zx1;gkKG(BUCmp`iMV(6HYcjBO$KDHY^;3}TW^D~)vzQDp$DA<&1NM~9m77FxamsO7 z10gYPXNdQv9-ld$Wn4&YDZ&!73B&RJB>-|6&uM#`jwecTk`9M53spjg+XS?zm4{%2 zjNYrAJa8Hh5m{r{SF!A&z&n$97<5&shd5&6FIER=!V;cO0}sY3->CZW&dCv*TmnID zIaKGL{lXG7^O4<_FcTvO8KCV;;#Xf|>7zs((MZFp*8SsGdWeAi{6OZEXFlUcM>*e>luF3px#SDUkTA>|t? z9MgIff#FJxcwZ{&f<)^L-8@71vz7{)=ubW$aVTaEYrd`tVh%Sl?EeO`4@ zNOUM}xkkjK;@+dqc0>S%moofufm%E^5PnfG9WpFG*egIW6yG#^g8e0A2(ASKAA70F zJMf5JE``3KNFc~Prma21-0p_cGH^4*{vUyy2*tZos`$uw2eya)K`9kQud!Ll_W8nIUu}GPUxwYIe^w<6N4+M$(US*SLS>)f)M?&K7{xSK9OsJGKbR)*Iuopu{=w=x<7{i& zs>{I5YB`1Ssr#gSwyZ3NFgv{+?n~jS)xj@F^1eNJ=U~31zzoDXlu;f>Y}s7;y%N?5K3@awJ4~T%pJUT< zgfy$$27SAm9C%lI!fYx@j_srX#K&YuZ@XgdT2w0QVek3Z zMX}%qBQ49UF{$9eD^9Bvi-Mo@Tb3;yQOpnr;k>N;6cOj!#9_T1Ab=!!j8*RV=PLgx znQ9fTCcpz6`c_`H zXJ&q_!`Ed6G)c#j$?-QD@JiN{&|BYoE+ID$(s%5(6!wH*c;W55{Pr`D7sTZ)5zwd~ zu9`sazqx+1R?v z^>-fj_D0cG8Cm#S$`TNv67Hg83Vv?c`)YJ#1A@a$W?$j-KA5|{bugOhej4&X%MjP& zqa&t&)u#BGNp6#bW@+w_hi+}~g~Kz7Owp-C(t)KW!`XB*Y^lkhW=f@@RQ>U6259HZ zP@$uu^gaJQ`fi+sAX4rzZ(R>7fqp@AD)zO{L1f7qFfa;2pE1?6PUd!#Bh<-RAtXe$9^CaWf^9BrPQ7Wi}MtDJ?}Bh5>CQptDvn(n}lFUYx>_u zqHYk(b&UomFUf>43jftV-AIa-^EfCGz;KhA_?4i%>+E`E+#O94M-!8<7BbxYffFU zN02CRz`#+VAS4L^+!v?TyP^3*DUpVYF$!i~Qf~v6TY@b29Q+VB#j6Ko7vPtormF$4 z@x7ME5A}!P3}YvVu`YL@-@#KxT!3BPLa#@_>boFCIZ`!hUvo&-7?d;Fs3rn)7O=tU zes#a9+S`%5N?bz^wk%wJ??OAA0LVGkrfg>5+I7@Tq}q*}O4{_0E&a^bQyo0I=@eiR z{3Ne;Ms{lRSwUHO_&T`n9bRS2OJ&!>$h9zq{;NpHbA{D+#0RmtbQPTUS}i2PYv5MH z>+x>lx`+%*oVtnZCLV1q@;8|A$K0HiR5NvT4SrtDj0-~5Qk2Z{v-*`hMbILrKjL;D zHl$U)R&&b0q3z&ji1W?09P36y3V7`Kl)&2^CYqNY%`>vipoMeT_nB(311FscworH5 zftRm@EUzoLqI26EFjVdm^*Kus0$zO$u z1kGwBWU`ge{h?o)0yUK@h={skXv+suLe~Q3(vFvuk{O!9Fx*tDaJqU0un`3q#=I+5W6g;-z5Wgd{6>#Q>B^1Sop$i}?g=txq`T?Tp}8 zHU{QAOynwVFFf$w)-l{-Fo=r{R7SyQlTn7ZPAdWSZfu?B&y&%IpGln=e-$Hm0FjT3 zS7gb3O^8Q>5?0VOcoaL~DIX!UFLtDvQzb~n7neLUbYhsgn|m2!oS)}qvcr-^#HfUn zym~?F`uEC!oUH~1H{pnc1ytBo8)ae`7w=O5d3N2(Y}I>ij!+=6pz=_vTkRIWrwiHU z#xQ4FJs;89l#00ogXN11acwS(;WfLfr%03V z8Yi#s?bQOl>2}&(kW5O#mknF5G-@y5MVluM38XbY&tPF;dU-xa;T;%_6(&E6p8L!- zit2DbfEwter@cvwrbl|#+BAnT4YLuNEwy(- z9@ARH5pbpEIIM}&|4nS~$L-u4!avbxO}eOn?S6Q`eV0ehp)TU7-2QJy- zav8=pn4FeNdUhOqZZI%@0x%TmT$PW63VX(gFBeZPInp{HTFITQqz*{!D4Cro!k9ML zhfl`ZYt*hIwi)vz`+9cDz^=eTM7|kHSjzkC{p0lv#76gVOJit1Iy>=7QCL`T!VZgh zlGz>U%0o{o`cAXemL{%Jx7hkRuQf5ijUW;^m;+XVwS${6M1rM3+I>FOOu)QcCNgo= zXeajpL_Ue`^%Jp znX;Mv_t4SORT^CatNivdWekTucNrf}EM(0^Qkm}$<~4cnFsIp)fI=0kUd%BCNsEFL zMP~ad(5;SFN-4Gw;DtHQ?yZvl8XP{>A%v=;Eca8%_kg;AEZTc=)J|o+U16QHV2xOW z&v?U#LC9!S(qfqSc-2n54d6i`aw$PmEFVhYWpevplm_QSnP{~|FcO7{ZX{CEKgj+U z>#uzptj$!`#>18t^6bNYrQwRD-l`YKh+83jHXrFELplr&#gRO&-8L=3T;^s|24x%? z*j$%$Sil{FwX>Kp46)H!k^d=0v%|cW<#hsf-0Jv+UV#gZ4=@YgvzMtSK#LFO^*1Gq z|39!=f1U{n&_3+;J{`m)f8N4R2)(S}V|xE!MMjMN`8JDhRU&q@c>3{aQw@5l+37yb zmMXc~pGUDNJGH@!1LKRi8&nrdcA{iEbsBhlq3D)Io=A_^#klEd%sGhGq|s}Wx%a40 zm4&v7kTFKc%*b&9K5$Nj>LPn!4WcRTR^}T2wCior4N%vqDq3#tYfq%~OID?Q&i9*c zG+zf7&v6dm8R=>&y-xTL|A|L)!^klZ2t9-2uC1T%%f}0z53pQSf z9LlA2e{hJDgeAi-PjEZ7!6#q|4;34;9>H=Kb#teZmw}f|FT87rga$({ynTmVsYXE$ zA{7ZD3!0Ab4m-QfQ&7$^v+O`s)^cQ<6&+|_OV|hN!`0F#ameW(k+8Io73;fvWwGq9$M_Lb7>vRXCHI4nMt;a8vN}=YZk624 zgIsb;kUJYmJhQMl;;8~N|BN74hXCTICWNpNpcHqH6TP-_ z1Od=x6N_=SptlHhsj2z1Kv8l#)lAq|i0_9_%x&_ZNSV~_xIQf7OXwwut~&e88U=A*a^zDu9oEmgti6ckApjveWAvpp3eUa>tGo$Qk3+3~#Oc=rC)C_7c>;e2Scpt;-O?Ohl z&&NY+zn|;CFvj&-s{RXrbMzGhoT3Bgf)#8-Aw(9b{_mfoiqC)PNwj9Y7n@ilI1M)< zNE$3g-zs?pwRXj!U$o?b;d{+e!loqh}7`k=qJ5BgFGA|MbRXf=ASw zjgT=gaIQSAQ?=%Nl&kD>ZR`#lS%HZV#s0nu&xNJFD6Uma9(JRkByGo_;rN`~GoYic zl2e@r-__8vJ46n0PYpn>dQAwrtLV8tPE!{!2^4d7PoUoh_S2Xzpczt|K@bm8Yn+{f z{zMiEq|k_uPrQ{btty#$v3)&)&4v9a-$|iCA#6=B#Dcp4p*Z6bk-uvBX+uYw%5?i{ z(r1?V$Hu{#97j36mW4g<e4{OOdbiJhTiXejYpJ9zDa_8;aDUR8|nXqQi zu%7+@Zu?7o81ORwpyBcK(yTe3%aTyae?QrA`9Irp;GqaoSVcVDu6E_IQgC|G=v@MRlHu;v@fG>{u8e1tm#7$Nth)93uJ|C7W?H!hujAXH z{shOu`yMiTC?27;@AxXR8W^3Vq{2&Qcf0qak`|D{T#5Wmws?4sq0@vg!gVa#8?)zx zgb!M7QHl?g5EsYx3Y2Q2)VmoB7!5iWf#7XY1iBE)r?hu+m*DlZ1L#s^yV7l>)1tY3 z<~yh~z2fk33r4SDgDfk;faFrbZIzNC@yfia`JVovfNz`ynz`P|Ja4GpzaT5%%dVCU z|4rl4#hl_#@nB80{5C&{Dqa6>^#HPu7JzdkPe_k7Rv6fUS_4*!_?&w>@i#^YThAFt z$pAQXGHPFIs^*sLCb zrUQMtKZK|qL>%fR{Z8SIT+sy_HPzOwUS-H@eo`=l_iU2($fL}&dpSLhz~i_nNI!kq zFu?dahN_fLr@*J#rwJlIt2S1?M`vFvk6_jiGBN*wCCRPF97M#w5+FQS=1Z-|lB9rK z_Ce?=m&WF+hzrs#y%kSV;p~IMNM|%CMUQfHxQ&=ka9cUTdRjFgjlWl4yj+3GM7+cj zJ_B7Q_Q}W(jzS@oDj*_TN1og(LNZDmX(awcToj_CRF8{*je>WgRda^={%1Tef>-d4Tji$n1EXdxY z+vquG^hUX*P{K$lAuTgzc%0t81{f@(V$D6`nX;j8qgt8RzL;CPCAK033G;O!TtW6C zj`}$^H7*p-_vP}5d36*`+A^MuKka)ZOb6HCuQa{mP5xH_#(eA}~W^7fUb8(QH(q3gI&pbqvG5vhI~x2%0)=~_(TV#-0f z+Z6YHq8(k178FIuMbXZb@}>w!fP5Ids1sYp36^voIeh}lYJ0MUd-40E;_r(Sn3&}G zVYG%JitXl>z2@hjbJH9+&O(?2Y6GI@Rrz!sgz%w*aQ<=-EYhU^i~>=ri-Rq{B)C_O zow5ix-{40xFSR}M{xqVVXouPjV$2>~j@D{EIB&-vS5)zr)jMWO>DZV*6VGv~i^ z=8X57A-PC;zW586joEPHS_*^HS@L0Tnr_09;srY%bXrlF)~;a0E?+4l+9izR>fV>o z&J)&#E|?byjA@zQ5^NQggR^!{1gaL)PuQ>0XMDhwOPl3)}QkYb2_e^w#;`=)38W$Ea~MI zdhwNge5mxesa1!6q-8c%ez^6}YQ`!dk>ay&1@X|>9mpR&VEDYz6(crIfdmWLJ(_v6 zDd<@eYGruF=bmcqRez&^2;%}cnFz(^1`(mk-`i53Z`!B=X`Umd38u_Z3x1@acd`du z*vsK*Dh9_x){aH10HRO^73vWbyT_NsNVJJ)zGcj)uPTvD=;u{SR`QL#EXLb8y9}rV zpqYS8^?5-Q2m*Rzn;*UJZJ&EFY4)r|k2$)3gQ0W}WQ==*uqZI=*_VQY11) zn4E9cL*Z74Ht)==Ym2*8t?=+GssRzc9`omcF|G#oF0717^IthxmY`Wr)O{DNS zl1KI7*~@`0|2Jd|g;PDO`7psV_k0pXDk5S5m5#`88O`~?Lj{(7(EHR5ux7w0=c6JV z8`+plqK$vahLthw5J4o*^DxUK)=Xx6;5(nbwPw(V?7h44;||Kc4sTSNNu;QbuS4%K z=yRQG0IcD^YUWp97B++|j9A=>zYO*<#&ny{pnB@Dy~ffIV*FNWShHzZmM<7ckSEuC z8S&3Hb@xe}&H643@}?2kKNOwkNH>y;eLiKW5WsF2BTyBm==yn-S(E&cFDsH182}+Z zV~+@A8Wm`*o{R(jcrLnw9duGgcJeOElX%32Rh7c9^%>-c^s_DWHF&5u+tV3L_1(yO zjPtsws%fgKQQvec&69ADMdFKgP-IVU52`}ajF|fmbsHgCK-DY#|@^`bJ6Yt0nD0q^E-7sK-MxUNeAR-zF z+gm2C((TE}TnxnV(WRyLHWug;NJz!YCUWo99Hne@@{@vB-KI&!Yuy?ZU0H!GUx{CI z-Kpkgn8b3xA2DTs&1J4R7j=UpSSzD)--FShFVQu;D#fS98unma3{y|g2UeSe`aFmO z@Pj@NU|Y*X$&Kocqtpf~tW$#r1|0)clS zfYW>udkQ`_&p*Whw$O>Ke|F)tr8>`d65)l(tOyKgtIgUk(b&=_Y+up(9 zbbrc%IhhwE71xa}!_=|K{%P8&JjZ%eC*~U^W-n9Qyg(#R1j3s?jv-d6g*qJr61n-U&p~K#g)F2b- zqoH-ay4SlqJ@Des3AElV0*vc;v%o-H*_bW&>dd^_n+crKeh>q4<8T@eO4KU~p}$y_ z!<-80$ge~8A+2hc^FD0)i0(0%@2LQFD))FogySR+wnQ6(oTgA@W(GKVT^CDIbWkWd0cE#`MqK3rE1^=$M|jxh8N?6$oj zEOY){ethdKg6OPQ@UXopG;@n_QXjRkR`}r>F+UfYSFJ>tC;9^g25Z3u!`*LblF<8L z`S^d2n<$1J@I=mB?Ios*ROTxLgB#-pkdQDjU=$pp6thBDFWNJIx-(ErjC{9{K6; z>J!!6o~7vL_4MRT>LoGwaV0XkiwiTZ_;dM}FuF0{qosyBigohxH^A}NrO9;#KMOSbmFRazI6;J>v z_OwwdKwoLeWIy6*XBMQ4`R#!nO6@k_H#`Wz)}FCFE-^~>5`=jr*)s&ssdM^LjAvSM zWD7Flr@i?k&4yPclc{vOL}I(?>i-iUhk*BVyJwsEg-k*Px9Af+IgnecsyKKa$Vjmd ze_A=a{54{Sun)){%}nsvWuwTBRjs4$CBA#BYuh)gYVvU(jg4a6GlgLt!1|TujG^_% z5e)z~)k&WSWxVyEuiemE@9W4{5R<>W*`Vz>B zC+N@O1ElA+UpXa=RHEGq97w+ylRS@Nk?!%*Q@ho;tvk+}1FH>~{QwXgi4+%ScuZW$ zjX{PmZhirs4$K*!lFILFRJDWu|ESIY$Ol_<=j)I%INl;qFE5tvGvkmJKXBED?2o5P z!mpb^ai0+ts8QFC&wGhEJRA-zQD!05rC4gs6sIz=%BlW5+Qr{EZ1UN8Di=g*#P(Ty zM|05Mc)s#XRAaQ-NS%B**(M@}jRV}O%dY@1#(V}guhn9Kgz+YxUE8cKx4uf4-Q_!fjuknZZ6~;xr-EXor4fRzz6h)m^IpX1!{-> z(W^5iK*smO;bi?P(t9f5DLZ73!<=-M5lbecm=blm3$m$bqDSCck2!K}&tq(+=R^k` zqgY%)BI<`FjqPpZyl3ql@F@xQ=nQ{Kzq%zPJ4gctBVSdR;ZN6u$BZ3tlJzXgT%s01 zP?hF~(x0Bgx`t)wxVR)bjZup*n*hph ziasZ@9g89OHWpE`n(BAeB+tZ)5z#A();g7segF(#Eh!{ef!fXd>%(jU`JqS%Vw12a16@L0-~+lTYmNnu))R%&?1+;WH``ctP6)1+O#RM37w0A zxtz%H&EkN~*%bLeVQYy0xlQceD2Vrc`)U};Oria2N~(gI+$cQWxqtknrh~O6__2QQ!7~^KZc{n!x+GW@SBrhHxH105uI#7W_d(<( z19&!b4_*W}Ot|~D{>}E|NGFFii4$C{TgjI(l+Idj-ij6_0a^A`4|t#5TN=|}Qp~+l zAsS@RhPl3fq%yaU#wjLi6%T+)qqmNZw!%q2x$hRlHe%C}?mu(6dfL9XGbqj_l!)Q) ztS^X>dB?wuz;*Rv^iu=#U$usNs=TL`Ii$ONm4P|Z8 zHwJHnO1EZrbcjZ_JQb8TLUfVRz4k4c)+gM_=cXxxTtLj{4vg+)+IENAU94RQ+ugB- zFqO&=cmqd|(x`F?)PgAZSg=3xHqlCyvD;8Ze0)Nu8t7ik8XYV2YXCFUxpWMSrrvW_ z?kHYvVRjR~O?JhNFUYd~@JiEG;TiT%m={*wAdvxl?Ak$rtK`^4dh)2X*O}~qV4wr) z)sBwL8wi=%qJ451Zc%?P27Jycua4A{drudlpmAECKJJ-I0g3DT3+D&q zX?%QjG?3T4GL7(Alpli4UaDsD2Hjh_f2_xmFSIAITJ)7rDiA~l$L15|QcuwiZbqpx zrlCvkVcEvf@}Xz>0_?U?C7e&C};heDp z)m)Vl(3DoXH!>`oE@nSXK8?Q$TpgCrn;4)iaN3{$ zk=)I1&3u3?10SR`K1da+t0>8GnWwfC;ibs6DZjeMHKeABd$A8;^uz7!T-Bf%X)qCi zx{;721R3DEzCK%$ubkbPYQ#UopuYmC=vAq15~Cv|b9Ar58m(-9ABcDb zoS$JqK*ee;aw{M-@Wobw13q~7b%Eq6oFnyn4A3Wy91#0-2b*x!wij!B!+B16kT@wa zoCaRfhWuIM%?!Vl*01X;p(&~1-(=;~I`!$^`(g2gov`L@=-+FsCZk{QCmOP$>V~w; z3_4bxO>|uJVJm_=YIxKY=a`wH!uGc+1%|0H`{N(%yod?jhL8J<-#Mk-tS`^VVH$nm(_h>&vKj5n`%Is&H7&p$@UpJ zO}t$g(k9Bd$RY?}Mzpx`A8^uh%x!6JYyQ=VlN}u;%+c$a9O8El2fX5^OIE?PWBVJ! zDl?tJg%h7p`}X0$w{RRvzww^JXa-W*9AN>+*GQgfNWcYmMZsINW@lT)5BcVvgsQ{j z9Z~i66sNg{SvkIpQ4^r@_<@o?8p$6?pVZ2ZC5B#-D1!!##;#10LF_Q*bOvI{)!CsZ z@JN1SpGj?V#1FSW(=ckFZocuB5G*Ef4+#Id*_i+E4R<_Pehy150sS|jsc(M79!0h$|LHD%kI9HN>>*z7 zE=kGU{gr_WrwCa+QPjW=0#-;rivxUd^PLFW?{0uv!7ih~_iM=^Ycdp(?npV%XPBaK z=Tb0up4n~lZ@EF(OJ$_dQ=|K%#%!s67x8VCoYTVv(P5nE{*@R?>!HdtqYV5(3o%)k z&8Oe}t@vFXfQo&K4U&aQM%g&wkiKe~M>~i5mrI(3j&Bxv;w*m|{4_qCWK#N6nxtJT ztcQS*){unT%~%I_8o8gsu*ESc!IIXeE-=e37{u@LPVlH}W}) z|Dg#)33zSCoILH%7`f1mQW;uazonGl0~S5CJ-uUA=x#0tw^!WP8eX%2dC;){d@7Qh zWt_VI_nN5>=lSO#f(R9uh~6{2ARWX4i-dq6Q`B4Qyr&{xt`$<8|17;wKbJl&GHwDf zvQRy{&~)Wj!`~WUxr34yFI4UKy<6w%{GIvH^Uwt<@~mLsV1uq3`O*yrB;U!!Ui@M- zb%=ON)k)z-mV-Pq>Kg(8)hqVu#npZa{A37OmkS9jNZtAj5l`bQ4GASg1Y<#lC+KM@Nu@UX-PstYiOzwL6!2LH2b=={Jb!MBFf))&*ANv#)6w36roYo_ZYJ zkj+|79Bqy-_Z}-6XC7o(hK33IzC)JOfN|n1gvM&amm3~UAD)@o&|exi@;gJRfC1R+n)%V$0lc z?gFBP8!OO~Z)u$FBFicm-#6njT|t{pGHZtXEpsCElv|f*1TvWMS!N*2RWsiFe3w~Y zS~MzVD9&>^vAtVQYbJhOWyu+c39kr4+R4KhkldVL?f?kOt&30IB*ykCs}6<#)eK$x z*AfwktjF52iswMK_Pe(a7F@weC-6vBy}l4nB~A|14*|?(7A6+n#*S1?y^tn$GrV}2 z@qF3h!cZQ{;>+~MS3Vh#lZvmudJ2WICRsE{A=GHcn)@}%w9=+~Ve#8n>6(A^Rjp;) zRA`(Ox#*kGr7)NregPuvaO#Ih`T9)Nw6HDfOzbL|P)rF9njy~Lz-GsV#L9DA)2}1p zqV|pf{Yy^SbYGFc#-oOt6q7L*h5H_Pi#Y7`Utn4X-wd7<=lEw-2bAaDuqDrCc!>Z8 zQ0Ma{;b((nkIoRyi2YL4-Gggl{8?0%s5$ppQ3rd9(^St%Okvh}^=^vnIW@?70>?dS z*w!6k!Pji_=kmiEBLA_zIXIlwvt7=`>I=^&fARcNWpU8B`C6u2I(w%w> zw`^VU6M}R?4>$u?K_H^Fjs`g1#Gr*EQzow<%?$af6$jG`$B$PGsdx@(RBH8RDW6`M z=h{n5KH$)%t9pc-AR#3MAsNPTJF;&E_7@HJd?#(LVB2+SnQ&DB!F;N{dMuOf8$1GM-d@D8%xvvj z;oZylWaz9NJ};nbg3UZyy{(C!P`1coGv_&sn^LOS3R7hRogQ|Zn7vJg)QcFp{m^gW z-EU@-WtFjd$AWnK03_Ycstv_+&it2u3$lG>G_BdZC;Iwu9v$78gXXRSkGS2WMQN2l zIH&$(3e@p2^JN*DQH|NeFYdL?|jyv*k&*UE!OE5w(&H;yA%EuAZPY zQ2~pt96K@RmZsPh{>_UkZ>?24y~%9l!Mb57NAVq-ej!HrIS_|<&@G&DG{ue7VKhcQ zaETGR;ysIb+z1E_$xbT{TpA*KDq_8b! z@u16!Sqd};v4@}29+M}Uu3Y+kMgetVIoLt@l;&SVFef|%Ecstl{HJo>wB1nbIm7NW zysW>MNkW2V;Qe$y?PXM=d`2&Vm69cgX~QaH{ItvO!X7)Y17<8LNu3}<2x^7_+Pk% zAWQiq$i7Xvd2RlVOoMMoF*VaR#?Al!L<~oSIcU-NsB>DSSmRv`?#Bh z%(*WXXf$Ofqcn$8gsrMC)ZHj^C+Q=NQd3j3sxKcKFBG`tK8sUzl#`(4n7AL#ta@U0 znZ{Yb{>;aJ3p>bc64;DH9+c~VYi1~#9rsqTey;|x1;Z4CpMGw=i*)%_-0dJKBz?HJ-Vs_H{CDJD%Zf9J zsJ&gUNLkDr6UK|8uCCvM-FR;&sU|hSTm*Q-7uRzIDx^cxO^;xZi5vlJBQ>4BQH-CC zf^wo>{5~hSe2YjU<8o|=_pjGFD|NUV7Ze1)B_p^l0Q7>k)_;aKlu%#RJLxMEw>@8C z?z_h8uO2kX-oLJ?x9t>=xg0EsDFeoJD}bcCmgFQU2$^_22y~&#m-{5Pf=>9#mx2kQ zO*3!{v_6DjOC)Aa7l;qDe5J;Y(UtK#{#vPoHlm&KW!R7^Ko!8yDc5RPMQ>`cOZBNj;Y1I%*2il$u%0s6mZ$FDP4Uy|O-0ZSTaDGr zX%9}2O4ipZBx}%f#{(9oBX=bZgwAo2{fFBXa*~mm_bT@PH3AapVi5)}oQ(ESo|wFy z=rSX2z1R4}1|UCvv_2VX`z(i3SIaj>TKvPGMl@SX!-mkJ5aotRI0yA~j2U=FQwOp( zF0wpKSo^81&UhloY)y}jWGIhFUrZV{PZ)*$Qj+yb@4DQ;n&W=q>F<$fO)`koO^jS- zJj4#A8GoC1j^AxXN1v#hBL=*QUo$D%HGOsr25T$nb&;4CQgPQB>IIeD4NeC=IXl~o+@unREILyxQIrpl|98=j+M^`~| zoSDibLIuj<0DMP~i{SNmh5wW5r|l5Ax`{CWZqj9J?}-6x4Fd!~l-TB^tJU$IJXnc- zRIfLmp_% zxq$F^Pv_#&q;o0v%Eh8Co6I2NRd@k1i;-AEd?t*ti26ACvEtz4V!kyl5+(d{1yMzlOKP6{uVw9#+DcUT{vwz)o%C zMHP&_?SdEN__7QV;sZ}ipOSZQ%BWPI4|Zi!8t~mApXdi3P=h$b_g)-AVTsq) zPlJUqucsj!OT5aHR9Q4A$T^&C*bvLMeE_`WF`EtCsf|lOoRkTtxlPtly&_EKqi*YlN{KJ^co75yu7z zt5t70(_;EEH25(hUUQ_&+@8f`C?Hno2RNfzV}S|Ca02`=VxuFsg#Up(t0KKBtki9< zXHEBYCLW*jv$8?iMSnyj`czj`N`MTvJ>1RO`Q9{FwRh*HmfE6+C;17~thGfxP{%V+ zVGmAYnvdXDB4Ky+e5&_(kjZP8XuMS4RfnKUL5yg*S~sxT7{Mq#({CH_q%_uYLZWP6 z(8KaG2pBmGm&tRqpFfdC1w6hy;>Y-RC(AACGpLH-Xnmxx%_A>oFIFm}O_$xPU#yVw zd}2QjMP7Ua=ts4+zA^}FzwL==1H8Vuq7TIT=e-z70Y;Z+%31WEf7}xjs(B8>UKZSL z96i#($FSkd8L68M_RRt}H;jFM6k(L{PhrpH{{ptEE}L2yy-WkJk`NqU$Fm*^yNjn- zYvDr8Kt`F*IT1bZxX{|-FA{L&E7l-w)Z6>Ft!6Ry^Y}#8Z)Nr4@H-rLL3MxR(%sn>sYVK3tM%fgC{h~R7Vhp|)yDGWbp4Q&VIqW`cz+Y<}ks>kQe z+qp14ix9OKer4NcUTZ|_+i6!QAzEIPC5OPtF<>)>>Av0s4a5%>G4bmHXJZsX`0?|$;|FQB#i46wecR_G8(o)a#HRsW?m}3gtHD+H!BI&2<^~B(l zPg&*K0%@vlXd?OGSpvo+s`-jZ&c^ur)>A3I&KV8PfZa+v+I&VW9GKPlI*UJ%Jt#K( zG`_s2eV;fr0yzowLBtAiN(_k6XgK?Q#Z6hxTz);bCavbIEvL-FZP_!HBXFzH87)9D zMe~d=Tn3>$0IsxbRh&&sO;m_YOPC{0ho(a}XNY?!qN(UZv#||8bB zD;BIxB!euQW^we|Q^${~j6vl{!9ED&aVKCQC`jCXlUoZ{Rmg|!*Haq;sz`jEnIpoe z_rl3OxtmnqRFx%{2*siT=WpRL;+uUY@f2(<2n1gfandbPll7nV4IYk&&QcJ9Cth}SJh=p+5dR#3AdYx7CkGzwuJ}t z4;D=($|psmw*a^sMGbu}TBd_@g!Fqwc)~Nyuj?v}#n7}iG&ScMHK#VzB+rT-!u7YaNXkcVR%wG4yOjRY;35n({zPb~Vpm+k!{vFNbkn~_!k3(imQExGJ} z_m>nSrMvtxSm1QS5r6yIY)(F_0vrgA;?fnSL9w%?0`GVBPB0!NP2=f z#yXojaz1b!c>+GZ&NXt8^NZ&_y69&$y$aL?ZI2=xC{L|t)e};{t_ zcola5mV%i@G!`ewN<*mGkX4T6>kaA9U9zJOJs%k=XaF>iPbOu!ln$HoLt{T{KTQuI zzC4V&UvlG8lN&QN*=!xD$nKU^#{udga{ub*4-YVbv(MWVnUF;4wO7+eImnG>mfJHqL{_SQL)5x%Y*6`2RZroE!Z0*=cF6j9e7>B< zz?l~cO$dc-(ArM-%=tiB{Am!BHf;ys>yBE)Hzd<)tDSS;h2K4ciRpW;zO||OETMME z1WT0p{AIUye!zKhfzx6@L4{*CV=RTNV^gK-9yqg0Ne3) zuNOSUjukgAagsZd?wkdXy5v)YJDI%+2DB6aI!qF+*$J#oyCe$qgsWSw_4#i8*GDYS zoWPp^F+k40wK3*&L)&2Czz;=J=?*d!Xj4JTVIf4Xz&He}ZP>Vla$&2*$o*8dAhV11kL6`M$LdppS{;sl6w#Jmb~1kjI{rL@D^`B_(Dy_GYGB%O^wC~ z9jAOf<%o@S3xUb&U@I=)_hAdMc^FQ=S|=?*8qM@jU0#9{NuB4)3`oO^c34uqW>*wa z-$sPD@Nw&poC>v0WcC%&uu!I_ZiNk?LLAI%A?fn_=(?eg>60P$${W@t=n@l5_ohw{ z@ftn%C`Pei1?ZYtR)mfzk~jTFCHz+k;2r+p_2+^$jAJFIQX&PK1*i0$4WBut`lxZm zg`oWksph1X;J5d=yW;Ksn@+LZVZV^U&Ju3!hSwIB!K|&~5UK4+#aGQQ7kQ;Xu1yh; z*^o>{`yY;()tbt;WH?eqmxc-9DlPKa;Jlxr$t}K)#o9dc$ra61yxj*;n{qltrI7!GH;Z znFmP!1|O!Y0-VreNX?#9j#(6BxK6rbI`wRg-eb8=N0>c+!Om$(vxZCxk6(Z%_i4I@ z+#B+1Q70P9gsgLBxthqJ@9y7e>fKimN*+qVLrhtzq<9ZQ$>e&HmH0^Bge3849O+yY$As@boUe zrE%a!F#568BgR|C6-k$(y&pdN7&JcO?~jdpW)zpQB*B?Q!dsDoKsc~!O7#;Dk7ur0sP}oTv&|Q2->P^PSw3h7g;=Cd%)D((1k=P2$dMH z@a?-RvJu@(o5R2ZF@1);f?6zN=A2sZ(z6vlgy(~~v2UD2;ytn1 zsyqSWV9+m_iFE`X!+(f!t)fFFa$uB+VPVQIzqd3g0Vn{mX-<$qHi8Kmg=YBs?P_#; zcokN$EW6?)e@tlF-3ecGG`0qnoxVC`$8vT5QF;JcH+Pdreoy@O)oTAy@KzFs?HM>E zqzg|^39CSpQ3|;qi#^&XwtB$?`?tW`ZsNFoY69J=F?z-j`fZ9lNX~a? zq|D%HR=;|2uuy_-?VTzfu0}9CgVz;QZh#EL%2=^^cO{v(I=lb7YK zt_$y!8aJRKDfk>1-HPWb0G9+~NM6MA62IB#3UrIeK#yAa=20L+D_%KJU3&TMSC?5^ zIs>$q;a&dg)omf;C~LgbFx*rBye%F2*+*HhoEl@}AWV-rOPQw%Zx{JjuG*~mIxmp4 zj*-)}Woep_dFwm@>AZk{k9vw>C)Y5UKs0)6Nw42hbsiafutsHW^GQSYOt%m|ZKkdI z7*=+P%S|+i;Z~kzWWb;#fN5lV9g)mq@h~2Qy}SYRXOnf1c$x5!o~|&CjH?cvf=6Je z9(5fW;)G6|k|U!i9nOl(r4E440KJXCZ`FQLToh3(iTP`jAGNmc9io~jF6 zf6Bg=P4~Z1oqI%6!CIUB9E8eq%3i2l2lZ={86CB$k8R(z^VeTckV>lk&HS{I?GPo7 zkVr+GxW&XI$5TXJyYG1u;J$6Hznv3PW1-;ZSDWGpN$^DQa7pCY)_XZBb7c=}m;c*h0R^TH&FA0I zH!Br>9ac@gMA*ubv>fzVCQ=PeH^v#%po%Lb)W)K8reWi3;W6u5U-FV$e*t`sy}ZJze$v5#mk#2AU&#We+IN}PKBtN`9-dS{CeKv}RIvdt0O-Tj(xJg>_satE0 zXG4RSeTH(%LCz(WyMKNEneC0pc~NYO)Zw=5zx^8Jp^CE^S!QGHsRX+cMyyMHXXDvA z-RHYAf}BMmGlCcLbz^o+;mr&P90R|Dq=<56wMR0bWVmQ+f|W*2o)xMI0yd;Lu+Nzn z)3!MM)E`oe8CVPSLbEe1t}>5<&>RPVUdXUn+3&61+I;=_e()!XCR~73WKfjVSud;+ z1DyyBcc?*P;cBpb1Oj<;dl$-xM!hKHr`vne7ud0@9h!gRH3CLD(P72kx*y+!hNRMq zKiHapi$>gS&Hh-K>Y!Bz|Da}@J>>^J5~Trus2Vc1=o_m2Q_{K5dyEXcmmIIo5L8)u z7%{64Y+9Luo6eNUJXz;z5cv#Jb`ZG~{esAaLJ6{JijoT?sOcmrlb`ytv=AMw&hXbp z77F^qtnlE|Kbt9sAW$j(T}BsVdrS#QOD==&gZ>@28bo0hv70Hrt{k%aGjGmO=BFaQ zXVVLlXQPy(f3dolf3T5A_q_+f1Pwes^EKzP$y3w0SO3Cx<9z8fGOcP#P-YhvX5R-FNfyb zw6v5Wd1NTFrL1ng!VyW%2v;2Ii&bdQ5JaheA*ep4#=?`0fBwKcgGBwbw8|I$E>l#} zpm;92sAjq-sA&}{9oDHMVML0Z!v&(8prc2Ycahm+;h*OV!ggm{nMqD#W5xR?E zyRViPo5icm+ci~?pNcUXH!IKv{Fw}C(9eqA0C7}R{xjftkn*Jdnc%Z z9q}C!#aM(cWU23Fk0#&|=_5n0ot+@5G-ZOZU;Z!1efQ0EDIKBiF(d^3v5UA=lwCnA zAtTUa6&KrS7$amhnt9S_R|t>bn*+$%BaA~lGx)b&6&*EuB^22+mRSnm=jGEVrW5xG znnPkSbX^V0el+0=Bh}!U^r<%k)@gu^&he#UlxEATx`vjARCJ?s^g1K?R4w(l--I=`pPdCh;$I1d&2 za)nP*pK96v5#kHB^>T3BN;fm1B<%{3sT4E3>h(_2*oImv540H~`# zF%mLd-d_F`r7bHJF!fFcVi zctEVSFh&zoLVV~E{vk@pdIH+>Ju~A;ak-(^nGX$Nh7aQ(TWl~QiL48RAE6}$dzh<4 zE>H-_?qphIbG-Tx9aRU7^y|lw2v85nbm*^BimMk)CawnHyK<1RkRc_QdE^r7zsfIC z2pGULb?aiehi*m8H>o-F(L3h77k8ADi*2`L-S(xgwVU~VZrJ7R?-k0lu=q2>p;7~g z8;beZHsoFSb}*z5NpC46H>#@=?E*1o zt_NlURJbk?e&mKKc}PAB(~iru5p$R*4)4AbCj@KLe(w#(mOUkMRvQqfdjSsi_64#P zLAZne zTOIem-^)euEa>PIy;s@>P{xlpQR?p=;zxr$$gA^L{DZH34Cc0bZnp?L5#`CtNzo1} zxSKf}Lkps|B%LiBaAj-LHRVS;y9GakQBiipMk;;g?mR0Y2uC|T4|Z6B_D&27sy<0_t6{WUC zyaittOz3fZUsBclGbGT5#1eKLxESo-i}k*5!L|s<$*Z%6X9hZ+t|%UI>uAa_XMJ#U zavoY*Q>pcYJwNIJmW(MlD{Pmf%**wbFLCDjZ>k%AYllEg57~--_eFss^~9+)DJaUh z#P&GES!Z;G1S>@U23EKT{)lhi-t{U;QF>&BFP>?rcD+u^j>d|ir9of@yN4c!&_qKa zApw2QQzuh(%LTA6L8_k{V-}x0YA(7+RJofOnFaIu>c>X0%)8saDd(Da$vPfA1>nCy z(?&G+x)3fxr_y-JLB(K4mhQIaM8;mdXW@>psQsF|L6$LnK`42OFCEzl!2M8H4C&}{ z%r#e9ZzJIO@bg*ut1(fLJK=9}A2_~ZNgXm3@AovIw!U&pmeh#?EfUtHqsT5i%ca7R z?>d_7W0?&%mL(YvVIdXPsh$9R-k-9P@CB$#ee|w-Vg8__z=8FyfqGdh zB3!YnNk1!~z6i^)+~;6ho5?s7H-(oUGHO#1b6in>wOUfGnlEf+|3D<3nMdEq#WH{B z)SMbbb#eVgSl1|sXLC9f&FZk>xl4Z+TJY=iFH^A(DP9bPwqnFp!e~<<135$H?Go$y z_1_l&l53EHhvDVSNQn9N zqkJAAdt1Q~=s0xsj?c{k7s){@62v1sFlA#|Gbj;RrA9)zPnDQ?Wfm!=eVb$Ut^y7V zve|I$>R#AF%KLLjyFuFz5?x)&Jb2d7U4T_6F0=A%b0!y4Y%8!303x~Z;PJ~XHN_KF zPeSb$&^jAe7?e279!xULNyP%AiD6^J{?;67FO!XoT%p|IBv`t+cw!!Q({aw-oL}59 z@vl2xfN@*Gt2-&TU?y)~&0cw_9LN~~mC35bq-+(LODdMYPfqR_T&6SjfKXXcAL&lgnf2iAuCB*ztW(_D zsx9w-PC|q8O|}Rgo%lCiTc`bqN~2xM(qf=!s1l_N`>aj24MBGP_viucQRLt59_LEF zU-xoz_&oQHHW5q|u_^HBjm=&7U3r?X6Yp`6F2NZr zcHRdQh{$+^-GlQ2j^Qcq&*fC;w>v%qtUc-eBE*n>aSTu817-a?3+>fY>L6EAeNw|~ z0d|;*ApqVIus|Sd4TXRx6YY5ts$U_%KE1{C`S7(&jA8p0KS9V9u*}O1%;df4Q?fu|OmKeW2$OZ=m=;|G9Kn3c zVumcN`vIu$J<2% z#(k&C^$VS~>xCgq*Pu=~jW0`c;L8d~^-kB4EgO6gB;rr_t|SxNs#5d_=cfvm+?uWd zR~6x3_fd#w$_w88BNrtt`(SJZ&Z}yv#mUE+@3v`&4Tz3m3Sehrit5-AY^Ehzqv6_Y z!L+!wLPG~Jh_w;nnb|8&hHDJFl(m1@e9)A=4Pve;Xjxx11XCa!A3KG>(sBFY{Yt~- zo}~V9Vr)l_Fhct)Z9Vi#8`twI1ew`K8e@{WB0kF7(_vKYQ9z%4O{5X0`m3;n*ZVOw z=|$zDIS?%mG=KeK8sA{B;RlrLVG>C2p750GyDnah$)oe)i;s*5h@2PKz;tcE4;J*a zd=$>oCD7O6e>}C`X6OUy1VKoDgm;>h zQu*)|@QV0gIzN;Ep2nmNywCT=32h^26+$O5h;^p&u4wFMm*?U6{ac~>1gF4mPQV)W z1#+pPBH&s?M-Bj;+=rbODcYZ?j7+T1oSQ@lV>3byAchBo5Sk+5rneCM_4&aD8Q|$U z1mq7Wy5;o^cG~=&o;3*Q=y5|CnF7cJ_ESsV3(ZXlsbnHyzPE#sjTPZyLP1U3mM7D# zs`(bKU2qW0Tm5qS9Km>ava(ZtS=xrbk$#_Y$Uy!W1AxlwDEC`5yNM{phHP8_CU~o610WOJ*>({q-|^#d;T|g~8B!4`zyrKj2H{l{^w7kCjETrD zICa{ug+`}te3)a&em&1R{!H9s`GZM|4a5Q~ zf?JWfvF8tV44_+6ddQ7BMQJt5Gw;M{q|v)^_esMx%W(oCwcGq!v=c#C?gnCsCTpQ6>5BFgXBvZs`_D2FMbe*+vLApa%?UH`!ltzJ|{J}0grF%8n zH>2#;bTQW$cRI(BFnRXB7m5ho8TQB#Q-(Ba$pYL0km0Rj_Q57Oo6G65klA{%(Tl1-p64qY`uR(^dvX zh1`!I{&QdXz{79r1|F$G@}Idk+{-z5yGipCW8ikVk)FaT-BAB89}*xrC$#7(&3H(u z_vZV&>U*z@2seWs*pdX|y?TKAh^d=PfR#0(nW8V9qta8Ljeoo%kAR_0*A8>vw=AD| zYSX>EF_yT>$$njeU7V|uBK^)!j!FAPr#J{_*+l)(?6^{(yPzIwGa-ojUYJvj>-~8{ z3Cj?x9fzBfJH=?*5Pr1YtwN~Q;yKw>DJ>9?=9#X&B(F9T=BNZs6M55&DoF|=pN6AS zG^p&GLsebZ;laMSIr$;SI=GnEN6UjcBjs2N)n^nsSMIxi{vaRZ6Q^GCV%Lp2WHG5n z!EHBFoJ>k*vR}I>3yS#AKYYZ$?o&p0R?s17{Y&BLhsoXr?YmqaL89i$&B$!Bwl033boWP_eIf0xspk}_vHAM2&g_|_+nk9C|r(zE9g=T4j> z@LH9E^EQql`3-;+l7xIS0=wBRxCviAOCx|-D0Ar)w?k_J;r1M)>>1axj7{r%h-ie$ z^d=3!CeBhjLX=r_{FW^sJ>i!wu`wf!_!ndFTUq`Xyo1u9REPi3?iPHsyiB zbBEbv$i=>{*Es;gd0F7zkk;Mfok>PQiM+8<^p|ATG=3Q4z zVYjE9KA*K-Mxh+ZZ>|%#JU4HxEf#^0xhTH{HEG0_WYk66;nZ!_NKT*v;rn@a*?c@b zEf4S@Q*?MWq4)gSKvqFo4~VrhPD~HH2`IH(ZB3>?5 zSc+|(DKINzw0V|y5FRa?Wtt^V=+~S}0HvIi^&XUac%tfl{))oL` z@X8i>Z1K&@{8E*F^onBSXLQ;^SC};Q20p@-f;Qxtp0B2f%Os3Q=&NNKdIRi~9f}0= zYxuO!p9c*^jNxWiFoYSdzWQX_0j|{8aMx`e3Z)!YVdyP@9YM%1j3jI+r{rCYEFH3J z8FehssJb<9_C$0KF7-g*1!hs9kN|^0+!dTJ7hlSR-g1*qDa^qk~+r?1PGDk{!W|061Qs-kl7y54z0;&x`t(Dg5{i*4)ul}L|}m`DDS+jZ_qx(2thf~L%P;M z4yQ*@-S$30vQ-(s7_IHu7Uf)|W#80>VkAKM4A}f7n}2P)Ajw?qWD$(hzI=e zSA#+t%FL1{%5!Ccm*z~W*6;p2@H2;RjE35PEW=?6p=t4c&~HXBcWg8N7tO(HjI4~T zzopYLA4!POkKnfGMb6tM>Pd=yZmC}ssKMK*(r~207QveGC+5({8_ACoMZapKJm$19 zRX^HN_O^aR-rvK0ZyENotJc){XKs-8kPXbbo{6~ru$}qAp$I0qP*LAIbuLFgTZY5= z)nVblViHHYqal-k+zz(@$&#-9EQXL#^9m-d zxr>LWyD)reH{=gaHcs2C7bB^r2!NqxQ3kk+?Ez)m;jvR4HCN}@p17TQBB>Hr;^G~i z^ipj>0oIMcgW*Vb|86&*9{HytJV*0mOsr8xj?ys~fW=$iA8-kkevlM4Sqlpx8Pon% z)<=X|07sdXsNqMh4ZSI4?33AJi2JNdCNMBI#Cv_oi$a|8#d%N~BOq&^KRU!j7X-cS z-!Ld+h&-fBY>d3CNK^Oj;h0a!Pu`%Q$Zl&bdCtw5#h*H7815IJoNz`*aUFrJ0@K_Q zE2UfTb!ZxJCUyC9#O$RsyEQU2QDSd&2EfD{#^!V5|0#d`R2 z-7kDfZbe?;x|`RGVf|u14o8*ua_)bt3+8hA+0qLAm-I#-q4;IOQy$f9G(R>g#G{J6 z)i$(b0}M)2E@s))-lD}326OVz85u@#Zvhb&en+@06uPK!!A8VJsT*R2&DztN|kF` zul4Au*sYv-K9A>Ar6=r>2Z@ler;F2>0ILE^4GF=#vt6I&wU->`_F-0T)*5r?nt@)G z7|M>HB?+;FxUZ0?Si)!9eevK&SiO4%fhEc4FazDX!U;v70cTnm8rByv;?aZ7ix z>I|p*a|q7wg&NZYxQBcTP1j0lnlRmkp^E=P?_vw=G-&%{VsG6PFDIkO?#I`;s>5Rp zZ%jbtuK=sr6X9_H5~}wQ$;+|OV4*JLJ(e4kqole58JilIV)>#mrQfiKVw zi=`1A?z@59R=k9NR_)YikBcR{1$FAnSQ(a@_t%EPgMM~Af3{J3p!`lj7=0=WjroCI zDO7Y^Y{d-0)P!5xGE4w-KrDlEfi>smf$*tuT{K}T2}0=m0M!JQ_1asymxZ@Sog0cO ze^Rzbh_{Hr;oU77`4Rb>B7>w%ue$Au#>Z5jzDD6l^zdZBJjKDK^SqEL%iXn&+4G2xJSevij!dZLf}dlt>bTrc+Y1g8~zy9uy5Z zaiT2ot2`{D^z2&6c2?JTpPHiVl#Me?>b&B*>iZPL6(<4y0CW--4&fgBPPDh&RRr%8 zQPru60yP)vJM7hQv1lIesyN<%AQbCv50#XFBugfff5d1|X2vebreGwB%uRfx6C`?p zuM~Vh&L`%#NC1wsVn&sb(N!;eY`W6?$6;g~t_M|Uriep*3nBNv+6wGe2qv%nk4SQY zRN0DhkKU67D>`}jdqbK1xwR3FbQS6k3Y7G8(2^X!u%bR9p*ny!z%o@$8G$d3Y8(=k zTg9+?bdAE|Z`VPd&lWbEMv~uADRvG&gv+$f-&2T8LI=MerBfrV(LEpS7xuHh%of7` z=3cJ!sXw|2x>ww^!8Np`BIGZ6VKC3C1EY6UK^cc`6%WUi zv#=Zd+HktJmEVUH+9jpeycpKP^sVZ#G$l^3FB=HGK z{nO8ZgD0mCMNBc}GpSlp4`xGt!1fnLVb&SoPrhj_>||?ER&>iGlw3dS-DgZtR~;pQ zrL;=UJwVL9GROa^O6C26YeO`Kc^@o>;C&Drxp5}t8M8lVh|ZQf|9&f+(enIcpQr|m zNN)Qfpq~cp2?qiG3g{q;DmOJn4{~#h)qY&qB0`wH@oL~Q&nfV9^hxq+ zP>&g19qaH=|tDrMT_QpNvJVK=n;7jTHSkwsr zE5`Hs?7JA{*QnKmxBGJ-n%EV97>t2N>)y~A!_P^2y8+G>rnAMhl=8)yupavJ$lI&| zT5-RRz>n}BNTwHQUIGE49@0yf3fVPmt`@55=8rD#3Tq{U-nP?uE17#ZBB=q%zXu86 z`(_aFpl+p64gDl23-T@t3pwkf_PA$grOeF&si|#|^ zeiom-tv*GZgSf3G%py4h8R4rCh1#04h;k6wFB~y>2s{un#Vx<9^+fzamN#fFYLUuY zc}9Z`Z^R~BdnPB{=-a~lO)Q5}2tUhlq6f*UYr)=@gswbE;R|H`d^(9o)vO^qQ2}dj z>DJ#erqQrysNdPA^ZVK_i)F4;V*=M&f9X)d>Q1ZBGuTFDLZck0OMEb==bYFlkw?Lk z@d{;y45s}F*CNmhV1>kgBtQWgpmgn~6u2u%JpooDXY+n7^hDshr%W!61dXwr z@_n#j8Co8j<_80#(gj=p33cO8T)zGS#*wp6?UdTP*G3Ru%zFskA1SgsaLiojnPx-Q zHuWNK%iI1F-}1ypF@Y2%m9DDVkn*hN2lB*?)$Dh6lz#e$1*@<9D{qMVaqnHS9!Won zW-ZSkm$P8x}b#@;`@LSHcxS8UuIrC!#Db{FD zpstB3JxpjF<*1{};u)mo^2peQ3|I@N&t>|_=WZ~~xx?H|Ub2XjxU8m=1-)1A>Sc9; z2szD!yk6JQv*57y*OBq`8EhIj$$*3Du3uzhPk%$J%P{#?(*JDdeSpvha(P~e0G#V< z0FO=C_Yk}yaesHCZfdFXOKJ_BBi__8QAl-n*zPv5*)QC?3*t(jGI{&aBgs!J%@OYa zA0E_-I<@&i(d3d6#?k9p;R|#j!hn^{Ijf^B${p2hGD16SUj~%W&YFj9Ii*$0XO*uO zVmMgzs8e-qfnZluQ0p1k#-W6D<=j!GiO7+j9dbsXJCWAXKReZrVJX4ogWmgZ?fDcD zV5G$0$FWNOpU>ii&{i0|wQ4`|X`o5QL_me}ff2r;-vx>eJNmiX33c$Ar2{~f3KTL| zM!Sr&QM-Iwc^O4c={Nhi+d!u4EM!-@Eu@FL8!0^I)UBQ=5^IKedI{C1riYopeD8lS zt;!ra%PYE|s^~y{8SrkZ%EIS5_8SmYrrTCba%&FlV+Rpex?kVLPt4gv;*OV*d6V@J zBle$6oW%v+Fe-i=YGEW-H?X-3Qi%yWmOFm);xu;+VU>y-Y$pQLH_9cP5ih47MUT29 zF(^k^Nip$ni{rsao#Jl25ritzA*^ri&kan)^1vcARK3#N<-aMoz@fN^BU+&Ptyna( zP%{_1_pi@RP(rxWMT^kJ=Tnp?5D`>6cQu3WFndE%CWA=`WW}EC$^TpokTN;+k5oK- zL#Qun+1k63&h~^%sC|bfb1a=nm;t|@%^c(it15wp1F~CYUBXU^Q4(&>2x9RJ}1X<=>c%&3^hRP=3M>IN!1~xw>W$4 z#{RAf`x=W3T9Uu>fk~mr{vuqpipz4R#qO=^g@2yk@-k0RzoA*KWgE*eQ`%i4$f7m^EW!=L=GMU` zxu22NA91=D)2B>$+sk}Jt!6A~0W9Rc_x`AgKN49QDaTxMNom;H)oo4}xtNMT zy81PqL(er%`iFR4&7gdHC$)dweob-$qx82{lT1O}K<^?A-<>WFZC!T=L4b8XUCZ0` zBVE@go{qNu^dB3IYI75+G`~!93syZi>UlC}WB<4YFri*pq zPPP)=h2^IC5C6&KA5u$H%Kj@5kbT&1z8j#qs!#$nTEKLeW;YPyZOgb?D{6nG?NkhX zo|^JbNB|Fv5>O2y47|2&P%>$~+%Y5Xes0pX+cMJ+kHF~gzoxsM&vF&%dXDw+(W{c7 zKUMPtB6Tgi00}?D02c2xyyxcj1SEV=EgZ}zcykvN1*29;G9)3dJ(4&l z3#LB$X=M3`;ntsPQI85=5kMvAFXHYV)VsJ_5r!4vHc5jcTyd#@Vtmid$)i)Blv^7$ zjbAAf%sA-X^9*nMzopqP1HCEw0FzdR#5_-E@j z>=0kiDKiJqKgh#L5+gnZId}3FqXF*5jG~I2$Qd;t{@Qh+QF6raPrhxz zR67y5qQ-4x-<(;q`^m2?>ZnGuCJL1(MPHSY=^5!?N`R@;`)Qp&Rx{?(- zyxS_vH8DZ2Uyi)_Zz5I%`ed0pIMFF=Gd4~N5;YwHJbkgG;xSO{mA~|=ISG?ySq?`< zM64J=OEE`wr_&slfPNk4%D2xi?0QJB19U%@Bx!MCrVIgA7cBN*f&;^Q3~gk^$2=?! zKDXj1o_L}xQ|;`JRR{cFK0SAs4-MLr45j$Uv!LZtbpl;%MW`nzGE;TF-Xe19jmkrDrNxC{4lNtu@7;}-wO?`#RC6g_vaRg;Hs0)HTcvr<6r}W7VVa` z@>$~!F+$-|f2a-%MPs|$d1>4N4|W9I6Djx;J2pyFb2dnGd4{JZEs1N`c(MIbd8#Bp zABd}jwMy$4rQk4Bqao>&J}Jd$7sv^oN=B%elTvd5kqFzSr-g4n#~pLCcO@Qc0%Q@l z(McsTB}9!Kt=z@}ZA~;|-fhfAk1(a_b)n{hbImZ36d7o#%(sn?%#H2>aJ*E7)G=HU zG+)gg?(T#sYGUN-cJRCgnsUwfGlMz5OC=0lcd=G9_meHmt{D6_VyLHfUq(1{d1KX6 zXAq@os?N)_mEU!Fvy6g3-Jh?Zdsg? z$Fp*2lBuxsk9It_`iaU!%?!DkFVNqbd}m$chklPKL4)zRJ+aKyv31IJoeqE-k&Y5P zpA$a3yX%h8YUL%0-+)26XgUZfnAXskvtJD;N{_vz)rg}k&74LR{aA9D19<;1S4fo- zG#~?#xT{PLhz&fll?%o2_iJwsGxpN|pwOikodGqzc0ZJAphlNsfkl#8Kfh52C|$(| z?1)DQi@-o=jRw;A4W6@B%GQQXA@Kk!z>b=^;So068t!W~G2T2)W<$h>Hs7LfN+=LPjynYb zb+`e|2*;MH``7O3d?JH(Q+7xzr&IjZuJ)$Gpfr732Tpf)%qLk}8-7cX zZa<{(g0_YUO&WM!z5}N-z^$}8rW^^(+<%14jDXpylT2&jDGliSGUZ4*nR~i+2O?a9^J7gJ zSfPXNb1Y!V-7AWwwgP8D(doBJ5YO#*zhGbOC&ws2;l}ZSk%q%R@$pM*Oyj#Od z!q8P2b%Bp>jYoQl>(-pn%N4Gz7*M_ESf%MWbR+8tfB@4ek!IcG!<%=~Vo9Jq(kg#8 zhqeZsyM3^MCnlW*m%e_ZTG*h802(X{zhngkO%cPF66?8JDXk80%8%XrNDQ*ly^nP3&gH%9?W ztcx@7gGQxx!{2ZUU6fDq&4tna-8DoaHH#V-vIu{kIn=3+xaHj4idJ_}Xt;E*UM#eR zh*U)?U#uUN2N{RPSpfh#t|eZpQUEjH3X}L~NIr~)q4+xDM)yyyoSz-#b;Ap|5Ue=V z)dSr%!p-!1;>0P;| zf4tS8op?d9_Qh43uuU=wxy%~VhZ7{`$`f&ecZvp{ID9NXa}L}{TmhB(euX`yVz905 zsjA0)$Fa*aqI3}fZ{>pWOo=7D^><5b*lR=w9j3ik*pcH?u z2zlQ#iMjQ&PQ5(j0ux1PjDOzgP( zUZf-L|)Tz07W>hN^2BKiboS4Jv6y1t2{{{r<_=2Egmrv4Opf zPDi|~>iI4Dx%hWtkO@XCjeEg&gT|wiYTSG}(_@OoK$mS)m2f0{4p3f*L}wA{f>q+< zOQKd!q3+$lURinG{^!CM%k>>V`aEG7TcJdLe5u9 zu1Z~yHr7-Lp5^f+Sx5oWS{J_VU0t(53WIp&zC-k18mn$jcBt?^^ev6f^F}nVB;w3C z21D0?z76j0S|tT_R&ykdP^YFOSC*89 z5?HB0eVvhW^EGtQ3c*x{-V<2<@(uB6=a>dMA=TA@Vs4D=USgTSf98}zD_;g$;47W%Fr~Syf;Fw$3w%i);#lp+=4UYA@oS-CjJB9*} z{3D=qGXuWA#aDa$pJRF3W+ots$FMyd*F`ldYg8My*&!-o$s{(cSRN%+l-tqn&llbU zUBQetDZjd^L>K8u{4C|Tm0Sg!`I<;f1e4qBika$X-DnZvJRAe)k4n1Oly&WTkSn8; z%>e&XA)&FeY(xaPL{0%$1n_2-4IW)icuZ*rXl(OdEf?(Az=--4qlH~^qM6*cH(Myi zvp?-P6-xaMR75|f7>=VRNw-W`rQ4j%OA%7Zp8d~t3aYQPO3dI%vJ8|Fe=DQKO>56A zJvl`qImAX0p1{FP!~=MYnH*36P33U^6ACq9T+fLUYweN*ykw7uv3qWZe$qdT7i7q< zi}1fskz@Ylu9uDyet|>RTOfJW%+c!~&!RbUms9FgB@Pt|gRc zLv%mcaY7w%#Vr5?MwM>)b1_-1Onbg^Aahnh8VFU!%#nWQ#U#xoFjpy!92LJnCEP@M z+%he2kpOFAPHf9AlPi!O?$C~FeO?_*D};4Z&xERChGkcA&sNHa;V-ReF}NRMF3gaz z{T9>1=*^<=g{f;R$G#Fqs%4eKO_PVDKD7kpF*L{cGCK%{>##klwgemUw;q%flZ57DYdA<7&iKjEcrr` z>8a@>|IRDjG!;_t7#UdTSwx)PQ9KPkRvl^cb$0o4inh*l3Ed&CLt*X6XPovo$|d3k zW1kxN;?|7_;aQ3RK+K)|phNVul?Q{APzT2^rAM8xwa7VKRlePQ6vuahd22!YBX9wg z?{c=eX{|fr7I1Oiz#6HWz#Uvy8}AV~(Z5OZc-BXWT?)85YPD6lndQ?5a@hSFC%UB2 z<|(=Bl0_bZfOew;FfWP7OJh7Ri~vlg;|at_068Ns zD?mLS0~|#{%Z_!H)Epf`1K*3eWd-Chm6Qncg#BzU--k$Fa06@We-b~A4!X^bZE5V@ zxJ~~mFu(cJ;40l2mdHJW^3xh0-*dmdvfdOePa^awr5KzLoxB`OiBOLWdw}dS`zDC) z@wbKVYL%h{6TcWGg=SsZQsgW zcjL^Q79)QP`b9p=pqMm@!b(F@0%XtsR0N}vR%Emr?s#VaRT?Qo@?2;9NsiwHO@xyd znIDq<2Z(jhrjNM?u3~{eRr}u_<4vIEJ4f9V*EmtAd+C~UA}Q1$1AzUhs&9FkAQ?j$ ze4$QKGs^w;8w52plA{j3WdEI=gpyLXfs?RbdQ2e&`!0?ZjOXHk{B|KPuB7cVOmG@{ zZTIgi(^2yWHq}LjSNRnbwa-z)-u3U{)QnWhJ|&x3R~xZypzq9N$alg-|AyfOH37x` z#=v)lv3_PSG-p1pgF^gv*+^v4LW`cXk!(24P+&m!X;D zlZ#S|BSgjOSvK55wAsWR%~hwC0c(esGeB=bCWmKyHee0PN{E&kF9JKESaa}E9^oXZ zzFdnp$9Q;vLGHApd`zRGNN{`Yegn{X5+#}Kd4LrAuB20unlR>}TJ!E2P7jU>DI@RjIDi%j?h9>nyggW+a6t>oJ5f}XSRDZMX2=B~ zD9ChB5kTVR!lEW5>VSvH>>g;sq?6iu6CIGA@s4jNjW zU=6E^#p)q=_sRws1X=GUNQS{Uep2?SY-yoPs{u-;t3KTM`ILtcQh^0Uo#8zr+FU^Z z68m&4XO+CkJ~

qWen&O(SnZdxHtdtRT+>F#mk4uLoM#knvepiAl^|acbY!(-7xn zGxbmg+iF&8>=ohuvW9}iUMVBFH_42>e-Wzp<__8c{AtiVrF~69Ts6#uWt=c=^;fp1 zH~LnaJdK4Gy*U&No%(;6e~aaHYM&Cue~k;^#{S4^)ZN%?(bE0q9!4%7Z?6!Pt(31R zRiy~piM9}jXg#ow zifZ?ZO8j{r-y2I{F{%fyc4tPJr8ZOlR2A&PyF#&Nt5bM}z)KX{0c$(ThG{FV{|5mikwcf4XzoZ6or|78r%;3-dNO9T9wv3ffaUAK z4QlacN#goLdH^**%D<>!pv$f*(5rRfutKdZlxiGR2`PhY6mN%jUgG6WXh`8#xR~qr zyuq!%HR1R!>#=O-I?S+>jpaxC*Gt}zA*`9E#jk$FC4)$oT-LX7FLHlqsQ56ZDGjHD1q~eQplUzQ-YzKnq8*0ulavKrxX&i-~!+e;I zON2gubO7&?VJWp{Y9)13;yDQ85ny*RyGYdE+bw5074-cJg1vtKHX?HHrg3^GGVZNS z{7-XYs(qo0%}8_>24QWd#Z4XnNw^g#_Xfk`v};qq(xezv3L3ZLZgbVb9VXRX;m{E& zDZa+`K=*-CQ=HkvMond3C8mOuEa`R2)RIVUhNdJ-=X>gql{O8?NZkFpFkO3y>#FCe zh7xPn$gn!q)>2NP)W~c-O4th!BICJ_r+|()-WxsEC}(@$u`@^L*vE9kLN^~WwX!8U zqfGkX#_okDChz1p;#~K6_&z$s5ikNGtNXZ#KomiMDkfMi;|f%rr?Gnv)P4sC)8H`+ z!b=GU-VBpu%v+R49>#71`yvsDA<_QB0|G^hXv4fI+d-|@mGRC-BIWF#6}%L~j;B-h z)|Rkmj*N*XvMw@m5O*41Y73R4t`5um<~pl0iFAsv{Hw9S zKqj9INzMSt(pA`FJC*HA0L+hvBf80`mu6OWThRdGl@m^6i9P*&7CQR*JJxvnYc+H- z#(w%8VO@d9LR74|*$yaVMs#uzw+!X;Uk-F|tDZ&hAqE)tKDR@w`quw}$9oSW&4gjpIYc5GS(?29k%EpP z7YhPV3BY3=_`_zMwWsw|T%HHq_eE0?;2<6dx{QGE-4-X@{N3}muvGT} zTOwEaMf0isddW3gIzB*jh#kWkvGbffmYBn(?<0)zaC+L8erDOuHj#DZ(b{)pap4++ zS9=fScFkO+DlMQ%UrhM|3Z(n=IMpx~WIT7?`^{w)zFTx?Rkw^oML%NXYA}`aQFu=; zxm_!HKT2YOj^%{x?dG6DkP^2!eZ?Ms!S2TtQ5L*JpVr!@|8LLKfk4_2Rf>`=(awq+ zfSGj_Z*EyYsliF4W7op|W^M5f`l6;NR#x{IQtpF!szITpx>+7p-M#lJyZmqhI6O29 zn#Ux9H&=g~U#SU*{6SeNqORH6(1fUyDLiJN4W~uTZg}2q&Y_UhyF+AVOg^7oK+b<$ zwjEPkTtKJi6&D`47)FG^9xA%P){<{T-{*+`_mtLGI@0VW6RFW&;!B09ZxCM(Q=?b5 zU@jciF@Yy@09H@-hb{qM@mdTckl+E{GRb#MmV~oXHst+$#F{~h^1;(YHfKMns3psL z9OJ7Dv4?bBEF-66%e=SoR^~N1G#xQr^JUbMPA1-EEg04rVGCdr{q=6Adk+zPi_5xII+?_Z+pMXPe*+eGdj6j!g|qPV3wUJi;DN&PARX1dt5 zt{Z1(0W4dYH^YgvmXkCUXsipVc5=UNFkjq&{=362F?u|fXh;{#ZD*znb6Vxk?~?jYe)Qu({CK@DM0H>w2auFr zEUik#a{x@epLDw9kC?BTd)XDkmYBb&kp&rAd>UvmTcoMx?AO$jq~^fJLNYxAI{?7# z9P@Ch2yjVaA)kuc@gTGtfR@*9m*AIr#X0i_^%-sgyZz<<`Tjk^7Jl=4h8%@{v z^X`}3fA6MXG&o{Zd!Z#C38*(hc}PZPPIYDPHw96>pduq1k+(=eK-ZqhJX*5b&F>^H zl2`I9Iax{($GrFGb*au=E90!t*GaU*NQDs*K$U5jP_bPj&7L*5I1KPr)6;rT1rUw_eyNbr_PVQ zLj|2%(EBbUG~m;izl$2_t+bB3pPOQR7@L0VnC#iA&YmW0AGcp2lp@GpJsJJUh&t)w z#z&UbX$7shyuCui3(Qi5y%I(XlojUbgM%*cY$W*eWNLrGx%vh6NDm?4ZY!=54Ic?w zyyilNJIFk;XB&Q+hb@QthNu$7VHhdQ?8`s8gGcP6f^xqrCKGWTu8nN{a_nL)sh zei594ql~dNc1A~u6Hs%yIDAu-rUh;ujtuX7*KTi=)JrJ={wP z{8GJgGsVvPq!hO~BSqu6T+@?EDZ5>WwFh$qeLVe$t#VW(|AfZ{U(dLY+}io&`!g5c zCD%TI1J_vSRu!(eZQcnLC1YFoM=zaFpMa|(5IeL_0 znTA#<&&EZ3zmJQ)|4%j(=?Flym41pc34D`_B(+^uHo#LdKUekR#)}LQ@x>?M;}Di_w1h0(H%wbcd(s$Q!%4CXQ7LW`zcJK&W<3O_-?Vj#Wg(q>_jkhwuC`T+q)BGY0RO9fWf#vi?pFIn6}~2C@xdPm)%T!4nAtDO$=N7hweH?h z&`JstIz#bhE7?Eewu$`Se$22Z&#J*N$KB;mab_)5+}P%`u6=JX{V6gL$$1-)JSHai zNu+dhXorkg*IML;!()qDNwPM&k=bocCe`r%%hT+yt25kHzH80t!9<`q@#D~J27XWX zlKsjXuN^u|q;pEm(4d*D6oSm#D&LgG8PqZlK#Hb;saEE3g?1TG3ZBiLWvY{j#VCYd z(*>-gjRWMQ=qn$>e9*J&K;F-ieW%A<7xPODZqg9HX8P3QtW?n4)u^Hr$@5p=`-CZu zOifm8mfuY?Jm2of9*XfAN}ep?V5A*8dP<=mhmzX>9aP$BO$pZ*h{cs#nw|5l4y@%d z-p6ZhzQw+#2Yf|gk(m$JN7cYXF~E&=u^KOE9S2fBx75?{fMv9H0~6* zaWmo8dqyqPPGJ@Q13Vr2e90q9;X*+y4n_f^_=kom5l*U%zu+;g-lUuKJY!+4(ry3tw$C1$%L>5GEUTq+Wj|(g@-i*<7t9 zDruP4@X-B!tLK0S?EsiuA5*7+^}hi2(LU!Y)Rt2$jX4EnfIoMn!yhq+Ksrjair4#A zKCdXxZ9s@0DK9aBJBDZtx)t8Ds!7LhEibJ?$CwHo>6MmrK9AOWJ{^9r5qsA>CM!n5 z$~v4AlFOx*thOhLJ@Ry>VC8&h&nf|2wI3Q_4*sZOS$yq z11wVRhezUwdxLgO)?9eY|KjdjR9(%!XIP%+F+H7{bSkq!p;eiL9cs>w2$vLj`sK8V zEpu;zq7;q;B+DtqSQ<9Iba84sNfl!2`!8C%X?O}K@7!hCOZkya14-K{KhT{;MOGKp z%u5LD9>C=oX}k($%&@?Aa-Y@-MzAa-DP~Eqnyu8NUOtYaZ}NP{UPcayvD_@7c&tt$ zCIKA;xbw9uzD@@U!PlNBm{;og-PdJ%v|bn*WlRjvGF3*eA;|@~x?~Gc5<5M5a@6FI zGV(7r33$124;J|(O@=bjnmJwab?e7E)?x*iMGRn%&)K3C2>*qxrc-E0 zy!CH-^60HpR=`xK+a7Osyy$3(C|*5@9E&XqnOz>k{%v;E!3xeE+d63n9y_u{Q!_r) z*)#ctkzMkUM7#XvCq#se*0cu!YfT5b)_R^Mo>$nfB_X_9i#wPQmF}a=Fz1AFe6W$q zmHKYKflFbqZ?txoQ`gt&1!wCuTQqVuNA`BySdE@JnW^R;&|hpnibV12>UTUo)YI{O z>Lf7M=$8RjM{s2wO4P0slYH53bbgG`&5lvb*HZf$!Gn-+C8|2ewmgsz4^jY^3r(tT zndX%*q@5jkDwMajdy2Z*V)PEQ@kGJ#^crpr6@T`Z=~TY z6PkfgofC+L(5~E(2rJ4wN%`AgQe2Emeg>BK&%q6IRwDHS!x*H%RgCH^(fk{a>b5~4 zFuq0R5KIqkx@QFvY-K|3#~5Z=HgN3&bpvhmm#N|gft)+ioB;>j!>NamVGZHU2$;>cd28c>KU0FKQSsbC8y3% z{ayn@^<2xs6l1UQ$`ADu!&g``OPw_n)@Ng!c&%4<0+a*D*p zk}Vx7+QQl|J!4Cz^8E@7py5g!AX9;Kjzph+x>^DSyH#kEUxr||>1%LzcVa(P7bnE5 zm0I?I>EvtznpTA|z@CJ0+87(tPfNDNoL&*fLtGp<&<1FD@%-;kszowIMXKk9dMYWS?!+r4Nm-Uh%^urm9uxp9n!BFK zz^g^4jJ{0{PeU#!EtTavtz%>4FB(ZkJvM078N#EW_75i2N?#U}gp)^}gpfQ0OZaaf zy-C$Outn+JNMV?to;i;A-g2U}Ru7mc?}EB)LoeTDe4qk?*-!!~nUOqqG@VdfP$AIpNcF zO?5`ZRK2J#ege}jvIMmO`xCuT?#X!f8X^xXwguwyGxwE*eyBPH{@jP`@wWNFo675A zg?QsPO+s2MyGTbmEI6qI+$!&$GOy)bcCAYe zSDmtM%)iMRHb1s~Li>8CU~-NItxaUh+5Lk97_e!hlKP9KQOhjV4{OtHlKIw*KK<|X zZ%_EwvaJlX$Mp^e&JklJ(u-J{0Z(Sa$kL>^kfOKK)L4_Uc~+@aw1CeX-Fk&Wamcyn zR$QQpLGG^!5$tUX^wpS1Df!gD`DOI!!dQ^}f)f;ozxN+`B2=R8*i+UDcz}%MbDPK7 z1oySQiN{rrB`vF!aQST{o#sWwZ=(E|JM$Xn1=u)~Tk>y~MAKud+gE9*t*$e`MK({L zF3Gtfb~S~U)QqbDvID${HEgN^b|Wq#=|2esV^9k9}m0)g9!L)pHL;YKq@;NsB;F~!Ke|y(3ydxnt43zUP zOR0Bra?b-1Q8qLQ56?kyd*2h{;>OhX({)^gHSQgHG~JluqcNYL`quEc(vTR*oCG_mr-rh@e)8`EI>^@By$PnT+G z5NGrp+cMIis$|>aBzcDN>@!Rx5!4C0fp}DdLA{gyJT#Ts(5ZCKNEPE}sdqcy-MQZk zF=ice@@H*9UX|~O12C6GyI=vNCqtynug_CL@~h1VLlN9UAc2**Kze6;r({ro#!2ngUoMhNNEj`ALqE>9_hM#V(%{zG(n+ zt!&gR)z;n7xpXsHuRA~uO*7g~_o`Ud`*sX07!}u+$&9(9#!dSVKA9ZN|M2Vy`24-4 zG$5d5!c?c-rjUVHJl#D2{0i#0idvp(Q)G6PzB1tUdfD&(B)~ z)Z()tK0ep?2I9n`F!s=*Lwpc{-<5jdfhPUWC{~LdD6}QiLl~>Xrg!$^q^axQ_nK+w zM=>8J1KzxE&l8++)Ap2x{B})2oQ(Rj9bYBTihGV;eAJrAG&CxCOK+N7-$jX>zH2I@ z!@S81Ekg2rV63Uv@mELs6Q}T_^QrZweza)IF)0+l>I2Mve!W#4(iI)55>9fME|$ro zZ)lb|*WbFyYw8IlQfbDja{A%Q38$|#B~P-bGVYV!m$4VgjUx3=fbxdT!qk(+EHt+k z9e(!`xew%b9Dvn|vo5h=%bp{hb)RG+rhzJ$n$=gk@5gijmd=rn1s_DWcncR93(n_0 zNTs@7-;~*)_APbA_ z=%+)C;tabJJ`@qbFtF3Q{0i+D*6+Nu+(&M0a#zC*#*W<-4%wQu@x!ja_tU7WKt#ww zd8LoSat8Odks8STTIOF-jn`T7<0)I;o7{gJ5gHs|C09iaJMMZz{8`z?g@eQIW42^DCM_m`)xNoLJ_uW9h{8li#qx2W0;E&9@;ZF=88>IWC z5Bv!!B3{HpMN@h~0;uYEFEOKX{G4aazP+TBp_8hghlF=l`J?>UWamYr-n^Zw)JXzR zR{?p1NS@(1;=E;6)Xr+laHI${nmFqBJ_8G14YeBH=OUR@@yP#0dHUd+gNFl!Q-VMF zd+o;B5;S09u#X!e zo6UEM^Y;ML$>ydI0{U0Y%`%S;Pm*3dZn(@(MI?Q|GN?#K8o^& z#XUjXj7WbdH0IPYtv3^zw4u}Df`n~#3*1k0(OLn_!;Huh2#ubzk(7kEgEWSQ;z^Ea zR$J1vD^m?;%=$l$@%#L2h#rY>N-M`jV5P;|V~jtGZ4-mA6PjVHBg5|ST@rzLXl-$t z+w(gsH!~rP@2E5=Ts$<*t9tuWx8fsgG)L-frC+Qf`G>O3;_1Dbmrh7ymJ6rXWln^L zvI&aji>kD43wX-v`2Lxe3K8Da@>ZByvjN5aDb-Ldc&uWM0I?HqvI@C4A+0_|CKzRH zHV7G$N=^%dLt$tPwQjQP26Uvo??eY*tYbenQ&CE(VD zL%ks2q=qLB@QU*h1SI~oQosXaDN3qOcY-kvl}i(}>YneF-w}=vlYyl8NqX@;T^QhX zv$wyVm)?(IYgjp$g&lDy;LCP$w4<@tw4oo_LUfOWc2a*Hsr5LQNjmugwdUJhC<>M) z+Lc=o%Ki-s8yi_Z_39;sbUU`NfH(6MTtBhk5_zLw!A1Vc(jn^0Lc$fPg8{eY=Wb*j-J$nsHl$ zZLJ$n3lt#W3bWn_4cJJG9M-lfmr@#kU6jbC;1bhsiwir!)G30CRgyhG5o1fv%CmI& zxbiA>HY{O$;v9XGaS`iqttNkCqrcR3nO)R3mV{)j+U^8b6&~OszU`uoLV}9D(wipN zPgZq9M^eNW4(AcCZ@(vSf2PzTn7(Ak$kfgKztR33XDshSq=6G4;&FA8*0@dcW4eXd zjSLJKuHFAX!c=EL4^qJj)_ttX*mWrY^f~zL9cWUe9gfuMIcbx(!4_M51#3=eS2#~f zg4~b#PJ9Ekl=ud)T&-u0H=%(#&&KuCC}1(2{O;1o?lZGqBmo<1A?-GIvG|3E*TGJ%Lj7Suv)3vUmOJ4IlZmOUVqki+<K^K3s~h3g0Q+AOsdH`eR049sg7u!$vz`vv;JC z^mB~SjTE(rxE_0bRJ9p=Ive`>0U{HG`>S@6}iBgWdZwcUuj|KdM?)a{Yl6W+{OQLcrC5fTjc+M zP)Mb5lvY5Cs72|!?{foT6Tgkm#u{MX7=bBDLz?Kn^F$rARe@<(bU{yi$y2H)CWjtK!MzJkB^oZf{_)b zlR9So+yV`|3H05`4)*my-aKQtd_LJBRXGP>EtwVT#Zaf0UkTo7UgSfalU>()u8js} z6l*=LUO0KR{2iWtLz|gdaRGnr>be^<0>{1HM~N2KFzXZ7s#3N@#LgDeQr}TzRe8|| zm8j*+{2}$1p+j_9ypAg%@Cz|X568tZmyJc~wiR z`ir4;Y@{^&gMpd~Nmop+EQBQdBfpwKi}gVPENdxqwgA$?HS{*2E!WqXP8pO~wgu%p za6m?pc(n}NkWT2WE1@2TmuI{YoO`88mvm5n_5%;Wj&l=UE&T3;8|7Xu`J3&Q;^VL& z4MnAkHzG$4mCDT~NnA!R-oyicbp)3OGL`7{`+Tu$z>1j+-&%H~k>z-{5FtL%dK;y; zjNGGZ^yH10G%pUiLzvHsH~53FmJ-whMw2)x_E%2ez3Bu;8PaW$ZL7J_s<|u|;pDT5 zBt9@9Nct}TO!Ym&i<+ZVo^*KCKpyT@rraw;!0rlIVv9=_qhLtVB-t->qFF`6x6r%+I`m2ju2P03}2|aHHk9_3WmX6ho$0d-4+l8Y;Z zqPc#8btuVvzsy#jC}u3_uJ8%fo&JsJGR)FHly`8{58rh}W{@ker z|2de`ieQf^#6hQNjs=*E79NZN~NS$lg?8ram5D zP1ZYhoTa>hgPOH`WAZ3RkzW_J`YDa<#HG3^kOZ+>WYf5#)K1oKnx4@>&PlEYWMNk~ zlRvLoE~6n!;&=cRpop2mNogQUlwEFyVbZhJ; zGR1ho-0s(|BJZU4ZGz0oT3sbIFmKgvIyhRw=LQL_?+ZkCXp$ki{~yVnK8wF#=B~L1 zN+2rB6>cKxL@PzWndEbyBgm_lR}Xh*elXP@#`yli;8!)KCGHD$r=5Ebf~b+4G@l!) zysW{I8~|VrgBx$e&(a{?{3BWDg&;HLI;=Ouq}%@o|2}iO`QwSwv^CBB4Bek1qtx{j z@_^?T%k^qM73!ZX@Toc)@7*U$_zAC)IC>HMpP;FZ#5dmPxubpv;4u9*4th4}j^*$ta^qAR63BPYfY6$+DKbfe}?{*Q)~hA@%lA(O1yD!5#?4sJ8`B< zI#C@HFS0w?*eKh5N;_}wbAgYFufO$~NBHrnmPMo<14`CSN$?N)P8VudF);Y)2+*?W z-<1UMAg}mqQ6i`=QP_#*+r97;GnaH|?z#tYK=72l?9^Q17>D*iDH}9vniRj2cAS6B z5q|AGW>#GGVp<6EPd>*gpQO4I`rmSKRDwr&Y)~uX!Bw2*a*F=bGg0;Yr8Sb;kV6~rGU~jZW5f=HD|Jcpzvs$iQJ-@1e5pgk^Bo{6Qh-b|ryxFNy z5r)HAvgJmYU4@E`ye&a)L5mOP)9W}c{8Q5HkT>CSX$n<$@_A}FqNlXQxcRv4z%gER z-0h9;;?}v)*_>io3%;6}DlVCp&}rUow5{xP$A7^aVJG$M(+`0XL#EU)A@*EXOT{MV zufWQ?TChHueZpxB;eD0mhK|d*#Nply5wUEv=&;^pFFw*j5j!Fo2gXH|<^Zp!_M&Vz zW8C^=6esQXFi<7PL<%&{7+85tkX6bqVw#8+*=@-)IWNzZjCZV~3pFz0_WdEN4<64O zs7#aWZWHIN`{GbEoXPYWP?{2%w~sV`hYbrte29U#>BpeW5r<8v4-d_}5b#vT9@+p_tqqZnvzg8=Qivm(p!}r;*KQC2PI57euy@eZcSQukG)vFLd?p#2XN5&GRTrvd0H%ixL-TuDe-9i?IouiJHf!~cwRquvJ0YAGT4gjhrC)7eF!D2w)*IG&5pN}cL zwlD-UbQSaK@X5XM{M`rM9!H2AZ_*YafG~bNKyHW+i#)e{jDnzPE|($PS|ZG*s|`1j znnNau-0%!SMmh>-n9DQN?ScisAh~ai)3*gutFzu4@#S>CCK(PTd^$a&t5LAX!p3rQ zoisY5<+y6-Uc4zB#W8kgOz-2mSLgPQf@|Njo{98X8q%|GAnpWdkC$f-VUYGuXa`M9 z7X_h>izk$AnXvlw0G<=NEsZj6+{oY1!PWiYOlXMw@EW;y%#Qaq0%ef;ij&kb6$(ww zRU7mu!BW~(VBznGNh|fUqjU3Tsy90rfwwH0na4*{w!{Mm9QK!gpsq&_o`J8@HMO=R z+r3e)G%4K9-J5&^QW^M&E41O3OXh(UF}hZa%@=qUlos0J#rT}A`_|IvuG?}oF{&WT z@u{Wl($prKExw&N&L)@zd8O*hRapBmL37+(fckJ%;b3ti|{ zefn+K`_8)fJ{xm(y;gj;Ups-~tfR7I^Z%HUjtx+`W9S>DKq6>9OW2gKP0;wUw_8!v zVIJjRT1&LWjtlL&;|RHT*LauS71-rTVw-P^#{ax`O!^{|HBB)*J-FYS*p|(!EDk;f zvKgy`X_LE{>(Qb4Z-AL&!XEbe5 zRVVVqaGT;#yuH*i4IypK7te;CLP8J-3gHOTvKqKC1e39nv9DJQ-`swzrVPSC^uKN; z#CmNY0L%b}C=h~Cza`j>QBj@nVC0cNse%JYFF z=VtS$wUNdto+JWg65<*l0h+cq^?V>;6D)059HrEzZ74*$!ta=IY#9Ez!!TTSMhnq} zCc<`$Q^MsuQ38GRnVGUse$4mfh!gJb;ZnHpDip`c3}m8mE|*lLnQ|-zv(fd9aSGOH zEaO;<&R!oON-jAtZ_B<`+oLawi)B};Q8TG&VHR^ADk;6|-4bxk+zE^?2+9$a01A;emKK_| zNEqnluCm}2AY=6T8`9QXbyh%WCv?f3gSuD}u~ zB-t1>-55|ZZHNc7y`Grj3)}2564M`<(~UJOEHh3|M@1Kbji}-_)MBP61HfZW^2Yg zx>p6J7{dV1#j%^uB7~d%Rl4cq16lq-JW}^kYDi4p+3+K1_~mnlc}+hNO21M{CUx(LoQg$Pz`G{*BFH)(kVVpDVF<6(AnB ze^(H$KZ;?S4L=~Pf9@M58^oGmcGyfHb{9370H^1jdlzAZx8t0J4Je#Hz+7ZKk=@6yUk3Ehr7+%wUZS?xklsu-mZyvHga6oYcY8AcVeLs31b*qCqs< zj0j($VS$x#6YH<27dk0@Wf{}phKr^Z`Rh~d@@IN^TAnteUEe~ zeq1fak0!?!4tb6|;x=lBJHKMUSH7!>FEw5^nx_U#p}drj2o?dL62OrUUZ}RsgXq1C zTgpI&0g>A(1ejg|#HRDG1S8fG-hNdwlw+>?P_xmo!>^J!-%Z>zlxFSrZ3jplFW`@Q z(48Q~F#+>Iks{7BvjzuaY`B<41^MfxqeVAqB2Fphp{XbNPaDdBOE*l&0HVaTg#; z`tK*JCI>&KJy9Tz3b#pJG5Xwa^00){H}k|Nue*MNJ6hedEk~iGIw_on4Z2}b@J0#P zA=QN*z-Cn92chjsJz?eyP_+md41!|lSH~gR#HLkcc<50UG(D0OZqwHEs8sX7J z6=S27<@Hwkzr&;+g*v=L)83s;V|p0#x=_`5i>f<@@K{!$1h@4SSl+%#l()=dU1_)@ zQ8^5o^& znrBZb30G-OkfD+eL5h3OfG!Kg&0h)d`?;Raq$ZM4!NK6LJHk^4&`rgP4ARVgq1Fo$ z(Me?oG|lnRL6)2e?S@x9&QOuJo<23xP<5n6V*H*V$=Wxf84p%PeZ;HjeO7y$nA^k* zMjSo!(Nj@9_-kxTaE0kfj8-^Vd_+(q@P+Sq(om?iCEgKDm69l&-?rc~K1%5bKke_ErYQ zqY7wbkNP%*Nz#fQa)iksEz|0~w(h+h5#ru`TZXpo|`n!7qH*fh3Nb?mtK+y+F zq7)_n6dg8{rKghmY2=6*r&dSln%&WQYaJXK|(da@kPJX(MdCANebsIgh zWJjd^Hdy=YzMv}9(5F`cM0I?Sy>uAR&}gJlJ+LZ&e`Qye(-A&MhevSF(s=%iRISUz zmUZED%ZxJkT2Ja!5iL%JMDNa`vHD${*A2m5o@V!>(_iqK`sz%_&-n=#>n6D!L7U-A zJflebKgd%(hpvl6&77<*#jdDi5)*2^InR*Zztzy~)7q(Gj^u6;{OT*)MyV{+iA?MH zLM(1e{jXIXX@$wKwK64FiyB?vo~Ceze9Tn{5k(f?MK>>%get zKg?~1(HKdj#Ia<|`p>=ie_LDdW+9Y5Nuio^#p`q}4%Wve%kp(E$;~4q*kHu$?q<2j zqK9$gD9%Dt;?jCzZ=K}6m|<<7!YNDm9!rt+Q&ehIp{h;0`$9S^%icoW@bhBO7DJ#F zF!%GMjZ(+zo1ij$enj&+cZkfTrUW z3-KVpEcEZ~t!XOe%9hjiPj!XHe~?XbRH}^dY_~{8`HL@^S0ybn^Ds9CUsMKo z4pm5Ajy)%=73Aje%WF_mV%Nl>4hBGy#bi0Za}6?-kY*}XdZ&f7?37ni$1q0g^_wT< zf!B-K_*-r%vbjzY_9B$ZYIR6W*pG%Y4Ghy;bHU*|oSeA>|CPpagY#zM%N>mmW!L9J zmEVXLMkYrSq5%NRT&YZO0-uUg!}oFfPNU6}K*p+p+=3w2mC3;=V&>W3Y4vMl@*RVcmCdbkxL>GJ%pOaoV5CwG^oBU zQlW%N+^1;`O0=LOQM>qVG4BLwl6~LH!M~grc7HcCP(vz;9gX|RSvC3;p>TCnTPuIV zm1ewmL`H(Fm>R@6AJ1{X<>U-&4~p#fY+wLNl369*xeW6VRN2j{MABt^IQoch28 z$l7K>uzziIchJk3^oD_D2Mp^)HA)2qVJd0iAzpCXocm;_-zuQoSk7k;1fITIA>eD! zte06s#J3MpYpbfGwXGYXk_%d`IFXe)E~mr#$=a=gZo77Dr=dKTk)Qis{H-t9d>n^- zEvG+5C_DCX#Y!};`^Oy{``6kU{7H2n3*cof8IOai^3WKY4qn!D{AY20@iz%hi|(RB!r5!9avsoiQS&ej5=?>LB*&2`WFE@WWnUDGT7-a5 zGS7jME~1v8*36drj@uRl*!Qm>Y25Q=OX!5X5LQhysTjb%j-j91r@FFTKm&Fx@F$IA zv17MRcD`z0KWSY-Nyj$SY6f9(WX{cc$D+Qm7ZF7A?8+#0I^xkqCohjL9X6uzG_nIy zQq|WwhbIlKxz@g+*Ae;Ewj|e)~2sqkR2I6qg66WiaL0%tb$s+1Y zcNd$6{4Dw7-nu#nDI_h??HSBnCJQ#FW=tY z)Yfjb-Jv&iV#aIUD#DJ^pMuBey<$`U;`~! zE*%}|)Np+!bo@*d1{vfpo~56}jP~J^?f)A+q+~7hARDGt)hv&fgDMQ~56!Ka;s1nv z<~7Nss97cynA8K&PbZYmnAwblv7~0FCSzg5BU#G{N8S!g)4`lcQGOw2O_7DBGu`bu z^c!qe4*@yE71?!EFi9Y*<}#>W>BPq#BF~Xr4fiMyTHHJX@BH7~7*jM~IoW(~Znk{n z*(U2u2c1*o9>AKu>SLY}v3&W~iLF${jl*u?GeG5nf^+b(8VxKZfgJ=7Nb8K+m^~J% z7M$!`XrNvf7Hl-Js=?ENw-O>_N)p|MTouLcZ4kD4NY@@2Eru>L{FVUs_fIo<+d@~s zI77>Xh3$iiVC;ey$w&f=y4j#^Uq?A4f=EqJfPr@18GIYBW}XRSsw5Z&>iL|J`)UBs zRQv&3QhUQ|CoWyVZ?i1rYQwQW;jPi}rYdfbGQEc(RVv?TXwZsMlI0#YEL)?pty3Q* zJE1T%Leg&m04nQV?42^s(Nu(YIEt76D#)pkWV4#3X}4ILjY<|J6AC$P;GR~6HbZMb z(Y8^hSJ+5xzQn7Sq6Lqxq`b;wER3bR5=A@Q_ihbwF=#iLnvAted6;v^zHPfesA~Irw;`>x8PY6P?$;4OxO8Ouk7OV^mb|8*r5z(th_& z41W+7t+Q?+zb>+GZQo2K6n*)lkjXbG<4}~2IkJGb;a~f&w!6fNYh;%nWyXDT8hWJ# zl0r~q<_AL23f;{h*?h7iBxbTQ_2i2;f1=}B&6w2o8xTq@i?XyU^%{h^f|I$7L!>MBF4Ybr=>CCGw#5sL%M8SS!XOYRTJ*=4kJwU?0Wo`XTMUsebJG-u%ySXM^y&*f9#d9e6FTb<+ zIcH|zmsX0lg?v;oF5oGEKuhn{ovk#+4vUAB&`|hMjVr4FuSRIP_n(^G_32&XI%yn4 zZuC&qJ-;a#VEKxoz*dWAH?kr!IVELkw1`Z%7KF{D;J>Tl^!ta-1Kn)5=pAFBJxZCZWI$TqpJ!lFns}qgqheY{!lCLI z=AuNAoJeNvOA0=1d-i9Sul%`{NlYszc3njvkQ4;CujyT2%W0gwoBe(5;nxA>#K~LbR~wS^go7PBN^B{RWN+Ty1KA_aIISO? zA>9t+R;0%YUxYE9xH&4jk(A@APVF^@@9v^=VRFH7qRqZhi4QUxBOtIg-GOW4NSeO3 zx*{ z=+620+~KRcx#M>5)AGb0xE7#!kn4T>qLn!|wa{55DuJPaKckBU-mEFxqVG{pvy+2b z+RNUn&`x6leIXPh z)oz~XlU6&ds6fsFZ(`GLcpgZQtm;PvY$2Y@OCKNWTrW69<21~ibkHQSucGRlZ;E~4 z9_e6^#mAlKh&%6;rIuU@%JB}8Io+emDCF!i;F2JI1BkND!)&Daei@y)>Yg++qV1GJ z`S1bYv4v+$^9o=OvPe=dD{nHx74w^uEv&<7yzgfxT2(aYp4H>@pf{|4AVX)7qvrhH zM06zj z^#p-;6J>_n$s|Eva_H1ovPIeA*W=I_S8G_&!!!JJt#k>$1t)JkFhMi-6?9>%N%@t6pvy(35e!FgE86H$nkI-}+rZ=z@ zC9T$olxE(kYF8t^&}W4>2FiVo=bH?F4&q;SFIih+f1Ni>q10#o)O* zSDC=`tV86fKN6C6t{ERr;xV^+70t!F`EpIf<#9u-JehwAfVbkkzHqCVtcd?pJfrmv z_f^-K16wz&#z5NL5g7Q+*ph2t4}9d~O*0%gAbwBoJ$b7-e5sujH1|WBrO}dEcdD~n z<`I)Y)C|u>>-nXZH_s*96ac^A2F>s~;*^alsiLNu7N~Jm8Psu>FUz$R==G2GxL`EB z{c)u>G`Q zz;k^3y4=$VY9ccb0P=}S0R)ln?=q;`&daPeDc3XGX1vZsv6eaF*h-}wsNWX1G;ySI z3&aO{b9xY-ty7k#FG_ngTPQ-vow5O``NhYjl%DIGgh05#XxK2I{x8p^U49u&L_`?1 z;+*y{#T0F|(^4e{fQCy-K#B!v(X%9EbP1Eq{N2!4*Uj)344t0Jq z#c@AN-ODg~W$G%FD(4B1YAbBBi(ZTU%A}Ek@N<9zH@Rd&0l8F~*x{o-0!Nk9&X0`4 zeSQBtSaTyR-BLTjf57dBRc+BF+IIWJW+5a+-=0eg(E&g=w=MN?G{7F_c_wsn;E?a<6dW@kZNd$l25g_AR>+ItBmCmeiS8!Qq`7j#w%;!IA zu0ygBkK2J3YYD}g7-9cS@?mC;o8yGM%=Ri%D2tZ90yaRL>LIa$dO6NV6Ig<&B-ukn zA~7u9RcnEX3w}+h8>GG?bSSfk;psfrC}Dy_9&y8yd&+*Cmeta*kb9T2L}=3R#yn5RDBbwGnP z^usQuBy0@=Fyd_66i#tzKe&xL(8BSWjnFyXxCZcxe?ddYN4?*eM*^<4 zu5)ZVX0NNY`gox}&6Od|NdTuPCi5LX&mLS%x4WT#jTlU!Y3sE3*`H8y9X-ti2 zJf#ES}SJT_3%+Td&6st5 z#j&H4q`WbOgjf!OWgwI{?rnh3gIHIuK#Oyqr`9&ZWu#Y1eUv?6kE6PIrc+weo* z6>>B7@TIPMr=AG%U9!=d&p3Djk`S*`Um*i$`e|!IVAKQRtD@S^5>U+*=dRf3P*g+r z%xRj&y$+NNxIO?QDQv@T(8QQSCsm@sz*2z(7siXNAg!je50fF29j8T1c&E%xBrJ~s zvwFfZN5dHsvIDX=3X%ktmgKevGm8s+X_CtFUk-ox2ws*o&;*K8AM{6d2~2yuINT>X zKDAcWQ2f=D9r~kW16&||f$P%%$s-r29%ZkU#L*sG$Ih%H+BcCqv0f@L@U}Wii7g!B zu(yTL0a@Du@hV=<9QzVs#-#6~5eFXXiQC-n`C-y^=p6$fm<}1@9iy3;2Pk}~=eYfs zj~HYME}4rPRb3{Dq~T2qLSG=q#U7mN6GH}`AMA=UAEW_snxr)I!Je!+78ifdXQjpv z8f%Ts1u%M#jWe~4S;>te&6|6Y`KUSvKQD@T(k=+da;wv|2h#F;?3L(pLr1nkR00A~uy*cwe=s)M-68LWc}SvNOQ7o^b} zGtqn!TD!bW)l5njkY|j^M`Y8_qWrbW-TSm*87&KTKdP;qaG7=~&_uA)GTbdh=y=rT z^SV+~-2&f?v2ORrn}qdlM7rR)StMsc?TP{_EM$q5zqnaPD;keiR-jm4e6JOybnJvr!?b;>+I70J3rKj)XwYr` zrqYoZuv`QaE9{C*1gu&X`(VqqJ<>Nl5xgjHC$}()G(_MY=6OB+Dcg%0Z~9h5YO-O@ zX@;oK(E%dwXjgLIE%<7^4jk=Hys4M2p!4UwD%bbQt^fXzJ14sVJI!lQM zB5Y2}AuybVG&yE_#N0c=$-oGR$!giShU1nxM5I5P^FE5O6p!o z#;^uD=I8h!GHYi&83<6=&e#2S*_`KPvM+qLT#5JY?_Y<`A~p;7WvELk z`n9z`;1d6nGAH8qBtV3J5X@!lj-29=>!Br^?XZ-4?(qp!o!vWRE!e(Fm-x0*v1W>P zMS#`gyLb)X66e4E0|rfWKW_9COo9-Ye2Tyrz!zWku5T&w&{eI@5|aa!+}8Isp@N%w zpFV=niHq5(W!8s6shB6W;=hXCZFD|r1p80de?9bu+N@m|EYX%QcI*M9!qq7uE<1mk++Y0SPLm`tLxODCzmLfw7!QQtbLZ|ok{jQ z?sYlTi7u_13#)2*;xSJs4q~klJXl2vpP24$1YiTO=b@5t?>y27Ql=2ej{Hw>-+P80 zNvRNVywVlBX}u7a5XE!A;Bsoy@c#77OFAZF^XusId}sb?27tP`)E{)8UX{zO6#YSR zjW!Nch$Kwkn4gaWdwg4h(`tFkvdC<+wC8h-)ri^k4hFCm1m+!{t8R|3n$N60Qh-7F z%^5l$hJGr2(uf=Xa^Kqj_WXf44x0{GXzFPUD|Ee7Uu3_4e+tLGN*R0NG%jl zk65`g7|NziP?M8-gpsGk)p+{Q*7-RTz^*g=S=34g@uq_jRZF=sT&3EH_U<9l zAsAtifDlOL$C8UWW8N(>h1lFX#gnxyrKzJaO05*0P2Nny&Tq=2_=~e_2n>ozEd5T4 ziOXXa!$BomCswSAw|FNcC>YNB^VXtr9$YxRncz6hHg3=XIsVOT_|%vzD~ZZv@}y7r zD9Gq1_GYG*daeV|A%^_=szzE*+0aB4@AnzjgTu!qES4dl9)rm|9+|k4YtK(S2 zxQZH0+#VN|S=mZG`8xWg9lQQJVLQg{_2k%ESGFSQj)#OfgTqS+({+Z9E4`AIKMGq@ z%$ClK`wTM`52wocSZeoc&r+}Ad_dNT7NI3ngxR?zyq`GYUExn)N!u!o79* zgh(B6AQjA7I7^TDUmafIK3MP66z^L+%3zl#lu#3uZ$}T?x+8xRE5qT?wAaQha0}ae zu%n&^c2^`~&h!D9>hMM7;~~rkd9`1+R75*%=(fwU<`FmCXjiCK0Ceb{s1{^Egf~_X@H$JV~8-iM#ZVJ`?H=B^Nu#Nsc zTJ2Gt*e5?e^y`XcKcVS4*6;(~w!GxA<4xkz0Z+&YePPB$CWSMV;;`veR$c#12%hmt z30Mz!R4QqqHzl@Ly)%8*!NwfGor0P#4cJ7-$^{Daf)J7dT&#={FPd(}S~*kKwrca4 z4-|^Y7>A-f>|L$JPGRJI@cMt!l6<*0&i zE22dr>QO~=QK0?MWW3TcR(ztbBB2ObtrSG%njh)(kVxHY_{BVo-iLR|a(fvHEShEP zqHGV_7*d)MUi>Gn6O!sk_C9Px4M=z+iW|Sqa+8IQv29*O3ws#oJf^RP6_K{e+?MAu z5&B3DG|o5d7c#{E*c5P|Fx)TZVc8o+)LhNfr0hMtRUQ@Yfd_R|kaY6}h0iW1=MhV^ z9q33WYg{0MvTZcx1j(g9HcgeVbD#Nog@=APZO9EOH;gDZv!np&DW!eatDoKt#^RES z9OZC35@nTr3T2b+970uHZ@+Hm&S_Cx0Y^Se!Z_t1iEgroT>gGofRsnkZkgq8mih^8 z*zbx#YJa(M+)*!JP17{x#CG-DBImuc49G1hF(LsW#q?R`p<W2YfU+y6l@wfvp^?3*mGPGvnHKkg zo|^TS8`T)vG~t zFiIE3cBTazUt4ZjTz-~>h}rVDQF;6jU+A&8qUML?qP5a#wFL!Z!VwzlN%HGesIX3b z0<c`+z5SenMe9g2Bsk~@;=EiO z-LwaL-*`LZSh%u}?Ilo_Ekze2MmAQas~LRW2)_>6M&JK8!w=R~FoMG6It5P;!lZ1& zvUaRt+>`3aceIp{!}$aDKI7nFQ@QpFXAJh(80}l|)Yl6heWc%}OC_K`ho~^(Ey!z{ z>4i2Z7f&kC(V16FiLyn={+?`H`QvDIj+xeJ=9>9gf;fe2$g6&`A<6W55PKYufev

nzzx2fjZ;eY%BL4NZP|J42TW3!F^$}*jnNwc86VH#?%p3x1>eaX&PR*~keez3E zD<9tbP)}8KgiXtGm|!z0ibWHYRarp2D}KybUd_V7q@QWudxAK6pLI9P4fqE@uYSSA ziyTD^3TGk+wh7;Qtzu~J2ffTPIs}11#0uc1#wl@Cg5|D{krSOwbx7NWjy7}--gu6`~Q1;)*5ktsga#kr+X!1YSho{ zT?5HLz7>6F500GfcA@O=HIS9(nE%!TJWVOgub6{FU6%{B0+N&dNs$h(4(;1tK(UN_ zlbH#H-yip$AC1=&qf1l^6)oSse~Jq%9c?)=GHIH^cI~sdg~9N2~apo^)vWcVyIyn=vYP=4uJj$SUUdcsm4kJfFnQOYvI{GnqP!B0gQ=6)|Q<- zMCUclc!2SB=-%U8cF)@mI!fO0TWtM}o033S{r+BtdA(_szNLcOAv=1xK>=c4d9KqMb}<5>>M^6 zbfze7=n%-`CDg=(+q4;b#4fR-HR4dZqjgeEiYNFlpkQzk=Tk981njA?d7?lQ z8jFv`R>V5~Jw&D4thi5@CJzVoe1n z5rm}ElKPQ=+Y8ggVgXO5r*R(mSDnpgoY$Fks7@bW6y`aZ75LZDHZajWX}dO_(NCr| zfbxuO?vxGkZaxSR1us+Wx5>qL$dJk{uEH`2zyRj{?v19PmnuP3D#-KTct=rkZ)5Iu zIODM{AN12^Kc!bH&B&d>AR?;u2ghQFc=)B(hnZELJ`S>#O)2)IR1lwU3(-bPKAfUWZjNR$%5GZJ> z8hyK#7@mDl+n`<;o2!mpGRG6zT4X$B#HO5Wa&)EQXVooC;6np(yq*^ltcKRY0*T1`u*o)aPi}C z1u8-;3kMIJw5fj)9aQuj<|Qa|{GKqB$b0pQk_vf7h!tcwNnAjX|v@DeU30O&)V}2m*6ki+2`k?s+0J z+2|!(nV=03LB@IKo?M^F)092v2o6gV8}zmBqgpM9%Y6G>^eJGGm}W_(C{yv#8dr*` zUs4rbSEUhOQN~=!$er8AKPldT(g(IAE@G58*k>MO{6uGj+J&u4#B_yn#b%^|`ZX02 zfRTa219d<0LkvXZ4%m|9Z%O1RID}lC_0ib21Ttp6x~X|#Kdt5-;NjN46!b6lxqS;H zGj_3JBj^@{RE2%+@N&kXdf=NmOLO}89w)DJp000PIch8|Z_@FDmV~#N=P_#qz8YuCM{C zt((FzYC@@9JNxg^{@hFoMr?rT@)*c&K2-UOkt)rUx{6-TlH@?2Qm4Li4y?0JbN0qm zUqdmcI#aTqRFQ0mpjO%dx)Nu(Yuo;9dihNr%xT6dApjnuhOX~c!bsjYJD!A|8gS|7 zGbp^N%Of{JTvq4hmJGZthLUlFsiwZ|hMGwmgv|}1D=L)e-}uavY@)=9?OU|4M~(YP z(h!|l+>iIBl3i7dkd2YyiwV#3ZDlS=z~ESE{t_=&eTRZoj&{(cWXsYULo&QYQnRJ$ z#7wp5|2Xuyt7Y_lhTFGuy~=aq&C8t=v%_YyhkNXkl;cF**;B>T;o^||R7wRXUYF@} z`KR;CEOB%?5C~1Py1%dc|005C+cXr9*sVku{rCt=VE0DGD2}yUsXF&=eus=rb(ZD@ z*M4?8U`U0kTq?O38cN5dW=yAZsk~|+;#5Xl1lv&gBITLmcz7GBwB!2IUiU+$`n2?W z#x<#cLS6U=<&f(NtsPTbz!0K8YytXluWzy}Mk)lPn8YAOx(NW=jvL12H9#d-BjQxJ z>@nB`!tcdw#tgx+q!P5p3H?oghQYJ%svytPkkxfcT95mA9SR+l9k+P>#C>Q+&`DKe0)WL-SEMHQR zS7W_Aqu9x56~zdBZpc7lucTYF9-T)81eu&i9lGWJ&mSBpxTbZ?98FFA{Q2?wQ@5QCd-)hj4)g%m;Qdptb&bM6-hQ7BHmP9%F@6lkfKn6HpL zj$=dqc+?|g4>550(#|Qf5hIJ{Re3@L#pJIz`LZL)*497ZF6q3Lm5;~LZ` z_+K#m+}B`(jNTL}^6+;eg_XPDzw2w8K^x^kBtyf+4;)hiPZHgkUZ{kVC7)?%BCU)E zn>96-#+TDX5~KkQ1ECv52`}`ZbhL9%7xR6ri%SO4H$o?`rfG);H2F&s*JrO>m<4&2 zA}b@?MATG8%8LOQws|M5Z8HILF_NL%Y?QT~C!Gm}R|Fa5zRyEY)cdwC}Qq(BO38B^*4#}EdTVca}6q7{`H zy@ZvCA37Rga*H2YHY=Pgx-&2nC$@9pYFn8mbA6RK^8Wpv2Az28YCcZm?K9VN$-VYBAH?Px6utuEn>D(Y=gN$Roi_Ml>OuNX; zQ7*hcUx@qEO^ym(Ajs{`!JR`w&$iv?Do>uMldFL1&S;-RWGIO$;D*_d_M*4mjkij1<~VZ)!|1s=L?h7+l z+u-Ea7@#sq$A|Z^xAdQHrSUStkSL1`iDS_Zjf0^4vbV@w8Hb!SetqBKY=M^{Ishq*c~mJN0*Kloa<6aH3Bk=2(*?A)7Tp|R%TF06ciEE>3}eGbU=N@h_@a$k40Qxq12Jc!Z;~pXqm;wk3lb>=LHD#^Ry2(7w zW7fo#nn#1&Q~#|I)=L6Q(auk0{*h}vTYc7o@1t&;AN{7_jj8tQ4s@_6-eEsntKsm{ zEX-j9kgtu{^h~-5h1E!;!f95g`w0}hqdYwUe^7gpz6ta4MxY>IU0c| zvP`)Ly5GlhOFNxU1E;rASqu=9C%-}$_WSm5g4RcrUl}<*w!_DzfMQBO$x#?9jL{xG z#4{`Mc|Ny0pnPe(kKQRClNI{;JCya2-=wBh5+I#aJ)=Y)#j|r3OeN<)6-ZIPZyPH* z1}^nREO~<$x1u>9P)A7B@I|_MeOmOC^so`q59u9V%2R9NA&c54iS0W@A4;cmk3d*gx(b&tbl?V#be%e6t$auioh ziXu~wK`$YLgMOH9N*;NDGRu&s3%VjpnCeiuzz&{xCX6Y6h$MQUh-ml zH~KsD44^WJWBvc14VG099QN-yb%1SavN+7ZM_tG#+Ab?u<7&>Hs+&<(u@tt#N;ZT< z{o}aQe#1c*VEq}eeuV&e9@+v!kRKuR|KYTAkXg!RG-Bet44Z1e_@Oqbv1Iz+$^^~Dc0(V&aR=1nBH-}X}@QK zpfC2LA@=}jvLzJq)#L~#2h;c4lITatTsva?JOO?L1h6jD@fE2_?Ll45d2rmM)HsEbdI$s zq;->{qt`0UIxn){xA;W~AjSAk+F~RbIP1n(4IL^KKmd7TzcXjd!eCRcvAm*rI<_U) zh20&op@+?NvI3d%k=|*y+C<86sH2eBt+xQJRVwElv|5koI7ygUEo-2ov()=Zi|bIX zPNZpcB5czhg6j;uJT!6fUp|#vVNgfIn@~$Kl(g%vHSn5$@;m}nol!$aaS=H)HA=*@ zy;IQZr#_+08|&xyq%+FlTNXKs4dV2Nv0Rbgk+#8fia>;6F4l#`lMWxOe8}?Ovi4G) z{VHJ#n)QKJ(Fku2y;v|fD_9a&Ay{Wu2bq5m80ojmni9B(Z3_(S7Q_TP0dM~5^R;3H z@KXQD6;H~5`7=icp-af_*#M%^h3Gt`t2Jb_>tLG}!6m&#f7m*Hg#97VJeQvKu%8l0 z@bZ@4dW|pgz3+-arwD&3Fp!e$UPBOGaKwW!@L12H>650QW4U1`vK;fWB^um|>rwN{trA90bU&M5FiUx<9$_5pTAE?^M4t!-t1^ z9dqbh#zj>OJid&YxiszkO|pH6rekm|-l*-Hc6ZzH)DE^C6TothW}>qxO>XG&*Uo*+ zXBnD;RO+-iJNNrFah#f-Th+1#m1)GBfqQqQ4Z>8{9n}AjF@#iw%?kE*lU4d*;9r6C zpw#5DPXB*H@C8o{##^CF{ESR0GAT7kEVT+g~io9qq_-uU%1C(7xVk zF))LRClKw4$D{}@cOg2PJs*qB-~sSriyX%6+A)=q7x3Eqnm#hoXw!gk(LN~6Tp8fX zJ0v;L*X?Q1<~RX#ev^Z@+M=v-)U~gWHhFEI(CfW&xy5LS8`vpj&PPSI{b?Ge-b{x@ zJeKuK@0v23KEc7eY@aeWnu#PY#&*_@QCq~6z%}jMP7T~TLVm9iDhOs>zq_{iUlKoE zUHo8a#srd!s!Bp&Qnj)6KGrLy`)Q;!sH4moii6gsFS5Ih52}s71`t z#nYXd_%IJ1ql^+>bQ6c8%wIf$kPOAQagQW|DXxr2>gF-pX+S%1Vk#jZJn`8~NRWGI zrKZ?rL%>ZHt&{>2f|ZicHD&qlkB~0FS3~ zt}9KX9B^mnZZPwI$zWNh?W_RHF{s$RI!ylZWCyWT>Ad6mH-GY(g7YkG=^JV)C(2~j=voK{|(=FkG{kETJWK0|HMkb4ldI)YL z)UojowMWS;(`;s^`84s&DiB5J*^{O}aYa;?AT0PvQshk z+{->rrjd~2&TB2hM!VeY7sDcSCPFQHM9Ooo!m#00rpxpvYmhr8^kVw3#C+2RLIML> zRWX4B{_l#cT?1;AdB^L)c$=?Ecgss$AsFB0rG7M%4f((zH>=?Xk5b52^b3^yKhmpy zTygL_0de$f-5BK;D;Pt&*oF09Du~#1^_(PW>^F}$V-rQf43N%&*XkX^isn+IukP)M ze`FW^tbGJ%{?csE#7yEPP66+Ja37@Flg>Mh6wQGGCcZ4OLef8_CEX&w{FdEAY2-qt z3pn=Nl7?+z(0XZ&lQ40YI#KP@1@H}c59vo^%{U% z741${GEcI6UNe4h8iE^`*niX(LLELQdzbf2MCFZ1fc3IxOCkgId-N9+z%|EdXmqH= z5)NDh<#PF{R~UJma8)C>x(sGza`CyK{^yWW3Kz$qrNJT!bl*?`TaFO!^o%L9k6HHJ z6MIFJfbA{S!3@O4)cGIphv^d5EyfRPa8(%G)74}8!x;Szv8nA+H%4y~8f5tr&DYMe14es=ju0_cWGDY*Hb*yj)zlFj=mxc7QiNycKrRZ z0K8?V_IU=8rfh5eps8Pqukqo4x!sNix57B3et3}ER)h~&!z(poG|=SPs&eo@b2`aM z5sUNG!y-1_vtA7?!Ih!0HtA+Kx{n1ZT!2GlW6VK*(;5AL(dA74ZnLh^Pp`YC|9qYd z$iEfTc-=10-_qxm&e7%>4b4|`h_m+3;YFjn>vo&b6;$e=XqF0!O{fS6Gc((>PK1Y4 zFR7$lRH2?)QDtB$jaVYz$A%I-60RKWT}N1jRsY+|Aol#QA2^^KCjYpEI698LJH`l} z0D1y?y#{q`zjzZaos5WoyC z2z)&BBY?DryOy2YpsHwSP-N`a*mth@uCb-v{cm8;=aRH%@g1H)BrS#^@KmkxGN2?N z&vpV*T|b%jY+f&N6xVv57_fiVkqMF^v~l*Y3tQ>0_Kcmw*7n7op%PMBc8QP!DodAO zBqu-ium^`?gSg7g9Zstf#!aa-v|1;7heBN@9i}N_BT*lI2l{-NZ3Pw#?B(-g?bI$$ z#ZN{iq@BK7^J8SyDJX#%?gT&W*)7rO8NA%k6vI;Kg)srhIS%b8G|VS2EzW7;=_;3@ zpb_sGHBy&YevMSe@lD7}&Fc0?i5vFZK+5afA9u0MPbq&`lU8x3<~Ahu2$XjR-Zil| zio%Bu)nr?e0tLmMJhsK?<`%A8R6B;wxe(6e_yo~;2~9+jF5JOZ5EgO>*2Js-1U;zd zj&i_;p;=rkh;pV67Q&i8Fm8M7C)prRxvYwsPSI>_PJH0Lp%)xX%B2#IPM$fUv*`Jhz6%}jQlW|q(Jgg3jZ`uZuPDDiSb4tCX%O%X@h5eqwbse z;miY=SB1x7Vzu@~&h|qMzAK>?47_$G9i{C<&m0o^>x{phx*-O%_caAU@&_BJ8S?1t zVPu?Vi@3fCYpOmzb+nPoMesA}b`C2Y%u^SjB82MmI*kuA9HJ^O)M{P86+rJA-pW`0*N`mUt^Lt@|Jr^s~K>N6_SZ z)WtU9hT^qWb&yYOJ%i@)%qceQU(inD6X3pFnq450EshOsYcV^(-amaZZ%v%qfdN;<{epKWn>3Xss_FZvs*|7g?vF%O2|DKfrdtVcL284v~05d0SP zuyK!YWG0b@U8YyTd5QY$y`yGbxJrQ>fQqo^_iT?0%e@N0(knIg+j9=0uPLa7kDH3B zq@nZvjs7x1Lm36jME?RTEE@`4NiE2{K05HVFVnHFOcxWt^xPtOW929$J1FL!Oj;Gw zkRn_xpe!P@z0b#Q%DBe(m$KKC6tOxxQ+-7$VzE+wuPGNaa&|*z6Uo zB$4?#x+L{IiThYLazoR>v&GyM*T8?$U0D9W{_B5!K?X4Zbgz@jIJU^IPX2;Iuv%WGI3O&=W7MWlVCbaUGTAAyS35C>{%d5UV-y+6sHn2+)@JF)DMKE+j; zs*X{^eG!~f5W>-9J~zYbJ}8h`n*R%f~Q?vd!vFF z_#~oo7z(zAa%S1y{17j$Ri-T5W*Q-XW9_Op8?YoR9Y*b7B$YhF0?lwhAen zab30_P*|pc0yB;FmHM+4SonS`;jW7mQMI{Yb)*>3rX?J9vQtbG1nX73%y)*wR=78@UK-LG)2$ld`-fx)P7?(hDPO-VjN{gzE8OKE)3x zA+$a`k7HiT+F@MhHSB)oP9&cU7{X&FGAFx_IOEejN)H-{gf?G$4SwyRd(GxNjEK#y z-~}ih=0Dvc(#D#wTlsA!=i%fB4lY6~Zw8($w17NmBcYqEwxCuY0oRqa$Vvww6C#QCJ4d9LfGDCa}L)TU>x@KZy6* zcfyqdPFB&RHU|jIjn!Gnbc|HYRaNVG9WcoclK;lm53{tfoknMMs?ZzX)<5_(6#c(% z-U=7RbIg9`SrKhUy9uTG0Z2iiiW6}ya`z1sBra_;<-8p`HY3VG^&isK(D9hTcGk8k zH($SZ+{g(l_**5?XPJ}1@SinR!%(q6sP~=r0HjQUya#{h;pdr-Z{#cdf9cWJx0ti;kA|J(-yacHK^W}E5!0KvgSg(|cX-BN56kj9*EdXH@%`1++79k0|05 zGlqnklmXiR1oDMm{7pURMJLwSe}6aUYX#CWjwh%Ipea)Z28M$Xi5JH|_?3WMvL@Z# zFf6D+PVFB)Q4)O^eHG(Ve!LXSTn$6`g1mM4c5Lz>gJ#1 z;bww}S>yM%0nnN%iN#Ai#zdC(vm;&^t&|NtL)XuLexd<}C|23FPG6~-Im}Q!EI%l4 zk(6@T{V)3#_ihlxjuT)(D{#Wr)u!DZS(tjeldRVQI%EzOpB;#Ub-7GXK#w)S$*vwM z-91-p%A`x78JLoqOTLm8pEfuSJ9sJ-15% zCjKg>j0u;o@D8%;!Sm!8t2>!>X|oJx6~Xt3Dc?R z(J#P|+Sd7z3oi$GP#QfMiBX^GI_%S>QPD>+U?9PjVg0##mpBOL-}IxqHPmPyQc89p zYw^F%VGX@rE~Hd~57X?ac_wV6*78m)S8MPF`z}ThMpn6tog}+lZeJx|rD(CH$ecEk z)>aT%qNtl!Ho@|T{~NmzH!OGwNf02MjgWuq=55AdFa{4yp^I44BrgSj#e$)(CZqH3 z_X0+UWq&3P{)`(dc|A4@+QuTwrl>1HmaT8rhd%c!g=_Bc3wcGp`_4G$d0fXyEoQ1W zp%4!xJs$3dJTOgVPwM*cbVD*K+9&4IXepOJx88F*ZgM?%)e%9u8R~=VpOb2Bm%Qj6 zp_;EO3g{`(gNVNIM!RtB^+fC5ZDG~^uGNeo$+`jcx7t)fQf*)?v!s_56G(fpzgVgK zAF$jNaM_a&C?H@4-lcTP6&JXuEkI@>!{n-w+I#^IFB3Zuf*>5VclsJyfehKbf`8G+ z;B1H*S2J~dXCB4qwR;!Y5aNj~yA}eJO3qeiYN_yHOinMm^7a~Aib}Pdcul2@nNS_! zz6HKo%&I4OuLc38pc7N@Ok=vVf7~9QlV&1*}d)J@*hE_=B$CY z%T%Q!!DlE)Gl%z;+EB2=up}KM`z4LJK*7CCJE8XyBV7ojF?y7U(lg|PMiw{3=gIky zf#dFC;mNUcn5Vfiw(Z3X-yETwFbI{vjZuU9VBNZEuvliJ9QWiW-tJc164RZ-1 zR7yh5&1~As!R*I8>+d8$>s)#4wuTHrBtY+bnt+nGc@5ISD9{dCo5@g=AOnFm$Qzvl$ zM??X_TWj43KE*ICrs1`L5XQS^^2Sj6y%`03gAM)_{E|v!!!j|%GZv}OVYg9k_{a|= zH++JkU?WY;x++B>iBmbaDhfl4+YxO!sp?lrsk-B~sr8JJKmbEPyua6(!Ubwft#Bw< zL3El762n4B{X&O_h&~nG2syhwb_Wx?B9i z9A!0-)P^{ZlG3dWWlY*_I=H%PxM4h+9m{dMp|8Ya4tij=^O%+0CwCLo0Mk9FVoSKn zNEfmD&Q=4U-KgG4-{2mPH zKJ$PSgpE}6P|NsJf8)p@S5)3Slxt^a9OZEa8^>7%eca__I-uC9a@&bUlC|4KBUsd@&3~l-x%X$3QM(Oqey8d znP}$yGP>LL6IIDXjXx~y6S%l3sZl(H-SBd%`bRCQkEvGUqtMD;n) z3!R?7gu=>R!~fs6%?V$JTuBfDJT!Fz>Brep^(1bsQc-$V_h!9>jRImFdy~&Hsp&H; z+nXb^ZOlnchOATFp!SOHdiMcV5l_(`4a&KBsvl55<3>%_m2De=O~9v8yG5c9r4{`Bf6>4joQf;7v+ol^oMf5y-@cU z^gZt)K%xDkdu_pQl+%;AdQ|b!@0PcJJ5x@yi?t^tCTp(J&Jms5*W%n+$~}GV+AAh9 z+MOcu@np}mXJ0$haT5SfJCE>BBm7~~(5e_BacVn+ot~B~|;uqVq1 zYsFkOtu|oaj)VUY8Cbo+J5^ycEt~`T+ZkeK9P5a94{g;1Y8_fWM~kNZH~%F@LRKyv zAp}X)gYZB>BEJw^VF~5@?|fab zi}~;2_%RUw668U&UNk|1nd$^l1Uc{_4u~ZdmYR2l?;>NkRq0y9dT_Kj#g*W-TcSIQ z?Od2=%H*_^H zgwt-N2x%m%MD(T>z_!hE6j$++|b12BBrpI5P=ud%zO-V)?2c; zt-y1{@#G>*#&fiqO)aTRTp?%aV`ruMnw<3X9NVuJt)IExxJzz8(1+wRFPz%yN%1E+ zM^!y)gvBzZ%i>+1dCefNwFKct-eBouuwyt*pUF5C8+t&E{xnOP;-}OTf|R2MEqL}? zxQ;_8NE|GXTZ8&Bqk}_S8xBzikVef)LRUoQrw=BF-r1@a(yRckpw74nHaVdtxeHet znM&mn*BYHnaHvmFi{rjEmd%7hDfa&`==)3pccp)sYEP}tgAi_dGjvXL9tH(Puwi&LvBd7WZ6ino zo{L=5Mj&Le!RXETth0Z1AkI0mLa}Ct4m1@2N_EL0h_m%%vbIC;tQ*Wf_cJLc$b>{NV998Q1KJ;H_u*%B7K595mdpRkL_paQT_%3#c(Q( z6QKCfot@-qo-|tEcZDv>pa`?bq>jb>J{p&>6<4EY(j8YavMy8jLhcNNa8xO(KW*(I zMCciOXog6HF&OmXcB z8Z&mat|0CG#e3Np4h7#gZAMu+*HF_}YVrXvbB=3&3?A>3MY@m{$*n2S0HC;4G5p3( zfH7~_DqqY&a9>v7sVp|b_nzW3M54?tn&7Lv>m?%V46N%C^D0rFO+M}>UL+!sKo@no zvx#DWP)AhyybMqoUh*Ixe+z8<4ruFnjpd8{cp8g5opft+fnr!Y#`3Dy2lVjdidGZl zh5!^%wN?}}mD$z9HDfDjcd#}Jkrl93Zvn0EXSZH_(g)yh-e|)Z@7GX&=T8H#Lgl=m zwK<@wRu26(m#CmkT#Q>`rc-07#x8zKwRPAzFYje2!Ie>k9ECH{)I(tJuZbn#5J(y=PZC~WW=N-V+djBRqqc-1 zqV=+l&p3bkI_LI}9I_8TIG~wmkRss0x6z&2%F_bX#Dk0%RJ1F4ab9roT^}FOS$*P+ zd&R@6zIboMtp!9E5hVUp2yYCR%c{lZz@)i7_y^OJ>AfBi!kuT z9@xfXVKR8}H&Kr8eLiwx8uQ7IAYLivY{5tBsgLNj@GacWE*62!hJE-zj=Z~EAQX8$ zg%pE?dz7oD9oOyj|Com&pFgkR$`GHz2c3sH&! ze-<6Dt@OR_n?K&*+mZ6&*xVlw*6fhp>=e=}Jj?hAr&dXHin3W>C$ItJF&4GkjH*B9 z*HV@Hp#$NT6S}y2rLW=EKy_c4OQ{5H*_awhi!rmnARj|#*c80pp%|IEr+23z7xSDb z#2OmA1J1fVLwMjkZ2NC?kkvDG^5{N9iwXJvGq64+O)bF(Y(GQp8{?E@CPM&}nN{pw z(lTiIk0HHtNb0B>p9Av5>TU-svn}?Mqk@x3>Z%0KI~E<4ul9UAwOTNSpQE9Xt62&Ge3ZM0?FgYL zBMOxg>RHY7mAGZ^uM}uKn(0=>bK?;*Fbi^J%QXO>fuKtw3OrSUr>Y4bcMKH_*j-{L zLs!$zM{zOUhze=@7IZ;jWYn}K^Ehq^iIpREH{r^}a3lD$jH{e5>BUO^dN1Sq4tSCj zjdqYVU*1-bqcABGO4T*Wt*1)*^_$>z2X{{u5yZLS_ddVo`YxPBG??}27i=uW$P(%C zCF_4P@%Y8jD1|O>Ab&48sXxK+FX=&tHvZ+jr8wue2k~xCxm*W?krAvEJvW>Jp))=6 z)}(!LT9;2e>vfCGTGN-HJ(Mk-%`VK`bYhm6-qige%SP!yLU7S`6BjanNpp6C!0v`z zLabuW@Z%4LB{8DQ5KQRdr%HAO^$@R=eu*C4qV!QgI{jUHUDK=)1@H;d_TC?v=&Vo@ zn*Z79wRT_x)6|eb>dW;2u3WS9MKQ3a&E;Befy1%i46as*nJZ)cSZy|U~RWL!t>Q4K`ZNF zKmH>Gfy|p@LTB_8ALP+&bjbO&zWX#|G495=IYO{s?pUd~?y`)YRie_taHP;zI1hwh zNzQSKCaE?cB6T8|V)>h3pti~kVAwDXk+_{V#&|UQS#6kGuEGk@`66XL+d45#VeO!o zFyx;W1X6tJp{KQ9BCz*%YnYY%SB}ohw`7kQ6i|-TXt2t;WT{|*xVwr&Sog5otW{}4 zeO9Jx$F4-h+V4K`)cYN~6X*#{AS5ExHX7+BDp4R{EI$43W#V*S-1~9cBmiG@7`yK! z!7?F+?L#aWFtug=5~+3AT52hOZDmV;|1Dq;A)*0Am~|0J$kYnjuoK0oE0%AV=}^x} zAWIieN+nqOI@r9+QtsbV5&TFU^QiqgW%W>uXa^vZVZM@)hrkZd5VSj5;KA~@=Q62P z80qcM>~xiPZuHTDLIGb4m3Qk<7L*FzVXg!BeWM3%R(vBFZKQXzfJP9rZaun!@m}o) zR;hQ)aa0y@rFbWwh*N-U?!aF{+-9?5>NB^qu&Se3n~ug{Rn1E6zQ0+VRMdPVP-YE$z8y2DMI z`OO@`-7ex?2W5(l$ziL)+S(RwNoj0jB7#oOnvpw~)~fClsZT$XCE^F1X{av-ek zjqbG}hBM;B!jrvc0h}?x9W+S<#;Chl_6;el2A7yiAIKbrQe6X6zoXZROsvsNq^cI1La@n4h z1sGJ6NZ3sNGUaV|ld^(4savSYv=v6tdqSK}Bjp8fC?$Rn>r&6q(H9p9K~LuX*$hcTsHcI+BfrKNk3(Bp zx#U*2kd1?UX~PU@J;G*PS_x1j1CZd1FrhC6fU3wZ%+nSGw-h|-ixaYz&?}|b_f8Sy z+>JxY(OA7Ruqc#bmV6eb)jHcMR#8nH6v-j1>fbF2gIA6DSKfWLwtP?ki z#o~<*6@)i{)p@tat}wAof7LC;P+1Ra)gI=SS05lm&Ng|T0HI%|`>(xu1>Efg*Aa8` zx{=gD*$u7T!NWVDy)Ov-9QJBWQ6k6*zT)^TUYySk{+Y_qp`|s5l_@r!e{&O->nK7z zy6=K&p0e~y{Sm`JDE;zY300NqSOC8Os+zzVT6m{nnFb6IL$_O1C^>=K$3UQe2laX< zpiG~@E;zPqBeD>zygwMe0;*P(MHk_TBm~PIZEOvWh1n8UWsc4 z-sa`?1m*IMqO;&>6(t0Bs=XMP+?Y6q&Yab)ew0j&@s7QHX8Sq*9}S*Z?D^0C1kJ+1 z7&Nry$Kz*b3(lW?u;jse5+et**Y-3Uup;K~uXt)%j+?8G6TPeM`8wkASUeBeY@jTd z9y^)5f1=g%NRWY)kC)Fogh-JaklQ2X)Dvb6V=yI2)OoKxI<7AuVt`!P+I+D2o*If; zzMuB!Pty!%Pf$mv9^tQ57AFI~)nS~0cW|)_DYu&M3Nf_YW=dy4Kc3LC^?SJ6(Y6)K zkZC=(8LlTbe<99>VB#4+VkKUjV|jIH+(o8i&Y}Dgrm&ZH&#va3KQf&>;$%tNY5=c1 z;mCP?Sa4owH6Q-$pI5_zNiA-pv7*DDpp9+`X}%Cp2&PpTfL8JyDon(%HpQKRq|}z; z1Fx4H_3&^=Wk{^-zbd$^iD`+UKQ%8n>7p%sNXk;llL4aO z6e=bVPiy1~p}u?vv=|NTA{*eDKS`o?v@eH+8u$6G(&L!5n($n4jCa4DXU6Z(9dtgKUET6N9@-Bs6=etT z1IXx*CJHvb3t#VP)kWY};=VvK*D$5QKX8x;c+U>Y`@MntYM*WxYV-p>-jpzK71GLM zQ`~dyE2coPr17s+6F(73{OM5Ee2|^8Z&REsIa0ZSSM&?Mxt@X2Bz?fZK|0z|4e9qabv!fI80 z!mbqQVXZyg4?gP1TV);dJFrT?WX+FA`l7P~|dTXzK(lck&aOo|}S1rj&{c;7d&=Eb=mn z{`BdLVk(pS)f{rjTx5W^Si7-s}8I3i~oOce1?n0-ka;rjCu5+Ws(ns{ZllB}UM<3;ZXXEemZC7~_!!Y0sUX%?U$?G^(!Sxt^{*peHj`ghGgee`THmAC8I z;jU}(hkbUatpjf2P%6a8rZe*7;RpqFsLX#OT9`+DgsuoOx3bv$4_ns$hKBhvAfQXw zWm+%V8CIhk@O?p&$wJd%+QFi>*eq#BG^fZjm9aa)y@E4l21N#4&aKg$dUtWTTbMH0 zsP1?5lh*z_lq!f6#kb4uEs}b2DphT(6;Rkm+Z%$yRUuBx!oWRa9K4LAV08tc^?5jH zom@jmtv{<(t?t|aS$EHPCNVg5;3a_30aCdfg?63pEf-b@&b3K4vLG+YKhh%%3YS8v zNnHGtw#6yb1gDfR8$H~k8hFx$u=9au`^Qx??k4xm$_>|wNjZFn<-M&tIpy_xUaq3X zyI-4bMT`QdW4i4)J-DOJc$g=jO>-Ie{xhF4<>a%-`#)e(N}W&K>OzPOqEQ(D*CBFr zZ(P+yMANaB@G@HlHL?@nXn7{*F7b$lVq~mRj#j5_RIL6EvCR^wFh4*xYpKL-;pyg=~1kC*hchJVs)gQBD?u^c1weX_h+kT=s(9Saz-Fx*G% z_^5uly&%4+jAXyBsI-j2HVB^AgDxPS8d8p*$)3W}ak9A@Z77pVtj7wbPZP>yEKwiy z<4tIJ-RV+x#aN-xOc9v3$tx*#_n*{cwRYSBzAv3KgU5<4ICiVKLU_eDCj4qd%Nll&A#zeK`p# z+QjuV=np|WW%Vxc?BBZUG4&@Bcf9zdZ{3s9T|h0;=ias3Nc6hsFLiPqT)N690AT<4x_gS0{V4@(<>*Di6t?cdIp2EmXRO|kmpxfibp={z`csG_K|4iULm0qL)4cJTX?U& zJR7f~KD-(-3e~c%xYw;zRgx5n+_7oWdIF4}n{F2KsYG6YVuuM(v5U_>GJaX4THQYH50>~D7Pk-$| zg=u6bXg>-!L?&%SX!hHepAj0v=>$zuHS+kaSmTt-WdSrtyKXpy#bFzC52#-*jap9Z zKYyt;4a^gl+ccGN@eEdy@m6r5xt1~lYeXhTtZz_G=3c9O)FPyoGFogxyhd``R3HjA978QALdx>L8*Q5@JJk%kUtI0av|^dTRFHJ_Q}5 zrZ_R1N;3HvW&Uj!3r!Nn?<=XQNY1$=rO_UoTjnA(7Qm=81R+@!rK>(Z*6kb>Vzn*b z6VO8R;Z@sWt<20yFdV>T&7iiJ)_1LvqYZ<{mgth$8IlWqP+C+hYB`AInCb~f0Fge2 zzFSwwT`NU1DdX+2K4k3&?Q`5EMhbNO1N6dF7m$F~lAvSdiv-6|54ohf9~aPoWw{mv zeMD9N0P6gS@MrL1Ti}^~cM2liO!3(8H)kp&gCw19!%*ziNN5wCsfyKI&I)4+9mxAq zZ_wY`YJ7IxQy}!ikvUqZ1a6Coe}}5u826vy8MV02qcakk7?-O%+5q09Mmi@rwCt<6 z+-nrDk)}1Pjh%<<$;jeUzN_2okqM+Dl|rWDjdT&zv5ubZv>pH5l4#uGy{!D1L{d?2 z63wA^rxN?AR^HB;$yS9OYmUxD>LKD54`tQFcX>g~UIV{D!8syodVE0T@!k`zH*7Q< z<|5BFxMSmr4MS3eoxT(C?$}Uy+OO(yNnd(!$Vl|Ft&wll^!H?~z8$HRBx8>~iVYVe z?nI*!nl;H`eT97TUYnLHnOjmE8Yo} z?nXKT=B9<4H=$_7EJ99H=dbGOG=PTQSI1x#z$5-F&29Rjx-_8d_fX} zm9HcWQCYIEpls4GKVEn*%c6p5kTxLIjAf*lwa1$AfQ|~}H=a4##t~mkM>$W9=Fd%O zysN)W{DGugbA0~GLcM$Pg?r8*T2b}@FmN07akaiYu}7LgIrV$Bf17quP+m1Y>+@cX z>N7xU`l!ZWYy;>r^Dk=j%QprhCeq%z(!H0uw{6cw`C8JdLy0s9ZuAoQ1B^V!H2QS& zvH8Zc>%niC0hC@PB>%LTiZ~hCv5{=db3`p&D+r@o4^{Im!c9DtBeD>EWfuM4M!>HZ z>Y4CG(x%VsL_e!}0xcoC6$7C;SUpi6E+-&9#TA-b6BO*{P`JjTamvs{&pgKsMH1Rn*>weobyd&@$T(eMKG^jUm{n0DE3S3;se0*PF zSqnMpHAe0eRV_)44Z&B<*)H4SBx{^w%N8#%AMIbUjT)Wr%Opr4R7|H5-TVpq+$f0O zJq2r@L@-t6(@ae^W}{o3a1v1t5i+5O@4_D~^})v&Z1(@cahCB(7W|>dFrbBAUvdHa zpCDuM(vu>pbIEN-N?G;Hyegze+%Dc%m|vvKn;OPDxv`Q6t41Y3!j-pxCn6YhdFOU- zHV*t^t`~{!7%(C;<@LA9AxCIxl5UXn0K&cmI9wg^0GN;oyEkReD_QCl~hQ_1QMw}~Zc zV`sm+^7NIj91W`o0+N2vjn~hS&A}tziVc-(m@A9?*@~L}5(77wzaF7d#|Wt$&RONN zny1DO>Tdos5I)I|EI36a{Z;J;b3Qmaly3oz=;s_xBj&q)@tWJ?xNr%AAkteYFVasA zp6S8+QK3OnWLDn*)K4(>EAkr-sl$Vzc7%_%z$9+xXR=Ah}9`E675qbelFy%DrR054Eu4_Kb8{o z2_23e7t=D&LmkibmrtH^;@2sgF(^F0fyQ1p{|`TM#TA_Wg<8-X2JM32Gb1alLC$d2 z#3^#mzT**#1nx3?T{~a@;?++cW%tN7x%=>WVqEL@bdhU!j-Q)CzSzzTvjS2k)Ke`~ zpTge9zo*iVKh78j{#OxJkT@Kpc`%|F6OlO`rJJq$+B(F``&wZIU~VhC$-8gLefn{h z*6H7^_C(kUUKWe`fFWZ=TFwLb07ZLLO?ou4CN>sn^d zQU0mnO!UIW5-l1BMf(p5S!A0~hGh;sWnwb5q+%T-@~NHJV#lI|CU5MBdYlLfMKUE- zPnN~93{!Xj#v3SdF1OkhpNFj8kQxz!I5DLA(~X zLoQgAuYVqrY#~pM|@hPl}_tnk76hYMd0O=4Nl zy{7mM^2!mF?&J<8(pUNr=rbxX3;q5jfre+~Z&@hgbrHvITZ}H*1&bQx0xoifq8q~2 z+a4hT!5_J>W2@5LxNxPf;zIE5-L8*5d3&pThio?PI2ZseUvsrF5hX%p3O-26@Ls>h z;||z>S24P>=i0%KS+Wu7LM>`nD|VlkJZQC!_O6@}Ln5d{_fShId#%}{k>4wy?c|5H z5v*y%6cODrX{3jFM#6=2y1D0o%Sl^BpVEqh%GQoNSP3u(&a%)dsP;5 z`@u9`Zy;M@SAzpkl~*cv$h-DuRuIC~_N!Q6P1hbtnz733Tl8@pUm;b(&b+RK#2icUr7%KCd?|JwMhK~T z{P0g_@u0+StNM1LvlG_lD6z9;g|k?lI%7DL^yb(mQ$n4JvYlQnuC)lDNVezTxYrj$ zdDJQhEE;Dmnu)Tf^>ILKK_+I*fwgKPq?dR+0)HLR&zRL}jYpZ0wT{IAbqCvnTydtj zpcLd|_6zAVq1>+d&AN`% z8VY!6OKva+op})?{)OVx|D$r& zl^bfky=vkSIBkBJtm7nALcvYDvo+A??d6FCIu^a0l6vJd(k5w4Bm{jYLhBTbl=8ZA zXNgvyl~GF%v%*26x^R@^(pBc0z2})njJc>bKq6{axNn@zqL9FtMf~nPAPCHX|e?_R)k4dn70# zf7fU2$7iU9&AS^;Oyp9lpI%#~DDO4GtMh?^tZq+Z<1W4Xv~zkjNVZ#?zTx@F{s+Wb z_#_e@UZ86mUGO-#G-tx*?s^B04)Y+fh+V1FwbhI|(=!Cy&p74C)O57~Ht90|-aeYy z*bj-TFi_(LHalrFr`;Qy#iu2e+wX`%n6MG&<&eI?XuC%RXDi-Vm=~{Lm_TxvU5zSs zu?CrN1*31jd)*rC=n5!9_B4d-5xC-7@$gVG?c8F+wSd*Gl$*}Z>?Y7ZkEZ_YUZ;9n zQU*otC47ozPEOb^Ub^HXmC!GtOI)A0xvV0A{(z+ zr-CiprsdM88gKg78()dI6TretYgDMH{GfVFJeU$UmpC7FrpS%Cos{6ajeyze3X&g3 zHzB}JmrX;&fny$t&Q6`qXTKXC&by7(0`kg8o_n4w(s;?0=(pY-c|!m;NFnit74_`f z1l!i0);MvJ)}@3XVYIAk=_D6Kw&EQfZ9Gng73ccl=-62A*g=w*j3z?LhPow3lZnEA zvdJ(&Ne}KIDM~-B(Yy6-~|;Mp&AxlS5>mRD~d zbDeu!uXpCo-;0u4K~tPq+kAZL{I3}{KoI9|3dCC-z$9hT*l?X0toEf? z{LlZFeLNw)(zF|MC%un}G@Hf?QQM1a=i;pG+A!8Rm9{@`RY*4Ufb|}_Az*AQH`Dhy zhWorCzbS?J4QL*`~p~TA=v7Ro@Tw~U}v_c#BD1e5_V=~vpxXKW_;jOMHWc5p+f8wp!!jvKQ z||Vj0G}>h$P(r181&|pybj(v)wpBNgc3f-D{wY$)u%`q zDirV=Bk+P*w3pbmJ+u{L3Bb-pA&cf*VQNon( z@C}Jr*G(-XRFERQPdfPegxMCNdq~IBz+seIc;a`VUuIPcriGdAPoU{M zlobydn5?&K0YIH>>U=i~U)OVzLEA?}XY@I~LyZ<|U5ei0Y}RQV^x!D^cQp!#T# zx2T{)?Oz{GP<6JNr=I2qd`YZlmaBkr3%@uuf;AR2I=wiykb{TeGMk~dLLnw^ADc;& zqd<~Qs3>ED;2?Ys_IC^dQhz2@2YpMi#At!4P^2ojE;ZJh02kU)sh~kObj8?&ku=-L zN{TKDFnfq99&}1ai^pVNrGwm5Y5o&;AvLgDT3xGM+i$DjT|(+Oz9CF%-Xaz=9c`3I z@#Tb$4N4Styn;@jE9Aph`QznPoyDYpinn^YBg^P{&w>Cz$oPV0kp)Ni0K*m8BFw_h zcO{xhvZCF@>frEth>Eu_$Jg8LoT?naPVxQW{5W(KNv|(Dm1G(n#8MEIln6KwcMu;e z2_B3-cSIV`tYfC!-;kH6TfotSP)JeQDz%9snee%M>M)Rj%jB z30k=bbDD$}HimpDW;k9y77k*YyAPLv?VoNLu$G_cJ=D)4qE?}?-tsbU^62=-lZhbA zawS0GTzKLw!+rXpp&w|&0Zsy5)q&M#tst-aSo_f?C-|(&3_h6{v*gWRdVHEE^&kPd zUS&oUUU-Hc>K_XoRB4(ExaZ;_&6q>WaPk<(&CNNEgo-ZAVOCu$t4h7WbLGOfO==py z?fSHnqDE?r3>s?m5j{rndHd(T12ABQV8pjFcJ#6c)p`-vg|5ZamidbwUa{s@$hV(e z+vU<}Swv#$*8nvop_@rQHYdckvvI_L7Bh;@^}ME@44h034R#_dS-6eCx1bvF@)-dS zcZ`gQ`nVx)i}6;JBLsP+HJEPj_Q`gvZgfHHD5gH=qypa>2!#ika7+E%3ZG=7*iW>Q zb;*RHI?BMXbQY#i7UxAnc7WAOl=`;A=$d^hO6VXMWt!_|qPB|!$2}eFHX7!Ot8tV; z2gMki3x=?JF)@LSFUvRy}DF<`=Q4CMxEWc$-TtAXE3C5dVF4ppn^=Qzgr^!XJrsCH4ud zExDD)HJLC3J9>l3(ag?RCuu1Uof$w)q)PdRk7UdGrX9~XL7G96)LHvt4^BM@CklvG zjO6tYvL&HQ9!uzmW$lyBHEl{p-6!BNf(XLNmTb$pqn7obsHSxgbfJUqx%X6x_B61j<xaAH|K0yW~*25@gN3q z6|-$5O#XypWp>>{D4WWYVH|U(zLnmrrgww>Z zC%Im4xcY}mc#dBv{D&!$h{Z#sSfKSXDlY?p7SyX^FmBblPldM(_xfeA&IthJaHLv) z%Gl%WT5FBct{GGhLMI^!9JtDX$q#xrBrdXzceO#LYCF1C_Ym1t;I!K6YLli*mL&LG zIKA`6C5VDLVUUMYUnvbqqXtji(6trT4ymH&3eHv&Z!CfAo_9xSeCPjJwgBX7Q#s?W~}7apgh>gV);@6Qr_ z?xY&#*eD9{>j9Xir3A8l#G|rk_U3t9MNR$LTVfg_Pa;itj4&LpJ!syyAaGICc&Gom z$l2qs%Uh8BID~Fu5NN~ORfOYgq#SSoQwhQSkGFd3;`1P8txbEqZN?$e!4atHQEcHxsjHi#g5b7bY0%poz*8qZKS zUxMGJ%9KDxEr`sqmlHsBqEkcS^_9e?oL=ZNJy!JHVn3+prH1T)7Z&9gv3e3Yu`%9P zoZR#?maOgw6HR=5>s2k?c`3;60Ac`Lfzcsr+25`5E>O$kBag?u0JPwW4Y<6a`n@PZ zxZpM>>x$=%OHmze7p|+_ed}sV^d-uhMK=WoPj;3x@7ZC!V1PdV%Y}wvo!>owe23^Jbn< z?vF<&zA8#Ga`R>)F@G9Ssq{RiHw%>SCkv%`hPKQCK_pmm#Q^!ghytqOw2h4ahRG8~HOh~^}dhPJ%9Y<~mB zlXK~TU8K}qDAe#}bvZpx^;T70F#mjJpBZ<7-nM^TSw&Co=;5K^{KxGN#{A(Px7~pa%^;8v1LhHB3V#vx)W_5;E^9GUqwaGU_BDD& z#yc`6ms9tS#(6$q@>%&+@aP6Y9roP{U7QI;E5V964I%0uhR##rCq_BXej65URG_3& zbts0QQ)6vW7UgP#KjJ0g<2*m3D$v0A_nh&jN-4nW(9e!8w>{#4v-Lc zTr#HbXrg3H4E7vwiBe`KIa3g~CFMn(Fq_;)n&(O4x_-(StF$a=nIN*zvAUlYY|b^k zIN%J@VJ3fmfs$^2K#*+|#oiqwO?szhtp${nX~i`S;r1t|`8+^s1m(1RbM5$_UL0O>R;gWR@_n3yTBe zKLr~jI6lC!>x~P{p~f{X1zfg_{{B1tyD|gHB&-dK@!-BJjZuV8g4^`9p987u`ZelD z;;ui7EStjufyi+O;*TTL^Z{Zo*iA~A!CpPlzl62F;V79rNmD>rp{ zQh`VnNIKOWmJr*ir+B2rR6g`6sI2n-MAW`gEQeB$mGX+NQoI2)^2wr802^HyS~(F= z3bu1JwX$*f?McsJ_S{34#cpS4A?Nj8HbU+jaa8L>(w?OD5VaTaJ3o}qa&gzNMY3<-E&y)(=^AqYO{7yRAxT9_Z;V&%;+#WK}JVdH{7a8bV;}a4K$D@oWu{5?`~CT2o&n)-=bp`pC*Umr{5nz$wX) z?ji&lu5X91v_ygC*xn7Xx3my3G~Ku+b$&eQ{IC;0$C<5F(C#iPJ=k;S{_3Ezk=8zT z?*IioeTK=zobzb-aUeTfuXTYXmwC2&!~r?I28F`bzz-;M39c&D8f;y1#$Hq|J2hHi z(KI|Mb3WJDr!Q0es2EpKH(@{vKQp`m|GKC0T_-cSUWzzxwA-Omb7t<7)t$)LuC`db22?!3s&>_+d@ z-$QqY<#^R|A+L{sa-Q68OI}2P30|@J8u9j-z5y5B!|@{u1iqwmFvKkpSiv=!-3FRL zXkZeEbE3@-zUk~a=4?%5tPUhz+uwwx9p*z3dD2tE65NFa8q3;?Uuan^C!o#2(Sq=tyGuLjV3DH=(!x8@U)_Bbfx@5iVwJ>LSkV* zH9`0$ttQv}`wMF&SM3BPsb{d%mh850)t#NNk!Ntt$7rbcn1>n8ilr`U&6?Pj* zPy~knC6&$%R$j{H%>qlyVuy~u1uyry@nUEXobR1tfi*`O6i=tzwo6IU{(&m9LX3dt&{oO02xQ|oT%aMJH^Sp-GUED;HUr-8YJ zT=hqYSog$V@qv9E!1o}yVG(F-rI}YfpLqJjmH$2 zPKQsN6C>^vWIDd>rQ>9(L`43uNdxN!>1J&KFYm`aPYdQsPeibGUO7@2cP-Uq1f6-0 zU$ib_lRfh7YJpKza(gs7xWzA=(DQ#SPt@#qwWSxtTp1&E5q|UuBKDW*ZE{J*?3tN- z`9O&kz>))?Dem#qC4qc5P$6$$_ZtVREI@g97kCsd{Hrcy?jGJ&s3(H;ba6c);((-;gS) z1+O9X*oTIB+@PN}zjt{PEG6h?X5NQ^ zAM;1MxkD(yb+M!dA7pFaa{|4!s_AM$+egy-TE`OY6Mi93H54y$WZ33m=nK+VZ^j5uD6WZ>Iv>HxUA=AGgg2DPN%mAK9>~E5r(pbQ%%pXLJRo;#OYdEh2~y8*JA$-#H|J*vy2>^j0~L3_L6(!j4bmQd z-u|_|N^Rm3P^}H0dp=;C0xOV9aO%<=p4^mJxWwQldkkGKB|iKnU*z%dJHuSFSuN3;dRK!UBp2e8%$J2i03wCD zL!!=V;^$Ls?=-)C;_dF}zIKZ@96)#qu5A=m^<19? z%t-$?SqiA9C>j}FI1w6r?8N#`gRmJWN0%@#;Et2eD@$B$zb&7KAqhk~1J2G{BRk9C zDy;_k??)k#GsT}ujqBnnB;+{N4FsA&IXW0dlc|ozIRx%y>=rM-kg2AhJbt&GzMuza z?9u-1kf7#p>AK@%PQaVm@-Y5kz^R1vj9Wwq0&y5gzjY5);@YAi9{}NN<;yPBh&|Q# zDuN39cCTVR`iTN6TC)DR0%ByOn#XW8pLJKFhVVh{K6_I1-Zeno`$?l3@F-M!^5Ra3 z_)5K(-Fb8@j&7va=ScaGXc>++9JrZsZn}DimzeKyKe8VVnpPHnGHjJc5xkROj70rGVlTpZflw2%ko)sBZ$E?Jz^&O))*~MqkUx0jihy$PS@-PZB@O|mU zxr*c$I?YEdj`K?#2Fv`h^w^~7)xe&ftvEX9kfX3fX7SdqgZ~`lw5Ti=0{*#svT-aq zb4P?XOiXLYOI8>3-sb{%NMFOj-6Xa;%~*93uy$&k(E%yvuVjN_i*Q&cc4tN8Z_n=V?W=LttfZTEMHcU z@Ewa119hU}Ktr^Js8fcQA?kJ@YR(*B>jI+=TL%IS6-D4-i-hjoF;3}SKYy2%gAPpO z1k`g^zdT?L_s3_Ju5Fg(BOUS5(P~V9tFF!W{y4Z#_UeqN5i{C3W>=A+tQ02N&8}m@ z-|XwZdLe+o{IA4~EJe_qd%_9DnpK@lSJ5bC01rZ3w7G*dJbx@TR*|>Do3N+EW|LeL z^#)x6FViTQK>YDpIQy%TZPYZX(R1$%^!X{kjC7G=2U_A3)P#?6I!XvZll+80vT_P> zUtK%NBq7-C_7FVU7|IEGUNjAvs>*J| zr5QGZIRc=1nZgkswk9!Q!f>|Zq<;n-#_iAtSqqmx9g>hW;gl;5C>~odM=FLw0*C+< zT&lpLxpigO$HfbdziZp3jOM3Ho~*;2O?T7&V0Rin_QM^a|DA3%Y!^X4EZFllxWz%# z4*kKQmX-7JpW8}hSKI#X%(LkT)BcnO>f1SR2b;U4qw!iNnB-bIF#J*L_U)v`4MI{HA5@TZ!M8>tFa7QDm_M~2K`}hAgR;@C^ofqtf=O&y82cz3*W(A8&SVJiSwl;F= zS!Dc8^fMv}N&g{5y5pT$L!8(KGP!iDH;576V+s4rf%EJxL%0m)<{~Btu6)dIcL#ca zDXZ6KG-LZbCO(tXP%Ts36KzS;!;0Nsn0gXF)BTnp!wW?reD)!Z53{1rI~t3~cP*CE zO3VXBh=Y*R%}UYz&eD14Ep~pO5ZGp-h4qB`BOd9mj&=!|(&OCna(exxzO zO7;2YifX3JbpVFOqQ+PX1n*-eb^9i?83hpuUcY#(k#D8GGxvwj}lC(*4f~%htwkg^I z>wcff+P|<;8e6yB#k}_-G=^TG+t2(^G?x17Oih`^`w=v&83lHd;e#7Ky>qC1d7{L! z8*@J@c}?=y4S;k__nU5%5lI-SBuH2$Ob44s3Kqi;YVyzZg-e7}#`f$DYm^!;B%Zz8 zK>NOIX{pchVI~OV?PMISO;C|0Xw7tNOR{e>=1n-+d`wSA)1dm#LmX@8FQ9@i$NX`P znp90h*EeTA6aV{#U8*ebJ(;auUQm=}jQTas~K z1^0u-C3Wr&v$!arI*P^?t~b&o)BY$C2b6H03`m)i`^lBuf)P~Ja{OSYND;NzPs0cDHR~sLzeAdn=F>DOAL@T z!GcPD->OZN1YdkOwrQhKNUocoST2HS>dlC^f_sTm6$jNiV5B38RV)C~WMd$sJH4|Q z@SDQnr^f;;kkUO8C&eN3@%Zu4)sNaJlS~gO3kgY@04Qo01xtEa1KOQ)$ zsy9q|p6t{T!;;6=!#p3JW4{bX=^toDPbRZK#@MzEYfLL|=R_oF#boEZpB?hrQLh+r zgv}?U+J!!uD!FP-sIfLkzU@%4MB7$8$9rR-A~g_e>$~M4p64X~Xv*h<%_yRuhq0G| z)wkrW1~uIObB;Rg&Js5J-7#S*;kIgsxzlVC{kb`{?K!P%oOU*-#~+Ra+Wula*QlD$ z>7nt^ghR554W~{l_W#LzhE6bc{If1V5SOk>P?P}MX3H-E9sN{P<4}tv%c*loF2aUB z^hfbtNY83(m$uj=RWe2^W2G)8^r8mBBQ$H_^%SV(gPHi3044Q&l9xXFGQN$8;~TTa ze$7%F5dr$$63}Uh?Vh@f!#8!N@0O)jDYg*wAH^E<(G9cGVB82!#sS&k?#>wjt^s3M zRZvf$4oY;OSu&LG@;s6ke&xD%W|=Jo5V^i~5TGUs#O*@9@{K;Vke!>|%bMF7ryQ}T zYS+f{dIy$UE*WC4M*=;|-oW;J5+uv-s=qYfpK1&ur3q|(cWTyyaoUIVZ$v>`zWdDt z88Q5-#r@E8!!k&qQz~=RGh*(r13RH<)E0ToHHPWJ(#I;-;tnpEXMeowT~DFg{B&tQ zd#`%9qZ3@n<&8vEMbmrz!xdDx)L?G!Jv<)4tPEs-o_i~?vtN@B)JJ-IcaKD4Spe{S z@t6)36K;`_UM_}I71RchvxhR4DK!TOB7J<#c%we1Z(IB+;Jaye>HcAfNKH%IOKUzV zKO=wtC+>WZ8k=tXAets9yr%DEO?5~R0qa8J5aFlTm+{f5H2Yiiw=@-sUNj+p{*K>vRWLI3IGJ$(l z+pwteJG4U-b&?-3mL{OtDh-LXNo#A%TpjhoD2}r)aV2pjy%5&9W0q!r2=QwshLJR$ zxl>fJbc#jX=a>s691)2YWGX~iVW&*Y44wPzpV{`7ARXh9fSkMnkADa7H7 z;#eFlKF8ZZ*ww9K@v|tM;!f}xU)^jSrU@}^;Tt7rYE?6ekJk^N&IGPZR;7JOMgttV z83*}g{eSkyr-h=MosWnqgkfJn#afY_&^hpw8)d*jgo#s)kd;X6>6$-+oI4esW&fKO z`i{iLFOc2V4Hzsu?Vy)R?aRih!h{K|al&V|(PPK`mW3~tcjxbl@lOm*Ne;1K!`SAK zpq(mhH^YuyDz9?ivAgkw_==ORkLiQvf_9NLq4Srpgd6hF#f`b$bTvZgY*1QN%*8qNb1k4T(N0;6%-DoxdX zJeuUjj;Akpd>p<+&1DIcOh$v)7HgPt^j0pYHNzrxa8=zgkV(8)-=MDCCe&$5bD zrqo+~(wB&-Lg5D(l`AW1$P2r)$}=j_+5~Lb8{BJ~8O?jIR!C6P+QciCudTFwZi55A z(5IAMHj%CNG76&O{K{bz6h;w3--1hskO}9W$P7VS+9--fPGX3MnN^pT_UJta5vRj=Vz1gem1~BYt3Z~>B?5&78*{;VtS3+tx2Z7P(yh$EVv6MJB}edKO|+N z98bO5;IoQMs3cBTsEgeZdz*5g@7?QPJjlJP?m1bi+0~XshqOUIXJ_#s5==EfDLhXP zfwz*W$y!wCv5xPOi6ZtGPu8g9F#1A4WP~rfB{cBqvwOMIA<+f07ZoX)P;HI*xYxAf5UY9_ias)0*-a6Qh+MEax;IoAf zhd@VDHoZ`ZH6lJo<~FlRwsT~|lzR43a)?No8)_ZlFf7oXx{&^{uG^$tFqSSu!3nl` z}L66!=yk-jiK^;Pf8iqN){n-Vi8AIx>YiX1waAx1m*FUaC|{T`X0SBskz6f z!rg=+PRcx{H>foJ8=Zup%$6bN9RR4odU4zx{|b+Ik4HwuOWcg3+qV|!- zVBip+dCG`?D5qL01u9yhRHJ_wJs$ae5MMs{{&ZQV8HRC_TJ!j|NR*$^HQHd`FB6fLOmT$G*2)BjYDS1aShJVSeFIY^uou|v=K?xA*OX(d>hcH zWI&eCdQkK$_&^?@lxi&Z22iJE@aj+@;LuSMJ;v#75dievv(+1An#YgOkc+{eQcv)= z1hO&QFY9VtvJTm>b4|DWJ{gw*!!S8wE9lJr*!tB5XX$61I2fzZT|jLnEoL=@v1^Nx z7|JvD6lM>d;v5_V#9fURe4B*tPv8U2rS=x6%{E7fL7Zu*lD%g^POWn`N!qfvScU-j z)|y)SU*>zd$;Wf)%(M6UBwb%v+bp7kBDFm30yq2e+sv5&O z1q(oZ533)WKvPGZhPN=_!IiPgsV0>X{%dlgqz#0y1Fm>aei_=R;1_!^g&DTTd8>HK z)E7x>%H*Q?ifOd@kyjp3TE`t`vO_n?`l{QkdEBviK9_~?)Y?LxXpfP5k>Z^Qfc5cf6?{~G&0oyf<3j+tHwyY^f&9fVsCrg5F(94> zlaovNVVNcXy^I4Si>7az?mVg@V!6^u!EHJgeBRXqwB0Z(0MW-&23(K#u?Bt?SRd;` zXUqT0y*SzdB3KE@i?QZ(s=rV0NY7;2qA^zFasq6d&V1WV5JT;lU|eowDg+xmH*(-3 zp^V0kJj7oRMf@6A_0Yn>0~7*idVcY360=0%04}N zrQd3auF@f)bK^L3ddIl=zh&J))Ni}OP|PJ%h%|=f)V1|HeNpx7VN(CR1ge{TbJt!u zHykKy1T`%J4vbO>HXkcy7vMdgOJQ*hAu99v7ljqzj4XJ_v!mvy^X2Oh+Pw!Hh|BmT z++nwfF;7NS@HqQ$s(b>w*#nx za~xc&M}})kdy8X3#PJAp1`>9*x%>3SuY)HqAK|^Z`{6Wha%?lrA)<30uy!wp0<%;{j2xH6(gfu zkL4}?A1cV&##4cOX<#)SlE)?3Yqq@yrNlw*4gE=YlPTYSR z1M(W*npZwki2t?o?xgjFtuZ5ry42iAh%ZoJ}@H!|N>Om9gLsH#&(8 ztPYpiG|scF2=T()fnuvf1bQ0!a_dV)#sR`(%f#MtFK?1&aq(F5rv9x?fI3;KuoN-7 zsWDc6=~ePUmYR~d+{Tolp`(-iWwh+$i0!&dn)|}1-=h|9HRVQIRuD*yfutftads=F z7VgOT(33}6&ok#ycRu5=nbN^3Zz+q4k%Kv1<-v-jod^y|rP@R~l@P!Ukpz#WFO!32 zrfAEomTU5MMw}EPB4oih!k}tYxbM2tYtrs*#ze&|(H;Oz70zlXgW{R66x3qB&7g%c z#KM^-5FV7JrJ`1uN&(xiHu#}%SwL#NqFbWIi1qwF85{yB$MEqMMiK>&1mMXy6c&%` zv8`KQ;m<8DdCvOtC(tG9FoOWuiI~uaxarAXd?P9^2OniYPvLghFv|bjc5di0VapqW zgwXHe(@d}kecOXE2dlrToxiN|l6|`N9#|SX-un%sg9E!nES3(icUdJb+X1peCN*jr zmAhi2mSOLi8P5Eorc(C5M-$vqD>5$r;nS9BtPpsCpLlPyD$~x(X(0+_Aj6-cD@!6uFYv_t3a}Oo{dlp&$ z5L7cs^rs1#g^UZ>r4U)bww&PlC5M&BuYxM}*b;QU+lIZS^Mc;1zcu(=A0={X(!6*k z#rCH2QeGQv+=>1!?%kHD=Y+b<`Gc!lxskXQlcd05!=wf%eBZ}4l^L{PHU)e*z`!Ro zJ*A`~D)wRF;V9%*J~$5E!55YWRVKKT})dMYBlyAS2<8{;y3P z*kGD`nbvU&z}L##35jLmI9ubmkWE9t4>o00r6kEr*eMfz}K(IOivabGO2&5bMRy6y8{c;I*Fasva> zET!kPpW;vflZdt=afigIEw#oM-#_psm^=e5Lp~|*xe><^OoGh$n73S;Ugx-FAh=sn zwHa&Jtsg>12QTz-WVfLi`(LEOsS1X!b8vq;z+9@#i6t{)z~0wt#umY+_Ueuw1i!ju zbA=abHi5DQl6GW8(>v@;M=Ax?HpJ-LTjO6D9;d_g_gWY12c{s>v_GFNmqXl7%i@-Z1PZwMTO!tS#=7c{Y6ev6{sWHk6O%;*20co!{>mb zpSG*`_@CZhy^{i3_rhD8znX-?Vp3U`74a6f1l%xW9Yowq+ss1v2NTB53qqps?!8nd z=a9 zObv+&Cb)L#1`=Qckb-0HRm>6FfE5qV`(X%wo6apd{}p`~@E3GIWmNZ4q{J{qkSmt1 z6Zo?`a5B8_Ho312oTqkQKHX~%`+jDxdhxvqPIdVjx_9heKmQa^oZ%130QZn89DelO z$euxbHMhpk)hw?sEv7ST2kM+cZWwTBaz87$HhJQL+_w)D(+lIbsSomE#<2c7 z{FOewrv)-cXgspJh#3^0se-IDGXgX3H;sjH&UxDy9n7+IvNeT+8qmAr}bcGmd9(q%Bm zixbhi=%4IG?OC2WE@3i%^eMifaX)26r3#6?HNHn2QJTEz9ue`5)WL?dU5R$z#OWv~ zOzG~tQI>XGy9i;U9@zFW5unu2pT=Aw_VtLQyS0{MTBFVf9Bvdc!(l#B?r~daE!dFm z%f?z|;y9;t9IKu6$EOQ&f5@8mWHXr0Ml$C*CFJ@`+5#*f4001rBu(gPoh$=N)%kQV zo6Splp~e>iHVW`Vvl}}WMHyahFQUdsq*QU|?v+QeABxi?c*(l2J(cLT&lpRH8CI*> zwc&s#<{c{<$`=s9lJPizQG<#aFXv)Z(E)Si9iLit2E92}vvJ?QI>_n5+NaaRoB>9W zt`0bKyRS118XqwL@R*>}P-db;>JjOz zV>n)U(k*DWlN_Qdseb{M+Ha3?d}Md9!rALFd=H0&;jehKiiv_A(yK5ui!MZ!0>3Bo z$SzfAmE2oL4dh9{Hb$0x6%~IrAO{JqtU;T+?AW&yXy1bM23waf0KR&pm5X@z~GdI~e+jNNpJ+2~$$Y zn|EnSiFpc`(fGSov`sVqmq1qq#y$FTl+MZx8}b{X~W)R&n)5R>$$& zVqB-OR_`x`Y?GQD1pGc;j_iGiGphg`Sc8|ZSj|_Nn2LX*Y1F0AUE*mrn~%^h_lR`Md*)4xLy8z1R{pdP zG=SL26v@+%3C;a9*Qc30h$!SkZ;=hi9o9s8tHzg2pJmp^oZ$Y%dk|BQOa(VL=MBoD zpMXUw!O1|p?SRyh5@=mk42HONU64BulQU!fTqKaastM84sK30yaS zh-woE+zp@Hd>0AR5v`x+IE0)C=7MFD|rXJ&1V7H49yc&}}RN!*m;X&lr)V&EAB^LJ(|KMH9>Xi@^iD!%9$RXJ-hb$H5^3J43GX zs+i9dn*63xL5rKAhp*Es$NEi~R#$9ku92BaUxfF+sI)y8D_bpc-%CMkI-w$CtoTT6 zOw0y6IBI014R}DPdzB&%L%t=H5pG->qO^>|to8+JbE&Yu;@K&QlzSj`ds7E>Wt(vk zcZ?-(-_i-?jDl4XP(0AC2Os5JC^_vt-mxtA$381}{wpl7F+>7Yk)pOR*Z5d=PL>;~ zGVGF_BpI%Q8FbH-iRoxUZKed8}+w>6lR-ctXe$&3I(D#Q_A8AlMbrNLxvtpFYVg+k(cOzihU#?KM7yv zZY{nv6gYoo9u*_p7*S<;*t#OFhs!VWVwI+P3_(=7ni_aMErp0_&M`2)KG$H9BU%&pwhRGi8_yDx0Ov34 zwO&!EWcDM!hO{G0R5)F>m3dK^AnRsm6(VFLq9dLoAFKG2a*X2D$YTU>xIEM@${jh5 zty}xJefOWMqM9gt=dkL?NP}uOB-Q~(){B}s6@-_OwVbA+kJGQiexs5COK8N!J=GEUBY zhhTbACUuIMb=&1a&Z#S!ll!GxWM{u(PLBNn?rP~wmE0WKNVk^hwMbc$gGtvJ%Qj<5-47w`OO|4d7}z!mEuj)j+T&YJ{lE8MJhDJrditNCF+ci z#I!^T>|X^FI|~Y*1u%psliDyiZnR(WBsnq|_JZ-5H_VQ|hQBtJzM_}SO9L;Q-;}Q5 z7l#_QU;+Y)%kzadlj%eGj08XFC3`lAsnJ@xKhDw3bFFJ0h!gDX70m{flzVmA-SUmZPbzh8GxYU0JY?%DqQ_<7EQZ& zOkHp%JJ*Uqs3${zYZ&oBQ}JoDHxP=O8JG@7z6VnJ5pq*4aR1^&Z_qR6Z6*##ospr{ zuO-d*Fu4}Q2@rE-Q0A=(Y{74oe)7CN!v`T`-AN~g_aU(iTE03fgM3fNv=4EZJE}3& zCJt?r1df&35qzS0ncN#W)Wyq9S#(>B7`lX|1oWNm*GHgHa8XKSoY@;DL#6+zZu<~S&Ywy2@?6MlHMY$bp> ziOJM}@oyMFubWYlzyA?QMRIs1VhQmL@xH#k5-oAWP*JG^qbi_kjqo6LYxpRRcUSfVuRK zk~yRNOo>p6UwoJNZa`9^RKqwtBbb(zf&~OM{6+^qOpwX=z3qWq}a=RBO-b z+xPaN(J{7im-5Cb+7@``Gc8sgHryu7LSNgBT6rs7 zyZ*>;?AgAzRk$$YzX5HE1h0hY3WqE?IU5v7@fxdqRX%RR*1T5|T*?ywh5|`12hL&u z6GcfF7eWmj6=6YnE(dv8)exECXgBy8WIb0V;tmTW)nP>M+4l?r{P6Xe-#Um2kq33d@NAt@i;y*7eEHk5`k1TTL*-iucHf)lvW^`;9@**=OFBuZTSe~fxbp4u@ z{b;@Ll&G4Ze&-=uG20vt2;34EzfPdpaHb;re%!CCOswOCl6?;b(S~f5WU>9falj|^ z{@U6I(Dt-MVV-xz4WGeu3n3-CE|9+wq>w1XXvw|g+FT3RjYPwEnK0w8D*W!xhH8xF z?m}CO#Up+Sl|qVtd3$5P=i~Ad3!3$ze};aIS6O$FZKM}0Q{`T`olN-lD$8b39^%$P zmq?wqqi1w(QwQeb#ly>NE0ggN&z^DF=ovb^eO1SWgN8jZMMlc=Uv3o~-V>dQ*3wK_ zQKbYv172?lAW`%LoZUK`wstNCoY{GFJ^XDN3 ztTON$q?xwgV(tX;Wna~9gr^l)dctEP8OMM5_=|AY~FORUao&MNH&f4VQnhLj^ zxo`Gb(3TF+`|c(m~~97V6i#0pMmK9B)q7~Q62C5u`m zv3xiIeW@y!k?lLEu95Kim*-|+v5pc>4&@07h&O8PR3EeRlILxTDCpmEja$8?TzAON z3GBU9i`TFZCR*8JhCGeu8fUv^!HcM$g2ln9vJ-xnu>&InoY6^3;V7Pm`+ZoxFWyEL z!_jYmBpIeXZ47kEpy5WDcWyTiC1=!zhT#1+!ZAjt(u=`Qq<#_j5KCC+aboz3P9q{tI>( zCNC<+K4^VM2N)J51Jb|;D@Sfx3+Jkr{hXdiRzK_ z9?J3+B3A*qIu?Izh>PuOZ&`e`75L zdM+$fVic~i%=v~r;3{;YR&QC_i~(+{@c^YwSADeF(wSX$;-<`o2n@VWs7;AhBvbB_ z5m`;_xXxIX^kdOyU%kZ*C$PYLM{;2M8>@OWO-fQHs`(5x zS}FFxZ7Ov+%OC16YQ*CmEv7(ET<@Qxl*(u8uQ0{;`A{H>W#ex|1hh^vQug4E)ZCND z7A=Vb$&DT5&)X|CVXc*ab88`+|GLxk_lPfe3u!H6=6Xvbl_SK#!=Q3gg&GOvZkI}1 zhW+2UUfJ4$TlC!f3<2hzW_j8aqc#oYrlC9s*=BfToG)-p53}C^Im(O)O4}ASz4Dh+ zhAd~*k`8V$^jH+oRsrh#J(?p#29(3}swrs%9-_$Q1>8u|#|C0m*Ek#iNONYux6Z@< z6VxZs;T9v_xgW=i_A?{A8Vs3|0{-SWx^CbAeehIt?eOvs)dhVTZ+A5a9Kl6*TrZs_ z89I38N~tSXTp6B6@kiDE0yp?DR0ts8BoB^1>ItY=FP{FnhyOKNR;Vgi64_F5`K~zw zkpjzst60Fw-TL8M;R`q`3C@PLtfY1Fp(5OJ`h|54p()Q#p0Lr2x2?wWD6Xg*^SnO< zx~o3B8;|CW@{xyowrc#~EO+Mjqx&K_%bz`wA%h}vfP|x3nr=|l!OT4*9P`v8;=F=* z(=VU&aZq#Tx#9xW*$C)El)gh%rZvSj>b)iviZJKZ3h2{zcm>nSr8rrw^vmlEVB7vm zku*%wV))$b+9b$5u|BLowrUngxJ;q)O9hZ!dQtS!LorpTDfKgH;o)U{he@6w2Wy7u zHddy5$KD6}sm@m!ysOd?;83C#FgD_+_}Sv=ehZ zG!bNFcl5mfGTJJXIq)J5#?H2?90mSJIX70U&X(b+=X>N+cNOONe znn@yvCy<3mzamh0WvwKV=jE|8qSZ8rA1D7vj4{}g&&1>P0lWY1?&GM;$$<^^_;Duo z!0J}@;r)UU3V(F6e}tt~(Y1O?P6#ZJOAtEIl)`Ku;qIp}a1_8oFI-aUa~7y}Wfbb0 zSKQrlTmzy}gMOF2ixo-@u3e7!)cg21^VNSS8|c=rf}eFB;~ig-N~v2@9)>kwd41)S z*6QiPCQ0;xVD)1J?RR{4r+Y}&A$99?gRYaXYR&!(i@wOTMnXZVv1;v9WXEj}&a5dI zB9RRrXc^QQb>s_q&d9>n5*sTwqvZ8o{r%Ubh*90lgs%#R{g$V4Me+N709-B%Q7bRS z)hUdIQZ<|ia6~%ToBPQ+Mr2Odb1ih0oHPd@Yxa#3meN70zp3fnho1!jAh&bOSvok) z-ONX0Mhw<7GlFKJh;RFu32z=bR|%PYe7I2+1Q8dVhujtjdMN)dlSzQT<$WX2U*Xt_ zL?Qxl--#y9W)c@r7l7|ZX^jF58z~0g%{Flfdhf~i7d2AVfSJf*sZ^-~ug7tOg-P@J z^pUIYNt1l3@AThdjNIW{bH9~VV^kGLtVQ#YCYr)%Md=5U$6VW@F^@DN7iEM5q776D zYAPAZwN*`<-~y>2TeCL}x$k8j@F%LPVNPnP8}$AW#hB|yTic(e%i(xjJppcZeY&kQ z>$KaUopIol?39Hg=>)H+(WRm0`8vWD?rBLrxA|GVrhagPtc(kwKvj5g^DG#-`lRt;r{~k19v;v1PrzCtOrbcP6SRp6b^Yb0= zIrq5Bhl=(K1q_n4vc|Jozkcv&D(=GMDk}Z%ZYO)8WApJy`^9r4?}Ti@g-UEolNxSA zi(I7TD3!(bgzL$+l<0Qdv*NYp3*&{0Vj**5<+@|XmV-?tghPN^F1P^zG(TZd#*IkB zOyVg&C_w&RU&-uV1q`@cuO%3S-5j=4ruq?8h#EqGCW zO~BI*riJR06P%@^Rt#Zg1>uWTkHnP&@X_Tzp=?CI*eHHI%`Aj-uz)8u=1onkVpE8C zglejVII5mUG(Ph`09a|R;1v}VMP0bh#-{>c2h&kt$4(+)+UDs3{)d$8LdLsWv+9;X z`9&8Mm5}y7k`%>K1&uPMAD=GjHU}d?aaW6y$?J1Ebm3jKAcP{awpv8j%@Ic1AO{ z?*XzUo_rlKKj>aXEp#OI{~T>Tx2RW(;SV5zQqQ7No)I{nX}ieJwcGuN zsJrsT!ght++YRCadcX9V-?QOmy;=aU$G){OpZiZr3t2$he{|z;r3u!E!R@RTZY`z{ zwA@z(DeA$HZBzD(I4$>Fgj8n2?Q<{tMzieDZzK@9H9s!N1#ndo)kF`W0Am4=w`uWa z@YMJXE1!tsb_Qq92J!M@O^LSDbC`JRcBBNP?!{ZwkOB7zr9e49H*5F zVSB2QV{jcAK+P~>J&CTSTD0PviPRAz`TTW)jhmhs#?{V0tP9rupq?;va08uLYb)C4YVQwmV8bm{<<{Sn2mH>(3a-$S<1?tA6S4H%6N= zyW{|JD>g{s_#7CMIW`^2_d8c!lqE+pq5Ub#71i4R4NN@s?&D-Jr_G9b*tznBQRc(Q z4`wfl{mvjsHjKj$+N%PmKoHYI#m(SQpri$U7^CN90Nx#3MZ3@3Ei9N zCMGtxNYG%B5t_By$TT(p6BnY7u=7NmEM%sO8GJ`L26 zt%ukg2pBQuJ>r?`=-;(bd~i1S88GsVU>frSU?=E)pxv2-5abem_})aew;^gaKqJfK zhnSGy4gR%qZTlY|!>O!zdVLx>=9LhNhq@b)zG8=1g6N#JfEn&=fS_j0)seYQt|Tw3j6YZUbO>My?>7f2x;MB2A5GyLB&eZj8ul!2wwd7y=u0TN_pJbcmNYamiN%Q=tB5Co|=e{W3Bv*^mn!6flzhxKC26#4lah;$e|We}+bd`#3y2)0O}wa``pO+oeuGkX7?C}d(oh{DRjfu4!+++@!q`62?QCm809Z$5+29OZ|`;lp`)vHdj z1?`<6E`kUCxfGDC(Na_vR1*1rCm++Q=x3xxzyP<0lQa3lz|44u^w@73I4#GYApNiB z#E`L<{6Uiskhx8W2vF5sJ~NYxDRKpB+Kj^jsTq9qiBV!Uqbj0OgwAZCOI}Y#nHX$= zahJ>upmREq*k7P>!e-yns6b5!*1qHzx5q+*f--?*{$UOMMoU?gqNhYAIjMr8niE5Z z$OZD}&w#cg@+hX>wu^I2norykwzW7Eg*!=z3zGl2%4QO`&;ZxR`NlPG014llVN-6; z3J7oAIvN4`W^WP2V$LUe?3y&Lt7Q=ignw-#_x@8&(4C}r^z}$?9MGl|y_!5J5;liW z)H^@*cFYhY%HGl1@n%mS`)6sOc7O$mID}NdVw&64Dq`m;mgCY>Wi@*fFn3k$qQm&2 znpj4&34l1jT1MMuhww^#{uTCIRc#aq!vHSUwDV1SlvhE*2gJ#m_;*VjgwREF;Y0-7 z{OKVU&309Bv;JrzFV_`UUOal7Q^rd}pE7Z$_nb~RHL#Pc1s^sfQlq$l>2gQi z?uLXon%D^}F0>LPsb=#v#LDZihxEbhX%*PHrCLx`+&IA48BaCZWamxzJt+4|nu$jLX{Ot2 zu`qO)2OF7{Z4@G*Sn77$m0P(A1G?B+2~O^6lxW-Kv}asFP-O@4QAVPpZ}z`!WKqc( zvBd|n3RA0#-)2H?b66!lwytvSgyZu5*e@_gMRIsk{cK%(JX3%8KWEO^W{#Ol?wR}M zlH{Jsib8ZnB5ZQo(nYye*c4@@5Mm^gt?iW3MTrtc=%$M#m8gUeg;J@0pYQMW`u+EN zRm*lh=ks}<_w#<<&-2;roE0p&UoP#l7h44-TK<+=Ka{#H*b16$<6Z|T6Jx;_)UNf1 z6n8MsHHQ~AzDnGw?N8-=^Jnk0zGMzpC{Op78)|(bByXH^qPKhfr&p(ouPJDaTz z+Xc>pmODiKtYs|te0|c z|1H^a-R)^s!rja#IU@OcNx7?PD?;vRD;5}v9@#s!n)in9Sa#pw^uD&)^lR-o2OZA)I)zvHw5lGJl5xh=xE>EUi|vSfu6<`YTxTMnL+Jt zkE|CJlZ+jxnJ#;YmL0Z8GqlT5=Zx94=YRdIzRu922MXq4BG-EifA7Iv zA2qo{@u z_28<7Ui$}3zaBR$&(N>?`t@CCrf4$g@uRUdH#TWqU(ZQiHDgdqFgjTB*`sz^pViOh z(#BuBEggM+BK1Y|WPjB##B~o&coVzt&Uko6lP$mEoA0fZ!l~1yjZ?kbOZ{3G6XpdD zX-pflbT_t;XFmVvN1oTUTylgrvG4plufI4!uD}U3r#;*AA>f>CG+lk+8U&r5kD*t!qBGrFhe&G^z`xQO7z$OJz`0^VOer9u0`fNI> z`OnBT_g9bg##@sUp5k6+WuSzYi*l+;v%^fBN>W3156#_?jkI1^Hkg8$mL3a-EN6@mMw-12K}Z4@zv`X1cX_=~2bpzqV` zd(Mv+1SD)nG> zkN&mYEeiAe7`q~*pdf6c)?+n`mCxQ8whcGUlli`Q({SWnWn0FV>wAw_6H4ff?{|Kk z3?5p#rrN&X#-9Uj=}OCA@r%=j9*u7oA7A@(`fU2ueOFxr52^)iXSn`v zU#!jTCp7AHQub6ZfEPwuI&7#d=uR{NDS6)-i zOl&Q+mG+Qp+SKgVkKFuGc<`>9_{HWoTv?P;)I;c=t$9>}Cu{G1XyEwc6F+j3NfTFM z!;}<^zO<%9uRWFSdPt#K|6?p7y5HXrmm4YdTu!%#_p_H*Z#J>J{2}bwThrP%%{PC) zx?n!zTsZY>)0wj!XW7>@1B#X-$AZsL58SLMywSbA+o5daUW@*hg;&$c&u_nNJYgZ# zQ1|>`%O0M6U~|s6SmlOk#IMi$IFr+y=!b@fo>+AqB8(rQ_Ut-v+rv~?P>Jm8?!S<^ zxS`dq&MR@HM&f(r1zD@4mf+gA*}AOz($_=j*Mer!ZRN`wwwHR`{! z9gbQycDmo{OKwibd!M@>{+PI0Mnq|RZsDTcrQFEfW*A;E-0J+H7oGds3bh;<({VTF zAK#`K@V7nw?OSG7OAv9=Ovskn_2#zq)`0`5iK{pn_IY}XGN>Z#4iF1 zet9LJ+{g}pX2ocTPImjH7V5HBj?vIp* zJ-cxyM|Zig;;veaIPiN)D?Q$%6y~bFdn44}SKiiQMcJM^*Hy0Ed$;|q%7_m~ct+PQj4Vh4cL>L&b)I>T?`4W^iz#C z7Okz>^EBA7kIqpt-j*XA^t(ONOgN4rWwu2-IK)ToKq z@3-wbHtF_8_fz85ufnoB6R&rz#;NJ0Y}BPz`sa15dVXltMJKXB`&;YJh7UJy_vxI) z`tDA*jQy^jcy)5*s&(?2>OcK=+b+TQoj?JZC;FbmU3R=-k zJ6Yu_qeP;QW!?+J;lGkt$8lUu+zXP~yvvOIj;#aFR=M*NdUCdJe6+(Y^zLI#zZD6a z&h0)^;N!ML-t|olvi@$&nC8M8kb;Z1u)YlWciOG?@NAX#3Kfr8s&aF#PPvtXtI_Xw zddF`>G3H_V3k#Y}Q7z-mL2GiH^>>a`#Z%pHCH2`<@o7m@ao193_V#r(VbB z4=6ILlVF*G!jR;Tk@SGM&+Iw*&P}72kM;Y@-!PpqS+H2U>6-YIBU@YX{H5CB`A2`2;b#7OVfTa?&iuo%chgdLRdgkK`V`BYAw@vdJ2i@L< z9V6(BjCDSb&`#K8^XS~BJ-UX67DT_}ocj9LLE4F|)y^JudC(B#zVgY=1CDL?rQto} zmnXZ{d2*%*HP1!^I@Ld9D1UNoKknLLXLRE2kU_EAHSN}rJxxbLx|>57XD+40d$>&Y z$#AJ}@3=fmYAKnq)MDPDDOuX&<@XWm%y@%bIMY}S1gH{%Epn~$VFRqA~AJWyC zjC{N{m+shi%zua)vQ<}Z{J#|jv3|iDqw=Do%D1d;vMawn-gf## zyt6TOt={s=)gOiArT4q@9vp9DPABB-dH*_RuVme;ePz4_!As7qStgfS7`)w8Lu}Yp zu|qD+$HK<6k4A~i3GVK7?rk6Lq3m%FN_qaGt=z6*eMX3A;%(8x^y{?K3o;+Kxkr2t z%{X~wUC>#{`S0ks1kbSo`C?{dqLeNb*Yxl-c&{iwR@*s_V0-!eaiUDNa*>)(5=pR^eL;=SZ7a%#?cxe( zi@vAmxLqtydY>g<>AQbQM=(GQ%eeYeQTdsfS+o1D4V}?#5+BR*S7H0FI-d*-a20$x zIeF4?=GAoQL3~(sv+q;$Lt|y7?yQ~6sasQ}!5IZE6vd#D%wvs42Va(#zZ1IEp5V-% z|GOE_-6m}PIPR;hxg+uSwgru(+uFp2hHOeA^YG4Fosv{v@x=2dA<4Fg&*O`(0)2t= zmV?9IZswqvIyhN_oBE&7u0}d@qW&6_=H_iC9O~zM z9N9CwzX21|nK!dqtO&(LW8z*f$s&T_9WmK!x99yx;ZUhRLLHOj^f2XDsb(+cFCbaYs+ zeY=b$zrmi%zYMmt)$8INfEQ}m9 zTlNHZalhQPiHnQ+;U#|qpS>;AJN?tRFG4*s`bN`!wsT&mbS2@OAZvKn>WAxkKVAx@ z&TTlayK+^d@3ZSW>YrI(|G3?&gYdG}zW4XRxA0TTt_F`qJAWQ|IOMPLz}!E4*64OZ zv40+BdF4orPT`J%=G?XX%DGlE(s6YqEmGl$CA2@$?S(N91sogPjWqJhf!8^Rw83C)B{i}9{*IA}3(ckVHglba z4G!-1Eu60{KVQ)-YfDSxN1IWdvByR1!lZs;m*sB-M|*|o-!3usMn~K`uNVKx)5w-2 zHmtg~Ei4IAoy*=5Gg6j1e=boN zBe%DFE;PE9P}fsZc*ajsc~PzP(3tIuq~Mg#=P9+>%=We+m)UA{(NTP@e31YL+|fvf2Q|j1pb}8lPmdn{|2_GugVs5sw3u6Twk*H(>4^_9Ro45{oCd`8Nv5M05RE#%voEo` zI#P*8LA+`zv?5qk>eNAp`e1=!E^960`3n4np4WGBKL%|We^R_@O-`tA^UsB<_nw-^ zHzv|2>AA}FpCZ>UGF*S(k)d#V-T~sjXDV*7uE{yY}&;z@YQK@ZJyhW|#!kPrVXp!badx_T_5aGRy(h@bf_v2}2ce3lI16Ag~YUX_E4yyKH2ZT1^=YF_p zYfH437-&sR#EsIkf~SpD-2b$Fx9za}wv@MegB@{r!cty2YW&r*&3Q!@N`Iuidk=Ox z*wlqQ!fubyo}76(Z&Rks%;)s>r6<2mS;Rb(sH7ebTawyPVH6@xe!A5Bbw-=MeNx$} z&#Q8dsoZW7iaG<|RG-{ES75O~>v@dExte$CskwnqVw)8lZv5&udUI80-IcL+O*xTb zhrU_<*`g9w@$qN*z8)_;7A`ta_CqzL{90Aj zqAR3O_NdJ0jzgM$sQ&4B+tWxBFA_a>_gj)iH>lnDQR#!5rr7;?+#1qscKW?&>zDYS z{e}hQ(t@Hv(#d{@EJd3A2j;xqxUSmCarFWRTcfX^G9MfHlqY&9j8_#a*BMFaKTBL6lsr-hAs%j&<#vZpjX5?6mDk+sPiKu$1g(*rvXB zum8ND;8NbH{dSV(uddhT>=c-*Y;LsV<^M=`k`F!M_fV9|kmnhlQlo9Gy}AF0Pk{L) ziPg!t&KYGhHLFBz?>MyFv$2r+WNOFjtD-@*eG9`caXh+;`gUjTE&g>y%i2cz&^95h zWAyrhDaF`JB^3{$zi-8pU&s{;Te1S(j z`ZYGku57ioT(Hw8j@wvJ#g~ZpnL;g`np+eMpC_a!qrY=5fb}5^2v`!=dSP) zHXd4IP=0%5g?7Y!2G4zWzn$#RXm0;?xUu2y-txZ4&Uc|_r>e8}E>qlLoGhCcb*~~V zE=8gwn!Vunee&Mdz_*_9M;2emBJ6b-bN4>7Y3gHa#-D?<6$ZQZeqXJdyz4sGS#9+9 zq3b<+_vt6W+tPX~Tpp+9t5%-c_-_8t&xS81m&#-#Cq@(wy#9H4+Ft{AKkLj9zf!|{ z#nVv_-|hYU@|yt4D0_40z0CLhtjfnW71awQU0v@5iFwO?~zm_~O3y zTN$JCrv2HDR`paY`2h!~_n=n1zvic>e&HwlgOKk@K`sIJ^y+mGS96D&w`9PCM=%c6m zVJ%&9ZXN3{L*EOrtCdv?!|u9hIos~ZyLIy$xbb1YIh0$UB$)|)b^F>~wc)=*4_HNg z$<{C9e79O{JWXw{Rz3T?_ET!F`*D+K>n`8G^`%`*^!YY%+6}Ug-Sq37;nA$;3*9eH z+b7@t`{}*w)6K`-KRyl<^i&?vt1BOGc&4koaEAv6(yd>9!CLq=;k}K|_nEChAr@b3 zJ=5<;C#ntXvXNz`=(4l+s2!MJJL$LB!S3R|w4gQCvE}COLrPU2{*kRB;MTz1D{F&u zG;jFVofz8jomu)k&|?L4zE*WH%Fak$S8;1o;<)ZUS;ne@{zYC6LaMW$LTuFefJX&= zOB)JgyOOG(gyiRP5mlp`zf*G+BYFxgE#c4d>CN-<1?#-J2kg`x#b=1})kQ;(PFfF? zoR}#8Q0Sk1YH{oe#bJ)+%!bDw_J{}VlTKaoX-Em$HOKMYoH@?8cG@H6LeWk8;|A7y zGGkL0id&3+hEI-lZ@B&UiSmx{?$nZ6LaIgRsN?SLlcPDl*36(%|51I8*x4njK!vk% zF4%kO{QLQirTG<=Fh81G(HU;Izy-9TipC|N(J;1ehuQ+grw_IvHB%1Hg zb~I$6C(*vs;i9SM?x*2r-G2B7Ub*n7hx1p#f5o3S^E!Aq%a*FI&-Qkov%G}PmRK*^ z>@*>q7_v|O3Q}UwFnIXGOeoZnvCEFRb^LY{?aI!$*kzInrPZr@h75aptn`DGy>{qu z*g1!`{oCfjH3*2CZyNadg&XH?1AcC9b>j%lb?!YmHMw)|o=2+>MXE$OIHv5HoD5cR z+UgemYlf~1ZWnVpHu7+X%i`pnH|r}~hBFMkjS_aJbe(v+KdS6++N)B1@rQRtEmxO5 zk*me9DZh=gzNb~>gWp{)vaL$|6~6Hm-Rju6q=z=-;?AiEL6V`aUP`?7WCDNHlh&)r zi#oRXztB5>Nw2HbDyHzv=r_*w%HxB0^g`vPGEJwk*An%aTZ|?eVezvPCHyPb>SyTv zD@R_h+*G#0h}%}aa_V21Qwi^C68Ff4FEMXIFz?&-V?Esp3eUW(YE)DB21!{Oxf^#r zDB81!ytHwB($iEO(^aaCI?jV`Bm0L=6C5pnI+{w|OuznBZu!9JVQ(USOI&?NQ)1($}RG58L}?ZHnf=O4*)E-4C1z>*9T0`o>6kD~`-N6zv<5c%$p}S912TC7#V>J3M;okDvQX_0hS^nzfPhVeIJ?Ii&RIq2~x{daC+6=^d zTXfxy=LC7leg5~w>~Wyo5zP}19wSeyyY*})4n(eu_m3<39QWra>bj4<{l{kUpSKq` zhg7cr*u~v?*(!zb-M;#ITPJkhV=b>@MTFo;x<}lqUC^EJYE`+DhWb0# zS+eRm_cv_3&zs_29P1DUKh4;gORDNvry{#1``cI1PN8!z|K9id_?a_ID4$`yLK0c)r+6!AON0!8%Qc>Kk{Bblk;r#xr zuM57Jtx8SaxK(@Xsp99_F8@al^fr4f@m!nu%eiFW-$mQ*-nW4crq5pE?kmX!>5gw- zes?uTm3$X= zcOG4F`egZnZp!t44jw_O{3SlY->NzOA0ifM-)-QgjYqEA^q?<#C`aXE$qsAwyEXYI zmTv$1HC#hQ^sM#3oaekG`{{sdcb$Uv?fFOzHQbS+!kI8SlR* z94d<2cz#}lfm+mbTAs%jm#HhK%R+-^8vhowU9GcURnQrF{;R^{6UV*xG-P+2S*6?m zq%-sdGm=?E)yBpjl)SlFdb9I+YuyRMSKp1)20T3mZAw=%`nTEkskOad?(650joeqh zv>toX_38bal7AMn(?fc#N3KDlw9tf@kE_3;u)y3n=<(dv#p7E|-S4=}A#JM1 z3VAPx{!K}3@~5LRN*_LH`u<5LYG>?OgTbKo#ioQfub+M%qUF72q?U%W@9(#kP&5Pj z1GN}Ef2=&t@b-?H{cZhP(f*spOV|5x3EC0;`m!QFuKspYzp`jUTNiO&3;_dg|jBCBDho6F+L`eIv~L+3L)L z?$5T8GhKcpW)o`imC+-4_kM)h&m8cHv^{h8XRGLa)#R&!D+D?v+9_!+C$0Ni(z0Jo zOUJ*4UC|V3DQu5iATp`%z?G$0v#3XH#kq@zs9gi(a^*p_{lW=M%@3eQx zS(N$2Fte+wH(>ef;!5tCcAXXlj{_C)4t=$cC!!0FxgwurzwF*_qKt(_osS>5RU4#J ze8zRgqBQM?$h33^sE)?nDJT1$H(f8l{qj>U&3`d!7@>WD9O~gZKyvΝyp_sKTxq z1^lf`^c4%$R=?AFe0_=k5WnzHgdg{owoPbm8-0;hh>Jsaxa|gc_UP{SziRfbAA$9l zP{gA*rP+S*W<0%L$rhG6n}2-fn+f)ODU1EGW(-l0V$lm+|Ri#(UM1ZxzREmmQtg>v5@Oyp2XKz~?1X zNs8Niv_D;)$U$$}_T^u^B72G-2uS+-v3&mdwI7>~3~$?0Y-8y6uH}j3eo3>6s(e;_tVXej4#QzWLqbZ;k__JMAb=_X~U-N^i#- zbp(l&Ialn@d5DqK_tq=h&)oj0`kVRhk5-5;$*-fa|JvOyscUcU)tlZ0v~LO{ki)&m z6PRm`OVj(&dlvsTYvw8WE8(w~Is_ltWVat~{9LBmn>e2BJAT>2=2EV!mUw``}SPEk$6g09@Qd~-GABISbF_A;I7ii8k%wb7Ir z$qrVl!FIQHHnAa2(rLE3g^2qZP+OxL6U81r8ZY>mC-(8h`7h5 zJW;9V-+Ei@Wt!>s_~HlotfyE2$#&w^#+vONrcsA8QW+c4qpVJiZpDY!jXt<^gTKuG zR9VHP*2B>&kLmcnKeaPN!O}F7#&Vu3y~xfZg?u9acKNA|*Z)Yh=(Bqrz4?yQC&I6p zMc9)e{V=6}hkd5_sqX_KuXIY`0j_qup2qOv8(N-KM@b)Nt~yZvLf;-dCF#$aW$hpQ zQ22i7krFa(%hye+8`~Qhx8eJ!P2UN~sa|D6$BMVJ77FKGYm)Y_naKSv_OzMH@EKmA zdWqDuidSE(+Y+a?E#pbGTTfx!aOivq;R>$8A}}iN@Y`x@-HY&pCZdDLLApb)eL<) zc&Ys|Dd@A^Z-tqGBM1k7ImN)PTf%i35BTJyz3vF~=fs)$$8KL>&mlLnR;86I2fI-s zwkDS1p5N2|T(a#?(Kc5U+DZMx>!%!&)tWRj>xaTdk3N37P|YlRKa9za}SS0wgi+x@*-!zS`XsNsIjGoJ#WKPGW+=8&TbEjEJ;qyr- zsba%pp=?uc_Agg|d+yNCf)ks9VrbLOL*DC(M4QbPxk zmVNMM;Nop+=~+R?*jW}?8hd{l4v`9mMs6wpTKu%)t=0*(6V;lJ{zNLCy8r!6@Szo6 z567+G7c*}jpHs{F`qj6@!Xia4@#Qwv0r}7yuU1|7=(J_~{HO8n4>5ar)?Wu=I-CDU zPbSvOPwcbK$M6 znN;6laSOLYvH*N%;nQkIG8pgQ%{oS z{4b_y3$N@{ht!+y%d4NqhkLHx|)Qcl4CnRTFj+u68`VbjT-d?^Z(~% z2!blaa+9uknh->kVq&2OCW)frOgO?yj3MP#awQm-i(wKWmw|zMAxWg%B8)2&3bmz} zkb+Z^VU?Jq5T*!ON@c(qPQ}dP>gW}vLLJk6JnLH zjno4S=VD?k29s4{GAm7X$yvunJ82-*i|J238Z4U?#u~@E60xGECCI?|yI2Hh3Lofl& z-vE;Y0SGym0DfQq0L60{GA@-C85;#KkYJTEDS$v-$`D4$uqZH4RwToOaF{C?x`>X+ zWD?BuAh1#hueOT-fK>v=qrjj50iKRT2GQYI#$cjs3}b-hHYOdcUJ zA#2r+0TYQPpQcQf-E z^SG`UGXeMq$oNk5@G3Bs_D`eav5LB*E8|W4wA_q(=1TaW|MuE{1F$S>p2E-8H zF+voV3csTW0we+<|1JS5Dq^nJjRYwQvNA@r&rTrB!T|Ij(E*zU%+36Bn!tUTcTfyG zpeqx~Or=h?kIw985@Crv^gqRU+aGU9l=z5Xk!?0Ck1{lT`5w zDKS#ui>L&Wh?3QWG6ql%nmkO=6>)*(KvCg+Y7!Y25)0X}6bU0xCgjV*evpiri7+J0 zXEq=yg!fF$BF*q-b>vLm1$A0l(5vPhF-NTq;z09NHYAQ+J!X#mP1 zC=-=3VdP$<2m{EN(N;DadjumAuE$T>YGZB%pe4+eaf8@rEI~@ffWiZ!j*;M0NXWE9 zB=Q4z{zU?yCjO2t9@H-ZPy?otE>-~`2Ox^}lYwLi;7&4L2E_9OfdKf05(!8pW5AlM zD8}-e8IXjjXRMbg#OQ#?B~kDNvP>e-rhPD$aAn$DES!p4h~EQ+!)&OCmVrd_L^~0V zOydZF@2&=%6f-2WG#ikM!ow8Y_7YH{iw5UY!hxAVs0#rJk`63lmx2VPNdUjrlPkI0 zDpa0;Zu`#?8CwOyP(7ksl@Js;zyq-W5JfD(0%(3Ho}de)i;KkwC8&@<21m8=&4_|{ zAOjM#cDjOG12P4e=8Af zE)q#pvmF^o3WM1REkt_>Q03rLD28o^wiBRYb|I7n2SP<6Kixu%Aq12K^`9xBsK_v$ zjCP%1Dh?7or_iaO;bEE*im+_vdNLNkrK3{>aKbV^I>2_~;b^6vU1oWCN`^No^r%hKnmfrWFrfgi6@* zKzcU-8Lfa{6F~jv*HQlK2d!v$nlBTmgXjRh4k%27947NP0jyP-XxS%198V zZ%2Ud8-OBP0SW-n+J%q>l?r%Tgqo3|e7IT{ltTv4mrDE?1vI~y6unL2-%+E0+9i`P zZ$PrT}r`P%#0hqZtG$&dkIP5;k8k5Cn0-ib0q;e<^}R(O~AJF324Q zi;7#o_k_gFHIx`ogk`Nph)|%RM}SvD;ESW|VTwd1<}$#QJ3y^r$T0E110+bMu}Mi& zTPBfY2?N-+6dI_PSO z8&$A72m&#eMuKjS`D|b-qJ9uHAVCbYo(u_51r&B=Mh%3UmjJxyiv=4nSw3hwB-xwH zLH{m^;>bW>Q;1rVpmb^-5vpX9*&$@mb(f&8h_r{aPJr!y8RE@9Yb?P60p}%LQw;4O z^5rO|Vz5aNU4w1ZKvOOh(oq!>jgR({`Erni{|rb}2?I8!aAZ~73g%gJP@{!n?qSfI z$k1|vZcGW#W_pRNc_CRy=gLqW5}d?L;TsW(7($cp%sI9E06?N*_MoU-7f>r35Z4Hu zCTRke0w^TRwI(Ov5`s{y#U@9tAmGIqO&8LmZG}VwRpNbYEo8X_e-a8}Ue%?85--Lo zKt38{5*!GHB%2{d=My0g?aO~1t0N+*piLl3f$*#*hyp!O+n+rj;Y!g0Fa*6%WEv4L zDFi_T=p{u+ff}p`Vul8x6c8(p4E?D@2Juxy3zQ*@dP|AD zfC4I*7-KU)N5TQr%VFz65-N`XnNSPJ1O`{eTmi{wLl7pK&)*HmT|%Rxx{3ol3K!H5 z8GDga5a2OG2AU%YXbM$IXpc3VP{0dGIO?fP1`T4C=y|3VMVrf!7Ew>4KwI&Kmh`@3jM4!xDDNa+Q_hC0HAyo5NVG0fK@A~evAU@ zY64kUgpoi&1vPGuikgHGCB;nPOi0KU67Z2yngJn9CPaTKLqX`YGJ24}JZB&iV=}OP z#k|ozB|d_7st~wBF1sFNzP1qkss#EDF18%r$v;L@pt%BQz7r8_v&T>N5enEDdC2f- zmO#I_{t`(s=FJiFmjY0VnIc#Tszg&s1B*Zxf~qSbB1x+O3<@ngA7~|HhW;j@1}Jk3 ztilkm3xNn$g6&=*dP9klCj-q1zQ;hEQNp0&>}5QU4IF=}1t?zVB4v2%*ZZPhs1JdP zQ79!898os#-9RD%1O3G%QsxBQjy@zXF+^;N@)eoybipQDQW=GDU~cu9U>cL@WyEr)8DR`;CFdi$FyMslgmUZK>2^#uwmq=i}HrLv%Gu&W1~I4PS*2765z>YxNt z$po}mT@VM3KuUyYf)YALhGYYw%{L)=lb4`IxhNCFK_vDttf(fz{K19*)g%IqVpQ>s zp(Q2*5DERDs0PR^EE2L+nO+F!Q4ZK)Tu^0;gbd(745t!xr9g%7H_<-)G6c_|>Jc>+ z!1hsC#9*7!ZW0*4kW&Oi#s!F{gU+hZU+Cqo36xNp1*$n0V&K1n4n&s%Ht|6c(Fcgj zPA5n>Qn&)aWN4WxK_U#KEdsFuGAYB+6(D%v3auN;gXPKMAdG=}C{x!{A;F*xUwvSm z=4POI>VSsUs`|GmpbTQ+s2Y(>lu3n=KhOjeY%z+!qNhMb=d(c*`~t$*l>k5X?f=qH zz1b2GYDbC&-XJo0@fBzlN+P0<3BXoi0VXWw-$Zwk_>JhK8d{@@{!niHP3{n3d>|>Wf9dH=;opcnhXV$N@m_CQ|f&Fw8 z&>1$GrAh|dQ4tzTKsXZg4tb#(^$?le)qD`)NKqvUT>GCkkE=ytkzhL7G4@#kCZEZq zAy^h3MJXx>XlkdZkT-1&AW9T?+enDIXn1%q-+?9tL){c18C+-pHk_ceO9RBzSatAv z0IEiy2)LN22%gkrfEGX`DITlBEL7n`uqp%7VPy*efXDZ89#zrvuNM z)!}oZVRk2?Qi9(KG72kTX2N~YJP{D1N>ooBja9@uYJv_7yvbL^OoJFOodOztSw3y0 zu@DmhK`W6686p=1XGZUai||p15JOW@NPzZgh-yI1ZimDuMa>xWKUk$u3d$_ImY|IZ zsQpy@#@H97_Xqd z3o-N}`XKqbcuk}X#z#UM@bb_bq9tUASArt&28aMA`V-vp&lB|^Tl_r84jj%UszOyn z1^gK#e?NW+YdEWoMASCXFE~^^=>)oIryeV`Htd>PofVA;`#9U}O-hi$Mh2w3Z zEnsywyfG97ytMi6XILJ=8G|P=Fd4}Yz!w4-Hmt)dgA;X#R!}mYOxJ?ii60?%ya{AS zv?F-rcN32)4ME*1*9n4h1aA&`gK4PvEU1^rB%UMsL4S!$kqn%NHDs*fZ*l>mD>0y@ z_!ZD)m;tNOc9RjW{6xG4l!lkLss=DL!F0$AzDIz}^XVY~GgG7-#%CcHaQGF7F%kci z0KE3PgItvZHZCO|K)g24r-(!33eS9Pq9)|Aa6fdF-bZ{$O!0c`>jTvgvw)j>i4uAs zq;IkTe;J~~0UmBp9vy|oi1yG;;uf_FFd|3sQi+2;(U(xDh#cg_Y+&z#R^qR)707@l z&xkboD@YB05mEC3EfduXb^syv?e%~RHeldfoM*lPzJe%$x`=6j2L4{Vyp}=-iCRFA z{r)4-QoJG5N7P{c0%utfvlXD}^>}1MEIy2$32^~~mct&D4FF|*yfS@1l;LHG7c%R} zfMil)C}#xHz*~CdL(4rFCb57@VnpByA4dck#D$p#NM5m02H7M3Fiigf(S75fYeX}z zR_HM2I04KDpMpH%_CtR7oxl=#NIfygS(r9A%U!l;ctoDp1Oyd?-G`~w&b`~UZ{ z3tpXIOwSWHKvee0{{k-=k#O$dwUPXIJRLGzSnahKI!ZJr8dB^Ke5EIQD?Axlg&5M| zcE~^`0@DmE0}oGCo3#-GkL{l!PZI%UIU<+g2`8$brZ>>1WlK5w&~lR^$Q)nrpUA8+ z(Sh#J&ks1Nc>s~kFgMxlWdR-YI)dQ5(0zpez;mqM;Xboh}Q{$0X_$FKo zTnB|hHasw#CGRFDg?peKkP}arhq4r!7%Jbtf zSn5rr7kvnJJH;w+MebK?WF zSmthO<|Ix0F`OsAinoD9Z&HgPIpNE3>-YdX7Q;WPBrz z#RpijEZsEB6^rqYaWVX>ydaiUlSYhU9lj5@iw~e>S-Z`XD)!*N;F9?_d7-QYP4k8o zXYqe=JU&30W#gu4PS%u1(c;T#tO`0)eCwYCjTs{DvW$UITB?rn|%N6q<@}gLa znzV+=$?{9(D)=A{EIT)CbERVWHF8pZ7mv%bZ_3|X#j`sT{T3XkQ6`P00;tR+qQG0JrceR7k05K`7s zxA{`#9)&M*>9hPf-v3Z^E)GqV{~xD9Ma7ATctD(ph-VWgBH}@D?k1wr!b^5ATf6U! zY>}BA%qEO6c0Fw4c0innXdBt0@>{cP?N(9QqOzK0i^^&vTU1uFZ2g|!->~O=p7;Cx zdVij&gu?pcVD1(xop-FGm2-&_uEDOn?re|ruFD~S;mOHc1 z3`RPn$!ygLxxVGGEIfmmUZ?>FK*(bi!3)R?embnlX*CMB`HHXwo(w^HnFgE(VGV1A zb3r6Sm5yi%S_$EH--_6RWQIDOr2)r7Sj#Ggv!U5Udab6o)h^uOE5@>s*^cyj4LB{r zovfA4Y;?9e9o3Y!_6m3TR>rdN+1_-a1{@w?9jgRhNX`zYW15QA0pY#A5^SMob~L?7 z1J06=&sybN7@3_&$2HZh4&i;iRk4N1*{Spv4LDlDdKL`lKyxTF^cr64uyD5z#&VE3 z)EQO{IBCK?ESfV1ox_+xYU*0g3-|hHu^fC3bHH` zmxK+zQY;sl>zFa20jE@WfVIY%i_Ue=h-ePAg7?&YYhtOw}lelx)_9KGG`XHfdem; zvdZB+GLt_OZtHG+Pbl-1V|kuT!OXHYaQ=mI)_P}NBvUmLX&Y<}3l+ZgvAkradM2w4 zd=5g4RRQNi^N5+XZ6mE83zfbKEFYQYm|5Qjz7Am{YlAZ%o#&p3wq0obO!$azLo6Sk z=bb5R10Rd<5UUbiLe2}%#M&;lej$9+SBWk0%!|%!Y6IVnP{rElToRd=n2EPtZjA~b z^KFbRNzO~nY-t0ZlCX(|zy%P6GE3hUYW-ID|9>D`fKaHjtZm@S5*}t%ISWt9g?e3imu ztSwF|I^R7j(sr};PvO(PEio!S-#crv4Sb}+<18k;jGP~y6>GcI`nT{I9}`>VnID~X zV!91{vqCj%t8-aoeqvU#?M`b-c*3_ewk$b6H7iXEK4DP{iv<@#S(J~@vk=Wa}1Rh>Wal&LVVyI|J30YH~#;-?ms0zQCIy)Pmt4GP7#n736|&2Bxjh zRBm6JAMX z^JhD?`!sY>k8dZo(vvNiJ*EY7M%2sNE{Jv*X3q^T60^xYdP!54aGPinzX z5)HEW@G5d)cy>&CM6*eB%E!l6c@{=zPiw)95;<7+Iafs%CT1tK$2CmRY2SUZRmp{^ z*=g-yWQm4Y^)L+OQ08Q`Yc(9v8DBjHBRSMLS?yqgiH2FboiLiim;<$&H8rBMzTGhx z&tc9fYzG5PG{V{g)5skD9Jsw(vr}}=w+Ewnas+eA+QEDiooDTJ(jqykIY|4UhA(>7 zw>L&h=BVee+QGOJIa&K)I<$zGQ`0KA6G<S?43K=4u-JkGD`rjB^QV1o`|*I(i{`L z;S*qMJ&U7rr`y3$7I|3@JJ&`QC*~&G?`YJbH+>Js)+QIH=BDYuj28J=LbwcqD47|$ z^fryi?-OEW2t>`y(t(jJ3a~`ZG8AHDLb}X0ohaxN#maDqnOUd<6I>KxiQ#o5#Lt9v zIc-MKTRt(i&I1WD%XDC%i{55Qoa-WxDihHav=O3re3IC@B&5z{>A-v!jkBb1Ih03a z*6NDe?4oylQmh=wb7a=*z_=H^$C5eA(L8r1sw-{l6}|71#mezKZ>CTOroZS4OAfCm z^TL^!uA*%~^np)~t@q?bGn;haG7yDX3g`MrULq6MRkt}rANmxr^~tYMPnoCJ@!Ez(ANeq>0?DV&v+BTAA^Mo5bXK7GjCrK4uI;?&6Q43xf#);l4e7wm zA-c+Hgg21+{CQ5@zP3@(r@lsPgC}1wZ%hX+647U@N1PiX`Koyy-N81u=yTsAu?@+5 z^*o;r+$*98>kwQCEg|NO>!fWjiLUt$VU@@d$Giz0xMoCOupV_*qD$QKBDzCuuZX_% zJsPXTmw4w*>cFidy3SI;8_6Z%c`@CQw%0^o`Bd0O&ywi8X&tzPL{Zjb&W({JiFrxg z@iwpMYu{tBjmagcd1)QsP7+PBnqUMfpinY8v~2;=H@+qeK?!3&s_!VYj_iEgkC!&PJfp8|Jux4kF&-gg+Q@)QUtWgXxG6aBz? z!dVq5P*ISM!M3pIN8b~%s$_wh!s-C`nJC6O0&jwr5|rAGk+zRTao-Va6SCAnsqX;S zn`nylq;nIx)J;JN&)JXFE_u?}!?ivDCh?c5Tfs^)t-uD2ybfBBw{ZAntq^L-uQ9u*~7$6+S4 zjF>;(aii@w(QV&xjEO9B%%A80*Q)4m)-z5fy39R4(s8rxPtiZVXJSlznRot)$qsPK zitez~@K$nJcz&$oR@>jAe|>6ft7lns{&WYpbVVuF3Fp?xvc&vk$DOv6=s(|y*w*B- z)ciC(xP!%M>=u{>6;iS?^y%8W#drBzFcwls&C1eeXw$`avs;}kw2+Yn=`*!6#rODI zV=TOonN_FKuZzN z^0&p<$wGA&3%oLrCC*@LVGgvM$g0&BYXQykYcUS8+>uqU2iLiH4!hmSL6^I;P(Ao_ z8Sz|ydyIoG_ht$8;C2^hvUM<*TprHC^cC8C@jSl{<9e1yvzqkavKLd>9Zqg!c_Itf zS8J)_`TmX=H@Q5O)uIP?zc`Dnhj~yDWr1GL(-w&r`1KeMDWWd0>VXsxXR{4X9$Lg$ zKHg*fvj*V8NIk2nBI2+vMC9 zDN-%)=nrZc;>CVbY+JHOy}+jjx5tp)i;a%j)a5iSRs68NF z;d>*vV_0T zX}GFw6mRekVE1}T1PjLuKp2TD*(aU%MoLr*J%;OAl^DE76}vZCqF(4T0G%X8*n==1 zT16}zH{8%37FYQPF+Q@&v2elwWRrLk`;?Q9u5vFt5i#7<9u;r)pNjGERo;b@1|Xxv z)ocfRAGs>LFlM-=Jtp4bcVPEj6jizYuIOF}G6Lx*-p3w=?ixK!26-q#G-MdwEZPxjMWkW<1jVn)o5V8+*XBI=X1u z2vnq4z<$B`KxB1dQPOz4-79|B|3d76=&IVTFS_UOy>5t#S;IEF%&Ol<`$ZOA#2_K-M^N>rFtPiW}Mga~?p~xN}j{h4#X8O+5)CLndi zN7%2y50MQ1VyEe9`;_=e|7+Mo9)@7?mhbELrI2u zvCjmAuo!1wh6T`CV)3}?M*DB#X8&bOfUI>aJ~3eex>$US{f1M3u5~Ytm~OWJDSq1j zMofUO^)8+?0hug5&i29&lWW6^W2Rf}e~X{-d$ET-Yom*&O+Yn^)$BK&4@cG}7AH-2 z+Ed~a{x@R}C)cJHrmb9>aun;N(bvSdn?ruq|--ij2GAfj12AW!;Vf&pzw2T2k z=1kp8Nt@pv6XIn|sL%`~wnWPgz#_7Y55eXf-5g1~KY)omWdf+o3>3FS#|}D0kunv8 zmqbq=WB4D`6f$bQQyM%TF^ z)Lg2|m6-f*#l-kJFC;Vrc`h-t--adRx-f*9D|Go1i~ns*;#n7kn#@40O9=KmPDx~4 z0>aJJI;y18|4vMjT$h4c%s{+Ltn6`E3YAmx^k$x}NMiGkV^XA?nrAfwEibXN-*rmS zaz-9$uG5uBy8Q3Pq@^l}q~lS1>uU-jO$920lPC z!2ZA~N7uXaBIZN7O36w82QfLm-kUdR2DU&l$PU8_a(y^2W0|m6l|Hqip zQz6JNvyi$v$$9oCPGzJ*m5*2kb$rRQ{!e1cWQ97PWdVjm;$&Zi8=(zEeywFhw@2dg zU&R`c4UYVJ3-BM3QTC_KMs$NaAGKW2?Uy{~|1{Q!Z}8>|Ex?XQF0em?A0an{^D)as z-2ut-{?D*SJR73rL{ev1W|70DPo0w01Z zDNFR0kWMUl(I3GMA(hl6RtxYhl9$-moQKd##uCzUMJJcM?7tQ}gjX_`3|WAckz8bd z0Y6Gs@|QR*S9Oh&|M|bb9`#fTmW)||vyr^Q{?hqqq*ArSW4W$VNnZ7T8GAHYsb1o< z0HY)Eu&=`^Xd|)Y#JJ^#?y%&N|2n2ZHaeC}Sb*`7yvF{@sX{lpmqaW#bw?$y`@f2* z@QvOjlNR8EB$wGy_%U*0cuCB1OLt82hChlu=GhosGHn62NaAIG?R+ePRdn~yzwIq!IE=l5JPr^+QLMg}~(mON~zkd>ILJ(>}76D9?B*6a0*@PmD0*J`$ z&`E;+Z(>b2!Yn8xfQOQV*x$mBlL)^6CUQEAlDGWdVvlgbpeY24ofo5z5ySGs)&MGqPW8@dDnjfJB(C03hD`9xFqkfzjq!+ ztK0=BQQFZfdEfti>@Z&CEf5mGe@U*ee}JDLtHK2sQPD9V`N01J_JpS@TF^uQJ0=OU ze{?<(sY(>!M0JNl@}d97*b~XBR6z>?+?iy89fOZRnqv&S$(g<;=;}yx5{-0w<@y*_)lLYX6lI!dQ{1mx4yfj7} z>3B`@l|O+!<=GrvI!yo@D2cLvaXuBE$ajpQt|`)cV}~?T17=V2Rp)&AN{|_nv>OPDytJX zN=b};3qA&IA*i*TBOM=0;{IFMF=UH_THgr_reuo!hw~V^#Z5&!FLZn+`N{uB>=?eq zOBHqkzbU!N{u6$h+!Cf@ofkX4ko@fb6MNdTB}#4T1ol&sVE^TOI)ZUbOJXjnPw;9;}DawOy3#m_*U|(KZzYjnABy~PGC+Yzp-ySkE2Y+GP3hZ z#}AU<{kLPsaVB%wP$%%Jl3VP*;b%xDf0?uMYR8o15C7lTGaja3*;psAu9835|2UtC zFjdPuo!2`OlE3`_#GXkq)yq!!I)RgwB-wXhHMEsjHr{!o<2T7|{~b(?Y;`P~=mf@A z@;CcmryAYrUKZ)R+3~03AOF8GHNMrmY_b#hT*)1F3O+$@4KIsz-s<>U@-KKq`h;g| zblG$#u)UHL`#cR^WoAX`D1#3&f%nW?0kpcT4XI zq$yhv7PT7;I4QJ&SDl8T7gHFrgQG5wUR7;A#Baj z&ymgu+^uZ&umpu=R$!T>GdcITS|cn~A!04iQ>3#3_rzP1EOjBv3LLaFgOg6vKy0G0 z)>^F3md*~OD>Vq)QCM#UhFUs@GsC4p+3rHrTB^^L&JE0nYjC!=P-q4ITAIn3Noyn7 z;X=$>q0g7j3(QoudDzjyCM&SpQVM65t1ZG#6ynxuJykkCFe~1cWTy&StiXLsvp5+v zEySTL*IRk|BI$xahEj`gsLQQZV8*4{oY^id%3&-gt#$em>B7M5xEALymk(KiH<#vc z=Fr+n4u84Rx=&A+E(*+1wtF~&=}@kFdBl21UnyM@m>1XKT<`Kp zEAaKw0uF`NL2|>(W7Z@3P12t+bevO*4=@ei3Zz*6)3-y@|3a+F47o1>`S=G5<#(m9JHlJQ7E#p8mGgL!ap|^$fV(Gd7q%3SxLztP=j6G} zXbocpX}hAAOV(I1jl za+bI(ks8&C6CT@jy-K<@`A>_`;j_G*JedS75wyeqjQRor3+@rtyb z1Jk;pos^Y&JI^pI-5r3H-N;VrN~^urFe2T zPW4KkU1%7SqMTA%FSLtTIc}F4UXnHhN|n9HF2~9VJ4kAz2RLh7z349Y%831t;T7qF zfi>}7e3y6Sq#Xn|(t{iZt&iLlUKz6=F}x;yD8Nwmd3HrtPTN6_BNcGgy80r!5-XGT z;|8zv;lSE>UvgJ!Wm;E@!6y}R%4j51M=8nZ(i#F%QJ_poB6ZY~tS-GFBo%Ykxk$8* zQ37?D4R1>&fpu{buVa=Jc7f1G3f_^X^^I5ZaT_6LJ$~o&@ z{gFCV3DPxa2ul@#_3{2>ow|h81tK9S#;KqUK=%?QwOu2IkEO~$g>nG7*HKd61rj1@ zBWHta0KL~;f_7and?tM)upvHx-|HH&KFjT{c9ej|DcyPbTk8m9%t$I7!;XL1=>zpR!8d6*7D)eLR3D z2N6DXm9-0`OVY!fD%T*&XRIQ-t{8rhJ`t#j58{00s-Z3rHc5|gHqlOzeEuqD*Hy!m z^vS>`(5ary_jSDo@vSLqhsgU~~LblCNIn>jF`f6z5da9MFBl zs`0KHhTo*kfoi1#xzDj`qAO;&B|XO3;&Pz(xmQKHZW{iSJ{{N+ci{JVS50eq2yBg?PTrSV zmDUYnELjVOMH_ zdL~@h4MHxNmcyZ)A?x`t+?`{bBWn+ElxIBk0=TRjWL`2IhwC~MsaL^BcY%>2>j-e; zXOi`5nAHs;FqxjiqYXp53Anbq*qALd1bE6}WVZvZ?}m*zG9zc3YZ%?_hSBa)W3J2; z*cKngcY9%BcbO5AnK?DIv*hkDjCEHS^JSJmjqzfVn6#npdLu0B=Io@MBlqxW&hC9ix~wMvwlq9@1hlbk)L1I(!3|^W5x>E z0OwxUd33Lv7U@1@tdyM$+#5fS@Ac9qyPJ%NY>>mJJxlHl(_-C6jGJVq0(|AOp1o1p zbT@9SmN_{0xt@*eP0*6v$Bj(c>A-#QXOnwVw6q=&+sTGF^)x56k3!Gr(Hc3jGl6=g z6WK?lXZ3&-Pd3ci?Q){~7<8z|Y^;%;4eX9P@qJ8sVUN{VD;weLp}EL?d^+6IZQLn4 z7uch8dG-nDWj!F{lbz@6b-5z@RCJ_g(8!lP8`v9nCHJZ6tR4{g$()>hv{C4Of?nG* zV%#Hh1@xxoGLQT%={UDyKxLD>b)e%f>7 z{b4%RbJ2J}_IzN!@;T4_QF>Dk$OmO^&I7LJBKIfgc+X{{K=wl5f%tRD`&0Cm9uODG z#yBYL0<@p9THg~gie)bbP~`<=KXtXWXWS^2y~Js7T|oCUR+BwfjB?q_frj`6d_Qyb zP!9+bWfwUIXwQ@T`KwPjd#)NAW&aBtP(JV3FIYX+1HwhwE1U;i&qwyFR(pD`8&$Gb z0}sZZPwrQ*_Vs|QQRd+sq`9F7h}GjgH;jj6mjVZsZsY;S>WLl@J<49=Jmhkt54cxH zdTttz%3cpV6nEnfcvnyMfFx3OnIoXRKt2#&9qYMeJSKZ1AW**Gc_6xax(5W4GB4+0 z*9(yc604IvcZ_P;n}LVpFC-sGtxoGrGquQk93gECLMf#gz3C>6%pVXc#}JfSn$??O z(#irHk!uV^8KqEfrb#CY21M~O9A%ak_GX#%vJgj1dyz!>rEqVK$tZg(AXdKUK?S8{ zy&%+-z0HxhUW}lsQlz)QM9AI=Na8OhQFSS+7i65WagLPs64XGH*7g>g?6P+QQsqlX zgQK**7et=2_c$`wOK5|;6zwfF^~&B4$l@>I4c<~=FGxUTS2%Lo%Va~i6zi=p4ahzS z$dxa98lt65y&w>kg*gh>%aMjeDc)Oca>zamDB>?C8&ah$y&xZzO>i*UMd$!!jlP#> z8kT(&z?2t}1JpIvUJ#ecKISN07tsTZHDqs{>AdWdfHHm&Kfqiw)C?SJF} z{u*cRKGUe|(?Fy0f1U$^HDkRXOqG4cdBpX<$N|+FPwzpKTlRV2k@)|T2h?kPy&zka zML36OuRsqHYsPz}rk7;b0*92ZAP+j$O!R`NRrUqvQP(TzgYGqv-b1EWWM2jzjlY6F z=v_0}+hp>{u5(nhSIGy%Yht}eOs~nl3aFH?dLE3fneN3+mt|4TW3E>t4<^B2reOA-B>;~sB?GkyA&w%^7P4CIR4;)rr@*ET}%KAvt71d0&Jo&c&_e{Hwr|Aru`C`qqI?Z`$ib-Z zbDFNorZ`W!UPB*pGtj;Zrq5(Q1)hw*hCk$G2>ZrN5!p@7QQGU|LtzHichU5P?B~Ey zciohb{S|mR{zg)uUhC_d zG)>EroZ~bv^f0k@yzhqTH`(pLaite|*s*q^FJ`(W`zmtPp~!a!-dSULNd#oAy4OOXns=2FN4V(^Bnn%phoHU2nA(jBxKH%&*Zkb z{1Kt53?U256#1-RTil-%s>@hpp*c&Q!PU|NkccR&C5z43^4USHGJuF2W%VR%&XLdI zwz~qT$X$k#rRH4u++ce=fQ!6kLbA*Z$uqe+T96cl%P_LSoG+gj)G31=QM9ayM9c+p z3b(@*jEE9tI9Y9`%I60=;=!aSRn|hX%!TqSuAUZx#FTYxAgT+CcIMAn;Oc@Eb|dy5qF*Ez|3X1aV)(5QUNBNnV1 zBT;jyJeO;7y%iCw)_KT-W`=xm&=h|wDORuZkwSBs9O9a3Z$lDd-8d;Vm&@~lX64(6 z#IbII#LN})e6GdyHY#zii;#!RmGUJ)OZ;tI;$1gMHklE50hge?LrTKyV&oC?Ci&7J zp?t?9iLRR_adWkt%I$Q$6Okm=CCTGvrhHkjGyYCel3JJ64{~>TA=gS9hoqG9jDD?| zBVQi0D#sBiwLGgI#PISWuFW-$N*U!)zu8s$f_A-K11q&guu5y&UFt)82z* zM0suhh!hVqG%S*Yv zwD(C_xE$-hXg(lc6YN#K?~z5zoBBQGgK`G9&-H#pmMF*jFPjDOwZXpl`$<`xiahssl1%q@4AA@8S77w{a4I#`TAgg{0c5- zt{>{3Fk|ux?f~rrQqEuR?7wPmly3+QC_nJX1?$K9Bj!W$O72P52NAhyy{G@WStZ{X zJQ@EWDOa!e^-r3c-l2`T;wTZ{nVEg;9ljeWd@U z`KWwz@KijEE4=F``vFRjS92Y-4@pIMeXReM`Ivl5(4qX$qlm7b?gwN+&g7nUeHc+B z)+hV#nAP&F!PD^%lZw>(v;hDb0fCe%xIypCZCO&~<%nI;6ltnM+affLikr=-M9>}p6<=cY8%8xvlprUL5PziYr z_pIxq2&Sq)1_~^Me0%V0{G%kMu3!xSNFlG~j?g}alte}CK(WOx-w_;9evBv`74-vv zS;%*C&$&JZe=A*q4wPDY<-3CC;veHmZ-sCGAPjjO_dM+rQW>tm1}ZEA@_U2lm7jQ& z(Tb)4Ks4lh?z65>BFaPsK2U9O$nOh28~-G!OjWcD0Kg%y=Q?Rup+?FE{Q%E0EZ-e; zDz74q)D6}Fz&qr7xGvXKw2`ra9H_ILm+uX_;#cuT=7ynxdW%!Ok2^~Hlx*a0a1QLV zjLPp1jw(O(GzvD14WO0_^8MWBT%Sf7RU1452Q6;-1HtFwpC%jC8+-!*j>u8&1=?rO zBgBUB0jcFBc|-7m@-yTS$A*akz)Iu?xX-&jLmzQ(hzuOEydr-v_|)LTMwG51AR1UtXK9I<>XR|YRCzd#;!RMww#TCU0)x&L#0 zfj;W4L{DC@d?tS+_`moU_@mxR;mI*eM1F|-3hhhs(QqYp@}lJn`J=&ClwW!tjaD|D z^jNOTRoquyUq&8HRN^NuTcYyEg0IHEOg@^bY&i*lj=YKMpJ|0G}!3rW@VF|vi{Lb@u6loenhyn${eaH1(aj_xV&3%vdJ$aa4(BXkYFhKg73$_PaalR`38kVnS$hAq5S|oK~#+oN{MnsfAEU(2jmGy z)x;o1R44|xAGm%%pKw=21`iRHij%<);y>U|c&jD{n+QZP$PLqeB%cUZ#RiWMn-r&l zVdamWC!$r;gE&#GaBx3#{TO*7QI#A#PB0axgCE9!Og@pSN;}m;uoOew30e#~0^U42 zr6o9uGrM|iv zyb%06{!{YF)TWkGK0>G%<3?yVp`(<|C-kR6gjn%nFrvJP9Hnlyo*E~likG<8TsP69 zjLqb!D}-F}a`0OGCVrH;dFa#xfhjI>zo7k09_4R#p1MjjD*hMzLiw}js9^KhsR(gM z@e21#*Uyoos?DBL*9n#4)!>)$pOZ(`n|-Gyi6(`Id!3emo+37npSnRDR$K~RS0<3B z9GfRj#fT${*SKG~66jOz&5=_#iKB|wgI~oH_*34^lc%N$TydEjrTs!a72X^>b&EKr zcq159{^EHmx_SCkk~psLa=&){5_u}IIeF?1p;o*Z{5t+i@~PD3GzTDs3LkfpHVxsF z>I_GEr$*rqPAaDnoLZgb$mrB60^Dz0(1nbn03z!4Q8?zgmG zNt|B|J90XWinoH_Du4Chg6c8{z>12ux!<{djo_+k#8J>mDBcNv7ymVhtE*WKKp7R| z+#9stpk|`F)=}JPSG*g%q5KVLc2w6p0CZHm$Nk>*8`|uyMjfS{y^8mP-^YK$o4wUS z2VjtjE8HJwzmv`3YRpm5IiUC;_=ECyPjj@o$$@kZD#F|!UB5?~6Vf2?z)8@V{9QEb)DxG zp9JIaTlg{NmLUh=mx`<0DcT?8G5!{(V_)Z};?v-i@(<53!Im)xz?q8AxIel6h#XUG z@i-24x)q-Xe~SN+Jf_~_a{#)jh;VPx{)C<;wv0QZoi8b_1#c?6^;hKS#FnJvc&AtKb?}$?U&*IaThdMgGOC#5PScXmaSAizw6-&#_$D~5 zOd`jr%&gP;&XD3;?ys&SdYr+8PMbU5R(u!yHJ-$eGns{_0XJ3L;QmItO&;en;nUrn z?oBpSVe;Sm+PO%GYKYs`f_Jf@k{Wp_&>>K zQp^)ArvZ^wOmma8JCK^PRew6v`K{vDU{ZMpQB${CPXj=!_>Ftpbq7^5wvwl>bpD|D zJ$O5Q2Ujz<4xOIpj45t$|EB#*s`*=;r>}NSDgFrlt^C)c7Hl0m9qGKO_>=pO>)(i4 zwbgU_dS^oMSMZAI|Hu>Jt+CU$I{#Mu8%!zx^PGro zojwiFuOh|$&-GvAL}F|5^qtO>;=ka3@&A%1Qd`r8013v@cxm*sycP;8V+cH}fZY{J zYfP(Zp|Y}u02s#Z=G`@#*3iOWK|`5Vuyq>(kH$5(Fj<8|Syr$}%e$L?SAPqi1rOy| z!ES8m?#8<=wFp>cLy#4$yYlWCz3Wk?IYz2F!q4dVPt6Ckb`XK<7u{pdMqjxv7x>@K@sTFL4hGtCN-Q4PB35Nh<#xi*` z>G$-vhFRE9g%#{rkx~Y&7y_kES9@K z&=BC$SPpLveMY~A&vp*&vw~H^(458@mox(Q*buA%50oj)#hMN3;~#p<@4r^&TMFNvm--?tYCRA zG;eBVbDNhvIRw}?R=}gsXZ5#**|DJ`RSN9-NC6p10Xs^ z=PeqY)6nkbpl2>v!LCtg(bSygb}vVG1~7H3l$T4N+ut7MU}r8`!A4Lhw{h;J_9&<6 z3_$D{gSU8e?zQ#==LCM{vK8#@gceWDz1^PTw44FR9V_EO^vpaRg{wakvV!fJ5Y(7i zrK57KXU45yVTPACn%SUZaLF@QtY8NwlsA>xtYdPA&Hx~fRq*oZ^ZIpsuJg=QE7(*C zn|MpXD}xLz1sEzY4bws(-X8_yBn7n19^RIOzxXCkjtYFO{v}|hr?T!>TZ8*&aHWzq> z^sGERg_kj$ZUf5*p~A+jDm|5#HJo7s`vtt^qgf4l1`is}w1Jg@(DJFQW<8TvI1Gva z7>`#(U(m1T^Wfne8z}aNiW(PO(hGQH!=M*{)$mq~F1V&w@sQyH8>ruhR!l9ptyl9{ z!=NUB)$)qz*?9(nS36v6110%TabtFs!NIE^2CV^XCvW9wc7wsqLx)RkpvoRvIhEaP z@bZMiphSSx@k;0m`wd|pHe6u?h4WBJkw@!r`gEdHZ)vs1EpymoxZ5w$lvB1-e&_n=n%be(Iul`+t@H_1J!2U>d{5ljH+#( z;e$5NRt~M6T6Ei}-sT$?+CTxBS4z*#GZEXyhov^qDGrr3=2n>;+a`uF8>k8M){N#h znB3bU!-s63`5Rg@mD_CcZkrr#vVk%$k3nDDZwhaV4Ii;)SrN`^K}l0JIdQUy7U^M+U_}f-3H2Vq0LiEZxib6zO$f? zf#JMrIyJA8*gk&th7DBKLe-7bs!qrDiL;=cfgR&*8KpLKy0=Hp-n4;YS!l}?wYk%~ zeex_QYhcHDO!~6^&hYlw*;_VH{|Yf1mtE?NZl692x*M39w{>*cwa&!$qm(9sHQ zomzIgGqpW!1XMYcEj$*zFwaV<%@|3ygCwW9 zBL#NQ9SU)$mfyCjYgr?p2BOsSc=Vz?8&O+3QfvnepAfIHsLJN3tseoc5T%i~ZM3Mt z=B`CYO6{PB6WTUa)NJ$C3P(UmL}}*L&{y=^!nN2)g&nkILN$#mF4>~BO(UQ)q9l0R zM^{|4C2H}JYCEX3gtkwuxNS?-wv2!ZiPFler5ESfDLeEdJUb|!glZd$tL)SrR`6aE z=#ubuj21W889T_4Iy@X-j2yIsszqqm)XLj-^$y<%sGlfFULCz8uZ!3*J|eY)LPV&p zv81ZYv14Kcv{94;yn9DW8oJy&A|r?FptcaYcdDeh%e!N81e8;hgFHTcRex7_M{MMX z9h3_~{Ki$6x}rO#M?hCa>EPWry6Re2Vn=f1xE(YALibIry4{u9k#-JLSCm7%dODof zP1%`oPHP98K2+ZbS9Mc&W}O2~7UeK+_bA-Z&DaT@Gur`85AB|Uo4c7i3(tXKi*kgw zhfeG7=I?~hb=v{+4((~AUFsI>EIUWq0mkO-9i?6CR_#R24cY;j4(*+y-R@TJWStwb z1K7;lN2llY5Ibwnjo1M{4()5CSM@k{)}I3{809GM{!w~Ek9#M2?t&dq-q8J1^yVJ# zPT@IFicwzR?WeEq?+Ncbft|Z(2P`(Uzj5`Yp6Je|bD$)nbn_kJ(o6GtDZBLNLUzD4Lug}ZRWEgy_1w4}ph{lDXlX+)V;6bu ziXG6$P{UMdb1!q((76da0E)Z=^fmpx{9Vp-SM7iih7L5YxzsDzHFgdZY?QC?9voeB ztyi_nbMCqwki5`?Q)_Pbs(1O$fxeB>!#hZ4h3B)n0Ds|$>FfGQejR*1rwbsJ zkhpQ(B~nmVb{-UalyCDSqwB7bsygI+K^FiYA<5Lb+oZaVbsqG5l;b=py*#gB*6Lt9c>MoGShZIxmZ}+F_TF!%xkaFVx z7&_murph;tlaY}bl9@3wBr~IJA@wt3WJtxfL+L<8Wr%=^ird+CHc=TOprWE8pbXf? z7z2ikoiWD$ZNM0?b9RP?WM+&E$;@b5NM^>!@b|pv@>O2Q&Uv2u{@$O@y_gL0I+EOf zbK%vPGLV|vGC1pK^5D&-SD`WxgvT7xu4Bu?HcPK=Dg)WJ?T~AoSRS=mbrldqhi=7i z$REq(tj(IM31uLGwsANgo8&2*4OanBbm(>rSNpM3&fjdjnp_5QVjI`>u~(kE*?JZ5 zMTf>>VDfrWRqS|^g2u5vu&L^g-iOn-t0gQBLB1WM7#91}HdG=~f8Hg`! z0@o+rs`<@6jrnCDk&Hpe5hR5_%dfGZ41|m}gcCti1hWDf3(G)m7$ek1uoYpfz{a97 z5Cz(Vt_ZOriWSmWS_aa+7!jE!Q?OW}jgm4D*x5uJnn{tu3U34;)1e13hqW}Pg3pR* zRFr{O&34#D^D1&#(T#v>Iy4oNNsc5{7qg&7br}f1Y?+)$TD6iD-v}_LL#~)3+DLY_ zo|V|BD+3Xg?T9N5wPpDEU)a^$07Yv7-zGNw%Y$PfgVmETj=YP=}^t zj%h!2R!_6C8hgt?++#cD`qW!J&&qEEOw^&7m@M)JlF~n}u+dfqLK#~YX9G>7RCSfi&5wDh(Ut|)PhC2q2DrVKRcF}dW8vYL^&sm8}; zpzpTjayFW3CgNrqXUjk<9h0Zs=&YHJn{9kr2D)Qgo@=AGW`SCwZ-v}X0-(LR#*Y&3tFu*K3)?l0qqj~jP?t5tv)_cLzRFI2s`8YLR@Q!Pt$}; zKu<$IOa4+;I})FvStkK)3U-$BrKxrz9??WdK<7a}r~T4dI~|{;*&qSU1a{8#rMGrI zK3@|p0sR5JkQ_s*^WRdaiIISkA1ma<(CUJ>lxiResN(78wK43vuq{%}CJ89avGcAN zab46Fl_p*SYHj)jGF?{3+M>}UNI?0FUEt77btzj68UT!OmeVh4>CQU-7NaIv0*YMh zqKodW%iUtt0Dg=UNG~EYNcF{A2u->Kl%iM>he4}XZW+}8oQxAhzoccb>-Ag4G_V8| zjMya?LtJm!GN}P{8D|B(n9P*bk8GLJWJ*9)hZS>}ruvC3Ga3M!af0b3TBfspddsXP zM*?auti;9i*3WP8X#(sSCxl)~hDd7vt$s}f5>O#wr5uQ+4&EBjR44&O5B;(hVynZp z1~wH*Kplf!c0pow)YgzDK&Ek4)31Ky@LF;tYKtpMpK6b+=5sI=WA2L#8#vU zP;Q(sdZqSjXT$W?tfpQGxT~;A*Vo>L`K|d)0D9wu)1~B1q^thh3Y%;aaHC*S&L-N` z;BBQ%galj?beVP&`)b%WY14HHIQKD`Ym@kD)HYSqs01A4bUAsm>?&)Urs zv)ObtWt*W15OSRL^eXLU=T-hTW78c8I76{2*Jkh4+-=q-0L*bB=n686)L6WYXu2-} zCmp8XuxO3SZKF+qpW{T*tFW9%wto2h9c0nq}gkHexjYPUI?rW3NX8_L0N0&8?_^ES;V z@F%2hyrVZX+s)fqIk3etICa|?(+ojq~<=}6{np_ED zZPa#^79fC}1iF^IU8ZGi*Ju;UwVdr3_|K$G*>2D#mg_h>=%{wPQ_J6O)FzkfIXf}b zwcV@D-EP&Ul^ZyTbRBsI2`%1EXw%EvIBX334>VM{eN>xK-oZ(tH*0sWQT_HYEnMEk z*@ZQ`c8F2S_DL4J9zpoQf&&R%+}Hj%9h z+YyKsmE)X!SgR{htc%(af|iyOoOF5{nJv??c7&pm@?p+?4E)?CUCNGd6p)CV1N3$+ z+o|L4h(HzPBb(N? z#mM2)yS2NV&C@%w(BAS1jsWX+?eaFy@5o01)W|{TMshMq@4vGUwUyuF2r=;cqv?Zp zmZC)YeU6B3(k8R@VLPSh_3{Ut!aV>HXSNXUp`?S@dc7JSPw9ccpq;=6Cw&08Givr(4NsB!hpV zpU$^p0rw;ZK69EOI59xySK-S&MIX?nu?=B~fx5*N3%RGU0au#X5S19B1E?jpfIdjx zBQvlPLv_n47IDvD-~lliQWC>;fV||Mr4MQMI1T*72;GVbf9^SK$hF67$W4sa0T7d0 zNVk#ql3I%sA>HbVrQGuvc$#Rf%EWjb;4--v=$LjdyH%fia=+;#PaW7%u(KEG9BqBOMZE}n0cI`fA>vUq4ZbL;dw*<4h_IX?9 z6Z3Td;^dan3358A&7WPUi>X+}y^Mk9l-3r^F4aL5tGQR`4sANSEsQPIZK_zqEyEnH zba7i0TcwMy2<1xX!{q(4HWpi>OQ;}o%Q5g!o7z&?1|1+ixfS$l+WpQpKHI2Et_b5+ zV%J>zy=}Q{s}2C5Tq*rJ`2eZCm`&)?D?Z}NFz^P@+Li249pFK^a{4#g1MGG^drSvc ztmjr?-?$El+b!%#9l%4m3i=57psam_J*CU6h~!pd;B_&zPq1fnfEMK{={K|oo$b@? zSzS&=6t@Pu;X3GTpJ)3t19+6HqK}d@NFDx3e$53HUvO(N@NUsMf|CN83oBx{b@Xqw z8SIX*q`>B)3I?|x`_`2q?ubeXX$EL1S53bOcENVAl0us$6<={1Fz_OpI#QCto23<- zxL4`lX%9I&_(>7XiV7CD5&O<{$lH;d6y2<&d%witmfW|6mARlgNy6!oKMPcwpOHZ4fHWGOzQIARoHB+ z*voCjz0D{7ajd9)&f+d&^E^JHDD zU7F@w6&!9S1`ZEXSIRC!^H>GU?V|sv_->R+m*Y^+6;(Su91F+ z%qMjh?;@J-SBSVK44gEyZso4gW&pr)&GetNe0H~f*I2Ws;wZNV`^m)@cUyK%HUl1( z+e@Dy3uN6RyQZ2SS7dYhFmNoHx+iwcGy^=AYoXuO3Y^{3yJnl8R^)Q~vAZsTw|jn< zj~>vnTq}K&jF61}$$om@%2V6{44h#!V{meS-mkKNJ4pXoi?EGh$$|RCm1ns_*v~FR zY>Y|{(Far(a&7c`WTDK+N)FX8tGvL)FmT|Rj48?CdVtn)ar!S>q0`7uj?k~DEauv= zUtB`3F*i9{4@g@sLBCHHkxa$OkbZUL6|MsVrz6dzOpezB@RmDF|5YnuoAk+vdTM0_ z_Zs%AOC&Z~lGF5n!{uJ5KOi5LnMRT`^y@0++;1>&l$uNv$%r0cx!e)@Z`#97({yr{ zenX{_djtE;b=Yf~PtMl^N|!rIpCV_H%>KIz^)Z!o+;1^(?$XS`yG!*@rJ8$_{<}7l zZ4TQl)o-f2%KZ-e-IXaeN9|VW0mI9^MR$>p$jq$W8ht{gmis*h4r!A)Ww${OkY4U> zx?6k1Y3A=X>XR$=+#fKv>xkE!yW6S4NeJYDXg?{@6rF%X0dz2QUY6wDsk>F*q^Q}ac@*gNK0uY!M#s^L>9|>St+3{ zlFDK3uNb&kOuZ>7;Vl3q=02eRr4>7S`6&@Cipml0Z`fZhv9~ujB`>-Kkj307`eSl7 zsjoN%YEf6--sAp>feVgi2~I6-Au8{4AJM(q9JVDaRoZgB@&WfR%JtWvtS}eM0|Nd%|hqry5)CR8DjM z!Txof@LFxnKE+>&3v;N$XdpjGR|~S^r4tRLkSaIqrWLxMxlM6R9&T0DtDr)1PVc zoc+_Ovn@|6=ef_YXRbVN|9q;C0npH}55tF&PqO-_`5Anr3*hH)a4FNQ!D#^oKdCRg zfbkrf&$fo81sWDh7sAiu&$;u()~K`)17M|LU&aE;Ntu7HN2GZ5?a6>4BI0$Y?7{lU&CK=7l;R=_NWZ;(olFAV-e+y zY>>4_V@Qya;n#6cb(jWI_81HR*@l-hUPjM22l;!9hGc0N{09EA`;2!mcaPNo2yQr# zv6ym}G*rBYFr-UAf=M_iQD{TTJ);Hybi+Z6SJ1QUA^n~)11wz+zlp!%J}Vxw?3pwG zz8hY_@TZ)U4UO!XGGt04;kR&5&zOcL_RJUn?hOYsmZ0aHL(_X^4LQ;%cqP8Xea<^H zzsIK)(BE(fBY;v!via}zYb}s|0l$rd;)iAn-W$+bD2;(vF(7eg(gagG$L{OW7OV3Q%!)4dXTRywk?tn-|fl zkh0+S@Ymeuy|&!F(X9X-heH|5C>Kar@m{D^E!_&gkAu33hAH>Pw*rzJCNo|~FR(HF z-o#d&bO%hqUw2;+W0t*XtpF^Csf^{6i!y9vZ$@i}GztCy2Su0(o7juA0?r%`W4wW0 zbYjzcvs!zlDezkS4fjPaHorH&6=3IZI3ti!M8f^|6}H->d*KgpP_fbQ;C-d7gft!g zh(SV&*m&4JY3p_A0r-D7$z3GIqxPv0<)3n}_a^R0~P~MsFlzoQQ zF)0kMXS|7Ca^n1b#@0JhKKu#(ru&i?&)sKjos=SQ1Y-rIm}D>BN3`CTieMTJYC)P^ zxo@;}N}35rGTuUq*>?TDu~v`tDEuk@mb+MNx9pp2osnk28yLZq5}AEu-&E^kX*T>B z4hlz;ePZ8C>#Q^fj$*7tOPu!UeY34krMd7%e5Jd@YoFie)8-?~hoc!Glu{DmpYGS@ zD?0^$j)N+bMg*q^wE4*j;4c_&qor&jEIqJovFt4TCH}U%R7^yrhqMLA3gH;WD#~RU z!AcKpTPC{z({WIenuwJ2@U}o%5zJt`gI;zL{Pc*n6|!QOiNE8%>?Ly3quWAcr7*-; zO}RpH6sJRNt7TW0-hzAaQHfxlwBi(X+n^y!IhR9OZ5HU6&qir8UEPiqU8 zN#RY5HIy=$VmATX0Yr(}sihm$pGNHN2JaJ}O}khwYcPZIWGux8d)* zCF0?z{i?QjnFdZ^kSXP|Vb*?4TY^jrZ^uCyZ5mG5Z)i)D>EIm<3R>AL>`zxD#zAiM_$)jaKb@PUB#LYWQT z%lHtLvag372y8Es;qX5EL$_3XJ?cP6d#Q|o(-|L8WU}k51EK8_*)Y5x2c^I1ddh*k z@OG)}I(&ffKUC(t&OZ>*u8@tu2l4;8W!~$#2cp}78USZ7)=}i7Z;B5?>K^pNt zSlDip-GhZVXhCQr!3Rs*3E6#E#Gs)H_DI-4Y5R5A1Nbmbb1TFnQ3qA+Kstak8IhD~ z*$C^Pru~-81s}mdzhW9mIcR7H4g!3X@hMvE9N`}{w%?IW!^iMX-PPWa+=JG3pd`Rq zj13ee=|=HEqW!+?5iG_xaFw(h%7dfrz)*m*8K0p__6_~Pv38H_349#?%&ipPupFFh z2f_lJ!-%5P$Zm`roN9k8n}bi_pc69Pm^e7o4*Uf;m$4D8ao(6dINSbIHV^0F8{IYD z8}kQ!I)Ki=%V$JWRHRY=48IOv`2yZa9JEce(cp}L4nMgs?-b*6RK*?*%LwdPEMLew zjeqV|iASR{LOOu#z$;*UL8+CEvNA$DmdO|K&fuWOGL5EWgm(b$qFg0?SA2-*NSA-a zli{F=qOam%c02*L>|FYFj$mE*>@v{ zraCg^k-TagbeyK|CJxPXWXU)1l#DpE(fQr_h(j>dZ;^c*PO5|Vh8gS6dnr@|V!aITN!Mn=XhBi5G@i`Hl3OS3{h;MT@d2i)% zqB~Xcc%FulK+%%EFXljH7%|(g{Qno}RH2MV;SIbFw;n0Wqb`)3DIV#WKbuf}B&9d7puBP*roWtwHLH}*Kox(MAj>%zO7h@ON?7YqA z8awaE`Mhp?m%G_}JC|$i1kMT1$VjH>Nk0^GiO&0S5zmAtbM>?zl-$uyprG)~jNPc7 z{ezx6*6EQS<@MmZ-Fop47VcyxFj9EEj1)?X?1vHVROe%PHm?r{oxSOY3GPfM5L9>; zMk?Ck{9&3q+xb+U%j?He-7Vf9=D9vyK2`ZVDj5nK@30}3x)%_ z{Hh9ggN!|>fjt%m2X-y4I?Efv_qYw>u_!pCE1;^7XJhQ8w93XID(2boeePE8SS}ph6;f5oBN*wFHj=X#hPqZ)UEw)! z5HQf3N;tkNv`WGoX6#4X*iJp1*hQ_X;9bM_yW7N03!K&!UM1ySXB?ok%bX){M%TJ3 zIqw@Bq!A|P1dMb=R4I5PjDu*q(>V=ib#16p@^0V<-R)lIJe=PJJQ?07BZJaG8u#ZF zcEwcH@xH}DbU_;r=9P9qRchW%#v!zWJs!rBc5SM<%KHvK=JH6w%JZl$lYqg_C`;f*tRXczlOJ#VZFuIl3bi1XZC;vX%%$u3~v z@a{19ly2FNBfP1u%qlbQCmbXwrXMGGGhIN);Y~0EXt(pnY2IvCPL+js7Z7q1_4Q2Xq%!*?9LDLe$8<6UGnhE~>(Lzu-c* zQG6$gpBK_yT1D{gGei`V><)_`+AXOX=KYF;aK>~eg&*E6t-8*8z&MPWoOk&Ah;BvI z2=6!iu-oLllgp3pR#lDirWlzNGwG*dKGdzQy2<+;2T2a?CnZ0=TT^w5=VBZ|&Fr7_ z{KRfu)oq>|KjJovf3on?x(!uhJP+e2rAPMD2tT8{qiUS@2M%H%(@zt8q}y0^hd0eQ zhW0prn&xM9_f}2t9^%K`J>H+@`T5<}s!84qBa6~Yn(!ACcH64%@&3d?Mnszk7L;}q zRrh(17-F=SJrO35c3-c0!21gqyL-hGQ36%>Xw?+&F(aGOC!1giG~KtVT)e+=5Ga`@ zQUr$Xu__Pm3FA20=bYdRjNNytrg{J1$K8G2iClrTd$MYVH_OPOSV(t^1w{A#sz*F8 zp2M}!?kWYN-BVSMd2@^tsD*u3FBt3gR6XJSi=S{?#CI)%$?ln|S>97dE~Q^~cSJDN z{kUq5_a6?TD%0Hw!A$pT)l=R)BMq}wW-KsL!jLXFE5i})|tK?r7=OhLkpfr`cah0N3FfOC?ML>N~n{P{21PrC=aletK= zF+>r-_hS}N21!2`Bam^mVk!S6J4noEKP!=VW2j;oe-ZNxI>`Q6k0ctYisk&5?PuJB z;-4)@nlW4v$Y0DnOBs^=Jc48x*C~Sduh>DnX8L&oL5vZK6?}i@IdsVR^E8rW+@J{N zFR`C<4|#u{NAiu)iV%JPvyfsV-SZa~8eycthy+c)ObBO; zS&9w(U}gzwcm6UhoHgbsqWCNAC2qU-mwBO&DPIxI4`G&42-1Cjk)NqR@df{FJID`d z_k%?Nrb0ywe--mGO0e&TiShzXMG6N09s6ZBA-*3a3NZmai@%zAh2oIiXNf{h62({i zckLi;G~G`Tg`0q_#b3iLLmkfhd{Kl+pzgk3TCZKllsmuz>HQBEt zq6|}qB8mTj9psp%UnfL}37B5|FlHrs&H3xJD9hBVNa3%wSGuoxf1MZQo2-g7emGM~ zxlVfEf4I;dbr#&k=;;eTug390Ep%3*_POab%PGpo>VoDcYijix&aKK~PYmHQj- zgWSVb)1(67M=%wX5z=qPhY8btg@{kHE4U-H-;{?(O;d_Yek8LR9bx~bKRjmgD30FS^N!5CFO?fw~@nBrpJnG{%3ZOy_$ZTI6PyTRpjuam^J7P=Wo-8 zXH8EPx%`dx8utzFZ}W$J%s@EfM>AEFQPLFn&Tq4CH6ZcrAgHBH1!o4B{i*?|&a6d8 z*;8Sef#${40OPjTx<|!RQJEp;fa*ei46}~%t!#>w8ERfueSuH6gH+cvm69254y-QX zGnn=0x6UbkW`ub~H6Tds_3m%IQ@NSZW+1EaA*Pyglk|IWCS+b+4fsPlh=6IoD>LKG zz+K~i#cV)tvVYfSCYq_$0A8~E?8`+R1;8HjHDc&3JOi{$b@QfQ8;2GoTe zgv&Hn@R3q8R1Kg6W)pgg?Fu_0HE*g0BYS(3`e6StGbt;!fZi*aJr|D%$jqmK}KqCasS|T&mZyW0fHSrjcK5a zkv#rK{dx+jLBeJSfjG?*d^9hhr?46%P|Q|zjO__K8rV}*4e}X#t9wlBi8>n6Q(6t; z3uYU|Df6(7hW1FRL3eKl@wv&9ax}b0S`B(%W;^P1diX~pdKA^5HMF<8onBAw(dZr^ z=kYU`9h7m>AH_$Z9(6TnEbJg_r~RQk8s7t4JwAuoiH@`X&>v0g(N%-0)ZXbH7yn^7 zn$}~e21OgQi}IuFkCCGpJss7crm=%y-t@=BQKZLM4Q_m9H~ORVkLjaXJ-yZ7LbP|g zfAsz_e>A@bh<$v7X{6jCP5U1!?6FmY^<(EywkbItUW*qUmr}$3I~Q!vxdAiDNT8z#!!3 zGW*fH&WF>-W_zAi&-3%_{qDQohx5mLdVQ2Wf_$cxGD({8&+_Z_RW1;mw1de5Z6-J? zpx007D>%g*KquKVVOfE_iz_AXN{5}dJv zF@;(U^h~5=Sf5AEXko#xvOm0?mZ-_ELP{_1V?veg1&VqVZ zE0+q++rdnO_NOu{z8AQOf(uLxy~qAjpOx54RW27?v}5jj;y*1}X}v&I6cjOW$}h4% zN3t?{*C~Spm+W8wV)}C;3+V;sqM(>*M}KktIh~c&yFnQ&D6!k!zj*(g&&uzOR)z>l znFQrN>5;#hx@+xQIuHK z8?Ou%NSMQvUuBP2Voh&?k}N2|hQ<`%5Vv?affG6Udq0pbyx8>BVEcuyVbi z%KnY}f%q?rc(NB!MhFzl5z23}zedDUy_w2LLA4zWZA^bnh-Z4Ulp6#}<_+{W=U>y} z+1?ywl%U3b!~L80uX(XgU%oP0pkj_vrbv(dv-AA=3Y1?6YVBacLwg*Y9ne>(j1kl^ zzeT6mkHfM9`-+qdLB0K3_muc?RCY*TsS*;XnKvoF%O10`L;EDkuLKQtFb*<3PRS1M zlPWg}t}?$ve|Lh1GonwSWCSaR&EtE*}?3H z_O~)SzE7h}5NMg-qb~N}`s~C$opOf&wSVt+iT}1_r}Y_>i2@z-HpMObdn7xfuS1z6 zXtskvlj-k?Y^2YqOcv;wKcH^s-_zMyeZ9&QL5uwdx7+*oe0F}HRhcF*Fvln!(i8vV zg?%>VUO}rJOr>Z~f{&N>5z2Hy8`Ft;*iXWaOZ%=X4+z@rPPa$=Bv$AH>1oF^fD(X4`u(19G~iYtjrel*}-tk^v}fcnZ8+Nj=;jai#~MzGktuv@2N6Z z&~Lx%e(3#Y{|pF>noY?Gw*=M{32^2w z=$}q-yhK=5)D#Qs_Fvq8dS`QUqAkES6%fq(lt(0QaSmizU2{d?u!9*L&8y6bw*dK6 zFwFcFeZ=*atVy`79%>rCh!FA>X%3m_?NKS@jU5#AujU5d1Ox}qc z!~)b*!3gs=^e?A(Iw#Arp++gVVgJqjm)AR=lWzg0s$i5kMR`n`^FL8&iK(d*d}{~O zKiXXIiBbzxqZZs`{*FFo&xM_kS~k^O6?|v^-ThcR7j;5q0UoR17Sl!fTQ? zXa(Qf!KlzQmvX{j0a~lzHq(v%?F8qi(UM%F7yMv%yZ`pi<({xwfaNL}V|pl0NdFd} zAS~%MZ33qq%oS<>Do>1Bfb=RDXa0dcVgIW?F=l~lx&%Mk|8PGM|7$rhX#ozb;0|+| z@{jD_krPvv%o?-cCp#EIn*NM+GKd@^BQ_RPdInsZ{xlq5l=BD6xI~a!2{!`}0_XF8h;9~xb&awZ~ z=O*^+YHkbM_P^b8;{PnUY5l;x6?m9WDF4d-8_CV+@2D9U{9y-^Q`3JFxkx`yaRt-N zf6#xO|4rv+_4n3H2p-!1asTW6Z$3A_ADFp<8RjhIDQVt6udv@%b5HQ69gJCN^TBzg z{Y1@u!6T*@eafB>%aitBuX!N&%kFhQ6$6)8)epQ~!DHqe|gQ!_32$NsPTKks~Qp0$6nW=1f}d`g)oJuA*5`tR2~ z5_s)Tx%0GV%DmD3shY=vIp%-pJo}kGZ>--_^F;8k{Xh4-_?abdvVW##R`8TLPkAPL zHj+2h|F~vO@Shz_dri+K@@D#HYn}?`na|K?&S%qkv;9wN<^|8}&)m|jHPms@&d_SwNY5@Yi#~AjB^jVo7VD(e^A`77BbUsNwYx4uGi&YDe=ZWXQcTM|j z%nz{wUl{R)7Eqs)`^4pkT9>I7AukYMj%@Zx%@4N%eHd8?J+FIi9Nh5{))gv$Sp>bHTaffzOMar2 zs#=b`OuXP(ko{bLewsC06^JZ`7E+&=KX)TP!@5otguFt43AOpTyZMMULbU?%hhEe@ zKmOdq{4DDRRWP!Ic+vCx+;h+J^R2){Mgkx|s&CMOB_|85F{)L_s{|NlM=n@-veXKx zRwGNHmvp{K3)Y^LS~scIAg>WGd3>`MY&@y5#;Zb+WzZtWL9gokl6=>m3LGd>F_3qNS3Q2&z8g=4 z3;;nISq&|vz9jdJI~6)0QGJEHOMrpB**Eo6_yF*wku}h3x|ha5nHDjiP_dBrh}S$X z&H3h?iXH&^G!hCeqb>@1q2v@apjK@~-Y39xKk|i|Q}F}9rbfun>$*irFSMLW9MGwD zAQa+t&!X%X`cI_|0J$2WLd&Ty%U`&0Dr2BSm4tji02ILd!rfEI0C265Fz5~4%i}LR zJe4)jt4cxE5^s23o_pchsr&(;UL)a9Aa!xj!X>8*2W+an$cF^r1|k=(JY70KsM3*- zAd+rz(!#Z;r32Sh2ax{}B+ugPg&R++21ZpG$T}#9`igvE+-c3gEfoj(m;jK0d12~l z!@!sdM%F`b>RuTKb*ORRj*5?bLcHmDWo}{KY3l&cwvh;E1=T<3#gfy+zPLB=%iyMiA-qQIez1VVkY`~*BihN4E6!U ztFn>L2!Jn`U%Y#IW&k+eNEEbEw`Baqho@%;o~m+@jl@dNlDQY3o%R{@sm({Cp%7|7 zkl&I5zd_&HQ^@B8U>zd;Ru%*d`qdU7UqEl`0+RgJ76cA1u04xu8%wM7U6dPn!_I4HCu23OP;BTV8Q&#QBO zc?HpfA+@Cl1g)kn4SK1h02*9fdj*Ll07Vh`QcXepAh5@gub_8zOOsw|DM%co)>a^2 z6YqMKX1~;5kTwWpa%2;vN0p&R7S5rj8^+RO<4emrKqNgXy(x2%7-pN94;j zXGRAzYCDi5=mXvIq?cRHj19uIUC1ut1JCm8m;2954kEQiBpC{$z9E14#+j+X%vv+D zn*f-I`Q^K3@@59JYI~6sXszyz@s}T-nH|ijwIHd)TF)DEFF!luGXxZOBn=9u1_mu& za@KFCpmq@1Ljb5Ga`DQu0YingHe@gKp)N3K@!GS2Lq)YXvXA)C6PUetsC%Swp?G z6NrHL#PjCdE6>j64*|6vL7)igiXi_b=L&~xwf7Jq0YI2Y|CQ%ThltwyhzO$TRwVha zJtrNyUi$z!Owc?lvi&!nQw;&}9?66vsc*^sSwBLtvk%>Jq83`4-fM~*_D z>fRa$&7g7UPVF>ujQG^^)|`LdIqMM6@{ufP12s5kNy#~4=zi@ZL`-aegCm#JoEsei zmOhdVeWnXeTGDcEY{*ml1UXK8<_XST(tmDp2uS-#4irUQDPMBq+|Q1Yn5Fuih<0Y+!eQuoz`x+q}{5$Abt?zLyXyme$7Xq-tx)@;{v_%)1v~2B#K&+^aA*?61ctW$6ZM+bImDWK*HMEubzI<8S zg-}dV_m!}L07R8}S?Yyw3@khlUWK;l-X8~nMFgg(V+k9HZJzh%mgQZD#=!mqp$1Bz zl7n6^xd376x~;+{0^nDXuh(3N$24^bLM^miM^1XZbnP0zq0l|!Q$wED}Q%4zp{o#cythX*j*h1{|Q088L zb|D|L)};vzP$HEYw0z0MLd;gTSJ+Alw#NG4hY)` zwuhR%eB(tGHd>bNvtqA_@M$ygc=y0UN7>giEKLVwdNGx#f8mt=ME8A~ZtD)Ucp8N-h%E{W_7*L?pvuk#E#o9L2yI2B8_+tqV(f zqvhfl=BYa>>>+l0!m{7!zc`73tqj6mD22LK{>F`qQ`qCWY+)af0vDztM5<@)+#Am>`ru$^gU|}4QNx1*mlXNozV)Ys14J4e9vQf@ zC;<1XFAxqwdvxJRfoqEb@x}FLg+s(1Pk46V#-b2BpuSLOgZ5HCln2HYh2qQVF9nnuUi2a_AvPu0#X*k#hA-oP9p#D!z zx>1yYud9~}zab95|1*>B79luT4j~+Y4(k3lPI_3Bg>R@=3U3exJ^z~{JuAw`!LA75 zD3n267ZkMQQXw8wUnl&Q$bi>H2CckQibM5k;Z5j}Ze3E)+DlS=Q~g!pcf=vjy6m8h zmsEIsy+(Ko;!r=92gO~|;0g6w;r9dw{@5IpddYx;#S+5X5Lfr{ILOqDcyhg7_yfW9 zd^{JFcgc!_Jrlw)2&S$Ndb8vbfv4BE37rHCuaA7Q=F%t*)=mh=A)ao1(wi-p#&Ed4 zOZX$f^Q_N)v;R`wBo4Mv2=73A>L>CyZ(N$fGwaR5p9ntuiTTaDmu7IVltMTG33Q)~ zzxnXeES^(u5#A*Po=@i9e0Islo?mYjPC^JZB51{uVn2I9{h;t?0)ZnUSF9`!uou?b zg!dq!E+T2g+TuWaQ9UmFg%ElovR7;@4zZWk6TMh${}YOX`P(zY-#tW?qq6 z9B!A^Ul%@r4(n*+AT^AzE9ystzY&K$w7C^|#nE|HTm8jpcCh_I=z)$>Kb60Aqd3Ff zQ9myHgE$I*YJTf(F=7YHFoe_4G2N%*Z#^u|viH_c2p9auGI(W4shz05FMI@vbsLg`*Oo}_*Xth$|02Ym4cWmPOH}sJ z`YGXKD4Y73JUFgIW4~4J68=qO!=IUhQ%ekXut-Dr1Ujz!Y#hX*M*E%mY2iP_anEOS z!FeTCJJ_osoP~0zQ9&z9N(lS?`bR=9kpo9XuB<5;wNKSQ7S2H@bWuqwTS~_4p86-k ze~A;GsO**fC6o4<`dQ&qD3`iXzVb%Nl>Kr2obW#)7v5-IdADT7K3o4(I1lCNHjb}+ zSTbvWT0bv*M&x-m&aHe_;zNL)93r3Cd}?%1$dXb&!dJaO^qd3Gm60JUO9Kc$wXbMF z>`7g8QpnoUKw`0aq3C(XNl$ck$i~tTB0%jY@{K)3{ahXrR~kwzQ!f&|-~cG5IV80- zoB#_vLpmX`xo-rqLhUbl(Q(@I`CLd|X*2=$d5HXC3#eZNy<%(}-|&plEUIS?ZVaw{MhY z5bM-IqE{RM@-)AFw-g~F)GI{(vFCJOj=%k|G>h1v4i+tOob!A+_x7{Wd?H#MA_|Bt zq{alTT5`FNh*7T+z3Ko&XymGumrDt-8bq`-_Pj18Y1P`xQeu;Ojp#MUc~4CCs*RUb zM7%mwv@G@ll`dZucUeOusL7(&9RMFSuS&gaAQIJ7(el`fI{G*Wwv9xxI!yG22iQp> zS`~X)2PLgudnM3Oq-KcTaa{I5*{e5R32~IFA<^pCE7Vx|>bNVR4vG3J(Yp@7yqZ_1 zUI}-AjU}Qrv1Pj0aS+`{I23A@=sic7Cw6Xi-j!$vSYRRwjg?Tp3VOHX3gl3$w~F3( z06;eK-I^=$4zSNeM2;=jeU=gsJ^t>)D_M?Sb&6=Mqtf&B+`G@Ny6u`=DJq%~{Hq>k(A1ET*qWS&jgYc`gt z9HZ(C(YjbUb+dd;T$#pkOU)5|>;Rmuc};4W!7-+WMeAd$beqS)Y{KZcqvne~aa4IW z&#lQTvpOc#h$tdfL1hKKS5ihe?yE&2nnMAzBHybi8+Cv^Dx%2PY8@--y_T{uhev%> z^r@rT!^(cIziiS0)~bj$#44$A^7n3(O*tN`vqhgd00C@%?{3+Q18i9lMa9O>5 zST^f;s?HT{bkum_=H7c&<}>WmkS~gkRZ-)ELYGMVhJ71Oi9UA#TsSgxr6ge3uc1Kn zMQp7uJ}GpqByf0f!&%Xnj#^KAcIZY)$Z$YIp(rM{j=Du28Yc-IUe<6yM0WtT*c_TF z2_Ft@C=xMZ>vdbk!SE$wctu08i0P>JY?%wqlSB`PG?a>W*GS@r!TuJ}SFsJctx4~ei z4H{8Gtd_c6PL3bMcW;Khc=T_%MHU|GmL0QEUMc+4yH@S;p7IrXr}}9Y@Z|N zm0O3wq8L$Ptd6=Ph*DBc45v4=iP#PRPe)Q}%JW8tGa5QXNwLkk9Z8gy^08sKp-Z&O z(d^lgP3bS697Y<9qU2aTb*G$iqkL*Ov%xId?Et8?nR2&$W;m;%SCkUlqT4x6d00L> zoYPqU|)@h6WggvO8TIsBJrB8;kJnD==3CIf6!l%b`7kr5y7!t)LrrqZd7Dk>u4Ak z@f-jTH-B)q0=WjZ*@*bD-MU@lA3Utcy4Kt9|6=Yvs);j!`#5!Nm+iWR?W&7ex3I12 zvfYfqJH|`!PG;2L9U~f4R8&-CE}^MyXlpHPtx#(?=Gl2vM z5Qb1_Yb|Xvh3&dsHtUx6dH1~My_~cEzX#4CgvmMc@bCP-pU;X;`kJDf9-=J+}9k4U>_ZjSDfz0y-J1oa!`OyPLVW}36=roNf)j!qxB>uV&j^(*Dohi)^1s?pY z`eSyV2%mta`)698!r$G<2ePonz`b6ZTge#rIU=hXM)2k5y0Q@($JRffOggEM?? z?vSa_5A`1Wy!uo2fH-&5RP0}7eFiV{eHuLwn>%4D@k7i9FITt8^JMc9DoXuptk2@_ z`ru|?IWM`Q+z)LZ{DRuY=4t0CDop+j*4g-bK3_D?Fi%xs@n>06@r!C+evqE0sbKuu ztk2```{2Nzo2RSr__M8Y{E}K=4~p;+uh*Yror7QY3DJYGdB%!Xf37tRH>v&dL$dj0 z6~;Dyo^>8x;e$1Ra=y7D=!Y;6POID5L)!W7im<=Hx(L7GYmXi>%x|d}_CqHKuT*!) z57YAl6(fGas>HAQU^>9fAF3GhLpBJnQU}<>;{4HyasO%S3f$}qL=VU2PgKOULq!O; zsDtu+*@6T*zWt1K6@JYJI|Jo{WIC}OqC$AJx|7YO@(?dPoP@aw+LXue^AicV=S zw`Srs>Mr>adVz*cZNF$m@me2@61W9AI;|a&L%3Dl%^nfqL+teSO6w-v=If3gi7hbF z8SPLY!Wnf)epI%wjLvMYv2MZbK3Fg)7nVquPaTI=h{+<{v-L>IPutvR?$-Nzmi7mm`!?YFIaaksB8 zdMvhZf-Y%qwdUd;by!{?Ta<96wEeDiKYqgp+X&^N&GX_($r`*+Ol)qB5ytitQ}^vG4O}p&?yWnbMJHE5+}szmT7z(>0Z;9aC*@;h*?m z{K2K`D$_cWZ0GQM>Mz+-BAg+lcg(b%$3OLb89fzCH&$kJq}a;wHuZgZk!(p>WoE}L z+XdX`gQbXaiMcYX0|HDqul|ZH(k^jVW_QSKmvF)NRkX;kq@^;q1A0ujU;RLi(@O%C z`5g;vG~VumSqZmfsIssF@=SP#`fC;!myA{xcPz72;Q`;*Q9QO}qOzm|YE5`hJt8M$ z%7m+>9cygWc&86GC`x7W)$$IAH{o6CZ&*UBR9rQ6Y_M7JZr?Xi!k|=LwRB|J7(Ar@ zR!-7N%~hsjo6Uju_+YTYDRoyp9gudyd)42uqzHEw-i{ob3-9xN7bRm#<5gp82b7-h zusSNIWJ}Agwsqv$>hOLa%v+R8%~yjR5Prf%wZu}|rS7ZYjsjaFKH!t0lwoPh)!`24 zK;aSf_i{bGG;npKgRpVc6tZ+GUEWq`*1bJ^V}G zU!%o_WvZ%_K)J0Azpws5{sO&BQxfr$_=vRWmTDh8e13sjSnU{%H`&&tN;Y3@NdA{BOR0Mo$}7sLYl?&^CfURLA5m(JM4&CJ?fHi~rpR%OGxr z&g=<5$_oEc{V(<<5l&^ifr#yU{2#u5MPG`oFq&HfP`1Ly)sN&a%T#6Nw!nSc5BNWQ zFe6f`%;sQV#P$%6ssGKstW~+q;Xu^(BmOVnzoRc3R4wM=z?f|uf296NZlqNK^GM*K zEr$Qw2b&{KHDn$OjN2aJKdC2Jqo^7+j|U#ve!?ew6H#MKHDQhmLKTaML*tMVSw?~- zJ~)|~L_EgBKuMXAY)K44B#W4YJ_bs(8462Ma0>GT@iYE0slvZMqPnRsF{Ith7& z&d^v=gHxF&iN|?ZGjSO@OIi>TTEr9R&%i4p+ybQsXEIL_Kj(iYy%NhXS~7xA)FR^1 z$;hj+m1UO9;4J28;ukzjpOh=jmaJeZGnIG}eH^^1UFo)D2j$Fk;+Op6(yNA*EtcG1 z8k0m!L7zZgqgMtj`N0K@jQAA~J1K7Ekfkt~&dem9LVpfk6IYH}ii69TXNUy;=hADj zl@pedUbePh5-ZEMYQ;6f_Yjm90srj_*9ftRnuv!xBunCb>GX z6RKduv*@ousdkN`I;ry==`3e5iCO5=$eZ*UO?7JLMFu7Q$ir-m zTcfK^>x5<)F&q62cvFOHuJq1IW)txz{x{N_u{Fl(j7~_15vk}@OFCPbTp|rkLe9z7 zC0sAz2+SlTR!_ zWym}9I?Z*aGsGMv7VgggHSh=6@%>6I*Az-r70L z6cXv^Oys<5ecAQ4&ihOev4n?RoN~SSda!eZAqXYjT$EIv+9yVmS{ZI&S^Y^|8)z<}|SaO$KG+`qAs-osXE82^F6# zmBrRiT#xIDvzHJVXbMs;%S@<=@0x6Xl~~Ec;!c^FT$9)pZ$CqFU zv&3rt4^p`yQ&p4Fm1r*|)}YTK@6wr?n$)hT_P2<&Jk0aBOkGV{SCai4u?~F>yeq;z zUV7I|`*~tL|D5!0EYnz%(UoE^Co<7l$OW0YtR}N-mi+>u=3&dHRGVwEx>D^I2^9S! zxS&{Wz@|C97yOg&Lk(gnFSA`4ALF3Qk^+S0By_G)4?4=X_>np|7n z1@$#z3;JhpQHv^SOw{( zwMI`DwAqLq=wHD5B3vALyK?L>HQdLtZnUrR2z|v%8^SltgN=JE6-j>?BZd7 zsKm^*!LEFJ1EEC`a7l}~Yr|ay_C{hik4TpcSWE42SE1cYL z?BikDsN9fjP3(s18?hgq3ryM#3TsmLIr}~006$kU88)b_Dc$AvHX;w5hg8rTG}hGa zi*}wk$iohj+n}?ib(`#d;t)C?RETg=nciJ#?;sBI^QDT|2BS5j+hPwA`RD?KmTfGv zW_H)uyNDw^j3$*E&DN}L#vUS$q6-17-RQPvceD0h;uybBq755cthwC~kRuAvMaUI; zW5Al<-C!4q<2)=Zxs5~C!fxov5hu{a;EK3$)LPtq+x{7Gl3y%ciEW&)mUOq;hlxTo z9jTOUO0bo7-?e{1oZ?}Qsoa!oEAMWz-zSRDC7@EfNntZ}`|S@1oL?eU8aAnHmhPZ^ zgdk8Qa+Thsu`%5t`?mzi!*-L~q_cUt!}cgap-aJ45pG7k-4Xlugq~k2U5#xr+FHAZ z?PEkSx(un3Y07ME-S_Q35HIjB^i*ohwqQ4u>Ieh698_sFZd?y4IMk;WmE;sqGr@K-u=k_6Jg|4k~yZC zu*HSq9C2g`nt@nkSqV&hXtHAx`3escQe{>$lNgG3OeSANR|1PROTi?CrZ}D;U*lIw z7DJYbNeLx7;>k1UD&!iSrD0M-Qyou|uk)}s<+5~4S_m3=<0SdTjFqQxeK>q>$xkCQ>8Yl3*_lt#Ld{zRSbxRkN)3FM_%VQON?Qg%8}Aj?#Lu-&@G6a-llP+_FQzJWGxT7UT&Mt zk=6szKhlbB1$Ghcn$vqK9h*oSzg4owwiz86JWcJiJwvcunM!?GL zW=B>JBmqeWx*a&Q+ue@r9@epgbn@FJhhck*Be%!n$R=5I2jZl+2ORl54Gt{{cvuW` z+lL&5Jzhr+=|Xn`r?`F8QQULev6po7J0)jq`-G#Ur`3^5deCfymF-AymiF9r>?d#V zFfUf_NOqR@v^ny~I&>FcwL26}Q;*+qh^*&#NvvUq%4z8dI`YW|REq$5hsMeDgd9i7 zn>=ihxg9#Erzh+vARE!$0Elq7?d^#;PLNIfZVAM87@e&>!;V7Ii{>CM+0HU&ThDz* z5!uYca9O$2>2(t}-^Wcb4-4>EmH+t=wg1vwBmV7fBvH2ginpHotd)lP24FSY~s(hSs*mlOU`u*4 zoE9>O<|7Ryf@QXLw2D@K!aAR0H)pzPAl2XACVdiS`{!_ zdb6Ai8A6XDH)*W~FumKH4zh=b(Kn~n0Z(tXlO=o6W8kI;xA)%O9H)!yanl{)W|U!bReSDT}7CH0+i-Xp){ zPf1=wj>?tNSMF>h@1sRXGo7PxrS@HP^5j=M4Ar?Dohz-+^N=5Fh|@BD%MCl5<}rOxaQ_Cacr zjG-@qJ6fIF9qx-de zH|~5y{)8Gqi>MoQkM})t{zOjjMyVyHn{da4p-V}{VI@ebEH}XuAD+xkq8<}q<{|0fne0>4&xO~d4`aDTPevG;mQ*};2Kh*~ugsGfp2a>* z{X&3xfO4PNlNE-ookc#T_XRxp z;RURW`jr5w0dC)rr!bt(&ZM5g-T)tq`$j#*;brVIRD$q^^l@z8gr_8&!KP4&SSfN> zwm;!UX?P9$EcI&vx&+Go$v4Wwnd~g;Y3xmKSG!+v!)OX`U}sam5#E&U8uqJhSi)Is zDm4{*3;BfJuerg5x3SMt(*&p(aQk&PJmG9sPEE(&2A_y9fboWN*g4d1g}0?oV*8CZ zTEn?)8kK~dL+;5Al-+0x=dtss83H5^ln2Z=g5i920VTuU0r#{A+&9AE0(KGgJK-Ja zp5Z{tjp1-1n@-Kd&Lf}F2Ld-n!UU_NelI{DfjcmCV=P?EE~B2o%D|`Mfzcb|;nVC2 zDp@F#K8+oixDnT1!e&q@SUJ)r%S)(>??1z?qW&O2OhK8KT$k8i%C4cF#oh&N+B`*F zQvW%29rc{>uGD78Q`M#Pm$RAFEbIc}qw_R%sr?sOl=`CpoWQ+Y!;P@T|{`Hk*=Tmk@zI7^utdZ(y|)B0$rDJ2+HV*bl`| zY7TZ82;#xfy5j!Z>|RPCT$Y5`!HK$(eh7e4X_yJ|%MK;fm-gRf_fvBP2tX(gCD)ht zLl2aihgAT-_K>39)bD2xQS*fg$!|EMs<-q9*?ejNMkDR?Ax%BgA7YPE3k9e}aEEmD zp8hafKrON9)J?AF(e}D#0uTVuvT{pf+F8kR(n4XQ|b~H7RJwS2d)FiJ+8PgHM|T@X~-2JX-aLttVlO~B+!sA zE&w#OQGj*^cVwucP=vxMwF$FmDK$e3$s=MhCp^i%Jz?`5*gcX`s%mFTHr{I)&Vn>ZPTSbVaQrQ@b^vaHv z-E0%{KpnM9fQED0XS-p0KpIL*wD=}u^8N@_F`_(CmtKUIWC?CcPO3Umil7HCT_+Jlz>($7xN%t zSwTW${JdupTcBu68aM~;Q3r$@QrJ+SYD^g@2W?ay zR)_S{1)9dxfs25r4hoPG;Rs3 zm@2>;kpcR6pfP`-0f^Lb0U{;b@u9}T0WXM9C$J_kARZrWEFQQGKBGoh)l=8@LaCpk5H5m7+XpZVC>JfQOU;`v82VJ?U-=4@AL_)QiFg z(r1Q~EltA%kdCEJV|S1t`edMKWZ)r)Q7;J)R^d(#HH{5GNtSvUYXL*z$vPpSDd~CYej}-g$dsH$Yj?f{VM`gSd@jy-o!|}YqI`T>_afDEmU}u zB2!#X=wA~)l!gt3DsM_8(G{;hgMEa2P8VvtsgbF!C-tujkZj=!b>6f{l52|oEcP+@ zT!f)ydSs^SDg7J5$I|DqLZde$lHy9#mtuF3FJ!06yqS?%uBY{H3ea~^o-%v0BB`#a z`nRx8z!%z6Zf|x(?wYQDTlhr!!f>j^n;S`UCF##$_mD5?Qvq*&WPwYje@B2A40meC zTNr_Yt^PdrDfm)6HR>&nEOR}hFB3kMzKork@Rmd%Xsa*B+K~IQqJ(BmX=IJ-S^c{L zlw*`d$<5^v2;Ax~U_NkPTcl_)#W6(tSgbs@W3Ba;54oVm$H{U8HGdBHLWg z>)#h3Gs6|>nmv(hmt21d6Tnv@OhLVo9M>HEWkHa>iWM1~TO+xyG`$J)BM)SFS#w(? z&oxh9AwZKxiJO~)k$l$zJ&m=42U^_S9F7#Y7U{1D?a~7S-qJiAfmW`*66-*|rtv`Y zNQ7`H^;ZQ5*l_qz^H>DZx%w(B0KOLS(dO~UY1az9SqMm9$MA{fxIrlC>Md9h8IchQ zT>Rh}*DC!r0ctl&BAH7Zgs`r@8tVijT0+4k4W4tY(_a@lr4a+6;!+07U77kCtPA;u zCNx~?;6)dzuN5GT!x1_zZ4k1%dMnlqz7b*Tnm$viKaBSn8A?isQ#t^IUkPF-SP~ET?P6^Yye0ijCs9-5!VTQlQ1Ajlj7pyW|MJL~6rZTt`0ILKT*-7UKJ~x69XCThZ8H3t6gMsIKR@E)rcIhrxiBv2 z3H(6*!Q=a;O^$=VJXSny(zGWwEQ9y8aB|XrZs31@Wbz*;CH?1?{&UR>|LvNE|LcnX z_P77*^V5R~GiJ_^&*r{={C^)h{juWz=cxR~>E)x3Pp|v_@!9m}lV``na}NAm)pYNb zAK^LkW_X)dZGEg{`klv8|I6F|<(&WD{=w6Se|Qd_cXs^rvt7?kjf+ctws2q3De@qJ x94IVIp$-<|M^2tdnLB6UoVm!>O)ImuW@@(0kk5>ZgWqvgkHy7ht=uyAe*huc{YC%) diff --git a/utils/local-engine/tests/data/decimal.parquet b/utils/local-engine/tests/data/decimal.parquet deleted file mode 100644 index e1981938866ee9ab0869cd32df3fedf76eae19be..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 190487 zcmV(#K;*wrK~gal1Qmed2^Es!2}~8BoBETN-IeVH0R6J{F7K<2GbktTYEZz)Bf}X+Vbr-F{`^1h#r1K z#4RHawDDb7i-AHGz9nxKS2Q4>2^ zL~6o(Soa~cz>yh7#TE3^f?R$jw3FwM9DZktY3+;`Z023ho{#QFcgz8O35{lc{~K8C z6b&O|{mvZdKOKkIp_~jRThJd4aa^;Djo7XkQ2i{IMK;Nr621FgnNLLbI&r-^xDXO( zMuRPlS6W&x9=zI7VnlWFQv}=^p3G%AX{Bi|{gmAn7NAwVV6vLozyFG6UWD4lGV=xh zJUs76&lBvR z)|>Y)$&ebBRG~SUZ*uS-EQcMIz8)&L3zmMOkn_v}V4$kg2~46Oyk%C4l~LFHtx^Zo5&~M(KHp4ZHTqm8Sg?5Z-t05-cjEsM zQ63<_kD-J1nH~fEA{ul6$h9$Uw4NGpxK%c6-C5y0} zWk?L?lnmGKhaT+{(IM#TNtwm*J+-h{05KrXZbP0ftOp!ZVgzWS4GnI$I7I*EF_GxF zqW^T`N=O(1zdg4Kp(;WixUbAZN*0OgGLaS70_I$5#y>bEaa<5Aj_^j)wo_QsLu2vO zOJqIgO81?k1I=P0Piah%PI-whvY5+(imP2lTScIF7`UKAFaa*SMj&D}R*bMLxYTfa z-=6xU`#!F6^P14}xR*>8iaRs5SyQpM}`8)IjApH87qk=~Jsr zXm6MCYZQkuOpoy$ccWI6R7$lPxFXRSR32lp*ds4Bw+OYQ4dNf9BVzVU40$w@1Hayv z8lqB3u}eb7h|@c9>7;gpIQygfe^VLjmue4+so`d>TizH6%I?+^jE8nZw}1-2m-Uiz`sw8nQH@N|aJXwH{WY)MoHW zEU*9A2JU@n_3+Ox?%qFG$7vd`AaLJLUXHkSL->4m zww4br4cRe9g(y0ACw>M|%4@F-w2{nAh0I)?NB>sS0-^HImoauhPWAMX*!S=H6lXd>#tIp5#KX~CpWSToQF>U3_j;D$*~=vBL8p)WR0u?${vbmu z4o}pUFR|E`M z@<3XJ%w6;kU~8le*yj07Eqj zfuc?D3NYVCt%t%8KO#DM0uyzJ=Digj@w}`{D#yKP=l(pNhtUo=N32wQAD`GEnn~ zsjr(!UUmf07dyBKfE%9rFv0T)xfoBiaZ{0y7bdSBEgrg`()>!w4g1gvuG(a-*5~mI zlFei~M(Fxj(odRmQ*74FH<&;Q0pSWNs+wx2L57&tZwd83dK}sksB1)0!xXTf^Opf4 z$kVvEk_5D21_~2d_c|_4e@G0{wvxU2D=%SBMvdO*1TRrPtWltLV&$nfa1KbbiX8ad z8CvDIW7a|+F4NU_nrU{RA(jFQE?o|{V)c;@P{xPDOZxTo zL$-Zm7aLTiUN>>f!Ad_L?x{x7=5zKld?_ob?~yZazLp_7wuLWV#NKyJCD%f!VW*rk ze)DcDie>OZc-IR!{^y&ubS}|aOc=tt?+6O*=(w{Rq#Y3+W-%R89!zzH(M4(JQ=>)}OmNmTI)?NZ#Syzpiiq_X=8aXT?YV~~31(9*ZEO+=MF@E@iPHl$Pnd%?fYgi_0l!qvXL zZVA-mRI`M%FyFf;tsIBtGCW7M6=yM zR{Q@cReNOk;Q|Z`MN7^Kz?wVPl_3xsYHQ~y5D!v~C<9zwZP!I5SGhQaK+2}Y_5|@h zQJJJCp=EEYTJb%NtUBlr+1UA9-;aJOI8o+>Oa%KmZL_Yhw7_Of{+TEJ131gVEFiTB>6>9WAb|rKa zmljy&cy%?6?_i%6ZuqV!SUq{+GprVB@TrY2LWwtm5r9Vsh*xt&`r{Cg{A4oqgC64i z3f<`n(4f2C{r@y{{5K4gzWV;+76-w>a>?vB=b@6q=5k+V0fyJb*}iKa!|MWBESyQj z{>kd;iMYxgRq~jgjg&2|9j{Z0OOrimsZbo*0aZ=x2jiVi4;<8;YAgvSpY*{R7<5#@ z1=&)I3Uuu{12l+PAGjtZtf@54QS^+B+4~lt<5--P2)F!_2wiOUo{V|J-Rh@lCtU0y zhO%lbltrt&Z7cdvqXeM_L)|41zIzAnBWCdHJvkVbqG4U63U3zSwqpJ2mj<=I{r_Lf z{KDuD)aeGiNk=YN@eQ}7uwcab`Cg`TMB)@ z^Od|~kfV0Cve4hMy~wu!VoL+72fmR_QrJL51EAk30}$q5Xmy@=vnsF%0_QKvUH?nl zwp`?DV^^9fB}7};)g|F_7W^P*jXD}q?87p$XqpMZ@{IKBJq6t&0L5~zOz~{dQi)dw zh7GGc(Q=UYASeaK^ZC2`n%KAeeyuUH-3TI6Gv4prY_FM0G;?65QWgX=`D?t|>v_2t z5QM825X-r)B+NF*D1&W(&h$UX&XH(fmkoHwigDO$TuPDAX#%{$9rv7JXL0LUeKz;tK| z_xTdDLRy%bVw*>KFwbeHD}8`XqAEMvS8eVG$Kg|<_6o%Se z`XF=(-r77mB{Ct^An1P2nme#ZR>FU%ex?K^lAW{(oq;52=V&n03Cx{ObMuZ_6K=TU zj({Y_)2fwnMq=`zdm4$h;y)jGf@6CqXyhm@#UKcBAnp$lIuVBdM zGk`Sk@cB2qL^i-$ZKn#-pPPMIAh{nD|EML*X$}8yx8w&w+~8sHQ8bP-w3mdVd2+FW ztPyn##zsC-7wBgK*<-cA^zOjmdXE5vfm=-XbOj{cvk9dg8ghom2FM^1LboMMH7w31 z#3F>8G%aiL%il93K!X{Wo0(9^_TtSaBYS(20ba1cL{vo60h2Qo*PWp`X#tff?Sml`z|hQ(t}gigB+3BqwtF%U{llZF zZ@JeJLY>@|X!&@Y9_*|V!%3q>u#L$8INoTBwjZ&C)gz~kAvgNr0xgc7xpP(Q+#E5Y zkw|@644X23zC7Q{(2~CE*gbTb?H?_m3FQmgp6uC~;+P^u#%XRj~{gCwBLJ?Q=%K?YqMRi)PB@Su^-oPv7-T>#e(O zXDbpea<6d~P#OnMR1f&C2xUQ`@yjTnC8-P|PiZ1?zN_Z(TE>U(pEaLi`oFYdnre68V*QT61bh;2 z1=+o5v0B(y7?5d`l2g9);*_uo&ZR##CL6JzjIgwZ%(*yHG9!jnk3gNo_aJplmCsL> zMq2cZQMGM#c$I{XxDeT>ge4RwoN8T+mvaF;pE2&XLEesw=D*|~0Yy{nmL&`)uPMtz zGLF!@&s=SXYiUe0<8LtwdDb!p*PQp9q5=d4#i^q&E!0ldPY5X6il4qK-HSOa2Hir^ zlSfNR%RI~%$jCS+6gV6h8nM_3j9sLI@cMCVPJcxQlQPRIu%duz!hHtZFS5NA-X#P5 z&G_*ED99ZFc3<&n zwTA{AgFssdU8=CNVvsuo0X}qTX-A zOi=7BvE6+ZsQSbFFqt6NPlI05iBjW0(NXSrd8?>d9rxeo6Jy4~7!Tp0<^b;*#fb{FdN06LYS zGkTxEMSP^(*LOjvtI5_tb~oz{=#pGaTY zG1uGi0rHJJX`T#E@`Qik>f4*8zprr5Km=In=1Q{YO{m&_fQ$X#mMiuok8cAD$5lS_ zv=U!s*Wm@|+N)@k&+LQ`1&iZ3Fbl=cZ2g*ZmzI$wxc58j3Ny*;p0W>RP2X}UwhV3c z9{(ALUfqkY9<>i|?GT6xfXcm!x{bg<OXYuzLW%2CK3Ys9h%t$mf-3lF+I^Z zFXvUni9j%gk4Mm#pJ_a^2tL}e>~OA{6ERz#$4AX8e5#jefu@2Py4tGMWdcPJvhf;= znB9#EnD`p5^6R0Mdt%ut?V9>6FRzF~49D6Rz(4&QgM4%P;=6PQQ-RLEM8le4_>Mod6KdRc_Q#CM5XNm zpXt@zi3yQcXU#^vti#4H&0p)iP4I5A;ouyj?!nGj)Gb9^^jEjyVQHXW;(J(@;~Ze; z&~NZmk3Lu8_5i5=Bj^d^hMI;`fZ~%eqGQ=$U*1}8JN6@&06FxM)JA6-;_Zi2=2L zdYse!R@F1q8o8%CaWqXt+{!82JopHVWA0>w(*DB2Xy%6|Bx{yp-EKhmes zK+AyAdDchgmM|pV`FlkhJ&Ll~cjDy^NOCf<%@pZjDm$rK5qoOGaHkozfI{5}VWiLaKZxS3XQF%zVq8%>H)Vax@2t!1< zYmRFU!%^kXwf<-9bDIQEAa<5UrmFz9Aj`nv4SHm)OD=-tM zZ!F6v19cB~bu#IpPw0eSk)Ul5n(qD2=p`mJ1|n7Or6{~D0Ovw(zwBtd@<~_Y&5wD<4P=H)AN#9(>^_q$Uxum24)P*E1@q z_HSFSG-Wje$dDIgl|eEKig;S;;?%EdG`r@Rv{GqRuS$0lE!1&; zRC`sO=3H#+PcMmLD$gE&&&d-#d`~E&!Fbl?6S^Y*=;i$BFKGi(%m6n&vKO?X;#(})t zog`K}iDbBQ_Z~+|JVCfqpp91o8{#hGUmBitX{Nv5k@w~`HO5AIM_my=02K8U)Cr7U ze)cR`1n~QNrShZTs@l2%>Z~_gW$%$P5CAv>*fP6E6$@n+&E@D8lJG`iiS38+?DnMV?zrXBJBwUQZJ+cP;gZi&{>0Vk9)J5QEv~O=qv>p2!iww#kwv zU|q%@b$x}a(5I=|khA(C2!PjV{eMvwfsM6O`PuU{l6R>%66*lAC*!XIuS%&k7FWZc z;{hKBYORG}OpM%@rmZwfr-T6y?K`HI!TbXGvxbLsPnuqg*w7V|UOj0oGq;`>MuF%W z0{*%uiBux(M5)n)-~9}Rc4awb0U-<~u2XaVX_v`-T!Z2TZG3IQzN+8ykgkzI_lAT_ z5{ES|@Q$_XDSj(*7zsK{ugC|PYV-A~3+H+Z-h6K+O-_;Nly3bfS1y3KH+sqnVyx?* zP*1n}Kw{Y%Ik8U^Y^6>hk$2O-De?VA)w_+zRLcw^0?80y$#^N)+JLL^G$6O8TJ$#w z6S>ZevJkIt{Oz#j?vS@Poc4ob22-=4qhE}OL5&A)qYFKGLGAG046yG;7Kp7F*%RAj zj43yzN@K7KE2rbrDK-rDk5y0a_D_0ZpfIJLQImlD1wC3&B_gS%_?jz82q?UqdaGqJ zNVcdrfChwJA+@6En!(k%hlxlEGJe!(A#)`-F~BP}z*v;vV{6E)D({ze^v*7Uhf8_2 zRnY?rI|4m2<#M}|xoBX^j%?9Dq1@oAndzI*Hhsa*?FM+2FQfxEDV5@{ci4`T%BS8WS?$P{aJWJ=s8~@1Gm(j^x+)_Ol+cRY#^O2u$iP&4<`R z+L?iXg`%&!2Agx+bcc?T>Jd$JlJoegc~1(5^w_)aeSwNnTAIYQ1T~-CK6hA%=`Q)r zvm>i&#;qxuNru9x)*(Roi!rQjRNT4KEpfg?B?A+$ym=dibP@049y=v^ubd#()+GV` zy>Qyt((pqXP#xCCM4fK32;4s}bn7QEn3?8w2Hs<)>10DN+_YrmfG0ZI}|WoGNFm z5#$vzxh!zzHYPu9!c9|TVWRv^sI`^ zkw@3ISb#$E*~ZUj>?aHIt~Z-?eOol`9Hlg)eVB;{DAS3t;djfPnx|dcA-A|Z80=MH zsIc&i%4LoR z2pGAXJ-@X$GIjG}LjyqB)hN@Nq&zKK9YZkTB*U<9pp{1CYC{vcc_7Yu zA1_z<=TG9l{Sv83;ARl+0C4(ud5dY)3@ZLNg{3&y-cE+4z@Vpa!U*y}rFLm>l8c(E zIrBJI3c?+Yi|SgbX?Zr@6SJFM$no#dqxSn`fq$0+eOeTsQyXTSB@%qnH%<`Il2j znHQu!xXWy-dG8X9-e5Hnqvyzjv0xvz}ginCxAd$ z;(i`d=gg#`7pi(-Iapype0LnfTbyG-^2iTNnvW`|0=GWTld z<9<)5YW>o&eNhkRQQF6=)QzD=d^w;+)xEe%tX8bk^JZTOTd=kQA;z|bSfrOEIa-y; zIU6JHt+QXk()w&~Enh=vosu!k0boTY1^Hv(A;kV^xg!n_$@+etLdNdXw^SxSS_VLG znYvYtX`DJeZx2yY8r3Ql|JnWRPRVR$x&eSX&0lo9LgVoHyomN9PFfH8vN4p8O;rh_ zCKyzu@xQvxGAO9mz6tMBu*C zKovo}{Vq2pE%_RIWq?+R(ea14^}vmCzMew>J_--*-{3f+tWcWUhUd#{BaRRI3|djx zMZv42S<27WcBXOaZ~a3o?%g83h%em&FS?NTf8^bNUPY*ap)Zg5BAiwT#g?i2&QJVF z^!hRW3lodV+vSF_$3htqcy9%lc8Za&n#`lb;N%GIip1&Wf?xR!;3C%3{Mk&*0Q#lO z@m^>1{(`V*OdqG;Q>fah>pnb-?884H|}RoOUuoLp671l6f3lX&sFkR3JVVsXcf3yJ^aaqqR#_Lt>)PU&`Yg zRjP8=nOouv&z4u)uKyH-oJ$$i5nzdQRX41d(s2kGfTWx8HTkPgVrAO-t2W0^@3>xf zy_x&xRxRQQ6mOcZHXNso+R1qsYD|n9uJfQPRm0YX{sRL+_^=y{$xAf#^sL3w#xL%^ z2hw%%mF;r1vb<5HHWcAoC$ii%G%LNDH<4$Z905ylP`|mCJ^njxg^@=?bXm2Dg+xQc z9J-fvZ3wn{6-MiYkO1EGF$x*%^7`?}1z7`T;f-k?>s>!8YY9;ugPB$v--Yah>1-Tp z+CAA*iRszSntRYFsNrdKSceABLu1WYV6AFTAo|$m*bNJMWWIhtxMv;m%71?~g9%JY z5OznyeOQV?L3(|ZqP&hgG@ID1MC_s18X2Z=q;xjJOWIHtHRfudKf{$R_#reB0#P764H<$&%)0&q1ql)hEvrt zffxSn?%hbz z0x4;RL{GLS=|9yOP=M>7`1>$k&O^GW^5=TvQoa80{-hX`lMBK8`FYg{dMMqAuc<&i zt{wBG3-*rH#fORldF>`z3`?2K_QGhh&xUoi9u>`nn~zWwQs+XS5f|Uc`ZRbk$i(k> zoTQxa&H6K>AS#P6SUfJ@e~f*Si>t6$%i-b66bVWN0Us5QxaJq;s*JD}$I$ncfeoL_ zcCxwMg~;N8HNf~>Kg|;6LuHA>;bb>ML|>}$T{K}*Z|3Xq9%qj3*78Du!5~`BTJhisOI*|geVu-iI9z|%Sc=fYtBu| zeZ@4VS2`T_t`)-+3ou3ukf`w?k#31h(UW1)QUIdLYMYbFNAoolvek3@#PG^k($Ej1 zMi0KE%askf|Bv&Jea#6!Ei<@#=X~xll(N+e0$cgZH>@?kAW_PTgD)y zKKAy~7~Xbh=JM+6-qJwZ{6y?X4({Uz=_=1)T8&gT;p;2+@>2$JRb!*7e{V!=UYDu7 zATa>7CSQEJ{-`2~44>16w8iA3r%Pa7UY3Wh7(&xOCjP9cx#oFI3myegT zJQ*kDw<)S;MQ*F7>vI_C+#b~qPARSSI-1UQ6f@Y4LD&pfybx|{Jh7egI*e?H{r%fB zB%i^~ny||TR6D)IUEu({?wkTuSX+ENWlGkQwjv0D))0)UA1Sn^K~bG{FVOz#ykG73 z_7}E!)*$dL84HLrN_-14+-+WSavQQoGJe#FUT7=E*ClWg?|xteho}}QsTm<7g9XLM zssU&qH?5-j``Jl{q6K#E=oU%`^J2JYHj>I%Ue898i>S+GrgJd5rdwQ0lQR#$ykM&;8Fd^5YfapuXOE#d5{1JtqLCm2={Gpl#pE?%#*skQvhYCT#Re)^S| zz!o$VIW5bl*j|=EWQ?Hg_HJ6F_ztgMWkbsLQUuJXI(jRXF3XI7JD|Q=DTOpj z_42Vk2h`K1kwA8ea8U2VfAZuaWRjKL4wU1y^s*1BA#r;xJ) zF`WHZ)B3VX#0*{!`S?vPel=^w@Y(Yq zUoDRlOS*cN&b6UYx=}0S7CPj`0_^Cjc7_zc21hC<&vs@DrR!I6 zNKpYZj5EUO{h%q(eX|J@n#7Zv2eof9@AXQ+HU~XeQ37UV&CCuZH%h3NM4h68EB=xT z#~i8e_WK-SaJH(9Y<>MJWFJn&EGt0C>zYKF*B+#|IZZ!TYz+xy;&Q}+h`^n?3zh}wk~jZ%t*@_t8Rpk3agG;O%7+Aqxw??b?GP+ zw80=hla+lwZnD+SDj+BOJ|8%$82^VB#-Tihfe1TL z=FNo|TYEtlZZXz~inV$+Wh(h|t}G;2q(3#%RSfX z@Gbt9>Rx?9)4;lnxDNZZqW$u9Kz56B+zvbp=byL24mj%_UX47xzCCrdq8(aL_79EL zFb)Xof+UNA##**pu8p3m&Zs!SNkm&YpTX4=BcZ$L#)*#&x>^pwry zw+9`g_nvRPKh|jKrVu{En^_%g3q-t;MxU*#PjC0eI#)ePvGefShR1WwAmQ>m=nyK5eE~?Vme;+w^_8)?; z+h-yBU%iy|b!O(N-l#TKm3pP6n831egQQ+So&~dErRrqsPum8MOIu?oau4OHN}Uh$ ze_U_54*e69f(j}n@(|x}#EH4TbkO$6>FYO9{g-Ua?@ z|HE1T^&)OTTyGL-l@)xv`JTgKsrNL)+_&gZdY}c4irlySM&2WbE$=3p^09g54@d{9t&aN8 z(dB-wC(DAs5KWQ>9wcx(u^Vh>t@w@F;1Y#dTXSxq4*(;$gb2DHgut95Na z2m6;mU{RFtIUOf+8SO4IP|8me2%{A5^pgt@9X9fFv5=aI+nNSSbqp}uyJvV;FBeB9 za&EpWD@D>Ylu=RFtd>s_K>>IbFYEczn5BvFhVHUv-}Vi@PVh#d)#Pwj71?fRTCGr& zl#I5e4nniravtsg-11bxp;Q(%a%M{vPbhkNTXp1ufN!($H&_@e## zKA^@8X6RvXOC_36KULf1_oGv)aMe66kVFCDHA(+7VGmA1UCp9cw%NB>{&jYk4m}20 zggJ? zwP@R{+tEq8tetAi++6ymgZ2UKUWWH!>TbYv()C6HPo)!uq5a$x3h!sbr^ECrE>joo z)N+fJ7TO=nuEU&=ACp2{!e$KH6AA~?caeQsf>BTMpmUeUk^KOAL`%ukCTrW@GA&Z zV0$;fR??w0E?i7>bkELqKGB9#kX8LJwcxGV1kf$v`Dm=pFy>EYFi zG#je5`g65`m5+bbe8?>96!6vLSXEgx)h*GDa7@CN8M6awl>%4rv^dS+_4wgPULL@>D6tdO@yN>{roMm#{BGospsQ7KB{L?dYxlbPe zBt$2RW0{rhfcEt>vA{O}=*n__Id=eKOh99}o75vZEBb4;M6-(6K0n8`tT?$Q`RU<^ zQrTp+Y5b7B`?@djjs^y)vUa5)I|R z_yF(I{eI@otTPckj0#jL_H}}EgXD8C6joDGg}lg@!&2nj^zD&}Z?EsSu}v<3>KsHb zg3_mIIPv{%rw!`N30C;%1&tiTcDO7jbxjL((^b1TMGMuwnUB-M)cb~iq2^D*Cpv8` zF=)!NNE5cCQ$>J1-Qy!n_)UXAq2L72A(zcBwfq|jh6aC{q@({X>0T~k&G62~U~sZQv=-lf!TSe-kGY%>p_#Kjp~w6lU&q(4y!?hPg5@U7YMqK0+S@AQXU?M8 z)a|u`Wy|P8fvSNKYHL#09O_27)(Pl2YlvjU2rF+|Xj+9z6=k>z9bR-KMB1+fx?e$# ztN*h73qy1_klDWW!zbqpiJb;Kcur>&AROV-VQ89@C1w7fl)l!3EW7fL*?4`>l0e=4 zvt`#ech8S-3a6pwwQog<^=_uALLl;Ih z^*|rk3|4I3NKF9%UQ7UseQV{c8o^4rWX`}Jo#69Ga)nwa1ifG&TJn$C{RZ&bSl9EH zkF`NA+*CzyjJPE?Fw~#)VtNQP^9h&_2riEAoS&x1a&FPwUsj<+hBK~waESecxc0Oj z;eG5VEP3ECuPH`5Geq1Qx3KoEaN*VoPGC}ULd_ng)$&ufcAC7O-ItL;*r8^}bVnQQ z;HkeV-Hi-u1(W^a>-ke4W`1dLncc=UR&=<`(CF+9lob-9*(+$@u zR}qoxBJEr+X+zOaQ(;Aw0!y?^)>%g!v}?+vL@K{H*3_~VlL$Rp+WeEU5qKUXAboi77T)pbOspG=e!3t4E2O`8rrN#|dd~5cU2C{$MNq1a=P`-AN1w z`0N0mQLdYxdeFf>A}qb=e_jCaBS%}{{=m?_w2FyW-vp!_{)kSfBvlB=e8|aOt4Nz~ zQ0_Uy(rPsQ`PYtnb}rw);*nf*$Sqw#>u59@q_+{}GL?IR5djD#6Q`lHAh*@Y;jpfJ z@wBK#KCzE3B;Jb+l@MLQlLbv6%wA}>zNMFvhQVaXaC3_?UYOKkkjJ5DI*&+nJAW=H z*gQcKh-;(5l;zXZp)h^eb5vCWgK`wFJ74lVEQKWXg)*cO=-l(#B7=DmZ?nQsNx$r5!eHynTR=F1{5VxSYqm`&8nX^-4i&cl>xrW>0#MN@1>w z-H0|vfnu`gSu4Y0daz^suTbc1yEjjI)28PM!>y^OhNyD zf(s2`VUk|vvrTvr-=lh{YEL@}v96y|k({H;-}wjmDaBZmG|w1Z_~JOaPc%t_ z3F|krMJsQv0+JB^-3QBZ5W$@*WvOLZ{MWBgu|U%kWPnf>4Qfaqy&CS!!f2c6z-p{g zXIT_XB-tqc0zOf4jOwcaLAZfc5b?GSqoIOUTFlmCitOu5++o6E;#~3>4AEs#!j6m$ zYlTUQv>c8oINYU(1Wj0*jU8`P*M89n-TL6%v#9d=jLsJq_6R)IXib z|8Wi=yRYnYJeSfUlgm*dJJm^ygpH=(<%F^qKJ%b$`=N;8JX-9Ve6CMbFN!oY96 zxvXXDE>{GE-K{B(s0z@E1>S~N5<5P{*nH{%Y)E$WQJ86WtVlW@5I?b_#Upp|wCOwR z$4S~jmnT=0C%bArI}6<;5NwxVnO%IoO|u z&Q)HV`2#&+nr;*IIz4$n50Ga?EgV(uae|$W-rEGYdH{DbkswZsLjdFoRSd?2gIIYr z787H@LFih_s~FrfNTTbYPy!k-a##j9!md@dw_4gohMI?GW%T+p9isd>4vRv1>GG(- zl@Q5=xi2hmS($lEA*M}{^x+$taA!2LCU2(qvKfqXF+h>Spbs@1sxR1Q+4vA%aL2Y|n!pKDC2=a6udt)uo7x}oAQ`bf*l4Trb?x)yN7&XJrw zl#KGdM8yGzhmE^X=EJIG`=47ZhWj-D(OI+^s%o?l)Z$z1|RFV!BWR1pTD#Qf2FgM@#;6{RQteF}zK zZXh(cM6pDfX$YiFH6gd=`n{G2@N`nV3}Wq@#yc~>!-eK+9FJZ>+!Q~wHGyD~uszN) zJg=!BizJ(zNF$`Nj8MHAwLgCM$4q;g_E&ct7o+@L%0qy&e?H0C#uVSvb-!|jUt3Xw zml~>=qs??!_Glukg7s}E`pIR^$SQR3XUD= z2O|a2D_6YM$I17fPDxQ(-6CL%y=;j{1xEN1RC+fNI|iHXEy;w4dy?(f44`2bE9CrX zDQY9em&6mWO(@`3RW$M}vB1Ws;2wHzDoY5%7CK9YQ&{@As_ri$su{}F;`ahMwdQ%WUXC@kQxVO8U;F8! zgMKN#x^u1WibCyX5hTBSwu3qg9Jq%N_|-A_#*U%=A)KgVZTw7xQL&+g-aEI2cje))tVBE5^0@o69UTwAQW(6 z=Uq;!lRPUN!|RS))7tXVIcXnqzXVpmL{`*b`)6%j`d$(G6BMVHnU6N_ugmZJxgR~B zubBZN(;!S#(QLEj>5#u!?W_~;w@1H0W7i=k=OHoN@iO9F`FZvPZGBP@N0j=O&vylS zhpS3Yu0m9Ka~rorw0*5^GaYcBfUYy3+E4Zp&hetRO%4$Uk#FQ3bXEV{bA6B*QXqcS zJ^Ko=Kp>BqVMRvyy?N3R{CiMH6{XaaIt|X59 ztUux_mQGi8Y7wHhg0A&F|AqB<=e@~aZVPbDJzEJ7ACV|7I6JP4fRA6JEz?Pon>Sz? z?3Hddj#AH_1nx=n?-0`yqUf+RZ@6n6~rz!8H`^fmsrth= zVM8FQ>l+F}4R?LU{Beq765do{IbTajLE@&Yt%(EU6sDN?WGV^HE#;P;pN%mq$UU%PnKik!)9RV^y{kp z-(7QPnsU=sCCVvX6>}{dZq#SZ{`8Zhr~p}?%t&No3a*xe=F^}h{Bee`K-6aE(^R_7 zI@-+P6FOqs$Zr@2nbS@L@igy-A*FNIy}%IF&8;*&%GdkbHiZmD<_>6SFD+hTxH8>q z&lxXW+Jz>1Y9?nCvy{WG!q0y^v)-u`^F&YHs>e7Wj)*TMybzJ=Cj3g8{)NPX%HnLM zGfqmjTdS-jqUCio74;UoP_akbf%O z`qW!9y4^5+U(0+yTydSzF2Z&qt$sfoYS0J_@1RKICGd8`F&xUt#U1%8q=pt9n<3nwR7A5?`{wOqrU&g+I8`-fMh5HNCL6( z3G&JL8OjV6;J8bA5zdA0k}Y3$|Ci?@6M^)-$_|^1xcQS=uP_QBuw17e>1`+8xJKFC znp*e@HSJzkS5et2K-k8h^ z+w{2MArSZ&QAqSOtG`q=;%aaod3~M2X^pofOb982)n|Ny-?DAS%>cFwCj;N8(W_Gu zePmf+;B#036wb8rCox7;;9{cAT!6xs<%i>K$BzApOJjMg@xf(fHa|swE*(m7%i;6y zG6OVZPq0Z^_Bci#f8fY-_Sp2+K7=A$91qmVlmF=bXgap;geWE$RD+Hh<>^z zdKzj;AwOzph#cO*4J zlY(>sE`%T?xNysYw!NnKq=D3=0CU@pilq5{B@(_h2%^qbL`vg;myFWsgCP{&g;N)P zk~R|xQ(mNwsp1?Rt_j2;nRb}&Ir~)HLEZnI7Vzxoh%@~-=0fQ+;z#=C7L0$J_VzJ# z)0q*eo78QYrIJlBPhVc~%yfKxi?qmAQyjoolsAzRogY;6x(R#}bh z{e#DRDk0Q9&SMMTuUAPxuM=6umZj=xE$G?9sY=srQGC%w>uX^zJaBc?#;+5I1`s%d ztACyk$3=li{6RTZV9D0F3;aF1A`W98;z;xGu`8w)3aDB%90kW)^2mX|*x21%#m6x3 z$7Jksji&{k70D$raXhl_co)13B%g0X+@I}MjrPS0b}CekMT7S9)4aa~h9dBv624o) z#4b008(kjq-N^=4-R(F!1Qso6e{Fp3x0tt7cBLs$^JiHdD=L4kxAlb8{mmd9PWuqNgl zrZ9Bj_@j>)RxsA7sf{P?cuf(BrqMObMLz<{VWPO`el8eF^!Ho;^opj3Tx%E6pMd)Ln}s=RDliK#Gb$zro>7XM{023A&La9_w2E7+*HV_ zm}$@TsoLm3^S;r5`N(|FHb}<+;Sb0vMMwTL@TR*UKG5;fea`J4zJWxUN(u%RW2Cv1wtHQJh9ty=bl{5y`>YnzCng%R_X-$=T63kX>@LGo`h-KB-6px2B1CrrqheJtNkSS0U{Ms!9pB-hiGK{{N+RHzt33$BBq5IqriG!aqG zrA@FBbhTtojl5JAj3J?AVvl68UQulP{~1n8Gsh$u0({zm>Vb5?-jGg96DPhj7_`X+ zl=I*T`TB+^xqKvfhD?SS?8e`m(In@)yJld+WV~6y@nH69 zLc>`Ugx6XJ7SHxeNlJ5}PWfXUnN}nScR%L+fhw@bH%B7%Gn#4C_&@i=ieg#MR(i(b zvT&h{I!juR*qpL1X7!SKrpWeUP5>>}(I7RMj zO1JQl204i`5aSfX@v+i7*~3CU+-d4~EQdGm&NtdUgF(s!(nV7vkOoL~MH@N6EzZZC zyH;Uf&|pB9{jzhM68<@j8VtCKg|7Kq6UpZGG$JCo_(!lHooE7Tu@I@MiUxqrh7;U8 zNQnDZgI-vEW1vPJJkb`QQY8%8q;^VJzK}v|5_>Es-^8J#Q_#MAC>HC|=)U~6(!;qW zMDxOuQ>}yl7I&V1n3)BOxP{#UmZe}Tw%sN)>2uHh-pC+>vNm!{vr$r>3V0m}8K`8c zO{(8kE^8MIF0_=AT$wA?h1tXZPZVJiNyJs%%-C7U>UBYkCRhx!TNY#%i#h_B(r#HW zN2C^k4Le{5-`Ow-)G@721-1%7H;VXv1(WRq=BEo%cG}q~f>hmZuFnrAL`|XJ`h}CH z|4Q#P&&4FAyS9-nU=GUS>dq=f{hM^dGJaPT4WX9C54+!ZwOKlix z3vn(XER2lF9Lex_C#d8rGa@JWC%|DlVBZK)2q`3UK)%)7Zy;J zL~%dTU5z`qH~(%jl&iV~#bZI;BTNd`CdR!K>bkCtrr3TICm8QbSp~w< zm2eD?J0I+@#?JwulH}Z>)CVG%Wv8tH8BFC$o3!)*n%JI{XzBKkO@*x*9_a)aPlAqf z0!Zn%kxtLD4~^YO?`NRvkJ2yD8wwroOZ^al>8&Qu-nNwD7BHx&?T|RJND5rr++?Dw zMmFn^9$ktBTM@jCo>Wz$C-Cw8+Zne*O@EyA__{Af7CS>bT$ujyAS>6U@uk%jzung) zZmeg-jD-E$VHHJ7%UX$H7o*19v>`0AG36hIYe0R#ouS(Hpjy&56a59Yb}k*N03)Q5 zNs*qZVer18;RXhg>VP@Iau(`s!%*p!j_au*wmhz-wM8jh&A+_Re2A%v@T8u%g+uJu7(!-$r zoCei@L{`QRC_9KmOH*L#8qOL-0kw7oU*q~gWnCru4_G@cR3fFZpJY}}{BT>i1&G*R z5*2XFAv8j?R8Fp)q{)Yp;DbltKshb4xpW+IwKHk-#^h2ITKJP2x~r=S79T6-AN4&op$y>rXx4v_#y=Kj}AIsL+`Z1IKEovSRvzySKRso8C&f87^-1KaoSD$g!46nR3uTmsWd@>4L+oSdcB zGg*Tkw$0T<$nTECS`2+sSNjSB3(2jZ^mLb(Yry7!m)-mBCv}#upG)eID6wxN6V`qk zTj^a2fMev0yw6WLctsyx8=ZQL-OtOR0=P{Dz^k+63HQHVGDwYd2p##SoF%&={8G&4 zhfj+0m%ljQ#0(Pm(_q(5pqa~vx%P;`l+^i+JfF!jOP+mJn{3wOZw{fc9FXtiQWq!U z5+lso9+psJ?YQrf1jPmfKxBlX3`Roa=s5RB2JYDQW;5w49y<_U{}|g9PVX0mcCbmT z1{*V&5lrdIhT7z{gdfe||9=@*$0;Sb0&GUdE5&*Cn~j5c!WJS`@MHCs5yVQ~3w%(q zQ^+A-XgeA;@)HLJc7X9MZQ8LI^=|EJQxqc74AV`rdRyFEV5OHZWvi#2gom=8>M0Cu zSTW)lw=iru*RJwLz48`HO!a+)B}@8F$5v7D#b=ASj^K(S+`v!3CmA4-TpM95YOzfC zb?0miQ_bs|;&Vy3)_}AyFHWKX84F|aZw3bPQUk~Gig@HkcGu+bM=R^(k)wP)NoIk#6+@ zfHG;)x6OSs7f1A$Q+f0SV6Wg39n3C@viW3mNeZOpH($_3$McA1 z2bQzlZkG@A>)C1@D=`f1!X}o%as>zx@JR6Z*$-?xWPd0dy%@PS-An*BTZxOKuaXIK z`fIc&YlWymaUYt0|AC@09QA#O33#@CG0o%Z&)xYA*Ds!P!~XUzE(5q=H(ye33{$4GnPdtUlT&~ER4V8N?Hf%6QvTe;`@6_>Pmm_M)@9fK`2&6 z#CWD=aw`a-G%SIX?_@r5a#|kTdP3)gV2kSI`i|aVUkbxw z{AcOSdDw2tS|p{*vv=o)?#U5?odw!T-cHkOI&9@3FFK_0;gc}T74To-(Y2@7VOc3l z)383>dwzPKgI`WnF?O(_vB*XtH@-3K;` zS`@#(LkhZfF}JviL`fZhDiG5HF6K!)t%hD*!}yky(t0=EuOy1K)-@wFfslN93)^=Y zui{Yr#Oy{_9d!lzQ%UcR@A`-+3u~lJ&@`5l_e_MDNxIhida_o1yHJWK2;nd&RRE== zO)v*;8Yi3Rs1$%MgU)lOPKKgxF**mA5m5jfCN&_-$S~O}IPnoQg;V$dV-?L;ajCLP z`C=6f5WRq%U`}h2RlM;W=Am-y=YPxg#+!=f`m4hD9-I0YQberh4BKu#57@kvp}%Q5 zq1cS&sAyS{2|{Dd-ue`Xx}GtsT%*E21KUy%-yV)e`J!gh5wR<=FdB1geo;XjiT4AN z>;Ri3viK~q=Xke@j!Po#bQ^h@CI*o492b-p5aB4eth5-_Gct*q(PA$HK8CDP?N;i? zX@$;!CMe`Es~zXtEVRC}vmeI}G8jwEbVl$`S|Qgp-aUu1^d_PRLA4Sa^W*b=n#+21 zr4(*^N`{VZ5+YFLMlziG@C0sT6vY9g)8e91|GS2Ystr7Dfl2yhbIN^B8%7KkRtgJDO>p`d}zeN*FK~lnR|6IM5Jp6SDj*^%O5>zdm zk)DlYy2hZ2q^)27;nh=-L;noWe%p-}PF{pR`ias?%yL}4uovlqTZ{WsDRgtD*!p{T zS#aUOzU4nI+6btK!6pSP|zO5_sEQTdPP*4cQAE>J=yl$M2E6AU8mS* zhFgQWf_M}95^N*1=rTA2Dpy-F!S5mJB-^-F~DB-H5D*Z3~_5MR*>LxtSvN6pSJE{TXo#ybmUJKLl}0h z$XiQd`Gee5^#?v&QGU*p#`ld7$-bXH4KE}VgPjpMd_y`IxXBhzO^+y$4%9C<{`}`L z4`DZ^Ur|~Y#JMyEsuGvXAl_mkYk#%dU)*6W)D>(uegNZr)TzrKnFYMF`!5AhQ8$Kg z_v8u;Zy96njhmXG=S;5A!VE!lN&0z2xP$!bYD*c>DwNd4qs_=oeRZ_st#6m2U+B^ z{1PA%1iKI_kcSDENeD^ro9y|JKtN)s`(<+Ku@+ZFeRl*)#>@&S+}m(2*aR-^Vumg( zCQcwR>Uc+8aH;Q-=4OMvWegb3_}zO41w6^GLMyvY#Xr92%>X{}bYXVE!wIaDrnIOj zW3-TLl)q~r`1G+X3Q||kJ#7H1d~Og>Gig`F&|o8n0zY%-59eM1 zNJ#0b=ghdueVCz^;8zOKDKt%v%^Y}MKa{dJObW@B@LWg;@&r0ilg=9sv4RC=-ys@R zmD!R3NeNFct7+hTJlvu(>cvjDf1R3LFTk!mjP zZWv4K``YV{rI?~`#pQFD0$Bm=81d2@Ve?UWAko#^7{?!x#MK~wWl*5*0_jKDca3*{ zk$Y^d*v6T5vyhX_LbBcX9|IlC$nHv4{QbR>HgDxqyR}GO7bpc-xy+Gt8;QrAdMO zI}Bv43HugVG`=FRBr#~D_b7fC;O93JH+U+UEurhu%Y~wR?DgWk4WLI6n=RaU8kJ^E z4DhgPr53dVJQ@q>NBw*O5o~?B6H8O|7!Lxc`GIy^7id^p3J{iK5kJa|rmHFCW5SUP z`ZM*X^rxw;U;~OIE-Q`>8!4@Nyaa;pvjUFoH#0D&162T4H)6}As}d-B)iUoS9ccR1 zZwM_glO7bEpm%Yke02}p%P9~`0l_p2E^v}0@R7o>943tzm*0b>_grs6Mpe* zQGsrf+UXRT{koe?Z}Td^0=sf+>aP)~ zVu0qv=$Ety3G(82_7-soIP5~l_mnc_W7?kg;3)MR%nGvsnp+ib@6XnIyL>xi1GP#2 z31GN{Xrh(({SO2Qi@xsdeM747n$Ls`2rM&9`BpC#S+@D304d~T;qX&6h$~Y5fT~)* z;Kf=vs`=5e-!pDHtqbBeGXi*Rh04F zA%%&{4(QsTh+LBFBJnCT%AV`V;#0oe>st5jfryZ*&;;75hC0WcUi!`+oD-nhcPItQb! zn**9b;RJSrE#nkxob$mJkF_!p?5jKbY8h81j2nG4Dz54|Eu0702>vDARa;Q|Z#sX*%*B@x>_$ z)92_2qidAB>ZZJ~!Cc%A=jLRBAX%Kq5b8M>(4ak@q#3*@%{~RFD9B zI$TqLJ6~$8C9NBRX>(?qfYpDgr|Dn7lPAy8`|tl%u$7C%PXvVCEw6^VF2zwuwG3N3 ztNNtQG_~26-U9(6!8a6B)0GCYfD@!NrwB?s7+oeWPUOm;0O!C|j`Im1A8t1GW>@o* z%KCXsk9p{oLE%!PPt&T`xlA{BR%FmjPd?^sBC)l0L*`Oa8u=ofzFhQzd_>Cr_$gVq z!Nt^sXUrNXGLRhqTc)&l0(7oj#GnT;=*WR;3+g*8MMS$Em$X5cy^v_q1~$?m`$-~5 z?LE$pdNA4f!85AaqK8cjpYpjNc7z!cyS2c{o-%z$`XvbghmclaxYmD9xMeX2Q$z3N zrSeqF3TefKRxK*qgCkrKE}ii~bDxxe7ohm@Z^bu>4BlyU8|zPKBlMCTRJH+r0%Z0C zAvSJx-y4^3knOe%2(X~bi&=*raZtwCD|a}&?5Tn@rm)bI9$4Fayzj6s5WnZ5DzXT5 z+bKamvYDB*r7>0Dx5L)EbLK$XPg6>z<{=0>rNsPV1@cJ+qbV^ptgd2U`NA!8a<1;k$~xmD*v0(HB)CR&JJcvBhGQoPYm4xO<(Uv`B$KaKtVMcI!;XA zlGRD_#C-@7wDK>#S!W?&YP$ArXmg8dOyjK@BIyJ|l6)spU%nvQ{pdmzBP;oQ+t=Ih zJ;F=HIKrrQ>tKQRw>!N3-ZsVAmEozhRzHSq8zyeEX>5Hl|NlB2^A<%=`#h&DzIgp6 zqd`2Y^4m_0>TMI{XOm}~ds!O}L%OVc$v7NzcYy;*Se2Y3Rt;W~n&GYQ z?)i>sVC;TuN_BZ%oZ($gdB^;B03jhTD-CF#=)k`G+&-kCg;9q(V^@6eBJakNyWrd! zZ*^-W^%ILpEAwZM_ydWqyOQZA9U>R!xQMQQQEDt*BvET3K%0-tRZj()E9_M!oSIiI zo!fD3QNCYu!ri~;ZKG^GD1WODmF8j94Dc-qC0}Ge)coHM-9%)m@no37EA>O$La;w) zUof-OKdPqpq*Pk+H7ASap?FpF$u8SrY9KJey|cZEp)$xNvU8{HO<>Ujd)QvUm}w0= z5Bv}1R0fLG2hYu#AW#r`(@@{XjUs3AyifQH@`1Q?Mh(V(e--*hGo32d3DoW!wwAcV9#MDf7{D0CllwXg2| zQ6&c4X+^G7L{MC|)t0}=rFJW>Pk;1>E|C-Lr3L*S(S_EHHx|AHv*Cjs7rB34c)kN` zNOM;}56y1UWy~W;gB<0Hfh8|mWMsh(r8)_in5x~ROFvfRj8AXlaiO7KZpOo{^g|{u zV_Jp{Xg?1!BdF(toq_-p;R!hKDejnzcCTc)t zpj;E5#$X5dI8XJ!5YtfQE*glWqmxmy$u{^?0a{4X>fn^!N}4Xwq8JjyJ^v|JuI8Np zX?a1OpFG4k?M3-wV8gq!PJPhF2fWLEN7lI8Zi(jIwOHO-POp)Ki;J2vXYdxk^o!0< z%D3PEcXhYNSV-Dj01$MmL9#3YxM-(EQo&2BWYl=0M zga`GLXsL6s4tyV%tOm&)p*Yf2?y9T|Npl@tX&`AnAZ2e2Ns?$uTqV%s^Hg8 zXvMPcbMN)fyZfLYltm`}_X zWdDO;);pZnc>Tp_`P;y@EJ#BXH)gi)o&*8dWNVfLB+R>(7SxA9Z%XM=I>~QE--V8C z!y_%KbW|`~K~Z%Od*s4Dj?I?2g$X_7&x-X+M=Ro8M}@rlGjrW98{^H|RNu*D7=2#W zN$|WeFI*x^PZW}1>e0s0G`Gh4q`Udw^2Xf{c~0+bXB3VB2OPu_)}ig>UFYPWNOAmr z18Dce5Fr2WTdFewkU!!&Lp?X}b#>|9NrzW=U$k5av+Qy$h7(ko>>fqyYf0Ry$RC~T zQ(IS)atz3ecPc|?@X#aV0**bqP}aWu+6VjTz=50a%$QJ%@iigYbke1Rj{7;D0BdYY zh)MqFve={AAmW?}PEDSy*BV-V0$7l5TcD`X5#stUY)boQ$25!ZY3KDGNE|0Q{l@`!bo22 z8>5Ufb@QcuoYK=~Ikkj)Aw{-j`VhQ6?7mC|ckK#6dC&JTBLsQ4>+%cSql$FJoB-xQln7oqkLH+!nnATom4xQ1Qg3z&FYkPTu<)^0Mo_YR4 z5}->3ZelG4p%E{6ii3Si0PJ0ELYDl5$19}U|KFm-_nTiehjs<#o%ysm;NocS02N^@ zfA5L~e+&g;2jrb3*zFDr(Q8{>%X8w|AMd^GQN_0Hm#Z^TRhQUO<|TG_DApxy@UPmTJc#;W|KQC4uAKxhHv`gqMRKY$ zYD(a=p*9y`!dX0!D-+nxVCR#x{Q+`Lbibuq!rwjd&f;mBsJCcJ7tv#?P2WMlUSVm% z-Hp$-{V?$^Ei9NR?!o(>G~-!Yh<&W`M+B8Ma2rJ3#ttss;?Z*rTmd{JPghsU?g4&4 zd!F;`ub%hesL7c=eVktl^a2}2kqOKp8&uoN!h}=^cJuq3Ov-W@&h-w)=zzMkFFuu4#;h%EzPg+fEZN{m)-W=_z;H?6u4evwQ|oVgLq)#oi~jN7fR9MVLLz zLjJ9Ae_Gw7kk*X|FBa(lv#D81i(x84>Oj%*hjt&i?azanxBn9JtXwGq0DCX3)pGYL zO}$!PA}MwPSbd1a^Mm4w9zF=rmV8aiG=fGD2|yoGm@Jx{2PlguD3DpNeULui5H;AY z+sAiqUI+-)_&* z=^69@x0DfqLcEfE%LAxciKeL7pTa08E_;i4j(C(hV}f{K*iA7pi(w-%YO~c|j>TF0 z2P7hRj&#~p0L8A4n(&u9yNkjjfr>efrnFbWb9*`12RBQU*P02-cRehcO5m(3^o;5Q zG-2dz=6XCxORF9yPt)$S>6}`wH>^ar`o!`0;Auk-N9wSp5L&6%j6lk>qEm5mPH4C?(P9OjIrv_TvK&-Q}_|LoVGLlnFi+=f;K}G0{W6J6i zQ$5bqM3gT)WItVhuU1>i>t+AV?_?86-_luMEt*P1@h!juZQH5=_E)%LKk2VACg1W#f{eD>VR2;%b5u!W1!<#5z*@T^&f>4^`?rzaZ-e>K<0!mFbT`1ogIO2 z{|I*TZ?YXrfxOBmGV8LzXIZi|5ix}k_i4KOJt3p`TLZm`H4Mbbt$($Oz(JOfI{`LT zou$!(u9RS#+HhddpjGK1z{VoMBmBw^X7$sX7$f1D9%qIn{na|XqQe;Xo=Q$#AW7l- z6pbsrqcl&SQz0D-!>}Z*rwq}I>Pv{Cii(o&c+ zt+&|PbT8Iqs(3cuT{PQgkDD9YyORjoTBNYK_Lu! z^v}8}3}~9buoRu-wKPb~;!eLkzkaI`><1BHBjlpV^wc=_)M-Yu)epmq&rpuwtDxtR zDmeEqSyzFO*waEx(hK0d%XxEZY=5Lq9AV^V?;LzW#Vd`)Y^V-=EYCIvos6?zwRljJ z(on`v#PkLGy$S)LCF|?2b(>BF|-fnW22qdj0ENutHOpBn< zykyfOCPvi*YsAgK-%p?D()Wsy;e94++;foLD zA%5%n89^fg(F_Cy_Ki0V&BCvz(xRRF53w4Aba-A69hf=3+MUq;o-%-V;a8&$#w8a77RY zyh0kNrFwqg_H=caQy*{B09c z>%;KZ9=|#Q!`@hfaGRknLY30>TJIi-39@*REwq`f9{Fk41Qy})cIHu|w3rB>N@=`n zqctuY*?MPCgnFpl*`DKAe1GVdbnGG@xEW{~tkfbw!8Z zbj`rZ#WrL~Eq}mMdH2I7r05i-7~_p^p=j&KGQAD&n0nZU`K}2-bcZhRcO18a==EE2RCnCYJ>71|hBfOcs581ar%9{MNyy zfX}^&iEB4EaBUuCf8b{{QwZbB`HUQRTN5=Ob}!p*7(e41ChL8*{LI7;xO@+5ZL1&C z=+>QlWC}ypG-3f99}J4E0XuR>+SBV=W6_+PJVF}*^1+@W3<7jzVUhmcXUqYGME0yY z16-cI;P2=NY^o!9r!H5XJSz73ruVVQYB4Tv>43p*w+6M-O@t3U*y-jX@D_;vlPI?F zrP}oaJ(}TfC8Q`if?bpEl}!n8J}~{{I02c^k&)oIEaoIOYsZuC5!<1g@AoH_F$Ga* ziGPkNUMUfp+YJ+s96tK^awa>0x{cEF$azGG& zQbow4fUY%0ALDUrq?w<7g5Jdj3X?79QCvUV)N}IFa?Vw;7Na2)gVNjsg6oxgupbTT ze>C9UN*n=OywaX2f~ADbX)i93#UK_D@0&y7+tPzqp0YRBnNqevCtqo^J+sg9OY;Ka zgs7x)Dmqtyd3FjOj>z3@R-j6(v|9x!*9*Wdm?lub+|R<~9L*H+rHu2uKZLID%;m-8RQ1+SfMs|!Af8dT3;r;iAUG9QCa4L62FbgEuT z@2Wm|O2?$M)?{9_xai`i#YjrSRxdmKUd_T{6zv9j=^WQOz2%xs<52*ml{o-b=dW6( zRq-v3jIcP2xk{)k+BFg9#r>$nq?Y7NIWW~XmBG7n@KQiHzhPpTLoU{^fE&@Ywis|J z#4`RAJh)PKt!YSxr-Om4G|iW3va!2yEe2pZM}}+TgRKqG7^+fYMm8Nl=}M39zJYW` z5iblr+4Q9|2{Y}-d3WGpQs+4gC2SCDaEp}{G7~CSD)sfFRR#cG{i~w=>`k3%Up8MN z{MhV5FQ|pmPZ3iPaZ$>0Q0 z#>DH*-Pa;;V?^F)HBGY-8=Ec0#_r6gq6<;kKGFgwaO=1N~|a~liM=l>m5>N z6xI2Y3^fIwe@euiRW~qOL)Kj$yJUoYrSYm%(pjLiyw3~6&LzfcGX4%M14s^gcsADY zEepM0CDCEQs$m(1%&~n3|Ez9)o)w&#s6*sD z^c8_~|K^Tg>%fLYPh((>(sl=4FXwX;p4DrwMF_)^CcK zkwCg*+pWkbGkXnh9Y-f`-c+j~SLza}4M;?!5~EQ&dIU$jqTjcgxq3D+085AgkLzih zWBZF*G=uRM>Z-mKwl6yyWMDDQn(%11)CclV#cDMPkD@U6VtjM5B`)0Okyg? zYRPQToN&~&ivCyejZcb($N0!0r4UDI#s)XXMLxB2E;OXcYGrG=qBtHO;q0k?vmPLN z+PDvfEa59khmrN0WZ_ST8$~1GL_zvkSW9t^KqdMb8HPfPh+esRYswLWs`7n@q>yx4 zG&c0A1}k6V0oC+9?8~~4Vkooy8e$7f^pf`9viEbZbXULdNQgU(CM@I^M^^8>qms%x z!ev}TB=2_kL~)b|bnVDA_H0{yIVx#NF)mX5OTgtI&|j5`;B?|DPG45SHyqge209g_ zUoR+&LcR(~5tbqKS*RB+06w1^i07@Mh@qj8*TKbHYzt9JeV zU-t%{g6~dHjZcnj*-ebmD&sC%`d|h$Ik;Ou``(2ChS-k;w^#&2&3%em{PM)KIuoR0 zr(``kIn-!LVAz=cJ-@9A2y@M(+XNxY`!YC?VEIR3 zS!<<1xA|KA_EA8}WjW@mJi%~M5F*~R&HdEIRt_XjmcW1kWgf?;h zbDG@<+^L<%0LqcrAdyFI@N~VkUP@my4?kR}KjtiI^gRWRdz;pi@)ila8v;# zV^)+qu400;2q^j7@hC?XvDw>4kIRclqyEBB0mmywp4w&Gu2wOOk7{}g_RA^_qyP9r z3^7y~cpLD;CDBnp6V4?Rz)nBKt{*3Xo1A2(9Ki?L{ka=5vZ$j#?DFu{MGv`}3FDG% zbaQ9GKX1HVD;=s%LmtJ$CX;d$8rbV~lMJ09kIq0?tzTMFAWom^_;hPk7tY55WZc$* zKvugBBnZyAFhK~iFm3cZMmV3kz@`7zjFmH{Z8B1k$-B|+-sYqcrVJZMxv!NS(B|dx z_2+!+v)Qplm`7|?S!x9#%gjU-g1-&zMdNFTj`1phaDMZ0AnE|kV5sqO0dfO~sfC?*a%dJ9!m)k+MbpNBYmt$MTS|$;eHx6l) zaXC4yH8D`5B5CzsUW;nTkIRa!+$2%D{a~7qX#tBj`adUQ&8NT>i@(tf>nCA04lFEB zh-(Oa;NREwtn5>>SJMb~5Q@v+u~(I*L&nn(2!rH`i$(zvhLZM_`_Bt%u-jsOR%ByH z%(_EDas_!Lu-C-VVF5(+ZkT*!*PS!JG+C_=S=wntYijpc94`^}kFagEu(dtbpxZ0=IV)N_BV-c%UhR*&?22+^POge5v}MvH`;| zJ-WbE1O(E9(rp!VYQ3#(zn)^sPP1A{Ud{R~9O6ukBK%|E4klj*yN0r1+8<6#5wXxz zl{nSdgu?7ZY6A{IqHNW{?UnKW`T#1Ml&k(K#?kT&VNF`PU&={>Rlz-E!h?^dH*4E4 zjR^<3U?uRgFud;MxmL3oD#F>Cv!q7F)Kr7zTv}bAA=YsjD9^ zeC3oMlab|YVx14WB-gc2Vhq+Dp0@^hy^n&&`V-{I-FZAa^eN}Cb=KzBVhvD|q!}K` zp?YqD6c<~ODxuU&uz0Lqa4K?06>t|*%Imd!h)tcp2D+^Yp$=*h6L$k^<2m7R`Z|dJ zFJBNVM|DBr1~`8|Bo;Dt3%+QquoGxWz!j@IBoa|_b>mLo*^Q^*km$gnq0xXH{Bjb{ zzawSQq-Al@P0|>F#MWcd9lDW0c6+Dv0kd)!YTtk3-XAO5?ftHhQK_H`r^Ks>I`7KyD)w%>4)vSJb@P->>15>o{BsJuf2(R{{B8U#{*$E6dlu>c1z3N_0WXc6IRiNk8CTk z<6%aYkoOQFh5k_sIp5R*1ot~rUZ+;q$9M!fA?Ec`!3Lv+gPS7Zk23w2OG2@CGm)t1>=qIu_d3%xnTxiwos@0F_f`)Tl_M=~x*E5jhp4t^xvbQt z@HG+oVd*W4pPseiI$`=o8>n7O=L)Dl%>H?>nQ`UBFzk(Xs8D#BfinCsl0K{FXWa&a z+HMy;g`?ASYI<`KnF|L;)lN&A5YMXXg#BSP%|f9bN2ILDG)Dbo#I2t__i|Q$Dv|Cs zROhPcmIpS|2bthwIw1${m#~W4(dT88X*xW(tk+;Ak=G5z%PKn^EX#6Tbb@sU@d(9p zL^O&zxzGr5B_|K@N~y`=A#`i7&wAV+ir12R0mIa6b)8fLcK{--FLQsXZ+n zYWC7oDAZz@C+^s43|RD^7Vtp&I&Gs+4}kv%f0gU!Sd-kyvNC~%04 ze{i=vGO%<+v~UjD=PwhQJ5Zg;^qgp&dmj!|f6^SQ!sPQF@d8~UqI5%Dc4W?%O4Hsk zczn55NBk6w?oGzSHaUCXV}5H@;3CnGA_Zvfi_P-a$2E{upGlKOe|}yzF}eDQ8)<#Z z3SKFy?mp6geG{Snh|ZmYXFp$Boe>VnMq{~@IY)_rX=$$dSw=TZmq)DAYQqS>=}aP* zn5I^#Qio_nrR`oQYfAnc1Qf{A8R`9#Bm`Ag>(IYb9RCp|3F=P5;n@oYAMsP=kyomX zGzC02rY1#BM943^J%G=L4URUHdDXPAnH?wSd&H3yz+K%gqlH3O9hNG{@Z*&k)EcM%Q z=Lmt6zI#saR`zIQCcSRdr6;KWqf)R~4J#S*6+kS)fn3@z!QT4DkA&#sQ?X2V5;_EJ z)J$3w{bs_5A{bw81r48s$Hb;4EH1Lz+#H3;wJgtBGbNuNhpdHe&1vl=A&35P&3W2? zt*kOJA?LU#N)q<%CG=Dz?hK+wNU*nH)xa+!ANp=JU~(9z)dsYetxIb<4XTAPF%$|3=VvdMLh z)4PtHh0@Cw6zr;V&$#lViG_u(>(Fr5LYDFl@nNA#qQE zBJj+pq(K^XKkxQilBBN}slth3Qg`{w=%lJ9KDo10l&MAUUqYndG&!S=3qb80HC@)0 z9Y7vFWq;i4@$Ms1{pJdIE9dkiyDttORksIn7#*y5<=YJwBAHP-Sv3QDbb)?odJzL4 z>x(8ahi82s*>a}^nJP5inO;6wwBm&~w1?j`u=m2}eIC5EB0mmFD|Wpqq~;9@ z!Stpg;2N{PKIlzBe&p9tnUn}4Zs!<~Kyh?9<{`ILR`IcTaVc8^cR1oHxKdE%yr{Ez z-hX4#^j*?`IvSKAVIYif=6-pdN#;8@Cr>Ks_jGD+jUe~s%17q5kmogHUHK?&?d#3m zmy2X$u!j?T@8%DAl6`YkevjzRy*s;Hrb5tWj&e>KgUM}S9a=}%sSJTbda`aOmI5I= zH+7x#kjunDNViICyxzEJC;fXjne$<*>JVUsEq#IRT)V38=pq4d5}c$hyit)`A|gSj z%!f@oz36D$@VtYk_tJwO{4z&!z)DxGvYlsM!CQx*1beS1fH}z?wKH4@;@X%4sGgVe zBSmZwFeP`v+J1XSX>Tb?)V|nMU&7S|9Qo58R+wS&MOu#mK?cw{6;jtwK3K!}-36+? zqedrm8)`pv^z zFD$3y@sETxb=&4>YjC--C6IZKZD1lH#kRK6c#j*``UZfW{(Y_l)tugy28k)6FX!kC zdb>-x^0u)5{MU=+WXBnqKwpU+K!PgCOG1kQ*~n+_pBK3OGl8fQ$*4dxStt~<-Nz#1 za`~ug4tnXnub3tVwME8zF}oa9PzcPnF0rKNVEiLUu#WPm2Al}!Lx-TEog0+kh_mP- zq?TYu*o82NN&GX3>SNC`za0?n7AnASG-A_glA5<7Mxd`7bo6oDGy zTZ()yh0-qbdl;&}Z~HLxM_!!LkQpIhyQ0s5>Zs8#_!KS--dxyAt1|oa)3|dH?o4#z zr5_+JeCbQM%Hro(hJ3^-W3MvL5JJfBK`ogCd_g`jKYWiW~=l! zBVQwmjQ;{bhX8xCCfHP~#JsCBiuLnJ%l4>mOJ&obV>wSK7`D%+_*~3QUr~kslyHr- z=e{Fsq*Qh;GATl1VPLp$V$473$;qc(Sxit8NdTLEl4}wkZX$LbWh^00iaB}P593e~mV`@X>=GHYwIKBG>#zfn!%!V0WhE1>bSude6@i58c=XEUo;Y3{{ zIFkU3`@Y|7yAhe8(_kgeD?4!e08@P-@Tx2H*vqE=6w3A9`jQhR+H;A~2xT(~x!U7L zaZP$#6fo422VX5mr1;L*6^jVhwNBVVZ;ymk=ijAI^6+=CZ9chZ4P0t}a|EoT=Yr7w%2-_PLAimvYLP4@`tR(-X>;C^ z?6)JfIyB}_?C`A+8bYlkI+3xeEAQ}W612JxF2n$W){wa3Ps+2}{5^@9QGL0245nVg z!41Oervv~8Ks8+%^rNc$pVi2)^OHPXHbG@$O`kXi zM5^}VtinnODTDZq_RT8iy2tf(K*iJBv7E}YjMkUvRw8b%P!cHsWvw!K?SpI_Ecu<+l{yHYP1)z_G7Y$*I3fz&yTdE zcru)QqmjB@yv!|40Z8LY(&M$(aD5L5kX)#jvo8l5r7nZ#^+Ya+E|Bt2JdPeWEK6;lhfs>*rqG+h8JWu=2Q+?M@ zS7+!@{v2);PaJc^IqlUG?0VkBQvuZ2#i6<6RLIrzeaF&}WS#Zob{!-wb@ z#(KGNQJTrwSNceSk97cR7jpi!-46=5n1chdKI){wSXhJP`UUhw69`_(?eX=Nvsh_J1n&w>om8Hk2 z*IJOs>}xIUq59EP@IO5Or2pxA-{hza-vDH zKpaNcHo~|OlD~00!!u%5wFDy{#4q(mNi!r(Dxvec895NjuoQQ5Yt%>_Ue=7rWPd8|p^ z@}jdAni6gQBLxt@U*`)!OVY~zQHTZ`%eA0zMW5-*G&fC8%KS#>KVNiWy->@Bo>?}f z6?;(Zb9py|UAqe8ZrOx_xk$uSZqjLRIl*0T?5}Fift%D6N$TAx4itJ3%?FQrp>F>L zd5_>>5MBHahPMzku`AO=X(pM*?iK_^6S;k{GE5cuD|Yy0VuDQRY>Ax1I=tT|iEE__ zfOgY;==5X=sp-nm)AD^_hzJ$>gF6fj2rn;3$sYk5%d~D;yk_(st0~`AW^AeAu_F&j zr0QC$RltNzMDej4vm6JJ2Xkyspv5QSV$fpL^n@T}oZ)HEnK8q@(z*S)_(xtHP3pP&UQyJp}k zXrC%KgJMalh%qdMb>@DZ7Z^9P@c1DSp4r`l9+IJ<6E9&VNY~PXB|+_qa>dt)bAv&_ zhU%gx_i@P+3Rx5H9>{Yeyrys^Ub*^(1o;CEn=2~gZUJpWhTYqXI^Yn{4H#<`lOQ=a z)ettp$Y{UWR!$gCMlk9S4)}1S!ZXRwMm@iAx^b8uMZqrOj-=~g2 zr+M18PnPGHP-%=O!)<{p(WlF<$P6BvT-$_oIe<{N0E%ei21J*T+fjN&;7X;d4~}q5 z1?^IyL@9`!OhJ~AH&3tIl_GhRLYxUDc@S@2=K4{{!|-0;6{zk?{=SZ5X?1z3p`Q{7 zdx{2C%5NU+O`MyNXILnl@bVPuG1((?jgg9bQ!WQjvkzM{xq~&u_lOJuw)5GuwWCaq zaAW0?lNv$U1NEdN~>u~E83s;VlX5*uyJPWbRbc(geIOOhOR+VQ%nShe5DmQ61LsMZzz*^{gz;)41G{mT8 zA#q>m&=U=WppQ50s7hx5ZnYUJ=;-e6X5t|OGFx?iDpxC!&sR`VNbD@wPKNaPSl?R| zHNc45Z8AD^|u{9AbIQS=BUJ+%!Jj`w)`H2 zv<}rN&KpG(TvB65TEdRSQb|o8ZQ)W505_&ah&TBwcxyV*BSJ2jfPSWKW}b2 z3t+smZtqEjOrHSVm~CHlQGk|Z8FN{z+fZ9`Bt@cRORas+-2x^or~DkoQUl&=H46wg zBzvN+aP}U3?3Q^Z!2t<2~Q4*R=;6MNY2o#HdD4?o&;} z1X>odewT{@j$He7>Al5=8W+qmd4u?+AlMPxrhBC#hqP8C32s}2KPC9=0l3AUO0 z*vlqWsh2c;m+Kn5z2dkbsCmZ`z#yA^thWPYn$IcT(Q~OB_6TBf^QYRRYPtskWvbm> z?+Kn2Of_X5Q_?>~w9uYw*=OFHd)8us7mVx|Cc)l9yVbB>rcQz66#rBPi&iA;HnCsC zIopoodJ4%XYBYj_Qn{wBW{^gNB*t;tu|MYJFLx!M&cAZ)#ZUFuv*lr{FrK(k?)j}I zpHR+R&opz67ov0CSJO=+Xk>fbsT2Z+{IxMO&T}V+t~$^krls1s^R3siL@chCPyCG| zl$lBTu`QRN>`+2Z~GX`icIO)j^9@ik>-&cP< z8D#)XXi)6rp-9{nTVb{14}eBXOJl2aKpZr-YCDqXvIeJKvyX1J?pBWdztcxB&K;Xf zJA5dEt^crKUYMKwI@&;rH-oX=4O{?2d9)wslEkp&kS_t`s8Nqat*?Cp#bvDVEKuN{ zN+!U=BHloC!nGTzyM)C~!0l6GTSz=by#)36-j2B3mr6#;0_s_hv|iY9ZnzkeR) ziEKuu1Ktv_4|OU)Qx!crIOe}hKZ;!paW}jgxaa``$|w|-i|GX|h6{MV@m?2;oS)k$ zAKLHL$0IdU+lP}pRbEcz2pjrr`n{uQs2AoGiST~~wI)+d@!{p;MmW!FCtCC}rmqT2 zUZulRtAXgVmJ!SBL@P9T&DZdS-P=1Tl+;6>u;g|&y;`kx!o1$F3q|^V+p{5P^>7zq z%~A5QGL#CS>D5?Vwn!X7W#{P?Jc>{>k*>eYbQ0KBMC^sqXsY}K8@J+dj^ zUrRTv&v^HobZ~S=mzLN{2&xD_2D_;Y{%*#0S7q$zZy#1E&(@jh>SuWCSg|QH_7MM7 z#JYYAXRku>ydp9QNusl1p!}U~J*TQb!x`917#~*Qk%0J)C-^g3h%6#fX zdb_Ga4Ci~;fLwZyS6|7fD!9s)~-@JW;18{rbU>txXc(n3O95_YpRur)z~CKTV7 zP7L{Hm|!bpE3)OwRUTTp&Qj%LT;fDcQN!<|SDe=LwiBLaZlb7;9KpBs#^Cdy5)8q* zb!#~Z5*}LVUaquPi+vV_a|Vl==Km@qNm)?kx&{K?Es)cncLak!6KoC2#ry?rUaGhb z{;iuI1ECNwv<6And}^ZggYh3R?V)EA%fSS6x(bg&w;%FP@x&z?5@f0}4i7|mllyxl zKQEv^`A{(OVGzEHXF4-PeIuZo&aP6b(S2FT8e~?KQ#P#-}1kpMC4kd(>0#Kihc zlPVt)V=ez=b?^Y98*Rj@xAOlbn;-!II}AP&AmM+96v>5$OlnXUP*Xab5JXyy5Ac~X zRySA@_8%~$%HKZF59Ll7O8k;CU^j)~_LP#Hf$)u0zAId)X~fH06zUOi^71a(1TBdR z7j?8W`I$b)&#SsPC{UUmko!XXlGQ7yTs@UJp(EZ7`=$!ZgK{?>=$dHZLwhq!!-YJ@ z;pu}^gftz_?bE(5OiB%wfiU7^PZIA6gr6}pUOVj~n<0ycnWt8Vg7Me9$zR-slk}L>8nS%Q**OITq=(*ek5;IXXd66J%nf_6qwK8I`yi|y;w4fSpW)w-j+CS@q=ui5ka{Zm7F;`QX1=D>3?tRuK! zWbJide|6(=D$py3M_bh5IQYuG`k;a;gfltayCFwjyW%TpJfqFz@ejCQe};h?dhG#p z#TaQE@eMKXyXjI1)~Vu=Ba%1+JYatu^2F;=qPM9#s3Gw)g|CP^w4BHL~rJ@rlHW3Yb82^64_?$aKBfIVj4Y4vrTQ&Rd`*;6J)a#@dmw=h4gCe~#7KS)U zEgDf*!`U$y6oYjpfl8dEfoDIe_^8`x>%mdLIZPz%NQ%KOFiW97&V+zt(B8rjjN_RD z!7fl(oJVr$l3j3Gj^&%NklW_|ODB?g91G|pjM=oij7hWSm)+Oc2yc1cm=KF`z8e9FO0VM zPHlmudMmK5(tMD$>dLUj8>D?@&hs60=~d;yvp1JL`&5sI9mL>KswDD<qo%SN;?t|D*M90D)%}$5M~?Tp7D@JMzCGEyj526V5ff1~p3AlEh1P(eRrXXg)3m6#4WJIsFT^OE`azO zp5G})GdM1?CqLzESC_2iA*iq3vgz=~uj^jJkbiWCzreeb;)6#Q`}YfZ^RX5+9`J&X z0HHdx&E`KxH62aEzUq-E14arunh``0e-Xv3*;KsvGdXi(Uv@sw>#vb>*DR@BN(VqqbC2!7_WQJ+?P(dCq3_&la9rEetNc1DE42b8uUN?IrIte;{pA9TtPBOg3Zp}AVgbc*&DoI0VS+gjG3)+=MAG@u{ zNL`=DARQ6fV_662pR9N%b*D(71YzLm!^7WK%!gomJG?TOgQ2~Wx}=6nWwdf230#VV zRowI|Ibun9jdp6P==P|UDA>GHaU}IQ8yd=BKhxI7m8nI}SaH5JqMgyAZD{$aQ6UEX zc31XU+P=ghZKAR!DE$7Vg)DGyGr(e+w|xZ_Ya>nDYh*Li{(MjmOpmCeC0s?147aY1 z0o)|1POb99C5(Ur=S!viFA!XTq3^uuy4;z(K665Tustzv7SMjSEi?)&Af*qrjNS)y zdL8VA@}{iisCN0FeGnqW>t^MJYn?m;;k*?f-Yd?h%AD&Wzb@X(p4`FT`CeFW$=jFE zbp801&>Z(?&tMa|IM5eEQ7%-%eP6plHg)Sr0XlFS@Sd<8A!$iV!&g&N}mimUI^0tBu@z zUQ70FYIkIA7i}mb87w=-KqKk3u7}ki1fP`5!b4pDy{_ zfbg;@5cWVQqH_Wq&U4HX`%#uO`$~uv+kH{xEJU4xpjM6aJA_3EA~5@QfZ|sVe?s>1 zc&NaD&a5=wpTsScnnHBj!fB>=wB;{`^h$hdRl;ES&CbxyK9rK&R|0eOye*o>TTglv zWlpc-M6=Rkr)3onyj78Q3m$ZR?n2{hOmpQDs2Ogra-9lHVq`T)(!ZZ371XVd`>B5` z>Lb?}a3hwQJu}@`R#ik?E=)lZN}lt?+Wk-sze!s@#Y-L)*=h&u_(h)MEr_em^eQ7n zD?E6u9~0_!RTN*xmd;6#s>=u!a}nydw=zzfPmnBi5PEOkioGgm zifp?q+{)>~RD|@1B{Aq;dw$O`fDP($6jO!A0H4Ih9<xsgn~<>Y*H*>w%Q+OBQ4~FC4(oxi@ga7#eVCH$0v8ndW%*L~|o|VazK} z*4o^S?1F4(vGy}O>tj_)Iw*RH6rcq%&>B8GAyXquwav749;3NM-sVNT0_nFEW+E;L z`1wGRvc5~%<^4*CVQ@V>Boh-f2%6i0$X!#Z_VmfN-ZTbvk-?Ctp#*)2vq3Wiojrn? zf#=}uHV9|hMmDN@cR-m0;ov$<9M1c&qR?G%3cT=dG#oQheEn+v*`U7?=XVNCV^lTm z(_J2>l(h$SNMI(XFTV2@cbk0^woxLvW#|KB(JX8G(Bp+LyMIBTCY{vK30j&;@5R2r zxI+baF-tVqXBja$>%oNgXg?ORg#ro~xWfhG!!Co+oE@<~onH0XUGi-O;7bVK%oAh0 z+N#>Xes(Ts*Fl@`I)&RY12^9YYmul#zJN*gE<#*37aNYW{t)CX=}I)6{~h@`h-XP~ z0$jZ3So0YSWi|mp9h{2VQV$1ec%-Zc-(@^#z z$I9xAANyCrpQ4xy29nT&`Ihq6zF-S48tdf0{A-#Eg6!f7O!I`5T46Q?P+8oyG`Z`KUm-EO$rwX|-WX0Lje`b2el4Vq(t=Z!Wj zyIJgRniu5~1duoeQpG}x@Ry_AmJlX*jGoK-OxwKwDfaX$Z+`>ceyEYvRV_8^2)kx; z4WR+}kPZz6Q7Ao7IxQi_~Vj;e8DB5@3j)bH9Xil!ZG`)MIJmO(yW{4CT9K)_X0CqChmsaC})YET@Pgs zSYx-+X@y7H>%*Zm1+B!FzXYSioi#Uin`?j{0sc?l+hH9bC*Qoi2Phm|TBZ{iHJij@(YL@LRM@ zhkS=!tP9W)FD4Tp(IwrPbk>KDn#nU_}7#14q3*X zxg_V^YD)N5SOJj%HK*hb@2y>u>>jdW%FjDQU}`ttVUaAQaU9xuILuyu{b$VFw>5Ht zu+{#7V&SK@!p>W#W25P7W=Dp#=0A=hLBl7RQ_jF)InmBp9j|`bgAW)t{*X!ua>P5I zLAg+Oywlwoc4@3p#>^wOvSG6Wd_L=ynf8!z%I3s48bhkyGJ$dHt_}m0IL@&>#C;zH zhpU7Pwu_V@)eX;v>epx&xp;o#*vfL|ea()XArcHLS~cYy9lH4(=TB&`qY|UaE@kyr zw6E>$mPN5si=4qodKH2HH^5qB#FAg)#;Ba5tjjH`}8DuGV@zeu?{0U*7j@P(?_qU(g=)He;mc!ts(E!6sZMo|i4{#AcY4bC=HU>jM zZn+o>q}V=DitlMsX#T(u=kxfH*QiDzIqj2)hbz8H$9CGGuD>thxCO^6 z?Q3rxwJ$f!tf3(9I#r&Jt@Ra~q-roBJP8({sc|fa2cT}KnE=aur?jj!{l;rD@)9Vc z#s|w?fd6T!=Jlm@8b!7q1xqWK!owY4v41g2?8p=#e3S}JJkl&@Og))*laM|)wP_X@H4Y1QQsTs z@rbIN_kPXJ_N=WKU;Cs{f{x{-vkth94#*zz$+{;i1?#dW`wi zXAd!u6WWJfx65dJy!&Qbi2UJMr^6$jq+ORX{&wq*k`)O8)nZ6s4wJOa zAWIB6Wiv>ux)OwaLxMF=sXF*?fSH=PCEZS9!43Uns~tBEEx)J5V!9SMp#dr)^!oW$ zc7q7f@Kr?S(cBRj@{@=_K<>Ii84KF!3YG{%K)(-a1FS93My%xN#BRxW;K(-vsAxrP z;aTBGPqm@en<8IJ^)s`4`;>WT`wT92R@(Jxy!eg?Acr4D2M%uA^4Mncc{Lw3coNU6 zmnC*z^4NW0d9Fxq=?(hhP6l(WrR&Ap;;*2JN_DEzy8odAs*{9z1HR+Sf3NdzBvwx4 z$%zkbl{`LhTfg-{=dgb3?8|w(GLqb3+K2S5gBkX&51Li&M41;xn5+J}C6>CZZeN$Z z;3VFE^25}`SZvVWI@Dd0&>2ny$#m4SS*w3I@v`tW6VgMBv zaH*2*aJw1({F0g7S6I%B%xhTC!3#HG-V@)KG{oX$RLa?by=7AtT!ofT%|MHD>HH?1 z`KHQ;;3Y0Nf_qL6`ip(xepmVV!k1|E@Su5nCPH!G30e*3=8>>T5lbJ`l(?Af#X?>T6!;PLhS)EYU?_yWux9O!z@R?1Y)J<-Oe6{m3Q9wrlcuDvPTEJov46W+D zxgNI^*K**z*h;FAtg&gp5 z4mpLPt`219I9&qQ#sp9W7bCQ%Z~$cS^3l`)yZIKn+WEh4GUT8tPF9|62}dl#eS|$8 zSb9s*(o&txEbo3tJMJ5C_k&K*7O*4o$>h9&!8adX+C}P=e{mRi1%5!TfqjO#%-MJq z`wV*U11aM%dMXooIFr8%y!2d>N3? z7@04QMH^H=p|XI7wZj<5kI|m>fw>95oFI2=$~sz0c?`Zc%x7JvZ`Ze`H=-Y^vXe!( zGP)%1O|#10WMFo#U;Z87I-*dRu7sf(NK7(@+Fo@5#{I_qa$UO$hE~-v(%y|;i5}9a z&KsQB33KZ^;bS(cIsIMSXt0)n@%7sdJQya*$)~LU_oJ^rT-xaPoXg7M9i&R5P_+4K zA3{j-&Q_V$?p1N(#Fs}s{&SX&zKayN#T1d371yc1v*OhTqyVlkvq-T)jwc%MVEJ~l2rBhjwiaL+UwUsAfQZ$_ zW_ri)?aZ-p;tCqVHjf~~YpP^bUVHxpGY)&Dz^>&)RuX|7IvUA*A`DA*i&)hqKip(goNv!YezR@R)xOtIuG~wOLxf z*xU>ZgkqeS^!r&Q(lJc>*f~@GqUMKA+NYK1$O~eIEln1xJ1>vFzm8l zxXBjjQ!00B%>!3R*Uv$au*v;aP|3Zq-I>ZK{3E>2je2w;bKa79{en%Hm+Y`mW$QzQ z8_s)1Q3MU=l)e$aiRE|#DPq^q=mq#`S56Wv_+U+Zk>^vKM3JuHqzsTT;ETy8$%1bzLDL)>7tHlo4XMU zi+gfk8DGb_YwZk)gbwJ#3gg1btWdKNV73G`ZFNk}glSovV&~3YCk%m~i_FjBr;SyE z#&%JjG_+^c0~=w(v=LIf&3)H*E?8Fp@o%=pA|T=`qCtbDh|hgo+`X`(<^YmS%Xao* z1QyX&|J73)VZoKJ-wknH=ByEea17eINSu-nv)~46&b5f$s`HLm-U|&|SaGI0_Eek2 z5ql30OB#lQ>b47~>FgLuvM*83D&4GDscNHMB0`OhcaR`dH^TMgC+3M|)G*;Zx6%_W z&NN-9xdE(q1!b$N0w|LzM%;|C!1$9W#W?^HXP_2w&TkaB@AEutkke5hvltupd*Rxf zVtRSh9u~8I8Tr=UEMo(u$=tP4fDJLu^huYKoZp4b;3AhRF%bMxb4jAJk@K8gwJGRT zaJ^mq$}745Zj7+fC?C1CT+D}iKY{<365@W7;zbC#(!2P0%}r4ZcX6_bErrzrq&0SE zI63&2FjZwq@|M@^6ndZJ5phgHhRwkCwy!#CR5;r`XGwipm5WfBxRQ|L5@BqF$XkX}803HcIMyYNQxK^PLIN7(QLaD53{(`HClu zKl?oRzs#?qu4i1s?htfvQL=6kfjfR6w_ z2{Rfv`i^^5V$cjI_5+kflv?f67N~rRF*m(VNA*}%`lY}X%okQ`LNHGYWhdk?4 zn&P_qYx4(W=RFX{8g(e#bn5+@Cs1Sy=O`FJ-&NRQLc|Is?Xcc&&!f{bf*6(D zak>FN72Zds50Y#LQYnl^5&SrEWpybLla0vHgteyW>BsrMnzO3V*HN5486oK_VltlO z&8YtB9izqp;{G|`AewDBRu)O-bDXDiqJmbnL&h=}xJ^w!>l4V|^|;DhF2c{t$t)axJiE_)8|2 z%?bnIXG#o3f)wFVjkSK5mIqS!2`NB;wVtxrXhrN-8+1gkTQj71aEXm?K(S|-V8hLm5?wo&U z{QZfYGl&DSGH;xk6(>8Zd+4ap0Cz51oy}X3qE^TrA7Bp zBNTJzwZ%ci?_({Tp7_v7ySIik5l^SDVii@6%K`Yo8S}G#p6OXtSV({mbUBej6QJSX z4xDtmG2JWmC5Kr=Z=0bQhQ<+#F&GH>K4=(~48)d#M(Cmi`pYw^K2d*xGpDs(qY z*F}7#+0vII4QzNIV@Ylieybe(@yv!UAh*TDzf;DQ78!v0i)f$wN$7+8BT>Z|R8;_| zzy0aL;Pk(pBNTYJU;`B~i}c8`8=Ti4rW|(m$xJw~iL+)k0p$J4xMOE@!AaD;hx~S4_2%vk1N_n672#bK)x-$LH(1Hry z@iMP5g3pFzMGefH=m$t@ozXTvK!tIjH#X0GIwgs}TO$4V0ljh$9?OJBBrQnomT-`5 zfS0!#KeP-PJnHFTOmn%y2DxYSUjNtF^QCF5B*Fczw9Mue*z++~Fzu6T+#xL#veEJnI+CCWZ>DVllxk8n zK~9fSQ)$~)Xji$J1ndnn4j!}a6240RxB?5qAOR<~-KYe)9TfQbDG}f|uBOyw@dy?K z;K~eT^nWA9)$4$$z<51jZZtJdnU-%s1(k;%h+n>yQ;31bxeBL7(*U3;BVlyRBOQrnZa3fEsf@T84m<%wh}!Qu z5brXTs07~7H$PTGL?h!R5JI?JKV9?msdsP*Wl}7&E>jm?kM{6}mR@bGzM1{?(t)Zx z5^y@GTgn&`R#{m8hc`0Fq9{3OeOn?+ z#%v;S3IWS#j|s#syy+f8C~4AIrY=&IZ&udkyWC}@@b=x#Q6d$N!TJsCZ*1!gcIl}0 zYGm6_OvrS6IXV+wuHWMj{#|C;*|ag;D5p3VrDV;@ML;f0gcQY2*!{(vna)H0Nj2p8 z#A+tzII$OgE;(%Y$m>pc%eUY2cqMu)ZqW13!06Uid{u^R=h&Ve)7R#Bq`k)+$`2z( zj%-xq7%FGT{Z&YsU&1*oTkvjYR~L74Kj2URb#&hOGdwK$ z8bn^MYpw1Z6iXdoc@UP$?`w^?75bpMkL|o~#iW^LYy>ZS_<^ryFx^%(fudPP+$s_D zc*qo4yhV`AT4TVTOyrdp^@PQ##4L&LDb%)s2C&KlZag);8Wipo5G({3HBj;F9D?*Z z5EC69c{Oa_O!@)-XmH6l^i5aNbYL26@l=QY9Gm50UTED$15g*OzGF6_st>XGh?7wtxmI+;&=i!A};_E9MY zs!Ae>Mkcjn%?gcdtJgYDz<)WK7SAL#h-s9mwMWl52_=+#xMt^0z9UQ>rGI$d(3HpK zDq0U9g~-P$c6nefrIh+tFf+I&e`Ae_+uk%e61Z!|NCWmw{pu*;t3ny~mSdn)q0ui? zHX>9{rp9B7T*`^IQyty?qz{W+0-89aT?0R@5{D1n|4{0ylHU4%^97P(sPRZ(v$gWe zX|6{aDJcTjmhz|q*0O(VGBNPNd`uI!TUA;|GFr1#%9VOQOW!5+#2_JIjl23a zk9XfW5~qk2qyXJ+ zTsG*85uUkJ=YaOdGg)%P`h3WO7v_1v3yQu4IeEjH5|zH`=+M2+ zLfH#In5)VChW0B>@e6}vFpHK)jT|6F6sBWFcJAVQnp8Vq4ae_PP>0^Njb_col$x(l zh@n8^d0q@YuiL5rsjiNqxNJ-UN09g{*B#(WYksUy0Uwh{#r*69Zq`6dn{nK14I!L% zF@6Ci3iAI`1d%PybOk!3bBNzVmxsC_Oa#S#kg4q8m&^TSFUlbHG8$z=AyBa@5ucwM4$$JAPL1;tBfyxFDAl!~`w z^}|zj<4i!RB)~_US())^C^o1;?(FNw0;DZiXd8wVuT@VD*mf)3vqKzIp~GFC%@W&( z(>bc{$OjnDV4x^cv*9gT%I9sGY!iT&BBZ(>C^*!12&fQ5fhWbniYs6Xq*I|e_ZbF$ zGkT?(7f?Nu`JK6X(LmTQS;(_psU9SUVhXK4%R!a50Ch5+m*UA>~-| z70bQJMK7gmw`4d-teYkJIu3m6`A9Us5fJbsOtJfA9xLt%>GxY@am)5K=t$g7P&GGC z=KqADv(ga-Lvkv6%giwEDRM?Pfpy!D_*&gDBDUFc#VE!FT|l<{Rzf?LKi)TETxb{X zLA}82^ZnBx?OHf<3Cg}3X~6IY0~22K@7EieBp^mqdw^hS;#Ah{tJcDB&@UnVO!jBV zCl>CK+h}vL5()263<+Z}xU-3%Lk3VGEQ8?*Cwcuem1<`+CcdhM*;mx6)8>Lw#oAe# zl`9@2Y~9AD8pcS(=vE|gzk*dmenf;9@777GhotE#9a?cI?EEp$ec^`P4dF@?rk%qC zX#;eJP5`dXPoImm&F&vaF|safCq4_P?a1h zAq>q6ivcv8{4NxNf1u`HF}fJ6OF`P#Ya#J%Yaa`zF1pj;40;045b9%j&IRM>aw~Bs zKDeI*%8>SF)UMPmH16ll(FndSlDFH8h)W$4(ESNQOf&)bXMiuy7Cl;x-Ga`@~tqfIBeaVP_W_QQiTY`+LtHw1A0)TOm}w? z9uD2_g+GOtiVNWC^(?tl8Nu%|N@@ON1T*~$se9OB49|p9yf02v;SHtAQr1m=hwY}{ z9r*CCz@ndC6NjwjY@^zys*G3(l-Ee?{!NCzJ+x&9hAo8zyF}0H3XW#o3!Aqbl6n$3 z8FJ0UW=q?&B2xYbp(a03(3BvD1A23lPe1Rw99Rj?pxRh@b!HJ}(mhVvr_Mf3x|9r= z_gX_3Ife4ktMv}}ikRt%>xJoX-KBqLgpJ0t&2>XzKt|4TqA|35uiqvMqZLIKITUZZ z`Ky8LDgV!2oM*44tc+|6+bsxM4s{beCGw92V3>l4k|FDKEYSe+hI3XA9Nex=ISjSZ z2dbavCd;N}Eped^tt&)gXTA%VK%l8bO)4ib*!Mpj_4L1dAFDdaRx5b6nDij{zTNWg%)%cyRkKI&abyEJ~BbtJ-q%pBamEK8ixqSY%RaV%9g8eY7i7P z1Y$eIe0$Jle?ea?<;L?F5PuQ5ezX^MbV9Rf8WiWM2QP6ON;r<-mkbK}y@%~Ss{Xsu zr2^XV4q}bIJMA0+_e^N^u@@x{0B%SN^oTuw{{LFGHUcuQxcBXjVvzVtdjO*hZD-8* z#({=KmKeJvMtNG?Y^QnA2@Ec#e3)g#U`-FW=9?A*x*0@pOrE$MpJL71KyjL)bu`KR z?}<*T3@0X$#1ny9whUY+j?9)o*i2@pz1Xm~Q)g$e89?UWOO0aoK0ROv|L4{Xl7h&m zNobOT3%e?A#*FPJLo_Dw3n+1(dP=lA?vRIHSC~_qx)O$#qeu0+*Hhb_PGlN8<4r0r zUp8<+&X>`qdNcHaYsN^np0SJcz?Y^*WP+7)gTd4%9}2Dex(-`jQ4Un|=lutcf-AEd zQwqsQSv=|RBG104h}W5LyAx8B^W^a+9{*fjl1 zYOLk2`)OlIXYn9A*uXc&4ty4&VkG5Fz(*dii|)aGa^Qa|ss`+QMpGzgu>iOWLNjU^ zg~c?pb|tH)QY|+P!)S&r4e5od=^*rA!z5>uoXS=wq}}zi8=K?=v9YI|0m+@Y-{@j{ z+P3;{WGYG(p$4m$5#aEQkMsv^D*6nIqj;Uvr25+kENgEdQdX&D0O_<~LH1r>B0oXziLrJe5SVor!jsp2LbQ^Pb& zqwJBFTBFlN!G6v6NZ*~eL+n{IOmlWN=R|w1wX4`yUNj{4tw$`iJJDlpj&J!F4gZP1 zlDgch;#TyHig}SJ&GAhl%WDO_*$SmqGz_fVnn=59Zz{_r(@Q&8DRn=B`y`()%NNsx zqW!R01uDtE^8}Kyb-< z?MuI4^bH=E`XYP}ht2J$30SZ8T@N|9$2p@6*9=#`Ol#>h;FJw?eg;NFnCZ|k1oiB! z>?XoRplEz5xqoMektD<}k8~6Cj81wHR0_6+51;~1H%92i4MldX!mpGN{zpdifq(W3 z5Z~wbt$_H#oo5>aULJdJBk4HujwUAbdTuIcez>(n;c}b;55%#(53YC0vUHG<>mp;z z*t*Q4_J1Ca?`;ZDV9u|`@vv6v^Y4#F1Pifk5NEk5D#cR_Xzdf(BpvU^KLeb(klBbEQcd(($hDy4~m}D7M9!A zJKBCQR!6SzBkJ}|sFGCV!VDfS6fHx#G6zIOKQlvjXdXBat*7kk_LsR1FO#Pbq_x zJ4S;N^sA$o=}w0 zGb|A324%hkXva*jRZQjQCLS!YK01?#3Gq(SaBC_PBBS)zXc-87)z`+8R93OiC(tU1 z*0Hw7aeL=qKR5`~gvH{K7SFr`3Ypjm&YDvDk#9J<#|rkyO!IY|%D!?-sesv5hTr3* z#`SU$CH}~ms_U;0Hy@Q#+Tm>)#mNg-^T$$iO25JWK||poqW5WLZkakm)W4k^!{yML zt)1^wpLihg%|gb$JVeZVB$uBh;)e^0joRWA?W@(jolgwP(>b$ zA_&!DbwGiZ%XYM2+;O-EdR4$FU@b3NU#dh8@@iRLocgPR?n|g=#I9aZqhi#T0_GwJ z?7#22>hEY}3Wj?WDyG;lJPh5C$s>436s%Rtn@ynw8kRP)*hKdOR70sGCH>N5QT+9o{i|tHCLOzZY2xMP4;SR$1KkY7H(Z+nB({nRk@L z8;2j7>KKAiXto?dR}kz9OG3T5Lv22EDaqGN5z!(XXaGnaLc;D;t36a0^84)GU|LN~ zH(==d>Pb(#eoXc-ymE|vY6KW%Z#44+qR5F^3d%SDHAatt>!&8RRftQGXjPauaAbUq zd@z#Aw+sh1^9=s}m~()y%QVk7Dr~tP8RW4RLE=o^Cy$hQjGd9@KX9VhEgNu>Eaib% zv!u#N+n-tZ9z5XhE@EQYTWcN5tFcX^pdYn%CN;w~qP=rbNgZW_E>Ig~ka?~+5t3V& zN40`#IRP41QeDfM3dLRb;3bZZ2T_|kGJt7W{eVBKs(`{p!b!kDhbPRuGA6RVJaZ#)@v+0JK)Jr(SZgCu>y}@4W zw`G|%JrWYj{BaSYvv`AbS)Hv_D)ENeJQLnnEu-$gJMoy(v?6+=UvqP5rDZB+_obR| z(TB?Yi&>tC>W-35kesM*NZJq8Ok}CBWb)}3|1D*3;Jlh$_Gpy{p_~@OdFVGX!FC`> zZe>_M`#*jdHmT>M71_%fvB*C1Z3u0=|2KvYu z?FKF!@uwL_eVs~P9jxn?63AF@FLcDXW&Q;Ce*)fItV?9>=lv_j)uZT(%g^4IT1{7> zhkhKB>K{VxnCnkhm{SqSB^ zq#&3jV!dEFqhR;blp+%eBj`_$_9QySq>W8tR6#YZ5I$!q+b~fvb$Z<5ObU{+*)XO? z(!b^~`gV(ZdV_d$^+}zDOw{7p4}H4Ye>xe<2jy+CDgGB=o*XKR($TwLikECv;j=aV z{K;-VyJ`n6{OB9b7}r~1coT(tdB?w)6w_CALF1_%T-|~{!zO*mG%$5}X{Xp^IPDj% zr4=_anuL{S_H*T&?beKW6Mdv0;BbxjJ}-fjRm6Ymd~d_NLH#WMJo;Rrdf{+`bYAX)|}cr!Qav*yndrms_Ae) zPEs$V6Lt=)?Sy^jxZ?SO+|TaBbNpJLnr*YcdMwvy6s%Y7fDv;a zAE?Tc2zHw!$A*ZL5OzuYfj@xi{aC&K`jX(PKkc;G@8hvLc5Og)JzhtW`!)t{-kX`G zZmxyKdaB0bDbfj8?yI{*)0|Ib&Duy4Eu1dlQ7_61PjjVB1+FYI!fHNpdC_y(=9Z^E z9mL1fx#s*}<5FeQMkDd;9J5xa z7&@ZptmGmT*WI-`go;^o(vT~L*kJJK(V#`f48?$w5a73!v4B-and=_@Uh&6UgY%aFw#HTTI!H&lM&kmcD|~Lh`kI?O=&oCRL%%Jpyaodt|!Q>~bvFQY74##G;Ga_8k(E)Ena_ zPK%Z)0|Yo1Tow}apC63=lFz}Zq1d~|JhOE5$Y0SN;n9t!v6kJNQcV1-yNJ|DHMP7Z zA3BqneZh7Q$@{ICwh0yxaXCf+`&S0ezvh8+TPygNo-`1bJ3)U=S3P5L|64Bi*`PZs z`cSqG^Z9{ct+_Os_@5N|cc5QbYy zGY|w`;l6T+D`QR@VcRzo?g|W1d;gJ|R_V3eya>X}`bIFp<|aTUmp!7|$BN#P?C69u zi;fi}x$Qh1f-;0K_5a{(defF-QY=QLIR`OKgs>{&gB=Uu$B)}e_?uuDQse>?DE_>x z{v2uXMbI$c$>RY+In=|^J8X=n8AANILy8K&DX|oIo(l9wa>f)=*57Rwm`vDyIqS;o z2n+oonqVj=XKCi;tRPv?sz2S^VBYIH=NT3nq24SWI3v_#1Or$gprz)B{{-F({hh)r z&*hYV2ISq~6glArPen(|KdFMD(Ynb7O$pBR$`X5pDkRnbm#%Ysh++R- zFCD_5RbRxRblwj`_>b-=s;{B(dBcBe!&g?$e+SxZePyagl~X-FMVWU_<3DN(jBzgs z(K~Dt4*)f_QNX+B^gkfZF(6L-JCUJMROq~R)E zv+sjkY;?x;`k4-dAK%y71hkk*@aZ8+oZx>g$SlX^q=XXGhINidQ_<3-Lgb>_5leo( za>cj5(M80*_^OI)shsP;h=3YYbk-o-t73R(-o(8x4&SEbBH)25ZskQ3+MwE2gq_7n zs{R-r_3^&mni0bPZ5Q&s_v$W{eW?dr@}CQhk8TP9qoZ~l=>Tc^n3Tw;b zJ2Z@K-DqCcF^gvY@P}ur`|`GUuuxqqp+OpGr|tEfe)(*OnL-*I$A`i+5a3m>zKS9K zZ5*+2BsE!viMBc^D~tDbZGOO=r6H@E&><+59{FV*GHf>;sk}?xvb0l7qKqSA zD%8kObwY2dF24X(P|dEpr)|2naeu5a9ul16BC_+1`5Yu3+PFIFAgy!k_4mQa2;F92 zR)Ck6JgaUq=Ouw?$aHj?K3 zXS=l(o=?rg-)*b#n3B(eY`Blegt(u~(s1QegRtuaC|1}c(OBUhgnzIrde=Bbtel={ zf!RJ@G{&9izt!0tb!9;kK&^hlHu$~#szU3m^r(<_LLmFz2jGNDhdD69E|4YgulF}l zY*D{9lB!S#+>KnX2U6A2uWJ)qD>VD$+5CKgjHjOg^W99W{aW$=$77yLTdcr-Zl!w! zd#wb|oGQihG7H>lof-Oph}H?Tz9Yr@%hWrUZbUk`+O?(o0r$BBwYNSKcsqKpG3mjg zP25j4s^l4A6C5Sd54S%Y8~n`Os}`!x0m7h_BW^A4MK0me?_2*$)X~giB#Eji?zq$* z)?}8*Iq52ZU&I!gw>+E#wl-L`ra01UGLhV}Ro1U0ubk~9nt7%6k!OJ{Pw1_oNp!68 zP@IdlM!~#+1&;-boq(1ARjG2u+`nxHGV-arSTzN%j%1By3piLly~w5UD=G$4y40=4 zeaHoWHn-`7zk}Ug8TmF2V72$K6d@r5>Kxd_0{km$MlhrHPU9+L1_oA7eW{~y0(kS0 zrS&-g>PYa-21XrRbTS;rwnjuP&Iec2Xk#22Ip2<-WQ=hh$iNit;8t$0 ztAFd8toI{Vdq?nz3Y#NMF@#o>UdDwe1!7#5D`jc`>Cme9*dQ#Z)k5I#){2h+vVT8y zD-itet}AkIwWEo5*o8*cJYw8$WE`j{{oNb*g~%!}iPzM5_?hANSO&0fSC1#3z%Y~f zVZlZq*8?Q)`YS`%ekIKY`?-XY0g|d!dMKN<8{xCD?&=5pSA!@EC z3}NNb{J_Fa9_ zPg~!!*7k}n;8-W?#jG>nfOJrcnNj{pcJMl!O3Nsre4ygQvQq>88Y@JX08Yr*ApzE8 zS1fmG--C91J{m7MKd^2ZhLJ(#E3~H3!joJkrhG5-t$RUf ze-PYPCh#e=KR~u`NRDtM#(l}YLg z46$pT?r%F~CR0tRb*D}%9g)!-Z@6#L-PWT2mWi6M|xz6r$X0Q$OlTqu143gNf& z{8eB*An#}^0Qd{*%n!og(s4uIIbm1;d0w=JXLWSAysdlN&zs_~J-oKs9^p>fw}mL*F36~R5azE6G> zW_A)kXt){#-+xr}p+tw6tdn9}r5;_%J$|d%BzN}Hg z?L6g6b~O(H;rQ)ol+8(c$;?!!Q+6fEPYbPgZS@+@efCQ$EE;Rp+ge=Be#g6XPC#ZT1G(? zWHf>-uLQMH0E%d%G)pLLz+CLG0MAUARJ{Qn;t3@C1^9QZ(_FQta^HtF- z-*`|8EE2rLiJ+GG97tb`n)s_Uhfqqpq47AkkSbh#jd?^_u+mfJpgs~s^e-~RVt3)S z(yerx5CJm-6KN5svP`$CzmT?BlOkUFX9pYWMPbI}EA%amNG2i#a-q^i>JQiQs>L&h zs&Kw>LVEK0*NdP=XP+K2XFqY`ASXc2nui6^R)~|pmH}<)KYgH`%ZWC3I7$RC*_UGZ z)f=fED?>pNCta71kfD;KpQDjwDY069oVZr@+p2|tZ12D6@lTvKmHv_2Od=XXn?Y%7 z>tBLXn}9kS4YJ@HEd%SMmc2fd-Qil0(*vb8lbZ;EM~P%ziP!{5bViffhX!l$lq6~D zaN1*-{Og83*IHO6x73NgyZX5zm8kOqt@GLmkyS@J_mFidKg!6xyw&nM1zAc}|3`S- z$QQGoHS*E>%ztmi(WfuYYpVQ;^I`0hA5qJV?GL=?vTeLvItv^gMrB5aZmj5>eN(*$O){uHGbcdJfA_^ z^UMKLNIRUk)B=o{RN4QK*>(-6Ix~UQZNeR>UNOg5&+X)B8dp41>aP(Pe>#)CNQZ=# z2xt}UC=nZQ=%iT18%QQP6ac?MtI(9jxb$+b)}#3(uR*C+o&DuC9b83N5He?*+=S{k zDWVWc%cLk`u9d2siaPk3d3CB7G782rNFnIN6$LxTF9Knjw$4+WpIdxi7Of*qX7dsu zn#)ksRA$oLmhakN|-hYZP$jJ;BM9hlE3Y*>_(@;P#R ztt=C?F#>*i*ku*XqtP)7**o z@lIVRVyX|gC;#t^6Be-2e;go`l~~dw2{QoCo9|zYa3UgFF~|l(kI)VO(^DBf!ttQ3 zJw&@3l;pH|FQbL#?rUJ{wQW|mW&`{g=CPu+Ix-IibFgtFwv8){Fw3&{e>-1!+>L&( zjP6Hmae}KR4pTN8wzY+Dclw*$GZXss_VUwYe zoM}?{Mo5{GxDxpa~OY-gryR>ZTzhSIz5^sGl&&vfO??m=w^wB1Z3=px%nRK%K+r zl(2L_h(ti(kO7UUtA;Ri|3W0M8srZ**%#&=R8#>2Lef*0nTXJnb5 zUDaJ2K(ojhQ`;3pN>$Js~~4TA%P7}DT%-jVx?_o?|+p~6h}Nf&;VXm{ocv}guI z>Q;d<;Dp^Z?6Y`dmydRsz=8 zwOSCaUgj*FKYYCP0RWE%}~rj7M#pbv|~y!1ZY{B2=|)FB37 z&ZlAG{H@p*TY>Tsi}i8o#raEfE6t6D^SSz-S>*w4Xc=#|MZ(967Wf!>dunEf9*TV~ zo~HDHkL)sH?{x+=JRZz5(E5ZM z|Eas}T9K>H?i{;;YRFJPjs0NP1#a`(K|$}V;CMhuR-<>hi3=IJMvU+NSu@8 z@c3~CVMq9~9_j;u7x>2r^{!tlyr(mw2yoDO-f$jBF^b{URUbr}Lr$33ty0O@iO;pb zqwjlUNM^uGOQ~5|KfJ)5@m*mb6t=fl7uHP!Gmlxnd47pJP-{EL8-Nc6^mJ)`?bBeu z+)<-|LWMD7$JhQ66aGZ>0E0C4MTvgR(miA&_Rl2Rgc@@bWSEwckGQKb19sG);Gh2; zBif0igNcQT|NOR~a>NfofC%I?Etn4;q0$>{Ei;`J$as|ly)t6bGMqyt?VWxWflZ$)@@rYKkHs~&dOk(?g+hszIf1cXx;jOFB#}93hLIcPiC)< z{~k}Z^g@iEF^_px#BcL&u-2m(Gv!vT6K60e<%-OJ#ulE7F>zNM&aF#}D7XB!+&PbH zm4*i2sVr@vefs&(5yiE1A!c{G)gP!S6`RKnlGEibPQv3x0*iJi^Ou0IA zSVsz>KEEt)q0wKuKC#-}7FNkVgp~}qc6+ASd~|_W3Z3Ql5eg$cqGR3HVi8|5#i*{L zvp6!OkAX;=T@ubIohD%kE`R0Il0i$(i@Lff3{E0xYjL?0Y)}O^UZmqLI5aKBsun5) zD2%ae`b9garbs&@n9uGZJKy;xkln*RM)4s!Y&GOz#db}50?f8NYD)9|ZFg-_yg7Io z85`w3Dy9B~B8~LJTyacZ}s*$jk z{fskcq;SrWcJ!edQ%1J(ZF`OR9TAm9;<_wMETfmk7RVGOAr^cxfZ*T~+fgT&LR!y1)Wry>b3E>LaTyvi8g?KIU_00|roqpQ91_ z?Q(msRHIH=QTL;3Dl5+8@)HX8uYkI(&gp11_h`WzWtB@z+2R59hCHjt_5Lngdbm)f zW1P7IW<^kTno%`g>Gum_bPTY>tn#Id9sWL*J(A03Wr3$@2x?nS%^6C5))Kfa%imbm zaZ1W&9Ep#QYbMEFv7{t390G#m-lX`$l$VAxS`jsZQ)V+OLl%jW52CKP!Z#-3J_h-S zj+Q&|j3t*tu9&!ODwT^A`zioynGj_d1(yzyXhH{3uyib8bWAJm=rgcIgCYtz&hXfM z#6zz$i^80;hQQ7~zfq*v65aC^mmM^Q(!QM6UT1}&D6kkmQ?a3I2LIqZGaBA@xhKSI$Re@957ZL{svXDBvYV}|KsMi>lD<4z^7ltqb7ar2eK2v~hw}OTlPawUG zajuy=1*;hr)$MnZ|FEY7(HX9#0uhns@E4veOyH0WrMVOL318)$!fGg7kTz!3c6lA6 z27}z({KTggQ)RB>J0?+GQYv-W3TOx*`V7q8Mokm;JAS!Qqz7>_)>b-c9V71L`u*oZ zraKaki}x5copk*5qs0WvT{O7mP`{;Jfp{13jXuj|5?-xiu*m@Y`{h)1X9F1RS)bj$ zDda$M$9jM}j8z*UDYeJ;x&kvB(?|V2UXs?R1kks3i$ECcRm;sXV*1)pcI^)(LzMs} z(s;fNs4-p^#JD7gv|NRv&-Ptg<}4Yw>^|-b{hPmO@Bb{iP1GTyh4npq=?-l%IP!Lb zPr_yB8Ek}|IqRA-5x5ZR^Q(w;*DkO`$gTBvLojo2xtuA?7t8v>0)@PMpvJ%s(2QQe z@Kvr`cqL}$h;1~zNVZPwP%#B#_xjFW2h?>pE@MEMd#K$rfdV<}v<}L}GtvnOWA%E} zY(AiQ`M>O+w3w-T?^Q|R9|JZ%b9}n1rrOKz8#_!H(;;HRxroEq-Y|I8JBMbOIhjwaXrm(9(Kzodzee@) zT!>SK^HhM!A_zLgGL(}U=N)4Y@wK*M>H#)a;Y(18hd!H9i-7;(Yr<(Q=@Id84r$+@ z`+GH4@W?AixZYku2cVfE7Va3&M9~5Fg1EOEEw#o-K#)Csn%y3OTJ-@_6H31jUY1%2 z3k67pb>blM7o}%QnF#@xia=WKHQrM+(#TgD?y!q#5wXH5UloA$^YSRB&Z?&tERFWt zjKxxr#lztoxSxVuzil*h1ONe)!0W>jk(AgK1znB^hY_uP~eMJy*Eta$!v%g+UCsQ!0z zqe@!0j-S%E=YqQ&M_}pgdmAV8H@FQyOtYnM=C^u=Q~Y;Oxgw|$Y6cP{kEV4%ikM2N z!GTFxCM1Sxi7(K+u4}-Q1I|k46?x82oYj$K_+H(Qbcm1rZa&{245Y(cH-xR>MB=~R z$FyKM1dF)BVT(O&a21S=2$_|Y(1U^$eK*a}g$ip6t1KKXTwLa{$<}6!xh%;0Ke7xg zjmf!LFl*)?&<^8J&-g0TL%^|WWmx+tu!}!YaX~G+s@$OIS1)`2o;+(q=5|8T$5gWb z+96X}crG0xX}p-n0-R9qM;aoeiZ|)mYj##vJK+*}a3;~p+F2L|ygMR(0mvEw?Jl&V zq{ir-FTG`RBM=86B4l;|Ab-GKD(J=8ViFLw(uA4B^w==#vL?2)h@ty|D&Kikzwo^J zcPb`u5S{qEWzNB;^$)Rn?bvl{M5!0}=ZG&g5wSQ6R^49Q8I*ROs>80@B*^HcYDuP6 zCLt~JK@yw3?4NKTNz&ojN0FXv4P$A~%-7yra1}#}Gg1x(g+mFp$>)y$NOF^Hq3oY> zBU?YO{fVdQSfE?K9p{{+gNC?Cnqur|ce?Xi;GP4~0RNeIhIQSY>1aYf><*@n*)!+1 zOmZG4BDIRUdb6Z@Bq>D%MR8!91IfuCVyUWvK)Y3>m3|GB2jtRER^Ae~S3M*zzAq2@ z14{XM4i~E6GA0} z{@avUfzn*EpHD~~Xu;UZ2Fw5Y4AtS52yM{;nHyc_`9zyH3!l=gAM_>01?uN$alS8g z-?Fq+s4!MU0^asI7>U@R{E7z05%a|r&n+{rGFnKRK zT<8I#!rRkQmy=d)h5!MZQWfk%-6-QW&&5-pf6?9(n`AqMZQW((kMIMZ)=*?oU|UQs zy?x{6MOtuB0i6TR7L3gpiFPSo+*S^t9%1*gIpsFGXIv7S?X$}&4zh#hHKl>-?=F`ki|qR(yC6B+9H*(!>j|bfPbg@BLl_hNh7K4@x6_+{6Bh0-t8921Hhd zsIZl-d+G>)DM=0>c-hAJIAeP*3uJJq{GL8Ek>C^b%1sfRWB~No{=MK)5k~izXG9B7 zguDF6)jaf}^p+_!8QJr)4 z26(Y^M()HdnUO_os)4IY&Y@>?LyQ2z%Zu@4bU@@v54(Z!K%v*E#0hDaR|WVeV9KbT zSiQQtHquv255h8Q8uog3VA{eC;+3)m4d8FDT)@ZvilsW0PWtn*l11(~$l6tH#OK?u!25W!3I z)-QMNKQKa5UoVSGae*Q@ z@P!qJiIpteP*e$OoB+<7?>fb**cah>5i-)EX`%DjIlcTRf-@P}>?Fm-jC_x?>U0PS z*pitZwd9R%v|{N4Y4|PLh5?f{;#fxmy=PzSc0HP)LVQs|wo8+~xy^@fbvg$4Gg@9~ ztjXku4j(%qIauMSUf+fr>Sjdq!FosxNi_Qo-7$7VJhk|`#7N03Tn6cMqn6=-vGvCr z45Kf5^USL83^aQ!^8rg^rcMuEjf}mqqhu{KHt`>CS#%WjRPF@VK z7cs4r68Dpt#Igi>eFHIUfMgtWlZxWR_|Hrb`+i1b8MY!J5Z#O&{AG(@N0+x*$231_ zW*B4sc)JnsztTRqdf;a&cNq9-MZr^m*ney(qAR}*KbMJy=(DDt-ME2k9mj;1+?m^P z6Z*UL)v5B%@T#a0dbA{x-w93P{6s_r@*b0+>otyHN+#et&*jzTtt6yn(GtU%oDD!w zN>rZlD7VOjcQi}$9hOnKuWE{!5lWivGU#9R-E{GuhgZWO$#C>g7@=lnj+bZpkKW`2 z-G`qHeJFfyWGf=WWJrdRAVzI@vBHScs99b0hH?9!PU__hY>iObt&}cLMgQHuG5jqb z88jOPU2Qm9Fwq(4JQBFYesVt1m+SrdY_;d zOlae>(>wTJLugSS0hh(a1)Goad&TkE)D|t^^4l!~@lC^hLg9aGkNhyXQ**(_Qu0u9 z$mYpeVPC4JQq#Yqql7Snz+4|39-b1#3zzu?1Y;&57_kH+qJAInycMPk93Bw+1L^we z-ZllW32vztiH=TNwzUPhA6YiItw<0E8};B%(qU}(cB$GL&+EEax%COLM$muTSnp`5 z^=7g!ZmP*~)4AD(IO0*VRo{J@G&zm^Rrv}H7=VU=gS?)27L04mb-X*$aivbrGea49 z(dJz)S`3JzkEX7)<55h2qdkw5CPjp-d}I$D5^&8=`jDGb)4{P7N^>Qs-kNU)zu<+q zO1hQvVw)~cKGTD+F2*2<(RsC^9l(dl>aE$Q)xj$>qw_0}kzf3HlLIQ6iQbKg2g=Jv zRduyLbjUkGIO(BZ4U17ip*a@X;6Y(*u@(B|sNM5mMAys&s~pad@pwMn7WrV0mBu4^ zFFFW6G5^BUsa2J`P59q}XP#+4|Es$I4b&_;{b+OJ5^0mg~x!e znM#!iOQv&?#+y%z21VoeqxA`QP_}OqWLtq)WrC7n z)6sbNR+ui@@rrB|%W%Cs-_faNVT1`?>0~X>^UoBc9&x$iHr|2}(Hz_!7Nsll?TE>V zYKiaTU3FgXCE|ba^HmS6(+vrz(ofD$a&zA8c1}wbt1ig@=hYY~;(d(W9 zAHc(E&p&|*JAI7xqdg30gNPY;Hk0u5kSc0MFXhFfHXM(47~n*fJ9gWG&pp?jRw034 z8Y!wA3xE@{n+JaDv~i-QL_hX|I7S=}4k*S$!3V8&xkn8_eVW)IL0;ZHM1;LZP)=nL z^j+8jW>wuyK>VX7Qsud4%oVofl+@9IjNu=fRcPaY2I+68YyTPA!*eqdtVP)z+v~6G zgnY#S2>V@qDqdO!eexpH?};Tb2ng-3@aglxoW`vY0JX8gMvJI_8O{!|WavTmHZ@;Sz{}uGeWakVleVh8u#Zv3ww$znaDLver)<)iZ}EMLO?AK5AJ@*{utrEjFg6+!{q6QSWisi;!Gj#T>pd= zqr$h6feiiU?^&pq%k8NS=$8|vDJZG(^4~hN^uu%?@=NDNVGxZ*Ls{KM$L40#WL}AK6o|@7DthY9wHMi(~ap!uhW2C!e}N z!Q3t&S$y~giM=rB8|N&gqxdwgu6s6qTZHweoip8_CrFDzs*y^gPX^E^fAEaYLO^(j zY6u~Pd6|G-i`XPeD?(dyjW&7Lh3G#po3kI87u4$T2#|p~znhAQV&5B6qmsBb`?Ytd zjyVt(>+2lkLyK*7er)kHx@j+qPSyOF8Wid1C!~~Wn-U|0!V5#J;Ka2!rWjN~pJP*i0>k8Nb3Zza1zrRdKt3$#UsBto*92JGbZ!K?lLkeTd)Tp;#T z60h0s)EPYiU!$i8-(c`oX^VI9{3>R(%)oGAA0|4Rq$xXW7x6Y?bSu(kxI$7 ztTkj8`qx6;TG5#n2Wq(9Mm}dVkPt%rHH1g_S$Y8xmuyCl;=*kFszpdD^=#gn<>R@p zfDKqd;piPb`qAMDsvR*j?QD?MX!bnmH#^&YN%6PDQ0BViXgE0(d47~wcJ!qMu(;;M z6-0`|AOzO8s}sFb&tp~p@f2;B-e7?@)iJWIAHBh2Ya=Y>6qNJvBIggW!hv?aHJEMS z*{%pwV#bTB z;cn)e=|;J1wC{F35y1<$U*q~{4DlzTZy^pd_4gira`Zw2Vjy7KiiG1Zj{ ze`tn@$6qL%-th6m8Z)xYm_C~aL@q&}rE34z1}}Y--4xBfCV*HHEoof+Qy>RSOPNiJ z2CN|(JW1zk-Cc3<3e)Wy0=Wel;;OfxBy@LpGf<~|j+;HUR!7_#6Bm}ce6lnxdL_-k zuX-Oo{go1;G65U&2%OAht~^^1KNZL-T6|29za4y8j;E@#zN2>2prmpY`CaD3TIPS%e{< z^>RvTKz4Lly^@JD+HC5&DC-)4i4y9QOHWy;)P5IH~SL5Q#yNE0Lx}$LnqBCJv zVGDQWuXUUcs@tnxih#FLa(I-nFRXuhQKlSS_}UeQJ)`uVKAPXgjBI%g9WsW_7{#34DzJ#4w%lcrx#@?C{NNKU=O$wJVa*;Z$3+P|yRV(6 zut#t`>*K|KX283204YQYr2HIuoqhKU*d1UxT4jf6fNW1TnQkcmrG<88jZ4=}9FF0% zywHze{n)&;TD7wk>s*4+!T!HCnn{#2RzI9&0sz=29)Liq^bD|X?{{y#*Q9Gg;S@K) z{%tU*#&I#FR7<}Gn^1Sqj=G+F>sXN$*_i%8;XJ?7*4Lm&MOXpj*0P)Ji#lfQ`{oGS z&?Ud&_)HdDBYSh5=4lr0&f22E#lB+;_S4q z6k?(0_|OPcSf9DCV~72ja1(1)##!&IVj6l3|7nbO^pMmWMaa5phdLBD(~tmA5N_Pg zGHD^AqgeT;IG07M@3~ZM-Z(IxK|f#5k`MZPJCwTdmy1M5whjod@ouvkQYCQeGwa~= z++Nile13xU)yNV%z<QIvYGWQvSnXK@ee)czW&yy^p!5_k#8 zUtk#2I+oTm&ZQX^1k3^(Q{K4OR~ZWWVkURFs%wpBQy`)1RzK7-16hU~wvvi9a+*TV zmlq{+v0OI~VVw9l_B%jHZ)a*T#bGAf#rGN8HEcns&l3i`?1{=0diO$ z8$?uYijr4<^p;{#Y)T{@UdZ==(*4_pEasgSeScARvO@D0MSNpj<=k{~Ra~=14}cLF zCAu!kH5Tn)r;QOHKhilydZyuQF$%u%YNoSrB&3hlXudyx2Gz3a+}k##rj@9bU-|5Vp5L>5_sYFYMcm3{ zUM0*&LJLDyz_)V1CBWi{8H*=RJPGlD2YKhZ%8KN`mU|OeT0FoQ&kPhgpOx#9LPmRk2ja*>mb$A{1Ya^^ z5Y#B;?gzXz@?>NGM9yjV9n6_7B-{ePi^72bxtH0lI=38Os#| zH_D>+^7swa<1zLi`Qty)$tsI;7!8*RqEDK^Ol?cez~C~|8c$(9$??i!e-Y-rVn^wD zlAf8jq}FOA=lCB%j+EfG)KWI4pXKpaM?dgtRR?YJ0l(Rb-wH-Om_-e(wKyGrLjS!Uo|qePtSDRi=?}r?#sOco6Mu*|aQk_=6ynPY z7!u(+Kinb^s)&6%)NkKD5GzUxl#*OH97A5^4^SO4Gn`hN7oA3XUe%1`b~%o);6aMO z11L;~GcbMbM?c~g&VId;2JS5PK-m<_sv(!%Z;1_tTE0ok_AtVjq8bKw~fR>=MOvi>k)hth@Q8i;P7&V4ulDNZCfLDevFS3#oZmm7VO-gYBrzw@@X2lQbBU=Y3-&$yzTQoHekP` zV7ufZHK^bV*A~yqjT>xv0%pDCCze@nlF-6)z+4^(Ib!4t9KE7p-g|{eZc75?bf9D_ z!7w$~J0In8fqAWqvkg&xj(d@V$Mw5*o#;naw&ol!GMY;&u<|aC z)}3lZdE3#EbZrL_8-Rr>zRSsa>uk>v7Mt}Y6j}tmrvkotexwbZBom&G#8MWfC9hm& zlU`V;8THKp?+o)1lfo^$2hT=&jZgHNneP(C6XqIlb$ujlkJ=ASr7}~oGTOO!s&u9` ziB$u?Mj2|wY!Ck=L##^=^nW-3RH=zBKFANzz72L5#@LvY^ls!(e_!Sf&}UJNe`~{- z=^H+|-&;-~{2}^Vr#TB36f&x-Dz+}Bn|_L)fC--F6BL-OGQ&zy0l(&Acq$)Wq4#!( zROHaGhbH5_y)G^N?8cJQjnN_<`tmEZI*9?R=~gyeO6jTSFtW{P+IevP%>U2L0l)TN(~z{=G| z6J$JPY3(CQKVd}O4iz^DJm?C_Ov{&y%EpVixEph0=G--G2r>9W5<>+VA-p-&Hln&0 zKVb8s*{v_0=zM;#VO;2cDh<^Q0{mj{4+Gk0U@>K4O)5*O0R%=HVL^s zg_nf;dK)8;V^eDo{vgS?+6H_Fk@K(%bcgzsw$KA^rfj`@8ZtBy^v;}Wprzm!3}<3v z#(GtI;Y_&r-flqJ_*~O)a#y9+i1fDnG&mx@FZ{#ul-ml0ND7lcTGdP28%G^VRsuQH z{h3d>w^hi8X*=`JN7=6(1hcAE>l$!*m&e|5JmB(BmAzDFNm*e6Z|Hn$fFjcaKL%Z- ztcgi>x|BQiP;2gdZM=uM-g=I!8W$<-HH<%12aSlo`4K79N_|7I0MFbrWO;LTdICws zjtxXC=N!juj|OxFB|A?YvSW9Md9t0)3G4ue;}fUb9`d&j^qc1eT?~u3Hx#Vn}Yl{@lE?HtQhj|HRcC* z(sL~`#7d`?j_&qfbp&~!)TE`_sviVNg^g!p?8~UDp)n!17HVO_mP4;m7r`Rb7_S$; zJERD84+9YLiL!Kcw%AgVwjSwE=C5+>eHbDxXQO+zHo7#>$44QLB)cMEJ*ob*W#))%+U;amD(r_Zv(q>UhqLhO^F!t2#Y84PX1eGcbH<@|B)Gz@NPKz~;*k zrgqQ{TUDd`KR$c@?Z{;ZF-|kG0xLcEkK%!1Vm4YKg|~_8DguGuB{aqEBRYVCgEV!e zBFV(W63XhdtNZ{Cvp$V{jWfyY${u~mYPr!M8TDZ8<3=`}ki~EI6ylfYeqIbwFTadA?2f|-+H$TK zelNS{907(!r$GgS>b;;q%fB87|03Bh;pJllcV`i^jo%lzSfC<9}OK zS%sS$UdYlDRtg>nJQAtFy-m~l)D!2;U=pl8vE|AXe$}kjL$gA=IB675%rEe)0?vIy zA;n_Iv^6WB>~=cgvk1}}Pl+wIT1#E)VJQ~`=oN=}x|r&;{1tsq)|#b6Dp+?B+VsS5dsr{-DI zLxb@v)N`k``ont?t1c|wXm%`vT0IR5IcVUfwznv{mWYlQN>ldK?ZG}mA%?7{rqZ62 z%;Emz8ZxiQw>Rx%p-po_~+Ic6v+-hJgZBzH#q$6L)W{EF=wAX zylLxkyu&?vrTZAA@IHK#Uou_X|Hl-_2j4E&_dPSqh~Gq-h~9LuxpUUB81cVfRlT%v zoV*pWE!!(;Y@HwJBj;hXgetb_I~L`@uRY*!UQkqfluDuBBw~6tJJQL|{cSrlxKMdSa(O1*()Gfa15-1x(DuJ>^z7`oKJ`YlqMJ`Y zKN8FQ*ju4!p)O#Ws%GF-BVm1~G=_g+r_k52@Oql|EHkQ9Fm)L%$h`YM7mo%jk=4)V z&o>|Dy;C2OKTN(>Gz#Mii3z4Xv%k{XO*8z3{@b)l7rO&%OES^D zL?ML`H)TAIrtyQzLJEaPiprNdezOCE^udZu3m1{6B{KTXo>7Y2dImSH7%zk2fOwIi z!LIXp{?y_c;8X0UL^bUigP+qdB^|H1nRO@QiR;w4PrP|Anng2+>NzHhIUJOWM|Cl zpY^-efycbGUob;moOVJzfMWXi%F4gttFfAW*IFhotQ5OPqK;vjAbky{WKtg7Rq$r? zK4>(-bDvU3A=psw(87Y9@^iOvEE;{#n$*UOl>1BB>GY2#stX)p-5~r3J0EaITDFv|WCx$o`j!7R#Ni#o?57hIf>QrmS^r&y_Y)wWkzsehy9R-G?8s1j+rK{js2z zvVDDMUQv~j@Ed}l8gAt+N!_xl{G!PY`8{0Ky&i`iMqw~Sqj%^gUO9oY?MR{ z5>UkO;5e(ssEc6KO&4n2RZm0;cT5zJSnkUjL>)*0P9|@nEur(Rho(zod%_|%ISaju z6ROrNfpAg2s9*PetI5vG;x$C1a19Ab%$sPj5}}!alm*duu1^QZau?(T|B7`4fd@cb z$qd{3bu2=j&8|-8W6JZU7ZJnuQ%(pIiN>5vZ9fNXNK>t+9;OJVvBq&Sif`KCFKM=H z-nIFn4`(rH1?Rg2VSBdWMFZ|w@PliRw27f}pbPP#I<$m-0c5IM$E)}cF1&eGDq*SlBWuiMzD_@t#VmbY~b9KWIb}yC($AU!PoJ? zEDXSNa0F!xfKSnk?4Y6!ZW7@2V-TlibfsG3`Vu-27(jQNKg0U_^L4Z3PI*MIs)<=N zXKt4d_=Lw$XW#a1j8A^ts=@HVFlP1R2XawdE~(>oFLA=NqziNVU{n$w1I`)-fcX{tYIlx;1EC@}h7m+rr~wJCWfpBd&!ZtFJm zpZLR(AX7MRrS$fziq(C$D1%CT7@?Jq#Y-P*)tFe-#C4+AD%FqKZA-SBa!Daf$sCpQ z4S<~wsVYR5969O!FL`CZoi8g&RLm}-E#A}fWba5U!^i^H#`Byyn+s0Ucy*QhNHH7e zydNs9S&aY80}GZDQhT;fyIb1vcC9!X zkAk!**mSu>$Reg+hxRS!m$|fOI(`B@he(;w$TYlOk0`;7zlHY(ny%o6V5X>ObkJVf{8{YUy7*dZ~GS z7X|?5^@%QF;xG&e?2 zWX5X;|D*u9%LwSbXVZ4FZK9+X@QASOjX1;Y>(BWP&YhT{YNIcyj@DM968HdV&89&> z6x**A@fsyOgAbNan^j-lIDN@!J0)AZuHO9uJ_GFm0j3roUW(4`jD}WWM4IWC)bQK7HoiVCx*DbUC4nGN3JM4fAPQtd25md?)M z?}mLK`DM&rRI=rDU~FEVcFf|;EuC=u&K#T7(!#B99bf2?tK>aGF1&rzzlTkmHLi~A zku4AVhgHGlu~?|R$LDEJYSJZfvio`Kc3KC7i`Y9uswTUwGAsXg=X17)p`(LFD=WVZ zc8W+xw8!cdBP!{nPvqkNOwYOkPOkF#*;4ZEsIi?%@rH%Z6`}P-mF|Kqhm9;0%|ZeN zsCHgTnG?rlxKO|HHn+ExW{-gQXQut-1|^enIgaO4#zLuc1=DBZbrpxjX z{662RhKV-am8PzYgDO;J4{hmBT$6TlmdoyrRjBrEP2H_44o-`r)t(iw(!qKE*!kMi zH{KgM@s?D7S;*G=lme((_ae(ev$+SD5GpCgIG=h>YR4ZguYCy1rJd{OiQaT*m>A9w zCzotJS<6aRCGDZPFc9t_QxsR5+{mjU92wq@11eYB@HZ%W2Q4+xc-r*5Be`_i7I2d! zJ&BADRrEn^k-yf;KL2b9uXFT~blKV8q3`S$#@utdoE<$JIc0f}7i$@px*+}T-NfzP>e+XU}e>C^|X!yvsEkNsohS2fuP+V&aNH&IQheJewf9>Kq8XvzP_c! zU%_jx{EL|H#VlJOrSqX%C%OXs>bx zK$m|p+kk#Z{-BrRj(i8~=?e8t@}4p!No?trW|7rr;!rD(4>#?nahvv{apo^h0;b9- zlIBkyKCUD&^D@(T7RIHF5ji*0CwH)#g1WvB*mwnde>`0h;oWD{X9-40vMfYW?Z09) z{TnEff!B0is77Q}2hr)2arfziIaA!4=|%&ElK^ki~+W(t4avla=; zJe)vdWK0tYqaiQ>~EQIEK z5=Z=Rn)uYqGX3!tDuPe)@d()65905dPjc1z94X#)F~OImF2X@pHUD7x(;WGqGAqr=&UB~^)A|%p2W#^5+YC> z-J#|J?;M=IQ)A;b4{BEo{%#lyxnXDdP60IZVgUouW;X-%INw+7L~ZG*?xIR&UZsAN znst*9D8E2zvJhO1SLDAHkB(4ZgAamMGeAGh)D!vddgwq6E?$imW&tKx#95tx7_9QJ zHCK3J!(k79sXatL=zspidU~HKx*GLw8L`3WwoOevNBEuXI*d^0z}O{spQk7%DPr&2 zeV_E=x?VNO`+>~p9eu_%x4J~XAQ|SWqlCSmDPlb`LWRtZNoLfM^`l1{Ae+Qa^KJSy z_(mqkYFutmG^9vq4z4#mSNdN=y25yN04(%jVvMM>GE47jT@*Gb#euf0y3xFZAxXO< z5YI_Ou>-HnmXPapIz9d!>>r>xM`l4{e65eo=9xsx+^fZbl8{3$e2?Fwi2)R+p-M8?4om=yojUN0gj%4z#8{YpkHC|VJWK}DD-#j9BTYN}?7`sHg^s9gDLRf*ZKHxbrJ+xyQdFM51#7P!Xk3LdlD?7Q z|96;>#KkT?`-R{dPW;_IK9pMzsby$R(UFjh&c2ZLdyU?oNN405MXsZ+*E;GM708P0 zX_LisWzFlhx+Sd$X7L|m#Drnx(!KP{Nas@6pijpRLpn0W}tq5j(w?CFRu#Mi+^}dH` z@~e)Qa8#dhM#rW>M&wqlV2QuHZ|0Dov&m2Vz4w)GB&3L!p43B_G=5*sX`hR9vU!wH zM1B!tHefx}gUt#FhhX6kPVrk{C>uqt&rNOj^!SP3w0ry@`OC@mOBEjjdQ0-Vc~5FMO+(XJHfZF-1t6oBlsE0y|1bDAkqs#&_(Oe~qXC zn{!^|0VS)E%est4v3wptQ=e|@KcRlA7WAA^v(VOBw8o(D#{bgkD6To$y_71tcsm{`MjrJK*Dkin}x$$}6)6=Q<@Wl4|uRoa_)U+LL z=J6Zd)hY+KX2dUo&HFE1Mf80yF ze;Vy?f3$pe;5f8t{qJo#RGUfolk(J2p6<6lFGWb`m7E@k;&h`FKB-_850cL=NO~RE z;hOgmyq=sG>4iOEdb^-gpgd;Y-H^v58^Z@#KH6@@{QNPLx6|Y_O?~peRub6zt zwwc{vKS)Dw+XY~$NhYN@Z2?pQ+f^?gn?8vZ2|!$vrtt14DR{ahG~EL%v>YnFWqB`` zyd-W;O1yO&6p!*hNnFCC&OxKNxHNSccqCizH$2rU?og6l5>|Y^IA>55a>;!g09W8^ zBtuK^4tCe^ts>+q(zs$9{X??;`OB}n!KDC9((cT{HYxQ%bD=Ft#>H^(J1~>we)=5# zOq%u0V*w&vx-6&8zNYqVm!SOQ8)6=&>KEKKbYCrH#XShayMxy&1Gia6^Cdv(VvI9s zg3{WuNg(-@FCim-9I#j7uoeEcJ_V!Z{%mMaR{W`1dzEs7B+IHQu_%+;x05bo zHBCRC@uF$=`*7jGx+gt-q-E0)v-4reY6`Q=nh$OEw_e@8y< zj7Zqt{CG$;-e|`r{!bA{P1-+3b$dlH@LdWo_wj*r_3PABE}XhS%9;yVM@Poza$XL* zyHs{{YtaXrIlXMzjw=31sQE74Pr)mPAmkyh*L2@J-O-7b7G6{8>oYZ)l%J7HcBMw3 z#2f!)AA^3;)^bK(SyC6Y0i!0SPl}pdnwHu_Z;k)%Qt$>i+KL+;Hf*xwslBCsxCYRe z4$V_fP7gWw^Nh9p60&voD^;ob{R=?HVY08z{E62ia~?AKV`gj<+uHI~du8`rlk?RN zab;9({+R_ub?nj7tY)-b^b#LmpNa4N3cz5zUg)FxDe;a~2)lH_aY|fZmL~-<*}s+6 zpSx$woVMehwa&T)xQRVMccC7$_Pw%`xa#aO262THOc7o&jHDlnycbYJ#26PU!$#dD z5}K=~Ze5ZU3^Z`0jsRzX4tD^^qUl=1<=?r8 ziZG(W^IZ}JejEAH7R_U1RDpKp-n=z0TfNXgaX`rzR%0dqhlFvLBqM=!nE_D+FmQe8 zw^ulv2-}$k%@?1gp#=}7D9Nkmmz=b#fBrFVw)IZU;yNe3`a`K9U3&~2+7FT@_)q)Z8FB_kwv#r?Fsw*SK+_i(=WSng+%Hxzc!BkwJiNT zvvStJpu*T)vumb2EM(?72vQv1Z4jTce*7K0$c26om{_~g@8p2=4XRFSv>z=w5c9+9 zqbc4WXPllfiC=ZbSUgWVv9|F8jA9Urr=TCqY)Nwa)Sh~Dy+&f16l zs$5U@&&pIeLrW(l<#heIMUwO8>%Nxjz-1qm!581rtFXwJYRmfb=5o@)AJ;j~XN%g{ zW_cncVvAr{lwjL_%97g;1txp7Ufye3=p=*{t^VEY%Q zDI!Xj9fY#GY0KhNdUy?$4*dO0?9wiH%(MKvY+TS> z#QGyUXv1#-#COPccKc$0_E32~gblCk(4^e7$5~5gXe7kD=6lf}8&AR?UXiT&+vAo4 zo4>tY+aI&1$!Dr!Ek^l8P?lgC$KpRMS@uK{iY@*i9NrQt?5Yl4f5;n?=cZ&5w(YT! zPfHi9zrG0|VP+Nk_&x!SU@ZZwJLnz>iY-B!35~*TnwEBKA|?1>QihzLd502>{5Lqd z?Fvd%tBNL<$`F5c<8)HY2iovPc@RNv7axi;>92X%-b%D8TuyMTX- z>6cdtr|+&Nb!UVw1tkU8_M2mQ3^CPw3B?n++#HNNpwbhr^!gvuy5=N%8r@}KW(3wi zHMn*wHuyg#WuC?@Y*X8WPUhv7pyuu34(1aU^?Vn>4$u(QN}Ql>=87FdT~p)aY#IB0 zVVkx;<;RSkulqj{MaRSt(4jfIJfnty$4>?V{?RBVLE6p1urR;FnNd-dj_(NWw~3Fx zRet0vORGBbA_Ti2QAG7!bOow`IyZ4z%~o=H0SVY4V^Kdu0KroqplYCT;LAmsoy6pX z575>v`QA|#>MQ+Cf6I$z?8I(RnoaGdV5H>wClVJem-RFU4kf4)w7^LISBZGL8YpB~ zdtcLMeLsI@bSuma(0P%e4QM2}fN1yq$DN@Hg!sw4EB($+qGGazm-I!dq37GMV53;F z3!mzRF)<)v08JRqY z(J#(~N#~AANx!yQko@iVR_&egl+~07w_%7c|B6C!Z4Qr$g32|K#D;^_3G{tE5z({5 z)=vr+iOcVHqr3tGqLgocv1N(+OuzJ7eV#Y*mZB@RkTU@vt_wcp!9eT-!fzhlboSu& zNCdr1Ii{+LNSe;lA-etM7{;Fk&Z2+aLHAz}BUIz=1zzd%*}{dzpOZXRZmdym2WK+( zW2F;-IhBU@z_Ef~@V&@@op}s!wp|EP6#t@mEh|yZ61xxU#^ZW{|o&D~H}Gha{RiQ|X$4{L3%GE9sm0(b@k+0{I`+ zs@w=td^Td_MBR>Cwk~Me$uFTGrKH`A5)PJ-2&~xbW2;$lo0|19-OC=MLz;WFOx4FX zG4IFQRG6ox^BpzL45QIn8_by#dAiIcSs>@<8RxH$3tvOqMSc+C`$yQLA04}uw3ov1 zgsk_0hSY9z2O-`&`2&Ty2US=>HLMu;0YTHm`qxPtb$6VY?iO5f5sn!(v$ui-U%7_w z$v)9I7Jezc-|QoZN!}Cc%7eL^=6&N{dTWrx*ye$Zd|q^<5Lg4%z93>4{K!m0zv<8{ z5PnKw>Rx%;Wc2^kFHQqA)5^wZ!oVf(t!^#~Yw7XexC1Vh;7{2YQc?w4UsgATKVOm#X;P5qMkiBa zX1j&F7j6|9VV;cX*iE`}(&8`C?K(+SnX*EPdpmWp)UrGOo9EIe%;*v=l3!BYZ|e5> ztWx`}>cwM_@)0hFbQPWG(l0lVK~QKx!uZYg^UBG2a{|w3lUtjG;Pe}WpP*ONDp3@9 zqeszm4@ykHxlU)>(SMzCzv8kg&FE(oj2gPl)xfH$3xT6sG~<(;OH`LLG;vfoWaT-bZ1Rtx+Iu!*0!I1I*r=n z`_`-am2LKl{=-{+$;Zs!7Y!>B1G3nSR|(^rw+^x{QY?P zA!f`r+Thw=0mag1!pNgSo4(Z=bQssCwAWT+3f)O-OQSlq+-9VTC)%sDIRlww7DSiG zAKa&c^T_GEfH6kCjjtWV0dg`S3qrCk#y8Th3p4W!1UC&?ym`aE%uK2er;kL_-N?V& zq9kw1tIySFc5fG){Ap~Xk9XlYCzM$n1j5nGbmg;<;91LE9hGkeX<8_P%Ac?8*>f{l z-$-d6J3Y00?h1Bitk^~R$;}ct?Myc2-dXfffmyX?%g{{(Kg!au!5*JHQk5t{RSM_Os~edbV3y?KLSk6GH;~f1nB`Fw6pyPdEi`Sm4FUL#?n*z1|5~fd(BXWL&>Y1mWXqrxF|tqp}Rd1_=i7kQdrk$SLXreM$Th?_p!I3$gdwA|{XqO*r$xw|`q93D8@1HCWesn& z&qPf0q)B{=Db@WM1RnQu^%20tqEl+-iS1s&H$pv zokl8ExI9bVbF*Lem+*uPeL;!eR92LcnS~0UV;98yFITgmT_!?#2Y@8T$>1L{d{m z|DzuHa`J2YyXJP6rkgFyS8 zXkLR>F7KaYj~XIn>AK9{1(J-2cqV!zuR8`9^$3PIOY->@a-ro;~i7xtAiA z+X;54^;4&)&+VC<@ly%QCb3dRwX21O_zd!|qNuec)9gD$XVIqwRShm3+HE^Q$4_;C zd6U%s4whv=!o=7TfhiHpOlV4&JAGcmE*AidlpnWcEM>&*+j7BtKRKR@ooK0i2y3@B zVsVSiKQ3?+a;sQch0NLc!dm^<5$x$Etdk7?S>Ut;U-ibEO5Vb*vm1g zMcd(!&e^9J2@&LX4Fj7(r-sJuPFoMw#$_M6kG}(4m7LdYHS{?qr3VgJU3VSj$pQxT z+FO0z2Ca+SW_mW=liI6TbCzwq67+Nk*eL(Sz!35-=q-&Gsf*v`_IqWw|3dDE!WzBa zR;V_e&v8OK#(lYuQq3aXen+%V-q$B$vk)SH>M8oq^>Y~6@>&fxN+*ANCsg+|o|m}p z_h)K>91DZ&-){^yb~{~m3s!o1Ci5ZXiv%IGff*SO19fHse{Y|5^E*wFDXia}1Z4=Z zBJQ~a)M%6%Y)>B5nN~_95xFP%q)0OAH^ZQC*?SVgJ3ghxx%O_UG%6hLe|Yl2w?S;U zW7BQMRhiNnW+K7^;$Nu8zEMVUw3ej5Ai!o>8uEK;le(-;ZT?p|LF-^#p)A9j{wDl) zWyQuy;f$1OHLu1!i(p5S_1ut+;&qs1B^!FEutrs!FpR#ss8S^}BZBhuQkjB$sl3Nc z2FyWTp!V}p=h%#$GC^r#Yl7^9ypL*(`vdn3M!MNl#B0Hmqt_Z1MNL1iOQ|)|wa+dE zu&N|1H(O%o!tpL|M{oB62Hduq_FXjS$LOS=Th-e0f7%pLjAZH0n}arQm}`xe+Sqj0 zM1R;HziYf0Ns0NpD4D!qOrytl*wPckeu*6W{h(}KL(c*!EZjs&{YpRvtLWIuJ~qu` z5rqAzYl@wPz6!&dFE7-p-7f(-)bAwA@yo~ovKA59D1`aC^1Gn#mp1e5vv@y&M7L+&<55tWOrtcPFJ1#u0-$|Um*My zgW+5NuzzUcHDS_8lyPY2g=tI<&ywm^Zk{*mR422fN<2io!mdFc=2UApD{abBf{)32v6~9Ehl8kpm1)l6N#mwy|1a(b-||WpetjWhUOhr zCwbYi%vU2buWeHm82Wuqvlszgx_c0L>Zs*ej85pTPD#{{0$}qt#-aaj%Gi_~dmJNb ziLYos)0OLfZB527@-?I?^q7{}IJ0MKYV}{oJoinPN0iil=dhOeI9g9RMSY}hR1z{j z=_a(9E+oV!X+j!EZ(}ULleTSd^YTbveEw9(x}a-yWpdqB&7Up}6 z*Dui$o_xfzL|d9EAe39fKXv1!DjkRNiAE@t)1s$X-tTSER^I&l%Tw)-vFNeZNIGkx z!5%FGwl}*bd2Fp-@39)&SS) z92%y8H~U=7xL3UX4jm$~SW61thmjV=+;e17GwTr<1{UT2I%L-?@&+ya5d;pt`j>xx z#!r?7@$!h|Ij6-WH})kaX1XlPM(?PA5QuNI)WY=S)GvzaKOP8Ia&96*HP>^)Vkgnw zo4bp)`fJVVyfoj4GHTLnXqmR=WhCP0qI!oOsr_#!XU;QL^1!$O;E@QCFIfL$tayXF zZ!7+Ww<olRHj|{SPW;aIC$UoXeo|!!+!iB^WAo62TRDcB9!2@%kCcix;+nt}py5(g z*8cZ{p5G+T2v#`+iPy#3NlK!v3iB!=VcCs5*Zrzu({j<4#((U)M6xXiagUD|nh~#d zn~pxJ8RATRq`BirqmSe+oK+Dbl9FxEnQl*bAjo<)T>8WRlOfmOHIeq0l7r~PF+WJ^ zLC&^XK-lof>Gq@g+aGL^LBaclQ;VAou(D&)+HTfFd=40?jTu>+fndXer{_+5MqBBL z2bbD|dE23j6+2Td_acXQtrq|+c~y|Z4nkL>Etu7qf~v&c`(HKoBL#Q=usy5Sl>Qyv>qTBk<=I4H&7C#rB|=-&-WtmG*QkHCQ~#d5 zk4C4~4ed^Jr>}j>q$2~oMsCJ`mPae*6~vT&vXP!$`m}iVh4r3lPxIz28zJXVH_eut z-d&f;E7!&R`01~cG#Gqj8kKU%RH7f9F8|&{fAD^z&>xLoXdmGbqv^o@w$6Zm{eV^7 zLkqZmE7deB{15YuOT*3?d<4H*S3;tH>n#guI!wamDfi77j3J&fM;WpMnO)zL8F^ub z!zDAER$ncBpxgO5P*XojCM1v?eSEN2;zwHk8z{89nDHgX=*vhS?G5Jsv>G+^M?ix! zis{Qx>XhN3AYv@6A$g)Vn}wvvzq9mOf$g} zl=C^$!Zfhw+@_lJv4N_*fgX6a`d2BHT*ZDQtIu4=o)VtwwWsF(E*H7cWZ-N^;2x;y z>XUx+w$M`#DgH|%=OQw;z#-c-Vzj8~p+v;ErCW7;3Rfye`b1&cZ-DvN7O`obJzT3i zymW@Nao@{@$L+$r4kadSYUh$~Z%O8>-d+CYEv*}&>00;CC|*5ia^GWv1G#sAKH4|9 zaT{A|(^JX^bciVna$#NxEaHn-BwjyWs^!kFLhP49@fPMD`5gWAeXM)KXWMVQ1o0~x z;&jX~Zj5hf$;%>xiA;=UA!0wB$@L}c%(>4eu$bcheR5JcoKbR;u+nHM>?ORr^XN67 zd)0>LZD8noFi?I1v8bE;pqNAl{h)ET_};>VWP*_CW#|rq`~87WvfH=hTy`NqC`$Yj zFDF@(d@y#oX_N(9ML*sjVtDtL)Bf~e_${`>eTAl@(sIG!v4gYE(J!H_9=jSgqcVQ1 zyTpRPr@C@BX=6Clkv$|x)KUo4GZCZK(HQsRA5nU^VWl$s?Ct(#nOs&q8KEn?k9p4R z98WMxpEZhoCg#r5Vm;`?wr9?!2zU61`ad~_MS1rR0G3HnKUaNE5@$im#cRg5q0#eE zbB9sJf-W`3$~=dgHi_tncN2BGyJ?F;emRGBu88se)GTrH=7KVpG{dh4vaw@zwAh)F zIu>Wo|31<7=j*Yc2LTD?^004u%1?t!i;wD34eKutRCA~5^OXK1VtbIaGOUjalDvDA z@s`p$4%eAVUDf~Ze^#sZ#n~a2ND=*|189e}mCiT-TTR0!rvjhs zFhXSq_QKrDI+BjA8gDf2)Oike=JD$VQhjw~vxDP~G)v2WnR>HHQPiwM8q8<7Fq=&a z75cLRX53Cv47&x69A&4Ac$H;Ut3SK^k>`+edQJX5TA4&+x|g~82uPJq&_;gWt|xTX z9=aMF@uB7#x;-O35VF2lZ0lvQ>S=!S4^z#V$X4(#>^+<(A^*b4Z>NFXHp9-0aPK8B;QBGm==f=hi(m~L*V41+k@$mI2Gtl zU5px7A ziUjauPJqQw01)OBSR6>CjyVUGKoFT?E`lYIL{6BiU?~()045474J3}k+y=`ah;uRb z!Lmr=3e01$9E!LD^BfEUl1yM;gXIwM=1_1*}0a#cN2!fOm z3l9QClJa8#ATSgu2#XW~2a>5{Q9_guWTsfO5M?Bp6BZ-nC5kKn3kXpGlE-0jKwcrp zbFp|Jsz~w*EPjX@io63$7@`iOn7|T;Xdoz7v7{lINQz@D5JU?_@rVV6Xagw$*l@^e z1SKQ33PcA<$&alLd4r+^VQWKlfmG_)x)41Cl_|C%L?21zgl!5jKv4x?BOr!A>Nsp` zh!KK17uz0UjHIr>c7m9os5`J-Af`Z?32YCD8G>dN+XrHfq&db8fLNet9C<;RJGSfOY^I7tv|Ae}l+8pH-cXNr>vu|?83;p9T>P;>z}NQga< zJ`SfC;((yf#VLh2BIzq|Dj@Gr^c^@g5GNqR1Wp6wJ%V8srv>7SWH`p@fP6qPJmT~~ zT!4%K+yRIyf{_t-1mcEd!)?@SvecAU_@e8ioRb@JONIKvs1;N@xUv)fA5w8i{0e!efL+ zp;!a(fY4|lTO1w-GzP(zi^l`~gk-D0aU=2|RIVJc4}{Pa2wlWIx6O zK@(Bzk9c5c5|9Ic4~HfrI2iF&peaZWetdOkDvASyuMJHDa;oF&LemkPruc@?&qz)u zd{bx!iZcKo0nG$*#o=2+vk+Xl`1a6jBv%E#6Ep|K)q(E<%>{B#;Cn#x5ZtTyKG1w5 z_c49|^b3mn5&t6;3FH9~ghLAuJd6a<&_W~+KS3O{2*m>;NP-pvdDRKhpkEQZrUaSL zZ%AGzf?Q|`iZ_4&2`vTk#Ss)k%Mg6I1f|e&BwqzV1@t?LuY;flS^<19LC^rLM7&rf zXn|HCUmO#3K&w$N9tnD&H9&p<;Q+K2!OuuI0rwn5!YOD2P(Yn<4%&zi zFeO}sHX#L^2v?!aD1iV%6to2>7)Q7bZAA#?67ECWkb)J2$Iy0^UdlfnjoBI*E2 z*dRj06hI3bLW(#67-7RGkpKV?HUbol18~4b5u&*O9@rRCv;x2n8%K$D0EA%^K(Psc zIP3>PY!x64n?#Bo13<8!D6vNX7&ZkI2N1zw(+F`!A{E#SQkmNj(yMgrR`a0OD}iCPJE#I2!gFDa}tD2irnPgNT!0 z+dvt0;xyO}LdKLh6Sj+#aU#xz?V)4>h>@^;pllp*G3)>#n@d~@J4DJ>5LdvCP_iAw zHLzo#+yrq0>;xgVO56fFMamr$cfih2a*xD4uyY^?Kr#TkK!6xYMqrmn5I@NT>=r5SM6wF|gOU#*LBZ~T3UMUcuzQ3;F3CRZ0jW?yat!;6 zQs^KFK8HO56(>lpVgC?{t0Z@@C#2#r$s_C;rT9pK21jEB14yyp=$2qcQam_D0hpf@ z0LR<}gGfo?Sga6rQc5_sCB&4J7LHQ@aUx}e<8DF%NP%!XR%je42OQrLnoG(9Cn$hc zkn+O`H=!M*!f*g9Y=TrAPGkvNC6$H~7r>55L2!~y*dr+zPRa@gkip?(mT*Qg6*zeT zoS#e`PO%9Gk!iyzS(Vhubm3H%N~UCnaOwgjCo)qw&8AWS83InrsvJjV4X3kI&Ly*l z(-$aLkU7B_HkCWbT;PnXFDJ-6;7pb;SIK9KEi>lDgg3uIIE=! zBY8BOtw4pJJPyvjsRANTf^)FGQYTM?b6UPKCC`L&6})mH&xLbuz6u~m!g*L#fFRtz-1as%}Dh3InvdhpR|y37IJ@$Lb(qkCx?H9U|m?o;llj)R5iKcl|Jrt1R*M*7a##Fv_^F&*U+P`jrXxFL(>lTTHwd(M?2(f5P9a$GeDo)XW5~E4y zwVEeKEa^h629pRPT^!Tk65~lFDOyHi0_l=gt4K^FU9Qzy62YV^V_I8cGO090=SYN; zu4;9zL^SDItAzdHS`4ZDfjugE=F@xmP>H~?Hq_SFlI1xv>F{Y0sW|PWO0ML2@ z>82I{ucwf1)dHCH4ASi}0Jok)sz@;~)(c2?v`8#@x& z&`oMfv4A%AkY3kXz#Dr>Z`4{a8~vm=$1J#w1Els8D`Vpz=`F2Qv2lp>cCFR2F+h4W zX0>e`C3U3aIyMGL?`U&f8$+abYjeFD!=(4da(x@;Nxqb${*4QyPVLda#zj(B?a}bY z2{&eh^4V+BwO*!rW!^xj%kGsHV^5?b3y5VJAopoc1i+c( z{@N2^Fpm7?*og=@n>>(`4}uWLUupBf5DNM0+I$RzLH=ee9|z%(2UAWmAOiBY+LH>1 zg#2CYNee_l{(kJF4Wc0rrJQm=0P+voQ!a>^{A2AYFT_IrY3!5_l1C1toc2TV$-~;y z0Z0LPr1o?eVk7@Nb~*wnCXc2RfRakcW7-06QYm@7wg8jlAWw`H;F8M8!IU$Mqzdw+ z_KYH_k~~#=#**YBPmi6kCDo8aDQ6u?Zt{%wtSiYw{-yS;H_1!>b?mG!sf`>?Ipdq^Y6DiNtowp=|DbJ6ewqwpuD6j0jE$XtLjQH zDGbW$@e*7Lhmruj#7GfP*61!NQY4hMb(btD3d*|iOSTjZB@uerkpfWG>n^)e%#;mv zm%S+#%Es}_zLY!)7<$E@l23u?t^`sFC`ol!!YMY&rtvG0lwwLUv=js_p`_?a!O&6) zw5}8bbx>gAr8sCg1rEK+fL2ftx~mFkB?Vb`)dF=9LFdd&fKQ&M%;Tu={X zbKNyB)Jxeie$5AMqhO%d{m^y_R(CxB?VzO9T@OQjl&#~}BhYS2I@AGz^-#9y9AH>4 zWqX|i1M^dMj5~0!0ZInc$$$+~cIuo8*brq`oznsfPFCbY}}3sUyz%3QDz zWp7=X7Z#@M8!z+0<|#Pn4L@vwg4f*$z!oW4bvMGW2xb5HjR-7?nhh-n!K0}Mbmd@p z4E11LIR*}*9vUyl!Q-g}=uHMZflAcfRKOFdq`I3HIG9QvziERfQz_6}4mg}j)!lNz z(NtR9EiW8HrH|k8!PBV>=xskdgUZz14!|?1th(D_IF8C5za4>RQ#sHI5Q0GE>MFnp z3YAw^fk7~+{P7ALfmG)YKI+l&hY@5qH4o|np?avtbS^Ne zmwLR;g+ckLC&pbk)BrUfTE##OQcvot6sV#h>Z!Ua3o1Z8Jzix)jZzDs)ecmUdPZ06 zLWQVj>#Dt|F!kJcwGTBH%#o zq1Ea<;LWA9x;hVLvx8PY?!j#?r@5icjLj9a23@mab0w{@uGzBLMQa*wwr#GVd7!U2 zHoIxfx>sDAJ+xQqUh!`B(q0{Z#kaYQ=7qlM-`q}X(Y+ej+(B!tdo{e-M|*Aj)yU>< zS{u|0+R{UNUFQXF>7~6<=f!OC)7~8S;Zc)w?B3dvCnew`HE@gTCh9vOw$9y%yNANb9P5ExaW{ zdw=}3$d)L2H?$3eiKc&`YXf6q=pWX#VK5;2N8@ccOgy~@`Z@!XK>t|xx&o6(|D^79 z3kFR8bo_N2CYjy~eZzr)(?8R_;liNlpVz(N#bD@PjKAT-q|^P-H~p9ldY|si049^( zU-xDhgQI^r{$>P|O&@@^gRlhpSGsmEmO}r!t{sDA(7zdP$6-12LFiiytbqQl?kxpY zLjSJrEelpb|9<=}8&*Rfg1+s*0`wnrZ@aK&`j2&Qd$AV!PvdXOkLfzVX{Ge>x(-a5gFZ3dflDi=2chpU(kkea zx_1<5mGr5)cPwcx`tPxg%}t-tz3WQz(0{3W*PG_0|2qDzFRhIphQ8-d zYp2iZ-V3C4(C6yj3#a+$zm2~aN$aN1Lw%sFJ@nsoKJeCF`X6;Z%vL}B&v74aYtaCG z0ouvfI!J%4>r`wVqW@LbY1tZ}|2^Jm+d4{LgmyW$2I>Fkp0o!-^ndHRyj#Qc|BZL~ zw$9Td(D(gY7wAj6_XArO>C1KRhqp%P|Bb&N*&4-&f^~z^qZv=>yTR!(jHm0nG3g-2 zGZWpo^ms-z>;p!60^?cz2a5DW#&h)_Skl3a=O;d}r6)6DU>`ct;fxjf4_)bK#tZcy zdebqCl@lNO($g6r*hl{K3`VT}qd1Uh^lU~vtOvA>z<5dD1Kvhq ztg7$9Y-2E1PxRonaTp1(j~Uwpj5Yd?7270?we=rcwka6vCO)=p(=ZZYpE$MwjP?3Y zT-(fy4fUURw^oY@?Uf8<{b!c#E(U7iGu!qW1{(Ic zW4oJ?s{h=z-NV>i|G9U&m$7BybKmwh1_t(pe|tLvtN$Xfy@Qcf|3!GakFj;)i^%qF zMmo$7+R?+B`!PHGj2#nx+>QZ82CR>29y#Mo8eXW0>8?4Ibe z?HFZb!ulOMf{Z=-e%Fo=V{d)GcSo49Z=&C~W1fM7ed*t^z`*Oj4D48BWYvEe-VtH! zpZGGeBZ`>~8vtcQGY{woz!@>jgY^TL3=s3s!~iZMo=Jdx#mGot67^pxG7_1j`mZb* zU?zFuD_ce~lLGtNkpX8?^WS|oJByhb*brz}2~(>d0`Dqi>gtCuyBtjY#1L**ITL{Wz}Qv6H0Xa& z?5boM>wmE9axqO4KiGEFFwL+Z9lPAj9Q}{3T^{D)`X9Z!yv!pLKl*mHF)gs4{JYwj zR{c+bT^-Ea`k%tPe9WU0KSg$RGxJ~p(C!}QF?|5MyQr6Wygq>0?Ps2t2;g=PF!N!< zjNOCGllo!B?jh!>`eDoN0Q2<3uxDl+8O=Jc9|dQ|urAb(VlqLjixZ=` z%y?D_Y>bhaz`CR#Q)DKxF4vD)GQq4X6Jxf_WL7C`+>r@qUDc1fGSRGS_2b@54D0&D zxGyuE<$z82Gc#CD{X`%$lT}tf5zfT1ZcI!>GP7Cbupnp;fpt?K1n;4+Zq)}ddl;nAOH6s&s_leRq?RwZo8u?Jw?*H5|jm{||%r@VVC ztcMd*zCC#?7i`+UC!bZNpAPIPU{%*ohxgc6FHcNI_7t;fU?I@n5>~A~1m0W9s;duS z_BvSg6CvE*a+Vu5!`NHFYS7Op_ExeQ>t`%`U96^w8Qb0(mIwBWW3QXltpCNe*TZ_H z{ul3FFYDEbUwnJpSYFt#{=Mz27X7b*y&bI9`d`C)eXQ3eevRzyX0^e>pnW~8*Y#oW zzFyWF^}OFygFH^h3oe%7)tzYuh)<>VVBT_61q* z=;vJfLacY|=e+yEtoJ77eEa5EKG<*meG9Bk{cnMNi>$8t-@^MMtoJ8=i|mVHcf;mE zxM=nV`gt%ehW%mvJO&41e>5?V!^N|EV81hP3G9#czbkNw>`&@{x8T6+PbYr2;gZ?C zus<9)IQujGA1)l4{dxT#UL1z~#l#;zTsqqi`_qrhVE5_&4B#@^{q=u_aX9vu6Msf< z+3W$>0tioFf2Cgl<0lZM12K$?d1stBk9)vw+;05e&^^X;J3H!VH#}>ST{r$va z8(za6g8k*d1MDC4f4T5x_K)>{dGQwZPZNLn@OkV2>~B9lpFOPqJAf}>kJSGi#@pCG zPy8Lh7qdrUi=eC$_LzPVoK?ymuV2JuIoJ~ui@2xhhYCYvfS(${lBg(5Brz;f4x~=_OBEF`m);CVc7rtS?%mu{r>`49qhUK z|An)B?B6E-7s)E>X3xVSp#44U-}Mpj{$BPU^%2Z|Kl{yXuXdjYn@*gwdAtY1>> zA7cMizhv1TVE;X_WZOT=UW6?>_6OPj=$BplL+pR+m%aPL?Eg(H`}WVXBe4Jc`xn?t z`u_s^7un18|AqHQ*#AxZ7ug@hiGoMPW=C_L0-_+K0BTh4S$N6 zoxph(cuJX_$a&8Flr_F9x4k%FgD*!=qyl5I8RZ(U1ca&MJ2__5g#kIv9;V zz~LmopJg5paMl3NDi26FYu(RU4=6b6g3sCyXgG=R=bQ%s&U)avssm=u2KRF<2P~Y8 z!RI;;=)?2!im31~?h;SmvQY&Q2g!d1#2U%N=Vy6yWR*#@Y{!ax&p@&O<@Y9w4sj zP>8eF9oKRw%-I)=>pV2i!NFhbJG8*T11}CATI6K8Uz|M@;p`8-xO6Crn+=bTB}8)% z0Pzq)4ELZr9!mgm4+Z1#gm^9i{t}arz$F4NDG7;OlKUkq0n8-_U$PUDxfJ*+Cjri- z0;{SBXfDmYs)c~z(u1oy3F%x0e03iogUbX~4-+!EEcfbJ0*=cLu3jQ!b2;#YSR#ST z1ri`c3YX_jz!Di;elP(~HB6#_D+Jaki4v~Jy~avZaK*tjcA|zWfvfDJ~l7p)cCgMruTmZhFNvhx)fb~jJCD-U)ZzZ|7rr>%zsfKHYZ*Y>_+#Fy- z70JUr?B38q@^X&^H*}KPxEA=vK2kf^3TzxE6?JfP-5X~~KJL-r#wAiWHxCYuCHHWT z0bmHZmwVg|#*+Qq6G1SZJiyI|Lzv`2?nwZmBoA>bZ?p^N4Q17O-tk`UNJm5mJ-c7 z49IlKxuf=Lzd?f?iSRl>XLMp&r|-n}5gPSx-#;YcSH;N1t1 zRa7(Yfg9OEweTJWk)6~$o(qoZqvrFf0MszGfLHBC%~EZ=mxHJ!YB8?{j*g|3@M-}x zgjUL{bEB~|2d_Se#?#7qZg?t_R>5lkQkAqyUZXqJN^|j=f~j^|4bKDL?4-GQPulKP zG!O3;_vRLwm-lLLb0@8h=Y?LEbw+ zS`|ISd)J-TLJ#xa3#N6_=XpN();{_IuM^lhOkd=6xwp>JBfR&6TbJlj{BC%9EF+r# z0gw)1#PC0Kr(+o){zt)dJR_dp1K-AEB=A25wka8j{7>B5tPC*!)8IBcBbna|-|l3< z`JVyXs~Bkh=kDz-3=IE^;Py^NI^Pf9(Z_g#{{lOP8JYZk_l{Wxj{jwF#}Xr(KLF2& zWfJ&b0T~b`h5xlX1IuLazX@jGnH>Hgd?%AB;C~D3R5B&}@7z1BOa=e@;7&VJ!ykh0 zaxww_55TS}rkVevdshq7!v86_tCN|>55RZ#G4uJu!0usY0e{53dzNYA{~X-C#4P5I z!ZTx8CHyfU6T&LxkGnImEC+ugn2BeV^Mmj`OjZSd64;|;Rr06Yd#o%Me>%9w&Z^;u z;Cr1cH-84$TgCG5e{t_^VR`w#2KRQdirV;L_`W_?JAW3~H_Ynb&$;)_vV8pCg8PJ;;9y;Fatl{$Fmql^x*!9mLz& zqx?m9mXjUi{{v)Iu|xcS-B~T{F#mtStWNekKLX$1$6nws0sDv9i~MEx{#kZ}|6g$b z5<5x|g~*QOL<^oWWJ5SHf~Om@u^f=#naONCCteVZIKbp22%a?@P;wFl&ovyda=?P; zClA;;$$}WfK_>?;SYbF=#X$>RXgJuy!3b7P9_-|#3qXiNeVhzItl`iwCsPpDaA=l; z6TCQiXo-_8h({1&xdg#W1_Fdj5v*z;V7Uyz6D$YM

fGL?%}tSYse6xe~$J2BMX# z5UiUd+PNA*B7)?6l0IB-AXRbAf(;F%7Oq9Gagx-@%@cqTXQBqJ!Xyb?i*fdb)`3ZM-XEYBf;O;Yf@aseDcW%4Qn2m@8gs}vv`s8*g! zfSRP*c{Ktwg68D81*ryF70)Bs+(2vLc?DZ0X`Q?_0R}xI)!;c1wz~>%oi#R;$dNdP}Lxw721UAN%4}f zSg1irVnroFtw912l?rtY60FD})K5zAqH-aCkTOLTLW4o76jch14N|MfB{WS+?V=i? z86k6u+`=4#tV&em5gu-kwTQgJBa^aDQJc_$koSq&g;s-nSkxiRZII83e8QuX@+DEX zFb|=K754~_859t4ukd(-0xR|lPfRNC;sIekLdg^l3Qro8O7W2JRD;qg4hTJ< zqEdv;DS?Zw8gx|>wCGxcu0?_oU7yr-O43CRguYLbA#xh@!;(x<`y*@4p+%MqE{LYx5&JrS0@j5%GyLvbkjaryQsx*WLVZAYHc_&EAxq7n>?~4 z>lU>kEV1$)(dz~aMBXcUqrrle`$ca~TJZ7#Q9HuQln;vDGFX-JA<^3nR;xT9dNgUZ z%SS~Wh+L;UD0;_`TO|*P-fhTjk%vX^P3CsW=S4om(LVWtsMBzCSiUIgYB)M8kBHu% zJh~*05_co=VinQi4-9z_MU42vhCHkSB>reJ53h(9_aKfj6$#>x4abyFOr=2lt>L6nDG`6y zaMG$&h`*maX;*5*Lx@vOB_RI6aH>ja7XR3AszqrL|1^23Q<*0YAWruw^Tor4)5FRF z@kqnzS*1<<^W^CzWwCe^Q4p&t5sw)PAgWUFctZhJUb({&n(fr>adHMx5(YwTovB z=Y~}s;<<)%vnrqXx5;x$s&4T-!WOIU5&v$mLDaqCKN@UUwO{<_qz$hg5HBF?O!c7n zvB9oX4~hS3uv^su@!yknyLwc-h$wWbgW`V-g;na1_}_-Y7Ij$szsbT*^}INODC$!$ zh?fjS!|FxxazoLqIwJmWvS>*iC5b{7$7-H1A;w~eCPwmfV=-0(k~}k2jMv0VqLJsB zngq$S#`8)|qU5>8^HvR5^8D0!yCzu@gS_C>z$Gh;7pgR9$qS7aS~M8R%Bc&Tnsf;W zd9hEEA&E6!9M)t?;un*$jeSGAX#s`T%|QjHZ)#t(OM)Mr!IGD z^CV#8l|F601Y*20tSyivHC~z3+9aE%t}JPbCCSLrSY3%E#aIf_l}ezErC6Oq0-Gwu z>&hi?AEH9NJp%`N3zZ6farTA+Z!ENylYrax4In z9-1o01MyM<@+K2VkP?kIl|Z7D)OgbhfTiTAn|2^sNYOn*JsF=DvkGs4FytF~ z(j!w3I*n~o3-V!~v0Z94J{&f7NOKz>&KiBvqf-x;jNQ^aq$}3cBRytxK}@~UFKE|yJ=KffUI_!g3>d_>MB!6dbY8; z#T1sFd(!DN%}Z^_m-|c$QoHfxVbh|tu<_+tQ$$)c_41M_N>+@li8V*d&KqkW<`~(9 z#u}^{B)d3OgEz;^N|3cobAs%Wu~unLlwEGDwVJ`QD^s<0bF!=yS?4svWmk=LRc5s8 zT4P;{86&$sRo7`wmpPF2edY|A(^x-j&Xkok*3X)8vKv$NOXh4@Ino`QLy+Axx*<6f z*{w!5Hisd*J>|ydaAXz824;>xcE{MD%#q0MHa1vu6ta6$4fY(3tP34}$(L0bn}%}=WYvvLvpF`|%TrBDImNOXq$l=piLBP> zfgCQC)irvshaIx|DG&Z|xy+4hW*)APH5i+fhbv``jm_4>E?Lu5v;A<5%!7QzdDty$ zHoj7I*du$T@s*atUfHWtuXG-6lX;P^_8o4QwHRL=KHMQ|ZG3h1uut~d)T>K}yJc-i zZ|spC+3Q9xRlox+wK-P|IVICQjy=81s9vPCo-PmG15|BNbYOx;~ zm31InokxPQcZ{u7M?$i98(UkBgk|qdwRRqvm-&#d^&MG|bsAqAKC&q5YJ6?>NJRGj z)N4yeqU7DkwpdHF`~zbf#1bR_u(1tm0m(m_YQtONA{>@Z7-pY{=BHvo|Je9;i`63kY3l7xYo0uSeAH*nmk%2s z4OPM{I71e9YJZ$t{(SH+Eoi9rB5(4t#D=xjcw`hnZU; zpESOs%&nA9HNIobb;+lv-m&M_$V13?ow;uLjPc#7T#x*h#&=tCz4BkD-tEk7lZTP- z_2st9XN~U-=XS{F8sD4E^~rymdT%MWTRxBU#UAaE|8Dd_j`qs`X!K!^`sII4`S3>v zNPW#bO`69B*c{C{h$JkYMG$jAGv8&~1SpL7M zuFj+L@(A+%zM~8BCFA?UM;GPGjqlGMjmZC-dVlF?lp+e%9h(=ec*@ib$%|1u-PDcE z11X-F?#AcEE22>!F!K@=&ze3^<|QhgYx=;N2Ua{k{ee9%SrLQ!(3uBUtT268m4{Zm z(DY$T9!9Zp`oqq=bOi|YQD0t$BG&ZLa9*Y&uIZ!MJe=ai>5rE3vK8^Dp4ejq#Y?6h z$T5myRZ|c47(=mox(9!Zqewt~%seJgtTBD8JSI`BZTi@HOrcme{jvR+Mv;j6#CZ%* ztT%m9b!^a8~Nll;59Ww{KqDV3ILXMXzpiRBl;|>LEx)*=ETmeUY#yno3K$t#L9ZgbBr^=N8 z>Idej3Z=pHgYs0R(%AHa^^{9#n*PClszzx>{pdX9R_2&~tUBdU9&Y-v<&;->WctU> zQ*BBM>ZiU_?MkcZr{Pl_%G{=(W>5K)N2h;UI@PVrLj____b88<0+7?a%HvG|>}kL9 z#B>0EdO(?v8fKmzRGu^qD^Cw8Pc;o&PY0Bzr-$vQN0kMr5$EZk@{DPu>U2nXwrQm0 zbXa+AdZhF8ywZmHx$pFX(r)^B`1GQ(u<7U7(-CFS^v_GDqg2JH(b$4$)p^q>q##Ch zp=lIb08(9?9>o{Lt4dH~%z^~fCDWL)AW?O>Y0O#xR$ZAMvlk?*N>Ss^0=Vj`X}qcc zt-97U-co>3U7sHBEJ#;5P!oLx87ik~Vz?kvRn{~yTYyvDn4VZF$X1o3g0W`^s+*=D zSr{@YxPkYtyf@XML*Irhi>J z+pTItg=5e4s9raPA?JEkZ#0Fm=lrTSr^EPj1FCk^Ec4u;>Mhf(^4yT>?WS4lxq#}? z^sN2dsHy`s=R6lwJ*g5`oeQbnZJKL27goJDJ=b||Ugbmm)^~0})oJ=|_}rqZtLeAd za}m}1)4wg9i&A%^=3{Nq>JLow5L=A;!=`zx4W#~PdLD0!SNEWPXWA0fADezx+7i{D zH2rS1fz_W*|8BP>t9wy@IBjtCXQn@@Y-si8O@Fl5FzPR+|LC-(tNo}y`)nENKGUDW zwoG+@)1R|Xkj*cr|6HM4K zdaSfd)ZaBdw%Qfy@24Nz?HctE>My4qQ2${1tIBRx|Jd|bi`}CBY5K2Dd!9Og`n%7b zuO2r2J!~&fk2L)~YqzO?p8k8uUR10eMJ>h_mZ--}i;%)n^?1`Fw$P!Tm|nyemaBuP zf0%_8>Pgc-%EC(ZRMS7!LYI1a`X75?jXH$-*IDRR&zSzLD)gv-Y5KRN(5wD+`rppN zHgy>FzrMnD^{nZC!-XB{xu*Zk7W&k`P5*DHuvA&HkMfGyif3rmq^?%d%s#nGCl%uz|jF`B16 zQE9~>%`>5>tm1f0H2Nu4af0Sq^HZwgM9p)ar*ezIn&(4L6&5FJV$e^Q6~i?v%uiPr zqctyho^CD1XjX=v?kY~#fY8tM7iVZ<&CiS!XKLa+&&(C$G%tppSuW1j#G|9*&J#2* znWK}=Q#7kQ(P`%yn$@A`tn(aA0{U6jd4Xn)`B~L@iDs?m+1&FA&AQODh37SzMD%lI z=K;-n^K;eb&6*9K=UUHOG#f+Db)C=CfYHzQpU>Ao%+HUUFVG};o}W8!(`*VozkI$} zlZ=juyHKJ@F~=lbDAhncF=-bZ8dxYM>q5B(j$XmKP@zGXSEw#jYLK24xffg-RA@!v zg&GYS{X*FVwFqc8WeZ2N3+cgO1jvq+3o?QUG!^qgg{vr2Q(SzSk}cs%}#Ty>f(@QmnSy& zVnDMy6kB+4RFjF0E4vuf>@mkxUkqvXdg59yhBf;_aa|YZH8}K({TCNBc=L-R7Z){I zo)_mXMl}0FFD_q<(q^OM<4U5n2h8zFB{ABAp7^v9koHh0KC2{NOF+NGDoM~1%`d4+ z6161HOSvUrEjjd3VM(%i}1gB+(Rxg)iYdPqIxJv{r*PM`aiK69s64EX)wER#))+LTsfL_D8B+v@Y zYgCscT9Ic>?j?m*99mO&Nu!mZ*OpxZv{Lih>Pu#=%(J%jl0_>It?jy$r&XZW^YZP(H{0}Xua&!9tmyey4zLs(Y^wI|IG)s-RbDGwy~NLTS{r&(|CI%;-Mne!%A&T=vuW;1L|YWvw0tE>SBy@MD~;BjHzy~R#^^41lG92- zx{IOYtkQU02|9&UnxMO6PEnO6>Mnaya!bLwE1{Ia(qvsJ8d_Eg*IhM3t4qj)uowCFpLN;Yn91x?3K2 z+Es?`b_kwzm7}XbBUo1jx;tis>Z(L{*MrEts?gmFAquZ*bd_ji*;PPy-;AujYSumQ zAX~3mbPq$wuB&-E7aG-nHD6a{MvYu8&{ca-b60J;mqV!KtHrt+G&=5DiLTa+PP$gA ztMj1Kt~qq|A#~QYa-AEU%DPseYcQv(u2t$9J*m0ZT)L)EYT>mSod>Q*9UcPnX#(tL%O#;*xc&@-J=k;@cO8(1D#fO zJ*a!foK}53q+rg9|eKk;nKb%6DshPD+t zlJ&jl?PU(Q{xkFTY6n{Xxo3N;1Ec>Uw7tubuJ@yN^gA;2edZk_j!b>OXUCiar~fjv zW7(0dA3$fsISKl&%o#~eivDX)Mw*kM|0a}?<>crG(K}gAf&N?bPL)%l|IV{B*QwBd zAKF>y)aZxMyULt^{s;4}YNuKMqi0vE)1v<=w5!Wml&24%clSH<^~2`fBhCW-h-del z)29D9w0qfEtRF>Z#+8-m$IO{YWu^LYPi9(~Lq8G9%qlC_2hn?2Wfl5K^Bz@MrGCn@ zC%4R{pAPLQEUVFn(0j|u-1-^w-s&=s{uj^Q)-tdD*U;XsvNnAfy|2HlT|aByH&WK2 zpY!aSEA#1p3+-Dj>(NNjG}+e|T_dH~jiPL%6IP1NsFto^@kT|JaOI z-5ApU<-zCP2KD;jWjBKQf6Q6cH$wVm%s>{Wli$ zOXmF}Hx~8Fp8az-BKrSA`Kw)_@5R-batQ-!k$T?VDjs{+6KG<500anf&>?%(O zK&glN%QJx3oI@k!nLu3gp}BG#@Z!v&w zn;ak^mB_j&0M_IXRW~KT+Gb+zO$D%ShFEx010<%B%5DO{`W#aAO*62enbdmI0&JWi zb=}MZz^UZ^oB04Fhdgq#07z;k&)u{En`X$%H;aMfR7%{f5+Egql60#SfHqUoZaDzh z3?=JUIRHj;jJ0~ok}aan$&^ zWro&us|~=U()(|<1K1q;$gK__t(iV|%Lifz}{w7 z>+LYGZ-&)%dmg~0viol@0QemC$n8ZStC>A_I|A&VVK3i~GGwQ6;wqvI2XZ(`6)}c` z&78Cfkm1k_C#xdfKuG1XDiRFD9ImP&(LieE=2n0Wl(F>I$@h z*34_Iz!>N=ysnCL10$8+Uy)&8=I}=Q( zq`RdCU9%+ZuEU_8k!0O1Hvp+p*4+w&AxEmZTWK&hOLOnK45k@r;oTa8IaOA6*KNqj zkyYRI7!EhfTJL%dM`mPQciRk>RC)j1c7ruXK61CiklQSuyX!L?oslo!?Kb44D&p?- z7>?yAlJ4~yjyEgP?)eQTW)xZX1`PSBO4hwW!^s?_>fVsyRI@VoUchjAMp<}o)KHMB zD!UgnoXJsD-wPSeHmh3ig$?IsR9*Mx4YpKu|Gfo+Jx4uqZ_!ZLte(3UF%-?Hm+wUx zi&HgmmC?rYIhv%(7~_RzO;ZY9`wWky?A znQSag)sg`L3LkZy!%8oyRR_b zn=usL*BC2Pjb--%^rjx^f6#7h$vHCe zpu^bOd}Qu{&-mKRk>v;7#p7OBhrPx(nk{J${l+(EELjf+jP0pb*26*L zTRB$M6N>chW^3-lfbr3cweaDnu_HCN>|xOOPEKz1!;taa=G@kYVdHxG<)juNNo&VPUX$>SCYw zK$P>IKm9mVvl+J02X|vUu?jJo5V=r)b}2MRgnd!2|2x z{e6CA-$K)S>xUa_!S&z1l)Ex$C~D2vuVBYVaS|LE~J<@xva@EV0V;!I5e4SnN*L2UwlPA@8 z&u6W%{Pny1*`eO9Gq?RO?~W>SK1$C$*Ic@K?0Zd=YO?V{_jvU}&y8|A@ZpJf*Y16q zjrjPD)hnOYuDeXg`S{jk+jr6Pf9YNBuf9x@enQ)P^pGOs-PQu}C-#&6@8AEc_PuXb z^OF`7`RlI~cyUC?{;XukHuXOnBFZD1T;l$BuK({_N&VEfzulRA?m2BQI;r;T*gxo9 zZ}4tMu@Zof~uG*gyY$p6jN$cUR0FzW!xq4oAe-e7=FZKR)G?hfNpi zvkqehFP=Om1${H(MN58~4J}Xq(thJ5Qil7p*R{~KF5vOTQ-{C%oBHVU$DNri50$3> zIui)&i`%yT`(^^g({_E`H%aEy+m(gLE>T7w2mCU-FZZwOF9+_HCkGE5=(zpW*6C8F z^+fU2R-^b~;L1ICUDfWmNDFMmv$szh3Ayx3&zLu^ zj}AUQ=)YR~$r|y7KL%>L_x<#K{G;x${PTttSE^s=^2WXS)|zX>hl(Zt-Bb0szx{aI z{R7nVuD+hSUv-=K8!wLbU4G$D$wxQx_Fb;lPrYy5hj4wH96f*k)Y-4_#M`ow^p7+8BgNEr@l;ReyZv7hohh1$8{UG z=C2cstUGbG^^5)Ae=~RF;%I2_@dWp(DYR$CcK_=1f97tkkVsomk^UW4B-*}%_G1xr zWp%jh>gpNiZ?T~q!@8NCOO7+E5dRNDK)k;i{I691zc%IlJc?hFVm%KaK5w&FEu+AS zYkO?wkXpwq0QvFeTqJIoYkEyh+NY2}CZDbN>DG?1_*25l*Q~>?ksLCfctv7RPzjJ~ zHg31Ad5EZ7w;X-<(*veY(+S?#&|L=3M$?U`{Fm7IV(qDUkf?741UVDy$9xUApK-Q4 z(gUz89mBPruN=%>z=w03#Sg2jx2=B?G+So`Af_O|62U>sNIqL=2N3Rb9H@GXER2NN%sk60SuGEVaqT+Bss4VMR9#ZrMH+r zZoVVC#=L|)cLlIIO(PMVHDrR8{b@s1lam*-3*(0-t+$YM2^@KwN=x`nqZ11d?Wu>~lT` zu+-xPq=zLiC#sM6KF<1qSO_EfF1-M~b`zy~EXZ~FKl2T@m~_IZg>SiOyBs(>lZWOn zrj#*7l<>GssEdN-Xmoev+>opFPVpfyY%8Qq=%57muu`IrrsBRXYNpUb3Bj)X%`)Aa z{6v48*vGJR;78nozO^a~;p)%bGsm+JgYY}NEmum??T{(fJ~%o~s=?;O?>G?Sa=bM6zp_t2DTJ{QPf6$1N+hFB{%B31Q?oV;V9dOQ_tFDv%e!hG%vN{p;T19JN0M=g!C6sw&AbYx+j+-nX`9b5u+9Rix z*4MN{v}7VI{w7`WO-vr{XB{pV^VPMy9%4At-VGt#4wc7_s3{1V=L`>6r%%eX? z9~P7B`;2ktjls88t1ro|8d%z?;3{GO^U>Aayj(G=X17>u03}VV5i9f_`c7qVEND(H zmVyR1ds%)1<5(}zQ&afCvbnWoP!3S#Nm;5NxbS)Sr7_+rAb)3MrEM%P^7om1hZPIgH2$~Fn4@f zBu_QyrR1j^CE4rU7dz-86HQ=#l>nl#j**Y zV1~)qacfJf94qq!$)tBLenYdoG(Fm?GXbF0n#M&JWDXnIy!q;ZUU)rjuuM>$DX#S%nt!<^@bM zb>L;M>NZs^np%Bd^5rj?k5sfiZ6}^)0&O(1;zer${v&2_j>Ck7hMBU)dfS5zgD}me z>stGM;A>ANnA0CAnfRPu$fi~oSEa|3*fx8ba|0}r`EqZVq89IZnU>xAjKj=3v% zK6-GD)&|&}J)!j41aPqRG@_BIB4(G26#alUQU|o!@%gx%*r#A`!Ck5!j^7_L@H|Kj z8T7R@bu&4MSB)!quOdaSzf;p*&V1}(ay!-9@Z0KlmWWb@_ox?*RHVaRo27Q?1DVnB z_p`Dcr8+hKPQ7-fs(-+_`na@0HacOsP|T^%7`dl(xv(+(2ueZF2RtbTrjgMoFXwQ@ zk<&#>^fo7}8Ew~fkJpTmWRVF+!ZRbL3caDZfO9_M@W82j8Y7FME86);YU_G_r|86@ zpZmU$nI%bQzU(0to=hx|sm9|?oh7R?7 zMP*}uBD4b>w{-Vd_f3|@?@pRxF-?|&R$bJBYF9Ph~xMAAGmd?*8`v(*{xy*Yyc zEK@vf=`R1OlsdM6dSy)bmQ98@D^u7krIBvXV8qE3`jYt|hUb4oHHAOSr_Ri@H+I^T zJ9VuK-l>@83&XkGTV&|4KN8qBEPA0n{X;NqDsl?r4jpocL=R9x2AH+QK96E3k!J+1UVV>SkPujob_TeAPnNv?km+csLz-E z*|QWv#XH1eZYv=hD=@zuXv^({U#ToJEzpKa_%_ilt;w; zbFB{tSjbivA>nL?9a`dld0Z5d?jVc{w9#Rnocd!_*3gIDU;s@e>JnPUtzN9qFmg`7 zY#+a&!8cq{-)JAVb@29{&rx8#lW5Yu%j?vWp?Hn^j`0m1FI6@KLd+r(bVArn7&H%t zUfo1myI>MJX+UBA^r1Vf3%_=98qwSs;<1<2F8QeQV0MS_{PttM`l9H(j5sKHTSvLi z>yItoPh-y<0*L-rj%V#}Qj1W$&$677T0L{p4l3|$aW^W);@KXE0 z1)q-E^W)x-4$C?_r3&_D@!la^odRs*uYyJcwbp}5yVM}N1gPhz4;XWMRkJG|Owg90 z31K9@0@-|S!NIAwcvElwH`Z@$3qsU5tIP^LyNK2cr&ij~E$WZH08K7cD_#3Pz?Evd9!dS}3JQ&k`WoM7Z~;i+1_CEr|`)Md;fZ4ec>b$$4my9Q?ws4HoK^$kO+w=vl;$}uF}TI9V|#CsetnW z72AlaDRlT5+`lgE*In_uSYWV+zllXKf~w^N@YYzPKwNGWO#I~o!u)nxJ!#YAz2MSI z0-pJCPYAJNj;4*pE*V4Xd=Q;tHFzzR#Sa0a+XlWP#U1D;DzY9T1%tZl*Rt$&xMlxZ z2ABA2di(j}GF3;}<=LXZ1k4?CNzl}KQp$!&XRYg&hX}LD-=b+4kU$0q5`s8l=+O|4 z-G|`zC8lfw7qJGAZ&fOUDsC}&3l2to0Jgd33!0?*6@Q?i%T}%^yIO^%m__}mjOd_}% zRkUwtcsWi5_8KJaEFWP0hESt1(u@G!sI24SUjKT>?Mr7E#*I`7xn>`}7i;L_4|8Xl zs?`3gsifn~+HXj`I1Zy^BgW!(zvD6gFO3~aivJYwU-jCVa_2@K&}o25sN;5%qlnL+ zYo*!v-^BwKZs0r^o48|K+N4>5!9?nKz}!?+Ld4fW!7ZGeV{m4YPXjaeGmOdBRWDdFcN%vmsYj zyd4eU+{wGn#q5^nM1cD_0_a%IqE-IE|LbxhT|ylmWQai?VV<~EAWSa_7?imgrs;cx zGIQAQ@1!rD9PiKDq>_%LzSUqR4rphtsJN|$&=I))<`hJAqu?%-ma2oLx=lr5opi-8RQI$3aTpB<;k zer0_g7{J_yvIu2c9}Wri+%W{K>V0#2(Vg0?orSFN+@h4ojA!e-qTXx1v(O=b+4RlA z;{)8$^@Zy=;q{!2XQYUbRg#E?}#WBIl2Y9`58BF%lEzbNlWu0 zJuS)MKWmaA@!G_zXu%(~>1G10!f~V*CyG{1s1KtdLLD&*6EP|0Zn7SWOHN#LsOgVD zjYa?6UaY&JI~>7^ttrO?i2Ah!mejRCdrO*g(-~+mB<4v5wKv>$vG?Dp3nUe6eSwL2z>FgXs>FY+;hJxhEGT{4@%tdeWkdFLnl)M<5Vu5E{$A3{0(?6Wgw@?h-@GP)NZ z!?;4&wHA%kIAQM%_^yED(Fpc_9&w-m$_qXoI6{%RFH!wqml#HsI&7it&ga%_P~wLv z0lEcPq9oT3izuANL3QIJ$2HWZbIxfEK!>OZs76_xq&X@NFqc-c8tja{KvuG|i&hzJV!gxAHx_ zRQNj0^3F;xgLGdtO6QvV-&c}qQgrZsRX)^{pil*Dh4^@i7lcW7jTO2)eN!HVOR5R$ z>TmCLWGWVc!57&&kw2nFhUw!8?at3Bm)?G!2%pVd0gzE=(VG5bdnX6PGa56>r9$2a`G${vCaDIO@o$ zU;@o9404h|Bj&&I2Ag*b;{Ubl{8jr@Az0!cfRQe%c_`D2=i~t=2Frf)H&NNX`nwCS z>JYT!BN}jr;lx@-vR*0fn5TH$kuxGyRk4zrgO4-B$;cYA-1veL1Q2fUPAJb_TMLIL zyRFA#xwW9St}X>&z6!<5R~eAEeK%h0fH2el+Nox4l`CU*gggVAaIk5dDa{qcW4mXK z-C7rXz9{_EvkkuXgK|gc#!N~4QBwQk$^7vY!o@P7NNxOM>t_a??euOt>ImbCM#Ayp z4=$_Ki&{T(I+ZJf?*h5222A47e}X}WrbX#z;d4j?C8idanqD}Oz*4%drc`^@Em2rZ z^$n27wi=ZR$HtdGcQGTag`ue91qYuD!mE!@qJsLVg15YddrU+2n39WhJg-vzUc~CK zd25MlDA!C5ATCvX%Nv+~%-svXgwi33 zTHCT#cXz7SbuPpM4u z<+Cfay}`;@vGPfDyqqB6-F8b*l7js?t3B%b%Tj77CFf{$%8J16c9|0@>yZh(CJ|8` zwTP)y39``wQHVFWAHQaW*0>i`*{aru?9@!gf`EUe1*5J2>zZMyv2tUM2$}}+|Cw>r zm@II+_^>-r+m&4)&6rTR>-LF`-aaAEmmKvajd8n-2wP@V*whB&x?~5K{Z-|*IXL~M z^49Tf);Z>TbM%34Gq*Z_F$saNt);Sb4?9?HLWktNZc(?$k4dxAys&*ui2|FoMS`lo zwY}w3ZBzK6rL@oHQu-x=fVCj_kfXA+iu^46X3{q+r4xtNj-kIoB0Z(8E)+x|*9xda z%`JJ&E<-q+Y9FU=yFt&xD*~MR-@+(mnAV&tw_R0Ov(e}*xF&f0Db)lN?hn`cNhl#7 zO84&{+Jk7^d_HtR^!}_vOB`{MiLVGBFbTBt!7$K2GCwXG)AGH_EG;OX@}(jD;viV^ zQO-$09m`RoC%CT9oog3xwJ6a#F<%@wKi2oQ=e4NOKV}{czyAvj6108jn0SQ8Bo8*c zDhv_*qdG~iN4GE03XFzA!{5oKF`Oza!)^OO{cC8lxM$fG*d#AfkX_{)F ziDp`r_40IQqq{Khbj`;Xtjja1;{qf@D{a9MTK!SsPwq+lF-XR#W1$w(cHp2bs8-kA zdV8bJfL!qpcKS^6K7UTeLB1Iwjsbp4I+ng;(aNWnkIA>I$D5Yc#Rl6Oft^X(YPa%2 z>j5}6_IyC!3Cd{_U|YGj+?5|%*Zf+jSr1a~bw^gb$3wQX;Js(2lg!i3X%v}bd6OID zC_K=0RCU@M)Gt$dB+)j}4z2B^wHdG-TIxtgThhJxGQqh` z+`S%emJgQnTFj6dK_d|Qhb5v;94EQ0od~x+-#z8%a?^2NRX^@BxLvBjrp~GiVF!P| zwByAZEqfP9cb8ym7NnSmw$>zbke8^s#GllyZ@g0Rn|NHG})kX^4y5X6?kSZNa5die( z^it@7437W)?_k=Ui?4B+Tt zKH07={9#W-8$^VfZGz3cGbk&Un<>M|CFtdQ1Nsag57}1<0#u%MWvi%v;N_CQ9u63R zvM%fTgT11TeEZ&DyEHO=@RS67s~V_Vb|_g<&()9fFe3MFySH4Ex0$mM*>5S%&mdpX z(t-XE(*|j`0!lAoX@qKaFq`8YMtuM~5_xFOx{_&g4~Jqd&>r6?0RwP<&Yz&r#6@Ib zZDw4Gf??^aixepr&cI%g zJ)HsL8O|rJH(85K_|<>XMlB<4NYr}&>_Wd9Uh1j8%XEPZpU#OdNmomDfA&YDQB9Tp ziaq6W!ig^OW3zYQ@o?^tv*G?^5?{r+VtbfqG*9spfuEC ztB)#!6i8*dy0FDL^!1Z{3mc%}SMJcSK$ewvAC&{=)2QGLYI~_3ALnB{`K(68t)*gZ13fGo@`fT)gV%Uka@Uj|PBgfxJy2DNh0--eBJk!` zAD&VC)iLkIY0kZ4U}8rw-ZaUN(w>eSNR|&tlXm&YJTC~JL1;OX z*q0B32g!WT96!Adfe*>HRwR*{R(nYiMAUgm^IhOvrw6!gVdY8xJA`yx7Fcfg6PJ#S zawU``10K4FVe_;**Qr|zR4I_>sc~zOonMRjP0V1C;{J@U>ljLg8pA)*@vKxYI0@(8 zO7$doX!phR2|OgdoLV-HY*hC&C3K2E=4jV`u20Bb)qvH)^?t3D4)vl84D*WCC>}jb zA&gbw*w=zi40OJ`WP&5^A;`yu7R+!sA!FjC5gO83|jL7Kl5RFz0i zpOySsh$cFRW_e-Blx_Tpt9d3yEh!xweWTuloEqgB|B*_6|E0O%Sq? zdtO0?=)u>*;sgw-!~YWcBqvS;6!~9~+wryN48V#Ir(&Oy9Fm2F?8yZ#1PQyQ?{;FS zvPx zaFjL~7=d*4>e)X1)l*)fc|)3~QNkp7LK5{o3=HPvP?!fBqg0%tARdxeu?pQ?;oB_$ z!-CocN}b(-_!?usOE?o@*0MZX-ftQt<&J6)+x{;h+Jh%{U@{uhNiuq>&xUf>%L*U- zu#Ez}I)5u@Kqen0Fa>fRF6MI#>^T6-6Y|)|5&GCdl;}AR45O*40ub?=_Sd$p6g7)Y z_43LsW2|?9DYAsRiwlHN>-IrP2#T4%d)h`qs9W`z2OQu&<`;Dw z<_gkPV!NJ5qP;RqJ-mc74X~9B&&@!+5?HO5i^WrhuMSc(tII1{Jt|{gsuIOdc0@9=>$5IoApd676KA)*r>Io8QS?PWCpj(sBT*MW zB6@Fw7-McEo>pr=g*IhVDK(qw@wjLV`T^^+Z@(B;sSthQ={lYd>3_9B=^gJeYuFkm z?mU%RwIdr056sk+*cH(v(pK^+fw_{YwVlJm>qf14C)?CS-QzP$;2g>TN*jL!3+VMD zfDh8x(vw9NaEvU4(L%8t!XY@KBbGuIO?X9fDG)ii_L^kqdP+bpgQ2f?osr{o+h;60 z=TKj^PGg1Z^X{zRUBYb=Fh`5-0SP^DODd zuxl51zwo;P@|^nVwjDxxSQPXPBqe=>-fGSr^dwnHuEOYt-|k5K zK&&kbVfm{=T<^mQoyJ?B(eQ|MyBt$>ZjY?6>{o|xDg5d&sWM+My5=^x4!z%7zV%Mw zR_|n=g3n}r{b8&QBX{?zX~W6IqZ7tdB!DvT_MN1Xw>;|(A+b^VXPG81QNd?n!<>Ae z)~_=D08#RnFDcb(v7Uq>F4h9En9WqT2G+@xrDr)3V?Z{i=bQCXV{^lbW5myeDyEUQ z;gu*C5^`Yn78MR$K22ZnvHInz46jHUl)xMQkYY^8g@+vw^E`l9fr%hpHf##nj2UZA z2c7!!zttZZx${x~to7#N7M-GdrZVlZrq*Vx&(_*Ci)^KhRHqtS#~i&Ibx@#~3o-|y zH#e?{QAdYoG)$hKqoZ^u!gJjjZiE^!UfMASZT8%k=Fd38u-A+RM&#E@#r2&>-wrXw zE*orBaIdRBS%~?ok`W--IzcRX5p){>?t4S@1Gska8g3MgkBt9QhBnt)w!jeN_7i>{ zxxHJW4+3AC;7-O3kQ(q^#`mx<_qU`4HxlpwVp953gd#jKmOoR|?LiY}C_jJ5RwLS3 zBXheQC&p6a6$W;!-rw*;tDCeg5`bMxCR~6=B?suhqKQ@S=WNIFOTgeJlp}>?C+}n)y zf#}csmB<*nVOAylOS3V%?9u=#*MS|}b?1X7PVb-krIQSNa?auv`2jpHcn^0wwW!u( zEcU+en=Z>Uzos-|?;dYFgBEnXU+|Hkg|hHz)k15baoZ3}GemGW!UBT^rvYl}596dh zz$H@9V(=rZs*GIjZ{8AIXTM`CbL7@A^O(sj9`psv2px%dpv1D<9&b@`H>H(2_Y4jv z&W)Vw7}tvn#Oa&9Ty7GUuDY!OPQ8=V(PsOmJ(Q!bS_7F^ZiW+UtU3s!>aEd*GcQpp z3JhK0KqVGkMJ>{X-;|@cn`Z+uk zI!-rJ#72ow^dpvLE)OJ5B39pzwO=X?)xCba+mhW?qSH1IE8={+qbA+!CZFymgv5S^ zZvWLN@s(pQe1@Q!#H561rqDIs#ARbHs0Hk%GS>`?yHBrSeq0dm4{|4ueGw7paVo(# zQq$%3O?FRZ(0OawSS$_<4#}O7f9H}SPHp@3`?BY})N6k*cJxGBamokA8Uw=_j8NSS z7D(Q`nCF!@uMq7kRDNQ-r1?jojsE&J5Jh-&R9S*Q7i6W^j#*=TLjm24=`H`wOL zdada2>w#zSOJd6%{0iIV)W4dOEFuBldxz__`?1mGEA1n#f$iX?R4O}iPht=-usggi z*S3lYJgF}YbqJ3T90lP1sb+-7#C49`;>{RH0zLNO@ZcZvja2Fl{emGB_wyfmw?zQq zg^%b+h9pduWwUQzQ{1!FrcYMhhzrnsiSb8^_ZI!9_|X$+^EAUAT>_wB-6g$N~- za~4zEY7KH~LnB!VfiDN$Qj8UeUcTPO@tX75WJD>?u4R*3#V%nOF@h5|mzA2V?nRF8%hVJtxap?en+dfxn+*@=v@vXvH6 z9y5!2ovWNTi4j()T$t%P3?qwF;6O?cPM`3f_va!GSTZsn3svv(+hz^} zcyzhG4!z;(mZs7@8$pSV@&BuNSq!ENfj^&ceK`S)W9V!iIw)H=`q=V6z^Tvy3Hit* zqka3vZC!az6$C^}z_!9#09=Bu@ptv5;T*r6ZmZpEpf~?Mv;b!3N75otyWvBOYk6$W zUIY|bJ!IuEb=lUcoYhn{p%3r{gXfL(pKn8I&)jC zITp#J!;R(13{id!TN3dBZR)qp;TsMLROIS}XBJ2g*RmFE`^tj`mP3FhpL&Mv*WVAK zYa6%|%;myF4yv~A+>=3Ihd$as@d`?kI}U8-&(r2l9D`UI)6hQWhA3TL%)SOBQxnp` zq|rm|fudepwOz;_oO7Kn`!RR@ z#}WXcD&_^!7SDXjVS71z;@A z<771VH}5?@%kNTg*eb&Keh9FwsJ+7N6JwjpNNF~#4@!xy%`=hzoFQZ3X~mVd+hZnM;8SaH)51UkNY75n1fvh(y|&<5Y-Z5CKl#uqwc z$G`2MGvFf7b_J#>ZoOjAC8?&IH|(*tC1h$?F&2V|0Js0zz2Dv~^{cD|F!;3BwqW5m>L;4o0fhP=s|lW`ie-8)PeXv3I4lQ&iq6>yfq&@|DRr8fUl8N1|Bl=n5^* zDmB^6Merp8i))%j&^yUzFg~#6HPt@-84yh5q0Hq2l@NW@MfBbos`)+zx%3o+ZMC!@ zdp|pDk#@&W(T?%vcgm@_!aZqtQpEbG5aGx0`76*ds+44;wA1a5bL%wB0jLVlc|Rqh z=&eT;B)fx0_nxpgZjig_;P01SSkt*ebE|v62u+%Ax^u-_@I7PgnQd@Bo^#e(#-xd$ znW!3|DQDHNcBykeBwUO3<+D~8_*k+wb|l#LLiDp{69po-?I|WT8CWX-w=FNSrr&4a zfE=F!1lLoeuY>oN4$Gnv<#BfdP{!2T_*y7m1+)_i`DFE2uuUFfpg^ifkaiJ#x2R6$ z!##6eVV71XQpkJozLL@_E%1ek(NEfuAeWtYu=lB=yCo&4R$1#=!N9me$=ASiPbpnQ zfUY-)HzFq^99N%Vsyeks5B@@+qr6>Ut`Oh3lxWtt`HjYBUH{fB&=6ZQ(A@#yGVGSO z!Ji~^SF^65xHWJ|>{jSaI<%R)6%;}O?xdG9D8XC3hK`PIu||yxlY)_ZG_E_-4=dMq z64%c4?Dl~|)PQg1{Fe>b@F_VsJxyaIyzOYxCe9WGDI_^JC#f;_B1?NSOyap(?nukEQ9|EyeF2h!?z|j^4ac^8P!3)MR9omdT%J^Mr3Knv! z%)C^wq%HsTkjHB}%|pL_?#dIyZ855T2CmP^9~$KvCpDJL{EGB8BdIX}$WkZ0=cRG3b;LPO->Rsr==B<-rSXw?BkBs_X}1?J9r}ia zWc|5FtPk$p;r$}B6an6L6uZ~zsN=xo{|l2bVGiOWq&Am& z0rOBHFBy$*0GEEvo-}rAJH+&cFEL1&XStm8(pXA7W4>pbU#>=YGRQIC7g^{Sv=76! z|Jr-%H8cH00wX=)OAChP;^|4%9!(QnGHKGel&So-EYM`4#@$fztcfOPkVTM*{YS%X z@G0R#W?5fk`KRV4d=8(Iq}d9zxv&LOvy6v$-VwqgB*Bpta=8Nvky<4?|5#Ztvx|ow z(wM7w^j;BKL-OP@xx{CrOq!N5hl?G&tZE!p(IABRfEE2gb?u^Wuo$E(2)khGeVv)h z7V|Q!EJ4%e7P_t5xBLl|TT2F|C-Gz$a0T`Cf>nOHs~cec6;Oe5v-64FtNo#dz3?DZ zV9@$?TRJ%tBIvL-Vx^Tw_ANSaM&bWo2!~hh#&PoZKp5pLNA%VAk;M4lUZAap4V%Q( z{JSo-eWe1JlQ=KjYpTZe>cZvW-W}q|i5tAup(rVGp2)oV=;$7@+~JM{FN50*IdTaY zH<8W^*n=*GnGYvRLwc}VoZMHuaHF145X47^k92fjhBm(99^%r`(p?bsI+h0y?mH1? zCXMZYKR6(Rxi=(<$&y1GeSw*RN9nA)o5 zjswqbHTvFVB7*JcTc`V0bU_2R87cqUF^56?5eGop`x?!sIj;WYwZncYRhN@)J5z5P zsyRm9kjH!7VIY`K-<9W+{Rw7BIb~_pj!8r5D~bf?RhS^yI>jUD$w?%$?~Gq?A(+)2 zfNg;DBYYxBCu}1o;i?)oob8TmBz=+QKL}epDE6A*xN~C&$4{?KgE#wO7GVxv9oedg zs1EBhk;_fJF^qtd2kf9<9<0Wf({69(3G0dm5dh}upL6%s7ZoL+R zZAL?_M8rdwxd-o3ozt&xcd-kaymg>G4`*RfYdPX7cQvk7avMu8CuYH>XkHCe#Y3j7 zMG`gZs`YI6_Q*p!NQVJ+sfD>PS^VN6EJ}*F%t&-!75Ap&qrltf0ly5=NZ!|svL_X+ ziC(E}BU`;Kya}prbAh0N_4X1haXU}D^_U}kjzu_V*zF98>BOY%LqTUCBRe&0fQ8s1 ziBs|yf`=%JKnX!;9ii2Vhrd`s(_rU8!*Z(W8C$Va&du#IjENNq*t%%yvZyFsimJj= zt%vSxJKwQ-BG^9?x5gkN`%D1#C;WU>L7k{kvZ0{HR}4Fq@5@?^MbpJUYORAD*nqte znze(_W|lC1tAn`Uf=z`b=rugdAr|nAIw7-RrsPZO(n1<+$t8Hkd^}hdrVbWspQprk z5vFx4&{;R2xLAtFSMGfifLk;3m@~qOOZxV-pC1-W%51y8(y7Vo^U6)hJfsgMYWQ_| z*dt6t&_KQ-kNA zrDrB*1b>TK=GUJtk-;9?ToB2LPQQ>c#N=4&vNZw1twvK(-)*Vyy&4|t&VkRm zF9Q8J`*pB$cPn29^^~SHL*;>4;q3ymmUKphdCN{z8!<{RSjCRMm!4OVRAcH;xDoNR zqB$#uXt!*)?sE3KzoW{2g8%Pf6NWt0qWS+fJ^IuNiT9px`sp816BM(!HWix2Z1x(> z!KUsN)78o&TfRt~h(T22#fHyK>~?-0!4FE~90q|+TIyf6wS|lV$m<7CfjLDhr1U%L zXCmDGuP-Tfq>EVL5koqpe3jqCdkhdv9`G>Vf)D)qMcK<<(d2O<#4tYi7OzE|(MUMo>=WA2jA=Q#*x;Y6fT$_+`PaXBR<>Dzy zoNfl6Qa(9-f3sB|)rq|K7vPpvFeMWcE$yd3I!YW}!oXC{PoC;P?v(L=kQt8SK&;j5 zS;kfKIRCWn@6HlP>*Ma2i(>6vka?=fA<^JW(jd9Ep89SYCQcy-axfk9Uw#Zz&zof7 zyEU{7aoK-9UvrIu$yyniloVQM_<1;2{BOo%H6SQYq39$&@FkM8)emNxQ*c*XIhtT} ztY>bc%Pb{H_;;`R5^GY_(ecSghMg+17`7J=7XE=bCr<)t;%PtXynH8$T~AKQ50`6Y zxh4(mvaVM$9Ok@Hh%zbrcGYSFY$6L5h#NWIyTlZCOSR#5R=p%z(sz4c?rgd7M=B*t zt*deRZ7!3{Fr}m0;LB|(fD;-lLZ}ILn@CjPLlxS=jn<*qGWWb2i7KYbpiNsHaq=~& z*s}HbLYAJiQLirv(*l6p1Cc~#?l$XUc#|*S6@TguN(p*&2bDu%&PT&%F@k5^qx|nl zi6ehb5Z0s}c;fiwGItuLF*_PoM%$GfnkT=e&^X$Urz72m=MM)N^-$$$;z$DiRcbLw|!g|g7Bgxwg0!k zCqwk3bMF4qd_l06fgS~6aTRZVkkjT#1a4vrM&MI37l9>DfY9&IuD|n>kG1#0wfKTU z9;0Y4x|%%(?N)bkFZR&^1*)VKKGq`IGC)7*La*fB7VJ_dPJShe&cUTs7fcV@rPQ*9 zz5#6LMPvEdnD;tzPDyw{pndPId@-&W7e?t%ZQn|kRu0gAR9{O!1If`vixJ}k zmY_(RUW2_MIU3&Kq~hDn9u{O7$T@L@ND{fOC$7@QRG;lY=MpKfC|MZ9NBNkBfMGds zdC?0WwK6&(cMd-*GVY~z6o4=hI{?zgs--s#FC)1wNL!Cyv&{H;`yuj-nn#1u6S3^e z4gb5349B&E#dstN5hyOtJ)L9 zSGm#St8{O7>GsxJg=mbwW>{*o*=ivHqg0`W*G-i02yVt<`f;)VPmdOHGv>#r8;r3_ zyFcJU)^Se-q00jzu45$h5!CmM!wO8cf|0(>Dm2^D&Uy3_qlC(D47*5@YwgUi{7oag zA#(#;(GdW{s8gz%Csvc{tU`fROk?^k*K5k~U5ySw9j3VXf6eXM&1ibikVTd$CcK6S zt0t-rU9=n`zFL6u#GCEilSY)5avg6^kc)iV`cSHA##i?7^r3O^o4_YR0j+|_gL7Ch z75DoF`%eL`k>t*|L?)$Qce`&J48x%VS$VQe-_=-*)XT!CPW$+!%vFISP+e9l5&YTK zdtE4nI%rXW`>!QDM0``yK>IJvNe@=}mM}WXc~}bMLRA~^U}&QsL5Prr|LroQ*m8w9 zkaMD9y+!95-g>5V-nlc*CBn%3MhuX_1H5RmV-howH=J9J^NOWaoCsL1eZ1hngtmqF z_L=x%KKjwmI8^kEBYUR&K}eJ9RX~@?3I{1z7vCR(seIq-mT{I1!hh-C2^`%1$Q-LN zl1b7?h8mvgd>fZ#$1{gvbF>+0hWa{?KQxvM>%xOBlA8%egr}5G42t&}n0{i!hm1?N zx$y^8mYg;0C`OlGZjb7xF_&bgAkl-8jZ#p3DUTPGl}YNFm%ngJe8@EwJUofj&{k1B zr6*GeFfiYYpFX9b{pLMZq(QFC*YfmW$_#e|Fsr4ci#RrKl&?cfzTLEKo^-Oc1DXqOL-VL) z-ri??sIiiibzUMSc?&2`dd7osad%2>Q}UpGsKVlF?Q%DHHdha8F?MP+fmLMXidYS4 zt`XiXTV*SpF~86s39c|o6YYP+CCt6ku1k+!K?j9Ob9{q!!adiVcV{&WewxFtdsPE~aoCG{&KKl+g9%b)AlRK}0Fw_Ac+z{YC&VA}N4IT&)TS zm9Z*e%j6Sdnx1Ps=E>)^iq)(u1077Q6IU9jC(0rwpL3WhmqcFUZWFi<}u)Z1* z5PV$lx{(OA5|K4P_pxXZO}H|5y1G#FBlGNGOSDdGUMOrymtqq2z3euP1U2RCg9&7a zSSkUP4B0m))`C7U3%YztYJ@TO$V(wSd4F%2H@dXD*D*5O1F7p-Dmp&9<} ztyfMn1@E1IFc_M8>^UM`S>OpaklmvMG?RX;507ON+3q$}@jV^qt$zcxH7@&NHc2{} z)JTt2nejq3zC&Q<4StA)NdS9iOr|7hjisL=>7~k$<5_?svOKAxGDJx4F8jtl_r+U= z{&xyy(IbsWRj+YST+`sjQP?F3z@zX%2ZZ>~-FCwX#cm|CBX{#QUyL+Fh9Du{QA7CC zs?$r`Fh5y|0og-AR;!;+l*w@z>gi(xOd5dtgNoBxhM>SC+Nr~jr&@5R{Ymfw0aN(s`=9{NNY+O*=6i8&nCbI>0w}x8hk8f`dGPMYjrYsW7bqVh%3f9d#AyL zL~NqYeWf*x5~FyFVFi09M<)Dkg<=Ox2{{!Qg|`_bS&Z-+3OeGv-<%4b&L#?$Y`;|U zTx+LU5iKr=Xh~eTYoPJ**zgov9mkfBHW?iVlM2(k=}8gMnzHM0(0xO!&Yns zkj5HbG=tj7N~yIF#YV&p69V(WoT(%lsW~lc0Q1W+n%b$2Z=(t27Fx#2IJdDxF3+3YV?V~pl z>y996VtMiaDoRhRL=RdbbvHn-`-_8&&Rg{T)Cw#j;iPGv_K)eXpLBA^Si(&#P4Xq4 zoj1NVle(yKSNB!ybekk6iTrNJG=7ZBuA8k;yiJyNBn~o4Z@_xW`EVStgy6Ydu``gK zeOLdoU9<#5nD+Iaqs*=5L9B}NKOjW&^njVJF@V4Kx9n6%@b1Jv^=rE22d|IM0q_ME zjwok965dcpxrY@$VUg;=ato2+)EmhV73&7D{r+}|U^KNw!!JTsNE9-6_1q+hSDkrh zkVeM`reWdJ+@@Rv~x^IzPYxFeSHPCtu!~vw7N1x2{640F4W0TNV~h zLwe?x<^qYqcRL|*Xgm_*YD>>SipJ7f69_u}PEPNWo4t3VZ%Tww_L`a5tq*Ash@#eg zVt6uNN_P^}Xnn+q(d9|KZM$kolZc=?wn@3hpJ0PDZjL8C5AGJoBX_GDK-|QH`IC=8 zl3gd$@T`mq30z2s8Y}E*GZ6l2JZbF(W>FrvR>OPJk@l)Fd?bn}Mb!|hBe?y+nbcV^ zF-wy0J{ht)!MX?x8%3Vk4y6%3Fl@41LE(JBVZ4+Ss1=WF6Jdsu9Fvw1#LTo8XSISDb*nK0c50I4#lg8Gh+*dEmNXfJ%h>JS zjXf7gam|QS&pWv5KF{2h>i*eyP1T^d6}e$UGN6I>d7TcoIZru3G}h>}>idL;M_(Wy z2G6-C&^aeG>gBvnENuVU7Y3rJc5Vkx5#yHyc*~*fCOy<9V~#h81w*M{L~5Wc9LI_R zUgr%%im98xW*Pme0=GXROK*fCXQ1fJ9BCPnV;{%sT4~qQ-LbsFgDtp!cKtUy)r>Jey&VKM6PYkfm~! zrt^8x2f184u^m#FbY>}Hp@I40WL+GZ7-(xY!2QGsgVYCt|8X2ff$u_T*DYnh@WlnW zuHa@iA^|{Ge!Mzwi6kfMd_F*|EqW-)$WVJhDTFNzW1Dsxnp)osxC!XYMu>&5t(0?& zYWX{pdSaEkdldW@9?Fxw@sfd>+<>PJoi><#N0bOvF`J6C_Se_xF2o|D4%8vHoRvS<=SER#R!li zHsv6D;+ zvU!)a>d3ZV-qM`Teyo~*QkRt&hj;xSBG{QpR~SGjJQLD=6x<;psM6mmM7WL87Mx7F zE0$h2P!6VI?yNsl+iEIu^C~6GyNFJ=1HTVJSk7AxDc!@5@|$5$?@PmeopXhtrX8U$ zHg>2U|y++pi0F~pD~&+ih(A)${b25z|xP8Yr+w%^cDeDNX-Eerc^{TV;X$Mb!6!5Wny~$a&?Y9~%Nu^g z7|{<{lgheXRV?m((2c9tohxkh!f0 z*Y3sLypyF|vpOg)eg({=kl}Q+?wPk|5AZnTG=D=N&mPV^QiX?ydO)Q=&ekXxMWYgq z7ste^gxHs~6JX%XyGvFc8n0uZm7~M|in{+zY2-EVjP)Rk{QW>ltwPj2G>_0* zGh%3z9oE8{_?J>5B$n5Ap};r4%o_e1_4+a>o$x0tKch;S8R&Lpe2vN8nL*9Ds-~aM(GP=>wEg`|AkeL1i%#(O^q^dji5dpP(XX^bF1!(odgV2lOgEqJ2P(-Xj@xLo|(-#|JqX zYMVCa0&+}&uxv0$LQYGh8;M}a`q%z_AY&we46tL))njZoN-KNLOVb#vfsS8H+CqYc zu;zSZtR!&#zX=mabJ$?H=Af!+5uVgGBYGmXrdqs~K8oZL(@f~|Np?BP;hTQjCPPpe z0YTO87Np$}z7ubJ+n!R%eK#=PE(bcLbDxd5?(J2P=CMLy8;ecBIiuy0m{%I;?*9eY}wIjn7s}A znvgFc{P&5I)9BFa3MxC~#nr8_qderqO^&y2f6Z_oOqNvi|A8mw=5Y%VPelFT{Bs}j#TE7+s z&ohnYk(^AwxZ*>Tdxdof7LgZUw4G*0pn6yJ*>CTg)S+9OA3rCn0>G0i;vZzOm(0#a zK*5nbrW6VRmheq);#j_y*+e{sQUZY@Ki>$mM8L+K2jx&X%H%a)uxFFdbok4Y-T-Wa z$9YaNyUOuTb*7FWz$DRWcwV|QSq~o&wh0>kNmr0S($3_ma56Bi6aTgyaI+hlq}ger;tQ2RIOHf!@Iz~O-psb^MRX&(Vo!0{&(GgxXZ`(eb?Vl zyJjtq47is&95|axF0hkXQme$q${YYZH;%$M_onRcaNXs=dW~k1>zfZg+<*QzU-x*v zS6?9nfrZ(5HY$TJM%`}z#&?w|Y;;?mK?RZKXYil^y=dC5QUqztH@HDm?zg3Gs(byl zS>c@f2ztj1)SAO{v}(uY8P`moEf6+NFU8zSIQ4gUdSiOMx*em za&olVKd;zf*#fAN=c6RyFls`yaG^uz>DPz5ri2}S8?*^_d|O6xA#gq^?syw#Y{t#X z&ldV!^4N!St0bra;Fpo5;M00&E|4;mgwiq(ePhz*))NW`XpJ_paQoKE+Z5kFW_)Q= zmNd-IHPkjz;tIJ+4J@87F^ul~bAeVeWIr*=Bnw{#|D`LjW(YyJP*U^JAMHLuj4h|k zDN4HFudqn_p$Va+o_3C3-PBLH-yPAss~2f99db}h(*C4y)Nk|2DDB?JGfd{%n}5W- zE5UmCgqQ)gs4-&3L(C$f9#**aI{dQyXL4&a+RTGLNJ9-f&D{&MOI`}wKHXaCjTpeB zoV+8j7O_oA>;J~xgqT$2b4cA!#DFJAgKHBnviH_XBHjQnuYOk>GNR#5khHfCm_F#8 z(l&hEW8`0R{*QJId0|N~sZGlVEYBheHyKe`x)(Us>74^Ks?nqHF~22m=E(TBpUHx2SMook4Kn=d%HnYe}+d>8f}XL+kuAA?hWY z0hvmGkZgZ1aS6ziYP~9Te894A$)HTHZJ;x!kGRjJfaX*aoTVUgyFfdr9pjy^-Bkc{ z7gH58 zabf~|we&xVXHE!NAG)Z~0pw{->MJ&1R-_bJdG2tTA#Px{KT*i-`kT?_wm4I8HApG!5d+6@=}b8XlB z;SLiLr{it@H;^eV*M~>6pz6|!SvLJ_b}Z6x4E9l_OT09PNL&uKOBP^M_Q@4X!_t7cms(@tE4YU z>&O7zNw!g+RzIpMT9qjhZVASNrB0BVkgL%h#Mu(*$9;?XMB89A7onO2Fva_fcf%2N zG_6uX=s~T#S*$1S+>ic?EANVpaok?NV?fte;JgA2N_bD}VUfW8b-Rj5)q12?d=BTbQ|$A%nMJQ3oQx1zB*TRO?;+cj#pAqlHrYxSROm@xx?LyJ zo$KBId3rJUuALUl-HezlRR@^&u|)QDv2q6qazoZ3+v!HTbpM}AFURUub4mId-@?$@ zT4Qp?u*~{7q-5;L{=F!@C{WW(V9eU^@!>dcs%j=A;!fok9w8O(9BbfA@nE)4ayM!! z!-nI|wD-UGZ~iNpjt{ujY=jB!-mkDm%DH@5{gs({sUiqNOb)# zkytg&iQq+IEzC`rkZe;u=`kgF~8k$ zoR!Ax;aI<7w35R}IJ@*H6dYAIn4;Y@OT#6PorH=&Qky|bZ+T8D;;|nGvhZ5>!mIT< zeLcGJU^|5qCnaQW!n-gBNPZ!baDG*oaG)0mIhU%$1n{u65EV-S275>~s0%B=Ug-Ya}vsW3v)!^dR~aeJ|^g3~%WThm;5j z=73coduqLpzl7&`HgZOme}QLOQ(4Yc03VshKSlea(I-Ni^@$MO@m>yl;7O zKwLS^-~LyhV*$GJ=i3*9(@I@6#%WgK8a3X6EInW_oJ|TwEa^gZ&K39~(Ja1?&xjgtG(X0kZd6&(6Hj+%i`L%oa=L70~<( z8FIcmmv4J>A-jBiQGO-LYRm~T-RhM;cgOdoCk6&ZvEO;IfQ^R8t1W>z@YB+Kn2kA& zP_p?q(RDJgQ9l+r3JiwF3v=MtF&k!Vh0n=G_E4_zzn{R^_L~LX2G}`j*}ag|L~Wz|`ay;Le^@BM*&Pd; z=|jqnO>Dorc2=|4tlU5xRX3f;A_TE|OPf=8Ld_=i!NUz*nWD$vwOzD@A*Nfy)X%~& zU94KQlsiN*2ZMGd`Gg445)zH_xsj-bHQ7E6y5o|qe%VkDY!Kkcsm>sbV@YPO$67km zn5Zs?k?o#pkr{baTLq?@gDNCOr{?Z~Iu=vK)W|) z=>DzW_bHchkxNIpiOQ65_FWHt$79=|#@f3IHc&Ua!7?k1spqZaT_q>i<9QyyK8P!> zPQ?X2r68y$LlRKMGsUCc1?!U|WEOL$wy5?1ZV-}G+7x4MT)B9578Wr3sQzNB8A=#b zSvYk_2M0B>`72B)m@a?Cx_fa%Qs0*>a z3$rg2Jw{M=N&n+L0$%GX0wIs0I~C=F&Sw2=y7q5umyCqEz#-#E+12I5olNygddZz} z@kg8a5|7ImW1M`E-`r+}-x*X0)`|kgn;p)Gq=?uQyOHIp@z^n|)Qqnk0kEHbZYM=` zS$xw?Ci8rJ9&RKeMlP_yUNfLHfB@bT>t=!G;I8a9h2j=4x5rr5NMhgD1`bDiy9t;h z*#%Y&SPQgs8p%JgYazMgq{7wehp9d7caIX3q66I@(f&m-j?$iANT0(0ef5!#;1fL5 znU_kZu)CXSwRupc5~n8hyP|Vm6uy!_(hE^%kti8GB4cGdPn!!HwVE%Y5Av3_KK9~3 zLylcRk?*Ns$3UC;7aIiHsxaGRg#5O76x9bTuo*F*0SGsqi*n%Y9*VL3fD*u90eKht z&})!JjBZZg|3IEgTPqj3!?% zJ7ydJsyY=ClvYy7V0X{eKnoWNVJ!-pMwlx?bAk;!lWjJRvj1L|2JZ8E;n^C!+hWhV z^YFFfq|L6HYT}$7pKgixTsG1LMc?M9XO~|0Gc;0)sIvK^!ldoJY?u$`Ue6vPI%A*}411xlcN1!?(7kx7`3RCNsm=sMc zX(Z@+ows1ta}4RWpy-vKw&gODcrs(=7^K2=Z@RXpn>b40B3{?LBf-gp{KJJkRhu6C z zo2ovAcG{F%g{WuR|L-5xTMf{q^I|J+^ix~oAktTKYQYpYrxI#`-*h zrzfV2X%Vkf9u-vQn%}fQE9|Hs3NcM-pO?R+`qWWNeAw5}jRyKtYdoSN?>Fmymztty ziSdRy0bHwuD(|?_eqcMV;w$IGN~A(_J_rGxaOmH>8#o{EB+-Q&WDyG=KSCxdU-irF z@cEbyxo1#ti@0<�?HKF!omhe!G^^`L41)iuj+vI%g=ZOFz6dJXq`t0xC;0S|#u0 zjmrp4aq+uG5X$oNm%M8oOJ=EbapOtCf^;<$KRr(l)CBRlb7hz+9IK)>+mSO^2J9gz z46^?}RbF;-CunPv_aY9qEM*a~6mRPzc>I371zM%%t{?o=de#aW@u_8~;TdgtiHJeK z0_KTAU^?mp5Ai;BR;Gyut(Fm0Q-X}%f;#6|=OSt%5Yw zKWF+;2hqi>6nX!>Yb#_@n%iXaJ;G`>Ac>VxnZryT^cyyqTEc0>A6>d2lxzoLU#JF~ zkw2q!!gBdv%trjBR}vp>r^U>i$3=z@mXPS!*G7@Q6V#h2{-?C7~sDj;gU(M;m60hlQh&O=I=8!Vc!j9N`{~bwi0V zQS7!5Z1CV261^2G*yQ-!&z}4QqXAu4gTKN|S({rSq%;EF%F2ql-u`M(%LREge>4Fj z$2;A+mORaj6zch{hQqqLpqpiO#~Xd+755|RS&-WMewk~Qi}xtI4%?JS+M|WPz1W&s<6I1w%{k25zoZYddtGZ{1rd{5+X<1k!I6rN=J9rxFt{#^UC0~ zz%DmwV&J7h#pTN+)z1U{1FC=q6oSWCVncs2BA#0HLD)*zdg}6=rfli&hZBCG6^Es} zq=+@!IskREom~r~{kUDl3$zRuEpBcm&;|xa8GnFsvL=Df9VAT31;rv8D<5Jk*E6OU z95c{);^6%T&aK9joROJI+|QI&Po)vp3PhJjJ|bf>WzMplMF-<5(Np@BBy)}gs$3Bw z=Yxq$btPVhU?mC_ zGQPAUF)t07{xpXd9%(%SMjh8L{4^6Qk0IyL;_EtLT~C;ZV=;>4|0MSZDH-EFw9NyS z5Z^#Z82V5j9?BUjkCk@K`GHj3ii+qE`}>F|@|wVN-u?6DnbFd?qVku{xU84UcU32c zal!Tvp(|D#1Xowj+6ay`UyYl<+8Jw)=__>NH%>U#fj~`V4rp=bk1Mn{Lpyl@ThR6M zSLqJMz1&bY_CRFh_#|$aFwM^;5!0nQoYWuZ&LgE`yRubWs>GM#`xBf7=vyzEyK>re z>D0346+bpkdVzU9PqTN`*d_CYZNo)pf5)34exrJctWqO&ojLJe^4<<6_CPNj)7l(L z7Srlc6N(C&V6*-TJJDaT)>5~7Fy`DjCnb^SvHLX%5dJv6*@n?zTt}|hW0tWo>v>h# z6fK(EUd*VhZ9HTmdzWGi0|MS@?H3c?{?Nh2=eu;k&?V;tUrj4-O62+IX`1|i1+6^D z-N2W-lMyDz>ep@QjP~todY1p(+q*YVJ?0?1pHv9}WCjAQCf$YoOYL-MtY>Kus&M@S zPB2X%kXLfM;Y-TgVjw^6PKjMBkz8yG=18Z8aTda6y~p=@d;N=P%FHM)w}w_~x?KjG zPWLRVi*Fo^6=OPFa9!xC;u!T^am~47_M;S#kMK@7p@a{X=cMf^r7Q_e2p~$F0wIA2 z=GG!*<68MW&*^)Vg|>QC8O&Z}eAwjvJz0|^i>?GA!R6197=VakYM>0TK3)z&1cL^la z;|#i?TD>^QQFO}w>7@x5IX~o_nUhH(;qx9QGK{%V2C{(e)rnc4ARaY%?Nt-(Ef(Xk z!{1|Q1Ir80p>qVPf;p|rZZAD0?X<9Vfvxk3lCT9t#t9E8Ya$#|p)fD8bR`?kE2Fg+ z%4gC=;vUXf!T{*lf!&KCitKFi;*hZv*CAlK8U;J@s6TQ^33{JS6|1OALA{I&%0mDT zMqPMp$hXLP>t0R9H>|FGfSKhZxEQx$gnSE~6^oBmYuc)SdGp&x{nTZ)y(jo3RU3mH z&2l0}$IO*1a1RL8&UT2-OQhR$rk-11lIRAWh&ecS<*@cMbK*IpwT8)BMX;s4$S`Y{ z&^!E;T)2?ItV}6K+~Aay6!V}2qK|rO#RQlWF=Ejm6?NnK_k?YUR&huTO78R@_3~c9 zb?BJv6RB)oX*8XA-;QZ_M*vUoM)JJfNbA{*oa(*@a}ri3i9#n61Xj)J@qfoC z&%p>r^|^S7qLQ&SbR7JoRuBT^=>MVB(*i2nXTIF@_UlW4xixVd)iT|cIl8rXnrdjL zA4o5=ZCdRBc9~3o4!F(!OdNOv(0=x(QAv~pJzKlAd*)4JA;w1%Spfa>9#8?9R}U!8 z2rLRgzI*BVW26@5#Q}yER)_e@bp&iChnBu=bUWdUaVpZJ_F`iWJskM9SH75!+8L2O7=* z8T#zPtff?PgrZQMOO}oa)G8+hpE~ok*-R-D>@mho21S6zJO(w8AkO^u`oC*gDUplX zn7RyE#nm2^cQ=zmpts60{IDc6_2-5O&e&z#Pvc*cxI_mf0^^hu+O>{_;*`a z^OG(ss)jDofayXAM#`TrC6OtUO=ykBNej0S07Xp+4Elbx+^O}}R&*ahR1><%%wbkO zU2LHbRwbM}N5gv$$#_rZW;NUg@JFy}B+UHe06*??^V3&Gfbq8_5o?ax!Mg=SIeNKS zat;D72SHBt;4`=H#-bxfPSx20>_&0CMLGyQ_rK2~<)*9~!C#qnqk1erK=%Q5u;zY! zCD(*`fc4ekX-Bk9sb(r`)oJaQgv#0atqZ%{d$$A1$hrT!8$zs9!5~HNbF#mn0}S6GUJt=u}i*521QrdmrFRWKR{$cFanP71PM_l z;HH2t5B3K3Axk}qIF>>xU`oXo1yd}k^@%6qyAP<#^zb}fkYCn!vk4|d(g|c?BF6;M zmXNq5RO%%7*;7%sjl67h2gjqzfjSX2xE=6Gg^$oSX(N$4tMp4E-hezmu_*c0i9U-r zC5>XAq`8ScrUFNO<^@0@TeaS_^GXyw<{?rrFPo46nkTf_WguR&J$%(8^b+Ud&HFa= z)}=)K`Er56($$O6SX)}nAtuDtTnSUv4%tIVat7SnB>6 zBcP$H&`&E*AL|hh(1HLBub75tT6q|%cSIY&({F;;K)EZ|pL#F*N@IUl_O{?J4LtZvP{`8brt@b#FmAxVMlcv-8~sLdA}dxPHQAjs3WM2w%v zZp9hsF@SWi#FgUTYK{_TgkVGvTQ2(<*HcuP@Us_z$-MDtAUE}M0Xr@~5DFQC_7HOk z-?V)Rf{>$VPw+FCS7-e#UAw$J%{yeIfX(kvYUYVB=hMPFB2^~k5fIzlGx8;C{Q>&HAHB`#Lw^Y9J{!3G`oAsS9lqr5!xhspjj-vP_WTWn%CUW$88V~j**kgSvx zL1*$N&m0)x8M~qv5H1=kyP4gP%k!ukoDq04Z!+^HDWzhMW?9)dHg{Z313WIQ!$up) z@nY&L;2CzosC5s zLb>{Y8XKMTbR5PYYTSlt(=QC_SS^C>+XaW*-vPzgzarnX zp7%l}chlv)z=Ix7K{e2-iz>K^@OfL5Zv4mOe6liC8iJ{FE#2b1N*?&a&`)^aRqk$yE4fQ)pA2%^BLYg|{Cc8eRNhCEnd+dE88{j0HtcDBjOxT21eVhiQJ3?92Xki`gsFX`Zpd zhlNDHohmb732)CPm;=9`Nj+~UOpqE4_g$sFBxCFa)huRgI{LyFPk(z*sH~n$p4n30 zI4W|7J$*-`1Ryl|VWHe5F*ZUdV+2Y?q8XlrJoLa-#y{U;s?<=Vr~v286BPi(`9vLh z*R-wCgM4a!J`7HSTRHEXeiVT_S|qng^6yN_J49X-Tf0Z19FF*_sls}$2%VwK!}qlj zF8xyg3=7~ANXG+@ZKd#me!tofvo_L#V;P_{N6k9_+%(KIENG?hN|uJp8{|c3R1mqv zc+6z$Nmb7k*|p+V_?Z(Losa?IeYfiPY9;S_i*2KmnkHzO&Z)s&tNHo@Lt{v;+mvogICb#P`qiQD&YuBLRD_WR=yiPA(|Tb;}We z!nWsCD9|nv#yVVq^m6~*6ab>hKQ!KZnKc*PUTqN=U7C6EjeCL7)OPLQ7n{-O7nI7i z_Ams_ggl`I^cF9~xZ$wnESv-{*qGe4cJ#gDVmeXM{1P!}4FqSML9WycdUm~J*2Gu% zG*6h%3&!lnVrx8lJU>U5Ve;~kW2Lf8Gt23fkvhxA2K)xEW#s8maI!rZdy>UUW@K?z z=PIp)R^dHETk9lHyZtUiGjT_%c=+$`Gg2R-DKTJh12TtoNcg}Cl5=wyYy&3M6>?M5wvM8+($PX0!$PRX$J2^?KSViH3;FYSAy-q(CT z7h*#9MqwaYr`&{Wuj{pfC4v9bXoBq+nlfv7n=b`vkbXehC8aH?X()Mz~!U46Yd!L?1_&xDEH8JoF!EYt%lqoF*AOOl- z(z;Q&RN?f{q{oQ@V05>Y5F~}0;lK1l?fw9(8q7S>2|0mPKw8OM;+uWA85+*)Z}Hwb z%CWIH`V_x)M7brE zZ{}H6rB{imUvXjG{G_Hi(FkB^Q z;Z)fkO~SJ9RqVCbf>LJIC5?8b&Ho_1$&eZOQ`w=j!<^|$8I25)>q-eo!G+gDs`8d# z>6G?^^z=Mrn@OI!(9lz{O}KNcU;{-3%vOG?s%4VT+#1qf135&3Xj8Wv)|I-1vwC@d zTQ~&#eCKAhfY<=Qn1KaN@8z?+T6$jDR0DR3-d}2i>mKsU5Rr*dH&8bJ)M8X`zlQzh zIh6>p3?L$=0H1w(J@#+Re^0BqhKsq!#WZ#{#(mI(`k&2O5<0f3#1A&w$yM%E8WcaXyD zJ{^D+)Sl>oc5q$xAo6<_h)JkJT%X-zQ|>8>bz9-F*fTqxGm}^4bwF;!iCOqaT}|{} z>3CbBVLVR{^rBI~9J*1ee;>}=$PbHX#Ey^HSGo_6nJG{$jC;_%89nyPJ=;h740ieX z;~CyC!!m(!T|kKDzFR6E#r}uIB7`NBOwCA=z*ccM$RZlzP1_UH!Rg~6zs?TqXa^1V z;=aXr5yv=mdHe}uly%@Yo5_rgsv~FkCEpCmV889uuZUesSO^!>n*Ve0p>+?XB2#(G z@hUd?VO`Un>@$}AA{8xa2$p}S& zF)QN7I5cqqL$6ONsSx!IxAXyUO0eIDO1om&;&D(2SR)hE|7{NEU^{-%Amv_E6!kHW z1q9d5%agN`RqE6?KwKDBfTJzS_J|*G@NNlDdrf<3v&hmiY%mi@dM5moAHVB@Tx|$^ zkWV3U>f5Ww<_zO-R#09`SwP~E7XK<{kp2)_w(hYr1Qn&4)Kmd(=E}W7f1_5}-lf>B zI`F!Gs2S_RZdD_5Ve|oN>_kjU$Fe@b&sWhkuX5;g2NW#DWVP9$J;vMJW_uchJN^n( zZZp(pYx{aMzi6#?FLwhVn8-*H({JIMK3eN3;#k|5k@&o!e@%?H=(N0#JD;WC8#b)?JE>XYI`t)^X>F=$E-N0&SwVJi$uDNmLUCw z#M=h!ka3+7I<5_qWii;6rL9_tTONx|5CUvbDM#KdxRv?hr1sZ-ZhoR`snA^&$JRe3 zo7#woe1}tvl__zimV2ZKv$3ou#Ise9UN+_n4{x47uzydCQPC>Lm6R^G!AbUExt9t? z^O=4~PoM%jS=nnNg7ReB#bHl^+$@6rKyTQx;Prz2078v-K)J%J`P}@;vI~W2l(9Nilsq)*Nm;J1ILcQ00lsYcTOGMZ1t7-2f@wL?3i4*l!((3kUDZ4 zZ;1FkXlkNr+d6RoS%w+I zs=`cvtuhqqrYK$4Z%>aaAZtK1IlHk=4QzaK8P*3h_dzxV0UOChM|riS ztR>X;CWf5u=I6}SdB_m{IvZA|(^A%6FKTVTi!_EDDZ+w=#44PjNs&MerzR%wXQo@}FnAY%CKRP^D1y>Mcq9eoxv3Vuy3H8V7Bu(UgspCf$K6WM z8*o!iL)R95UkX)s9NasW+0|20FrZh)Afzu-L5;|M5!G+Yxu7K-0(2O|a&msO4CC!Z^DtZQw(|90@k=;>y2*k+`EqNPccVfb$35R-TfeQxT zEFvqBeyBP=3>K{baWJph(>JMWZ`~;4A}}roKOUlt*WwG<5#pV~BVa4Us&q1K)^QIj zn)-N?AC5ZGI=W=5hT5K`bwrJU`BR_-@=%bn)AAo*u|8WDib3U6vpt9Y^&Z!FG-`p+ zx=7WoVA6M;;ghzbXc%gs`%6w6JO3qU|1_=QUX|X`%t;S^LCDT zq!{@;z9R;F;gv5ugyX(ZPIy`s0JA|zWeBEohC`e_Z_fxNAvKm4RXRR++G%1N?Mb=Z z=yHzL{@yITm8hR?3u{`ZpFnzfoX0aDaI-}N=aSJn_NF>i@nf%2vQ|PATo5;Q4fH8> z)bnKz-gk&g!B30o+4N_8jcQ**_JgKNB>JOj((AG5scM<(+4O4vprRdr%>vO!hc^4# zqDr(#cTZ|gDEi`zUizagS=f_yh#ut&H`0dZa&9$C!~)8$Bn_KqGrRzS;_Ql6c~YZz z5vd`I)xU<{Q2ZPEIsvLs)Sr4(Q2Wq{ld)Iy>b2B3PXz&C+ui7%@6!nRJq%34H7B#r zU}xD712>ZR2gKZu7&eXy&}lOdkCAms((1_}C_R4|MO*{vvOz@ zmJuw#zf%;keK!_~k!+B9 zb++${u$s6+9+YKIeFe;#Wo|Lq{c~F|QIsJMr8?jLmx7>_$=whow#s8ol9?oac~GK% zmaLC1%`A1cVy}%XF41ehyP{tv#xutD(>(XdZ%%_&=YGgQv;w~`7s3vg>)wPpiCy?p zTILFf$!qs`j``gMbsl({oIVFU0s~!F3PL}3x=?;0@t+IMa=sbKP$Fhz3|6S$wq|!W z4~(+*i6Fwu$bwkbWkYCRtwQTDh>#~oaxf>GWSgiBL0C%;1C=7#5Z#DkPa1NI4c;3? z3p+?;H1*HY_=D$&)@Aq7U98F9mNR$mWgAxk(_J1+o#Cs*vBVrX4%su7!3wA6L*@9M zM%1rX#2mtAC3g5u&dr@~pNzxyPpn%e3YWQh!efYf>@f8ZFF&sW>Sn>oI)m`A}?zCC` zEd*F%%&?}vd46_Pee(fqw^P5>;M2!g8RyKF9Z6!V=n|j2LLx0ggm&{#abMiyTa(;w zzYIKdgBtgu5?~F~W#tXaw4gfjvWe0oeQGw_Qcu^lXnP!k1kG-g)n~i)n+k{v6{Ab# z1+uV7v>rm>S%11Z6cczlZK6RKWq9o-wCtFMdTTSrn{V{S@e`N$gD|g-Mg%UA3vWRY2cx@GH**GrV!Cz593$yY%8fv? z({MtQ&{(12d*1!kbA3bghNWbG60(1ul%o-M{n_{o`t<7fP87ft(uZGnlogmB_)Xx2 z@>@%uMOu$xM-HAYXBLLiPWISdh*$-dgtT0ZP`}OzbjQsiWG-Rm4pO}OMq&~$p`5QGu*QN+!VH64X z^>Hj!IK-zP!q1pR0fpoLsB2}EHpKjYTZYu^-)lC`BodhdC@}ln;2KI)aP>9tFp=v5-L7X z`P-x{X4g#^CP^Q5%&#E;L|<>}g}=&wUdStPshtyjbtUYgVgv(Hf+3DPDt8NY`TO#k6t_Raxip4{Rdn3-W%|FCPu5RTR9`C z`NWr0H6GA>bdV_+&Z~QH;=}tl?OjZQo|K39?$g3q+r!F9op3NC3~Hcj$Yf!LI41JC z%#^2Ya$JH1>M3StKsh*4yfMXi4;nPujEybgUygOmoKN2=pDwM%K*=_M3V_u76O|?G zk(_t@B~SNjA{@zG2%l0DRal*#<76G@Z%^MY=}jg_HvpC1s7TMw<_~4&%vW~d= z$YA_LzLiH-wVQmX>MC-)b!7HYD3D|kY3)!hBkju6@9savk}hhxC)r6otxD@5cb#p; zPpobAmk(W&d5t>ipoBw*zBhj@5*{}y<>1T;j(LNb_-;2q` z4|$;DEzq-K-y~s$*tMkqeHZ|CtcZWlmNG4;UYBaA4u&*wsXrB`HUg$t{=UWpYIbIKaOR#3fXMgd}(%cp``;8TFp||9Fy(QK*e; zs4R9UhVlLZ`hK-8pL1I6kDv*J-IE|tjkxkz#Cw1`TCj%<&|eiu>=ZBr&sX|;8%|9l z62hof>1yyB3^*kKPC&80fCYqnGOlG_8Pkn>^TOx6-ZclUdu%{MPR^!oswGgIw*ys< z+M4(ntxctfIvB5ho>azW-IBC=|FE`>G>QY{o8pxm-k5omukjFlm1hOsm|q#?p#kxL zF*V0?lcZd7NvaG68+czJsj=>6^m&XiN0j(>=l9d4paNuAFRQ%HRz;FYHrq0Wsx1Jo zP_RhLrOTKMf1P5Qggj3~o&A7(AiXCj39RmZq?I-m=oiARc(!tQbh5_44o3nzw{g86 zQaV{%K8TE|%6Wy_GvyCIM?G@HGm9Jg-cj3nkm_I@^s?3cXVF(iW6N*mlusKq@lJ!x zW~#)M48=Zf$f7CrON9dL8gF>`a4y&c&dJHPYvo}6%B;Sfxi@9YT!{qN9k~ngj70%` zUq`Zi6P0Psmk)$-{_m`cmlbisJ4O#=ohZM}5>@QMD$lf7!_-tV7V7V1v}Z2KE;jqi zuCX#-;zsvFN6^}{s71B?Gs2c8A1x)&en%_$lZJ{dMOR~uqe21-b5iPW& z%aO07<9Y{>4bx@-!G4 zC!}fUJS~Pdp_aSn z>MQSDHO>G^zOT0iU`P$q_LD;Fo;4->cn(da@TnINs)v`B&%M_btd9@Ty*)1Bd^o^@ z`24^O8KYN*6%byWi|^guf3=IpUu-*w8yF2Q&Gf3%EMByjgUY!c1D_Dz_v6MwJSKxe z`u6cev6(6OZG#~7N(k_#+=Y=q%&T3+QQUF50-(}n;HI}v-fOs1ODCUX4$U=kVuX}) zQV`#lOdLYlY9saXnEmU;eE!%{Q@Vy&fUa*^$x;-rFF(t+5}_rix)=H}QkwgpNTw#_ z%I9aKy;b-x;0YWq_ya@)P9a06vfC>7^~Qx$Af< zbIOI81AgjnRByt(=dljD>Q$O)J3fB>0O(O-CxRZ4o#j;qU!OBpwOLHaZ%-i^XiRsv z|B!j1J%4O<*iXcz9gq4NyVvt4B$wh}0L@w^lF-;kjd#M!-L+Xw>0(=oxPJwaldne%i(A*nzCMkHj+pl#%&=|xEC67+Opd9fxv%)NHn zD8X^boMTpZyP`zP)T6%nf4(Rr+Twq>ri6Y%ej0Ol9IIP@3t{$2u!@=*kpm%kYs<(u zmf_9P6Ov(2N`~-ze0#?nPXwE{kcT~ZVY!>F+x=13+Thc!h^DmsJ1!Wz4Q znR5Y46IPjpfG?%tfCs9p+Sby+{?`?8O`dc|E+A-E@r9#odf` zuqz(u79&_x!{u#j!ij%ALU-lhAQw8NYuNe1%zU#1=r{H-nbXcGuzt&3lCzVmz0zSA zZTveQg@u4B)yEe;hPr`;T$rh*?Jc616c-ZJ>vDlV0ANllC@lx!e;>-59;{MtYJNpI zyr)9e9nG$BJ0?F%7(LFd@L#At%DGx?okZMmTUAr8e+-Ea??-4-sp8k}=D#WSr*VES z2w)@PCFg%%gf9I{-UW-8GI$R#)(JXZBJKB;@7kzG-IMan?rF-1M#9BX26b5#IJ;$^ zeqsIRFpu}F4U=7;fTMxXg^@sTfjYvM7#5y8b>s%GRgUtt-I_!=IaIprFrRPz-l`;s z;V@*lTB!E<%4T`KONqgbL{0##;s6{2t%iV?V1qiFRD0}nm4xN>Pb?P4cN6oNU*Rk^ z6}R+!9F5pxw5ZEx_g2@WjP)5g!YGLpCbnIE9ur0^hX~a8d*HXBx=YDdBEH59sEwLb zEC1jT5?$cOg<&N0^-rV)tJM}?w>1v~gJ_NA)e>dI)r6gkGcf2enr7EPH}*pleigZ&_k zl)Wxt^A0ppboMSdIz+B1cWt#?lafpTa80o|u{~++xKu5g?r4*=tYjVlr*7 zY6|&c--Dw7D$2=kOv??Dh`n}$+cHAG3B@Dj_b_YO{!k1_ZMz{krtmg7F_dMnZ;n!8 zKI@lhV6jfBS?xW~5>6yiaDoh*E-GdwlF?_De-vKRSTDXSx9tJ!`cj92$@RU%;5YIc zH(KoMk@02<$2aHXVzf&zUQiMDrMn$<_++Rkw|>@sj!~QY8`qFXK({!Qb1T!vXY|;T zFj_1B*~uV6ce9#i<)E69H-#KLEg4lzxBCVt>6LKciT{L{xqlquLv%Pk`p)`Q;k{nP z;(jsRQ#8XM_i`V$K~DhC-=|PTn%d?Jp2~LFHvZxetqZ*rST8|{@_kb_~FP7 z$T*nae6dg3g`;~GY$AT}?!JR^l}ETEL6*sNw4&tqnV#P)k(2O7@P82Wqdv5Q6z7V8W6@BikSbr#C__9Yn*9IT>@N|m)7@ZZ- z#^G=ut*=ilniDs`p8TQ;my`ynk~-m8UqBgKFluMU2p5>5CGoQjZ&;^>9c|Q%n=K?O zoi^kW3SI#^fg)TDvN^#YcQm%9{w{?p4}8V^L&c%9uCSZ8PhfQ*GNNVs zibO{{EXrAI-WhPaaQJD%cP$yuOSSIZ4r0-uOud+YdXd38(9aPs#D| z?U|EX`y#oFz|2%liV+8D@b{|r5Bm<V1@BoHgH5;_q~dyxe?PJDrxo z;w)+w6jg;{X(v8VqEkH8$|DISB(X8?XcIF7uwt@qd4N<8i)`0Q_ju1jYep;k(X0Ux zgQpVk&z4cMlPLvkd;?SnO%;~f)hhf9z)hYc=?5o$LxB0TcZ&>w|2re@59gqDvoIGU zT7A-EpaAnOG+s4?m48f|?Tl!3L{i$3r;;dB-Dh+XxOFWe#w+ z!5`i~5qEaEE3j|>oB>Z9-`xE&G~n-7*ykXs_R5b2M;yJu#t+8WXJX5MpRZB;Q{mV} zE+YB>|8<{nGhjpYAvwkOCSM#QJ=l;Dz$qvQBHt+)x`NsqH9{UCU8b zG1syq0wd62w&};FT26csUQUS}n8zJY+GEXgEiHbwSsG)fU5b8pz;2eBgj0sr|1yTjQh^Exff{5R0=e%hL2{S4#P#se`yXq>@P8Ru-zw`I^y8H4R8D`&T)R~y?r5=hqu>WWz_fcmsD zoHx9AQpr~Y4(Z5xxtB1D74j?k$IBU;=mP1d0Q@(oKSxVo1M$6dEUCHTk7?)up}G*E z#$5ATu4p=G0g43&)dFZsA(u(81=HF>7TY1RC5|AKUlu-!{UhD*oCCUFlu?xy?QOMgQMX%)%s z)Iy0mb-xQN@mg0>E??EKM)|QRT!AfAKD8zlbykT0%EW_*&yFC9dw!GO! z<$ofhUWyRbmN4MY66|S6-IXA7>eEu%QEd=^}jp|Fmbk#@0?i7hXv%34-p3)Ofzql2sviAip!Cl;N~Y^;h9WxguIgxVRtQA~M+Xy*_8 zW__(<{{!;Upy17vo7x7P&HpGx6~4i)OSY%T1-`wxbRcN@uZIoq@+fchC14(c`egK< zIANn)?^^4Ge#g8JO~*yLyRrm{ZjGCjhy>Xj0yZ}~*_oghQk>+GJkp?}``fK{Y!V`D z;DE32QspGJ7e}f66r;-b#|Dh1x__r|O68=R%YP>^ybE61VuArgWTT40p0euY)p?eU zvV$3t4181BLO6SA`J54^LgzsieHfDH+&bUrNZ6N(zWwUd@(G-gp8><}(slC_BH>xO zBB!1TodGNhxcv=#b}cT+mN8DaD)S1sfDAp6dzHb$;u27N&F%M7CybJ-7LXR1?1vlA zI{pd*@mm+?ubVN_)y&B>KhSprc`Zm5+7UH9)40bdOAJXJpG0=26v^QaLW5MpE}dIa z58@oi)XFhrJDg_ZTkRAGd3gyDcXMDDG=5s%*5GHsrBD3lL0r}tsX6#zs>5s9Nz!>j zQ1JQckaC2--Pb&zy6V@Z%mO0Yn;l z`UI+`LS~;Lu*1%h?Z9N;ZmwN!#K;{F%%OO}%4%fQ&OlEWz${NRsyby82RnIE9td5? z+MBGX0gt`4RuX;cKM)%@T6PoUZ@+q-pmWM;W%V7BNjTOz3~uZNyL2 z$zHNKs35^}QQJ5!mu4}9ktwSr^gquIOBA0tjKX?6vo$Z|cD~t+aBgYG6J(sQVL}w` z;Atbd|5$sW12O~QxTqa0``X06B9{*-{Dj;jtx(h~? z)pr1W7z_V1 z*g%$ioiR|$Gs25I=&KEml?FEDEw<) zh;%A2=Tzkq(tnXbbE*UYDa$|IjJJIC!4Qc^V^y)~=or;KlLN^IdbFVqF#dxlK7ZRk z>8xXB5HWUHKqyVN?UAdPsuyN#L7y^OYB!gKd?zu++@uB8GfTR~v(LOFn*3AFW%8Hs z3vrb(2+9jL71(_*_g{kH1BHZubj_;l3hV3AJOjQ5vh04vDHz<5i#Ai7aBjjj1n^T5 z(3jl1Vx;`8*mI}SmLX0w=LHHUzq3+EN|SfAk-AsMR7MH(|0T^LZWQdbx30`@@i9A` zPozWiBz1D_2_v=`?&cZ+S2!??0Id;5+7HKyp%P=_sk?*RR>`IQW5_lzLPlmqd6;n0 zoRt-z0$ixRVI%Zj$!#&w=BK+r016P_N}!m>uJ7}+E+0)EP1|3d14~f^gHbNe@RX20WvTJRgKibh%#m z{U@D^0AT+q+>KL>kg{O36G*@X{c_BDgHS|BG1y=u_#wEphc3Z;b0->*_4N5MlXau7 zx8Zh`v3~<9)J>cE4*h7rh!W_&*OYp#_gGhVd?wqa{lnd@Xx{%7uyq5MT&)z%#QVHp z$Qmm;KRByLHWv$v8ZD2T*l}v@+brQnN$>;jnFFOYGKjshElwtm>07jW~ zin8CF+14Hs7Fl{rgiJjx6%k9Sm#>9(FZvcJQJRJ@OD38e6gKd2RvON3jp;PGD)Cv= z)tQ~_)`gq3w>7w&BE#{xUg)Y6C7t-X`x(7ckwI13V*o`s+iOFwZ7k9;MSl3?OB9^Q zTXtJ~D?YrH;#3Ds_qtGX%D`rAS8ze}yQd~GacwKASk?@+U(mggiOZOe>`E7et5>Xn zrTVmQ=Fqdq7J-bi6@Y*~Lt?2@k^p?o;}tfIU}LhXUGs0gIBYJ9nu>$vI8qli%WrL= zC!bk%o!B0+vEbhe`875S=q&@gY8Q$r-vK+zoq6PfD;+P+3|Jb{+)tSkM*yCW60-r&+$EBSOR;IPZ2Um+U9lKX@&0@ zp%LOo!@2ClfuoFlIjPr#Ed72iS2SbIT=q@XL#Y6`8ISt%>x6@#x2zsks0RZ?55quN zEuSh;+CpmDNe!G{m?L_!m#$5TYw)tKzj(tu?TqeJQVjeQ=Rye9#>0ruqN_pN8q%Hz z=vkl&RPuh~(c%c*r76f(a>+bl5N-VuNG*7V~eZl($ z2{^w?_}d{MNB>p{ej^p~npGyn08RKDp=2H`1d-Oo} zM1)bKdl=^QVaN1^90hMNWBC?^bn25mTDsxN?qrd+^69L(By(OApj%I|EGzl}eX%wj zz6is2H$T$?{jGGV9UTJK&uv#embg8(xy73CQu-$=m60#`!DM1&x`Otq8qjmihU=cZ zXk`rmG>%G_)otu&PJ^}7xaJ#}L>r(*r9%Z(9mvyIx#pV6?bg{ao715g9Ovtxf|NeS7(8x2Htf*?ltK%-km>V%b1Ka!SQUgwFWRr2=CSo@840kbIIuLEei^LY9FqXH#GWQaiaT%0md0h`Y zhva<%?uo!EJKCdg@HCROnZgIv;0woQpJSDK!{C#tFQ2ghS4FyKPI{|Jk5>+^4je z{SYa^ScX3_q;W~*v)R3z^=XA10oDMBFwj4#*5pR>XxfdL0(vfIvgW%ZKKW3-ID!n9 zH?afLWZ%MnR)I8*50Imo?nJ_jO?t8|{*m?r>$%|{$XX6-n%Fez6-0K`AxXM0)rX9t zoEjDzATmN!fl=Ek|J_DDJSElLU=S|Q5m##|ThBSTDeYTgKH2c9w8BK0NM|s=)>0mCj zrm0g|%Q?!K4uV;X&!=5)@Sqx){lWu#xY#SfyEj=hBzBj0GltvYpeg5ADK6EPyW+EIyd-izb%Tk__9`#>*A_S*=bY>>~ddyI!+>wL{&Z= zvVjaLI&!|wk>1J^hq!TQ^z|fBZeG*!4v63jW{f5P<%vAya^#AJ$aJuZs`W1-VTVL7 zxQCsa|B|Wag;tk`a2Cv$IsHJHjru3KBat5@-CwD0_d$R>yspBdQ5wpDm4Dz5N2-X6gwLbsD}HmL8_d4@MyUqKh81zKzggIn0}GxrrBhC=NwTi7C992e~)(FnFV#&f0}K>UMgT?6{evq|z}*PzQC~3Pm3$=P8bx z#IU67mL5m`=miD_%BJWGU4tri_CmyuKZWTRZ7Lhpe)wDd&zu9Mixr3Ba2J9|B|A}B z;&;p}2(DBS|rObxyWw7=SeCP{ET(q?T4CWW%}TY&k^88EDAgaeZgiGZO(n| zitNmZNB;fs$7ITLQvs0pmFP(Yzu{4=%!Ty^>8iOYbX&NXDq4p#&Pi2qi8A0ua4OOQ zR{is}R#q&zVDnEn&zy14Vce%?vk_r;lndc#r=+;$o-Ci(v<`RuP|Dt|WA-0nEfWD7 zTo)9w!!?)D`Iq6f+C_MNr(=}4*sdXcJ>3xi@VG-64Mz0U9M(UFd`2~P zr(P+!PH6c>5Bqk|*-oP)#ew+eP`tT|;S-}OV-LOiyl~wp|5K`zIXjJa<>WDRuRn!N zewZ2zbFl!0lU}j|Q-89kTiHUW6e9#=Np0NYYZX{^ocvme!|^gf(A3yq7qzVn%+qy; zOIL^jzWp0~LchRhKxsAkUpaZjYaONF=e(_&&X(bGY5YeE)Dbbt$>-O6*VV(oKJPbw z**z^Y!!kuX*Y9WQwps#hE);eVfw%-fHaOb?+t5RL(m0c7oHjME|c z49{$A49h($q#jD5UvW&IL8mI*_G5knNy#@Z_u+vZSj`rY<2o*SIGanm;(C@Y!K|vA zs|dwgB$8SpAHX?Ps((b_>%tX!%X8bEe9UbZkf}?252WR?62HK8i*g%BJ{*nwBnEks zdm)(Di;m)1@p47F{O%Jx6hox%B}VqkE`;S`E1GDr;=|On8=Erqv3q(xF%l?ZPMK7i z7*-%VrdY*>ul|1$)X{6`lA4av?fL`M8jD5^=Xd5*VQ-~L!`p{7w=15-|IM69EmF)DV5%d9?}9`7NEaX_J^erILY_Ea0Mu5#5DKy)7?q5`EJqG0waSM zrK8qss5mt*I-zRB;*3)W!sF*-{#VZ{OfdT3SvbH1kN5MV3BGEZBkN()GQV*XyZ(M& z8AYA`59)Ftt!zyGxF*CghhB45ra2ePJK5zo7I-aH{gk3dBfhVl zz$wVr#VxAa3ESA6Iq<83Ecj};XaObVF6{;s+C3@u$nfL!2jCh}W+7Mm_@3m>J5SeU zh$OEtJ$Qi4+Aujxq0?Z6PKm+8dc%ujg5o+w8X)5a~4mhTe1jLA5R zuS?v}nSMAG+K0xcxrQi4&H-@xU^&AL=!#PaXI+UYhgV@?r(TK8L%)6#xpoqMrrFt< z>QSJf$x{Y_^kUZhSeLjX8yziS&-`^%nbJ4%eYF;3u&h`5kO-m~_mUSkvDS-(nfc*i zOQt7TS(QY(_9nmJ-mA_ulnKEx-l=uGUdNP?Y5Pr?H7&O-eEiHgv2IAE2xl6v#LEaP zFzDeghUP*>!!@OSxXL1w8w(fOh&xOz*&u@Va^rZXRjoRryPF?M;KH)={AZ`Pp*2Wh zhYhW?YqE{LB>5+TiZbvd$+JMvKY>2Zm8qvxfkS-xfC+d=X%SFmBQo3vc7U!w?CtfsZ zMwjf&l)gS2>8*B-o5jCewDA?3(%KPeeD!);`)^|AVi^WkB?K9~WShkbHxZ-17bZLo!&vE%H;ni>xZf^J8a>>|kwgrhybMLf`{J-=R@{SN&6(n8_O%{6* zBe$#ok4Le0Eq##q=P+RW+;-_xktNjAR;~{649CmG6|Guqv`o>BqPQrg!VcaKh2(k}Aa+d1!%pD0KJiXlGa*V&YeiS~e|8z=2f5t=r)xz+?a{7f1DE_j{O z%cg{vK>K~}##g_~cU~O;JHgtvHyhGmo8zCNS^3M=7}SRJX6~Q?c103_!~h z%VfM*<+C@LRGl6h+2a(E!f*-Fs-B9(BQ%HRdgt3m@}16nQLMkr{jezKU8OIFt?<@D z$|ojKR$AeQHa&>GGQ6qZbK%oeY1zPmP%CEk&d4x(^Oe%K)B~FGk@UV$!<|0 z{#*Aik&+F%$WRpbAHL@mB>K(<>heFrnmv|6^;nYWk{(-uF zND27!6{DW|7x(ePqR_1VVEaAM?<*tiF*^{2H~|oW`2pH#5?5Yh1aeMk)qns?@yiCI zx-XI0bG(ICxu0GIRwR__uNG$rUEG<3s#mh;#AT|Ddak#Kp@5MW*T-TQ749|TRCszvgq8&AMl+CS z5v2#XwF**mNNxVMbWDmBHrdM-K9xY-(>R219OzD-DnFFO^u)YKj=~jZ0P_JF)X2I^ z@Ar?qg9>U?!3k3P^Wy^Lu7(bJhU_&|BVp1LACRbH3}!kQ+74m{7@(%W_X~UX<^zgK zouX|U`a&E?;_+-)A4PsS&+TYXV}@rYu}WJ3Xj}413cw=KKhaWk?6Y$XUAi~FTF?pV z%2!y*d*jkx9$@z#nT*MHAlwm^VFvq^+K_5ysO*ddUJW`v*$KZkLGdCd+rL7;n1eKD z{b)arl4gJW=L?`qhWahuAy4f%UmmI_09#^;_BHuHP=Ze2SFiAVR> zT0%o>Sxl+YbT|2Z^6?<`|4j26gy6knusCt{vpRhQ&>Vj8q+Oj+!a@y$O%Og2f?vle zZIRo{!}j9chd6}FOZLFn2^E_wntDu)k6xi{Sdom5;RbD7*^mydK^|@8>fAJZ1GHbRZ@ znb;gqRT6!ytVi(S7Lk&GG!nQlJk~0Wzv0Ar2Y|kAD@VcyoP~X4y@qM6Tn=8hDKpRM zLSLI&2iTv#mjbjUfngh;PzN5aaybO$Rc{3oZ^A|w_oe!JdE`eJt3;O_#)jmVG84G) z@;>PJgLoBIrD zQ1~F#4I9iF966!Z@w}z~p#H5pX~ojzGpvQ%k+7z`jjIo92@U8OCCI_0bEJNOuj|9InXudH6;>C(4VEftRmQBC4*$%5 zPioGRx9OwaXeZl}l`UZL!^)bYc6I))8#(RUcR2&Ry(GU4gjl-X+i3`Ar~A7zdnT#u z`?&t9G{6Dj48&xs+B4^*4D@)bkz@sr%HG!CJg;-)NXHsrz`m}rW+ zmm?(H*%D1*TjCIMgUdhoLy1whI7)pLK5^n?@fLqoY44Izpg=sKQvjsOLG99h!OzVL z4G@q<2OM9~COvtU0ICvA3F8ZiA2>S`UcP|9Lx9#JoQ4A|E)-~~f(fa0+=cv4@7?;> zWpu(fTll8|~{B&4!VJ^GgS77nJpe%L@Tb*<`)Mbw)U3~i6cbQ)s} zd4{oI!bM=G&cO6#@2cU=VCW7{p^91fg#xLuSysb~zRnJI{BzS92o%CiVJ944l>2ls zvHWb%R|%vE4I6OEY&B$U-KE#SJ6{2PubipNP=uk)AyXKhuA8-WP!5pA_NpnYz5vk7 za9U99sBFJ-S;>>~WxpcH$YJcF{drjBZ9YV;xx2@%nJ*s+h*oo`LZgcJS)R;;alS7H zQd`{okc?1wfg|)!f;<3PR)j%+n>M@oGVzNrf}83f1~Jmo(Dx2*tX{6Z$JV{Nty^Xo zup5#Uo<3a6(<|9Mwv_6ls`#R$oI5U4>{!Xa7(G6TAZP_RMBRulp1qmT5^X}-=jW4>2xS8&m zKwz0_kiw=2LJs?shl6pq!&D1l<;ArYA+z7nPxzt$BK_aMaf^4wmmmE#T5ecVaIStc zY}7q}O90CUbRejSC-sQV0!7vYnnSZFPX;&OZ~P}m(wfy= zNHQiaR7aWfC^vmq?*un;Dl5`K^AT6NUE3ZWIWybO(`rZgkI4b$NSwYb%D-263J6({ z7!W+E9=GmGsNb9aj8QUlxlEu`mCT^yKp&}`rD~;v(g_e0^HX!K>JUH;a$c^LWQn2z zC!$bX@#6w3PD`)Jh<+Fm0|8c=I9usm*yx0U7z zGl8KPq)Ok=-kkXW<4u97?%uIuY|zQ3)aDvpk2^b}Yl@cU31ah@6BQ0k&*c1XohBwd8< zKRA)DGDHsZ2b}+h=`!hfpi<}eKz9)Zd&&GYs?{*clNmB^4E0zY8U!~Urz_qckp!L( zm-za`JBeMu=#so)I7N_<;^DUOEoF)!Ojt_cqT z3*&RnUSGrFVVETK!)Z|(lIIF#vQQ`6!zXr1^6TEbC>@nXp1%|jQrkEG*{;itkK9?I z?mYNU5^YFrO?9OkzevN!Zh%gqpeC4L8tEt;Qfy-!9*SZI2?)1saYTy%LgYavrKlmc z>lc4A1eyv&Khn=^0uRqk4Xo9OlUn=DA8cRnUYVvKuLTMtO^BN%I$qLmSac9Uh&Dpj zU4Hr8tHXVyMl%|0cEI#zJrYDwEmzQj?h_>NQx>FRkly7;hd0Zj_l=f=!J6h&MpM<- zqUTOi1y?zVp!++f%qm+h!=JXs>`DUO9Ij^mu#efvXt0onBxD@_n;IAZ!elA|(MHoN zPcHO1_NJXFO|qfCB%$2y*=AP#!(Ts*$p}cVTsS%pou38noD-bFtxmLQ)Pv5fZ|M)j zsl^^&cK9~@s%+BdIe%-xL%oYtRrqbX?0Nzr%KJ8~2HPUf&cv929tWn(dvewg;qwMu zMVVZ1G5(&`Jt_S7TlqV8X9%;|@D_NoXr@x^3G4+A-ee@9=4P0*TjovxHxrFH=NNNCI)J!;<9!oH;3 z-BL|%=-7<8#;~k+T{XHYriKequ>_(XrkLX4P1AeA<~dxSOFDf1C{=$IBHY~QEd1q* zqIWAg)6NTj+P);9c|MOAhT28UBGZGyl5Id2R8G0{j`|L_ISq?+Ccn0|@iJ(smfiCF zP*<hz7RVil=#a2-Q!rqph^z%Q+c< z(kP@J-GEcTu99uqqQ~SO#6z zHhDiLVornAxHJLBOzHhOd|`Q^}CJMj)8Lp0IDhU*|l{?4bcAgAV|D zw9VqFRE*C3Tfd!UEC)-}$*i7o5OjWvgeC)R4@?W-&+$hJ$9>sG5i?jhg;@ff5wl{{ zi(|i5F)2)zricnEL))*Nkh#i*Wy#ErjAj1B2M54(!;bcrCzd82E?_4?~9c1 zS&-L8xB!Lm=&vmuY&Sxm;e$Eu!Pn_&^s?0j;qV2tM7l)HSXLEGz2t*3W)oC7euG*? zt_78Cm;E^(=y|ISQ8%{qd%nEcPD~Be@YzD;TnWVw8C<%VAH=mPVWT{A9^<-Y1OqGNhYxF{7Cxu|8!@`-d{~s8>?5f|o6@q3dqKe@_}(>b%5%Qgg4&hLAE@ zhf<`|*=f#+x`yhi-`ilhv}^By0N|0op=yRh7ZU$?7iINr%bXa|Z%k!%x5bPv3hJA& zCrfS_P``_&IIk$7t+zF-K{hh2-z=l~lB;H!kglx4e*v4aiShwhdhgb-D2Nc=e| zOGyvu^Rj5|dGJ62KiYx$*gtA>C^*>dSE*Xz9){L-aDIq6io4{u8Klz!ggeKjSjOi2Y!^| z{5#}6{kU5-*z#l|lZQPANJ%t~L=r6tS~B}^k%cI-L;mHPB@Sl8fcT z=!BX@6y3G)7Vo?xgk}4TNiU};>uV69KG;0_!0jA%V^d)yLSL}2lLc3aEwhFthe*Yk zm%sFZ#h4wH^=YDa6mjDe8aY7^ix< zckj4`hcavMR3{WxNfCUeY!2Fbj%w{(5OM@!*@pdT4mB<(3ay3Qs$o=xIC|nDl}83n zY>`>S3%3agG;D$bN1RFZ zT?%{iNDIbg-}@mvV=CFnyF$dWF54=$4iK@63dMit_weVVsZEm_gZ8%TTsO)}&Ah5{ z1&BHG-05I+%uN13^mtNmjSNQ3R6Wycc*4{lmf(#b`M zNj@CnzF9PziH82UGEi1(3+jdg9TCW6wg`G^yQR@dD0&S|TWur8%TS1_Rx;L*fvbE! z4E*jndCFy1+lXoki{6GI*y#(@q4r8m?bmeA$rL1B0btmUYC0@Yrcz zRP^?E^r$6kR@Z|1wn`K=$}%3OFKzqdGJ~Fnr-7BUqG{}GO;*e&4xKQx5e>Jbgg2># z=I2$yW(!F?g7d|5xleN=AeXqFQFviuo8o*aahQl?L9-28laY-_nGpj~Uo59Z?M`jE zW&kD_8wQl2QdXdff=pMP-sypeoEaY4J)juF8HcPo4T)zEDnv1aElALEQLWGuAZimh zqbAuZ;&vvu&oikBe^^i_t3zm}dwD(=kPS8R{ERFTCToJ(6J|qczrw>823ZCpJ$n?coQOTgxMB>-J+vfar?rkV2H<=YC5ze@$12Pn z;G=1j zB8HL~N;blBfpsMuV6ox*6Arv=_!ZG1=atOV zshFN%w^|>`f6h9NDGL(ir?G=Yuh{UYxED{)g}r2Er_F6JJSJ*VpWN>o&*e5rk>her zM6xhR1jCt=R{c63a>DHbq5s^Nt8omnI@qvX=St6@Hjp7mBgTP$?|ovO9iA{|Cvy3{ z%Sta(xAKxKMNlGWeF9PN*Z6IJ;d&{&w{cPKfqp|>DGy~8=NL}-uj2sB1`tj*I7L5C zEf5m3!sl4M^5PB=st@7dwy+Ve)Tm4Hb+b98jHcHE0WyCsxmx0_@Ww9#Fcm_*Hef~K zEvh7Y{#_M*l9qKtloSyQJO*Z9an-PR|81G+EC~c{dx377MEbj&Ru4tPt_OZG3d;Kd zHwILTu{!(sqO%pSEOwauDtCJd*SOzKM>TGN6(+#spcw_?OMwBcL}|>;NFPOLQlAFHJ|m5!}F zThdtcdiSDb2~J&XrkHOZLTi^ER8%Ug2GgpFEYJIz(t9~-b-00Rqh1`8HsW49}OCZ$7Jzt>TEo5x3lK3n< zWsTIR@`zPDG?47wWk({RS{mE)-f4nl<=M2qlT#4FQr1o&5WSq)$GV4oS)Ngd#c9@DYk=NAaobfV;InvG z?Xe477{)4;q%Idhz%``g2CHtutdj6juyR<$UJa(9Ho>iQDN@`umG`+&v2%lgi6eSkTz7MxIEaxN7j$L}egv%Exa8do@DY-@EU1p6RLX{a5q>~cPhlx9 z2=w3fC+|2TXg2Q~;7jRq^hpTzr^ZSXidw;M(u`FhWoEQT7EKhv=j0%u>8=Zh>vMWA z(=Q?P&g?k2I|#l-V^P<*RMz=Aj1MhDO!j(iPRIkW!eE?ecf+;#0fp1FJgqy( zIa;sACSa1~!GrxN&}KHMglW|lb|@x!r$iL}1FY^SX&UbVT8^JWaMCGchV)TM7O2@J zv*(E5pY3YwXJFUQjZ^Zn%ZRn^PKF51KQfv5Ya4~#310+?=tTa$mi%DD+3sSdfD(!B z8p6~AVrXI23Gv=QttSgw&K)Ndcfk2li73)x9m7?bkT0}yE@W(&}_%ITx$ z>;LJOkAMd33YHlt7`7ZV^(2!FX%j=`XP4g>&-Y z^u1_ncT*Hhr^bhtL50&7>tp}FrR9_-{_hr@?}9UZ$F()^Z1;Za2Lb!=dQXyhPL9S+ zs`PW{o!?ICu}m%CP5$Bc+xXXZw(P5}VeJ=AvUF%W7&$P+5DUz_g=huU*@g}=&1?*Y zT6ec|rg>aqF&C48L2D8UeQ_t^apN+BP{UJtGf26mUur0NHG3>I?zXtA≦@ee;NYBhCda zaDx3}ZRYB(2|h2&^}}Ez4_P(hNaM6$L?+P5ZF`6paa~�RQt4NV6Y-ATB4w{cJz0 zd|*;Yxdu$slub94G10LG#(wH=@u}VToDrX%NGdfj0b4~3BJ2zVs`DZmc+0-=(_ZMip7Hh2<8CC$UT3)%h2|ocWJg!D-dO>Dq30#cWSP+ z!Yh4B%IhRTn4ZYRE@M&MT>2c`%TqTmuylKNX1c^kNC9i?9>pqk*Z@fj43~bA#o8g4 zL?o^u1BHLFVWW~ zzXN%yd;8b3&9UUN^bf?qwtJAijQIGvJKQAP)kJ6ZIQQsB1>2!%h?YI=&<9TZ9WY4i zH`ALhDt;%hh5*3;iKBpi^$FE-2BQmaN%K6EP;;sAQGMh0AZFM`ji6e^`DyCgNB9d$ z{0qYLf4M5)*9cwlnF3*Y`+uR%x?jN4lO+Y_cIs*D4bnYOqGa>OjE!NRo+C&e^&OsJ z$Q$TPhZ!V*vxPk7EyhTqeO=vuL0Tk#!B8o&ny|>y1`}VW2=8z z^cJJzE$`5@ppPuYVOziH2AtNyz;=MbqjHKoTlw|Suk8Yg>ws21WHHT(`&`?BDv!j_ zDp|)7N>SFL&lfPsX4atpWqzC?uN2ek%+efX>#jU z1;_eCo-3%<4GGi@UGNy{$TlF;b{;WEJ;Fm_PqWWNzeYWlt`L$h;BOkioufo%bOG0_ z9$)lsy^3Az-@XA+tj=7I2sV_lgXc;4ubR`|&TIC#HoSk8O2aFgfrEd6ENS%k2*OW3w$O zblFvhq|)qzDM!dSi|Q|o$dCB>h`Fkks$0BTL*Y;B@JBZ%uSRGXLs3&4J68ffgTOB8 zCs;TrqPFvKmL4U=W@f{-zFrxsuUc%@+yi@$N~{2(Q+cfTLV$>d8MEeL8P0!CLgkKfP9+;1D}q#}^w* z5#F$6&}Qz8jqAK}2&~E&8Ra4!w6xQ(==`T+#TFuu`#s;KpxiHC881-K9@r(J1#GhU zo=sOkyT|`qMxQQ>991CEb3|EqcczaRu(-IHbxRzJrLcoNAdfx!vMdBYzM%AlljJx5 z(VCmfmXzjxM>iNzh-sMRC7ihxtu)UXU9axYY|4m!TfCQWQ>>z5V$$6J!NP&`MxD47$yc?z%ndU~^;q>|Μtn z*zkeiHw<#H@{EE3c%f`q?tmDY!tZlV+z`_G{GABBKyk)aJTZIvQbGk;{b;E~wO>kq zQC6La0weuTM6Ae%0#Ewaxhf}2Fu*fF+f+w|irEN|7JRtd*3GT7gy0Ny2EVX*+v5}3 za}It7U^o3GI1*+g_$VvD>1=S!D7+V_Hk`PMBcM&7%G#7$ryJi0CqMG*fuCQRlsV^& zWbE3oYZ_2Mh<_uYwom|?g|HOQ!`nzmcORl;K;$abwDev*K&?Z_P(;N~zVOO`-+^qa zo$Gtb2J!Q+Tmx#E8#&Hv?7ZX_8_2tKb0gKiAv>{PKR2DOs^s&o-W)x#5jE$DB*HX& zo`y)`6mWcAQ=ZNxV})aPkjvA&2=eGxP};F z`FIFY17$&pw3r8IT6roR8&y^L&)QyS>%_l!Hvvr?(LD_n2n~clJU?W}QoxcO@8iZH z9`9hVj7gCqzNic`9zXF!HaxJ!5Uxf3|F#w20(=bNWmsY?8f^Pk18Nw*E$sKhKZJXf zDw>JpjNI)b!;7nyZ^w}2OVm(OW+ge;rJZ{>6+FX#LUG_b#ozju*yB=UdMXv= z7?hYP1^d%byQVY4dB@GCd+P;oyJ1KofK2k~Xxj8o>=WkywBD1C-{L zuQ_@b-1xu+k;ZO9>k;rcPOnbDel47)gKHol6)U#%YN@*_)*Wd#9A=|;rH~-kFfq|# z7TObRfx?l>&j;#PJ+KAL-}zcND8R47^<=IIxX2_Uy&Efo3V#G2v+$j^#5LLG_LCD7 zz5KLNI`)h%wqy3BZO284K6BHUuMi^NsyAcWyk9LIQHs&QI1%9}ku3^5Ch}A+acZ;^ zntLz+Yfa)KY6WuyIeth~tc5oZZ$SMKhRVL801@^&Y-v_6;ln3W*)mQRJ3L`j4R)jl zC3L-{V5Q@I2R&;$1V?ZiMX`XuJZ)gXqRaR&_Oh^#0-uZwJ=%8st0J zs^r4i{IbGz=@lH^yL(rGk*}{;VwI70Q(RbZ8+}X;$x0wMI0Tou)Jk}R+fgLiYgOQ) zHva|tMubY@&+2GWH%vZK_X+da8i>7zU+689<|_7;d7C1mu~_Q-v11mV7<&t{%EdFI zPNIZ;iH&Gt!=R7*;$nmba(mY}asAE?6FSFBAQ3IfW7S7N@8JukJv(23Fz+HT+ML`~ zj0uT2#@HTh4#bn1zo|n%Uo{~OaOLSxO~uuEDl0J?(d)L{9zpX=Qek<)_ZVeA!_6&kg-5_tB@gb8sRx<6)E%{7VyFSz)9 zF`QJXs*d&7RVx^`zN39En;+%Gb zmzgw)ONTIkvE-YDcB(i##xcIL$W}oJp?JwEl`2>Xo2x);sdK2a1F=Aoy+Wtvt7(C75XXJR#UHv4IcHs{FPr)4?4*SBih~#dxkG3(nkf!Z#9m74$uL@NO}dWT~{dNSo2g zMqIq(sujW%sU4?ELC2Jik5+~ECFs(BPi{6@Ir1v2!GP#`)1_ZtIt#D~pNoU*TPJ@3 zSYVZ2_gtu?cm0$Y4$ImU3G5X}+Yp@{vhi5$2$RisEO7IvwxP|T63HY&De)wW5P%MQ zM?MYu(yX&#X}KrjHIYlp_@O603o8`v13R9Tr}F6#nZr>gJ&c6c?c zy923JJLu#;x-bs`fq=H}z!8$n#pO9;q#_TS$5$pcjnihA< zU)N4oRiItfxXx8+=h%X242eLSXsBxFE>G1^#8qDF8tQh`<p($_1{{qfWBJJO4g6cZi2@RgM2Gu1L3Nq} zzOKTPWK3VQ=-DwfJ8w>FJ{Iy4!Ok}|1aG5wxJKW`o&|Q8E`IQVdrO^0L|SuVEpMtJ zbx6%=bJw$ui|=v>-Ux+*62FW9O^*4tufE^0sS2RJoEq|;kcwcT_srpADu3eK!*nC} ztQ{PCDc4Zhg6vwj3F8$I$E({Y=0oKrhEkz^?+D&6FfqAQDk5-|H5H^3-e*ggeCjv@}dAD9*UF%EAzEEuRYFaOPoLU z-Za{$D))br=%bqbk`;=`ko9vpN<~;!jrz1oPeu+X1Qo3BuL{4kh`Y`Jo2R&l5Cbw~ zIVp1ru1WgxLWUihno-EXLd&$u5hCJ$@C^u06-SWcuFTa^2vvaNJwr2>Z5olNYh6BN zu5%)cFP6dhr86*WtZ|Dcb5zm`$RK|MNhb!__erDd*P)1Wg-RjYuR0iNO=3%*Tm5LQ zZdP4ht;`$m^EKv03VMjmj_LBsm_r8Y&U!$bjn)nB{&Ue+p9dz~0NBT}fNb@7vl$~h zvsVZJvg{$=-9T9O`c%%INd>1zeK!S`xg(m*nm3jfIQ=NJZad|tB-OUd0X(}i%gbyV zi%JSStc9Q`8{Zv_Fwhb|y6AM%`UY_oX6b{F)5;3`(Gf#AGSbcBX2y06(1@axcO>L- zJ2=WikUQ2=A_&dcR~mgp|-;d?ok+3CjMH;%0+gzqYQ~^2wL2;}J@8Y5U&p`HLYP9e?I>;n>NR zsptdYD~{P|`{p|H0X5ErZ06T;sRJ)!T=*xod2RGXn}?2GIyXfPkG;6NIC0@Sn)IYJ zern8p@>BqDf?L;iH@6B#?j^;*D44eXh({l7|GP4UyLsHz6Sq7aO_1lcT2K~il%`r0 zKD2(<)o6KK8#g;~ZLZF=d~!%(xXEX?5#fE&RM?oofx0h5{bvQ%uTCOmwya3k`5tUE zQnnFg>IzF;7%-1LlOY6e&-yP#j4LNFFqzD(Q$p=A43hldh?z9J(r8pMS!KdZV^fzr zS){--LZiIAf5_;6JQ(gWoI76c2tQUw?1Xzf-m1mHa4rD9=h-@Ig_rT~UWv6+b*Y{~ z$P2%_IjO4~KWaK`<+VB+i8a#XPUL6>SjP?U4EQa#e>Yaxj#zwJwP+>9q>V3K7@+4& zXPEvH*j{Bic(e@X-Q#A6=I+DI=%u>ZX*IE*By!Oi>6ExPV5ke85f*%b@q+sduU#O29KJ*9&+x&uvi}#JKxq>D z;Q8XdE-jSWIIYhMpz+5b$aPvI3tFJTzmhw+Th2U2_?>} zI*pg{=yfSaX@&QSv{_de*f7N^GF+_{16~(qHgOZYv~`+EmMl@RD`6PHFWBYwxhD8H z8Spv7^WT8|y{ZiphrbGo460kX+N;{Q>L{aL0ef+%q0~7ntx&&{mNH@!4*b^2>Lu&v zin@-zZm%zv;mT&<$uv7dWfbUncOKO4%SGrAy>c_{1?D>FmFX7c z!(E5Iljoo*zahvu4v}yuvg|U6v?sUm-=$6($xG#!nB7mJEF*{}td6eRyQBd+rDA+? zca%%;6}WDfsSZLAov(IR*97`g)7wG zIBjnHzIUzL#rWH>L>{D0LWlrASzE6c{8yxL$UlKMwN-f;i9vG>0Ts@qMby$@n6r?| z$_n)i-a+VH=XX6*vzN*clymPqk)3Hredd)*IMjwN&;4FpTqwe<5eKti4n3=8rDIcH zcr>8|F;L_Z-qc|e4SkvKUs|5vlO?pu2qtpG;(V2crugtyx!2^It#lCc?To06e4tV# zM1v#Y>;>yzt|Y;`U(e3A_)6=i1XnB-|H+1OJj93Pi4!9r$UL12*slz;iImD-AQxx_ zFfk?0VJEfeEs4yyC?D9cYP-w&Q zR62XWN6Bhsl|mv)hf7Q7)enXEGGWKqg*7X$Wl`c{JFso_FC$=uFhOS-#a=IS*7BG`rJ5`U7Qx{kPuULi*;G02g|viUi==x#=KxI zqE(7(Pphp;l3h16pp=hw;j=3{Z8$xjPiFBG%m;D|RNzdKPnAPVg$hHV*lc9Xi@sMB zydQrNRGA?ux8Eh%lpTh{GWw-Lka6f^-2)uFVbp(SN4< zzfH)u3VuR90<{7n6&DqY%P2q=5%(12Rk#hmqF9W>vg_}5feW}$%XVL&~q*|Sm!CHv(`T;z@@+IE}BP%pmcQ3YY8_Y zQ<6&DgN7o#*DK zY2R#Q&QVLnJqsm1D;b*eAlSP2agc@#@I|#@4(s*#^0S#PTQq|^x!t^!j!22}$YV?5 zM@qC2FlZZT>S9{N#nk7=4#BPv_ZADhmlliPvS%xNA78~sYZv1xsmIAS9>P~*@F0%I z*d!8Kg$PA>?y~#N2OK#tIO!M2_O9O?4!@yXmKYh#1O1 zpE{;PNjH~Hl*Ydjvo(Ess^yh;YK@#_Z1+%`HZ?Z0u7im8a0de93=TG5>C6h8hykQ8@;BGE=jB@go~2W)W8{4)ci_8dhb z#1}P^Fg}ptXNktpBQO~ZPx|ZGxEiW@{QO_KfX}SD1GwlLYo`uAnLTv;c!88|n>vaT z!4Yr2G;D_^A<><@hIm0DQa)?_U8+gy)Kpk$9U4cmtkqAUOTQCgf7Bzqg_{oB-Sv{q2Ls4oM+r6wrO-< zv9VAv=Oo;6W-*KmGaOBSy@ak&Ydk$Efm%AacnPDHVR#f=I+}cGq?oSWn<)7uJb7J% z@yK5~5a*|aO*@K>$IUck<3w(l+)X4{{bn4c$ZVm_<%N;(Wtc;|M}JRK6oQtCQiykB z&l4q!xjJb33h}tdJ3e+sySNVt6H)K-rG* zDW`dsKXTw-bG!#?Q>A-2Z(0z8ZX;JultfC~ZNV7F0zXXzo%bByl=xjRrERL4y}>5- zvHxXYR^Gfku{1vnkJC`uy8p59t24gNayGxGPPc=Cms|vU_>(w2&>ku9k%)*a2{6l~Gm$Q0E2R9= zB(SC!>N#|SQ0OP$1rObwEUS+y{SV^4;zpWFkGMv?7Ck=bU$|Qst@ZrdKX=yRrbZ&j zlF@jf8wX|9WAlq-B-H5(isi~0#zum_B&ix4S?eLmi61QGlY%@*Dok-BSbQ+~sNq3;5|-+r}C z>K>9&9I91VTCKL;mT*e1y_%`Mswg4d=OMivD&FUVW0W1by1|ITE%m*E-n;m9hvM*N zgpZbU$)J*RMrq}kZHDWND@9xllVg4X6j~{AYz)2``z|YpQ9Zahk>a~S85Bn&tD&_J zz&ag+vpAUzVRMpIDCe8dT;z7EzGn|{{R9amqx1wj0^<>qc~7okd#Y*~HE=qocdbUv zR-v(dA-Rr4-RFph05P1m^+QN}Rf0*#o|l}Zvv^z{c-X$E0NTUR10~5cqmj29ISLse z`KV%m(cn$f0PXV=w;y)4-G*GU4LNWVSI&aVap~c7EjOw%DQaTWWs8i4ATYQrykgRt zaA%BH*OwZEqHw=hsgvRRAWq5|d1z4nQ!Nv_fe#^CAWl9{tgf zF`GI&yNFT+vA8U}ksif*l>u#P2K@Cw$Xd7;aYFCI7v0ggqkXo$T0hB=VJ1?(ro9vc z4&?ht!Sg*{08PCWOxqjgagbub*vEsqkqlPXSG@~YrJb7i_Q=}F8M_|V_Dl923;XJa zmHP_FSmr9N6#4PmA8={>=isH423r*>Fxu?VAobshtWg<9$p=1pe4tAg$id6JniI@% zRmG@*_iAC-$VGt%6Y<*wZ62GnUW;gz?_pnPX@rzuacmy|)=96jkKSe@J@(~Fwuuqx zsKt=*Q#$PWHxLG@iSSfq0mM{{!uJ)}mGA}>cwXN-%n|Zei?gV7xcPtOgxVqnyAnT6T&&*vMLVGXhDeRJ! zD67%Vod!n1MBhHE(V%6pfEu3itmHy@XjW`@g75dW;a-nu#Q2`&b@kPd|EODLckC5%8CP5`qLp~-*)-+DY$GitBiM_9(S{xj4DAHtz(&We*Z;mj zli>SU9!{hRuOiN~ki2{GCy$=d?WwjCtXkOI8m@CxB$Q3?$IimH7$qKLfu=5~Vt6aT zRrM^rcI|Rz=s7Ab&P!9L-22CmO!fkIy&i=&n1eu*v*J**?Y6c{6FPey2Fhf!>jm~Z zK)i4!Vj&TS@g%3ub{;=mz7@{ zJ(DF{S!t$q8-1VCaBZBsn>)y=PX(Fw<-J5Z#7173QvPq3%EG*Q2jVGM;WhxXjDy>$ zmZlZBZvxjXLm&At7a%H4*Wy9FMy}`~8BpiaUZ=?w@*Oj!)|Cw~p?Sx;kfa4viq6xGHt*IgY$bHh*5V&==F^D^aAo6 zN10zPhlW4J%)UhY0?nv{G#X+`sUw-Pfe*2_Duy6Lbv&;bgNT7Wg9F(A%1hk3!^`k) zaO@gGlCp3G0HX*hb-6Vz2|mT$VRo(4`lA-{C8+`lFawkM*ulN!ErY~cDSypPj|y`56o}!TtsIo+(`mO()`KNO#MN>Fr0b3kbHibJbIwBd6DTmU>G;Cp5fsaCn9bDVVJCc*=lARX z;P5VQNw4s|d>Y*sV0JFn3TN|eTtaDIt}K4LlYEDb3Wzpkq;oq+8aV6))?D6?_y9v| z-`AQhU(o4U-}fG|AhGy--LH`K;jk?*89Zk=>Be7-05l>JxgUc?q4(V0KgiGix_DZc zP>6ZR|IOcPn6~Txv$F0nwbu_o6m>tT6dZR&(r)y}6=HMhUlc1)7zxZ%!Lt8)2lqje z5?w-=fmGl@^o{sLMve66cgboL{N3p228`xld81Ex=8uAMz)1O*+WH&SR$ zk+n^pIU9zy>VH%-~w|5+U+Ff`?Hi8oM{fY zc%A*EutO^0#-9Ox#my0AYaxu%CpX;ZlzrlT3^z(y^fnwhTs+*ev9Rsb)?)fE6KCr9 zjDzh{ncwhH5%&`vfAV8y%7_O~gwmk0j$Lq7Fseo16lNb*9S_aHYp^QBp>GyvEbH%1 zRf~Y&?L`WP5MzMMo;&rYptvF3+Ye;YJ$8y!{8)UH#uKvGJ+>K>`L?4MLN||9Eb$S* z9Jy%iV~FyjlIaRguRzRJ-sNiSoA=o%_`LBnb%Q39BW4cZ;&aM&0~ohr5US5u0_{vB z0mr=LsQMaRgMSvxF#l$owwK;WI~IUC`{i@VY?(&srYc!y0;yJa0^GF6PZ@cN_}vD0 zJMCEJH?@`!8i_hm##XMifH3vWcH5Y|=Jf4SJ~TJXXnDJ{Ufn@3VdlmB+q~B(*Wp4i zN?|o&=#XSP3r~B`AHt5ssc7 zd-XeuC}jo6!KZbLDCiEwn8<`=m$eSc=Ys;DP3F=4)%p7igtjK9YHJeQ;EnDCu8@5? zG3MkM=}$142~Sd$>h?~k?(~o{Tj}ErSYhOADz7hp6%@8*{uy0xov9=UE5zn(|k$Ki2kFL*q_FocGY9^^f8ruDI z-0f%l4QzBG91bbR3(|$|28|kbSps?*|{n{WPHPIeA-9O zN6QmF>)ETsBH1;pUencfN14X*?M&X5GHc=!Jk4e1^hKd)#=BD)vnZ zjhlVs7wBe9_x@S-P3>2RKBBbx1Sg?vcd{0kA9S&Y*m6YreQwI(%9`eYlAKU7KqU)d#O!epX=e>^) z2XQJ^ko>dWko`XmQW~VxgAw)%2bt;*HXR(V%=Zvy?hFAq$ZetdB%vG|D@{l`=V~~} z!p(^9@ST4Hq?-Nvz13E$6B(iqJQ0gO1U>+uI@P@7fPD0cvflLOI@Zu~q5Op^qo1Lp z3v~xev(qHnAVqN|0PDpMre#^dQ^4*n$d`~2<#QRWQF^}du{o;g=IT4*U zP0Bt@HS}6z%5WQ`${d0d%3_e(-HogHF`s+ zQ+)&|P_3<6(9(g$pVlx*|6*A7X)1tgBXl(ju9_rQZ^~(dZ>)0HZv426125L#6%gyW z%u}kRLqVd2%1_VA3-sUD6h?R+<3OzLgK!7K8-T5K)0)1q!PB2*`yszm8#&s_>TiG1 zJ2004G0*lmy-<>tWhT|1sbrQsHM~+neVC>zNIb=@>7Wb+KJx>9OltYRf|La5YG5Z9 zp&lYeHeTsk%P1_3@o}5>L%9&MPvDyn=fRL2eN$~z;otqY34Vq7&xxqV|{2uKA>sY<5Gu)v}>ybtUpmlhkKf=jNczjW^tv4i`yO2G~J*r z5@g~)>*8&UTFaw(t20!HdXQtauOj~*pC3tR?hG~G9XG+TJ*EK8>fF@>(E7XZ zaI&m{f-U8Al35NQl`DjIN1zx~D(;Q0YA(WkeiL;;47l#Je5|%9zIcFvuZ^toM8MCEkwmo_uU6)0YL1f%ShLyz+Q_cj4CgU88$!KZ;#pOtQDKr8sZ;%4f$eEWem>r9b zUeQwc(P{K8KHu`R%MhQxR4=&#})+w>FHEKTS`;5HJuCX$#{DG#+U8+g7tcsYO|a&lc4t z2XHN{EhtT0(wB9j{)so!8h=u*-FlPz#(RqAvp=p;m^b0KT2h#@NykbY_PFZy4bM2) z+UXyLoP^I*qi!9?TpO}*#W)~x3*P@a3q|Q49NWsND{hRO!k=X)1{C{uOAp6%)O(|w zRYKe-JTB8*LX&eY6z`oEL*k%!waN+g#3-9>SR2HDyc z-Iye8rh-?81Qcyk^;%1+d;D9N<6s`C_tMlF-h6Kok(vS;^p5t(AJkt1uEOnxy4G}J z&V9&^U%ZZKUxj$M$Z>izWWwDE-&y!FY!E_+$=7nX`T1c=F#894L~a8@?pZE_=vAt3 z?$>dBp6KDt9LswJ@jLWkq|@vJ8B^Ojux;@-T32p0d(uFGLb;yVp!KE_H@;<)7F{pd%)CiMg)a& zXR=t?_n)KZlYmb2{UK_AoE?o5Ls@Lr3qG#ECBtDGfT@~+SPTzT`4kTU0tlWhs+7o+>GI6lj^xBORvD1YJPr;L{ZK*y@gR3%QuBf=U%(7@X4=RnM!IjY7Ull@J zb@kR5M(M?npVFp0I(kC#nNM$W;`mE{r_Af^YA2JBZx$(3PH|k;$O&!J^m*@Mmf8ws z)yK~NE}e(+c0g?P<dnq&-Y#UxS1S5)68)Nqy2DNDnFQQ67q zu7H0xHWGp7YleByVv9Tx@Q}PX{#7lL-Dyj$1(9UjN}TX=n2v@Tz=LU%tn6enaT3K| znyM9Lydi>i0=^`r59B4r4sW5dQ&^+!KBk9$aKv7$_x#Fo&p%kKKD##>sFUBj`Ux&; zwuavjX!U>lU|Uu#IgVNCq`TiL5qxET%2OH_?tBz9 z`f6#9PzE1ymCus>XZ3P|(U9+)ZJf|YA2!UvnpuwAGfF9{%34-E^nd;Lfh*BDjLp&e zq*HB72PzC|9Z&o0yx_lVc`J{n8tVO8+?!y?h8_DSk%xK>;lk#MECwx^$62CA{|^99e^B)I$(e7$^cfS4JBZNAm3oH=+xNB|2!nczsk3i(1Vh8<>h z;7P@1bc?Z?UWeO!7oM^jCGB4boSrbjn8e>TZd&%~S1KP_P+*MxKV)o%lF)$~)f1|n z&-SO=&Z0p5^yeeoP!uK=6m_{eo()m+C1y&0_qP`EGbOys{?Uxw7Fwj*Xv$qUWHkBS zW~lR084JW%6F}B|fk&@%QsAt=>)PkB!f=-o@Kcu}djmfttMJq-j-6f0^Kx@-!jk;5 z`w|WCRzhUNjXCNBoPUD1S?F}`ii`1&o1tUCP?v&*M}S3miHm%gLAD9Ou;tG@#phyo z)1~7w7l?JmijY6gkOSU_zSBnRTLVECmP3V~qCV6EUqp$4BW^=456~{U)rQ&|3EeZE zO)X3i=KTZHbxULL5}M}Q(xyNDW29Xd!T3@dAJu22PR=hrx|M@Amso+;zthCK4hp9> z#=RQ<(hASGdQAr7R6L>Z@!2}!rH)K9Y(Ltz@zmzW^%K}V3KGaz;gCwZt6PnE0?>5s zl{SE|bkMDdgeFqKbq#mpDLg=U>RI8_J1QW@zO+Cuz2@Tv)F$0XzZ4k;0{%aO7|)8! zOmMP8pu^1hEQ~Q*U&qZrjo+NrEd@#5za)ORDC@JgK3~bhI!WG=J(m~CPj7Fuvr+&O zF!-jsDKju^L>bax2TOas>ZcRkF+rsKBTZ5{L)i{5<}1F)_ufAFI4Qe~XMTIwY`y4U zw!<;@dZ^9C;XL?}SBA|eV!$$cNLSdI;Dc3JB;+co9_X+QuVyFW!~#Ho{Q=T2Az%tz zN!{Hq7J>3YUHnZ8Yb zJdM5iWxn2#ei6QU=;s0l-o)^P(yaQH9VHOj^+YJmXx7&|7K1?7oYPc}){v4nVg;^? zpW>BAKSIqWoNUyqydsi)-2x~vDd(v~&}YbYp-Np4)@}X}@rYaby1&{?ddGmR9*0M6 z4H8UO&rwweF>wuz3%93ck?_qaKg-b`mjOWL1|!oY{;IoYu`udrLd6$*i1@{UYHyFD z0=Bh?)7}9ap2kh{lJ%afAyMq?fKmnuM{D+wJ8dZAUTN{9KMQxaGYgQlaf6Q_Q zy5-Ja)52sOh)17if0ziykrB;m(kOr}>W)dMAIV?)wGG5xOq0Z}uLUi*BWo#%bdbIy5Q zug~Z6I?p+q5{H*S4xY*=oxbzOPbhsVymjT#Pp`D67ToA4+{N97V(!cTz2ViGlCAJd z`U>1bRq=z-Sm?KFwWfvEuW=W9$pidJO9H-3bqt1_Yi;@~?yKQ9uidNmt zoMt~xago=blV6y9Zp>$gBcb_1NvZiohXH%!d|Z%)5;$0vQazuo5AhuiX(ykI`epE{R6 zc25-beqt$8zhjll+UcHtGvl9)Y}^e)cUgB!!7jB?abYGmbAie{?oUk^XtV&etlL5@?#fY($2ek zcD~i?>qVa5KKqLPet9Ex47tCl@H>aFBxmi&xZ?K(-t=>q8g8_VWl3kYY_fQwD+;oi zev-aLwe$_5^JjQttaon$Dp0opJ+b-StALQ+ll{|EuE<}se=eIU#~Z`#Yg(#GpHu-)p?J+Q0c zCpw>?R#Sq$Ht)URael+hYr(Z8BlY&$2XM!c4+~50zz!}ObJ(6oxi$lje?f5bL&-cy z%Gd0BJ7~`{+n9Sb({C=T15g+6y_;;}gzpUe%Sr!T_eAO2;iXzlNt14a)4?aNwqUbB zGhy=(Hx>ncnD3+*;qDUNs`>-xv1FcGee9lx5A>#)O4z5u-CplV)pB|%K|a57s{Ic? zq2gik3t^Wg>T`}1j|bj(xpbEIj*Z`=t|bOmFr$U*tbw~%W)oHjmWf~6?dsYsJxFvDj6@w}+&FSpKlJQK zXSFkrl0l>f?re2Rnmm=csn0KO-nFOG9=zM_qqkZ)$Lhx^7ufEA$Kp&;|NZk9k#inx zywk=})jhZHRftB7W9rhd>e@JX{=-fohj$0fY&GC#ATpIS!A43 z_3drJAy?|V3;PGHPb{ol4$d&+!&rea0EO$ghoC@N5=hiqd!@)T{!wY#o_(v z-h*~X`p@FOFWhf#KapHt6#Y!4uZe&9-~1mN`@bDY%rm*dtZ3x2vS4-6=k8g zoq*`1wv&!~zg29`rDd=;wQfB2>Tt2Pcv%o@58T8KxBt@dE^49w^T!VyhnAfSH{DcD zi+Z2PEKG^Ep8Y!cldP+8-x=}zgSJE5w^JuBZ4A1Co^j^VzLa}i9lMS3L6g%PY~Gs0 zW7m#=R`2@ZzgFBxNIu8DU`~y$ahZ9qtHGm|#f?@KVlh^*b6|t;=j#iRzzeSbp6+y+ zee15D-Gz&9Z~R0Yf*A8}xn8y6B(y(zGGfS?t!SQkzw5H?(Cl` zh!|^UH$!#FI%C`9so16pLG87k^4Q}KvvR18p@#!PcUH6&jd(qhWIQsc*F+&3Hw66r=Wfk^kq*hmsz=SAf@? zvr&!K&Nu(;z z5AXR4Rj+bQITl`@a z=E$+-^WVlet-5n}Tc0Xu?JnKx1!pS?*4`St|MOK|=;kpKNV4Rq+9Nge<>jiO2V#-q z3(ddBPuqT#=AK@k^I|(bXrs#VM@aM8s;KLByV5>g(EBsk52Rn5Qn}Q@Y|V!8;#b~J zp2Szqd)d&C@nPf8(1VooS7vmZGvb`~1kwdjLRG`WCeE!RS+CzLuiE<}^%yU`4G>tib=u`iFCo=dU6aR0{OHfZ zwqJyNJ@~9RKWae%Y3ZB@O1jwY!H;J>;v05jH)ycMb?9|A((#DIa-X2O#lGjT+M`Q% z72i{yqZ=n5jt4VrpFgX9KiZp*km=ZnOC_IQ$TFmkp#;~ke~Lbp%K({M_o@S%lr@NSwOqlw7@DD_$AK z1Jpj5pR{Oc5W zd5zFp^ie!Vf@r&0^-on`|8_eKG<%cJ7SkGraEG7Ai>D&znX=YZ9$i=B_Wp|hk5jdr zPB^xs^vT^h0k=N9lgud&JR>19?tNR`hRMkvKj5zRM&~NNZux+uepvJV<2|lhCBLm` zDmeXwd*iO9g_B9Dh8bYgy$Gb!2@_}lX}T<%|1$IQ+{*tN-_VZ}HdJXZpR%&PiN1uk z@vhwR4$Jl@;i(5M->EOjK6zEQyU6P4QHQB%XZ|d%^jMdDk)B35BD9Ixu(`Np!Syri zHbJB6osINZjyM^=^nv%)p~1jyf;@c2^1%$Nt}g}1NnOle!pW!Ym;G)B7rfHU z&NLkU)Ne7h%lWW}!8lUYu4w;8I6f5-wkP%5-NDUHyP}hpd?GIk-mT9R%y~QK%E|We z4W;hN&gN-5E$e6Woqv1fapjxHk+s#*15YlMiKe`s91kCT=2%5=T|S?LS{NUF=9y{8 z^TzdC-|l8_rF0)3c&gOGM%NM8CAAK=yhr!F_tSopW&qJU767r$~E28K&ujb1pea6N%iAoRiecNBhM1Fm1wUV`4$2eP)QeLNSeMA^# zE!`092eY^wdvk;PKC7RqoI+Di4sxA$rtRhXL1PHI&^g6*g9m0JFMUq@`g+-~Jm8~h zlY;Ei?Vo%P!;apDc>ZF;D(~32(EL?c)wF)&j<>x;4*}P9tO4BEk+v%Z1#3O4vkE@&J!;-H3`LD%2dbqTf`E%DQ zV$j(;8`HmVmpiNgXxyfMOi0@6i&NQ+oY{9vG}Id8YqjM_^i?}oi!@k!?5)LsP>RI8 zdBz!4Y|G_O#?qgzZ!QrOPp%#QB3)7W(z|M-edPW0;~MzQmfV+U+32GD>xi($NAj*) zYFr%_i#~7bZ+WHPql)h5Y^wRPVB__)uSXgpGH?DlpN$ny-jnTkmT|D#sj+r(dic#N z_Zwcl{@mnS$Gv!q(t*o)*tKu}=~>-7Z`e4eZ(nw^CHah<>jhNBV^Bv?YT7)G@?oWw zQnFZ&0*PN&`1F76Ssvqb-kc_66TR4W&fPr0#aE+Z*yyn;Sn~mR)3KELjG$v~^Yu0X z6>k_%mjnyn;Rn2~o_*$ag|4bec^0tynAg9wGFBWdbH8Uv=kS$P?fy`+H`91%7GxB)=U;k%2-t!YANKxn_t-J~+%JHe z*DdaHO8jO%H2qxwo%IFZmiDl==Bj7T8@~$w(D`S-s%NzP*^#;kSIIA{*Vfa{zuCGu zzvr9tA5@F4_t>G3(bMwkmNSNcHI5Be{`0%En%xiCdjHudp>orrAAD;4&B2dDoCifs z$lYS&S?l3@_i z_%>ke+FO@nz8J4BgT?oRe>*fEbaT8O^da%T3~B$Z_F~P;*2fE#2b;5|J!}zH?^>`d z;>T9!8d7)De<%1f_lW_I=6X#10rXLRT}ktr&W2YGu73|kRk!#IM_1%|$K0=_3@o*! zn|!tk;K_fZQ-pAF**(eUyNM&ituI%5ruIx-Ykat5CviN@I_t|5_Vve>Ago zKlb`CqbBG})7e7-gPDI_!)G-DOzHMXB>Jdue7f4@L1j$R(&kN#=gtI|-A_XAK5qcn zeYCFbw^;kw(WTxs_jXsgt>toQyS8VQ%IBZZtiZ4To!ohE38`~kkl;r=^?26SqE~f(H@5!j zej9yU3KK*-p-{KQHm_swa@1k zMB_0_nV>|ImzOI)+~>2;qx~T8MilI z;{0OY`L$*mdAG%Sm-=nftoPeJxk-;(^lbE?|3$1IStwZc>c`K`>#sdoh^zwQb^cSf zGd`{PX+`t;FN^8?y)y9D%FR_xym{C~G3 z2vDc5RVBP7P&akt-Wq0LXL{!`+{&dJ^HHHa!*Q7wVgDr$tz7a++kO3y?Xo5KiOAe# z_}wc^Ga_1cW}{!Q6!J~YK9u!&zDYT!@X=N&cao=)n~wasbD1<|%=(xdNs{`WH zUbmU4FF1#Vx1NY9o0AT4s?TXMu7=i*u*Yd`l6P2Zb2Z$+`m;`s2_?wrnyIqcg`*rCJWuU}XUPF|YUXBzFB zwZpRaV3Ggm$hG+ECHV-4fw|`oS4~W{FAQAC>)LkNG1jl@gkPtn61xf#TSMBc^0C{< zTv~VLNT!cR#p&geq6>qTXR3}rut};C5>5uOB29^dq^*R7MM}|Dsyl7)Q)l}dFO4c- z=S2T|?~1nIlE0&}n)R<;H_m@w)Y}ee_#QQ9+RMa8h4b!&EbUs}I*Y!7`U_HYslr^1 z?#FFh%Kbp(Q?*y)a$@mITlZy=@BLazJNMkNYGQ&0&F@Peo~60mbeMcU=i{3dh4Z6p zcsJ}bmf|dz?DXFGF!0-z&1vVpjK)|9N69e(bz+H$v`_u2fPVCW)yp{-m+YfItl9fY z4{n#(rWAeUeX)>rPL5H^;?p{t@%9M}s}MKGXYp3gzVuIyvv(BlYAp>(s;a^B#v;B~`>7vWj-pwl<3>41;Lo-{~Eqr#7VO?oedB~o3Ky~5^ zyya;C9%EM|*mcS|<^1TTwH7nyEdaW8q!My~F1nWf!3$0y`xh2=&8_jxs_Z-R{2%TC^|*pFXsE!(+ru(Awx=!4e<-@!?0OeD@3@r7L4znX1|!OI>yp z<`N1cGjE4DyR5Id?()0qHTm?Gx$cX?`idGs2HMQg>p|QL_6L(kbaJD}^lW+jg`uGrOPt0@Tw&0_cDdxj&9(Bt+v+Qo(_jtd~c1F0n z6w^Ka&}RX%yT{h^*v4k(bFVi%4X)0=Ofr}|yUgFYc;4>BGRU>b?N@KxkF9l@?REV` z5UJq)yCaRr6s@aZ|Id42mvqUp7513uh4EefCv9$zyeg1k?lw0s#Kn*|R_|VEgQ(QAx)ijzXv}0u0KX=S*MzEOnrXrNblqAOIm5p|-uPMLlmO}!NoH~YvOMf|z?=X3v= zb{zxlg-qb)M^XU~jLsW|VhixI>^|T5mxiDFM_|s1-$Q$Pzj&Q5?uX^~-Pl>{qxcUT zM2J%hrtg`Se0p*DAZk$$q3iwwQ0MlwYwD~D*8Pt7eYjx|GPo*kHR}!k7&V`?=*(Bs zZ=lb!9rxVa%6DCvH5nN9uqWiyI&*%{sl9uw82kHfMHa{p8#i%cr0G1(d~Q`!rlW z{D}8;)&yej*4}fS!{8IXSGHTtLjYgC8`alZBS0F{7-+@8*l%R&=UW90BBh6xP8%}U)ZT% z2abi|Hcj3Oe8N1kuDaV&mGmkoV>4&+ZqewkD=z9203-Bxl=F^#PHEu2$N8%8AUd0> zrmQmVZU@}YIjwmf_}`>3`srY9fRtC>{{C_Wc}(3ej=p$Ne*WG2zmN?NI{tX%-{!V6 z1Dp@YO|?{s6yW)9x$@bD?13ZqA@eQ2&LhrSSH16AeN?r2nV->lq&1&%H2U73O?#7% zo^+TSw?U9>nA89i}av9d_N?6 z#pf1o^C7fJR2OROZhLOM@U#AT+&+kA_~X{z39FX+E}u!3?yqD&%k*q0l~> zb!;zqZc+A%EAE1Cf{ZBU`1k=k-y_$*l{G;$o|G9VK zCBtdM0H)(;?|06iOW&-z=6Q~nAg%er_MY~-9PRdH&c%o7W>HfL?0?jL{e1iQSe$*W z=9uo9diql3!jYW#O?z@W?L3MWvrgb6)+txb@&1mU>PJ?yD+=^n$x5;KR*wla*t9ro zl^vtzlLQnDc}2PQs6U%5pkM zP#_OQJ;t#LFE7sAS>ZOXlG6Ltg3-3>=5Xx0k)KUviy7-mWp@|PY)S6qH^2UO!Uym0 z;TvdlcSqe#;Ol4L+sfCMz7PHM?@3s$R?UPUeD?mGrwTb7$0jh1O5>k_{M)8?>ll@@ zj@!(ltl<@{xt_bAwKC_Umvz*Mn(5u=$vEVl>dSM&M1PV04OzZj>8|&n)Ik@NeHx0L z+z^@2^QpD#^!BiYB{?5j#&2p$qW9yliLQ0O=ZtPTmKF67Ur>=xJ-Ddt^xyfrnBKAUJM^_FOgv)a8iy?*`&Yah`n}iZ5GWj|dj4omG;~$g zo9A0OmadlzPL-}5$e=RJy+{pvLyWn&(O<%O`<~}M1MjuEL)!4Dbo%dS(xNKX@T1pH zIS)(6PnWRyv#-rSu9~HN^z9bppd+AS)3ar9koB|!_!X-s=d{LGABGdlORB$q?|gv7 zuJ*TP)C)XwPN+`oy4y8-Unj3B_CdAFz~X&tZ*z9e+tZGkz04`X@5uV2)P0Fr>rP}Y zJ$t+NZ*f}94`*+*?sKIZ(aZHsYrkn`zI}XeW~`Dtb)bCPiO~HX6R${C_{`yc8_L)< z`}g(fDH~_t7q4~Ne(~3#m#>8m{C!t_y~1DoI+?zCgSYG9m7A=$7-#H`KmGaTmbj;- z&%PwRVeeO7xPSSext-0qIqMh1Kgb8+XWt!t`MNd$bD_rL!^b-pqmI}W;pj%4)}W%_F)6D$HQ(05U+^W8ipE4`L~x$ zJ#gF6%&g_lyStY(klFB)EQ{CYV~N(+Vdc-}l2=y+ACGBxe_^rz`SSO5v(f8quH4B_ z_;jK1&|({0j{|aqo04`0 z_W{Z5sHhNWYUbWuWay7@UL)s2hz_uEq$$@f1peU&vVaX*w)ksG5zK?$ zA-j`_(p z_dG$lK)TlVPHy4d^j?PjyT<*zfP`Z!)fpcTUA$o94qG8Zt^eJ(0TtmHcYp6{^_vY5 zaqh@ZXAacVtysBF2hRyd1}hfSf8vXGbD#TnE$HyJwEfh@S*;F9j8G2|w@cQ$y>g~+ zzw+|Be`x8mv2`P_Ub(mBS~c|W-#n=EIdN)R`P>-?@7G<)&-hW)8Zt7p#1eD;(w1BG zpno^|YTvzqoTanAAC)8CyK8krX`qgK7ijkDSj$N}#}ZaN={x#mW<0&IvghEyI?A7U zdGm_`PQE>ekbxsCR&`#BoVNKed~5DjKxBEqXu3^C+WF<*+TWf<+@mk&Fu!_?OswkK z?>o^@KK2d$$7jH2VMZV++D7pF(8|@ki_P1Q9$xq`{J^sFvHkW#0r94Di;|mE=)F`x2BI5&J+(nUx<8BE;SA04WH{2Mkep5XA?Vo^tAA}rifaK0Px3u|S<*gfj zP@xq5ER8u)cJplywR}t0o}#9`LpyuA-+StGyZyjiM?{seb61XW@zxgylWGyuq`$5_ z;V(}ZA-;UGx)MU3lkqECIeg}KV#Tdp(FUKMS4EiGE$?rvc%OT2T1^e7 zns4LHR1O4M{+1^Uo|gui7t=iT+s_Qp>}8b( zG2@uU_e{V%Arv#O2hegM5X;FobCnJ^n>Co4XC@rbrf0GjW)EhVVSsX&70_*#F)8B=H>YS?Rq9kC(Qg)3bRQ@fb#*#!6zY>Qw*TZ=H61^ zWq_YuJ>j<-&~a*NcxSU3rUEEOuoy)*4Vigt?f5W^5I95AY7~ zf+*PxZAO_$G*73P*D(qKx=u}%HV-!sPrzaTnZO~?SSgGGhzCZ3r?PrUL?Lm3t!cs! zHrHc}V>YK-@Y3K;k16xzR1Wh$0E+n5$8oA;@>l6*AO*_>z-EMWRt=t<4(u@l00%m`n75Q=7tqbcyhS#S z!O{U?uqQrdU;Q-)NH8NT%p3+|y=DV%P7KxpKEu4RUejX%sjwE?sj)PeBVY)Huhf3kC!Q zG5{VJD;|&w@WEKifFb|{W1|C#0Z@!B)ItJ4V(fSpQa~8SUS^>JpfCATef2op^gm8 z4oniyk!#t7NtQW^EVY;vouk-Nhe?GxNi6#@(|Ar&%K=Q9%t>Ww#H8z-)Rx~dXsENs zas-pXbJkjpV=`sVMoSYWOXqB|1Yj{x7cdBf#qwMrAX_X>=7Iu&v3Q*e8svf{KwTLi z4=j=A$_4pgNitUv2!bW+T*V+LmI8H?fRI=!&rJ#n!_s7KDi8`w*SV=du~-JwT?0zQ znln_ipj0eN=57R`v22~Y353CNpdMf=0(Lsj17bzN&X9SatQgptIuEoJ2b&G`WLRZm zXYo9_Ryo+&GEb2e7duDiDYoKabD&-lt6c0{o|n`r4?9ohrLq!X=j*)GRs~ot)LUa! zh+V+*)>;)~7s|YiR$}ZTowv!V49kQ1fUV21e4Y=)x&kYZ`Jk*NSfS1bZC!=Uh59nA ztFeoDzFg~C>=K!;$XbeBs`C|F%dmM+KZ$icb{Wr4YTba%m-(rzRoLY^KehEmtO)9_ zv2Mh!;Q4E-AhywAi=lxMn||yDUZB)w0J~8ZsIoC)H|YY^Hs7&g zXpqKc1Y5!j(%O`cV>io!j5a20sV>N51HhF*pjh#dvDUxq~4F>nWTNVFXX zR}Bqj*k$7m@`AZ`Ik-czV38dcSECCS+wpL<&=84TF0PIjBDKrI9hQZt>_oUDx)8Np z0Zs}H)z}r{j`BjacEz}3vQVR)7M`sS88f zSK;cR;SBp~+zDPd*S;2aQWh?h?xrkS&aS?lvz*>oAVHBa1OQm~hRy7?T44-vW&VgF*PayjTd>7Jp9`ivol3 z_jR#ounWEw8pi;8;2-efxL_arLs^^%48cFr#fiaCd>b@g0!HE=^Wvr8F#Ho)yb6rM zKh?#n!LfJ^G(iJS#6RODXu+xY=duJN7>$3SOE7^k_;zR_*pYyL$xDPdQt+>2i6}<~ z{XO8cJbV{4S>l+Bf5%IfI_BZu%aT=& zB7C28LYykhM_E!) zP7=I9mx^|(!uLa`F`TOLA9>TbPPOpYI1l4Tj4P56JhER!>U05CrXxqt{jJ_h1q zOR$h*P%dDCWgiCZ;z9tyunZRuf)yXjb@3rs%dsLC2*IWgD|UerY+>fQZAgM0A18GQ zBiPGvDi;*Np%15ai6wwxc#TUU!I6*Gx}*}E zf}5Oxa%B+Q`v_=P4#5LPWVmJ%Jo!YfYYxFnP87Lv3Eq7~u`7?@10zXXa|ynDlGHVi z;3p@kTtx)`K9btCfB=D!HLit(06tmkT1*I(lZ~!oLQo&syVS%%^hQY6&57s>n@B2<@Ya-DHF?7)|0*t|27!G0^VqghUvV;od<=;xoDKU4&#gQ{=8Cr1UYx?m9v$j3sgJCrsnBr0xTR zG&xJT4q!L;3nMMyZk=-}b89ec5PF4lx@xi{Y6~oW-BT^~@p8md_G-a*1>LW{Ev{#2naciDxcxE`PSv zGmkh=K3nA}BF^udt@bP+a$$2co`u8({5e|BV&X#i9HXb0xTtTA$+L{egXMs|%87h_ z4#ca1D3Ir%yd*?nUk=)oAuSVhu{(P-h6LF<{zR^ofT-7(<+Z6t2$Td4N_P87oyX}m{>CHzHN?{VU0`68pYiCEgV$m9(m zmBDym9}sB^pJ#qUCT*4TP(EPNwmu%($Awf5<1>6bNZa{*u8$9Chnz3+fsl6g@x?w+ zQUy#P@j;Sy@dZ+!Fw$MPiXSN zkg8z0U|#}hA3qo3OCjx-=c0TWqyv4qXkQMg8n&3>n@u{%U(EH*Asv!07Wr~XHGPZ4 zzC2PbY>C7-msH1JBK6H99hNUq`HDzK`j)7D3rJGfQjKpR=_r4x*0-2+Oup3UD<&Q9 zTWazxBgtTSV83#boSz5rs~{=lc_=>#N!gc&_NyY*!hGxB_upNe$0FJJ9RT;_K*?>eH4+Gt{FuK+3JD`Wk*`rfP~@k5Yt)cfvIe$R14$%5z`9VN{W7Pej!P)Pp3 zU#|@)CilqK8w14T-oEvwfHJZURtyd-C-?D-A%PWSy}TF|C?OmAiqU~pLUM^mxzM2-k-Y(pr z6RcDdG(w)_muiE?$y4%DV~~mbudmb;1fT%mWnd_X0u+=%ptckXMHvbTrdaCB&`=i& z2)=~@^`KY@ws4_76l=v65fnnP(Qgq$p%h#ARtXeIu@h{SLc=KbimfUrisGQ(s)oi= z!0>GvXd=Z?uuTh1r8p_J8KG#3vwoWiilMl`%fT=L#Z^!afl(-KigFZ;L2=iYqhTD1 z2YfpNmQC>#Z0EvqC|-)~A{dwAt=}$&@hCp<9THeB#aFOH3d^JTDR!t}B8tC$hZo#m=dViX@rR>LHeC0SQ!NhuK>f#DKJ3=1YSXbD=JWM2?e3AK*Osj zNcb)WyqXd$*u{m{QbH8FL~to3RKH6Mmr=styCv{?O1NOR6y89IQ0!L0Rg_5mZZ-TO z1qH9vz#A!1f=VsCi4v`-G{V)C7=5J)-a?6mOTdU$N}NCfL9|ig6%rIeLrKs}(1>8_dL6EjooT3Va1XJ<)Dm2oC zN`UWUAU&u=!9Fh1he}fH6Coi~vVNZ!38hlt`z1&ul`7aTMTSvniv21iib~h-S0iJo z4EO;JGLgy@9MB?DsVv0-BN9zz>kpWa7%B%|4Gtzyrwghf!4&EYMKvmzL7l0uMhA1K z+3O93ERj`OUUw=p) zTtMZ*Yc#=y)CGbXZE!Jlp`ykZET%5f*O-FKs62QrIHa7)7t}&RDyRZQEhYBpLZsBC`Z{rljG6~OED5QnE)yJY$>v^@f)HY}C)pF(L2L(}%^m8LKZtqNWb4kys|3F;x?6xx18 zJt~|*JD{&ehjVDv@Dq&iY}!G=32t}}?U3SxD4a{H(Vq~9^Jum3lalaUTAkpeG(3-X zSaDJnE}|XLpHznz(4_EFn(#u}QNbx~cropm;*>F5OgpYWWeP8&$>0s(h;o`-&;W_3 zpeYm$s0ay7sc%3>RMG0;rx_8|v=f5U+=yD*NyTYVgp_tle_9+Nqcy96iZiMR745A4j5^{XO$9%ziD;yq6P(pXG||p0&Ke`svMXy3fw$mEn=NOS4w5x)1+{iB4HN`nmq?UGFe@+~!qcy?LOCtMe zHw5RUkpr}wiu0;SBkh*{ygKqbO%1=Gi5#Kb7F^Iqj??ZaE*K+Cv}XMUQzU@i0>21G zf#`Py7a=HH`aQ)(6bekgufK>!xzJnTml!Ay`UAlwF3N}gP;p6wg3ur7FNsl5dK>(* z1cjtO7F?F1!st&FmsKbf{i*)48Wl^|z^`afiS%cJD_T@4{kh_b5rw9|&|fj3F!XkK zBRGmce<^5$L{aFk6pg4T2K}|Z5go;$cfhYQqO$341XsCHIrL7&RZ$d|{#Jig9L1w| z!LLc8a_R2`*Q8N-^!JKuswfe?TYpU*RY2FmuWOsNIl$hSDzix^u zqwC;J;OKICpP&g6T|w6?no!XaxK+!i($ARTETGyhO4j@5=UXUDO*u-42HX*6&=T6cpx4y;<6c@!Ux>A9EO+j zfhdm4@HRXU$MG0Gh=-E6T!ydkp)@X!;ir74iW4#X4G-0E1q=w{ktVK?5g>e|jVopZ zDjyl+#Ec-rBU4-%1Bz$^$Coo;!Zt{J1p}^ZL&ZxN2tyk>zKVfFJZ8jKGlGSWx$(7( z5anZ0yp$1Ycr1>WF~Sf}B=Pl(aN!ead;=pw`9u}3VniCAsN*j(P>83R_(n#Q@ToSw zi4m=QYK&JiVhm4B@hyy4ga(|@%7_zcAPH@Zc%=rFpkX8!H0XqOMk3-FBcX$lBz(qA z=wc)*pNSH*j1W8#>iV1xzmDttP3E zxj^_l`j+!f1g|J(j+{9d|>^3H=nX3%lrsNi80YVE-X=Sb!Y9T45ZOk=FEhoC`HR$Z}=ci(J_k=J(84u<_2MpG-ZIfQQ4zPF)}w9dekZ3 znPNn*CS`AY0#@v zV_6b}L6e%u+9Nb*Q&U<0DGkO{G;6QHU`oZXsu2C)X$00jVLxOVg|%PVkDA6{9WeBx zr*T--h>whE*{p-YkKAcFtV7C=qG?=Kjp3ts8jn?r_#~N@%c>K8l1|HG9aer)O%t(> z7(S_|6|kg;&zfn4tfRuu+G)kCW6ICQX=2uK!)Md9GL{T608T4s$%O-uvj1U8osL2F0xdJK}}jC>zr^Ell8Xe2lAbx4kiR@>>Us`l3`?>O$5shZQF#Iy1G3<842snelekmM* zWKh_zlq0AN2K%*P1f9WQcOXU?8QJVN!clHU4!cu1D$3xp-x@~689a6u;k#AM%yM?0 za2%3Z!PYCsQJE69!7z@_tYY^g{xCAD*&l^}xS6%=Ps%@{Oey=b;g2{|#vVZYm1Nel zzX<~X}D zB&(nOM>r+T8esobPN}kt>zg!re)8evZg|7o+#hX$4Zj9Di3Kf^y$7Jvgl z0vs_Q4loxGfU)IR)B~b0V2))!AOqvV0U?1*j0eXm7q|f9!?CUhuE0PzHvPa743uMw zwAh0|a_n*~j$*<%_VpI$Fer{gzr}4#EC-CVe1=KnIObY@z@&1V>Me&bXpVEg_85E+_y?;keaNrVoqSa%@9`13F^0* z#FlZONLxo-IR}<&8-T0e!0T$_eea zE5XS)VMzNuxOz@_uKiJ511F;1{v1xliR`z(jl0M}AswFK8aYw94j*t$oalOoA)J~M z)9)~eYvIKHPjBZI)m9b#Ypkef38gJ9+^Pg=OIz9!O3R%R$Ym$lN%r2^Np_MA zCEO`c2=_aLQbk22h?-lKASx;TLw!c>NA< z;wI~yV}~!%kDWjYCM5Z@&tt*FqyY8>eK0*Kke%oVW+Vl(lSn(6Ng?cHY$u!)%1+Vm zR40Y8Qyn`sNfGQc(k^WhiJgw^(k0Q@8Twt8q)7Hf$1ZD9G&_^D+m;l=W@5YTNwMrK z{cdkk96Q^w+n1EUW|2Y&$w_QB7D7x;V{`N&^yEx7*Ac=XLD`NWa&ToW~YB_F9t**$`=;ExD8pWBcsM6>LPm z&zoG$mN@qLlIz$gDU^`X#Fk>A#FRF+Odm>5>0-+rp^TIsHb&adOzCASu>EjKA6u#4 zuTJS_s~r0^DT8dBbU>S;XXjuCbSY+buKs`}Wr%&ralo1~!pGR`h=9Q373vI|LJgw!c^5f(;Fon{y7!|16q>=H*9BXy2lN+L2-=hS15ehg(ut*j0{jYw8-insmsPy3W>MhwP~v z>>B+cZ|Wwy)^W&}>c^=gMG(^bIrUfsF)e`8ppT%Z1#%i45sb88P7~=~W?BfR8M_xw z3+1%v?^UOTaatYsYSJP&ZKT86G!myBJFH8iaXR#eEoqUQPRC(uS~RDNbe}CPhNH#q zv!}&!y7l*Y)8aT+9ryXt5;#315+OZ_a}6UA)6+N)=}GkTOwPj&5+j|(=_Qew=>X>u zj0~rPoJaL!bvnd(%t6+qqnthxMVpRs9>*xUbe!{qo?=PQ<2>o0SkntR{UoX_y_BQF zsP^;<&VZikO|RxW<)He~>o|iX8X=>J^E5^yX0&mh(bMP|U7TkfG)6`bM^8G!%;@D9 zup@9rAIGRaqR!~&m>fqm8G{@%>3(g7o^u_$UzcI#Jg2|kk}<@2-f_P*V}vtAdcc-3 z%CTS%*fYjB!}rFJ zkr~XLA{}LBhH&4*j>4It-1qfI)tO=34;)7|nGxJ+(lKo&iTfdTOqWUHexyHU$&BQF z>^Nr4jONaej@vS0xHjy#Ju{X&t3U3|jN^XdIPS|#;Led^2+Snzr&tV;na2H0A46wm zazA&(FqkavJn00J32?u_PQXl%`=$Pbnh9~ga-7gGQSJiiq?UiTgbkOU!EH{-BSg zXLWIZbi^{UdbkeKX=YY0*NL5mv--F${b_YpKiBOzt;rhXdPrxqS$ggb?2InU%>7A! z#*#I}{n>HGnpHc(T_K&dWsP#Z*janl7>KeoV~#PL(foWFLM8MFf`fA+znE^Hrv7d3yarf zd$@n=<1N`M+ zh^zo$n<0VD3IuLlNAEop*Rn-LV-IC=hdt*;4bHR4J!f&AYah3NWk5S z3py4JxW{n8!iog8J1`Y*{Gl{`w0U_jMCK~|uD3W0|2<$Z^tJx5+&zY=Yqd+J*Ma#y3{fZPF z8wU;;QY`E|;Gi?b$}R-L$f-7VDL_=D+SwIAxFOZct_BV{Q+@0@AcCAm;4}gED$PvR8#cy&CK3=;TFJgEXC^4oYa14!q0@#IdB!SCT= zWFeE^%Tp+XFu#wdGziuFexAxH)bIy+I9a6S>v=f}k&a($=H(hh7XA?Ll2c^mkMQ!y zVjF*yr&fsV{4rj>LG0y^^9r0|AAgcpNQMZ4DPEBRA_}H?#RiBjnBkQ;A%bu*RzLIDdw z{-XwkS_ttUb1F1Kl;1~IYK0j8afMPR#Q9Gclonwg|4FCPDlFvplT|ihDPN~h*@YGS z0fWjbtmZ%ERQZH;{6R8K5H<0iR^UWY8~+&tP8W6YpLOC4Q4e2F&S8pr`36M}Eb8MM z4LNF2Ki}lc(TE25W^%4pq~~8(?fEc7ZM?2E_<$iV`~% zE0{Huc%eAKC(aTdlpvTRmlEJ4!KaE+BAh1p%uq^)GX{s9=G7Sqo!=uN9YdFfRD!R@4NXC-~NR*$Nj57Rlu{xKv9ozf`Bv$zE@Nbkv72(h6*~;CHT=35R#fppwR>)O0WK`f)RN0X+ z!K$Iki;N3?aaQ?|Nx>Sqnjo1H{HmxXN~Q(B8LH`$8Nu()YKCM^uuj%6CG&!t3Jok- z5d2}#s3nVnKb;zlWLdC5uF*;yg1;0sI*CW{x1q)&SrPo>tg%Yg1e@eqn`B+!Q`Fie z8-guEtyi)s_}5wMllXytlsW?H58kG%BccJ|He($f4Fqp@)iKau(4SJzL_@$kl=Uzg z3f^g~SEFIzU9Nf!8UY4S8nh@0yj$6zLuueW#s&)-32t{aSkY)OkkV*FV?ctk(T>J~ zLB>We8VBxhHTuv5FqqOrkS2jUl}$ux8o0~YM3-iQyIoBTDGLmtG&7|DxJTIxOF?k2 zu~{vJzV8Wg}oDrPC%G1?kF8yKD@MGIn}p zTNjHYxEC9OX=3i9pGtYw@&T><Nc@)hu`tJ^AH1LG)HZSr-Hp}cCBZ-DW}t6upg zc+PdzC-)O3P!4C6x9dl7wt!FHw;uj|rJM3_f;(xx00s+CXLm1DwuuM0J!w0`4!WR}K>K08xW+kd6n68eM}7JXq93d76obh?Vzp4Q+IqBhDiTAUynA;|L#%QMJc8Q*KnHjk~B0c3gGq+b{P+o^~`$R_L zb#-pP$mF`N$sH7#DbHzh^`h&_=XAMd(R0SyTJ(M!f5`lT7s%dR2Dr8&_k#lpNaFM363 zfiEqHUNu_Omlj2@xh$GX%c3#Lu=bKe^ty6bcgZ7q!#HfYv?6-bHEg}KCK{)_V7s&~ zvMOJ&U)m5&7+>&S+7!Lzdck+ePdrH(A>{ds-&T$g^8&>07)R)Nf#P>vBaFOY@f77n zW?qQ+J>`pVUa0teDo2UxH1TJ~QMx)){JCqC zp=OEaDX%cqfcOjLE3g_Ae`$P0t%k&3xn9wzQSk!hRjnEmf318~r^dzK7+t{WAywk z@sF-CMt*IN*g<)nncpjRDqn~5`@}Be>+1Y|vD@{!CVx=up}e8Z*Nbl`-_Yfo#XlL} zu;dSke|EiL%^wl3P~NoVkBYs@H|_aj;#K3D-u!X#FRnLz`IF)`$~d85O8l#GoLDd| z{>?Z}FPIVk?iyzl%!$`2R%XGx_@>ed7c7YXFk00Gi{d|BR!zaOc!M&bEpUkcQcma! zJmSBN6PAJ%@jtE!Yr&d$lk%3WU|sA}zGW}i5N{dZ@)m50|8>3PEAWH-sFQ?3f9N*V zB(X36+Gd)h7Y0JNyC)fi!H_@oZDwHzbcgC~xG)sD)AY8wFbulO{kEns0t%qMqb(#s zcdOpf71E%4Oz&6b;jS@j`Y+y_xiAF7M{A*%aBP4OT^ zqkg0<)jk zC!uKS454HSI;xr>mP|v(Of&S78R)oshEXyH#ZYa`l6mNa$_AG#KqpN$b;%-h%5Bq> zEJLx>S#5~}I<1=3m3W{trddnL3Ut;zYb{xW;;5h4O4cEU>Jxj(1{81l#9OinopXQU zEAfL9sB?r;fB3v=j#wH1Uog$lO9SCV_Z*`%7*3*o$}A0mlU1L>rJ-<&=~H!S7@X?< zR8txOr%^xCmXhFf)n~d=8k}MJ%u*T&Uvz(FEscgVsh`_QV_>H0b9-qloMrmlTN(#v zyFd4pCcrG}JfSQJW~=6jWoaPuZ24vS1*TFUZZvHMGFSs@Hjzp|B;!m#QqdszjHn7;CsRl^eZ zSH7}37^N-{E;qqa)dKNyZ5u2zEzmD_!E*Nk<8lv-QNLzh?u8Yqui?vmu+sFk`f@+4 za(}J4JP6~|Z?u>7aE|I5-DNYJYx>4=c?iDb{>FNF1kR&=Yr8xOt5x6HFOR|brfK8ats(}|s(!Rr#3J3M zAH5ZE$W`}`zKR5-hw30yCLz~U4q{~*@{q|vugpXqb~_lAETos}WL5&mBPu6c2_lc0 zoa#yldCcw9RH8^9)upY(kjGUnT_uh@VRBh2^N=UqE^B2W(oc2UDoYWa%5AT#Kn6^1 zZ)G*|l-upAtV0H=9>SF-E_X?wG4jH9-nN{=1D=IHs zwSc^8@~W#Ak=NW_P1Q0oMqSlbIgr;?tGX%=@`h>EQniA->0Y%~ts&#oUu;$Dh*kBA zy=nuQF#Y1K+C<)R|Kh9ilT1?A2-W_Qw^eJz>Hx_*rZswXpyXZm8lyT`GDZEBSsfyI zPxUKY9V&U>^sBl$O!9&IS50+P@BwkbI%K32Q*fm!_L)4J7%>eN&@BB@5I)v>HtEwdxO@2A6ze`op5hlYHy`!>TEi zEK>ipX-Xw_)t`1vg=ER}r&m)g`Of{PPg7ebS*C6fYMLb9t2T%=ZIT~M8}ynk$&c<0 zMoo{zLH&za(<^bR{(@`zBrelm>Y9Fu+x?fOW>Dgx{;jRiOKzzC*43CLKbiiv)C@^} zcK>ay8Ii0||FPAKO1!Fn>@{PORntG-nsLc5?tgqWlae*+CZTpp@~di-SUWBG&9q6c zossrpgD`E=c|``P8+Gl0V(IIF}_G)GckTL-LnuOIPcW{B7E@ z)UHVWac@~`*CdW*3##)+Z^LhCK?Bfj zX1}PqK=gKxUwmCK>QB2Zt1bk+1HTQa3q|iV-Opj``HuX02;Jwoqoy83Luq$**JJ2@ z{LX=T96eyZbGSYaJ?Ob}qP`Ffqun)IUy2g(yO!!J&~WoztM%3BAt_jEg-0& z3B4B&2ybXZ51Rv`8oJQ?JOS|yJt&EGcUD6$O2+R-8v0O*`R@FNew6CDyQX0frP1!` zZqTDg@OuUt%;^2*dxjf^&<8yCOf-z3k+kiz4WlR>-@eo^hDMpUuQrUM4|=w5HB6$> zw7{UoDfB2F7~VLI9y13NJ%RC!b7%~WkkvSkp1=u6;{tlpOvrCsL{E7LHI2(? zEG?+J(Se@Eg9aKs=oxd+aN`Pk))O?*xQ52jcFZ=eqYQk5~M6zNKkWXKC=8&l7OexnB65q^{0<=9@&484L??IYDDc`&&zZsGW zJbP-IQ7K5<+ue*wh4|iqW?U*V?;UQ=lZriiCz=bT5N+RVbEy=@_boNoR!9-^zSZVx zsl>BytGP~!(n5n;nxs-ZG`yuvDl>;hwRB14p3wM~9w|oKpViVURp9%PmOiP{yg$FC zU#jx#uW1>S;{_(GJeGj7rt`!KIclX}x{I-6Bge(CoQEBvs&k+WjGOOU65Wj z6Z2aarR5%CP3y9>f)?K0>X26A;RCH6=@oPMaO;Y+$`d}(x+blr9hz-jmum1sORXEy z8uOvm)=g=x=g?NGpRA4+5!B`{tH&e4+X7?_=7^}aKv|)+GvNn+eor@{O~{+~F+Xp5G0(e9gVi;-#Z z`icR@>raS3UP_wI#@UXr!R_B-u5b6yBaDd&o?RYR{BC>>zXP)C#OX`B|`(I$JwOpEI1l0EC8#dq|`^t2;c z9lbIGegx_0lNrrN@;mxvCeM+YjzO83c7Jz=UUnV7f1txGd(M3SaL17BdC&b59V4suj5ezogUd6=BVM$71^7f zsEN)s**NXN+0J#D6@PH4b3-;^esHyOQ}&kU!L3d|`6MkmsLNmeHXa?`6(E1d939ma zD1X-z9p4o!pQ0Vj>I#v+haW|{Lgnw9kLGuU$v^NMt?7!8Pt%Tdcah{D;>QNMX!4KD z$A-Hi{-eXK2S~yJF-v{PlU%se4SmYCgN#Jud&nb9SqHQocru z3%WWb{}qo5zd9}d%^VkXbw>WXCocZ#oP3?e$htZ&zlk%Ds|)f!%#8f2i}F7`jGC*< z@(o&i_f?1dFFbzWsz?5}Iez%+iu@l>{KVBY`6lh$?A3L-4?lM+4pzQpKDT;xQ~s~# z+}2e;%(-$TTViFPXM;T&wtez0;j-2yIPbhZh_4D~X zVc1QZjRhV_ob8Fh2sw#MJ+WBO z^~BYlIBds_#I2qLEci%L(6uCNXHHW1wKQzk^`xk4nb__dN%7ZMSjds&tZM+aCnp)X z24Z`!C+A;-uzfd@Yp$VK=#iA}YZ$gaCuQIojvct3GJGu$J9r~y;#wgVb|iK7S}8`% zNnN^DfrVdBUA;B%f6FQ2=SLZPd|%j}p#3NO{qEc!dRyh5ZF_GCO7`jv+xJd9vz^k^ zx`*=jAKU+*R3-Z(Z~0C9_iXdKWj*=t^7GrXZGYSeztB5N37r zD#-WTY9U2J|DFFe{#W023*-N)|HrgHY=0!>)tlS@588_VP1|#G``+Q3+bQq-a5p93 zKh{5L4&U|Br*GWSs)U@o8Iu+9?OwlIR+axz_4}Ww|6l%N@8}cz{(BN7VDHLf`$GKu zXom}>#YJU`Qo6jbFygYJxFn|_KjLWAiKwG=Zq|7gH!U039~tI%OUmHqmvx(;AL~5l G=>GzI=tl?u diff --git a/utils/local-engine/tests/data/map.parquet b/utils/local-engine/tests/data/map.parquet deleted file mode 100644 index def9242ee305a0175cfb7976d2cc22ca0aec84a2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 13869 zcmeI(XHZmIzvuA=IUdZ4f`B<3kLh?26J|t21VoZ#$skE2OBOR?LR2J)m=P0TLPSJV z1Vkif5fBkDkwm~WU(cPV?o{1pW@_Hts;P3<{?yvNR#$cRs#;zDwSV0jE3{^di-^bi zi#IG&6E7YjE+Qrn*GT zhXh7K5>k-HD2&D!jKw&N#{^8oBuvI1n1Vks71J;sGcXf>VHW;|3}!~kLtbqztp@y|k$2w?06I#%Q4%R~#deDaf3}FOg zY=8+&VFq(pz!Fxlh7C5t7Iv_Q101mlPS^}*xWE-|aEAvxu?1f6h7Ww<2Y&=$D*~|% z+pz;Xu?xGg2SM12VC=(w9Kb;wLI@5c6i09r#}J0&2uB1W5rt@+KnzYI7N>9;XK)th za2^*Bhl_~EC0xc8T*WmcAQ4HpjvKg%Teyv6+(8QNA{F1L7tfG~=g3C^3Q>e&l%N!4C`SbUFW8u12AXhsX(;vL@O z13sb^ZD>aaI`Iiz_>6A!pch~872oh3KL&}mwTTD`^&9rjS3n=SMDQE>;&+ImANpee z24WBfV+e*~7{o9fBOne5jD#elAdOKNjWHODaTt#Yn21T3j6X01e_|@8VLE1DCjP=K z{0$k*hAie_F6LoA7C;UQu?UN?1WU0D%OQ^yP{2wkVilCI8p>D$6{tcDYoU&H(10ei zpbZ_Yhc5J>4+9v&2*%g|6PUsb=CFVztY8frY=kZBU=Ig4ViTOO8P0HlE8O4?4|rk= zyxBB?ja5L zk&Xw*z(YL3V`Sn9vXG4&JVh>^ArH@yj{+2;2*oHtDaufe3RI#B)p&s#yhJVPP>)w= zz-u((4Vuu57QDqfyvGN8L@V0Rjt+F<6T0vj-RMCtzThjq;X8f|5*^z!L?9$BH1yZ{ zXFgU!5v!nt)lkM7s6Z8JSPONmg9bFA1#ReHJ#?W5eHg$HMli+(n7|ZfFoy*!VFhd0 zU?Xf{2YWcc5u4zI&2WYbT;T?Hc)$}|;015^z!!e-M*y}W5ZkaFJFpYGup4_2guMvH zKJ3Q<9K<1n;4ngQ1V?cUVK|O(L?9ATh{g%T;3Q&k3a4=fXK@baaRG6-hPS#RG<=7sKyJ_;3aBNhkCq1174#MZ_tEhwBRk?;XOX!BU;ghc66W__1`w~*ZOBZRzeZ0poG;>#u}(V6>3-ub*zI1G@%7;=wLl` zp$B~!zz{|-#s-+c6lO4o1uS6&YuI2TY+(m`IKUB`;DpU^h6`Nb26uSC6Ibf83B=$e zVsQ$maRz5`4(D+Jakz+hT*75s!Bt#C0uqsg>$riNxP{wD#vP>KE>dw1X}FJcJU|8> z;t?Jr6HkzZY~=?i3)J8xYEg%Jyg~zBqY-Zg zi7sjCBlzcN{-3S5K>X|s@!~&cipP#)JKtBl?`TQUe+^2C{>PIA0tvy?f78ooq4#D zUnW7m&p^5S^ge^-TjfQDDi3`iBBmj;;Qc{q4cYr@JQOSTRBu-bl9Vz3PWcr4pWrW zd^AjHyo;ik^3=nR#8hMw6^EH=?r^5GeL|wrD97Z2 z%u!D1ZAzn^vxYqx?V3M(^%(bZ%_n0#Yh6~4^=dl&WUNnX;_7jJ-33p^1qj+#j}H_T z%NoCZsI2mYoswEv6LybxRh}3$H8g8tuuPKjr2TS*S(6URw<}K$Q5MUd9I7F^=8vQL zTG@YunYpf+5^fusJtfjPY0aO}UWM6z#ss#nnHn1;mNWHqh^)%Av*B7f)6U1bs!WfI z3(c7xpOB<7<8pFg&Wx++?J6@9vc#UwOv;y4{p&`#*3-Xk)w-(AN^S~$IxD3$N%im4 z?!u>krwKY#Wzt25=gMRZoufAUk)(F+?9B0QYO+~VkL1ec$Xr*OlPg!0J10-RLv3!p z^6+PK3pM7fomZ@{{cK*TncLd=<+ewj&98L6zIH*iSJATtHGv&#ap;;61&U&tb2V3uFwiMjC1LKasU&H4v_MJP<%Z_!(cZ-ctH*BZ z)Knh7cSND`#KUv7)=ZAjDO@wj0`QX{7a zpLC6#v&GAdT?^*vZE&wxU$()s&O^_{tNB=&iBH>2JyXA)k}}f(p)NhMzExAZOdFDxy$Jh-Au-zsE{M1@tT=6nO|qXxPa)?wzJ z1~%b#VHGx!F1HLeMthf5Y>e5~Wnde-SEADP^x^r2c4s4WEA7so@-(!MyBJn!AD?*3 z(Bbl((n^P`54sE;6S5_$9Fq#>8*RE#psiIc!7y`Lxf(9cjKuo zCO%CUkJtFLB;GdheRrp<#`nX6&nAAY*&|>2wHGWf_3y0Ed+Fa*x5YGIYj^YUmjS(P zw@tTx?J0Y?^}A5FX`rCLWNo0Zn4H-*5o!I}ZG9(tnQa%H9$ve>zihJEj)9BHYj+G* z=r-FqR7J9Grw_kS z%31849I0Qwcgkrmi{Pp8;q}4Olaej=%}gn;-#06x+hV^=j^wNTvW0S%2j*7lzdA6# z-plf!TubXfI+xUpaNc*HaHc^f#6^&6&89g@9&N)(VqFoCY zZam>$Y4GNRXTA5v7_XLyH!(i#cQ&5%>#cZmGC;U@V{Bl5>89B2VvB4~?UXibI<5)yRgJn}}&+K1R*>vWhLa*)F5Ebd>v!Pmx?9LrEG;BT>X5nLZKHNUC`Fx~n zirs~1pUUP7G246X;$nlPTjEZKF0#LPHqx-=;`!4)_VID?kuCA@Nh$W1E~iwsT)LXk zYkxT*NBZsMq{2lGS8h}qzP)m*-pAo;a!cgft10a%4%bq9E8kvA6aM0mklugPyMzp} z#g2)Oq>bJsW=`~VOv;)b^)4w#_O9dg+(lLIuIDLyalDbQGV1+}LaoJ{ZWbFFy}wy% z;k)Tpxqa08Ta~VNH{GuGsd|6AX8V^-$+f|wJ|x$NE_S-p5NY(`PUC4`rQ_cQ@bv+FSMU z{&(T8o6`jYMz^L54`1T^KxCA0>w~_N{G2mHXGFJV^q-UJ{BYpn>eh#YSAKPVG*oqT z+aodUB`%Lg7#X)cmaz15$&_@6Zp)N*OLciN+PAvx$=DrVU9!gS8{M8Y@yHU_?8#Ba z?b%b#__^jxy%gP^GyQt1>(iNctJ|N>did2fSLW&Hj$GNIC2r5=RvC9ZoBztsEl=)k zbVuHzj#RhjOTJWhJYUx5n_Is8fH9r<3d5JW7buR}&{?1~$=|(DdB%y(LX|o9+>6u} zzvwJdU-`|wSVMKpr(!Mbr5+_ZMjJkr=vw-FlcF&{C*!RV!GLs$OJj%`X zjp-`4II`5U!YXP*SB1?Pf6q$WODDQ2?XTbSta7~jqN~d3;Wy7}=ci*nSGyK1-SWb{ zYQyIjp0E73)Ofu;@wvvQx1W{dA-`dxTgEn!Ij^=8bVaZ_B4cQFY|tV)X1dgb(m##iNBua^Y-%Hn%=ipAAa|F zm+*A#mv>1;%Y5J8s51HT{?@Ai-w(-eW4?Sy=}7banEIvW%g3}nKYUx$2aNmLnlXI2 zU)!TmreE7KCvEj>&zf=aYkSU|`+gm{i(h{2$XoftuQOkD+_%m`?dATTij7RaeJZuw z>fcrFaPnJMrQ3b~&(*#!zkROR@x#BncHg+~-StP72lO;VnSSqSJhL^RcS`-n(c&Uf z^F#!FrR0RX`Z-v8n|hi%dRX`2N?ceXcAP}xMInjGjZ+RTpCbDIQbkJ5r1BII7Zy;A z{%iz48-k^G|7)-$A|&W1Bl@{tb`&~K?$p&j5Sb!D%7wR>R1O2XhI9x(7}4>LJ#^dfFX=v zj14e>Da>FF3s}Mm*08}w*uoC>aDXE=!3mq;3>Ub<4es!OC$_)~-td7h{NRrOY(*fp zVLNtUCw5^s_8p{Lw^jw zKn%iQ48c$ggBXTm1jHeMk&uKGq%jJkF$QBX4&yNa6EO*s@du{hPfW!$Oven x?& zzafL!ki{I##XQW%0?1(@7GW`#U@4YiIpnbd3Rnq6tb!6&Lm6wJ0#&GCE!43N8qkCm zw4sCb(1jlKVE{uI!5AB00#lg592T&I6|7-{jj)9s?BM`MY=RRu!x=7cg&W-A0Z(j! z7rfyEU--cv0oaN_Y{Pc!z)tMKZtOu2_97Vjupb9-5Qh+g!wAI@9K|t&;W)w(fk;Fl z8Yd8glZeGBoW>cP#W|eE1;pVZ;&BO=aRpa#4GBm@60YM0ZsHbhBN=y)g1bn?J*442 z((wQpc!)=Mj7&U17P66pr^v-KhTH< zc#TH9K@*zMg12~w_xOO1Xhj>^(Sc5ULKi-x8$Ia77ktGxe8-QUH#u}(V6>3-ub*zI1G@%7;=wLl`p$B~!zz{|-#s-+c6lO4o1uS6&YuI2T zY+(m`IKUB`;DpU^h6`Nb26uSC6Ibf83B=$eVsQ$maRz5`4(D+Jakz+hT*75s!Bt#C z0uqsg>$riNxP{wD#vP>KE>dw1X}FJcJU|8>;t?Jr6HkzZY~=?i3)J8xYEg%Jyg~zBqY-b=gl4qhE#BchKHwu-(S~+(pc9|ah0o|l z4|?$hU-1p!@q_hGrX(cY^D9_VU_Mqt5v!nt)lkM7s6Z8JSPONmg9bFA1#ReHJ#?W5 zeHg$HMli+(n7|ZfFoy*!VFhd0U?Xf{2YWcc5u4zI&2WYbT;T?Hc)$}|;015^z!!e- zM*y}W5ZkaFJFpYGup4_2guMvHKJ3Q<9K<1n;4ngQ1V?cUVK|O(L?9ATh{g%T;3Q&k z3a4=fXK@baaRG6-hPS#RG<=7sKyJ_;3aBNhkCq1174#MZ_q@z z^sn5g<^L2e{cBKCREVAa&*Nvf^j~`!_`iiq!oR|$U*XcPaOqdL^ebHY6)yb>mwtsy zzrv;eU&E#70-=AOF8%gT#`K^0QvJp$S#d&BQtU{W{%g_rzbPgEO~NEBWjs$H+1Jg| z)_RjUp%V{;yu^jYh1F(KdCn7&{Lg?(T!cXC4}s**pzG(Au;hQdDS<@(5UJ5pGygl2 zu_^!E6D5&bB+TcWA}W#kTxf(qFm3)8-`w~4{my-U|J>)?_g}Z`oX7cG*Z1@NmXC}A3=V+NEl6SFWIb1)Y_;b*8|9_C{K z7UCEDir=sZzhg1}z!EIQpI8P}s9`x)KpiWw3ag=kHCT&v(8PLdfEKi&gN@LI9yVb! zwm=_Su?^c{07DqT7$z`<8O&h;OIX1gHn4>qcEBDxu?xH507p1s51g?V`>-Dea1e)Z z7%n)1qc{dv9LEWqgd0xbG|s>sXK@baaRC=`376r4E4T_zc;OnZ;|9EO6Sr_1KJbMf z{1Jda1R)q92t^p~ARKpb5BCv)2Y84_h(r_~;|ZSP8J^<>qVW>1@ES3QMI7RhfJ7uA z87W9b8q$%0Ok^P&ImksG@=<_76rmU;c!N^BMH$Lbfp@4x6{=AK0cuf)dNiOBO=w07 zTG57fbf6RO@c~`vMh|+?hkks-Ck)^-zTmG>Vx9dW!otJHel35NV;NMThUHiRb*#iH ztcC{GU@g``6YH@7TF{0LHbNJA*o4j40)1@7Hf)Ch3}FOgn7|ZfFoy*!VFhd0z!rAc z0ekGkF6@Q_9N~mLaK>Kj!+spVK^($ixZntm;uu_U94BxRZa9V0I0JW_#W|eE1zf}> zT!sg(;3_=fg=@Hu8}Pa@J zCwPiyc#ao{#!I}yYs4THafn9(5|M;tq#zY(NJj=Tk%esJAQySaM*#{^gkqH74NCDA zWhh4l-k}mzs74J0s6`#>(SSxYp&2b`MH|}Dflj=~2XvtuJ?KRr`tcE;Fo4hag1<(I zP3#;kBs`&i^w;`lJ(fWgYFLgHP{&HF!fI$>4c1~EG_f8VpapH{U?X&)hfUawEzrkS zY{PaKzz{|-h6zkz26I@z5>~K=4QyeD9k9nv?80t1z!6T^183~TKJ3Q<9K<0Uh6|41 zD2~Au$8iED;f7N9 zK?p_&LJ@{L2*+LA!+k{H0UqKJA`ykhc!H;RhUa*JXuQNLyhaRS5r=prAQ4GOMha4q zhIC{g6IsYc4swx)d=#J%MJPrI-k=n3QHF9<;2kPag=*A5fLhd{9t~(j6PnS2R$H?iB(t)4XnXh ztb-=jV*|9H4IONRF7&Voo3RD@*otk~4g(m%2*xmhDa>FF3s}Mm*06ys?63p&*oj@( z4F@>F347p-z1WBSIDmsVgu`&b5gf%axZ*fY;3V8|3a4=f?l_BcIFAdsh)cK(4_v`j zc)|qa0lVIi+i|_2t2?;JVGR*@EA|<6wmM+ zFA$BFc!k%9K`i1Bj|3zl3CT!7D$lD8U<);w{Qhjtaa( zC8|)38VFE3N^DWxH$wk-)&J)xAtaGDPNHC(@;{F55~32KGE!pyH7Om3X8S~8;RoPe(aczUxp;9J3v7#~#m9fKQ zU1j6MZ*(%y+mYk!WdO?!-TQw za=r=QYp7HujMG$|oG7lXZI~#bXX2YEsqavgINrc@a*~vZhhdVmMSyS81e?gJq#x`P zCMU}{6c{G|=q&I}o_L_EDtVHN$dnXWS8=11$!>CfDO224s#B(3P@R%0=b>$sD(_|D zm#X0HP@Ssi<2ofxDZs-hZCXfxU)uEW$m+Bi5eZY$l_Lv`(q}#u_@&Q^?y63o9V0R| zV@`s&amL&fIsc5GGE{0Ze$G*ynyFHtZJar;#Kb>yewjl}=7LJssaXpJ9>!U}Gz9o( z{n`>)ll5Ci!qn_VT?NM3zxN6JvlkC^)nxx6G*m8UiHL+r&QdY?fSf-^&J*M;8>1$d zt17NzlB*_V8j!nO#!-;FLiV^^p1Rx>lf0EmfdP4|l%oWBtLG%jVHfYSNEzr_blP}cP)-f&A(K8J!+^FwZTc~SrT)s%p zsAb7H>PyU0b}}WvD`lfvbdBiJ_Z(P>GTI zyt)$O3u+2)OgwbV-k5rs2E8%!cC342?sHtB)FR-DS*c}6U{I-5cvM}fbwr}VTbsy2 zv$wWSYlGg}MR(V|-4QcXvCKX}!n|x}ihOX{u8ev0WxI3K6w4h7bj-^gOH6~yoyr{R z%lA|sSFCUrTrscM+YlIBv9BenzG8nzqT;&)U4`cF4j$~Q4SsiMpu7IvVIfhaN*57H zi^?Nn3L%w8N6v4kJT_*zQkARtMvJQBQf47lCuE!&s!qzDP^xy5yJ}H=N+~F$`n2-n zhUznOl9X!PRf;TX&MvGAsX4c(r=jNj647ab3#yWqf{W@3p@K^q^BV=1HJ4AT_0ZmE zS$jp#EVTBjzEfkZr@@J7bzUY{E$gmX1claJw|U%Hcf&quTD`YJk!Af&=ep4PTL*d? z>u$B52oiAa#NZh4TRaHr*A#{A}% zM>)%9v_=+ev~GF)Nfi6U23GIuoVL!#k5?_Ox^+ z%brwzpCadJ^FCE6IQ)H@@{^YL>2s2mKV+yB+kD7eSRei&Yf*2@hwLT8W_IPMj<@Z~ zRad;*m8Y?wwJTq9#mw#kZC%^$LOt`l-9`F)TDyx4PR{HpG4ZtRd1DcLx2M$RNo&tr z`{bFuWe&x*z2(mJcY7-i^tSfCa~U?PuhMnAU0;=(;=R6V_XTZzH5XRQ>KAzE+V$6Z zncwTL^WM|eU+;5r*2jhbPrHweA;I@PHibWF``8?jJnK_SWU<|+)~EINKD9;nwtZ@k z88&;MBVqiGfzA}g`vdPY7PJq1$XPM_b60`xj?djC=J!AMlzL)3qo0;aS(?D zBrzURkj4c302%y97thl%X6Ic!x?Vro2RiW{AJBzv^q?1g=*LHV!T>(w3;x1j@e5-JzJUmaLKMRwhT#~2Z!r=)c@g>! zqcH|!@jb>t91@Vkct}AS6Yv9M@FON-5@azMQ!o{BkcR>kp@eCeju}wKOw7V;%)wmz zgrA{;d6z(E|sVYuK3j^Y?xaU3Ub5^gw!(>McnoW(hu#|2!(C0vFFuHY&>;e~6sjvMgC zP29q5_`nx_@J9dw5rklbAQWM^gK*r%J={kG9^fG!Aret|j3;=CXLybmh{j92!fV7J z7IBD20uqsgWTYS!X-G!~GLeOBT_&Rj5V{1gJ$F z>d}BkG@%(SXhj>^(Sc69#|Lzw8$IYnANuhTpD=*W_=3NL8~eL%3=$H?iB(t) z4XnXhtb-=jV*|9H4IONRF7&Voo3RD@*otk~4g(m%2*xmhDa>FF3s}Mm*06ys?63p& z*oj@(4F@>F347p-z1WBSIDmsVgu`&b5gf%axZ*fY;3V8|3a4=f?l_BcIFAdsh)cK( z4_v`jc)|qa0lVIi+i|_2t2?;JVGR*@EA|< z6wmM+FA$BFc!k%9K`i1Bj|3zl3CT!7D$lD8U<);w{Qh zjtaa(C8|)38VFE}I@F^9jc7tMTF{C%w4(!^c#jY0LN|KQi$3(@BR*jOpYa8MVQ|kc zj3M|2A{Yu$41*YkV+6j%NQ}aF7>zL)i|;WG;*fwO#zP9yn1CN3gC8*wlOT)9n1ZR0 zgFFcr<1EhM zJTBlOF5xmfa0OT42`^m3b=-hAZsHbh!w0_bgFgZgh#&+b1fdAS9fad9?%_Tn@Bk0- z2$6`wV?4oAJi~LmKr~+B6<#9-v4}%F5|D@_BqIf>NJBa@kclj0BL}(2Lp};nh$0lD z1aDA^w+8XpNb4lNQn$?-ib~B)Dh^Ujci&ERMxL@X;UBdz>@9oQ=Tum3{HK7lY_UO`N #include -#include #include #include #include @@ -21,9 +20,9 @@ #include #include #include -#include #include #include +#include "Storages/CustomStorageMergeTree.h" #include "testConfig.h" using namespace local_engine; @@ -40,10 +39,11 @@ TEST(TestSelect, ReadRel) .column("type_string", "String") .build(); dbms::SerializedPlanBuilder plan_builder; - auto plan = plan_builder.read(TEST_DATA(/data/iris.parquet), std::move(schema)).build(); + auto plan = plan_builder.read(TEST_DATA(/ data / iris.parquet), std::move(schema)).build(); ASSERT_TRUE(plan->relations(0).root().input().has_read()); ASSERT_EQ(plan->relations_size(), 1); + std::cout << "start execute" << std::endl; local_engine::LocalExecutor local_executor; local_engine::SerializedPlanParser parser(local_engine::SerializedPlanParser::global_context); auto query_plan = parser.parse(std::move(plan)); @@ -51,6 +51,7 @@ TEST(TestSelect, ReadRel) ASSERT_TRUE(local_executor.hasNext()); while (local_executor.hasNext()) { + std::cout << "fetch batch" << std::endl; local_engine::SparkRowInfoPtr spark_row_info = local_executor.next(); ASSERT_GT(spark_row_info->getNumRows(), 0); local_engine::SparkRowToCHColumn converter; @@ -64,10 +65,11 @@ TEST(TestSelect, ReadDate) dbms::SerializedSchemaBuilder schema_builder; auto * schema = schema_builder.column("date", "Date").build(); dbms::SerializedPlanBuilder plan_builder; - auto plan = plan_builder.read(TEST_DATA(/data/date.parquet), std::move(schema)).build(); + auto plan = plan_builder.read(TEST_DATA(/ data / date.parquet), std::move(schema)).build(); ASSERT_TRUE(plan->relations(0).root().input().has_read()); ASSERT_EQ(plan->relations_size(), 1); + std::cout << "start execute" << std::endl; local_engine::LocalExecutor local_executor; local_engine::SerializedPlanParser parser(local_engine::SerializedPlanParser::global_context); auto query_plan = parser.parse(std::move(plan)); @@ -75,6 +77,7 @@ TEST(TestSelect, ReadDate) ASSERT_TRUE(local_executor.hasNext()); while (local_executor.hasNext()) { + std::cout << "fetch batch" << std::endl; local_engine::SparkRowInfoPtr spark_row_info = local_executor.next(); ASSERT_GT(spark_row_info->getNumRows(), 0); local_engine::SparkRowToCHColumn converter; @@ -104,8 +107,9 @@ TEST(TestSelect, TestFilter) auto * type_0 = dbms::scalarFunction(dbms::EQUAL_TO, {dbms::selection(5), dbms::literal("类型1")}); auto * filter = dbms::scalarFunction(dbms::AND, {less_exp, type_0}); - auto plan = plan_builder.registerSupportedFunctions().filter(filter).read(TEST_DATA(/data/iris.parquet), std::move(schema)).build(); + auto plan = plan_builder.registerSupportedFunctions().filter(filter).read(TEST_DATA(/ data / iris.parquet), std::move(schema)).build(); ASSERT_EQ(plan->relations_size(), 1); + std::cout << "start execute" << std::endl; local_engine::LocalExecutor local_executor; local_engine::SerializedPlanParser parser(SerializedPlanParser::global_context); auto query_plan = parser.parse(std::move(plan)); @@ -113,6 +117,7 @@ TEST(TestSelect, TestFilter) ASSERT_TRUE(local_executor.hasNext()); while (local_executor.hasNext()) { + std::cout << "fetch batch" << std::endl; local_engine::SparkRowInfoPtr spark_row_info = local_executor.next(); ASSERT_EQ(spark_row_info->getNumRows(), 1); local_engine::SparkRowToCHColumn converter; @@ -139,9 +144,10 @@ TEST(TestSelect, TestAgg) auto plan = plan_builder.registerSupportedFunctions() .aggregate({}, {measure}) .filter(less_exp) - .read(TEST_DATA(/data/iris.parquet), std::move(schema)) + .read(TEST_DATA(/ data / iris.parquet), std::move(schema)) .build(); ASSERT_EQ(plan->relations_size(), 1); + std::cout << "start execute" << std::endl; local_engine::LocalExecutor local_executor; local_engine::SerializedPlanParser parser(SerializedPlanParser::global_context); auto query_plan = parser.parse(std::move(plan)); @@ -149,14 +155,17 @@ TEST(TestSelect, TestAgg) ASSERT_TRUE(local_executor.hasNext()); while (local_executor.hasNext()) { + std::cout << "fetch batch" << std::endl; local_engine::SparkRowInfoPtr spark_row_info = local_executor.next(); ASSERT_EQ(spark_row_info->getNumRows(), 1); ASSERT_EQ(spark_row_info->getNumCols(), 1); local_engine::SparkRowToCHColumn converter; auto block = converter.convertSparkRowInfoToCHColumn(*spark_row_info, local_executor.getHeader()); ASSERT_EQ(spark_row_info->getNumRows(), block->rows()); - auto reader = SparkRowReader(block->getDataTypes()); - reader.pointTo(spark_row_info->getBufferAddress() + spark_row_info->getOffsets()[1], spark_row_info->getLengths()[0]); + auto reader = SparkRowReader(spark_row_info->getNumCols()); + reader.pointTo( + reinterpret_cast(spark_row_info->getBufferAddress() + spark_row_info->getOffsets()[1]), + spark_row_info->getLengths()[0]); ASSERT_EQ(reader.getDouble(0), 103.2); } } @@ -321,8 +330,7 @@ int main(int argc, char ** argv) SharedContextHolder shared_context = Context::createShared(); local_engine::SerializedPlanParser::global_context = Context::createGlobal(shared_context.get()); local_engine::SerializedPlanParser::global_context->makeGlobalContext(); - auto config = Poco::AutoPtr(new Poco::Util::MapConfiguration()); - local_engine::SerializedPlanParser::global_context->setConfig(config); + local_engine::SerializedPlanParser::global_context->setConfig(local_engine::SerializedPlanParser::config); local_engine::SerializedPlanParser::global_context->setPath("/tmp"); local_engine::SerializedPlanParser::global_context->getDisksMap().emplace(); local_engine::SerializedPlanParser::initFunctionEnv(); diff --git a/utils/local-engine/tests/gtest_spark_row.cpp b/utils/local-engine/tests/gtest_spark_row.cpp deleted file mode 100644 index 313c62de39a3..000000000000 --- a/utils/local-engine/tests/gtest_spark_row.cpp +++ /dev/null @@ -1,443 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -using namespace local_engine; -using namespace DB; - - -struct DataTypeAndField -{ - DataTypePtr type; - Field field; -}; -using DataTypeAndFields = std::vector; - -using SparkRowAndBlock = std::pair; - -static SparkRowAndBlock mockSparkRowInfoAndBlock(const DataTypeAndFields & type_and_fields) -{ - /// Initialize types - ColumnsWithTypeAndName columns(type_and_fields.size()); - for (size_t i=0; iinsert(type_and_fields[i].field); - block.setColumns(std::move(mutable_colums)); - - auto converter = CHColumnToSparkRow(); - auto spark_row_info = converter.convertCHColumnToSparkRow(block); - return std::make_tuple(std::move(spark_row_info), std::make_shared(std::move(block))); -} - -static Int32 getDayNum(const String & date) -{ - ExtendedDayNum res; - ReadBufferFromString in(date); - readDateText(res, in); - return res; -} - -static DateTime64 getDateTime64(const String & datetime64, UInt32 scale) -{ - DateTime64 res; - ReadBufferFromString in(datetime64); - readDateTime64Text(res, scale, in); - return res; -} - -static void assertReadConsistentWithWritten(const SparkRowInfo & spark_row_info, const Block & in, const DataTypeAndFields type_and_fields) -{ - /// Check if output of SparkRowReader is consistent with types_and_fields - { - auto reader = SparkRowReader(spark_row_info.getDataTypes()); - reader.pointTo(spark_row_info.getBufferAddress(), spark_row_info.getTotalBytes()); - for (size_t i = 0; i < type_and_fields.size(); ++i) - { - /* - const auto read_field{std::move(reader.getField(i))}; - const auto & written_field = type_and_fields[i].field; - std::cout << "read_field:" << read_field.getType() << "," << toString(read_field) << std::endl; - std::cout << "written_field:" << written_field.getType() << "," << toString(written_field) << std::endl; - */ - EXPECT_TRUE(reader.getField(i) == type_and_fields[i].field); - } - } - - /// check if output of SparkRowToCHColumn is consistents with initial block. - { - auto block = SparkRowToCHColumn::convertSparkRowInfoToCHColumn(spark_row_info, in.cloneEmpty()); - const auto & out = *block; - EXPECT_TRUE(in.rows() == out.rows()); - EXPECT_TRUE(in.columns() == out.columns()); - for (size_t col_idx = 0; col_idx < in.columns(); ++col_idx) - { - const auto & in_col = in.getByPosition(col_idx); - const auto & out_col = out.getByPosition(col_idx); - for (size_t row_idx = 0; row_idx < in.rows(); ++row_idx) - { - const auto in_field = (*in_col.column)[row_idx]; - const auto out_field = (*out_col.column)[row_idx]; - EXPECT_TRUE(in_field == out_field); - } - } - } -} - -TEST(SparkRow, BitSetWidthCalculation) -{ - EXPECT_TRUE(calculateBitSetWidthInBytes(0) == 0); - EXPECT_TRUE(calculateBitSetWidthInBytes(1) == 8); - EXPECT_TRUE(calculateBitSetWidthInBytes(32) == 8); - EXPECT_TRUE(calculateBitSetWidthInBytes(64) == 8); - EXPECT_TRUE(calculateBitSetWidthInBytes(65) == 16); - EXPECT_TRUE(calculateBitSetWidthInBytes(128) == 16); -} - -TEST(SparkRow, GetArrayElementSize) -{ - const std::map type_to_size = { - {std::make_shared(), 1}, - {std::make_shared(), 1}, - {std::make_shared(), 2}, - {std::make_shared(), 2}, - {std::make_shared(), 2}, - {std::make_shared(), 4}, - {std::make_shared(), 4}, - {std::make_shared(), 4}, - {std::make_shared(), 4}, - {std::make_shared(9, 4), 4}, - {std::make_shared(), 8}, - {std::make_shared(), 8}, - {std::make_shared(), 8}, - {std::make_shared(6), 8}, - {std::make_shared(18, 4), 8}, - - {std::make_shared(), 8}, - {std::make_shared(38, 4), 8}, - {std::make_shared(std::make_shared(), std::make_shared()), 8}, - {std::make_shared(std::make_shared()), 8}, - {std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), 8}, - }; - - for (const auto & [type, size] : type_to_size) - { - EXPECT_TRUE(BackingDataLengthCalculator::getArrayElementSize(type) == size); - if (type->canBeInsideNullable()) - { - const auto type_with_nullable = std::make_shared(type); - EXPECT_TRUE(BackingDataLengthCalculator::getArrayElementSize(type_with_nullable) == size); - } - } -} - -TEST(SparkRow, PrimitiveTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(), -1}, - {std::make_shared(), UInt64(1)}, - {std::make_shared(), -2}, - {std::make_shared(), UInt32(2)}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE(spark_row_info->getTotalBytes() == 8 + 4 * 8); -} - -TEST(SparkRow, PrimitiveStringTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(), -1}, - {std::make_shared(), UInt64(1)}, - {std::make_shared(), "Hello World"}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE(spark_row_info->getTotalBytes() == 8 + (8 * 3) + roundNumberOfBytesToNearestWord(strlen("Hello World"))); -} - -TEST(SparkRow, PrimitiveStringDateTimestampTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(), -1}, - {std::make_shared(), UInt64(1)}, - {std::make_shared(), "Hello World"}, - {std::make_shared(), getDayNum("2015-06-22")}, - {std::make_shared(0), getDateTime64("2015-05-08 08:10:25", 0)}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE(spark_row_info->getTotalBytes() == 8 + (8 * 5) + roundNumberOfBytesToNearestWord(strlen("Hello World"))); -} - - -TEST(SparkRow, DecimalTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(9, 2), DecimalField(1234, 2)}, - {std::make_shared(18, 2), DecimalField(5678, 2)}, - {std::make_shared(38, 2), DecimalField(Decimal128(Int128(12345678)), 2)}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE(spark_row_info->getTotalBytes() == 8 + (8 * 3) + 16); -} - - -TEST(SparkRow, NullHandling) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - {std::make_shared(std::make_shared()), Null{}}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE(spark_row_info->getTotalBytes() == 8 + (8 * type_and_fields.size())); -} - -TEST(SparkRow, StructTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(DataTypes{std::make_shared()}), Tuple{Int32(1)}}, - {std::make_shared(DataTypes{std::make_shared(DataTypes{std::make_shared()})}), - []() -> Field - { - Tuple t(1); - t.back() = Tuple{Int64(2)}; - return std::move(t); - }()}, - }; - - /* - for (size_t i=0; igetName() << ",field:" << toString(type_and_fields[i].field) - << std::endl; - } - */ - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); -} - -TEST(SparkRow, ArrayTypes) -{ - DataTypeAndFields type_and_fields = { - {std::make_shared(std::make_shared()), Array{Int32(1), Int32(2)}}, - {std::make_shared(std::make_shared(std::make_shared())), - []() -> Field - { - Array array(1); - array.back() = Array{Int32(1), Int32(2)}; - return std::move(array); - }()}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 - + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); -} - -TEST(SparkRow, MapTypes) -{ - const auto map_type = std::make_shared(std::make_shared(), std::make_shared()); - DataTypeAndFields type_and_fields = { - {map_type, - []() -> Field - { - Map map(2); - map[0] = std::move(Tuple{Int32(1), Int32(2)}); - map[1] = std::move(Tuple{Int32(3), Int32(4)}); - return std::move(map); - }()}, - {std::make_shared(std::make_shared(), map_type), - []() -> Field - { - Map inner_map(2); - inner_map[0] = std::move(Tuple{Int32(5), Int32(6)}); - inner_map[1] = std::move(Tuple{Int32(7), Int32(8)}); - - Map map(1); - map.back() = std::move(Tuple{Int32(9), std::move(inner_map)}); - return std::move(map); - }()}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); -} - - -TEST(SparkRow, StructMapTypes) -{ - const auto map_type = std::make_shared(std::make_shared(), std::make_shared()); - const auto tuple_type = std::make_shared(DataTypes{std::make_shared()}); - - DataTypeAndFields type_and_fields = { - {std::make_shared(DataTypes{map_type}), - []() -> Field - { - Map map(1); - map[0] = std::move(Tuple{Int32(1), Int32(2)}); - return std::move(Tuple{std::move(map)}); - }()}, - {std::make_shared(std::make_shared(), tuple_type), - []() -> Field - { - Tuple inner_tuple{Int32(4)}; - Map map(1); - map.back() = std::move(Tuple{Int32(3), std::move(inner_tuple)}); - return std::move(map); - }()}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); -} - - -TEST(SparkRow, StructArrayTypes) -{ - const auto array_type = std::make_shared(std::make_shared()); - const auto tuple_type = std::make_shared(DataTypes{std::make_shared()}); - DataTypeAndFields type_and_fields = { - {std::make_shared(DataTypes{array_type}), - []() -> Field - { - Array array{Int32(1)}; - Tuple tuple(1); - tuple[0] = std::move(array); - return std::move(tuple); - }()}, - {std::make_shared(tuple_type), - []() -> Field - { - Tuple tuple{Int64(2)}; - Array array(1); - array[0] = std::move(tuple); - return std::move(array); - }()}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); - -} - -TEST(SparkRow, ArrayMapTypes) -{ - const auto map_type = std::make_shared(std::make_shared(), std::make_shared()); - const auto array_type = std::make_shared(std::make_shared()); - - DataTypeAndFields type_and_fields = { - {std::make_shared(map_type), - []() -> Field - { - Map map(1); - map[0] = std::move(Tuple{Int32(1),Int32(2)}); - - Array array(1); - array[0] = std::move(map); - return std::move(array); - }()}, - {std::make_shared(std::make_shared(), array_type), - []() -> Field - { - Array array{Int32(4)}; - Tuple tuple(2); - tuple[0] = Int32(3); - tuple[1] = std::move(array); - - Map map(1); - map[0] = std::move(tuple); - return std::move(map); - }()}, - }; - - SparkRowInfoPtr spark_row_info; - BlockPtr block; - std::tie(spark_row_info, block) = mockSparkRowInfoAndBlock(type_and_fields); - assertReadConsistentWithWritten(*spark_row_info, *block, type_and_fields); - - EXPECT_TRUE( - spark_row_info->getTotalBytes() - == 8 + 2 * 8 + BackingDataLengthCalculator(type_and_fields[0].type).calculate(type_and_fields[0].field) - + BackingDataLengthCalculator(type_and_fields[1].type).calculate(type_and_fields[1].field)); -}